You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ty...@apache.org on 2014/12/19 20:04:55 UTC

[1/2] cassandra git commit: Improve cleanup of tables and types in CQL unit tests

Repository: cassandra
Updated Branches:
  refs/heads/trunk 4bfba6409 -> f45a1941c


Improve cleanup of tables and types in CQL unit tests

Patch by Benjamin Lerer; reviewed by Tyler Hobbs for CASSANDRA-8427


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

Branch: refs/heads/trunk
Commit: 9775a9c6808b8ed3cf37ca04d3b701284b459893
Parents: d9f5702
Author: blerer <b_...@hotmail.com>
Authored: Fri Dec 19 12:04:23 2014 +0100
Committer: Tyler Hobbs <ty...@apache.org>
Committed: Fri Dec 19 13:03:15 2014 -0600

----------------------------------------------------------------------
 .../org/apache/cassandra/cql3/CQLTester.java    | 72 +++++++++++++-------
 1 file changed, 46 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/9775a9c6/test/unit/org/apache/cassandra/cql3/CQLTester.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java
index 2226153..bdd761d 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -62,8 +62,8 @@ public abstract class CQLTester
         SchemaLoader.prepareServer();
     }
 
-    private String currentTable;
-    private final Set<String> currentTypes = new HashSet<>();
+    private List<String> tables = new ArrayList<>();
+    private List<String> types = new ArrayList<>();
 
     @BeforeClass
     public static void setUpClass() throws Throwable
@@ -79,13 +79,10 @@ public abstract class CQLTester
     @After
     public void afterTest() throws Throwable
     {
-        if (currentTable == null)
-            return;
-
-        final String tableToDrop = currentTable;
-        final Set<String> typesToDrop = currentTypes.isEmpty() ? Collections.emptySet() : new HashSet(currentTypes);
-        currentTable = null;
-        currentTypes.clear();
+        final List<String> tablesToDrop = copy(tables);
+        final List<String> typesToDrop = copy(types);
+        tables = null;
+        types = null;
 
         // We want to clean up after the test, but dropping a table is rather long so just do that asynchronously
         ScheduledExecutors.optionalTasks.execute(new Runnable()
@@ -94,10 +91,11 @@ public abstract class CQLTester
             {
                 try
                 {
-                    schemaChange(String.format("DROP TABLE IF EXISTS %s.%s", KEYSPACE, tableToDrop));
+                    for (int i = tablesToDrop.size() - 1; i >=0; i--)
+                        schemaChange(String.format("DROP TABLE IF EXISTS %s.%s", KEYSPACE, tablesToDrop.get(i)));
 
-                    for (String typeName : typesToDrop)
-                        schemaChange(String.format("DROP TYPE IF EXISTS %s.%s", KEYSPACE, typeName));
+                    for (int i = typesToDrop.size() - 1; i >=0; i--)
+                        schemaChange(String.format("DROP TYPE IF EXISTS %s.%s", KEYSPACE, typesToDrop.get(i)));
 
                     // Dropping doesn't delete the sstables. It's not a huge deal but it's cleaner to cleanup after us
                     // Thas said, we shouldn't delete blindly before the SSTableDeletingTask for the table we drop
@@ -114,7 +112,7 @@ public abstract class CQLTester
                     });
                     latch.await(2, TimeUnit.SECONDS);
 
-                    removeAllSSTables(KEYSPACE, tableToDrop);
+                    removeAllSSTables(KEYSPACE, tablesToDrop);
                 }
                 catch (Exception e)
                 {
@@ -124,10 +122,20 @@ public abstract class CQLTester
         });
     }
 
+    /**
+     * Returns a copy of the specified list.
+     * @return a copy of the specified list.
+     */
+    private static List<String> copy(List<String> list)
+    {
+        return list.isEmpty() ? Collections.<String>emptyList() : new ArrayList<>(list);
+    }
+
     public void flush()
     {
         try
         {
+            String currentTable = currentTable();
             if (currentTable != null)
                 Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable).forceFlush().get();
         }
@@ -146,16 +154,24 @@ public abstract class CQLTester
         return USE_PREPARED_VALUES;
     }
 
