You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by GitBox <gi...@apache.org> on 2023/01/04 12:07:34 UTC

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

blambov commented on code in PR #1998:
URL: https://github.com/apache/cassandra/pull/1998#discussion_r1061382544


##########
src/java/org/apache/cassandra/db/marshal/PartitionerDefinedOrder.java:
##########
@@ -107,6 +117,30 @@ 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;
+            return this.baseType.equals(other.baseType) && this.partitioner.equals(other.partitioner);

Review Comment:
   Does this not throw when `baseType` is null?



##########
src/java/org/apache/cassandra/db/marshal/TypeParser.java:
##########
@@ -130,6 +138,73 @@ 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();
+    }
+    
+    //the format is (partitioner:type)
+    public AbstractType<?> getPartitionerDefinedOrder()
+    {
+        if (isEOS())
+            return defaultParsePartitionOrdering(this);
+        skipBlank();
+        if (str.charAt(idx) != '(')
+            throw new IllegalStateException();
+        Pair<Boolean, AbstractType<?>>  result = null;
+        ++idx; // skipping '('
+
+        if (str.charAt(idx) == ')')
+        {
+            ++idx;
+            return  defaultParsePartitionOrdering(this);

Review Comment:
   In this case we have processed "PartitionerDefinedOrder()". Do we expect other parameters after the closing bracket?



##########
src/java/org/apache/cassandra/db/marshal/TypeParser.java:
##########
@@ -130,6 +138,73 @@ 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();
+    }
+    
+    //the format is (partitioner:type)
+    public AbstractType<?> getPartitionerDefinedOrder()
+    {
+        if (isEOS())
+            return defaultParsePartitionOrdering(this);
+        skipBlank();
+        if (str.charAt(idx) != '(')
+            throw new IllegalStateException();
+        Pair<Boolean, AbstractType<?>>  result = null;
+        ++idx; // skipping '('
+
+        if (str.charAt(idx) == ')')
+        {
+            ++idx;
+            return  defaultParsePartitionOrdering(this);
+        }
+        skipBlank();
+        String k = readNextIdentifier();
+        skipBlank();
+        if (str.charAt(idx) == ':')
+        {
+            ++idx;
+            skipBlank();
+        }
+        else if (str.charAt(idx) != ',' && str.charAt(idx) != ')')
+        {
+            return defaultParsePartitionOrdering(this);
+        }
+        IPartitioner partitioner = FBUtilities.newPartitioner(k);
+        AbstractType<?> type = partitioner.partitionOrdering();
+        if (partitioner.partitionOrdering() instanceof PartitionerDefinedOrder)
+        {
+            PartitionerDefinedOrder tmp = (PartitionerDefinedOrder) partitioner.partitionOrdering();
+            ++idx;
+            try
+            {
+                type = tmp.withBaseType(parse());
+            }
+            catch (Throwable throwable)
+            {
+                Iterator<String> argIterator = this.getKeyValueParameters().keySet().iterator();

Review Comment:
   "PartitionerDefinedOrder(%s)" can be handled more easily by just checking if `:` was seen. If not, just return `type`. If yes, then `((PartitionerDefinedOrder)type).withBaseType(parse())` must succeed.
   
   Are you trying to preserve existing bad behavior of the parser (e.g. accepting string such as "PartitionerDefinedOrder(Murmur3Partitioner=something)")?



##########
src/java/org/apache/cassandra/db/marshal/TypeParser.java:
##########
@@ -130,6 +138,73 @@ 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();
+    }
+    
+    //the format is (partitioner:type)
+    public AbstractType<?> getPartitionerDefinedOrder()
+    {
+        if (isEOS())
+            return defaultParsePartitionOrdering(this);
+        skipBlank();
+        if (str.charAt(idx) != '(')
+            throw new IllegalStateException();
+        Pair<Boolean, AbstractType<?>>  result = null;
+        ++idx; // skipping '('
+
+        if (str.charAt(idx) == ')')
+        {
+            ++idx;
+            return  defaultParsePartitionOrdering(this);
+        }
+        skipBlank();
+        String k = readNextIdentifier();
+        skipBlank();
+        if (str.charAt(idx) == ':')
+        {
+            ++idx;
+            skipBlank();
+        }
+        else if (str.charAt(idx) != ',' && str.charAt(idx) != ')')
+        {
+            return defaultParsePartitionOrdering(this);
+        }
+        IPartitioner partitioner = FBUtilities.newPartitioner(k);
+        AbstractType<?> type = partitioner.partitionOrdering();
+        if (partitioner.partitionOrdering() instanceof PartitionerDefinedOrder)

Review Comment:
   I see what you mean, and it makes sense if we want to preserve existing poor parsing behavior.
   
   However, I would treat an appearance of e.g. `PartitionerDefinedOrder(ByteOrderedPartitioner)` as an error, because in that case there exists an underlying type that can and will be used directly. If the type is specified as `PartitionerDefinedOrder`, the result of this construction must be an instance of that.
   
   The strings we are parsing here are generated internally; I don't believe the user has any control over them and thus can't construct invalid-but-acceptable definitions such as this.



##########
src/java/org/apache/cassandra/db/marshal/TypeParser.java:
##########
@@ -130,6 +138,73 @@ 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();
+    }
+    
+    //the format is (partitioner:type)
+    public AbstractType<?> getPartitionerDefinedOrder()
+    {
+        if (isEOS())
+            return defaultParsePartitionOrdering(this);
+        skipBlank();
+        if (str.charAt(idx) != '(')
+            throw new IllegalStateException();
+        Pair<Boolean, AbstractType<?>>  result = null;
+        ++idx; // skipping '('
+
+        if (str.charAt(idx) == ')')
+        {
+            ++idx;
+            return  defaultParsePartitionOrdering(this);
+        }
+        skipBlank();
+        String k = readNextIdentifier();
+        skipBlank();
+        if (str.charAt(idx) == ':')
+        {
+            ++idx;
+            skipBlank();
+        }
+        else if (str.charAt(idx) != ',' && str.charAt(idx) != ')')

Review Comment:
   Why do we accept a comma?



-- 
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