You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by bl...@apache.org on 2015/12/16 15:08:06 UTC

cassandra git commit: Add new types to Stress

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.2 57d558fc1 -> b03ce9fd4


Add new types to Stress

patch by ZhaoYang; reviewed by Benjamin Lerer for CASSANDRA-9556


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

Branch: refs/heads/cassandra-2.2
Commit: b03ce9fd479d3da9c51d118c50480ea96df0d73e
Parents: 57d558f
Author: ZhaoYang <zh...@gmail.com>
Authored: Wed Dec 16 15:06:06 2015 +0100
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Wed Dec 16 15:06:06 2015 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../apache/cassandra/stress/StressProfile.java  |  8 ++++
 .../stress/generate/values/LocalDates.java      | 43 +++++++++++++++++++
 .../stress/generate/values/SmallInts.java       | 38 +++++++++++++++++
 .../cassandra/stress/generate/values/Times.java | 37 ++++++++++++++++
 .../stress/generate/values/TinyInts.java        | 45 ++++++++++++++++++++
 .../operations/userdefined/SchemaStatement.java | 16 +++++--
 .../cassandra/stress/util/JavaDriverClient.java |  2 +-
 8 files changed, 185 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/b03ce9fd/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index c969a4d..1480960 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2.5
+ * Add new types to Stress (CASSANDRA-9556)
  * Add property to allow listening on broadcast interface (CASSANDRA-9748)
  * Fix regression in split size on CqlInputFormat (CASSANDRA-10835)
  * Better handling of SSL connection errors inter-node (CASSANDRA-10816)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b03ce9fd/tools/stress/src/org/apache/cassandra/stress/StressProfile.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/StressProfile.java b/tools/stress/src/org/apache/cassandra/stress/StressProfile.java
index 9c9b921..410f666 100644
--- a/tools/stress/src/org/apache/cassandra/stress/StressProfile.java
+++ b/tools/stress/src/org/apache/cassandra/stress/StressProfile.java
@@ -531,6 +531,14 @@ public class StressProfile implements Serializable
                     return new UUIDs(name, config);
                 case TIMEUUID:
                     return new TimeUUIDs(name, config);
+                case TINYINT:
+                    return new TinyInts(name, config);
+                case SMALLINT:
+                    return new SmallInts(name, config);
+                case TIME:
+                    return new Times(name, config);
+                case DATE:
+                    return new LocalDates(name, config);
                 case SET:
                     return new Sets(name, getGenerator(name, type.getTypeArguments().get(0), config), config);
                 case LIST:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b03ce9fd/tools/stress/src/org/apache/cassandra/stress/generate/values/LocalDates.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/values/LocalDates.java b/tools/stress/src/org/apache/cassandra/stress/generate/values/LocalDates.java
