CASSANDRA-14618 Create fqltool replay command#255
CASSANDRA-14618 Create fqltool replay command#255jasobrown wants to merge 8 commits intoapache:trunkfrom
Conversation
…ies executed in the same order they were executed
jasobrown
left a comment
There was a problem hiding this comment.
I'm half way done with the review, so posting what I have now.
| @Option(title = "use", name = {"--use"}, description = "Connect to the cluster(s) using this keyspace.") | ||
| private String useKeyspace; | ||
|
|
||
| @Option(title = "legacy", name = {"--legacyfiles"}, description = "If the FQL files don't contain keyspace information.") |
There was a problem hiding this comment.
Is this flag necessary post CASSANDRA-14656?
If it is needed, a minor nit cleanup to the description: "A flag to indicate if the FQL files do not contain keyspace information."
There was a problem hiding this comment.
no we don't need this here since full query logs have not really been released yet, I'll remove it
There was a problem hiding this comment.
and yes, at some point we should add versioning to the full query logs
| @Override | ||
| public void run() | ||
| { | ||
| try |
There was a problem hiding this comment.
Do we need parameter checking here? For example, what if i set notargetHosts?
| queries.clear(); | ||
| } | ||
| } | ||
|
|
There was a problem hiding this comment.
petty nit: superfluous blank lines
| } | ||
| catch (Throwable t) | ||
| { | ||
| out.println("USE "+query.keyspace+" failed: "+t.getMessage()); |
There was a problem hiding this comment.
petty nit: spacing around the + signs.
| } | ||
| catch (Throwable t) | ||
| { | ||
| out.println("QUERY " + query +" got exception: "+t.getMessage()); |
There was a problem hiding this comment.
petty nit: spacing around the + signs.
| @Option(title = "debug", name = {"--debug"}, description = "Debug mode, print all queries executed.") | ||
| private boolean debug; | ||
|
|
||
| @Option(title = "use", name = {"--use"}, description = "Connect to the cluster(s) using this keyspace.") |
There was a problem hiding this comment.
This isn't entirely clear to me. The user can state which keyspace to initially connect to, but then we'll switch the keyspaces on -demand in QueryReplayer.replayer(). What is the intent with the use field?
There was a problem hiding this comment.
this was a left over from before we had keyspace in the full query logs, removed
|
|
||
| public class FQLQueryIterator extends AbstractIterator<FQLQuery> | ||
| { | ||
| private final PriorityQueue<FQLQuery> pq; |
There was a problem hiding this comment.
Please add a small comment here as to why a PriorityQueue ("so we can sort the entries by timestamp of the query in logs, not just the order in the file"). This would just be a nice, quick little helper.
| return 1; | ||
|
|
||
| Batch otherBatch = (Batch) other; | ||
| if (queries.size() != otherBatch.queries.size()) |
There was a problem hiding this comment.
Only the length of of the queries lists are compared, not the actual contents of the queries lists. (Single.compareTo() compares the query strings).
There was a problem hiding this comment.
right below this we iterate over the Single-queries and compare them one-by-one
| { | ||
| private final ExecutorService es = Executors.newFixedThreadPool(1); | ||
| private final Iterator<List<FQLQuery>> queryIterator; | ||
| private final List<String> targetHosts; |
There was a problem hiding this comment.
This variable is unused. Its only used in the constructor.
| if (query.keyspace != null && !query.keyspace.equals(session.getLoggedKeyspace())) | ||
| { | ||
| if (debug) | ||
| { |
There was a problem hiding this comment.
Single statement blocks do not require braces.
| { | ||
| if (debug) | ||
| { | ||
| out.println(String.format("Switching keyspace from %s to %s", session.getLoggedKeyspace(), query.keyspace)); |
There was a problem hiding this comment.
I'm curious why not use out.printf instead?
| } | ||
| catch (Throwable t) | ||
| { | ||
| out.println("USE "+query.keyspace+" failed: "+t.getMessage()); |
| } | ||
|
|
||
| Timer timer = metrics.timer("queries"); | ||
| if (timer.getCount() % 5000 == 0) |
There was a problem hiding this comment.
Is there a reason to choose 5000? Also, it would be better if it were constant and not a magic number.
| { | ||
| StringBuilder sb = new StringBuilder("batch: ").append(batchType).append('\n'); | ||
| for (Single q : queries) | ||
| { |
There was a problem hiding this comment.
Redundant braces for a single statement block.
| values.add(subValues); | ||
| int numSubValues = in.int32(); | ||
| for (int zz = 0; zz < numSubValues; zz++) | ||
| { |
| { | ||
| StringBuilder sb = new StringBuilder(); | ||
| if (cd == null) | ||
| { |
|
|
||
| import com.google.common.annotations.VisibleForTesting; | ||
|
|
||
| import com.datastax.driver.core.ResultSet; |
| private final ChronicleQueue queryStoreQueue; | ||
| private final ExcerptAppender queryStoreAppender; | ||
| private final Set<Integer> finishedHosts = new HashSet<>(); | ||
| private final File queryFilePath; |
jasobrown
left a comment
There was a problem hiding this comment.
another set of comments
| List<ResultHandler.ComparableDefinition> def1 = asList(); | ||
| List<ResultHandler.ComparableDefinition> def2 = o.asList(); | ||
|
|
||
| for (int j = 0; j < def1.size(); j++) |
There was a problem hiding this comment.
I think you can do return asList().equals(o.asList()) instead of the for loop. (j.u.AbstractList.equals() basically does that)
|
|
||
| boolean equal = true; | ||
| List<ResultHandler.ComparableDefinition> refDefs = cds.get(0).asList(); | ||
| for (int i = 1; i < cds.size(); i++) |
There was a problem hiding this comment.
I think this might overlog. If you have greater than two elements in cds, and the first element (refDefs) differs from the rest of cds, the full log entry (via handleColumnDefMismatch()) will be printed n - 1 times. This is because handleColumnDefMismatch() prints the entire cds
|
|
||
| ResultHandler.ComparableRow ref = rows.get(0); | ||
| boolean equal = true; | ||
| for (int i = 1; i < rows.size(); i++) |
There was a problem hiding this comment.
I think this might overlog. If you have greater than two elements in rows, and the first element (ref) differs from the rest of rows, the full log entry (via handleMismatch()) will be printed n - 1 times. This is because handleMismatch() prints the entire rows.
No description provided.