Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Support Cassandra nested collection type #657

Merged
merged 1 commit into from May 6, 2019

Conversation

2 participants
@ebyhr
Copy link
Contributor

commented Apr 22, 2019

Fix #655

Current CassandraType can't handle nested collection type. This PR support the type and remove FullCassandraType interface to simplify type mapping.

@cla-bot cla-bot bot added the cla-signed label Apr 22, 2019

@ebyhr ebyhr force-pushed the ebyhr:cassandra/nested-collection-v2 branch from 563b6c0 to 2709120 Apr 23, 2019

@ebyhr

This comment has been minimized.

Copy link
Contributor Author

commented Apr 23, 2019

Let me check travis failure.

@ebyhr ebyhr force-pushed the ebyhr:cassandra/nested-collection-v2 branch from 2709120 to 88f9138 Apr 23, 2019

@ebyhr

This comment has been minimized.

Copy link
Contributor Author

commented Apr 23, 2019

I assume the travis failure (https://api.travis-ci.com/v3/job/194830803/log.txt) isn't related to this PR.

@findepi
Copy link
Member

left a comment

@ebyhr thanks for your work on this.
I like the fact that this is mostly removals :)

A bunch of comments.

private static String buildArrayValue(Row row, int position)
{
Definition definition = row.getColumnDefinitions().asList().get(position);
DataType elemType = definition.getType().getTypeArguments().get(0);

This comment has been minimized.

Copy link
@findepi

findepi Apr 23, 2019

Member
  • don't abbreviate: elemType -> elementType
  • use com.google.common.collect.Iterables#getOnlyElement
Suggested change
DataType elemType = definition.getType().getTypeArguments().get(0);
DataType elementType = getOnlyElement(definition.getType().getTypeArguments());
{
return buildArrayValue(row.getSet(position, elemType.javaType), elemType);
Definition definition = row.getColumnDefinitions().asList().get(position);
DataType keyType = definition.getType().getTypeArguments().get(0);

This comment has been minimized.

Copy link
@findepi

findepi Apr 23, 2019

Member

checkArgument(definition.getType().getTypeArguments().size() == 2, "Expected two type arguments, got: %s", definition.getType().getTypeArguments())

}

private static String buildMapValue(Row row, int position, CassandraType keyType, CassandraType valueType)
private static String buildMapValue(Map<?, ?> collection, DataType keyType, DataType valueType)

This comment has been minimized.

Copy link
@findepi

findepi Apr 23, 2019

Member

collection -> map

@@ -383,32 +356,13 @@ private static String objectToString(Object object, CassandraType elemType)
case FLOAT:
case DECIMAL:
return object.toString();
case LIST:
case SET:
return buildArrayValue((Collection<?>) object, dataType.getTypeArguments().get(0));

This comment has been minimized.

Copy link
@findepi

findepi Apr 23, 2019

Member
Suggested change
return buildArrayValue((Collection<?>) object, dataType.getTypeArguments().get(0));
return buildArrayValue((Collection<?>) object, getOnlyElement(dataType.getTypeArguments()));
{
return buildArrayValue(row.getSet(position, elemType.javaType), elemType);
Definition definition = row.getColumnDefinitions().asList().get(position);

This comment has been minimized.

Copy link
@findepi

findepi Apr 23, 2019

Member

You can replace .asList().get(position).getType() with .getType(position)

Suggested change
Definition definition = row.getColumnDefinitions().asList().get(position);
Type type = row.getColumnDefinitions().getType(position);
@@ -520,6 +474,22 @@ public Object validateClusteringKey(Object value)
}
}

public static boolean containsUnsupportedType(DataType dataType)
{
if (!dataType.isCollection()) {

This comment has been minimized.

Copy link
@findepi

findepi Apr 23, 2019

Member

This could probably be simplified to (i also inverted meaning of the method):

public static boolean isFullySupported(DataType dataType)
{
    if (!toCassandraType(dataType.getName()).isPresent()) {
        return false;
    }

    return dataType.getTypeArguments().stream()
            .allMatch(CassandraType::isFullySupported);
}
import static org.testng.Assert.assertTrue;

public class TestCassandraType
{
@Test
public void testContainsUnsupportedType()

This comment has been minimized.

Copy link
@findepi

findepi Apr 23, 2019

Member

I wouldn't test io.prestosql.plugin.cassandra.CassandraType#containsUnsupportedType, for 2 reasons

  • as we add more supported types, we will need to update the test
  • the method is not that complex in itself (simple recursive visit over the type term), it's more about the assumption this method makes, how it interacts with other parts of the code base.

this is my opinion, though. Let me know if you disagree.

This comment has been minimized.

Copy link
@ebyhr

ebyhr Apr 23, 2019

Author Contributor

Agreed. Let me remove it.

Show resolved Hide resolved ...ava/io/prestosql/plugin/cassandra/TestCassandraIntegrationSmokeTest.java
@@ -428,6 +428,38 @@ public void testUnsupportedColumnType()
session.execute("DROP KEYSPACE keyspace_6");
}

public void testNestedCollectionType()
{
session.execute("CREATE KEYSPACE keyspace_7 WITH REPLICATION = {'class':'SimpleStrategy', 'replication_factor': 1}");

This comment has been minimized.

Copy link
@findepi

findepi Apr 23, 2019

Member

Can we keyspace_7 -> keyspace_test_nested_collection (or other meaningful name)?

This comment has been minimized.

Copy link
@ebyhr

ebyhr Apr 23, 2019

Author Contributor

Sure, let me update it.

Show resolved Hide resolved ...ava/io/prestosql/plugin/cassandra/TestCassandraIntegrationSmokeTest.java

@ebyhr ebyhr force-pushed the ebyhr:cassandra/nested-collection-v2 branch from 88f9138 to 4ec5458 Apr 24, 2019

@ebyhr

This comment has been minimized.

Copy link
Contributor Author

commented Apr 24, 2019

@findepi Updated and confirmed failed travis isn't related to this PR.

@martint martint requested a review from findepi Apr 29, 2019

@findepi findepi merged commit 95e2533 into prestosql:master May 6, 2019

2 checks passed

Travis CI - Pull Request Build Passed
Details
verification/cla-signed
Details
@findepi

This comment has been minimized.

Copy link
Member

commented May 6, 2019

Merged, thanks!
Sorry for the delay.

@findepi findepi referenced this pull request May 6, 2019

Closed

Release notes for 311 #716

5 of 5 tasks complete

@findepi findepi added this to the 311 milestone May 6, 2019

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
You can’t perform that action at this time.