You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by if...@apache.org on 2017/11/06 15:50:19 UTC

[09/25] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
index fdbcf7a,a5fa12d..de42647
--- a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
@@@ -87,12 -93,16 +85,12 @@@ public class AlterTableStatement extend
          if (meta.isView())
              throw new InvalidRequestException("Cannot use ALTER TABLE on Materialized View");
  
-         CFMetaData cfm = meta.copy();
+         CFMetaData cfm;
 -
 -        CQL3Type validator = this.validator == null ? null : this.validator.prepare(keyspace());
          ColumnIdentifier columnName = null;
          ColumnDefinition def = null;
 -        if (rawColumnName != null)
 -        {
 -            columnName = rawColumnName.prepare(meta);
 -            def = meta.getColumnDefinition(columnName);
 -        }
 +        CQL3Type.Raw dataType = null;
 +        boolean isStatic = false;
 +        CQL3Type validator = null;
  
          List<ViewDefinition> viewUpdates = null;
          Iterable<ViewDefinition> views = View.findAll(keyspace(), columnFamily());
@@@ -102,146 -112,141 +100,156 @@@
              case ALTER:
                  throw new InvalidRequestException("Altering of types is not allowed");
              case ADD:
-                 if (cfm.isDense())
 -                assert columnName != null;
+                 if (meta.isDense())
                      throw new InvalidRequestException("Cannot add new column to a COMPACT STORAGE table");
  
+                 cfm = meta.copy();
+ 
 -                if (isStatic)
 +                for (AlterTableStatementColumn colData : colNameList)
                  {
 -                    if (!cfm.isCompound())
 -                        throw new InvalidRequestException("Static columns are not allowed in COMPACT STORAGE tables");
 -                    if (cfm.clusteringColumns().isEmpty())
 -                        throw new InvalidRequestException("Static columns are only useful (and thus allowed) if the table has at least one clustering column");
 -                }
 +                    columnName = colData.getColumnName().getIdentifier(cfm);
 +                    def = cfm.getColumnDefinition(columnName);
 +                    dataType = colData.getColumnType();
 +                    assert dataType != null;
 +                    isStatic = colData.getStaticType();
 +                    validator = dataType.prepare(keyspace());
  
 -                if (def != null)
 -                {
 -                    switch (def.kind)
 +
 +                    if (isStatic)
                      {
 -                        case PARTITION_KEY:
 -                        case CLUSTERING:
 -                            throw new InvalidRequestException(String.format("Invalid column name %s because it conflicts with a PRIMARY KEY part", columnName));
 -                        default:
 -                            throw new InvalidRequestException(String.format("Invalid column name %s because it conflicts with an existing column", columnName));
 +                        if (!cfm.isCompound())
 +                            throw new InvalidRequestException("Static columns are not allowed in COMPACT STORAGE tables");
 +                        if (cfm.clusteringColumns().isEmpty())
 +                            throw new InvalidRequestException("Static columns are only useful (and thus allowed) if the table has at least one clustering column");
                      }
 -                }
  
 -                // Cannot re-add a dropped counter column. See #7831.
 -                if (meta.isCounter() && meta.getDroppedColumns().containsKey(columnName.bytes))
 -                    throw new InvalidRequestException(String.format("Cannot re-add previously dropped counter column %s", columnName));
 +                    if (def != null)
 +                    {
 +                        switch (def.kind)
 +                        {
 +                            case PARTITION_KEY:
 +                            case CLUSTERING:
 +                                throw new InvalidRequestException(String.format("Invalid column name %s because it conflicts with a PRIMARY KEY part", columnName));
 +                            default:
 +                                throw new InvalidRequestException(String.format("Invalid column name %s because it conflicts with an existing column", columnName));
 +                        }
 +                    }
  
 -                AbstractType<?> type = validator.getType();
 -                if (type.isCollection() && type.isMultiCell())
 -                {
 -                    if (!cfm.isCompound())
 -                        throw new InvalidRequestException("Cannot use non-frozen collections in COMPACT STORAGE tables");
 -                    if (cfm.isSuper())
 -                        throw new InvalidRequestException("Cannot use non-frozen collections with super column families");
 -
 -                    // If there used to be a non-frozen collection column with the same name (that has been dropped),
 -                    // we could still have some data using the old type, and so we can't allow adding a collection
 -                    // with the same name unless the types are compatible (see #6276).
 -                    CFMetaData.DroppedColumn dropped = cfm.getDroppedColumns().get(columnName.bytes);
 -                    if (dropped != null && dropped.type instanceof CollectionType
 -                        && dropped.type.isMultiCell() && !type.isCompatibleWith(dropped.type))
 +                    // Cannot re-add a dropped counter column. See #7831.
 +                    if (meta.isCounter() && meta.getDroppedColumns().containsKey(columnName.bytes))
 +                        throw new InvalidRequestException(String.format("Cannot re-add previously dropped counter column %s", columnName));
 +
 +                    AbstractType<?> type = validator.getType();
 +                    if (type.isCollection() && type.isMultiCell())
                      {
 -                        String message =
 -                            String.format("Cannot add a collection with the name %s because a collection with the same name"
 -                                          + " and a different type (%s) has already been used in the past",
 -                                          columnName,
 -                                          dropped.type.asCQL3Type());
 -                        throw new InvalidRequestException(message);
 +                        if (!cfm.isCompound())
 +                            throw new InvalidRequestException("Cannot use non-frozen collections in COMPACT STORAGE tables");
 +                        if (cfm.isSuper())
 +                            throw new InvalidRequestException("Cannot use non-frozen collections with super column families");
 +
 +                        // If there used to be a non-frozen collection column with the same name (that has been dropped),
 +                        // we could still have some data using the old type, and so we can't allow adding a collection
 +                        // with the same name unless the types are compatible (see #6276).
 +                        CFMetaData.DroppedColumn dropped = cfm.getDroppedColumns().get(columnName.bytes);
 +                        if (dropped != null && dropped.type instanceof CollectionType
 +                            && dropped.type.isMultiCell() && !type.isCompatibleWith(dropped.type))
 +                        {
 +                            String message =
 +                                String.format("Cannot add a collection with the name %s because a collection with the same name"
 +                                              + " and a different type (%s) has already been used in the past",
 +                                              columnName,
 +                                              dropped.type.asCQL3Type());
 +                            throw new InvalidRequestException(message);
 +                        }
                      }
 -                }
  
 -                cfm.addColumnDefinition(isStatic
 -                                        ? ColumnDefinition.staticDef(cfm, columnName.bytes, type)
 -                                        : ColumnDefinition.regularDef(cfm, columnName.bytes, type));
 +                    cfm.addColumnDefinition(isStatic
 +                                            ? ColumnDefinition.staticDef(cfm, columnName.bytes, type)
 +                                            : ColumnDefinition.regularDef(cfm, columnName.bytes, type));
  
 -                // Adding a column to a table which has an include all view requires the column to be added to the view
 -                // as well
 -                if (!isStatic)
 -                {
 -                    for (ViewDefinition view : views)
 +                    // Adding a column to a table which has an include all view requires the column to be added to the view
 +                    // as well
 +                    if (!isStatic)
                      {
 -                        if (view.includeAllColumns)
 +                        for (ViewDefinition view : views)
                          {
 -                            ViewDefinition viewCopy = view.copy();
 -                            viewCopy.metadata.addColumnDefinition(ColumnDefinition.regularDef(viewCopy.metadata, columnName.bytes, type));
 -                            if (viewUpdates == null)
 -                                viewUpdates = new ArrayList<>();
 -                            viewUpdates.add(viewCopy);
 +                            if (view.includeAllColumns)
 +                            {
 +                                ViewDefinition viewCopy = view.copy();
 +                                viewCopy.metadata.addColumnDefinition(ColumnDefinition.regularDef(viewCopy.metadata, columnName.bytes, type));
 +                                if (viewUpdates == null)
 +                                    viewUpdates = new ArrayList<>();
 +                                viewUpdates.add(viewCopy);
 +                            }
                          }
                      }
                  }
                  break;
  
              case DROP:
