You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by "adelapena (via GitHub)" <gi...@apache.org> on 2023/02/01 12:19:25 UTC

[GitHub] [cassandra] adelapena commented on a diff in pull request #2118: sstabledump errors when dumping data from index for CASSANDRA-17698

adelapena commented on code in PR #2118:
URL: https://github.com/apache/cassandra/pull/2118#discussion_r1093074349


##########
src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java:
##########
@@ -266,6 +266,7 @@ public Term fromJSONObject(Object parsed)
     @Override
     public String toJSONString(ByteBuffer buffer, ProtocolVersion protocolVersion)
     {
+        //TODO suport toJSONString for AbstractCompositeType

Review Comment:
   ```suggestion
           // TODO: suport toJSONString (CASSANDRA-18177)
   ```



##########
src/java/org/apache/cassandra/db/marshal/PartitionerDefinedOrder.java:
##########
@@ -88,7 +93,8 @@ public Term fromJSONObject(Object parsed)
     @Override
     public String toJSONString(ByteBuffer buffer, ProtocolVersion protocolVersion)
     {
-        throw new UnsupportedOperationException();
+        assert baseType != null && !baseType.equals(this) : "PartitionerDefinedOrder's toJSONString method need a baseType but now is null or with a not euqal type.";

Review Comment:
   ```suggestion
           assert baseType != null && !baseType.equals(this)
           : "PartitionerDefinedOrder's toJSONString method needs a baseType but now it is null or with a not equals type.";
   ```



##########
src/java/org/apache/cassandra/db/marshal/PartitionerDefinedOrder.java:
##########
@@ -138,6 +144,38 @@ public TypeSerializer<ByteBuffer> getSerializer()
     @Override
     public String toString()
     {
+        if(baseType != null)
+        {
+            return String.format("%s(%s:%s)", getClass().getName(),  partitioner.getClass().getName(), baseType.toString()); 

Review Comment:
   ```suggestion
               return String.format("%s(%s:%s)", getClass().getName(),  partitioner.getClass().getName(), baseType); 
   ```



##########
src/java/org/apache/cassandra/db/marshal/PartitionerDefinedOrder.java:
##########
@@ -138,6 +144,38 @@ public TypeSerializer<ByteBuffer> getSerializer()
     @Override
     public String toString()
     {
+        if(baseType != null)

Review Comment:
   ```suggestion
           if (baseType != null)
   ```



##########
src/java/org/apache/cassandra/db/marshal/TypeParser.java:
##########
@@ -131,6 +138,59 @@ public AbstractType<?> parse() throws SyntaxException, ConfigurationException
             return getAbstractType(name);
     }
 
+    /**
+     * parse PartitionOrdering from old version of PartitionOrdering' string format 
+     * */
+    private static  AbstractType<?> defaultParsePartitionOrdering(TypeParser typeParser)

Review Comment:
   ```suggestion
       private static AbstractType<?> defaultParsePartitionOrdering(TypeParser typeParser)
   ```



##########
src/java/org/apache/cassandra/index/internal/CassandraIndex.java:
##########
@@ -735,12 +736,18 @@ public static TableMetadata indexCfsMetadata(TableMetadata baseCfsMetadata, Inde
         ColumnMetadata indexedColumn = target.left;
         AbstractType<?> indexedValueType = utils.getIndexedValueType(indexedColumn);
 
+        AbstractType<?> indexedTablePartitionKeyType =  baseCfsMetadata.partitioner.partitionOrdering();

Review Comment:
   ```suggestion
           AbstractType<?> indexedTablePartitionKeyType = baseCfsMetadata.partitioner.partitionOrdering();
   ```



##########
src/java/org/apache/cassandra/db/marshal/TypeParser.java:
##########
@@ -167,14 +227,14 @@ else if (str.charAt(idx) != ',' && str.charAt(idx) != ')')
         }
         throw new SyntaxException(String.format("Syntax error parsing '%s' at char %d: unexpected end of string", str, idx));
     }
-
+    

Review Comment:
   ```suggestion
   
   ```



##########
src/java/org/apache/cassandra/db/marshal/PartitionerDefinedOrder.java:
##########
@@ -18,45 +18,50 @@
 package org.apache.cassandra.db.marshal;
 
 import java.nio.ByteBuffer;
-import java.util.Iterator;
 
-import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.PartitionPosition;
-import org.apache.cassandra.serializers.TypeSerializer;
-import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.bytecomparable.ByteComparable;
 import org.apache.cassandra.utils.bytecomparable.ByteComparable.Version;
 import org.apache.cassandra.utils.bytecomparable.ByteSource;
-import org.apache.cassandra.utils.FBUtilities;
+
 
 /** for sorting columns representing row keys in the row ordering as determined by a partitioner.
  * Not intended for user-defined CFs, and will in fact error out if used with such. */
 public class PartitionerDefinedOrder extends AbstractType<ByteBuffer>
 {
     private final IPartitioner partitioner;
-
+    private final AbstractType<?> baseType; 
+    
     public PartitionerDefinedOrder(IPartitioner partitioner)
     {
         super(ComparisonType.CUSTOM);
         this.partitioner = partitioner;
+        this.baseType = null;
+    }
+
+    public PartitionerDefinedOrder(IPartitioner partitioner, AbstractType<?> baseType)
+    {
+        super(ComparisonType.CUSTOM);
+        this.partitioner = partitioner;
+        this.baseType = baseType;
     }
 
     public static AbstractType<?> getInstance(TypeParser parser)
     {
-        IPartitioner partitioner = DatabaseDescriptor.getPartitioner();
-        Iterator<String> argIterator = parser.getKeyValueParameters().keySet().iterator();
-        if (argIterator.hasNext())
-        {
-            partitioner = FBUtilities.newPartitioner(argIterator.next());
-            assert !argIterator.hasNext();
-        }
-        return partitioner.partitionOrdering();
+        return parser.getPartitionerDefinedOrder();
     }
 
+    public AbstractType<?> withBaseType(AbstractType<?> baseType)
+    {
+        return new PartitionerDefinedOrder(this.partitioner, baseType);

Review Comment:
   ```suggestion
           return new PartitionerDefinedOrder(partitioner, baseType);
   ```



##########
src/java/org/apache/cassandra/index/internal/CassandraIndex.java:
##########
@@ -735,12 +736,18 @@ public static TableMetadata indexCfsMetadata(TableMetadata baseCfsMetadata, Inde
         ColumnMetadata indexedColumn = target.left;
         AbstractType<?> indexedValueType = utils.getIndexedValueType(indexedColumn);
 
+        AbstractType<?> indexedTablePartitionKeyType =  baseCfsMetadata.partitioner.partitionOrdering();
+        if(indexedTablePartitionKeyType instanceof PartitionerDefinedOrder)
+        {
+            PartitionerDefinedOrder tmp = (PartitionerDefinedOrder)indexedTablePartitionKeyType;

Review Comment:
   ```suggestion
               PartitionerDefinedOrder tmp = (PartitionerDefinedOrder) indexedTablePartitionKeyType;
   ```



##########
src/java/org/apache/cassandra/index/internal/composites/RegularColumnIndex.java:
##########
@@ -76,7 +76,7 @@ public <T> CBuilder buildIndexClusteringPrefix(ByteBuffer partitionKey,
         // base table partition should be returned for any mathching index entry.
         return builder;
     }
-
+    

Review Comment:
   ```suggestion
   
   ```



##########
src/java/org/apache/cassandra/tools/Util.java:
##########
@@ -335,6 +338,15 @@ public static TableMetadata metadataFromSSTable(Descriptor desc) throws IOExcept
         {
             builder.addClusteringColumn("clustering" + (i > 0 ? i : ""), header.getClusteringTypes().get(i));
         }
+        if (SecondaryIndexManager.isIndexColumnFamily(desc.cfname))
+        {
+            String index = SecondaryIndexManager.getIndexName(desc.cfname);
+            // Just set the Kind of index to CUSTOM, which is an irrelevant parameter that does't make any effect on the result

Review Comment:
   ```suggestion
               // Just set the Kind of index to CUSTOM, which is an irrelevant parameter that doesn't make any effect on the result
   ```



##########
test/unit/org/apache/cassandra/SchemaLoader.java:
##########
@@ -390,7 +390,7 @@ public static TableMetadata.Builder staticCFMD(String ksName, String cfName)
                                  .addStaticColumn("val", AsciiType.instance)
                                  .addRegularColumn("val2", AsciiType.instance);
     }
-
+    

Review Comment:
   ```suggestion
   
   ```



##########
test/unit/org/apache/cassandra/db/marshal/TypeParserTest.java:
##########
@@ -105,4 +109,119 @@ public void testParsePartitionerOrder() throws ConfigurationException, SyntaxExc
         }
         assertSame(DatabaseDescriptor.getPartitioner().partitionOrdering(), TypeParser.parse("PartitionerDefinedOrder"));
     }
+    
+    @Test
+    public void testParsePartitionerOrderWithBaseType()
+    {
+        // default partitioner
+        assertSame(DatabaseDescriptor.getPartitioner().partitionOrdering(), TypeParser.parse("PartitionerDefinedOrder"));
+
+        // PartitionerDefinedOrder's base type is not composite type
+        differentBaseTypeValidation(Int32Type.instance);
+        // PartitionerDefinedOrder's base type is  composite type
+        differentBaseTypeValidation(CompositeType.getInstance(Int32Type.instance, UTF8Type.instance));
+        // PartitionerDefinedOrder's base type is  tuple type
+        differentBaseTypeValidation(new TupleType(Lists.newArrayList(Int32Type.instance, UTF8Type.instance)));
+        // PartitionerDefinedOrder's base type is  ReversedType
+        differentBaseTypeValidation(ReversedType.getInstance(Int32Type.instance));
+        // PartitionerDefinedOrder's base type is  CollectionType

Review Comment:
   ```suggestion
           // PartitionerDefinedOrder's base type is CollectionType
   ```



##########
test/unit/org/apache/cassandra/db/marshal/TypeParserTest.java:
##########
@@ -105,4 +109,119 @@ public void testParsePartitionerOrder() throws ConfigurationException, SyntaxExc
         }
         assertSame(DatabaseDescriptor.getPartitioner().partitionOrdering(), TypeParser.parse("PartitionerDefinedOrder"));
     }
+    
+    @Test
+    public void testParsePartitionerOrderWithBaseType()
+    {
+        // default partitioner
+        assertSame(DatabaseDescriptor.getPartitioner().partitionOrdering(), TypeParser.parse("PartitionerDefinedOrder"));
+
+        // PartitionerDefinedOrder's base type is not composite type
+        differentBaseTypeValidation(Int32Type.instance);
+        // PartitionerDefinedOrder's base type is  composite type
+        differentBaseTypeValidation(CompositeType.getInstance(Int32Type.instance, UTF8Type.instance));
+        // PartitionerDefinedOrder's base type is  tuple type

Review Comment:
   ```suggestion
           // PartitionerDefinedOrder's base type is tuple type
   ```



##########
test/unit/org/apache/cassandra/SchemaLoader.java:
##########
@@ -252,7 +252,7 @@ public static void schemaDefinition(String testName) throws ConfigurationExcepti
         if (Boolean.parseBoolean(System.getProperty("cassandra.test.compression", "false")))
             useCompression(schema, compressionParams(CompressionParams.DEFAULT_CHUNK_LENGTH));
     }
-
+    

Review Comment:
   ```suggestion
   
   ```



##########
test/unit/org/apache/cassandra/db/marshal/TypeParserTest.java:
##########
@@ -105,4 +109,119 @@ public void testParsePartitionerOrder() throws ConfigurationException, SyntaxExc
         }
         assertSame(DatabaseDescriptor.getPartitioner().partitionOrdering(), TypeParser.parse("PartitionerDefinedOrder"));
     }
+    
+    @Test
+    public void testParsePartitionerOrderWithBaseType()
+    {
+        // default partitioner
+        assertSame(DatabaseDescriptor.getPartitioner().partitionOrdering(), TypeParser.parse("PartitionerDefinedOrder"));
+
+        // PartitionerDefinedOrder's base type is not composite type
+        differentBaseTypeValidation(Int32Type.instance);
+        // PartitionerDefinedOrder's base type is  composite type
+        differentBaseTypeValidation(CompositeType.getInstance(Int32Type.instance, UTF8Type.instance));
+        // PartitionerDefinedOrder's base type is  tuple type
+        differentBaseTypeValidation(new TupleType(Lists.newArrayList(Int32Type.instance, UTF8Type.instance)));
+        // PartitionerDefinedOrder's base type is  ReversedType
+        differentBaseTypeValidation(ReversedType.getInstance(Int32Type.instance));
+        // PartitionerDefinedOrder's base type is  CollectionType
+        differentBaseTypeValidation(MapType.getInstance(Int32Type.instance, UTF8Type.instance, false));
+    }
+
+    @Test
+    public void testParsePartitionerOrderMistMatch()
+    {
+        for (IPartitioner partitioner: new IPartitioner[] { Murmur3Partitioner.instance,
+                                                            ByteOrderedPartitioner.instance,
+                                                            RandomPartitioner.instance,
+                                                            OrderPreservingPartitioner.instance })
+        {
+            AbstractType<?> type = partitioner.partitionOrdering();
+            if (type instanceof PartitionerDefinedOrder)
+            {
+                PartitionerDefinedOrder tmp = (PartitionerDefinedOrder) type;
+                type = tmp.withBaseType(Int32Type.instance);
+                boolean result = partitioner.partitionOrdering().equals(TypeParser.parse(type.toString()));
+                assertFalse(result);
+            }
+            else
+            {
+                // ByteOrderedPartitioner.instance and OrderPreservingPartitioner.instance's partitionOrdering will not be PartitionerDefinedOrder
+                boolean result = partitioner.partitionOrdering().equals(TypeParser.parse(type.toString()));
+                assertTrue(result);
+            }
+        }
+
+        assertEquals(DatabaseDescriptor.getPartitioner().partitionOrdering(), TypeParser.parse("PartitionerDefinedOrder"));
+    }
+
+    @Test
+    public void testParsePartitionerOrderWithErrorFormat()
+    {
+        for (IPartitioner partitioner: new IPartitioner[] { Murmur3Partitioner.instance,
+                                                            ByteOrderedPartitioner.instance,
+                                                            RandomPartitioner.instance,
+                                                            OrderPreservingPartitioner.instance })
+        {
+            AbstractType<?> type = partitioner.partitionOrdering();
+            if (type instanceof PartitionerDefinedOrder)
+            {
+                // only Murmur3Partitioner and RandomPartitioner's partitionOrdering() are instanceof PartitionerDefinedOrder
+                String msgPartitioner = partitioner instanceof Murmur3Partitioner ? "Murmur3Partitioner" : "RandomPartitioner";
+                // error format PartitionerDefinedOrder(org.apache.cassandra.dht.Murmur3Partitioner,
+                String tmpStr1 =  type.toString().replace(')', ',');
+                try
+                {
+                    TypeParser.parse(tmpStr1);
+                    fail();
+                }
+                catch (Throwable t)
+                {
+                    assertTrue(t.getCause().getMessage().contains("Syntax error parsing 'org.apache.cassandra.db.marshal.PartitionerDefinedOrder(org.apache.cassandra.dht." + msgPartitioner + ",: for msg unexpected character ','"));
+                }
+
+                // error format PartitionerDefinedOrder(org.apache.cassandra.dht.Murmur3Partitioner>
+                String tmpStr2 =  type.toString().replace(')', '>');
+                try
+                {
+                    TypeParser.parse(tmpStr2);
+                    fail();
+                }
+                catch (Throwable t)
+                {
+                    assertTrue(t.getCause().getMessage().contains("Syntax error parsing 'org.apache.cassandra.db.marshal.PartitionerDefinedOrder(org.apache.cassandra.dht." + msgPartitioner + ">: for msg unexpected character '>'"));
+                }
+
+                // error format PartitionerDefinedOrder(org.apache.cassandra.dht.Murmur3Partitioner>
+                String tmpStr3 =  type.toString().replace(')', ':');
+                try
+                {
+                    TypeParser.parse(tmpStr3);
+                    fail();
+                }
+                catch (Throwable t)
+                {
+                    assertTrue(t.getCause().getMessage().contains("Unable to find abstract-type class 'org.apache.cassandra.db.marshal.'"));
+                }
+            }
+        }
+        assertSame(DatabaseDescriptor.getPartitioner().partitionOrdering(), TypeParser.parse("PartitionerDefinedOrder"));
+    }
+
+    private void differentBaseTypeValidation(AbstractType baseType)

Review Comment:
   ```suggestion
       private void differentBaseTypeValidation(AbstractType<?> baseType)
   ```



##########
src/java/org/apache/cassandra/db/marshal/PartitionerDefinedOrder.java:
##########
@@ -138,6 +144,38 @@ public TypeSerializer<ByteBuffer> getSerializer()
     @Override
     public String toString()
     {
+        if(baseType != null)
+        {
+            return String.format("%s(%s:%s)", getClass().getName(),  partitioner.getClass().getName(), baseType.toString()); 
+        }
         return String.format("%s(%s)", getClass().getName(), partitioner.getClass().getName());
     }
+    
+    public AbstractType<?>  getBaseType() 
+    { 
+        return baseType;
+    }
+
+    @Override
+    public boolean equals(Object obj)
+    {
+        if (this == obj)
+        {
+            return true;
+        }
+        if (obj instanceof PartitionerDefinedOrder)
+        {
+            PartitionerDefinedOrder other = (PartitionerDefinedOrder) obj;
+            if (baseType == null && other.baseType == null)
+            {
+                return this.partitioner.equals(other.partitioner);
+            }
+            else if (baseType != null && other.baseType != null)
+            {
+                return this.baseType.equals(other.baseType) && this.partitioner.equals(other.partitioner);
+            }
+            return false;

Review Comment:
   ```suggestion
               return partitioner.equals(other.partitioner) && Objects.equals(baseType, other.baseType);
   ```



##########
src/java/org/apache/cassandra/db/marshal/TypeParser.java:
##########
@@ -131,6 +138,59 @@ public AbstractType<?> parse() throws SyntaxException, ConfigurationException
             return getAbstractType(name);
     }
 
+    /**
+     * parse PartitionOrdering from old version of PartitionOrdering' string format 
+     * */
+    private static  AbstractType<?> defaultParsePartitionOrdering(TypeParser typeParser)
+    {
+        IPartitioner partitioner = DatabaseDescriptor.getPartitioner();
+        Iterator<String> argIterator = typeParser.getKeyValueParameters().keySet().iterator();
+        if (argIterator.hasNext())
+        {
+            partitioner = FBUtilities.newPartitioner(argIterator.next());
+            assert !argIterator.hasNext();
+        }
+        return partitioner.partitionOrdering();
+    }
+
+    /**
+     * parse and return the real PartitionerDefinedOrder from the string variable str
+     * the str format can be like PartitionerDefinedOrder(<partitioner>) or
+     * PartitionerDefinedOrder(<partitioner>:<baseType>)
+     * */

Review Comment:
   ```suggestion
        * Parse and return the real {@link PartitionerDefinedOrder} from the string variable {@link #str}.
        * The {@link #str} format can be like {@code PartitionerDefinedOrder(<partitioner>)} or
        * {@code PartitionerDefinedOrder(<partitioner>:<baseType>)}.
        */
   ```



##########
src/java/org/apache/cassandra/db/marshal/PartitionerDefinedOrder.java:
##########
@@ -138,6 +144,38 @@ public TypeSerializer<ByteBuffer> getSerializer()
     @Override
     public String toString()
     {
+        if(baseType != null)
+        {
+            return String.format("%s(%s:%s)", getClass().getName(),  partitioner.getClass().getName(), baseType.toString()); 
+        }
         return String.format("%s(%s)", getClass().getName(), partitioner.getClass().getName());
     }
+    
+    public AbstractType<?>  getBaseType() 

Review Comment:
   ```suggestion
       @Nullable
       public AbstractType<?>  getBaseType()
   ```



##########
src/java/org/apache/cassandra/index/internal/CassandraIndex.java:
##########
@@ -735,12 +736,18 @@ public static TableMetadata indexCfsMetadata(TableMetadata baseCfsMetadata, Inde
         ColumnMetadata indexedColumn = target.left;
         AbstractType<?> indexedValueType = utils.getIndexedValueType(indexedColumn);
 
+        AbstractType<?> indexedTablePartitionKeyType =  baseCfsMetadata.partitioner.partitionOrdering();
+        if(indexedTablePartitionKeyType instanceof PartitionerDefinedOrder)

Review Comment:
   ```suggestion
           if (indexedTablePartitionKeyType instanceof PartitionerDefinedOrder)
   ```



##########
src/java/org/apache/cassandra/index/internal/CassandraIndexFunctions.java:
##########
@@ -183,6 +212,22 @@ public TableMetadata.Builder addIndexClusteringColumns(TableMetadata.Builder bui
             builder.addClusteringColumn("cell_path", ((CollectionType)columnDef.type).nameComparator());
             return builder;
         }
+
+        @Override
+        public AbstractType<?> getIndexedPartitionKeyType(ColumnMetadata indexedColumn)
+        {
+            assert indexedColumn.type.isCollection() ;
+            switch (((CollectionType<?>)indexedColumn.type).kind)
+            {
+                case LIST:
+                    return ((ListType<?>)indexedColumn.type).getElementsType();
+                case SET:
+                    return ((SetType<?>)indexedColumn.type).getElementsType();
+                case MAP:
+                    return ((MapType<?, ?>)indexedColumn.type).getValuesType();

Review Comment:
   ```suggestion
               switch (((CollectionType<?>) indexedColumn.type).kind)
               {
                   case LIST:
                       return ((ListType<?>) indexedColumn.type).getElementsType();
                   case SET:
                       return ((SetType<?>) indexedColumn.type).getElementsType();
                   case MAP:
                       return ((MapType<?, ?>) indexedColumn.type).getValuesType();
   ```



##########
src/java/org/apache/cassandra/tools/SSTableExport.java:
##########
@@ -71,8 +72,15 @@
 
     static
     {
-        DatabaseDescriptor.clientInitialization();
-
+        if (Boolean.getBoolean(Util.ALLOW_TOOL_REINIT_FOR_TEST))
+        {
+            DatabaseDescriptor.clientInitialization(false);//Necessary for testing
+        }
+        else 
+        {
+            DatabaseDescriptor.clientInitialization();
+        }

Review Comment:
   ```suggestion
           DatabaseDescriptor.clientInitialization(!Boolean.getBoolean(Util.ALLOW_TOOL_REINIT_FOR_TEST));
   ```



##########
src/java/org/apache/cassandra/db/marshal/TypeParser.java:
##########
@@ -131,6 +138,59 @@ public AbstractType<?> parse() throws SyntaxException, ConfigurationException
             return getAbstractType(name);
     }
 
+    /**
+     * parse PartitionOrdering from old version of PartitionOrdering' string format 
+     * */

Review Comment:
   ```suggestion
        * Parse PartitionOrdering from old version of PartitionOrdering' string format 
        */
   ```



##########
src/java/org/apache/cassandra/db/marshal/TypeParser.java:
##########
@@ -167,14 +227,14 @@ else if (str.charAt(idx) != ',' && str.charAt(idx) != ')')
         }
         throw new SyntaxException(String.format("Syntax error parsing '%s' at char %d: unexpected end of string", str, idx));
     }
-
+    
     public List<AbstractType<?>> getTypeParameters() throws SyntaxException, ConfigurationException
     {
         List<AbstractType<?>> list = new ArrayList<>();
 
         if (isEOS())
             return list;
-
+        

Review Comment:
   ```suggestion
   
   ```



##########
test/unit/org/apache/cassandra/db/marshal/TypeParserTest.java:
##########
@@ -105,4 +109,119 @@ public void testParsePartitionerOrder() throws ConfigurationException, SyntaxExc
         }
         assertSame(DatabaseDescriptor.getPartitioner().partitionOrdering(), TypeParser.parse("PartitionerDefinedOrder"));
     }
+    
+    @Test
+    public void testParsePartitionerOrderWithBaseType()
+    {
+        // default partitioner
+        assertSame(DatabaseDescriptor.getPartitioner().partitionOrdering(), TypeParser.parse("PartitionerDefinedOrder"));
+
+        // PartitionerDefinedOrder's base type is not composite type
+        differentBaseTypeValidation(Int32Type.instance);
+        // PartitionerDefinedOrder's base type is  composite type

Review Comment:
   ```suggestion
           // PartitionerDefinedOrder's base type is composite type
   ```



##########
test/unit/org/apache/cassandra/db/marshal/TypeParserTest.java:
##########
@@ -105,4 +109,119 @@ public void testParsePartitionerOrder() throws ConfigurationException, SyntaxExc
         }
         assertSame(DatabaseDescriptor.getPartitioner().partitionOrdering(), TypeParser.parse("PartitionerDefinedOrder"));
     }
+    
+    @Test
+    public void testParsePartitionerOrderWithBaseType()
+    {
+        // default partitioner
+        assertSame(DatabaseDescriptor.getPartitioner().partitionOrdering(), TypeParser.parse("PartitionerDefinedOrder"));
+
+        // PartitionerDefinedOrder's base type is not composite type
+        differentBaseTypeValidation(Int32Type.instance);
+        // PartitionerDefinedOrder's base type is  composite type
+        differentBaseTypeValidation(CompositeType.getInstance(Int32Type.instance, UTF8Type.instance));
+        // PartitionerDefinedOrder's base type is  tuple type
+        differentBaseTypeValidation(new TupleType(Lists.newArrayList(Int32Type.instance, UTF8Type.instance)));
+        // PartitionerDefinedOrder's base type is  ReversedType
+        differentBaseTypeValidation(ReversedType.getInstance(Int32Type.instance));
+        // PartitionerDefinedOrder's base type is  CollectionType
+        differentBaseTypeValidation(MapType.getInstance(Int32Type.instance, UTF8Type.instance, false));
+    }
+
+    @Test
+    public void testParsePartitionerOrderMistMatch()
+    {
+        for (IPartitioner partitioner: new IPartitioner[] { Murmur3Partitioner.instance,
+                                                            ByteOrderedPartitioner.instance,
+                                                            RandomPartitioner.instance,
+                                                            OrderPreservingPartitioner.instance })

Review Comment:
   This block is repeated four times in this class, we might want to encapsulate it into a method:
   ```java
   private static void assertForEachPartitioner(Consumer<IPartitioner> consumer)
   {
       for (IPartitioner partitioner : new IPartitioner[] { Murmur3Partitioner.instance,
                                                            ByteOrderedPartitioner.instance,
                                                            RandomPartitioner.instance,
                                                            OrderPreservingPartitioner.instance })
       {
           consumer.accept(partitioner);
       }
   }
   ```



##########
test/unit/org/apache/cassandra/db/marshal/TypeParserTest.java:
##########
@@ -105,4 +109,119 @@ public void testParsePartitionerOrder() throws ConfigurationException, SyntaxExc
         }
         assertSame(DatabaseDescriptor.getPartitioner().partitionOrdering(), TypeParser.parse("PartitionerDefinedOrder"));
     }
+    
+    @Test
+    public void testParsePartitionerOrderWithBaseType()
+    {
+        // default partitioner
+        assertSame(DatabaseDescriptor.getPartitioner().partitionOrdering(), TypeParser.parse("PartitionerDefinedOrder"));
+
+        // PartitionerDefinedOrder's base type is not composite type
+        differentBaseTypeValidation(Int32Type.instance);
+        // PartitionerDefinedOrder's base type is  composite type
+        differentBaseTypeValidation(CompositeType.getInstance(Int32Type.instance, UTF8Type.instance));
+        // PartitionerDefinedOrder's base type is  tuple type
+        differentBaseTypeValidation(new TupleType(Lists.newArrayList(Int32Type.instance, UTF8Type.instance)));
+        // PartitionerDefinedOrder's base type is  ReversedType

Review Comment:
   ```suggestion
           // PartitionerDefinedOrder's base type is ReversedType
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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