Repository: cassandra Updated Branches: refs/heads/cassandra-2.0 a6802aa47 -> be7914229
Fix millisecond timestamps in Tracing patch by Aleksey Yeschenko; reviewed by Sam Tunnicliffe for CASSANDRA-8297 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/be791422 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/be791422 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/be791422 Branch: refs/heads/cassandra-2.0 Commit: be791422972a2ee485239e3d724ffcab3d8fdace Parents: a6802aa Author: Aleksey Yeschenko <[email protected]> Authored: Wed Nov 12 23:44:00 2014 +0300 Committer: Aleksey Yeschenko <[email protected]> Committed: Wed Nov 12 23:46:53 2014 +0300 ---------------------------------------------------------------------- CHANGES.txt | 1 + .../org/apache/cassandra/config/CFMetaData.java | 4 ++-- .../org/apache/cassandra/tracing/Tracing.java | 21 +++++--------------- 3 files changed, 8 insertions(+), 18 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/be791422/CHANGES.txt ---------------------------------------------------------------------- diff --git a/CHANGES.txt b/CHANGES.txt index 842643c..47e611c 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 2.0.12: + * Fix millisecond timestamps in Tracing (CASSANDRA-8297) * Include keyspace name in error message when there are insufficient live nodes to stream from (CASSANDRA-8221) * Avoid overlap in L1 when L0 contains many nonoverlapping http://git-wip-us.apache.org/repos/asf/cassandra/blob/be791422/src/java/org/apache/cassandra/config/CFMetaData.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/config/CFMetaData.java b/src/java/org/apache/cassandra/config/CFMetaData.java index 391341f..9d69710 100644 --- a/src/java/org/apache/cassandra/config/CFMetaData.java +++ b/src/java/org/apache/cassandra/config/CFMetaData.java @@ -217,7 +217,7 @@ public final class CFMetaData + "started_at timestamp," + "parameters map<text, text>," + "duration int" - + ") WITH COMMENT='traced sessions'", + + ") WITH COMMENT='traced sessions' AND default_time_to_live=86400", Tracing.TRACE_KS); public static final CFMetaData TraceEventsCf = compile("CREATE TABLE " + Tracing.EVENTS_CF + " (" @@ -228,7 +228,7 @@ public final class CFMetaData + "activity text," + "source_elapsed int," + "PRIMARY KEY (session_id, event_id)" - + ")", + + ") WITH default_time_to_live=86400", Tracing.TRACE_KS); public static final CFMetaData BatchlogCf = compile("CREATE TABLE " + SystemKeyspace.BATCHLOG_CF + " (" http://git-wip-us.apache.org/repos/asf/cassandra/blob/be791422/src/java/org/apache/cassandra/tracing/Tracing.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/tracing/Tracing.java b/src/java/org/apache/cassandra/tracing/Tracing.java index 88239be..b38dc10 100644 --- a/src/java/org/apache/cassandra/tracing/Tracing.java +++ b/src/java/org/apache/cassandra/tracing/Tracing.java @@ -59,15 +59,13 @@ public class Tracing public static final String SESSIONS_CF = "sessions"; public static final String TRACE_HEADER = "TraceSession"; - private static final int TTL = 24 * 3600; - private static final Logger logger = LoggerFactory.getLogger(Tracing.class); private final InetAddress localAddress = FBUtilities.getLocalAddress(); - private final ThreadLocal<TraceState> state = new ThreadLocal<TraceState>(); + private final ThreadLocal<TraceState> state = new ThreadLocal<>(); - private final ConcurrentMap<UUID, TraceState> sessions = new ConcurrentHashMap<UUID, TraceState>(); + private final ConcurrentMap<UUID, TraceState> sessions = new ConcurrentHashMap<>(); public static final Tracing instance = new Tracing(); @@ -93,16 +91,7 @@ public class Tracing private static void addColumn(ColumnFamily cf, ByteBuffer name, ByteBuffer value) { - cf.addColumn(new ExpiringColumn(name, value, System.currentTimeMillis(), TTL)); - } - - public void addParameterColumns(ColumnFamily cf, Map<String, String> rawPayload) - { - for (Map.Entry<String, String> entry : rawPayload.entrySet()) - { - cf.addColumn(new ExpiringColumn(buildName(cf.metadata(), bytes("parameters"), bytes(entry.getKey())), - bytes(entry.getValue()), System.currentTimeMillis(), TTL)); - } + cf.addColumn(name, value, FBUtilities.timestampMicros()); } public static ByteBuffer buildName(CFMetaData meta, ByteBuffer... args) @@ -208,10 +197,10 @@ public class Tracing CFMetaData cfMeta = CFMetaData.TraceSessionsCf; ColumnFamily cf = TreeMapBackedSortedColumns.factory.create(cfMeta); addColumn(cf, buildName(cfMeta, bytes("coordinator")), FBUtilities.getBroadcastAddress()); - addParameterColumns(cf, parameters); + for (Map.Entry<String, String> entry : parameters.entrySet()) + addColumn(cf, buildName(cf.metadata(), bytes("parameters"), bytes(entry.getKey())), entry.getValue()); addColumn(cf, buildName(cfMeta, bytes("request")), request); addColumn(cf, buildName(cfMeta, bytes("started_at")), started_at); - addParameterColumns(cf, parameters); mutateWithCatch(new RowMutation(TRACE_KS, sessionIdBytes, cf)); } });