-                 if (!cfm.isCQLTable())
 -                assert columnName != null;
+                 if (!meta.isCQLTable())
                      throw new InvalidRequestException("Cannot drop columns from a non-CQL3 table");
  
 -                if (def == null)
 -                    throw new InvalidRequestException(String.format("Column %s was not found in table %s", columnName, columnFamily()));
 -
+                 cfm = meta.copy();
+ 
 -                switch (def.kind)
 +                for (AlterTableStatementColumn colData : colNameList)
                  {
 -                    case PARTITION_KEY:
 -                    case CLUSTERING:
 -                        throw new InvalidRequestException(String.format("Cannot drop PRIMARY KEY part %s", columnName));
 -                    case REGULAR:
 -                    case STATIC:
 -                        ColumnDefinition toDelete = null;
 -                        for (ColumnDefinition columnDef : cfm.partitionColumns())
 -                        {
 -                            if (columnDef.name.equals(columnName))
 -                            {
 -                                toDelete = columnDef;
 -                                break;
 -                            }
 -                        }
 -                        assert toDelete != null;
 -                        cfm.removeColumnDefinition(toDelete);
 -                        cfm.recordColumnDrop(toDelete, deleteTimestamp == null ? queryState.getTimestamp() : deleteTimestamp);
 -                        break;
 -                }
 +                    columnName = colData.getColumnName().getIdentifier(cfm);
 +                    def = cfm.getColumnDefinition(columnName);
  
 -                // If the dropped column is required by any secondary indexes
 -                // we reject the operation, as the indexes must be dropped first
 -                Indexes allIndexes = cfm.getIndexes();
 -                if (!allIndexes.isEmpty())
 -                {
 -                    ColumnFamilyStore store = Keyspace.openAndGetStore(cfm);
 -                    Set<IndexMetadata> dependentIndexes = store.indexManager.getDependentIndexes(def);
 -                    if (!dependentIndexes.isEmpty())
 -                        throw new InvalidRequestException(String.format("Cannot drop column %s because it has " +
 -                                                                        "dependent secondary indexes (%s)",
 -                                                                        def,
 -                                                                        dependentIndexes.stream()
 -                                                                                        .map(i -> i.name)
 -                                                                                        .collect(Collectors.joining(","))));
 -                }
 +                    if (def == null)
 +                        throw new InvalidRequestException(String.format("Column %s was not found in table %s", columnName, columnFamily()));
  
 -                if (!Iterables.isEmpty(views))
 -                    throw new InvalidRequestException(String.format("Cannot drop column %s on base table with materialized views.",
 -                                                                    columnName.toString(),
 -                                                                    keyspace()));
 +                    switch (def.kind)
 +                    {
 +                         case PARTITION_KEY:
 +                         case CLUSTERING:
 +                              throw new InvalidRequestException(String.format("Cannot drop PRIMARY KEY part %s", columnName));
 +                         case REGULAR:
 +                         case STATIC:
 +                              ColumnDefinition toDelete = null;
 +                              for (ColumnDefinition columnDef : cfm.partitionColumns())
 +                              {
 +                                   if (columnDef.name.equals(columnName))
 +                                   {
 +                                       toDelete = columnDef;
 +                                       break;
 +                                   }
 +                               }
 +                             assert toDelete != null;
 +                             cfm.removeColumnDefinition(toDelete);
 +                             cfm.recordColumnDrop(toDelete, deleteTimestamp  == null ? queryState.getTimestamp() : deleteTimestamp);
 +                             break;
 +                    }
 +
 +                    // If the dropped column is required by any secondary indexes
 +                    // we reject the operation, as the indexes must be dropped first
 +                    Indexes allIndexes = cfm.getIndexes();
 +                    if (!allIndexes.isEmpty())
 +                    {
 +                        ColumnFamilyStore store = Keyspace.openAndGetStore(cfm);
 +                        Set<IndexMetadata> dependentIndexes = store.indexManager.getDependentIndexes(def);
 +                        if (!dependentIndexes.isEmpty())
 +                            throw new InvalidRequestException(String.format("Cannot drop column %s because it has " +
 +                                                                            "dependent secondary indexes (%s)",
 +                                                                            def,
 +                                                                            dependentIndexes.stream()
 +                                                                                            .map(i -> i.name)
 +                                                                                            .collect(Collectors.joining(","))));
 +                    }
 +
 +                    if (!Iterables.isEmpty(views))
 +                        throw new InvalidRequestException(String.format("Cannot drop column %s on base table with materialized views.",
 +                                                                        columnName.toString(),
 +                                                                        keyspace()));
 +                }
                  break;
