Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
5 changes: 3 additions & 2 deletions src/java/org/apache/cassandra/cql3/CqlBuilder.java
Original file line number Diff line number Diff line change
Expand Up @@ -289,11 +289,12 @@ public CqlBuilder appendOptions(Consumer<OptionsBuilder> builder)
*
* @param filter the row filter to append
* @param hasKeyRestrictions whether the query has key restrictions that have already been appended
* @param redact whether to redact the column values in the row filter
* @return this CQL builder after appending the row filter
*/
public CqlBuilder append(RowFilter filter, boolean hasKeyRestrictions)
public CqlBuilder append(RowFilter filter, boolean hasKeyRestrictions, boolean redact)
{
return filter.isEmpty() ? this : appendRestrictions(filter.toCQLString(), hasKeyRestrictions);
return filter.isEmpty() ? this : appendRestrictions(filter.toCQLString(redact), hasKeyRestrictions);
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2158,7 +2158,7 @@ private String asCQL(QueryOptions options, ClientState state)
ColumnFilter columnFilter = selection.newSelectors(options).getColumnFilter();
StringBuilder sb = new StringBuilder();

sb.append("SELECT ").append(queriedColumns().toCQLString());
sb.append("SELECT ").append(queriedColumns().toCQLString(false));
sb.append(" FROM ").append(table.keyspace).append('.').append(table.name);
if (restrictions.isKeyRange() || restrictions.usesSecondaryIndexing())
{
Expand Down Expand Up @@ -2188,7 +2188,7 @@ private String asCQL(QueryOptions options, ClientState state)
sb.append(" AND ");
}
if (!dataRange.isUnrestricted(table))
sb.append(dataRange.toCQLString(table, rowFilter));
sb.append(dataRange.toCQLString(table, rowFilter, false));
}
}
else
Expand All @@ -2212,7 +2212,7 @@ private String asCQL(QueryOptions options, ClientState state)
{
sb.append(" = ");
if (compoundPk) sb.append('(');
DataRange.appendKeyString(sb, table.partitionKeyType, Iterables.getOnlyElement(keys));
DataRange.appendKeyString(sb, table.partitionKeyType, Iterables.getOnlyElement(keys), false);
if (compoundPk) sb.append(')');
}
else
Expand All @@ -2225,7 +2225,7 @@ private String asCQL(QueryOptions options, ClientState state)
sb.append(", ");

if (compoundPk) sb.append('(');
DataRange.appendKeyString(sb, table.partitionKeyType, key);
DataRange.appendKeyString(sb, table.partitionKeyType, key, false);
if (compoundPk) sb.append(')');
first = false;
}
Expand All @@ -2237,7 +2237,7 @@ private String asCQL(QueryOptions options, ClientState state)
if (!rowFilter.isEmpty())
sb.append(" AND ").append(rowFilter);

String filterString = filter.toCQLString(table, rowFilter);
String filterString = filter.toCQLString(table, rowFilter, false);
if (!filterString.isEmpty())
sb.append(" AND ").append(filterString);
}
Expand Down
50 changes: 41 additions & 9 deletions src/java/org/apache/cassandra/db/AbstractReadQuery.java
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@

import java.util.Set;

import com.google.common.annotations.VisibleForTesting;

import org.apache.cassandra.cql3.ColumnIdentifier;
import org.apache.cassandra.cql3.CqlBuilder;
import org.apache.cassandra.cql3.statements.SelectOptions;
Expand Down Expand Up @@ -62,7 +64,7 @@ public TableMetadata metadata()
// Monitorable interface
public String name()
{
return toCQLString();
return toRedactedCQLString();
}

@Override
Expand Down Expand Up @@ -96,23 +98,53 @@ public ColumnFilter columnFilter()
}

/**
* Recreate the CQL string corresponding to this query.
* Recreates the CQL string corresponding to this query, representing any specific values with '?',
* to prevent leaking sensitive data.
* @see #toCQLString(boolean)
*/
public String toRedactedCQLString()
{
return toCQLString(true);
}

/**
* Recreates the CQL string corresponding to this query, printing specific values without any redaction.
* This might leak sensitive data if the query string ends up in logs or any other unprotected place, so this only
* should be used for debugging purposes or to present the query string to the same end user that created the query.
* @see #toCQLString(boolean)
*/
public String toUnredactedCQLString()
{
return toCQLString(false);
}

