You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by al...@apache.org on 2014/09/18 06:03:50 UTC

git commit: Follow up to CASSANDRA-7719

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 83d15631f -> ad89d4437


Follow up to CASSANDRA-7719

patch by Aleksey Yeschenko; reviewed by Jake Luciani for CASSANDRA-7719


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/ad89d443
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/ad89d443
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/ad89d443

Branch: refs/heads/cassandra-2.1
Commit: ad89d4437bd2997f0c9b7bc1812e27d29a41b8db
Parents: 83d1563
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Wed Sep 17 21:02:15 2014 -0700
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Wed Sep 17 21:02:15 2014 -0700

----------------------------------------------------------------------
 .../apache/cassandra/cql3/QueryProcessor.java   |  48 +++----
 .../apache/cassandra/metrics/CQLMetrics.java    |  60 +++++++++
 .../cassandra/metrics/CqlStatementMetrics.java  |  53 --------
 .../apache/cassandra/cql3/CqlMetricsTest.java   | 114 -----------------
 .../cassandra/metrics/CQLMetricsTest.java       | 124 +++++++++++++++++++
 5 files changed, 203 insertions(+), 196 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/ad89d443/src/java/org/apache/cassandra/cql3/QueryProcessor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/QueryProcessor.java b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
index efd1ebb..419a3e1 100644
--- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
@@ -40,7 +40,7 @@ import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.exceptions.*;
-import org.apache.cassandra.metrics.CqlStatementMetrics;
+import org.apache.cassandra.metrics.CQLMetrics;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.service.pager.QueryPager;
@@ -88,7 +88,7 @@ public class QueryProcessor implements QueryHandler
     private static final ConcurrentMap<String, ParsedStatement.Prepared> internalStatements = new ConcurrentHashMap<>();
 
     @VisibleForTesting
-    public static final CqlStatementMetrics metrics = new CqlStatementMetrics();
+    public static final CQLMetrics metrics = new CQLMetrics();
 
     static
     {
@@ -97,10 +97,8 @@ public class QueryProcessor implements QueryHandler
                              .weigher(cqlMemoryUsageWeigher)
                              .listener(new EvictionListener<MD5Digest, ParsedStatement.Prepared>()
                              {
-                                 @Override
                                  public void onEviction(MD5Digest md5Digest, ParsedStatement.Prepared prepared)
                                  {
-                                     metrics.activePreparedStatements.dec();
                                  }
                              }).build();
 
@@ -109,17 +107,20 @@ public class QueryProcessor implements QueryHandler
                                    .weigher(thriftMemoryUsageWeigher)
                                    .listener(new EvictionListener<Integer, CQLStatement>()
                                    {
-                                       @Override
                                        public void onEviction(Integer integer, CQLStatement cqlStatement)
                                        {
-                                           metrics.activePreparedStatements.dec();
                                        }
                                    })
                                    .build();
 
     }
 
-    // Work aound initialization dependency
+    public static int preparedStatementsCount()
+    {
+        return preparedStatements.size() + thriftPreparedStatements.size();
+    }
+
+    // Work around initialization dependency
     private static enum InternalStateInstance
     {
         INSTANCE;
@@ -228,7 +229,7 @@ public class QueryProcessor implements QueryHandler
             throw new InvalidRequestException("Invalid amount of bind variables");
 
         if (!queryState.getClientState().isInternal)
-            metrics.executedUnprepared.inc();
+            metrics.regularStatementsExecuted.inc();
 
         return processStatement(prepared, queryState, options);
     }
@@ -397,28 +398,17 @@ public class QueryProcessor implements QueryHandler
             throw new InvalidRequestException(String.format("Prepared statement of size %d bytes is larger than allowed maximum of %d bytes.",
                                                             statementSize,
                                                             MAX_CACHE_PREPARED_MEMORY));