new file mode 100644
index 0000000..f079d35
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/values/LocalDates.java
@@ -0,0 +1,43 @@
+/*
+ *
+ * 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.stress.generate.values;
+
+import com.datastax.driver.core.LocalDate;
+import org.apache.cassandra.db.marshal.SimpleDateType;
+import org.joda.time.DateTimeZone;
+import org.joda.time.format.DateTimeFormat;
+import org.joda.time.format.DateTimeFormatter;
+
+public class LocalDates extends Generator<Integer>
+{
+
+    public LocalDates(String name, GeneratorConfig config)
+    {
+        super(SimpleDateType.instance, config, name, Integer.class);
+    }
+
+    public Integer generate()
+    {
+        return (int)identityDistribution.next();
+    }
+
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b03ce9fd/tools/stress/src/org/apache/cassandra/stress/generate/values/SmallInts.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/values/SmallInts.java b/tools/stress/src/org/apache/cassandra/stress/generate/values/SmallInts.java
new file mode 100644
index 0000000..702b6dc
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/values/SmallInts.java
@@ -0,0 +1,38 @@
+/*
+ *
+ * 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.stress.generate.values;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.ShortType;
+
+public class SmallInts extends Generator<Short>
+{
+    public SmallInts(String name, GeneratorConfig config)
+    {
+        super(ShortType.instance, config, name, Short.class);
+    }
+
+    public Short generate()
+    {
+        long seed = identityDistribution.next();
+        return (short)seed;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b03ce9fd/tools/stress/src/org/apache/cassandra/stress/generate/values/Times.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/values/Times.java b/tools/stress/src/org/apache/cassandra/stress/generate/values/Times.java
new file mode 100644
index 0000000..35bac86
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/values/Times.java
@@ -0,0 +1,37 @@
+/*
+ *
+ * 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.stress.generate.values;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.TimeType;
+
+public class Times extends Generator<Long>
+{
+    public Times(String name, GeneratorConfig config)
+    {
+        super(TimeType.instance, config, name, Long.class);
+    }
+
+    public Long generate()
+    {
+        return identityDistribution.next();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b03ce9fd/tools/stress/src/org/apache/cassandra/stress/generate/values/TinyInts.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/values/TinyInts.java b/tools/stress/src/org/apache/cassandra/stress/generate/values/TinyInts.java
new file mode 100644
index 0000000..0fe3f35
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/values/TinyInts.java
@@ -0,0 +1,45 @@
+/*
+ *
+ * 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.stress.generate.values;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+import org.apache.cassandra.db.marshal.ByteType;
+import org.apache.cassandra.db.marshal.DecimalType;
+import org.apache.cassandra.db.marshal.IntegerType;
+import org.apache.cassandra.db.marshal.ShortType;
+import org.apache.cassandra.stress.generate.FasterRandom;
+
+public class TinyInts extends Generator<Byte>
+{
+    public TinyInts(String name, GeneratorConfig config)
+    {
+        super(ByteType.instance, config, name, Byte.class);
+    }
+
+    public Byte generate()
+    {
+        long seed = identityDistribution.next();
+        return (byte)seed;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b03ce9fd/tools/stress/src/org/apache/cassandra/stress/operations/userdefined/SchemaStatement.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/userdefined/SchemaStatement.java b/tools/stress/src/org/apache/cassandra/stress/operations/userdefined/SchemaStatement.java
index e90de23..49891ec 100644
--- a/tools/stress/src/org/apache/cassandra/stress/operations/userdefined/SchemaStatement.java
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/userdefined/SchemaStatement.java
@@ -28,15 +28,15 @@ import java.util.List;
 
 import com.datastax.driver.core.BoundStatement;
 import com.datastax.driver.core.ColumnDefinitions;
+import com.datastax.driver.core.DataType;
+import com.datastax.driver.core.LocalDate;
 import com.datastax.driver.core.PreparedStatement;
-import com.datastax.driver.core.ResultSet;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.stress.Operation;
 import org.apache.cassandra.stress.generate.Row;
 import org.apache.cassandra.stress.settings.StressSettings;
 import org.apache.cassandra.stress.util.JavaDriverClient;
 import org.apache.cassandra.stress.util.Timer;
-import org.apache.cassandra.thrift.CqlResult;
 import org.apache.cassandra.transport.SimpleClient;
 
 public abstract class SchemaStatement extends Operation
@@ -47,6 +47,7 @@ public abstract class SchemaStatement extends Operation
     final ConsistencyLevel cl;
     final int[] argumentIndex;
     final Object[] bindBuffer;
+    final ColumnDefinitions definitions;
 
     public SchemaStatement(Timer timer, StressSettings settings, DataSpec spec,
                            PreparedStatement statement, Integer thriftId, ConsistencyLevel cl)
@@ -57,8 +58,9 @@ public abstract class SchemaStatement extends Operation
         this.cl = cl;
         argumentIndex = new int[statement.getVariables().size()];
         bindBuffer = new Object[argumentIndex.length];
+        definitions = statement.getVariables();
         int i = 0;
-        for (ColumnDefinitions.Definition definition : statement.getVariables())
+        for (ColumnDefinitions.Definition definition : definitions)
             argumentIndex[i++] = spec.partitionGenerator.indexOf(definition.getName());
 
         statement.setConsistencyLevel(JavaDriverClient.from(cl));
@@ -68,7 +70,13 @@ public abstract class SchemaStatement extends Operation
     {
         for (int i = 0 ; i < argumentIndex.length ; i++)
         {
-            bindBuffer[i] = row.get(argumentIndex[i]);
+            Object value = row.get(argumentIndex[i]);
+            if (definitions.getType(i).getName().equals(DataType.date().getName()))
+            {
+                // the java driver only accepts com.datastax.driver.core.LocalDate for CQL type "DATE"
+                value= LocalDate.fromDaysSinceEpoch((Integer) value);
+            }
+            bindBuffer[i] = value;
             if (bindBuffer[i] == null && !spec.partitionGenerator.permitNulls(argumentIndex[i]))
                 throw new IllegalStateException();
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b03ce9fd/tools/stress/src/org/apache/cassandra/stress/util/JavaDriverClient.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/util/JavaDriverClient.java b/tools/stress/src/org/apache/cassandra/stress/util/JavaDriverClient.java
index d82ebfd..30d0d4a 100644
--- a/tools/stress/src/org/apache/cassandra/stress/util/JavaDriverClient.java
+++ b/tools/stress/src/org/apache/cassandra/stress/util/JavaDriverClient.java
@@ -95,7 +95,7 @@ public class JavaDriverClient
                                                 .addContactPoint(host)
                                                 .withPort(port)
                                                 .withPoolingOptions(poolingOpts)
-                                                .withProtocolVersion(ProtocolVersion.V2)
+                                                .withProtocolVersion(ProtocolVersion.NEWEST_SUPPORTED)
                                                 .withoutMetrics(); // The driver uses metrics 3 with conflict with our version
         if (whitelist != null)
             clusterBuilder.withLoadBalancingPolicy(whitelist);