/**
* Recreates the CQL string corresponding to this query.
* </p>
* If the {@code redact} parameter is set to {@code true}, the query string will be redacted, replacing any specific
* column values with '?'. If set to {@code false}, the query string will not be redacted, and it might expose the
* queried column values which might contain sensitive data. The latter will be problematic if the query string ends
* up in logs or any other unprotected place. Therefore, non-redaction should only be used for debugging purposes or
* to present the query string to the same end user that created the query.
* <p>
* Note that in general the returned string will not be exactly the original user string, first
* because there isn't always a single syntax for a given query, but also because we don't have
* because there isn't always a single syntax for a given query, but also because we don't have
* all the information needed (we know the non-PK columns queried but not the PK ones as internally
* we query them all). So this shouldn't be relied too strongly, but this should be good enough for
* debugging purpose which is what this is for.
* we query them all). So this shouldn't be relied upon too strongly, but this should be good enough for
* debugging purposes which is what this is for.
*
* @param redact whether to redact the queried column values.
*/
public String toCQLString()
@VisibleForTesting
protected String toCQLString(boolean redact)
{
CqlBuilder builder = new CqlBuilder();
builder.append("SELECT ").append(columnFilter().toCQLString());
builder.append("SELECT ").append(columnFilter().toCQLString(redact));
builder.append(" FROM ").append(ColumnIdentifier.maybeQuote(metadata().keyspace))
.append('.')
.append(ColumnIdentifier.maybeQuote(metadata().name));

appendCQLWhereClause(builder);
appendCQLWhereClause(builder, redact);

if (limits() != DataLimits.NONE)
builder.append(' ').append(limits());
Expand All @@ -132,5 +164,5 @@ public String toCQLString()
return builder.toString();
}

protected abstract void appendCQLWhereClause(CqlBuilder builder);
protected abstract void appendCQLWhereClause(CqlBuilder builder, boolean redact);
}
4 changes: 2 additions & 2 deletions src/java/org/apache/cassandra/db/Clustering.java
Original file line number Diff line number Diff line change
Expand Up @@ -79,13 +79,13 @@ public default String toString(TableMetadata metadata)
return sb.toString();
}

public default String toCQLString(TableMetadata metadata)
default String toCQLString(TableMetadata metadata, boolean redact)
{
StringBuilder sb = new StringBuilder();
for (int i = 0; i < size(); i++)
{
ColumnMetadata c = metadata.clusteringColumns().get(i);
sb.append(i == 0 ? "" : ", ").append(c.type.toCQLString(bufferAt(i)));
sb.append(i == 0 ? "" : ", ").append(c.type.toCQLString(bufferAt(i), redact));
}
return sb.toString();
}
Expand Down
25 changes: 13 additions & 12 deletions src/java/org/apache/cassandra/db/DataRange.java
Original file line number Diff line number Diff line change
Expand Up @@ -301,10 +301,10 @@ public String toString(TableMetadata metadata)
return String.format("range=%s pfilter=%s", keyRange.getString(metadata.partitionKeyType), clusteringIndexFilter.toString(metadata));
}