-        try
+        if (forThrift)
         {
-            if (forThrift)
-            {
-                int statementId = toHash.hashCode();
-                thriftPreparedStatements.put(statementId, prepared.statement);
-                logger.trace("Stored prepared statement #{} with {} bind markers",
-                        statementId,
-                        prepared.statement.getBoundTerms());
-                return ResultMessage.Prepared.forThrift(statementId, prepared.boundNames);
-            } else
-            {
-                MD5Digest statementId = MD5Digest.compute(toHash);
-                preparedStatements.put(statementId, prepared);
-                logger.trace("Stored prepared statement #{} with {} bind markers",
-                        statementId,
-                        prepared.statement.getBoundTerms());
-                return new ResultMessage.Prepared(statementId, prepared);
-            }
-        } finally
+            int statementId = toHash.hashCode();
+            thriftPreparedStatements.put(statementId, prepared.statement);
+            return ResultMessage.Prepared.forThrift(statementId, prepared.boundNames);
+        }
+        else
         {
-            metrics.activePreparedStatements.inc();
+            MD5Digest statementId = MD5Digest.compute(toHash);
+            preparedStatements.put(statementId, prepared);
+            return new ResultMessage.Prepared(statementId, prepared);
         }
     }
 
@@ -441,7 +431,7 @@ public class QueryProcessor implements QueryHandler
                     logger.trace("[{}] '{}'", i+1, variables.get(i));
         }
 
-        metrics.executedPrepared.inc();
+        metrics.preparedStatementsExecuted.inc();
         return processStatement(statement, queryState, options);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ad89d443/src/java/org/apache/cassandra/metrics/CQLMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/CQLMetrics.java b/src/java/org/apache/cassandra/metrics/CQLMetrics.java