+             case DROP_COMPACT_STORAGE:
+                 if (!meta.isCompactTable())
+                     throw new InvalidRequestException("Cannot DROP COMPACT STORAGE on table without COMPACT STORAGE");
+ 
+                 cfm = meta.asNonCompact();
+                 break;
              case OPTS:
                  if (attrs == null)
                      throw new InvalidRequestException("ALTER TABLE WITH invoked, but no parameters found");
@@@ -265,10 -272,13 +275,12 @@@
  
                  break;
              case RENAME:
+                 cfm = meta.copy();
+ 
 -                for (Map.Entry<ColumnIdentifier.Raw, ColumnIdentifier> entry : renames.entrySet())
 +                for (Map.Entry<ColumnDefinition.Raw, ColumnDefinition.Raw> entry : renames.entrySet())
                  {
 -                    ColumnIdentifier from = entry.getKey().prepare(cfm);
 -                    ColumnIdentifier to = entry.getValue();
 -
 +                    ColumnIdentifier from = entry.getKey().getIdentifier(cfm);
 +                    ColumnIdentifier to = entry.getValue().getIdentifier(cfm);
                      cfm.renameColumn(from, to);
  
                      // If the view includes a renamed column, it must be renamed in the view table and the definition.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/AuthenticationStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
index 204edf4,47d54fe..88afc6b
--- a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
@@@ -107,18 -102,24 +107,23 @@@ public class CreateIndexStatement exten
              if (cd == null)
                  throw new InvalidRequestException("No column definition found for column " + target.column);
  
 +            if (cd.type.referencesDuration())
 +            {
 +                checkFalse(cd.type.isCollection(), "Secondary indexes are not supported on collections containing durations");
 +                checkFalse(cd.type.isTuple(), "Secondary indexes are not supported on tuples containing durations");
 +                checkFalse(cd.type.isUDT(), "Secondary indexes are not supported on UDTs containing durations");
 +                throw invalidRequest("Secondary indexes are not supported on duration columns");
 +            }
 +
              // TODO: we could lift that limitation
-             if (cfm.isCompactTable() && cd.isPrimaryKeyColumn())
-                 throw new InvalidRequestException("Secondary indexes are not supported on PRIMARY KEY columns in COMPACT STORAGE tables");
+             if (cfm.isCompactTable())
+             {
+                 if (cd.isPrimaryKeyColumn())
+                     throw new InvalidRequestException("Secondary indexes are not supported on PRIMARY KEY columns in COMPACT STORAGE tables");
+                 if (cfm.compactValueColumn().equals(cd))
+                     throw new InvalidRequestException("Secondary indexes are not supported on compact value column of COMPACT STORAGE tables");
+             }
  
 -            // It would be possible to support 2ndary index on static columns (but not without modifications of at least ExtendedFilter and
 -            // CompositesIndex) and maybe we should, but that means a query like:
 -            //     SELECT * FROM foo WHERE static_column = 'bar'
 -            // would pull the full partition every time the static column of partition is 'bar', which sounds like offering a
 -            // fair potential for foot-shooting, so I prefer leaving that to a follow up ticket once we have identified cases where
 -            // such indexing is actually useful.
 -            if (!cfm.isCompactTable() && cd.isStatic())
 -                throw new InvalidRequestException("Secondary indexes are not allowed on static columns");
 -
              if (cd.kind == ColumnDefinition.Kind.PARTITION_KEY && cfm.getKeyValidatorAsClusteringComparator().size() == 1)
                  throw new InvalidRequestException(String.format("Cannot create secondary index on partition key column %s", target.column));
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index c8c7570,8ae4d64..56d47b3
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@@ -27,9 -27,9 +27,10 @@@ import org.slf4j.LoggerFactory
  import org.apache.cassandra.auth.Permission;
  import org.apache.cassandra.config.CFMetaData;
  import org.apache.cassandra.config.ColumnDefinition;
++import org.apache.cassandra.config.Schema;
 +import org.apache.cassandra.config.ColumnDefinition.Raw;
  import org.apache.cassandra.config.ViewDefinition;
  import org.apache.cassandra.cql3.*;
 -import org.apache.cassandra.cql3.ColumnIdentifier.Raw;
  import org.apache.cassandra.cql3.functions.Function;
  import org.apache.cassandra.cql3.restrictions.StatementRestrictions;
  import org.apache.cassandra.cql3.selection.Selection;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/ParsedStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/SchemaAlteringStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 33a69e7,1e867bc..d86a47d
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@@ -945,17 -885,16 +945,17 @@@ public class SelectStatement implement
              this.selectClause = selectClause;
              this.whereClause = whereClause;
              this.limit = limit;
 +            this.perPartitionLimit = perPartitionLimit;
          }
  
-         public ParsedStatement.Prepared prepare() throws InvalidRequestException
+         public ParsedStatement.Prepared prepare(ClientState clientState) throws InvalidRequestException
          {
-             return prepare(false);
+             return prepare(false, clientState);
          }
  