public String toCQLString(TableMetadata metadata, RowFilter rowFilter)
public String toCQLString(TableMetadata metadata, RowFilter rowFilter, boolean redact)
{
if (isUnrestricted(metadata))
return rowFilter.toCQLString();
return rowFilter.toCQLString(redact);

if (isSinglePartition())
{
Expand All @@ -313,20 +313,20 @@ public String toCQLString(TableMetadata metadata, RowFilter rowFilter)
* key are the same. If that is the case, we want to print the query as an equality on the partition key
* rather than a token range, as if it was a partition query, for better readability.
*/
return ((DecoratedKey) startKey()).toCQLString(metadata);
return ((DecoratedKey) startKey()).toCQLString(metadata, redact);
}
else
{
StringBuilder builder = new StringBuilder();
if (!startKey().isMinimum())
{
appendCQLClause(startKey(), builder, metadata, true, keyRange.isStartInclusive());
appendCQLClause(startKey(), builder, metadata, true, keyRange.isStartInclusive(), redact);
}
if (!stopKey().isMinimum())
{
if (builder.length() > 0)
builder.append(" AND ");
appendCQLClause(stopKey(), builder, metadata, false, keyRange.isEndInclusive());
appendCQLClause(stopKey(), builder, metadata, false, keyRange.isEndInclusive(), redact);
}
return builder.toString();
}
Expand All @@ -336,7 +336,8 @@ private void appendCQLClause(PartitionPosition pos,
StringBuilder builder,
TableMetadata metadata,
boolean isStart,
boolean isInclusive)
boolean isInclusive,
boolean redact)
{
builder.append("token(");
builder.append(ColumnMetadata.toCQLString(metadata.partitionKeyColumns()));
Expand All @@ -345,14 +346,14 @@ private void appendCQLClause(PartitionPosition pos,
{
builder.append(getOperator(isStart, isInclusive)).append(" ");
builder.append("token(");
appendKeyString(builder, metadata.partitionKeyType, ((DecoratedKey)pos).getKey());
appendKeyString(builder, metadata.partitionKeyType, ((DecoratedKey)pos).getKey(), redact);
builder.append(")");
}
else
{
Token.KeyBound keyBound = (Token.KeyBound) pos;
builder.append(getOperator(isStart, isStart == keyBound.isMinimumBound)).append(" ");
builder.append(keyBound.getToken());
builder.append(getOperator(isStart, isStart == keyBound.isMinimumBound)).append(' ');
builder.append(redact ? "?" : keyBound.getToken());
}
}

Expand All @@ -365,18 +366,18 @@ private static String getOperator(boolean isStart, boolean isInclusive)

// TODO: this is reused in SinglePartitionReadCommand but this should not really be here. Ideally
// we need a more "native" handling of composite partition keys.
public static void appendKeyString(StringBuilder builder, AbstractType<?> type, ByteBuffer key)
public static void appendKeyString(StringBuilder builder, AbstractType<?> type, ByteBuffer key, boolean redact)
{
if (type instanceof CompositeType)
{
CompositeType ct = (CompositeType)type;
ByteBuffer[] values = ct.split(key);
for (int i = 0; i < ct.subTypes().size(); i++)
builder.append(i == 0 ? "" : ", ").append(ct.subTypes().get(i).toCQLString(values[i]));
builder.append(i == 0 ? "" : ", ").append(ct.subTypes().get(i).toCQLString(values[i], redact));
}
else
{
builder.append(type.toCQLString(key));
builder.append(type.toCQLString(key, redact));
}
}

Expand Down
11 changes: 6 additions & 5 deletions src/java/org/apache/cassandra/db/DecoratedKey.java
Original file line number Diff line number Diff line change
Expand Up @@ -165,27 +165,28 @@ public String toString()
* Returns a CQL representation of this key.
*
* @param metadata the metadata of the table that this key belogs to
* @param redact whether to redact the key value, as in "k1 = ? AND k2 = ?".
* @return a CQL representation of this key
*/
public String toCQLString(TableMetadata metadata)
public String toCQLString(TableMetadata metadata, boolean redact)
{
List<ColumnMetadata> columns = metadata.partitionKeyColumns();

if (columns.size() == 1)
return toCQLString(columns.get(0), getKey());
return toCQLString(columns.get(0), getKey(), redact);

ByteBuffer[] values = ((CompositeType) metadata.partitionKeyType).split(getKey());
StringJoiner joiner = new StringJoiner(" AND ");

for (int i = 0; i < columns.size(); i++)
joiner.add(toCQLString(columns.get(i), values[i]));
joiner.add(toCQLString(columns.get(i), values[i], redact));

return joiner.toString();
}

private static String toCQLString(ColumnMetadata metadata, ByteBuffer key)
private static String toCQLString(ColumnMetadata metadata, ByteBuffer key, boolean redact)
{
return String.format("%s = %s", metadata.name.toCQLString(), metadata.type.toCQLString(key));
return String.format("%s = %s", metadata.name.toCQLString(), metadata.type.toCQLString(key, redact));
}

public Token getToken()
Expand Down
12 changes: 6 additions & 6 deletions src/java/org/apache/cassandra/db/MultiPartitionReadQuery.java
Original file line number Diff line number Diff line change
Expand Up @@ -28,18 +28,18 @@ public interface MultiPartitionReadQuery extends ReadQuery
{
List<DataRange> ranges();

default void appendCQLWhereClause(CqlBuilder builder)
default void appendCQLWhereClause(CqlBuilder builder, boolean redact)
{
// Append the data ranges.
TableMetadata metadata = metadata();
boolean hasRanges = appendRanges(builder);
boolean hasRanges = appendRanges(builder, redact);

// Append the clustering index filter and the row filter.
String filter = ranges().get(0).clusteringIndexFilter.toCQLString(metadata, rowFilter());
String filter = ranges().get(0).clusteringIndexFilter.toCQLString(metadata, rowFilter(), redact);
builder.appendRestrictions(filter, hasRanges);
}

private boolean appendRanges(CqlBuilder builder)
private boolean appendRanges(CqlBuilder builder, boolean redact)
{
List<DataRange> ranges = ranges();
if (ranges.size() == 1)
Expand All @@ -48,7 +48,7 @@ private boolean appendRanges(CqlBuilder builder)
if (range.isUnrestricted(metadata()))
return false;

String rangeString = range.toCQLString(metadata(), rowFilter());
String rangeString = range.toCQLString(metadata(), rowFilter(), redact);
if (!rangeString.isEmpty())
{
builder.append(" WHERE ").append(rangeString);
Expand All @@ -62,7 +62,7 @@ private boolean appendRanges(CqlBuilder builder)
{
if (i > 0)
builder.append(" OR ");
builder.append(ranges.get(i).toCQLString(metadata(), rowFilter()));
builder.append(ranges.get(i).toCQLString(metadata(), rowFilter(), redact));
}
builder.append(')');
return true;
Expand Down
4 changes: 2 additions & 2 deletions src/java/org/apache/cassandra/db/MultiRangeReadCommand.java
Original file line number Diff line number Diff line change
Expand Up @@ -374,9 +374,9 @@ public Verb verb()
}

@Override
public void appendCQLWhereClause(CqlBuilder builder)
public void appendCQLWhereClause(CqlBuilder builder, boolean redact)
{
MultiPartitionReadQuery.super.appendCQLWhereClause(builder);
MultiPartitionReadQuery.super.appendCQLWhereClause(builder, redact);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -479,9 +479,9 @@ public Verb verb()
}

@Override
public void appendCQLWhereClause(CqlBuilder builder)
public void appendCQLWhereClause(CqlBuilder builder, boolean redact)
{
PartitionRangeReadQuery.super.appendCQLWhereClause(builder);
PartitionRangeReadQuery.super.appendCQLWhereClause(builder, redact);
}

@Override
Expand Down
4 changes: 2 additions & 2 deletions src/java/org/apache/cassandra/db/PartitionRangeReadQuery.java
Original file line number Diff line number Diff line change
Expand Up @@ -99,9 +99,9 @@ default boolean selectsFullPartition()
return dataRange().selectsAllPartition() && !rowFilter().hasExpressionOnClusteringOrRegularColumns();
}

default void appendCQLWhereClause(StringBuilder sb)
default void appendCQLWhereClause(StringBuilder sb, boolean redact)
{
String filterString = dataRange().toCQLString(metadata(), rowFilter());
String filterString = dataRange().toCQLString(metadata(), rowFilter(), redact);
if (!filterString.isEmpty())
sb.append(" WHERE ").append(filterString);
}
Expand Down
6 changes: 3 additions & 3 deletions src/java/org/apache/cassandra/db/ReadCommand.java
Original file line number Diff line number Diff line change
Expand Up @@ -684,7 +684,7 @@ private Threshold.GuardedCounter createTombstoneCounter()
Threshold guardrail = shouldRespectTombstoneThresholds()
? Guardrails.scannedTombstones
: Threshold.NEVER_TRIGGERED;
return guardrail.newCounter(ReadCommand.this::toCQLString, true, null);
return guardrail.newCounter(ReadCommand.this::toRedactedCQLString, false, null);
}

private MetricRecording()
Expand Down Expand Up @@ -748,7 +748,7 @@ private void countTombstone(ClusteringPrefix<?> clustering)
{
metric.tombstoneFailures.inc();
throw new TombstoneOverwhelmingException(tombstones.get(),
ReadCommand.this.toCQLString(),
ReadCommand.this.toRedactedCQLString(),
ReadCommand.this.metadata(),
currentKey,
clustering);
Expand Down Expand Up @@ -848,7 +848,7 @@ private void addSize(long size)
if (failBytes != -1 && this.sizeInBytes >= failBytes)
{
String msg = String.format("Query %s attempted to read %d bytes but max allowed is %s; query aborted (see local_read_size_fail_threshold)",
ReadCommand.this.toCQLString(), this.sizeInBytes, failThreshold);
ReadCommand.this.toRedactedCQLString(), this.sizeInBytes, failThreshold);
Tracing.trace(msg);
MessageParams.remove(ParamType.LOCAL_READ_SIZE_WARN);
MessageParams.add(ParamType.LOCAL_READ_SIZE_FAIL, this.sizeInBytes);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -138,7 +138,7 @@ public void doVerb(Message<ReadCommand> message)
}
catch (AssertionError t)
{
throw new AssertionError(String.format("Caught an error while trying to process the command: %s", command.toCQLString()), t);
throw new AssertionError(String.format("Caught an error while trying to process the command: %s", command.toRedactedCQLString()), t);
}
catch (QueryCancelledException e)
{
Expand Down
Loading