new file mode 100644
index 0000000..79a8e10
--- /dev/null
+++ b/src/java/org/apache/cassandra/metrics/CQLMetrics.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.metrics;
+
+import org.apache.cassandra.cql3.QueryProcessor;
+import com.yammer.metrics.Metrics;
+import com.yammer.metrics.core.Counter;
+import com.yammer.metrics.core.Gauge;
+import com.yammer.metrics.util.RatioGauge;
+
+public class CQLMetrics
+{
+    private static final MetricNameFactory factory = new DefaultNameFactory("CQL");
+
+    public final Counter regularStatementsExecuted;
+    public final Counter preparedStatementsExecuted;
+
+    public final Gauge<Integer> preparedStatementsCount;
+    public final Gauge<Double> preparedStatementsRatio;
+
+    public CQLMetrics()
+    {
+        regularStatementsExecuted = Metrics.newCounter(factory.createMetricName("RegularStatementsExecuted"));
+        preparedStatementsExecuted = Metrics.newCounter(factory.createMetricName("PreparedStatementsExecuted"));
+        preparedStatementsCount = Metrics.newGauge(factory.createMetricName("PreparedStatementsCount"), new Gauge<Integer>()
+        {
+            public Integer value()
+            {
+                return QueryProcessor.preparedStatementsCount();
+            }
+        });
+        preparedStatementsRatio = Metrics.newGauge(factory.createMetricName("PreparedStatementsRatio"), new RatioGauge()
+        {
+            public double getNumerator()
+            {
+                return preparedStatementsExecuted.count();
+            }
+
+            public double getDenominator()
+            {
+                return regularStatementsExecuted.count() + preparedStatementsExecuted.count();
+            }
+        });
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ad89d443/src/java/org/apache/cassandra/metrics/CqlStatementMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/CqlStatementMetrics.java b/src/java/org/apache/cassandra/metrics/CqlStatementMetrics.java
deleted file mode 100644
index 02b4ad0..0000000
--- a/src/java/org/apache/cassandra/metrics/CqlStatementMetrics.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.metrics;
-
-import com.yammer.metrics.Metrics;
-import com.yammer.metrics.core.Counter;
-import com.yammer.metrics.core.Gauge;
-import com.yammer.metrics.util.RatioGauge;
-
-
-public class CqlStatementMetrics
-{
-    private final MetricNameFactory factory = new DefaultNameFactory("CqlStatement");
-    public final Counter activePreparedStatements = Metrics.newCounter(factory.createMetricName("ActivePreparedStatements"));
-    public final Counter executedPrepared = Metrics.newCounter(factory.createMetricName("ExecutedPrepared"));
-    public final Counter executedUnprepared = Metrics.newCounter(factory.createMetricName("ExecutedUnPrepared"));
-
-    public final Gauge<Double> preparedRatio = Metrics.newGauge(factory.createMetricName("PreparedUnpreparedRatio"), new RatioGauge()
-    {
-        protected double getNumerator()
-        {
-            long num = executedPrepared.count();
-            return num == 0 ? 1 : num;
-        }
-
-        protected double getDenominator()
-        {
-            long den = executedUnprepared.count();
-            return den == 0 ? 1 : den;
-        }
-    });
-
-    public void reset()
-    {
-        executedPrepared.clear();
-        executedUnprepared.clear();
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ad89d443/test/unit/org/apache/cassandra/cql3/CqlMetricsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CqlMetricsTest.java b/test/unit/org/apache/cassandra/cql3/CqlMetricsTest.java
deleted file mode 100644
index 5dea2fb..0000000
--- a/test/unit/org/apache/cassandra/cql3/CqlMetricsTest.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.cql3;
-
-import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.PreparedStatement;
-import com.datastax.driver.core.Session;
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
-import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.service.EmbeddedCassandraService;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-
-import java.io.IOException;
-
-
-@RunWith(OrderedJUnit4ClassRunner.class)
-public class CqlMetricsTest extends SchemaLoader
-{
-
-    private static EmbeddedCassandraService cassandra;
-
-    private static Cluster cluster;
-    private static Session session;
-    private static PreparedStatement metricsStatement;
-
-    @BeforeClass()
-    public static void setup() throws ConfigurationException, IOException
-    {
-        Schema.instance.clear();
-
-        cassandra = new EmbeddedCassandraService();
-        cassandra.start();
-
-        cluster = Cluster.builder().addContactPoint("127.0.0.1").withPort(DatabaseDescriptor.getNativeTransportPort()).build();
-        session = cluster.connect();
-
-        session.execute("drop keyspace if exists junit;");
-        session.execute("create keyspace junit WITH REPLICATION = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 };");
-        session.execute("CREATE TABLE junit.metricstest (\n" +
-                "  id int PRIMARY KEY,\n" +
-                "  val text\n" +
-                ");");
-    }
-
-    @Test
-    public void testActivePreparedStatements()
-    {
-        assert QueryProcessor.metrics.activePreparedStatements.count() == 0;
-
-        metricsStatement = session.prepare("insert into junit.metricstest(id, val)values(?,?)");
-
-        assert QueryProcessor.metrics.activePreparedStatements.count() == 1;
-    }
-
-    @Test
-    public void testExecutedPrepared()
-    {
-        QueryProcessor.metrics.reset();
-
-        assert QueryProcessor.metrics.activePreparedStatements.count() == 1;
-        assert QueryProcessor.metrics.executedPrepared.count() == 0;
-        assert QueryProcessor.metrics.executedUnprepared.count() == 0;
-        assert QueryProcessor.metrics.preparedRatio.value() == 1.0;
-
-        for (int i = 0; i < 10; i++)
-        {
-            session.execute(metricsStatement.bind(i, "val"+i));
-        }
-
-        assert QueryProcessor.metrics.executedPrepared.count() == 10;
-        assert QueryProcessor.metrics.executedUnprepared.count() == 0;
-        assert QueryProcessor.metrics.preparedRatio.value() == 10d/1d;
-
-    }
-
-    @Test
-    public void testExecutedUnPrepared()
-    {
-        QueryProcessor.metrics.reset();
-
-        assert QueryProcessor.metrics.activePreparedStatements.count() == 1;
-        assert QueryProcessor.metrics.executedPrepared.count() == 0;
-        assert QueryProcessor.metrics.executedUnprepared.count() == 0;
-
-        for (int i = 0; i < 10; i++)
-        {
-            session.execute(String.format("insert into junit.metricstest(id, val)values(%d,'%s')",i, "val"+1));
-        }
-
-        assert QueryProcessor.metrics.executedPrepared.count() == 0;
-        assert QueryProcessor.metrics.executedUnprepared.count() == 10;
-        assert QueryProcessor.metrics.preparedRatio.value() == 1d/10d;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ad89d443/test/unit/org/apache/cassandra/metrics/CQLMetricsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/metrics/CQLMetricsTest.java b/test/unit/org/apache/cassandra/metrics/CQLMetricsTest.java
new file mode 100644
index 0000000..91abf86
--- /dev/null
+++ b/test/unit/org/apache/cassandra/metrics/CQLMetricsTest.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.metrics;
+
+import java.io.IOException;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.Session;
+
+import org.apache.cassandra.OrderedJUnit4ClassRunner;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.service.EmbeddedCassandraService;
+
+import static junit.framework.Assert.assertEquals;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+@RunWith(OrderedJUnit4ClassRunner.class)
+public class CQLMetricsTest extends SchemaLoader
+{
+    private static EmbeddedCassandraService cassandra;
+
+    private static Cluster cluster;
+    private static Session session;
+    private static PreparedStatement metricsStatement;
+
+    @BeforeClass()
+    public static void setup() throws ConfigurationException, IOException
+    {
+        Schema.instance.clear();
+
+        cassandra = new EmbeddedCassandraService();
+        cassandra.start();
+
+        cluster = Cluster.builder().addContactPoint("127.0.0.1").withPort(DatabaseDescriptor.getNativeTransportPort()).build();
+        session = cluster.connect();
+
+        session.execute("CREATE KEYSPACE IF NOT EXISTS junit WITH replication = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 };");
+        session.execute("CREATE TABLE IF NOT EXISTS junit.metricstest (id int PRIMARY KEY, val text);");
+    }
+
+    @Test
+    public void testPreparedStatementsCount()
+    {
+        assertEquals(0, (int) QueryProcessor.metrics.preparedStatementsCount.value());
+        metricsStatement = session.prepare("INSERT INTO junit.metricstest (id, val) VALUES (?, ?)");
+        assertEquals(1, (int) QueryProcessor.metrics.preparedStatementsCount.value());
+    }
+
+    @Test
+    public void testRegularStatementsExecuted()
+    {
+        clearMetrics();
+
+        assertEquals(0, QueryProcessor.metrics.preparedStatementsExecuted.count());
+        assertEquals(0, QueryProcessor.metrics.regularStatementsExecuted.count());
+
+        for (int i = 0; i < 10; i++)
+            session.execute(String.format("INSERT INTO junit.metricstest (id, val) VALUES (%d, '%s')", i, "val" + i));
+
+        assertEquals(0, QueryProcessor.metrics.preparedStatementsExecuted.count());
+        assertEquals(10, QueryProcessor.metrics.regularStatementsExecuted.count());
+    }
+
+    @Test
+    public void testPreparedStatementsExecuted()
+    {
+        clearMetrics();
+
+        assertEquals(0, QueryProcessor.metrics.preparedStatementsExecuted.count());
+        assertEquals(0, QueryProcessor.metrics.regularStatementsExecuted.count());
+
+        for (int i = 0; i < 10; i++)
+            session.execute(metricsStatement.bind(i, "val" + i));
+
+        assertEquals(10, QueryProcessor.metrics.preparedStatementsExecuted.count());
+        assertEquals(0, QueryProcessor.metrics.regularStatementsExecuted.count());
+    }
+
+    @Test
+    public void testPreparedStatementsRatio()
+    {
+        clearMetrics();
+
+        assertEquals(Double.NaN, QueryProcessor.metrics.preparedStatementsRatio.value());
+
+        for (int i = 0; i < 10; i++)
+            session.execute(metricsStatement.bind(i, "val" + i));
+        assertEquals(1.0, QueryProcessor.metrics.preparedStatementsRatio.value());
+
+        for (int i = 0; i < 10; i++)
+            session.execute(String.format("INSERT INTO junit.metricstest (id, val) VALUES (%d, '%s')", i, "val" + i));
+        assertEquals(0.5, QueryProcessor.metrics.preparedStatementsRatio.value());
+    }
+
+    private void clearMetrics()
+    {
+        QueryProcessor.metrics.preparedStatementsExecuted.clear();
+        QueryProcessor.metrics.regularStatementsExecuted.clear();
+    }
+}
+