-         public ParsedStatement.Prepared prepare(boolean forView) throws InvalidRequestException
+         public ParsedStatement.Prepared prepare(boolean forView, ClientState clientState) throws InvalidRequestException
          {
-             CFMetaData cfm = ThriftValidation.validateColumnFamily(keyspace(), columnFamily());
+             CFMetaData cfm = ThriftValidation.validateColumnFamilyWithCompactMode(keyspace(), columnFamily(), clientState.isNoCompactMode());
              VariableSpecifications boundNames = getBoundVariables();
  
              Selection selection = selectClause.isEmpty()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/UseStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/db/view/View.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
index 9e42101,39f7339..694fe37
--- a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
@@@ -555,51 -527,24 +555,51 @@@ public class CQLSSTableWriter implement
              }
          }
  
 -        @SuppressWarnings("resource")
 -        public CQLSSTableWriter build()
 +        private Types createTypes(String keyspace)
          {
 -            if (directory == null)
 -                throw new IllegalStateException("No ouptut directory specified, you should provide a directory with inDirectory()");
 -            if (schema == null)
 -                throw new IllegalStateException("Missing schema, you should provide the schema for the SSTable to create with forTable()");
 -            if (insert == null)
 -                throw new IllegalStateException("No insert statement specified, you should provide an insert statement through using()");
 +            Types.RawBuilder builder = Types.rawBuilder(keyspace);
 +            for (CreateTypeStatement st : typeStatements)
 +                st.addToRawBuilder(builder);
 +            return builder.build();
 +        }
  
 -            AbstractSSTableSimpleWriter writer = sorted
 -                                               ? new SSTableSimpleWriter(directory, schema, insert.updatedColumns())
 -                                               : new SSTableSimpleUnsortedWriter(directory, schema, insert.updatedColumns(), bufferSizeInMB);
 +        /**
 +         * Creates the table according to schema statement
 +         *
 +         * @param types types this table should be created with
 +         */
 +        private CFMetaData createTable(Types types)
 +        {
 +            CreateTableStatement statement = (CreateTableStatement) schemaStatement.prepare(types).statement;
 +            statement.validate(ClientState.forInternalCalls());
  
 -            if (formatType != null)
 -                writer.setSSTableFormatType(formatType);
 +            CFMetaData cfMetaData = statement.getCFMetaData();
 +
 +            if (partitioner != null)
 +                return cfMetaData.copy(partitioner);
 +            else
 +                return cfMetaData;
 +        }
 +
 +        /**
 +         * Prepares insert statement for writing data to SSTable
 +         *
 +         * @return prepared Insert statement and it's bound names
 +         */
 +        private Pair<UpdateStatement, List<ColumnSpecification>> prepareInsert()
 +        {
-             ParsedStatement.Prepared cqlStatement = insertStatement.prepare();
++            ParsedStatement.Prepared cqlStatement = insertStatement.prepare(ClientState.forInternalCalls());
 +            UpdateStatement insert = (UpdateStatement) cqlStatement.statement;
 +            insert.validate(ClientState.forInternalCalls());
 +
 +            if (insert.hasConditions())
 +                throw new IllegalArgumentException("Conditional statements are not supported");
 +            if (insert.isCounter())
 +                throw new IllegalArgumentException("Counter update statements are not supported");
 +            if (cqlStatement.boundNames.isEmpty())
 +                throw new IllegalArgumentException("Provided insert statement has no bind variables");
  
 -            return new CQLSSTableWriter(writer, insert, boundNames);
 +            return Pair.create(insert, cqlStatement.boundNames);
          }
      }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/repair/RepairRunnable.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/repair/RepairRunnable.java
index c9eed54,77726d4..52cc29d
--- a/src/java/org/apache/cassandra/repair/RepairRunnable.java
+++ b/src/java/org/apache/cassandra/repair/RepairRunnable.java
@@@ -389,8 -385,8 +390,8 @@@ public class RepairRunnable extends Wra
                      throw new Exception("no tracestate");
  
                  String format = "select event_id, source, activity from %s.%s where session_id = ? and event_id > ? and event_id < ?;";
 -                String query = String.format(format, TraceKeyspace.NAME, TraceKeyspace.EVENTS);
 +                String query = String.format(format, SchemaConstants.TRACE_KEYSPACE_NAME, TraceKeyspace.EVENTS);
-                 SelectStatement statement = (SelectStatement) QueryProcessor.parseStatement(query).prepare().statement;
+                 SelectStatement statement = (SelectStatement) QueryProcessor.parseStatement(query).prepare(ClientState.forInternalCalls()).statement;
  
                  ByteBuffer sessionIdBytes = ByteBufferUtil.bytes(sessionId);
                  InetAddress source = FBUtilities.getBroadcastAddress();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/service/ClientState.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/thrift/ThriftValidation.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/transport/messages/StartupMessage.java