-    private static void removeAllSSTables(String ks, String table)
+    private static void removeAllSSTables(String ks, List<String> tables)
     {
         // clean up data directory which are stored as data directory/keyspace/data files
         for (File d : Directories.getKSChildDirectories(ks))
         {
-            if (d.exists() && d.getName().contains(table))
+            if (d.exists() && containsAny(d.getName(), tables))
                 FileUtils.deleteRecursive(d);
         }
     }
 
+    private static boolean containsAny(String filename, List<String> tables)
+    {
+        for (int i = 0, m = tables.size(); i < m; i++)
+            if (filename.contains(tables.get(i)))
+                return true;
+        return false;
+    }
+
     protected String keyspace()
     {
         return KEYSPACE;
@@ -163,14 +179,16 @@ public abstract class CQLTester
 
     protected String currentTable()
     {
-        return currentTable;
+        if (tables.isEmpty())
+            return null;
+        return tables.get(tables.size() - 1);
     }
 
     protected String createType(String query)
     {
         String typeName = "type_" + seqNumber.getAndIncrement();
         String fullQuery = String.format(query, KEYSPACE + "." + typeName);
-        currentTypes.add(typeName);
+        types.add(typeName);
         logger.info(fullQuery);
         schemaChange(fullQuery);
         return typeName;
@@ -178,7 +196,8 @@ public abstract class CQLTester
 
     protected void createTable(String query)
     {
-        currentTable = "table_" + seqNumber.getAndIncrement();
+        String currentTable = "table_" + seqNumber.getAndIncrement();
+        tables.add(currentTable);
         String fullQuery = String.format(query, KEYSPACE + "." + currentTable);
         logger.info(fullQuery);
         schemaChange(fullQuery);
@@ -186,7 +205,8 @@ public abstract class CQLTester
 
     protected void createTableMayThrow(String query) throws Throwable
     {
-        currentTable = "table_" + seqNumber.getAndIncrement();
+        String currentTable = "table_" + seqNumber.getAndIncrement();
+        tables.add(currentTable);
         String fullQuery = String.format(query, KEYSPACE + "." + currentTable);
         logger.info(fullQuery);
         try
@@ -201,14 +221,14 @@ public abstract class CQLTester
 
     protected void alterTable(String query)
     {
-        String fullQuery = String.format(query, KEYSPACE + "." + currentTable);
+        String fullQuery = String.format(query, KEYSPACE + "." + currentTable());
         logger.info(fullQuery);
         schemaChange(fullQuery);
     }
 
     protected void alterTableMayThrow(String query) throws Throwable
     {
-        String fullQuery = String.format(query, KEYSPACE + "." + currentTable);
+        String fullQuery = String.format(query, KEYSPACE + "." + currentTable());
         logger.info(fullQuery);
         try
         {
@@ -222,21 +242,21 @@ public abstract class CQLTester
 
     protected void dropTable(String query)
     {
-        String fullQuery = String.format(query, KEYSPACE + "." + currentTable);
+        String fullQuery = String.format(query, KEYSPACE + "." + currentTable());
         logger.info(fullQuery);
         schemaChange(fullQuery);
     }
 
     protected void createIndex(String query)
     {
-        String fullQuery = String.format(query, KEYSPACE + "." + currentTable);
+        String fullQuery = String.format(query, KEYSPACE + "." + currentTable());
         logger.info(fullQuery);
         schemaChange(fullQuery);
     }
 
     protected void createIndexMayThrow(String query) throws Throwable
     {
-        String fullQuery = String.format(query, KEYSPACE + "." + currentTable);
+        String fullQuery = String.format(query, KEYSPACE + "." + currentTable());
         logger.info(fullQuery);
         try
         {
@@ -270,14 +290,14 @@ public abstract class CQLTester
 
     protected CFMetaData currentTableMetadata()
     {
-        return Schema.instance.getCFMetaData(KEYSPACE, currentTable);
+        return Schema.instance.getCFMetaData(KEYSPACE, currentTable());
     }
 
     protected UntypedResultSet execute(String query, Object... values) throws Throwable
     {
         try
         {
-            query = currentTable == null ? query : String.format(query, KEYSPACE + "." + currentTable);
+            query = currentTable() == null ? query : String.format(query, KEYSPACE + "." + currentTable());
 
             UntypedResultSet rs;
             if (USE_PREPARED_VALUES)


[2/2] cassandra git commit: Merge branch 'cassandra-2.1' into trunk

Posted by ty...@apache.org.
Merge branch 'cassandra-2.1' into trunk


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

Branch: refs/heads/trunk
Commit: f45a1941c585e97648d1c38880354bb74a5562f2
Parents: 4bfba64 9775a9c
Author: Tyler Hobbs <ty...@apache.org>
Authored: Fri Dec 19 13:04:35 2014 -0600
Committer: Tyler Hobbs <ty...@apache.org>
Committed: Fri Dec 19 13:04:35 2014 -0600

----------------------------------------------------------------------
 .../org/apache/cassandra/cql3/CQLTester.java    | 92 ++++++++++++--------
 1 file changed, 56 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f45a1941/test/unit/org/apache/cassandra/cql3/CQLTester.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/CQLTester.java
index 4e536fe,bdd761d..36fe957
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@@ -80,28 -60,10 +80,28 @@@ public abstract class CQLTeste
      {
          // Once per-JVM is enough
          SchemaLoader.prepareServer();
 +
 +        nativeAddr = InetAddress.getLoopbackAddress();
 +
 +        try
 +        {
 +            ServerSocket serverSocket = new ServerSocket(0);
 +            nativePort = serverSocket.getLocalPort();
 +        }
 +        catch (IOException e)
 +        {
 +            throw new RuntimeException(e);
 +        }
      }
  
-     private String currentTable;
-     private final Set<String> currentTypes = new HashSet<>();
-     private final Set<String> currentFunctions = new HashSet<>();
-     private final Set<String> currentAggregates = new HashSet<>();
+     private List<String> tables = new ArrayList<>();
+     private List<String> types = new ArrayList<>();
++    private List<String> functions = new ArrayList<>();
++    private List<String> aggregates = new ArrayList<>();
 +
 +    // We don't use USE_PREPARED_VALUES in the code below so some test can foce value preparation (if the result
 +    // is not expected to be the same without preparation)
 +    private boolean usePrepared = USE_PREPARED_VALUES;
  
      @BeforeClass
      public static void setUpClass() throws Throwable
@@@ -132,22 -79,10 +132,19 @@@
      @After
      public void afterTest() throws Throwable
      {
 +        dropPerTestKeyspace();
 +
 +        // Restore standard behavior in case it was changed
 +        usePrepared = USE_PREPARED_VALUES;
 +
-         if (currentTable == null)
-             return;
- 
-         final String tableToDrop = currentTable;
-         final Set<String> typesToDrop = currentTypes.isEmpty() ? Collections.emptySet() : new HashSet(currentTypes);
-         final Set<String> functionsToDrop = currentFunctions.isEmpty() ? Collections.emptySet() : new HashSet(currentFunctions);
-         final Set<String> aggregatesToDrop = currentAggregates.isEmpty() ? Collections.emptySet() : new HashSet(currentAggregates);
-         currentTable = null;
-         currentTypes.clear();
-         currentFunctions.clear();
-         currentAggregates.clear();
+         final List<String> tablesToDrop = copy(tables);
+         final List<String> typesToDrop = copy(types);
++        final List<String> functionsToDrop = copy(functions);
++        final List<String> aggregatesToDrop = copy(aggregates);
+         tables = null;
+         types = null;
++        functions = null;
++        aggregates = null;
  
          // We want to clean up after the test, but dropping a table is rather long so just do that asynchronously
          ScheduledExecutors.optionalTasks.execute(new Runnable()
@@@ -156,16 -91,11 +153,17 @@@
              {
                  try
                  {
-                     schemaChange(String.format("DROP TABLE IF EXISTS %s.%s", KEYSPACE, tableToDrop));
+                     for (int i = tablesToDrop.size() - 1; i >=0; i--)
+                         schemaChange(String.format("DROP TABLE IF EXISTS %s.%s", KEYSPACE, tablesToDrop.get(i)));
  
-                     for (String aggregateName : aggregatesToDrop)
-                         schemaChange(String.format("DROP AGGREGATE IF EXISTS %s.%s", KEYSPACE, aggregateName));
++                    for (int i = aggregatesToDrop.size() - 1; i >=0; i--)
++                        schemaChange(String.format("DROP AGGREGATE IF EXISTS %s", aggregatesToDrop.get(i)));
 +
-                     for (String functionName : functionsToDrop)
-                         schemaChange(String.format("DROP FUNCTION IF EXISTS %s.%s", KEYSPACE, functionName));
++                    for (int i = functionsToDrop.size() - 1; i >=0; i--)
++                        schemaChange(String.format("DROP FUNCTION IF EXISTS %s", functionsToDrop.get(i)));
 +
-                     for (String typeName : typesToDrop)
-                         schemaChange(String.format("DROP TYPE IF EXISTS %s.%s", KEYSPACE, typeName));
+                     for (int i = typesToDrop.size() - 1; i >=0; i--)
+                         schemaChange(String.format("DROP TYPE IF EXISTS %s.%s", KEYSPACE, typesToDrop.get(i)));
  
                      // Dropping doesn't delete the sstables. It's not a huge deal but it's cleaner to cleanup after us
                      // Thas said, we shouldn't delete blindly before the SSTableDeletingTask for the table we drop
@@@ -192,40 -122,15 +190,49 @@@
          });
      }
  
 +    // lazy initialization for all tests that require Java Driver
 +    private static void requireNetwork() throws ConfigurationException
 +    {
 +        if (server != null)
 +            return;
 +
 +        SystemKeyspace.finishStartup();
 +        StorageService.instance.initServer();
 +        SchemaLoader.startGossiper();
 +
 +        server = new org.apache.cassandra.transport.Server(nativeAddr, nativePort);
 +        server.start();
 +
 +        for (int version = 1; version <= Server.CURRENT_VERSION; version++)
 +        {
 +            if (cluster[version-1] != null)
 +                continue;
 +
 +            cluster[version-1] = Cluster.builder().addContactPoints(nativeAddr)
 +                                  .withClusterName("Test Cluster")
 +                                  .withPort(nativePort)
 +                                  .withProtocolVersion(ProtocolVersion.fromInt(version))
 +                                  .build();
 +            session[version-1] = cluster[version-1].connect();
 +
 +            logger.info("Started Java Driver instance for protocol version {}", version);
 +        }
 +    }
 +
 +    protected void dropPerTestKeyspace() throws Throwable
 +    {
 +        execute(String.format("DROP KEYSPACE IF EXISTS %s", KEYSPACE_PER_TEST));
 +    }
 +
+     /**
+      * Returns a copy of the specified list.
+      * @return a copy of the specified list.
+      */
+     private static List<String> copy(List<String> list)
+     {
+         return list.isEmpty() ? Collections.<String>emptyList() : new ArrayList<>(list);
+     }
+ 
      public void flush()
      {
          try
@@@ -244,20 -154,7 +252,20 @@@
          return USE_PREPARED_VALUES;
      }
  
 +    public static FunctionName parseFunctionName(String qualifiedName)
 +    {
 +        int i = qualifiedName.indexOf('.');
 +        return i == -1
 +               ? FunctionName.nativeFunction(qualifiedName)
 +               : new FunctionName(qualifiedName.substring(0, i).trim(), qualifiedName.substring(i+1).trim());
 +    }
 +
 +    public static String shortFunctionName(String f)
 +    {
 +        return parseFunctionName(f).name;
 +    }
 +
-     private static void removeAllSSTables(String ks, String table)
+     private static void removeAllSSTables(String ks, List<String> tables)
      {
          // clean up data directory which are stored as data directory/keyspace/data files
          for (File d : Directories.getKSChildDirectories(ks))
@@@ -274,19 -179,11 +290,21 @@@
  
      protected String currentTable()
      {
-         return currentTable;
+         if (tables.isEmpty())
+             return null;
+         return tables.get(tables.size() - 1);
      }
  
 +    protected void forcePreparedValues()
 +    {
 +        this.usePrepared = true;
 +    }
 +
 +    protected void stopForcingPreparedValues()
 +    {
 +        this.usePrepared = USE_PREPARED_VALUES;
 +    }
 +
      protected String createType(String query)
      {
          String typeName = "type_" + seqNumber.getAndIncrement();
@@@ -297,48 -194,20 +315,50 @@@
          return typeName;
      }
  
 +    protected String createFunction(String keyspace, String argTypes, String query) throws Throwable
 +    {
 +        String functionName = keyspace + ".function_" + seqNumber.getAndIncrement();
 +        createFunctionOverload(functionName, argTypes, query);
 +        return functionName;
 +    }
 +
 +    protected void createFunctionOverload(String functionName, String argTypes, String query) throws Throwable
 +    {
 +        String fullQuery = String.format(query, functionName);
-         currentFunctions.add(functionName + '(' + argTypes + ')');
++        functions.add(functionName + '(' + argTypes + ')');
 +        logger.info(fullQuery);
 +        execute(fullQuery);
 +    }
 +
 +    protected String createAggregate(String keyspace, String argTypes, String query) throws Throwable
 +    {
 +        String aggregateName = keyspace + "." + "aggregate_" + seqNumber.getAndIncrement();
 +        createAggregateOverload(aggregateName, argTypes, query);
 +        return aggregateName;
 +    }
 +
 +    protected void createAggregateOverload(String aggregateName, String argTypes, String query) throws Throwable
 +    {
 +        String fullQuery = String.format(query, aggregateName);
-         currentAggregates.add(aggregateName + '(' + argTypes + ')');
++        aggregates.add(aggregateName + '(' + argTypes + ')');
 +        logger.info(fullQuery);
 +        execute(fullQuery);
 +    }
 +
      protected void createTable(String query)
      {
-         currentTable = "table_" + seqNumber.getAndIncrement();
+         String currentTable = "table_" + seqNumber.getAndIncrement();
+         tables.add(currentTable);
 -        String fullQuery = String.format(query, KEYSPACE + "." + currentTable);
 +        String fullQuery = formatQuery(query);
          logger.info(fullQuery);
          schemaChange(fullQuery);
      }
  
      protected void createTableMayThrow(String query) throws Throwable
      {
-         currentTable = "table_" + seqNumber.getAndIncrement();
+         String currentTable = "table_" + seqNumber.getAndIncrement();
+         tables.add(currentTable);
 -        String fullQuery = String.format(query, KEYSPACE + "." + currentTable);
 +        String fullQuery = formatQuery(query);
          logger.info(fullQuery);
          try
          {
@@@ -421,22 -290,9 +441,22 @@@
  
      protected CFMetaData currentTableMetadata()
      {
-         return Schema.instance.getCFMetaData(KEYSPACE, currentTable);
+         return Schema.instance.getCFMetaData(KEYSPACE, currentTable());
      }
  
 +    protected com.datastax.driver.core.ResultSet executeNet(int protocolVersion, String query, Object... values) throws Throwable
 +    {
 +        requireNetwork();
 +
 +        return session[protocolVersion-1].execute(formatQuery(query), values);
 +    }
 +
 +    private String formatQuery(String query)
 +    {
-         query = currentTable == null ? query : String.format(query, KEYSPACE + "." + currentTable);
-         return query;
++        String currentTable = currentTable();
++        return currentTable == null ? query : String.format(query, KEYSPACE + "." + currentTable);
 +    }
 +
      protected UntypedResultSet execute(String query, Object... values) throws Throwable
      {
          try
@@@ -547,7 -345,8 +567,7 @@@
              for (int j = 0; j < meta.size(); j++)
              {
                  ColumnSpecification column = meta.get(j);
-                 ByteBuffer expectedByteValue = makeByteBuffer(expected[j], (AbstractType)column.type);
 -                Object expectedValue = expected[j];
 -                ByteBuffer expectedByteValue = makeByteBuffer(expected[j], (AbstractType)column.type);
++                ByteBuffer expectedByteValue = makeByteBuffer(expected[j], column.type);
                  ByteBuffer actualValue = actual.getBytes(column.name.toString());
  
                  if (!Objects.equal(expectedByteValue, actualValue))