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
Conversation
563b6c0
to
2709120
Compare
Let me check travis failure. |
2709120
to
88f9138
Compare
I assume the travis failure (https://api.travis-ci.com/v3/job/194830803/log.txt) isn't related to this PR. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@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); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
- don't abbreviate: elemType -> elementType
- use
com.google.common.collect.Iterables#getOnlyElement
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); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
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) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
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)); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
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); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
You can replace .asList().get(position).getType()
with .getType(position)
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()) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
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() |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
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.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Agreed. Let me remove it.
...cassandra/src/test/java/io/prestosql/plugin/cassandra/TestCassandraIntegrationSmokeTest.java
Show resolved
Hide resolved
@@ -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}"); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can we keyspace_7 -> keyspace_test_nested_collection (or other meaningful name)?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sure, let me update it.
...cassandra/src/test/java/io/prestosql/plugin/cassandra/TestCassandraIntegrationSmokeTest.java
Show resolved
Hide resolved
88f9138
to
4ec5458
Compare
@findepi Updated and confirmed failed travis isn't related to this PR. |
Merged, thanks! |
Fix #655
Current CassandraType can't handle nested collection type. This PR support the type and remove
FullCassandraType
interface to simplify type mapping.