index bf4a619,774be6a..eb82292
--- a/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
+++ b/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
@@@ -35,7 -35,7 +35,8 @@@ public class StartupMessage extends Mes
  {
      public static final String CQL_VERSION = "CQL_VERSION";
      public static final String COMPRESSION = "COMPRESSION";
 +    public static final String PROTOCOL_VERSIONS = "PROTOCOL_VERSIONS";
+     public static final String NO_COMPACT = "NO_COMPACT";
  
      public static final Message.Codec<StartupMessage> codec = new Message.Codec<StartupMessage>()
      {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/test/unit/org/apache/cassandra/cql3/ViewTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/ViewTest.java
index 7717a4d,136ae1c..eb9c855
--- a/test/unit/org/apache/cassandra/cql3/ViewTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ViewTest.java
@@@ -48,8 -48,9 +48,10 @@@ import org.apache.cassandra.db.Keyspace
  import org.apache.cassandra.db.SystemKeyspace;
  import org.apache.cassandra.db.compaction.CompactionManager;
  import org.apache.cassandra.db.marshal.AsciiType;
+ import org.apache.cassandra.exceptions.InvalidRequestException;
+ import org.apache.cassandra.exceptions.SyntaxException;
  import org.apache.cassandra.schema.KeyspaceParams;
 +import org.apache.cassandra.transport.ProtocolVersion;
  import org.apache.cassandra.utils.FBUtilities;
  
  import static org.junit.Assert.assertEquals;
@@@ -1407,4 -1344,49 +1408,17 @@@ public class ViewTest extends CQLTeste
  
          assertRows(execute("SELECT count(*) FROM mv_test"), row(1024L));
      }
+ 
 -    @Test
 -    public void testFrozenCollectionsWithComplicatedInnerType() throws Throwable
 -    {
 -        createTable("CREATE TABLE %s (k int, intval int,  listval frozen<list<tuple<text,text>>>, PRIMARY KEY (k))");
 -
 -        execute("USE " + keyspace());
 -        executeNet(protocolVersion, "USE " + keyspace());
 -
 -        createView("mv",
 -                   "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE k IS NOT NULL AND listval IS NOT NULL PRIMARY KEY (k, listval)");
 -
 -        updateView("INSERT INTO %s (k, intval, listval) VALUES (?, ?, fromJson(?))",
 -                   0,
 -                   0,
 -                   "[[\"a\", \"1\"], [\"b\", \"2\"], [\"c\", \"3\"]]");
 -
 -        // verify input
 -        assertRows(execute("SELECT k, toJson(listval) FROM %s WHERE k = ?", 0),
 -                   row(0, "[[\"a\", \"1\"], [\"b\", \"2\"], [\"c\", \"3\"]]"));
 -        assertRows(execute("SELECT k, toJson(listval) from mv"),
 -                   row(0, "[[\"a\", \"1\"], [\"b\", \"2\"], [\"c\", \"3\"]]"));
 -
 -        // update listval with the same value and it will be compared in view generator
 -        updateView("INSERT INTO %s (k, listval) VALUES (?, fromJson(?))",
 -                   0,
 -                   "[[\"a\", \"1\"], [\"b\", \"2\"], [\"c\", \"3\"]]");
 -        // verify result
 -        assertRows(execute("SELECT k, toJson(listval) FROM %s WHERE k = ?", 0),
 -                   row(0, "[[\"a\", \"1\"], [\"b\", \"2\"], [\"c\", \"3\"]]"));
 -        assertRows(execute("SELECT k, toJson(listval) from mv"),
 -                   row(0, "[[\"a\", \"1\"], [\"b\", \"2\"], [\"c\", \"3\"]]"));
 -    }
+ 
+     @Test(expected = SyntaxException.class)
+     public void emptyViewNameTest() throws Throwable
+     {
+         execute("CREATE MATERIALIZED VIEW \"\" AS SELECT a, b FROM tbl WHERE b IS NOT NULL PRIMARY KEY (b, a)");
+     }
+ 
+      @Test(expected = SyntaxException.class)
+      public void emptyBaseTableNameTest() throws Throwable
+      {
+          execute("CREATE MATERIALIZED VIEW myview AS SELECT a, b FROM \"\" WHERE b IS NOT NULL PRIMARY KEY (b, a)");
+      }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
index af9ec1a,6e6af19..6f3616c
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
@@@ -36,11 -34,13 +36,12 @@@ import org.apache.cassandra.cql3.functi
  import org.apache.cassandra.cql3.functions.UDFunction;
  import org.apache.cassandra.db.marshal.CollectionType;
  import org.apache.cassandra.exceptions.InvalidRequestException;
+ import org.apache.cassandra.exceptions.SyntaxException;
  import org.apache.cassandra.schema.KeyspaceMetadata;
  import org.apache.cassandra.service.ClientState;
 -import org.apache.cassandra.transport.Event;
 -import org.apache.cassandra.transport.Server;
 +import org.apache.cassandra.transport.*;
 +import org.apache.cassandra.transport.ProtocolVersion;
  import org.apache.cassandra.transport.messages.ResultMessage;
 -import org.apache.cassandra.utils.ByteBufferUtil;
  
  public class UFTest extends CQLTester
  {
@@@ -872,222 -863,134 +873,246 @@@
      }
  
      @Test
 -    public void testEmptyString() throws Throwable
 +    public void testArgumentGenerics() throws Throwable
      {
          createTable("CREATE TABLE %s (key int primary key, sval text, aval ascii, bval blob, empty_int int)");
 -        execute("INSERT INTO %s (key, sval, aval, bval, empty_int) VALUES (?, ?, ?, ?, blobAsInt(0x))", 1, "", "", ByteBuffer.allocate(0));
  
 -        String fNameSRC = createFunction(KEYSPACE_PER_TEST, "text",
 -                                         "CREATE OR REPLACE FUNCTION %s(val text) " +
 -                                         "CALLED ON NULL INPUT " +
 -                                         "RETURNS text " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return val;'");
 +        String typeName = createType("CREATE TYPE %s (txt text, i int)");
  
 -        String fNameSCC = createFunction(KEYSPACE_PER_TEST, "text",
 -                                         "CREATE OR REPLACE FUNCTION %s(val text) " +
 -                                         "CALLED ON NULL INPUT " +
 -                                         "RETURNS text " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return \"\";'");
 +        createFunction(KEYSPACE, "map<text,bigint>,list<text>",
 +                       "CREATE FUNCTION IF NOT EXISTS %s(state map<text,bigint>, styles list<text>)\n" +
 +                       "  RETURNS NULL ON NULL INPUT\n" +
 +                       "  RETURNS map<text,bigint>\n" +
 +                       "  LANGUAGE java\n" +
 +                       "  AS $$\n" +
 +                       "    for (String style : styles) {\n" +
 +                       "      if (state.containsKey(style)) {\n" +
 +                       "        state.put(style, state.get(style) + 1L);\n" +
 +                       "      } else {\n" +
 +                       "        state.put(style, 1L);\n" +
 +                       "      }\n" +
 +                       "    }\n" +
 +                       "    return state;\n" +
 +                       "  $$");
 +
 +        createFunction(KEYSPACE, "text",
 +                                  "CREATE OR REPLACE FUNCTION %s("                 +
 +                                  "  listText list<text>,"                         +
 +                                  "  setText set<text>,"                           +
 +                                  "  mapTextInt map<text, int>,"                   +
 +                                  "  mapListTextSetInt map<frozen<list<text>>, frozen<set<int>>>," +
 +                                  "  mapTextTuple map<text, frozen<tuple<int, text>>>," +
 +                                  "  mapTextType map<text, frozen<" + typeName + ">>" +
 +                                  ") "                                             +
 +                                  "CALLED ON NULL INPUT "                          +
 +                                  "RETURNS map<frozen<list<text>>, frozen<set<int>>> " +
 +                                  "LANGUAGE JAVA\n"                                +
 +                                  "AS $$" +
 +                                  "     for (String s : listtext) {};" +
 +                                  "     for (String s : settext) {};" +
 +                                  "     for (String s : maptextint.keySet()) {};" +
 +                                  "     for (Integer s : maptextint.values()) {};" +
 +                                  "     for (java.util.List<String> l : maplisttextsetint.keySet()) {};" +
 +                                  "     for (java.util.Set<Integer> s : maplisttextsetint.values()) {};" +
 +                                  "     for (com.datastax.driver.core.TupleValue t : maptexttuple.values()) {};" +
 +                                  "     for (com.datastax.driver.core.UDTValue u : maptexttype.values()) {};" +
 +                                  "     return maplisttextsetint;" +
 +                                  "$$");
 +    }
  
 -        String fNameSRN = createFunction(KEYSPACE_PER_TEST, "text",
 -                                         "CREATE OR REPLACE FUNCTION %s(val text) " +
 -                                         "RETURNS NULL ON NULL INPUT " +
 -                                         "RETURNS text " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return val;'");
 +    @Test
 +    public void testArgAndReturnTypes() throws Throwable
 +    {
  
 -        String fNameSCN = createFunction(KEYSPACE_PER_TEST, "text",
 -                                         "CREATE OR REPLACE FUNCTION %s(val text) " +
 -                                         "RETURNS NULL ON NULL INPUT " +
 -                                         "RETURNS text " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return \"\";'");
 -
 -        String fNameBRC = createFunction(KEYSPACE_PER_TEST, "blob",
 -                                         "CREATE OR REPLACE FUNCTION %s(val blob) " +
 -                                         "CALLED ON NULL INPUT " +
 -                                         "RETURNS blob " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return val;'");
 -
 -        String fNameBCC = createFunction(KEYSPACE_PER_TEST, "blob",
 -                                         "CREATE OR REPLACE FUNCTION %s(val blob) " +
 -                                         "CALLED ON NULL INPUT " +
 -                                         "RETURNS blob " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return ByteBuffer.allocate(0);'");
 -
 -        String fNameBRN = createFunction(KEYSPACE_PER_TEST, "blob",
 -                                         "CREATE OR REPLACE FUNCTION %s(val blob) " +
 -                                         "RETURNS NULL ON NULL INPUT " +
 -                                         "RETURNS blob " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return val;'");
 +        String type = KEYSPACE + '.' + createType("CREATE TYPE %s (txt text, i int)");
  
 -        String fNameBCN = createFunction(KEYSPACE_PER_TEST, "blob",
 -                                         "CREATE OR REPLACE FUNCTION %s(val blob) " +
 -                                         "RETURNS NULL ON NULL INPUT " +
 -                                         "RETURNS blob " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return ByteBuffer.allocate(0);'");
 +        createTable("CREATE TABLE %s (key int primary key, udt frozen<" + type + ">)");
 +        execute("INSERT INTO %s (key, udt) VALUES (1, {txt: 'foo', i: 42})");
  
 -        String fNameIRC = createFunction(KEYSPACE_PER_TEST, "int",
 -                                         "CREATE OR REPLACE FUNCTION %s(val int) " +
 -                                         "CALLED ON NULL INPUT " +
 -                                         "RETURNS int " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return val;'");
 +        // Java UDFs
  
 -        String fNameICC = createFunction(KEYSPACE_PER_TEST, "int",
 -                                         "CREATE OR REPLACE FUNCTION %s(val int) " +
 -                                         "CALLED ON NULL INPUT " +
 -                                         "RETURNS int " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return 0;'");
 +        String f = createFunction(KEYSPACE, "int",
 +                                  "CREATE OR REPLACE FUNCTION %s(val int) " +
 +                                  "RETURNS NULL ON NULL INPUT " +
 +                                  "RETURNS " + type + ' ' +
 +                                  "LANGUAGE JAVA\n" +
 +                                  "AS 'return udfContext.newReturnUDTValue();';");
 +
 +        assertRows(execute("SELECT " + f + "(key) FROM %s"),
 +                   row(userType("txt", null, "i", null)));
 +
 +        f = createFunction(KEYSPACE, "int",
 +                           "CREATE OR REPLACE FUNCTION %s(val " + type + ") " +
 +                           "RETURNS NULL ON NULL INPUT " +
 +                           "RETURNS " + type + ' ' +
 +                           "LANGUAGE JAVA\n" +
 +                           "AS $$" +
 +                           "   com.datastax.driver.core.UDTValue udt = udfContext.newArgUDTValue(\"val\");" +
 +                           "   udt.setString(\"txt\", \"baz\");" +
 +                           "   udt.setInt(\"i\", 88);" +
 +                           "   return udt;" +
 +                           "$$;");
 +
 +        assertRows(execute("SELECT " + f + "(udt) FROM %s"),
 +                   row(userType("txt", "baz", "i", 88)));
 +
 +        f = createFunction(KEYSPACE, "int",
 +                           "CREATE OR REPLACE FUNCTION %s(val " + type + ") " +
 +                           "RETURNS NULL ON NULL INPUT " +
 +                           "RETURNS tuple<text, int>" +
 +                           "LANGUAGE JAVA\n" +
 +                           "AS $$" +
 +                           "   com.datastax.driver.core.TupleValue tv = udfContext.newReturnTupleValue();" +
 +                           "   tv.setString(0, \"baz\");" +
 +                           "   tv.setInt(1, 88);" +
 +                           "   return tv;" +
 +                           "$$;");
 +
 +        assertRows(execute("SELECT " + f + "(udt) FROM %s"),
 +                   row(tuple("baz", 88)));
 +
 +        // JavaScript UDFs
 +
 +        f = createFunction(KEYSPACE, "int",
 +                           "CREATE OR REPLACE FUNCTION %s(val int) " +
 +                           "RETURNS NULL ON NULL INPUT " +
 +                           "RETURNS " + type + ' ' +
 +                           "LANGUAGE JAVASCRIPT\n" +
 +                           "AS $$" +
 +                           "   udt = udfContext.newReturnUDTValue();" +
 +                           "   udt;" +
 +                           "$$;");
 +
 +        assertRows(execute("SELECT " + f + "(key) FROM %s"),
 +                   row(userType("txt", null, "i", null)));
 +
 +        f = createFunction(KEYSPACE, "int",
 +                           "CREATE OR REPLACE FUNCTION %s(val " + type + ") " +
 +                           "RETURNS NULL ON NULL INPUT " +
 +                           "RETURNS " + type + ' ' +
 +                           "LANGUAGE JAVASCRIPT\n" +
 +                           "AS $$" +
 +                           "   udt = udfContext.newArgUDTValue(0);" +
 +                           "   udt.setString(\"txt\", \"baz\");" +
 +                           "   udt.setInt(\"i\", 88);" +
 +                           "   udt;" +
 +                           "$$;");
 +
 +        assertRows(execute("SELECT " + f + "(udt) FROM %s"),
 +                   row(userType("txt", "baz", "i", 88)));
 +
 +        f = createFunction(KEYSPACE, "int",
 +                           "CREATE OR REPLACE FUNCTION %s(val " + type + ") " +
 +                           "RETURNS NULL ON NULL INPUT " +
 +                           "RETURNS tuple<text, int>" +
 +                           "LANGUAGE JAVASCRIPT\n" +
 +                           "AS $$" +
 +                           "   tv = udfContext.newReturnTupleValue();" +
 +                           "   tv.setString(0, \"baz\");" +
 +                           "   tv.setInt(1, 88);" +
 +                           "   tv;" +
 +                           "$$;");
 +
 +        assertRows(execute("SELECT " + f + "(udt) FROM %s"),
 +                   row(tuple("baz", 88)));
 +
 +        createFunction(KEYSPACE, "map",
 +                       "CREATE FUNCTION %s(my_map map<text, text>)\n" +
 +                       "         CALLED ON NULL INPUT\n" +
 +                       "         RETURNS text\n" +
 +                       "         LANGUAGE java\n" +
 +                       "         AS $$\n" +
 +                       "             String buffer = \"\";\n" +
 +                       "             for(java.util.Map.Entry<String, String> entry: my_map.entrySet()) {\n" +
 +                       "                 buffer = buffer + entry.getKey() + \": \" + entry.getValue() + \", \";\n" +
 +                       "             }\n" +
 +                       "             return buffer;\n" +
 +                       "         $$;\n");
 +    }
  
 -        String fNameIRN = createFunction(KEYSPACE_PER_TEST, "int",
 -                                         "CREATE OR REPLACE FUNCTION %s(val int) " +
 -                                         "RETURNS NULL ON NULL INPUT " +
 -                                         "RETURNS int " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return val;'");
 +    @Test
 +    public void testImportJavaUtil() throws Throwable
 +    {
 +        createFunction(KEYSPACE, "list<text>",
 +                "CREATE OR REPLACE FUNCTION %s(listText list<text>) "                                             +
 +                        "CALLED ON NULL INPUT "                          +
 +                        "RETURNS set<text> " +
 +                        "LANGUAGE JAVA\n"                                +
 +                        "AS $$\n" +
 +                        "     Set<String> set = new HashSet<String>(); " +
 +                        "     for (String s : listtext) {" +
 +                        "            set.add(s);" +
 +                        "     }" +
 +                        "     return set;" +
 +                        "$$");
  
 -        String fNameICN = createFunction(KEYSPACE_PER_TEST, "int",
 -                                         "CREATE OR REPLACE FUNCTION %s(val int) " +
 -                                         "RETURNS NULL ON NULL INPUT " +
 -                                         "RETURNS int " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return 0;'");
 -
 -        assertRows(execute("SELECT " + fNameSRC + "(sval) FROM %s"), row(""));
 -        assertRows(execute("SELECT " + fNameSRN + "(sval) FROM %s"), row(""));
 -        assertRows(execute("SELECT " + fNameSCC + "(sval) FROM %s"), row(""));
 -        assertRows(execute("SELECT " + fNameSCN + "(sval) FROM %s"), row(""));
 -        assertRows(execute("SELECT " + fNameSRC + "(aval) FROM %s"), row(""));
 -        assertRows(execute("SELECT " + fNameSRN + "(aval) FROM %s"), row(""));
 -        assertRows(execute("SELECT " + fNameSCC + "(aval) FROM %s"), row(""));
 -        assertRows(execute("SELECT " + fNameSCN + "(aval) FROM %s"), row(""));
 -        assertRows(execute("SELECT " + fNameBRC + "(bval) FROM %s"), row(ByteBufferUtil.EMPTY_BYTE_BUFFER));
 -        assertRows(execute("SELECT " + fNameBRN + "(bval) FROM %s"), row(ByteBufferUtil.EMPTY_BYTE_BUFFER));
 -        assertRows(execute("SELECT " + fNameBCC + "(bval) FROM %s"), row(ByteBufferUtil.EMPTY_BYTE_BUFFER));
 -        assertRows(execute("SELECT " + fNameBCN + "(bval) FROM %s"), row(ByteBufferUtil.EMPTY_BYTE_BUFFER));
 -        assertRows(execute("SELECT " + fNameIRC + "(empty_int) FROM %s"), row(new Object[]{ null }));
 -        assertRows(execute("SELECT " + fNameIRN + "(empty_int) FROM %s"), row(new Object[]{ null }));
 -        assertRows(execute("SELECT " + fNameICC + "(empty_int) FROM %s"), row(0));
 -        assertRows(execute("SELECT " + fNameICN + "(empty_int) FROM %s"), row(new Object[]{ null }));
 +    }
 +
 +    @Test
 +    public void testAnyUserTupleType() throws Throwable
 +    {
 +        createTable("CREATE TABLE %s (key int primary key, sval text)");
 +        execute("INSERT INTO %s (key, sval) VALUES (1, 'foo')");
 +
 +        String udt = createType("CREATE TYPE %s (a int, b text, c bigint)");
 +
 +        String fUdt = createFunction(KEYSPACE, "text",
 +                                     "CREATE OR REPLACE FUNCTION %s(arg text) " +
 +                                     "CALLED ON NULL INPUT " +
 +                                     "RETURNS " + udt + " " +
 +                                     "LANGUAGE JAVA\n" +
 +                                     "AS $$\n" +
 +                                     "    UDTValue udt = udfContext.newUDTValue(\"" + udt + "\");" +
 +                                     "    udt.setInt(\"a\", 42);" +
 +                                     "    udt.setString(\"b\", \"42\");" +
 +                                     "    udt.setLong(\"c\", 4242);" +
 +                                     "    return udt;" +
 +                                     "$$");
 +
 +        assertRows(execute("SELECT " + fUdt + "(sval) FROM %s"),
 +                   row(userType("a", 42, "b", "42", "c", 4242L)));
 +
 +        String fTup = createFunction(KEYSPACE, "text",
 +                                     "CREATE OR REPLACE FUNCTION %s(arg text) " +
 +                                     "CALLED ON NULL INPUT " +
 +                                     "RETURNS tuple<int, " + udt + "> " +
 +                                     "LANGUAGE JAVA\n" +
 +                                     "AS $$\n" +
 +                                     "    UDTValue udt = udfContext.newUDTValue(\"" + udt + "\");" +
 +                                     "    udt.setInt(\"a\", 42);" +
 +                                     "    udt.setString(\"b\", \"42\");" +
 +                                     "    udt.setLong(\"c\", 4242);" +
 +                                     "    TupleValue tup = udfContext.newTupleValue(\"tuple<int," + udt + ">\");" +
 +                                     "    tup.setInt(0, 88);" +
 +                                     "    tup.setUDTValue(1, udt);" +
 +                                     "    return tup;" +
 +                                     "$$");
 +
 +        assertRows(execute("SELECT " + fTup + "(sval) FROM %s"),
 +                   row(tuple(88, userType("a", 42, "b", "42", "c", 4242L))));
      }
+ 
+     @Test(expected = SyntaxException.class)
+     public void testEmptyFunctionName() throws Throwable
+     {
+         execute("CREATE FUNCTION IF NOT EXISTS " + KEYSPACE + ".\"\" (arg int)\n" +
+                 "  RETURNS NULL ON NULL INPUT\n" +
+                 "  RETURNS int\n" +
+                 "  LANGUAGE java\n" +
+                 "  AS $$\n" +
+                 "    return a;\n" +
+                 "  $$");
+     }
+ 
+     @Test(expected = SyntaxException.class)
+     public void testEmptyArgName() throws Throwable
+     {
+         execute("CREATE FUNCTION IF NOT EXISTS " + KEYSPACE + ".myfn (\"\" int)\n" +
+                 "  RETURNS NULL ON NULL INPUT\n" +
+                 "  RETURNS int\n" +
+                 "  LANGUAGE java\n" +
+                 "  AS $$\n" +
+                 "    return a;\n" +
+                 "  $$");
+     }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
index 9ea5572,68c0b8c..646484c
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
@@@ -872,12 -708,31 +873,31 @@@ public class UserTypesTest extends CQLT
  
          execute("ALTER TYPE " + columnType + " ADD b int");
          execute("UPDATE %s SET s = s + ?, v = ? WHERE pk = ? AND c = ?",
 -                set(userType(1, 1), userType(1, 2), userType(2, 1)), 2, 1, 1);
 +                set(userType("a", 1, "b", 1), userType("a", 1, "b", 2), userType("a", 2, "b", 1)), 2, 1, 1);
  
          assertRows(execute("SELECT * FROM %s WHERE pk = ? AND c = ?", 1, 1),
 -                       row(1, 1,set(userType(1), userType(1, 1), userType(1, 2), userType(2), userType(2, 1)), 2));
 +                       row(1, 1,set(userType("a", 1), userType("a", 1, "b", 1), userType("a", 1, "b", 2), userType("a", 2), userType("a", 2, "b", 1)), 2));
      }
  
+     @Test(expected = SyntaxException.class)
+     public void emptyTypeNameTest() throws Throwable
+     {
+         execute("CREATE TYPE \"\" (a int, b int)");
+     }
+ 
+     @Test(expected = SyntaxException.class)
+     public void emptyFieldNameTest() throws Throwable
+     {
+         execute("CREATE TYPE mytype (\"\" int, b int)");
+     }
+ 
+     @Test(expected = SyntaxException.class)
+     public void renameColumnToEmpty() throws Throwable
+     {
+         String typeName = createType("CREATE TYPE %s (a int, b int)");
+         execute(String.format("ALTER TYPE %s.%s RENAME b TO \"\"", keyspace(), typeName));
+     }
+ 
      private String typeWithKs(String type1)
      {
          return keyspace() + '.' + type1;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
----------------------------------------------------------------------


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org