package org.apache.cassandra.transport.messages;
import java.util.UUID;
import com.google.common.collect.ImmutableMap;
import io.netty.buffer.ByteBuf;
import org.apache.cassandra.cql3.CQLStatement;
import org.apache.cassandra.cql3.ColumnSpecification;
import org.apache.cassandra.cql3.QueryHandler;
import org.apache.cassandra.cql3.QueryOptions;
import org.apache.cassandra.cql3.statements.ParsedStatement;
import org.apache.cassandra.exceptions.PreparedQueryNotFoundException;
import org.apache.cassandra.service.ClientState;
import org.apache.cassandra.service.QueryState;
import org.apache.cassandra.tracing.Tracing;
import org.apache.cassandra.transport.*;
import org.apache.cassandra.utils.JVMStabilityInspector;
import org.apache.cassandra.utils.MD5Digest;
import org.apache.cassandra.utils.UUIDGen;
public class ExecuteMessage extends Message.Request
{
public static final Message.Codec<ExecuteMessage> codec = new Message.Codec<ExecuteMessage>()
{
public ExecuteMessage decode(ByteBuf body, ProtocolVersion version)
{
byte[] id = CBUtil.readBytes(body);
return new ExecuteMessage(MD5Digest.wrap(id), QueryOptions.codec.decode(body, version));
}
public void encode(ExecuteMessage msg, ByteBuf dest, ProtocolVersion version)
{
CBUtil.writeBytes(msg.statementId.bytes, dest);
if (version == ProtocolVersion.V1)
{
CBUtil.writeValueList(msg.options.getValues(), dest);
CBUtil.writeConsistencyLevel(msg.options.getConsistency(), dest);
}
else
{
QueryOptions.codec.encode(msg.options, dest, version);
}
}
public int encodedSize(ExecuteMessage msg, ProtocolVersion version)
{
int size = 0;
size += CBUtil.sizeOfBytes(msg.statementId.bytes);
if (version == ProtocolVersion.V1)
{
size += CBUtil.sizeOfValueList(msg.options.getValues());
size += CBUtil.sizeOfConsistencyLevel(msg.options.getConsistency());
}
else
{
size += QueryOptions.codec.encodedSize(msg.options, version);
}
return size;
}
};
public final MD5Digest statementId;
public final QueryOptions options;
public ExecuteMessage(MD5Digest statementId, QueryOptions options)
{
super(Message.Type.EXECUTE);
this.statementId = statementId;
this.options = options;
}
public Message.Response execute(QueryState state, long queryStartNanoTime)
{
try
{
QueryHandler handler = ClientState.getCQLQueryHandler();
ParsedStatement.Prepared prepared = handler.getPrepared(statementId);
if (prepared == null)
throw new PreparedQueryNotFoundException(statementId);
options.prepare(prepared.boundNames);
CQLStatement statement = prepared.statement;
if (options.getPageSize() == 0)
throw new ProtocolException("The page size cannot be 0");
UUID tracingId = null;
if (isTracingRequested())
{
tracingId = UUIDGen.getTimeUUID();
state.prepareTracingSession(tracingId);
}
if (state.traceNextQuery())
{
state.createTracingSession(getCustomPayload());
ImmutableMap.Builder<String, String> builder = ImmutableMap.builder();
if (options.getPageSize() > 0)
builder.put("page_size", Integer.toString(options.getPageSize()));
if(options.getConsistency() != null)
builder.put("consistency_level", options.getConsistency().name());
if(options.getSerialConsistency() != null)
builder.put("serial_consistency_level", options.getSerialConsistency().name());
builder.put("query", prepared.rawCQLStatement);
for(int i=0;i<prepared.boundNames.size();i++)
{
ColumnSpecification cs = prepared.boundNames.get(i);
String boundName = cs.name.toString();
String boundValue = cs.type.asCQL3Type().toCQLLiteral(options.getValues().get(i), options.getProtocolVersion());
if ( boundValue.length() > 1000 )
{
boundValue = boundValue.substring(0, 1000) + "...'";
}
builder.put("bound_var_" + Integer.toString(i) + "_" + boundName, boundValue);
}
Tracing.instance.begin("Execute CQL3 prepared query", state.getClientAddress(), builder.build());
}
QueryOptions queryOptions = QueryOptions.addColumnSpecifications(options, prepared.boundNames);
Message.Response response = handler.processPrepared(statement, state, queryOptions, getCustomPayload(), queryStartNanoTime);
if (options.skipMetadata() && response instanceof ResultMessage.Rows)
((ResultMessage.Rows)response).result.metadata.setSkipMetadata();
if (tracingId != null)
response.setTracingId(tracingId);
return response;
}
catch (Exception e)
{
JVMStabilityInspector.inspectThrowable(e);
return ErrorMessage.fromException(e);
}
finally
{
Tracing.instance.stopSession();
}
}
@Override
public String toString()
{
return "EXECUTE " + statementId + " with " + options.getValues().size() + " values at consistency " + options.getConsistency();
}
}