From 96fd9581153f8480e790a2b96f4d92e4fcd584e2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Burak=20G=C3=B6k?= Date: Fri, 3 Nov 2023 20:50:36 +0300 Subject: [PATCH] Add nested field support for Avro [HZ-2509, HZ-3062] (#25269) This PR 1. refactors usages of CREATE TYPE and INSERT INTO statements, 2. refactors usages of factoryId/classId/version triplets, 3. decouples serialization format from user-defined types, 4. adds nested field support for Avro, 5. fixes HazelcastObjectType#digest and deletes HazelcastObjectTypeReference. Notes: - Type options always override mapping options. Specifically, if the mapping has only "__key" or "this" fields, and both the mapping and type define a schema ID, the one defined by the type is used. This also means that, in such situations, as long as the type defines a schema ID, the mapping does not need to. Breaking changes: - User must drop all user-defined types (UDTs) and mappings with UDTs before the rolling upgrade, and recreate them with the new semantics after upgrading. Closes HZ-2509 Closes HZ-3062 --- .../src/main/codegen/includes/parserImpls.ftl | 6 +- .../hazelcast/jet/sql/impl/PlanExecutor.java | 85 +--- .../jet/sql/impl/connector/SqlConnector.java | 3 + .../infoschema/UDTAttributesTable.java | 26 +- .../sql/impl/connector/kafka/KafkaTable.java | 6 +- .../connector/kafka/PropertiesResolver.java | 37 +- .../keyvalue/KvMetadataAvroResolver.java | 141 +++--- .../keyvalue/KvMetadataJavaResolver.java | 167 +++---- .../keyvalue/KvMetadataJsonResolver.java | 4 +- .../keyvalue/KvMetadataResolver.java | 66 ++- .../keyvalue/KvMetadataResolvers.java | 84 ++-- .../jet/sql/impl/connector/map/Joiner.java | 2 +- .../map/MetadataCompactResolver.java | 111 +++-- .../connector/map/MetadataJsonResolver.java | 4 +- .../map/MetadataPortableResolver.java | 210 +++++---- .../jet/sql/impl/connector/map/QueryUtil.java | 4 +- .../expression/UdtObjectToJsonFunction.java | 4 +- .../jet/sql/impl/extract/AvroQueryTarget.java | 3 +- .../jet/sql/impl/inject/AvroUpsertTarget.java | 31 +- .../inject/AvroUpsertTargetDescriptor.java | 5 + .../inject/HazelcastObjectUpsertTarget.java | 13 +- .../sql/impl/inject/PortableUpsertTarget.java | 3 +- .../impl/inject/UpsertTargetDescriptor.java | 4 + .../sql/impl/inject/UpsertTargetUtils.java | 64 ++- .../hazelcast/jet/sql/impl/opt/OptUtils.java | 56 +-- .../jet/sql/impl/parse/SqlCreateType.java | 2 +- .../jet/sql/impl/parse/SqlExtendedInsert.java | 3 +- .../jet/sql/impl/schema/RelationsStorage.java | 5 + .../sql/impl/schema/TypeDefinitionColumn.java | 10 +- .../jet/sql/impl/schema/TypeUtils.java | 363 +++++++++++++++ .../jet/sql/impl/schema/TypesUtils.java | 290 ------------ .../sql/impl/type/converter/ToConverters.java | 9 +- .../impl/validate/HazelcastSqlValidator.java | 10 +- .../HazelcastUdtObjectToJsonFunction.java | 4 +- .../validate/types/HazelcastObjectType.java | 131 ++++-- .../types/HazelcastObjectTypeReference.java | 208 --------- .../validate/types/HazelcastTypeCoercion.java | 6 +- .../validate/types/HazelcastTypeUtils.java | 16 +- .../sql/impl/expression/ColumnExpression.java | 7 +- .../sql/impl/expression/Expression.java | 2 + .../expression/FieldAccessExpression.java | 38 +- .../predicate/ComparisonPredicate.java | 2 +- .../hazelcast/sql/impl/extract/QueryPath.java | 2 +- .../sql/impl/schema/MappingField.java | 6 +- .../hazelcast/sql/impl/schema/TableField.java | 16 +- .../hazelcast/sql/impl/schema/type/Type.java | 205 ++------ .../sql/impl/schema/type/TypeKind.java | 3 +- .../sql/impl/type/QueryDataType.java | 237 +++++----- .../com/hazelcast/jet/sql/SqlTestSupport.java | 247 +++++++--- .../jet/sql/impl/connector/file/FileUtil.java | 24 +- .../sql/impl/connector/file/SqlAvroTest.java | 2 +- .../infoschema/SqlInfoSchemaTest.java | 53 +-- .../connector/kafka/KafkaSqlTestSupport.java | 58 ++- .../kafka/PropertiesResolverTest.java | 132 +++--- .../kafka/SqlAvroSchemaEvolutionTest.java | 34 +- .../sql/impl/connector/kafka/SqlAvroTest.java | 254 +++++----- .../sql/impl/connector/kafka/SqlJsonTest.java | 292 ++++-------- .../connector/kafka/SqlPlanCacheTest.java | 53 +-- .../sql/impl/connector/kafka/SqlPojoTest.java | 232 ++++------ .../connector/kafka/SqlPrimitiveTest.java | 384 +++++---------- .../keyvalue/KvMetadataAvroResolverTest.java | 28 +- .../map/MetadataCompactResolverTest.java | 41 +- .../sql/impl/connector/map/SqlJsonTest.java | 197 ++++---- .../sql/impl/connector/map/SqlPojoTest.java | 200 ++++---- .../jet/sql/impl/schema/GetDdlTest.java | 40 +- .../sql/impl/type/BasicNestedFieldsTest.java | 437 +++++++++--------- .../impl/type/CompactNestedFieldsTest.java | 89 ++-- .../jet/sql/impl/type/NestedTypesDDLTest.java | 178 ++----- .../impl/type/PortableNestedFieldsTest.java | 85 ++-- .../RecurrentStructuresNestedFieldsTest.java | 117 ++--- .../impl/type/SqlLazyDeserializationTest.java | 5 +- .../type/UdtObjectToJsonFunctionTest.java | 85 ++-- .../sql/impl/extract/QueryPathTest.java | 2 +- .../impl/portable/ClassDefinitionImpl.java | 63 +-- .../impl/portable/FieldDefinitionImpl.java | 72 +-- .../impl/portable/PortableContext.java | 3 + .../impl/portable/PortableContextImpl.java | 9 +- .../util/collection/DefaultedMap.java | 66 ++- .../jet/impl/util/ExceptionUtil.java | 11 - .../com/hazelcast/jet/impl/util/Util.java | 30 ++ .../nio/serialization/ClassDefinition.java | 10 +- .../serialization/ClassDefinitionBuilder.java | 67 +-- .../nio/serialization/FieldDefinition.java | 16 +- .../nio/serialization/PortableId.java | 118 +++++ .../hazelcast/test/HazelcastTestSupport.java | 174 ++++--- .../com/hazelcast/test/MobyNamingRule.java | 2 +- 86 files changed, 3134 insertions(+), 3490 deletions(-) create mode 100644 hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/schema/TypeUtils.java delete mode 100644 hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/schema/TypesUtils.java delete mode 100644 hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/validate/types/HazelcastObjectTypeReference.java create mode 100644 hazelcast/src/main/java/com/hazelcast/nio/serialization/PortableId.java diff --git a/hazelcast-sql/src/main/codegen/includes/parserImpls.ftl b/hazelcast-sql/src/main/codegen/includes/parserImpls.ftl index 7d6888f95777..dd5d170a07f5 100644 --- a/hazelcast-sql/src/main/codegen/includes/parserImpls.ftl +++ b/hazelcast-sql/src/main/codegen/includes/parserImpls.ftl @@ -138,8 +138,10 @@ SqlCreate SqlCreateType(Span span, boolean replace) : name = CompoundIdentifier() columns = TypeColumns() - - sqlOptions = SqlOptions() + [ + + sqlOptions = SqlOptions() + ] { return new SqlCreateType( name, diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/PlanExecutor.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/PlanExecutor.java index 0007a0d4b980..f743d3e47df5 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/PlanExecutor.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/PlanExecutor.java @@ -36,7 +36,6 @@ import com.hazelcast.jet.datamodel.Tuple2; import com.hazelcast.jet.impl.AbstractJetInstance; import com.hazelcast.jet.impl.JetServiceBackend; -import com.hazelcast.jet.impl.util.ReflectionUtils; import com.hazelcast.jet.impl.util.Util; import com.hazelcast.jet.sql.impl.SqlPlanImpl.AlterJobPlan; import com.hazelcast.jet.sql.impl.SqlPlanImpl.CreateIndexPlan; @@ -60,12 +59,9 @@ import com.hazelcast.jet.sql.impl.SqlPlanImpl.IMapUpdatePlan; import com.hazelcast.jet.sql.impl.SqlPlanImpl.SelectPlan; import com.hazelcast.jet.sql.impl.SqlPlanImpl.ShowStatementPlan; -import com.hazelcast.jet.sql.impl.connector.SqlConnector; import com.hazelcast.jet.sql.impl.parse.SqlShowStatement.ShowStatementTarget; import com.hazelcast.jet.sql.impl.schema.DataConnectionResolver; import com.hazelcast.jet.sql.impl.schema.TableResolverImpl; -import com.hazelcast.jet.sql.impl.schema.TypeDefinitionColumn; -import com.hazelcast.jet.sql.impl.schema.TypesUtils; import com.hazelcast.jet.sql.impl.validate.UpdateDataConnectionOperation; import com.hazelcast.logging.ILogger; import com.hazelcast.map.IMap; @@ -74,7 +70,6 @@ import com.hazelcast.map.impl.MapService; import com.hazelcast.map.impl.MapServiceContext; import com.hazelcast.map.impl.proxy.MapProxyImpl; -import com.hazelcast.nio.serialization.ClassDefinition; import com.hazelcast.partition.PartitioningStrategy; import com.hazelcast.partition.strategy.AttributePartitioningStrategy; import com.hazelcast.partition.strategy.DefaultPartitioningStrategy; @@ -97,7 +92,6 @@ import com.hazelcast.sql.impl.row.JetSqlRow; import com.hazelcast.sql.impl.schema.dataconnection.DataConnectionCatalogEntry; import com.hazelcast.sql.impl.schema.type.Type; -import com.hazelcast.sql.impl.schema.type.TypeKind; import com.hazelcast.sql.impl.schema.view.View; import com.hazelcast.sql.impl.security.SqlSecurityContext; import com.hazelcast.sql.impl.state.QueryResultRegistry; @@ -116,7 +110,6 @@ import java.util.Locale; import java.util.Map; import java.util.Map.Entry; -import java.util.Optional; import java.util.Set; import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; @@ -135,7 +128,6 @@ import static com.hazelcast.jet.impl.JetServiceBackend.SQL_ARGUMENTS_KEY_NAME; import static com.hazelcast.jet.impl.util.ExceptionUtil.isTopologyException; import static com.hazelcast.jet.impl.util.Util.getNodeEngine; -import static com.hazelcast.jet.impl.util.Util.getSerializationService; import static com.hazelcast.jet.sql.impl.SqlPlanImpl.CreateDataConnectionPlan; import static com.hazelcast.jet.sql.impl.parse.SqlCreateIndex.UNIQUE_KEY; import static com.hazelcast.jet.sql.impl.parse.SqlCreateIndex.UNIQUE_KEY_TRANSFORMATION; @@ -605,7 +597,7 @@ SqlResult execute(IMapSelectPlan plan, long timeout, @Nonnull SqlSecurityContext ssc) { List args = prepareArguments(plan.parameterMetadata(), arguments); - InternalSerializationService serializationService = getSerializationService(hazelcastInstance); + InternalSerializationService serializationService = Util.getSerializationService(hazelcastInstance); ExpressionEvalContext evalContext = ExpressionEvalContext.createContext( args, hazelcastInstance, @@ -723,81 +715,8 @@ SqlResult execute(CreateTypePlan plan) { throw QueryException.error("Experimental feature of creating custom types isn't enabled. To enable, set " + SQL_CUSTOM_TYPES_ENABLED + " to true"); } - final String format = plan.options().get(SqlConnector.OPTION_FORMAT); - final Type type; - - if (SqlConnector.PORTABLE_FORMAT.equals(format)) { - final Integer factoryId = Optional.ofNullable(plan.option(SqlConnector.OPTION_TYPE_PORTABLE_FACTORY_ID)) - .map(Integer::parseInt) - .orElse(null); - final Integer classId = Optional.ofNullable(plan.option(SqlConnector.OPTION_TYPE_PORTABLE_CLASS_ID)) - .map(Integer::parseInt) - .orElse(null); - final Integer version = Optional.ofNullable(plan.option(SqlConnector.OPTION_TYPE_PORTABLE_CLASS_VERSION)) - .map(Integer::parseInt) - .orElse(0); - - if (factoryId == null || classId == null) { - throw QueryException.error("FactoryID and ClassID are required for Portable Types"); - } - - final ClassDefinition existingClassDef = getSerializationService(hazelcastInstance).getPortableContext() - .lookupClassDefinition(factoryId, classId, version); - - if (existingClassDef != null) { - type = TypesUtils.convertPortableClassToType(plan.name(), existingClassDef, catalog); - } else { - if (plan.columns().isEmpty()) { - throw QueryException.error("The given FactoryID/ClassID/Version combination not known to the member. " + - "You need to provide column list for this type"); - } - - type = new Type(); - type.setName(plan.name()); - type.setKind(TypeKind.PORTABLE); - type.setPortableFactoryId(factoryId); - type.setPortableClassId(classId); - type.setPortableVersion(version); - type.setFields(new ArrayList<>()); - - for (int i = 0; i < plan.columns().size(); i++) { - final TypeDefinitionColumn planColumn = plan.columns().get(i); - type.getFields().add(new Type.TypeField(planColumn.name(), planColumn.dataType())); - } - } - } else if (SqlConnector.COMPACT_FORMAT.equals(format)) { - if (plan.columns().isEmpty()) { - throw QueryException.error("Column list is required to create Compact-based Types"); - } - type = new Type(); - type.setKind(TypeKind.COMPACT); - type.setName(plan.name()); - final List typeFields = plan.columns().stream() - .map(typeColumn -> new Type.TypeField(typeColumn.name(), typeColumn.dataType())) - .collect(Collectors.toList()); - type.setFields(typeFields); - - final String compactTypeName = plan.option(SqlConnector.OPTION_TYPE_COMPACT_TYPE_NAME); - if (compactTypeName == null || compactTypeName.isEmpty()) { - throw QueryException.error("Compact Type Name must not be empty for Compact-based Types."); - } - type.setCompactTypeName(compactTypeName); - } else if (SqlConnector.JAVA_FORMAT.equals(format)) { - final Class typeClass; - try { - typeClass = ReflectionUtils.loadClass(plan.options().get(SqlConnector.OPTION_TYPE_JAVA_CLASS)); - } catch (Exception e) { - throw QueryException.error("Unable to load class: '" - + plan.options().get(SqlConnector.OPTION_TYPE_JAVA_CLASS) + "'", e); - } - - type = TypesUtils.convertJavaClassToType(plan.name(), plan.columns(), typeClass); - } else { - throw QueryException.error("Unsupported type format: " + format); - } - + final Type type = new Type(plan.name(), plan.columns(), plan.options()); catalog.createType(type, plan.replace(), plan.ifNotExists()); - return UpdateSqlResultImpl.createUpdateCountResult(0); } diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/SqlConnector.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/SqlConnector.java index bcb89d030b16..82ddd92b0d51 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/SqlConnector.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/SqlConnector.java @@ -178,6 +178,8 @@ public interface SqlConnector { String OPTION_TYPE_PORTABLE_CLASS_VERSION = "portableClassVersion"; + String OPTION_TYPE_AVRO_SCHEMA = "avroSchema"; + /** * Value for {@value #OPTION_KEY_FORMAT} and {@value #OPTION_VALUE_FORMAT} * for Java serialization. @@ -558,6 +560,7 @@ default Set nonSensitiveConnectorOptions() { options.add(OPTION_TYPE_PORTABLE_FACTORY_ID); options.add(OPTION_TYPE_PORTABLE_CLASS_ID); options.add(OPTION_TYPE_PORTABLE_CLASS_VERSION); + options.add(OPTION_TYPE_AVRO_SCHEMA); return options; } diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/infoschema/UDTAttributesTable.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/infoschema/UDTAttributesTable.java index 46cd12405c69..6431549e6517 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/infoschema/UDTAttributesTable.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/infoschema/UDTAttributesTable.java @@ -95,11 +95,11 @@ protected List rows() { i + 1, null, // attribute_default "YES", // is_nullable - toSqlDataTypeString(field.getQueryDataType()), // data type - field.getQueryDataType().getTypeFamily().equals(QueryDataTypeFamily.VARCHAR) + toSqlDataTypeString(field.getType()), // data type + field.getType().getTypeFamily() == QueryDataTypeFamily.VARCHAR ? Integer.MAX_VALUE : null, // character_maximum_length - field.getQueryDataType().getTypeFamily().equals(QueryDataTypeFamily.VARCHAR) + field.getType().getTypeFamily() == QueryDataTypeFamily.VARCHAR ? Integer.MAX_VALUE : null, // character_octet_length null, // character_set_catalog @@ -108,28 +108,28 @@ protected List rows() { null, // collation_catalog null, // collation_schema null, // collation_name - field.getQueryDataType().getTypeFamily().isNumeric() - ? getNumericTypePrecision(field.getQueryDataType()) + field.getType().getTypeFamily().isNumeric() + ? getNumericTypePrecision(field.getType()) : null, // numeric_precision - field.getQueryDataType().getTypeFamily().isNumeric() + field.getType().getTypeFamily().isNumeric() ? 2 : null, // numeric_precision_radix - field.getQueryDataType().getTypeFamily().isNumericInteger() + field.getType().getTypeFamily().isNumericInteger() ? 0 : null, // numeric_scale - field.getQueryDataType().getTypeFamily().isTemporal() - ? getTemporalTypePrecision(field.getQueryDataType()) + field.getType().getTypeFamily().isTemporal() + ? getTemporalTypePrecision(field.getType()) : null, // datetime_precision null, // interval_type null, // interval_precision - field.getQueryDataType().isCustomType() + field.getType().isCustomType() ? catalog() : null, // attribute_udt_catalog - field.getQueryDataType().isCustomType() + field.getType().isCustomType() ? schema : null, // attribute_udt_schema - field.getQueryDataType().isCustomType() - ? field.getQueryDataType().getObjectTypeName() + field.getType().isCustomType() + ? field.getType().getObjectTypeName() : null, // attribute_udt_name null, // scope_catalog null, // scope_schema diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/kafka/KafkaTable.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/kafka/KafkaTable.java index af5d351a7fae..941b95b0b012 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/kafka/KafkaTable.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/kafka/KafkaTable.java @@ -82,11 +82,13 @@ String dataConnectionName() { } Properties kafkaConsumerProperties() { - return PropertiesResolver.resolveConsumerProperties(options); + return PropertiesResolver.resolveConsumerProperties(options, + keyUpsertDescriptor.getSchema(), valueUpsertDescriptor.getSchema()); } Properties kafkaProducerProperties() { - return PropertiesResolver.resolveProducerProperties(options); + return PropertiesResolver.resolveProducerProperties(options, + keyUpsertDescriptor.getSchema(), valueUpsertDescriptor.getSchema()); } QueryTargetDescriptor keyQueryDescriptor() { diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/kafka/PropertiesResolver.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/kafka/PropertiesResolver.java index 8c0824917f52..71bfae20da1c 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/kafka/PropertiesResolver.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/kafka/PropertiesResolver.java @@ -21,7 +21,6 @@ import com.hazelcast.jet.kafka.HazelcastKafkaAvroSerializer; import com.hazelcast.jet.kafka.impl.HazelcastJsonValueDeserializer; import com.hazelcast.jet.kafka.impl.HazelcastJsonValueSerializer; -import com.hazelcast.jet.sql.impl.connector.SqlConnector; import com.hazelcast.jet.sql.impl.connector.keyvalue.JavaClassNameResolver; import java.util.Map; @@ -32,8 +31,10 @@ import static com.hazelcast.jet.sql.impl.connector.SqlConnector.AVRO_FORMAT; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.JAVA_FORMAT; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.JSON_FLAT_FORMAT; +import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_KEY_AVRO_SCHEMA; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_KEY_CLASS; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_KEY_FORMAT; +import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_AVRO_SCHEMA; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_CLASS; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_FORMAT; @@ -84,20 +85,20 @@ final class PropertiesResolver { private PropertiesResolver() { } - static Properties resolveConsumerProperties(Map options) { + static Properties resolveConsumerProperties(Map options, Object keySchema, Object valueSchema) { Properties properties = from(options); - withSerdeConsumerProperties(true, options, properties); - withSerdeConsumerProperties(false, options, properties); + withSerdeConsumerProperties(true, options, keySchema, properties); + withSerdeConsumerProperties(false, options, valueSchema, properties); return properties; } - static Properties resolveProducerProperties(Map options) { + static Properties resolveProducerProperties(Map options, Object keySchema, Object valueSchema) { Properties properties = from(options); - withSerdeProducerProperties(true, options, properties); - withSerdeProducerProperties(false, options, properties); + withSerdeProducerProperties(true, options, keySchema, properties); + withSerdeProducerProperties(false, options, valueSchema, properties); return properties; } @@ -118,6 +119,7 @@ private static Properties from(Map options) { private static void withSerdeConsumerProperties( boolean isKey, Map options, + Object schema, Properties properties ) { String deserializer = isKey ? KEY_DESERIALIZER : VALUE_DESERIALIZER; @@ -126,12 +128,16 @@ private static void withSerdeConsumerProperties( if (format == null && isKey) { properties.putIfAbsent(deserializer, BYTE_ARRAY_DESERIALIZER); } else if (AVRO_FORMAT.equals(format)) { - properties.putIfAbsent(deserializer, options.containsKey("schema.registry.url") - ? CONFLUENT_AVRO_DESERIALIZER : HAZELCAST_AVRO_DESERIALIZER); + if (options.containsKey("schema.registry.url")) { + properties.putIfAbsent(deserializer, CONFLUENT_AVRO_DESERIALIZER); + } else { + properties.putIfAbsent(deserializer, HAZELCAST_AVRO_DESERIALIZER); + properties.put(isKey ? OPTION_KEY_AVRO_SCHEMA : OPTION_VALUE_AVRO_SCHEMA, schema); + } } else if (JSON_FLAT_FORMAT.equals(format)) { properties.putIfAbsent(deserializer, BYTE_ARRAY_DESERIALIZER); } else if (JAVA_FORMAT.equals(format)) { - String clazz = options.get(isKey ? SqlConnector.OPTION_KEY_CLASS : SqlConnector.OPTION_VALUE_CLASS); + String clazz = options.get(isKey ? OPTION_KEY_CLASS : OPTION_VALUE_CLASS); String deserializerClass = resolveDeserializer(clazz); if (deserializerClass != null) { properties.putIfAbsent(deserializer, deserializerClass); @@ -170,6 +176,7 @@ private static String resolveDeserializer(String clazz) { private static void withSerdeProducerProperties( boolean isKey, Map options, + Object schema, Properties properties ) { String serializer = isKey ? KEY_SERIALIZER : VALUE_SERIALIZER; @@ -178,12 +185,16 @@ private static void withSerdeProducerProperties( if (format == null && isKey) { properties.putIfAbsent(serializer, BYTE_ARRAY_SERIALIZER); } else if (AVRO_FORMAT.equals(format)) { - properties.putIfAbsent(serializer, options.containsKey("schema.registry.url") - ? CONFLUENT_AVRO_SERIALIZER : HAZELCAST_AVRO_SERIALIZER); + if (options.containsKey("schema.registry.url")) { + properties.putIfAbsent(serializer, CONFLUENT_AVRO_SERIALIZER); + } else { + properties.putIfAbsent(serializer, HAZELCAST_AVRO_SERIALIZER); + properties.put(isKey ? OPTION_KEY_AVRO_SCHEMA : OPTION_VALUE_AVRO_SCHEMA, schema); + } } else if (JSON_FLAT_FORMAT.equals(format)) { properties.putIfAbsent(serializer, BYTE_ARRAY_SERIALIZER); } else if (JAVA_FORMAT.equals(format)) { - String clazz = options.get(isKey ? SqlConnector.OPTION_KEY_CLASS : SqlConnector.OPTION_VALUE_CLASS); + String clazz = options.get(isKey ? OPTION_KEY_CLASS : OPTION_VALUE_CLASS); String serializerClass = resolveSerializer(clazz); if (serializerClass != null) { properties.putIfAbsent(serializer, serializerClass); diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/keyvalue/KvMetadataAvroResolver.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/keyvalue/KvMetadataAvroResolver.java index a2baa74810c0..43fd81cd8626 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/keyvalue/KvMetadataAvroResolver.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/keyvalue/KvMetadataAvroResolver.java @@ -18,7 +18,6 @@ import com.google.common.collect.ImmutableMap; import com.hazelcast.internal.serialization.InternalSerializationService; -import com.hazelcast.jet.impl.util.ExceptionUtil; import com.hazelcast.jet.sql.impl.extract.AvroQueryTargetDescriptor; import com.hazelcast.jet.sql.impl.inject.AvroUpsertTargetDescriptor; import com.hazelcast.sql.impl.QueryException; @@ -30,7 +29,6 @@ import com.hazelcast.sql.impl.type.QueryDataTypeFamily; import org.apache.avro.Schema; import org.apache.avro.SchemaBuilder; -import org.apache.avro.SchemaBuilder.FieldAssembler; import java.util.ArrayList; import java.util.List; @@ -39,6 +37,7 @@ import java.util.Set; import java.util.stream.Stream; +import static com.hazelcast.jet.impl.util.Util.reduce; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.AVRO_FORMAT; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_KEY_AVRO_RECORD_NAME; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_KEY_AVRO_SCHEMA; @@ -46,10 +45,16 @@ import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_AVRO_SCHEMA; import static com.hazelcast.jet.sql.impl.connector.file.AvroResolver.unwrapNullableType; import static com.hazelcast.jet.sql.impl.connector.keyvalue.KvMetadataResolver.extractFields; +import static com.hazelcast.jet.sql.impl.connector.keyvalue.KvMetadataResolver.getFields; +import static com.hazelcast.jet.sql.impl.connector.keyvalue.KvMetadataResolver.getMetadata; import static com.hazelcast.jet.sql.impl.connector.keyvalue.KvMetadataResolver.maybeAddDefaultField; import static com.hazelcast.jet.sql.impl.inject.AvroUpsertTarget.CONVERSION_PREFS; import static com.hazelcast.sql.impl.type.converter.Converters.getConverter; +import static java.util.stream.Collectors.toList; import static java.util.stream.Collectors.toSet; +import static org.apache.avro.Schema.Type.NULL; +import static org.apache.avro.Schema.Type.RECORD; +import static org.apache.avro.Schema.Type.UNION; public final class KvMetadataAvroResolver implements KvMetadataResolver { @@ -73,13 +78,11 @@ public static class Schemas { .and().stringType() .endUnion(); - private static final Map> CONVERSIONS = - CONVERSION_PREFS.entrySet().stream() - .collect(ImmutableMap::>builder, - (map, e) -> map.put(getConverter(e.getKey()).getTypeFamily(), e.getValue()), - ExceptionUtil::combinerUnsupported) - .put(QueryDataTypeFamily.OBJECT, List.of(Schema.Type.UNION, Schema.Type.NULL)) - .build(); + private static final Map> CONVERSIONS = reduce( + ImmutableMap.>builder(), + CONVERSION_PREFS.entrySet().stream(), + (map, e) -> map.put(getConverter(e.getKey()).getTypeFamily(), e.getValue()) + ).build(); } private KvMetadataAvroResolver() { } @@ -99,19 +102,19 @@ public Stream resolveAndValidateFields( if (userFields.isEmpty()) { throw QueryException.error("Column list is required for Avro format"); } - String inlineSchema = options.get(isKey ? OPTION_KEY_AVRO_SCHEMA : OPTION_VALUE_AVRO_SCHEMA); - if (inlineSchema != null && options.containsKey("schema.registry.url")) { + Map fieldsByPath = extractFields(userFields, isKey); + + Schema schema = getSchema(fieldsByPath, options, isKey); + if (schema != null && options.containsKey("schema.registry.url")) { throw new IllegalArgumentException("Inline schema cannot be used with schema registry"); } - Map fieldsByPath = extractFields(userFields, isKey); for (QueryPath path : fieldsByPath.keySet()) { - if (path.getPath() == null) { + if (path.isTopLevel()) { throw QueryException.error("Cannot use the '" + path + "' field with Avro serialization"); } } - if (inlineSchema != null) { - Schema schema = new Schema.Parser().parse(inlineSchema); - validate(schema, fieldsByPath); + if (schema != null) { + validate(schema, getFields(fieldsByPath).collect(toList())); } return fieldsByPath.values().stream(); } @@ -135,14 +138,11 @@ public KvMetadata resolveMetadata( } maybeAddDefaultField(isKey, resolvedFields, fields, QueryDataType.OBJECT); - Schema schema; - String inlineSchema = options.get(isKey ? OPTION_KEY_AVRO_SCHEMA : OPTION_VALUE_AVRO_SCHEMA); - if (inlineSchema != null) { - schema = new Schema.Parser().parse(inlineSchema); - } else { + Schema schema = getSchema(fieldsByPath, options, isKey); + if (schema == null) { String recordName = options.getOrDefault( isKey ? OPTION_KEY_AVRO_RECORD_NAME : OPTION_VALUE_AVRO_RECORD_NAME, "jet.sql"); - schema = resolveSchema(recordName, fields); + schema = resolveSchema(recordName, getFields(fieldsByPath)); } return new KvMetadata( fields, @@ -152,66 +152,61 @@ public KvMetadata resolveMetadata( } // CREATE MAPPING () Type Kafka; INSERT INTO ... - private static Schema resolveSchema(String recordName, List fields) { - FieldAssembler schema = SchemaBuilder.record(recordName).fields(); - for (TableField field : fields) { - String path = ((MapTableField) field).getPath().getPath(); - if (path == null) { - continue; - } - switch (field.getType().getTypeFamily()) { + private static Schema resolveSchema(String recordName, Stream fields) { + return reduce(SchemaBuilder.record(recordName).fields(), fields, (schema, field) -> { + switch (field.type().getTypeFamily()) { case BOOLEAN: - schema = schema.optionalBoolean(path); - break; + return schema.optionalBoolean(field.name()); case TINYINT: case SMALLINT: case INTEGER: - schema = schema.optionalInt(path); - break; + return schema.optionalInt(field.name()); case BIGINT: - schema = schema.optionalLong(path); - break; + return schema.optionalLong(field.name()); case REAL: - schema = schema.optionalFloat(path); - break; + return schema.optionalFloat(field.name()); case DOUBLE: - schema = schema.optionalDouble(path); - break; + return schema.optionalDouble(field.name()); case DECIMAL: case TIME: case DATE: case TIMESTAMP: case TIMESTAMP_WITH_TIME_ZONE: case VARCHAR: - schema = schema.optionalString(path); - break; + return schema.optionalString(field.name()); case OBJECT: - schema = schema.name(path).type(Schemas.OBJECT_SCHEMA).withDefault(null); - break; + Schema fieldSchema = field.type().isCustomType() + ? resolveSchema(field.type().getObjectTypeName(), + field.type().getObjectFields().stream().map(Field::new)) + : Schemas.OBJECT_SCHEMA; + return schema.name(field.name()).type(optional(fieldSchema)).withDefault(null); default: - throw new IllegalArgumentException("Unsupported type: " + field.getType()); + throw new IllegalArgumentException("Unsupported type: " + field.type()); } - } - return schema.endRecord(); + }).endRecord(); } - private static void validate(Schema schema, Map fieldsByPath) { - if (schema.getType() != Schema.Type.RECORD) { + private static void validate(Schema schema, List fields) { + if (schema.getType() != RECORD) { throw new IllegalArgumentException("Schema must be an Avro record"); } - Set mappingFields = fieldsByPath.keySet().stream().map(QueryPath::getPath).collect(toSet()); + Set mappingFields = fields.stream().map(Field::name).collect(toSet()); for (Schema.Field schemaField : schema.getFields()) { - if (!schemaField.schema().isNullable() && !mappingFields.contains(schemaField.name())) { + if (!schemaField.hasDefaultValue() && !mappingFields.contains(schemaField.name())) { throw new IllegalArgumentException("Mandatory field '" + schemaField.name() + "' is not mapped to any column"); } } - for (Entry entry : fieldsByPath.entrySet()) { - String path = entry.getKey().getPath(); - QueryDataType mappingFieldType = entry.getValue().type(); + for (Field field : fields) { + String path = field.name(); + QueryDataType mappingFieldType = field.type(); QueryDataTypeFamily mappingFieldTypeFamily = mappingFieldType.getTypeFamily(); - List conversions = Schemas.CONVERSIONS.get(mappingFieldTypeFamily); + List conversions = mappingFieldTypeFamily == QueryDataTypeFamily.OBJECT + ? mappingFieldType.isCustomType() + ? List.of(RECORD) // Unwrapped, so does not include NULL + : List.of(UNION, NULL) // Ordinary OBJECT can be mapped to NULL + : Schemas.CONVERSIONS.get(mappingFieldTypeFamily); if (conversions == null) { throw new IllegalArgumentException("Unsupported type: " + mappingFieldType); } @@ -221,11 +216,45 @@ private static void validate(Schema schema, Map fieldsB throw new IllegalArgumentException("Field '" + path + "' does not exist in schema"); } - Schema.Type schemaFieldType = unwrapNullableType(schemaField.schema()).getType(); + Schema fieldSchema = unwrapNullableType(schemaField.schema()); + Schema.Type schemaFieldType = fieldSchema.getType(); if (!conversions.contains(schemaFieldType)) { throw new IllegalArgumentException(schemaFieldType + " schema type is incompatible with " + mappingFieldType + " mapping type"); } + + if (mappingFieldType.isCustomType()) { + validate(fieldSchema, mappingFieldType.getObjectFields().stream().map(Field::new).collect(toList())); + } } } + + private static Schema getSchema(Map fields, Map options, boolean isKey) { + return getMetadata(fields) + .map(json -> new Schema.Parser().parse(json)) + .orElseGet(() -> inlineSchema(options, isKey)); + } + + /** + * A field is nullable if it can be set to null. A nullable field is + * optional if it can have null default value, which is only possible if + * its type is {@code NULL} or a {@code UNION} with {@code NULL} as the first element. + */ + public static Schema optional(Schema schema) { + if (schema.getType() == UNION) { + return schema.getTypes().get(0).getType() == NULL ? schema + : reduce( + SchemaBuilder.unionOf().nullType(), + schema.getTypes().stream().filter(type -> type.getType() != NULL), + (union, type) -> union.and().type(type) + ).endUnion(); + } + return schema.getType() == NULL ? schema + : SchemaBuilder.unionOf().nullType().and().type(schema).endUnion(); + } + + public static Schema inlineSchema(Map options, boolean isKey) { + String json = options.get(isKey ? OPTION_KEY_AVRO_SCHEMA : OPTION_VALUE_AVRO_SCHEMA); + return json != null ? new Schema.Parser().parse(json) : null; + } } diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/keyvalue/KvMetadataJavaResolver.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/keyvalue/KvMetadataJavaResolver.java index 502bd61cef4e..9250e8d4e329 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/keyvalue/KvMetadataJavaResolver.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/keyvalue/KvMetadataJavaResolver.java @@ -45,20 +45,22 @@ import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_CLASS; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_FORMAT; import static com.hazelcast.jet.sql.impl.connector.keyvalue.KvMetadataResolver.extractFields; +import static com.hazelcast.jet.sql.impl.connector.keyvalue.KvMetadataResolver.getMetadata; +import static com.hazelcast.jet.sql.impl.connector.keyvalue.KvMetadataResolver.getTopLevelType; import static com.hazelcast.jet.sql.impl.connector.keyvalue.KvMetadataResolver.maybeAddDefaultField; import static com.hazelcast.sql.impl.extract.QueryPath.KEY; import static com.hazelcast.sql.impl.extract.QueryPath.VALUE; +import static java.util.Map.entry; /** - * A utility for key-value connectors that use Java serialization ({@link - * java.io.Serializable}) to resolve fields. + * A utility for key-value connectors that use Java serialization + * ({@link java.io.Serializable}) to resolve fields. */ public final class KvMetadataJavaResolver implements KvMetadataResolver { public static final KvMetadataJavaResolver INSTANCE = new KvMetadataJavaResolver(); - private KvMetadataJavaResolver() { - } + private KvMetadataJavaResolver() { } @Override public Stream supportedFormats() { @@ -72,33 +74,24 @@ public Stream resolveAndValidateFields( Map options, InternalSerializationService serializationService ) { - Class clazz = loadClass(isKey, options); - return resolveFields(isKey, userFields, clazz); - } + Map fieldsByPath = extractFields(userFields, isKey); - public Stream resolveFields( - boolean isKey, - List userFields, - Class clazz - ) { - QueryDataType type = QueryDataTypeUtils.resolveTypeForClass(clazz); - if (!type.getTypeFamily().equals(QueryDataTypeFamily.OBJECT) || type.isCustomType()) { - return resolvePrimitiveSchema(isKey, userFields, type); + Class typeClass = getMetadata(fieldsByPath) + .>map(KvMetadataJavaResolver::loadClass) + .orElseGet(() -> loadClass(options, isKey)); + QueryDataType type = QueryDataTypeUtils.resolveTypeForClass(typeClass); + + if (type.getTypeFamily() != QueryDataTypeFamily.OBJECT || type.isCustomType()) { + return userFields.isEmpty() + ? resolvePrimitiveField(isKey, type) + : resolveAndValidatePrimitiveField(isKey, fieldsByPath, type); } else { - return resolveObjectSchema(isKey, userFields, clazz); + return userFields.isEmpty() + ? resolveObjectFields(isKey, typeClass) + : resolveAndValidateObjectFields(isKey, fieldsByPath, typeClass); } } - private Stream resolvePrimitiveSchema( - boolean isKey, - List userFields, - QueryDataType type - ) { - return userFields.isEmpty() - ? resolvePrimitiveField(isKey, type) - : resolveAndValidatePrimitiveField(isKey, userFields, type); - } - private Stream resolvePrimitiveField(boolean isKey, QueryDataType type) { QueryPath path = isKey ? QueryPath.KEY_PATH : QueryPath.VALUE_PATH; String name = isKey ? KEY : VALUE; @@ -109,51 +102,39 @@ private Stream resolvePrimitiveField(boolean isKey, QueryDataType private Stream resolveAndValidatePrimitiveField( boolean isKey, - List userFields, + Map fieldsByPath, QueryDataType type ) { - Map userFieldsByPath = extractFields(userFields, isKey); - QueryPath path = isKey ? QueryPath.KEY_PATH : QueryPath.VALUE_PATH; String name = isKey ? KEY : VALUE; String externalName = path.toString(); - MappingField userField = userFieldsByPath.get(path); + MappingField userField = fieldsByPath.get(path); if (userField != null && !userField.name().equals(name)) { throw QueryException.error("Cannot rename field: '" + name + '\''); } - if (userField != null && !type.getTypeFamily().equals(userField.type().getTypeFamily())) { + if (userField != null && type.getTypeFamily() != userField.type().getTypeFamily()) { throw QueryException.error("Mismatch between declared and resolved type for field '" + userField.name() + "'"); } - for (MappingField field : userFieldsByPath.values()) { + for (MappingField field : fieldsByPath.values()) { if (!externalName.equals(field.externalName())) { throw QueryException.error("The field '" + externalName + "' is of type " + type.getTypeFamily() + ", you can't map '" + field.externalName() + "' too"); } } - return userFieldsByPath.values().stream(); - } - - private Stream resolveObjectSchema( - boolean isKey, - List userFields, - Class clazz - ) { - return userFields.isEmpty() - ? resolveObjectFields(isKey, clazz) - : resolveAndValidateObjectFields(isKey, userFields, clazz); + return fieldsByPath.values().stream(); } - private Stream resolveObjectFields(boolean isKey, Class clazz) { - Map> fieldsInClass = FieldsUtil.resolveClass(clazz); - if (fieldsInClass.isEmpty()) { + private Stream resolveObjectFields(boolean isKey, Class typeClass) { + Map> classFields = FieldsUtil.resolveClass(typeClass); + if (classFields.isEmpty()) { // we didn't find any non-object fields in the class, map the whole value (e.g. in java.lang.Object) String name = isKey ? KEY : VALUE; return Stream.of(new MappingField(name, QueryDataType.OBJECT, name)); } - return fieldsInClass.entrySet().stream().map(classField -> { + return classFields.entrySet().stream().map(classField -> { QueryPath path = new QueryPath(classField.getKey(), isKey); QueryDataType type = QueryDataTypeUtils.resolveTypeForClass(classField.getValue()); String name = classField.getKey(); @@ -164,23 +145,22 @@ private Stream resolveObjectFields(boolean isKey, Class clazz) private Stream resolveAndValidateObjectFields( boolean isKey, - List userFields, - Class clazz + Map fieldsByPath, + Class typeClass ) { - Map userFieldsByPath = extractFields(userFields, isKey); - for (Entry> classField : FieldsUtil.resolveClass(clazz).entrySet()) { + for (Entry> classField : FieldsUtil.resolveClass(typeClass).entrySet()) { QueryPath path = new QueryPath(classField.getKey(), isKey); QueryDataType type = QueryDataTypeUtils.resolveTypeForClass(classField.getValue()); - MappingField userField = userFieldsByPath.get(path); - if (userField != null && !type.getTypeFamily().equals(userField.type().getTypeFamily())) { + MappingField userField = fieldsByPath.get(path); + if (userField != null && type.getTypeFamily() != userField.type().getTypeFamily()) { throw QueryException.error("Mismatch between declared and resolved type for field '" + userField.name() + "'. Declared: " + userField.type().getTypeFamily() + ", resolved: " + type.getTypeFamily()); } } - return userFieldsByPath.values().stream(); + return fieldsByPath.values().stream(); } @Override @@ -190,33 +170,21 @@ public KvMetadata resolveMetadata( Map options, InternalSerializationService serializationService ) { - Class clazz = loadClass(isKey, options); - return resolveMetadata(isKey, resolvedFields, clazz); - } - - public KvMetadata resolveMetadata( - boolean isKey, - List resolvedFields, - Class clazz - ) { - QueryDataType type = QueryDataTypeUtils.resolveTypeForClass(clazz); - Map fields = extractFields(resolvedFields, isKey); - - if (type.getTypeFamily().equals(QueryDataTypeFamily.OBJECT) && !resolvedFields.isEmpty()) { - final String topLevelFieldName = isKey ? KEY : VALUE; - final MappingField topLevelField = resolvedFields.stream() - .filter(field -> field.name().equals(topLevelFieldName)) - .findFirst() - .orElse(null); - if (topLevelField != null && topLevelField.type().isCustomType()) { - type = topLevelField.type(); - } - } - - if (!type.getTypeFamily().equals(QueryDataTypeFamily.OBJECT) || type.isCustomType()) { - return resolvePrimitiveMetadata(isKey, resolvedFields, fields, type); + Map fieldsByPath = extractFields(resolvedFields, isKey); + + Entry> entry = getTopLevelType(fieldsByPath) + .>>map(type -> entry(type, loadClass(type.getObjectTypeMetadata()))) + .orElseGet(() -> { + Class typeClass = loadClass(options, isKey); + return entry(QueryDataTypeUtils.resolveTypeForClass(typeClass), typeClass); + }); + QueryDataType type = entry.getKey(); + Class typeClass = entry.getValue(); + + if (type.getTypeFamily() != QueryDataTypeFamily.OBJECT || type.isCustomType()) { + return resolvePrimitiveMetadata(isKey, resolvedFields, fieldsByPath, type); } else { - return resolveObjectMetadata(isKey, resolvedFields, fields, clazz); + return resolveObjectMetadata(isKey, resolvedFields, fieldsByPath, typeClass); } } @@ -234,28 +202,22 @@ private KvMetadata resolvePrimitiveMetadata( } maybeAddDefaultField(isKey, resolvedFields, fields, type); - if (type.isCustomType()) { - return new KvMetadata( - fields, - GenericQueryTargetDescriptor.DEFAULT, - HazelcastObjectUpsertTargetDescriptor.INSTANCE - ); - } else { - return new KvMetadata( - fields, - GenericQueryTargetDescriptor.DEFAULT, - PrimitiveUpsertTargetDescriptor.INSTANCE - ); - } + return new KvMetadata( + fields, + GenericQueryTargetDescriptor.DEFAULT, + type.isCustomType() + ? HazelcastObjectUpsertTargetDescriptor.INSTANCE + : PrimitiveUpsertTargetDescriptor.INSTANCE + ); } private KvMetadata resolveObjectMetadata( boolean isKey, List resolvedFields, Map fieldsByPath, - Class clazz + Class typeClass ) { - Map> typesByNames = FieldsUtil.resolveClass(clazz); + Map> classFields = FieldsUtil.resolveClass(typeClass); List fields = new ArrayList<>(); Map typeNamesByPaths = new HashMap<>(); @@ -265,8 +227,8 @@ private KvMetadata resolveObjectMetadata( String name = entry.getValue().name(); fields.add(new MapTableField(name, type, false, path)); - if (path.getPath() != null && typesByNames.get(path.getPath()) != null) { - typeNamesByPaths.put(path.getPath(), typesByNames.get(path.getPath()).getName()); + if (path.getPath() != null && classFields.get(path.getPath()) != null) { + typeNamesByPaths.put(path.getPath(), classFields.get(path.getPath()).getName()); } } maybeAddDefaultField(isKey, resolvedFields, fields, QueryDataType.OBJECT); @@ -274,11 +236,11 @@ private KvMetadata resolveObjectMetadata( return new KvMetadata( fields, GenericQueryTargetDescriptor.DEFAULT, - new PojoUpsertTargetDescriptor(clazz.getName(), typeNamesByPaths) + new PojoUpsertTargetDescriptor(typeClass.getName(), typeNamesByPaths) ); } - private Class loadClass(boolean isKey, Map options) { + public static Class loadClass(Map options, boolean isKey) { String formatProperty = options.get(isKey ? OPTION_KEY_FORMAT : OPTION_VALUE_FORMAT); String classNameProperty = isKey ? OPTION_KEY_CLASS : OPTION_VALUE_CLASS; @@ -287,13 +249,16 @@ private Class loadClass(boolean isKey, Map options) { : JavaClassNameResolver.resolveClassName(formatProperty); if (className == null) { - throw QueryException.error("Unable to resolve table metadata. Missing '" + classNameProperty + "' option"); + throw QueryException.error(classNameProperty + " is required to create Java-based mapping"); } + return loadClass(className); + } + public static Class loadClass(String className) { try { return ReflectionUtils.loadClass(className); } catch (Exception e) { - throw QueryException.error("Unable to load class: '" + className + "'", e); + throw QueryException.error("Unable to load class '" + className + "'", e); } } } diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/keyvalue/KvMetadataJsonResolver.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/keyvalue/KvMetadataJsonResolver.java index 4d03235ee45e..9aa8111014b1 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/keyvalue/KvMetadataJsonResolver.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/keyvalue/KvMetadataJsonResolver.java @@ -62,8 +62,8 @@ public Stream resolveAndValidateFields( return extractFields(userFields, isKey).entrySet().stream() .map(entry -> { QueryPath path = entry.getKey(); - if (path.getPath() == null) { - throw QueryException.error("Cannot use the '" + path + "' field with JSON serialization"); + if (path.isTopLevel()) { + throw QueryException.error("Cannot use '" + path + "' field with JSON serialization"); } return entry.getValue(); }); diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/keyvalue/KvMetadataResolver.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/keyvalue/KvMetadataResolver.java index a805d0dd243c..4db27535e52a 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/keyvalue/KvMetadataResolver.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/keyvalue/KvMetadataResolver.java @@ -23,11 +23,14 @@ import com.hazelcast.sql.impl.schema.TableField; import com.hazelcast.sql.impl.schema.map.MapTableField; import com.hazelcast.sql.impl.type.QueryDataType; +import com.hazelcast.sql.impl.type.QueryDataType.QueryDataTypeField; import javax.annotation.Nonnull; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Map.Entry; +import java.util.Optional; import java.util.stream.Stream; import static com.hazelcast.sql.impl.extract.QueryPath.KEY; @@ -35,7 +38,7 @@ /** * Interface for key-value resolution of fields for a particular - * serialization types. + * serialization type. */ public interface KvMetadataResolver { @@ -81,4 +84,65 @@ static void maybeAddDefaultField( tableFields.add(new MapTableField(fieldName, type, true, QueryPath.create(fieldName))); } } + + /** + * If {@code __key}/{@code this} is the only key/value field and has a custom type, + * return type fields. Otherwise, return mapping fields without {@code __key} or {@code this}. + */ + static Stream getFields(Map fields) { + return getTopLevelType(fields) + .map(type -> type.getObjectFields().stream().map(Field::new)) + .orElseGet(() -> fields.entrySet().stream().filter(e -> !e.getKey().isTopLevel()).map(Field::new)); + } + + /** + * If {@code __key}/{@code this} is the only key/value field and has a custom type, + * return its metadata. + */ + static Optional getMetadata(Map fields) { + return getTopLevelType(fields).map(QueryDataType::getObjectTypeMetadata); + } + + /** + * If {@code __key}/{@code this} is the only key/value field and has a custom type, + * return its type. + */ + static Optional getTopLevelType(Map fields) { + if (fields.size() == 1) { + Entry entry = fields.entrySet().iterator().next(); + if (entry.getKey().isTopLevel() && entry.getValue().type().isCustomType()) { + return Optional.of(entry.getValue().type()); + } + } + return Optional.empty(); + } + + class Field { + private final String name; + private final QueryDataType type; + + public Field(Entry entry) { + name = entry.getKey().getPath(); + type = entry.getValue().type(); + } + + public Field(TableField field) { + name = field instanceof MapTableField + ? ((MapTableField) field).getPath().getPath() : field.getName(); + type = field.getType(); + } + + public Field(QueryDataTypeField field) { + name = field.getName(); + type = field.getType(); + } + + public String name() { + return name; + } + + public QueryDataType type() { + return type; + } + } } diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/keyvalue/KvMetadataResolvers.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/keyvalue/KvMetadataResolvers.java index 1bf3900921ac..bd58ace6c6c8 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/keyvalue/KvMetadataResolvers.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/keyvalue/KvMetadataResolvers.java @@ -16,17 +16,16 @@ package com.hazelcast.jet.sql.impl.connector.keyvalue; -import com.google.common.collect.ImmutableSet; import com.hazelcast.internal.serialization.InternalSerializationService; import com.hazelcast.jet.sql.impl.connector.SqlConnector; import com.hazelcast.jet.sql.impl.schema.RelationsStorage; -import com.hazelcast.jet.sql.impl.schema.TypesUtils; +import com.hazelcast.jet.sql.impl.schema.TypeUtils.FieldEnricher; import com.hazelcast.spi.impl.NodeEngine; import com.hazelcast.sql.impl.QueryException; import com.hazelcast.sql.impl.SqlServiceImpl; import com.hazelcast.sql.impl.schema.MappingField; -import com.hazelcast.sql.impl.schema.type.TypeKind; +import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Arrays; import java.util.LinkedHashMap; @@ -39,15 +38,14 @@ import static com.hazelcast.jet.Util.entry; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_KEY_FORMAT; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_FORMAT; +import static com.hazelcast.jet.sql.impl.connector.keyvalue.KvMetadataResolver.extractFields; +import static com.hazelcast.jet.sql.impl.schema.TypeUtils.getFieldEnricher; import static com.hazelcast.sql.impl.extract.QueryPath.KEY; import static com.hazelcast.sql.impl.extract.QueryPath.VALUE; import static com.hazelcast.sql.impl.extract.QueryPath.VALUE_PREFIX; -import static com.hazelcast.sql.impl.schema.type.TypeKind.COMPACT; -import static com.hazelcast.sql.impl.schema.type.TypeKind.JAVA; -import static com.hazelcast.sql.impl.schema.type.TypeKind.PORTABLE; import static java.util.Objects.requireNonNull; +import static java.util.stream.Collectors.toList; import static java.util.stream.Collectors.toMap; -import static java.util.stream.Stream.concat; /** * A utility to resolve fields for key-value connectors that support @@ -57,7 +55,9 @@ public class KvMetadataResolvers { // A string of characters (excluding a `.`), optionally prefixed with "__key." or "this." private static final Pattern EXT_NAME_PATTERN = Pattern.compile("((" + KEY + "|" + VALUE + ")\\.)?[^.]+"); - private static final Set NESTED_FIELDS_SUPPORTED_FORMATS = ImmutableSet.of(JAVA, PORTABLE, COMPACT); + private static final Set NESTED_FIELDS_SUPPORTED_FORMATS = Set.of( + SqlConnector.JAVA_FORMAT, SqlConnector.PORTABLE_FORMAT, + SqlConnector.COMPACT_FORMAT, SqlConnector.AVRO_FORMAT); private final Map keyResolvers; private final Map valueResolvers; @@ -81,15 +81,15 @@ private Map resolversMap(KvMetadataResolver[] resolv } /** - * A utility to implement {@link SqlConnector#resolveAndValidateFields} in - * the connector. + * A utility to implement {@link SqlConnector#resolveAndValidateFields} in the connector. */ public List resolveAndValidateFields( List userFields, Map options, NodeEngine nodeEngine ) { - final InternalSerializationService ss = (InternalSerializationService) nodeEngine.getSerializationService(); + final InternalSerializationService serializationService = (InternalSerializationService) nodeEngine + .getSerializationService(); final RelationsStorage relationsStorage = ((SqlServiceImpl) nodeEngine.getSqlService()).getOptimizer() .relationsStorage(); // normalize and validate the names and external names @@ -116,38 +116,45 @@ public List resolveAndValidateFields( } } - Stream keyFields = findMetadataResolver(options, true) - .resolveAndValidateFields(true, userFields, options, ss) - .filter(field -> !field.name().equals(KEY) || field.externalName().equals(KEY)); - Stream valueFields = findMetadataResolver(options, false) - .resolveAndValidateFields(false, userFields, options, ss) - .filter(field -> !field.name().equals(VALUE) || field.externalName().equals(VALUE)); - - final TypeKind keyKind = TypesUtils.formatToTypeKind(getFormat(options, true)); - if (NESTED_FIELDS_SUPPORTED_FORMATS.contains(keyKind)) { - keyFields = keyFields - .peek(mappingField -> TypesUtils.enrichMappingFieldType(keyKind, mappingField, relationsStorage)); - } - - final TypeKind valueKind = TypesUtils.formatToTypeKind(getFormat(options, false)); - if (NESTED_FIELDS_SUPPORTED_FORMATS.contains(valueKind)) { - valueFields = valueFields - .peek(mappingField -> TypesUtils.enrichMappingFieldType(valueKind, mappingField, relationsStorage)); - } + Stream keyFields = resolveAndValidateFields(true, userFields, options, + serializationService, relationsStorage); + Stream valueFields = resolveAndValidateFields(false, userFields, options, + serializationService, relationsStorage); - Map fields = concat(keyFields, valueFields) + Map fields = Stream.concat(keyFields, valueFields) .collect(LinkedHashMap::new, (map, field) -> map.putIfAbsent(field.name(), field), Map::putAll); if (fields.isEmpty()) { throw QueryException.error("The resolved field list is empty"); } - return new ArrayList<>(fields.values()); } + private Stream resolveAndValidateFields( + boolean isKey, + List userFields, + Map options, + InternalSerializationService serializationService, + RelationsStorage relationsStorage + ) { + String format = getFormat(options, isKey); + if (format != null && NESTED_FIELDS_SUPPORTED_FORMATS.contains(format)) { + List fieldsWithCustomTypes = extractFields(userFields, isKey).values().stream() + .filter(mappingField -> mappingField.type().isCustomType()).collect(toList()); + if (!fieldsWithCustomTypes.isEmpty()) { + FieldEnricher enricher = getFieldEnricher(format, serializationService, relationsStorage); + fieldsWithCustomTypes.forEach(mappingField -> enricher.enrich(mappingField, options, isKey)); + } + } + + String name = isKey ? KEY : VALUE; + return findMetadataResolver(options, isKey) + .resolveAndValidateFields(isKey, userFields, options, serializationService) + .filter(field -> !field.name().equals(name) || field.externalName().equals(name)); + } + /** - * A utility to implement {@link SqlConnector#createTable} in the - * connector. + * A utility to implement {@link SqlConnector#createTable} in the connector. */ public KvMetadata resolveMetadata( boolean isKey, @@ -156,7 +163,6 @@ public KvMetadata resolveMetadata( InternalSerializationService serializationService ) { KvMetadataResolver resolver = findMetadataResolver(options, isKey); - // TODO: enhance types return requireNonNull(resolver.resolveMetadata(isKey, resolvedFields, options, serializationService)); } @@ -173,8 +179,12 @@ private KvMetadataResolver findMetadataResolver(Map options, boo return resolver; } - private String getFormat(Map options, boolean isKey) { - String option = isKey ? OPTION_KEY_FORMAT : OPTION_VALUE_FORMAT; - return options.get(option); + /** + * Null format is only valid for Kafka keys. + * @see KvMetadataNullResolver + */ + @Nullable + private static String getFormat(Map options, boolean isKey) { + return options.get(isKey ? OPTION_KEY_FORMAT : OPTION_VALUE_FORMAT); } } diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/map/Joiner.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/map/Joiner.java index 89bb8ba63614..14909fe953b4 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/map/Joiner.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/map/Joiner.java @@ -113,7 +113,7 @@ private static int leftEquiJoinPrimitiveKeyIndex(JetJoinInfo joinInfo, QueryPath int[] rightEquiJoinIndices = joinInfo.rightEquiJoinIndices(); for (int i = 0; i < rightEquiJoinIndices.length; i++) { QueryPath path = rightPaths[rightEquiJoinIndices[i]]; - if (path.isTop() && path.isKey()) { + if (path.isTopLevel() && path.isKey()) { return joinInfo.leftEquiJoinIndices()[i]; } } diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/map/MetadataCompactResolver.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/map/MetadataCompactResolver.java index c0f4c2475d8e..28fdc59e9c08 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/map/MetadataCompactResolver.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/map/MetadataCompactResolver.java @@ -16,10 +16,13 @@ package com.hazelcast.jet.sql.impl.connector.map; +import com.google.common.collect.ImmutableMap; import com.hazelcast.internal.serialization.InternalSerializationService; import com.hazelcast.internal.serialization.impl.compact.FieldDescriptor; import com.hazelcast.internal.serialization.impl.compact.Schema; import com.hazelcast.internal.serialization.impl.compact.SchemaWriter; +import com.hazelcast.internal.util.collection.DefaultedMap; +import com.hazelcast.internal.util.collection.DefaultedMap.DefaultedMapBuilder; import com.hazelcast.jet.sql.impl.connector.keyvalue.KvMetadata; import com.hazelcast.jet.sql.impl.connector.keyvalue.KvMetadataResolver; import com.hazelcast.jet.sql.impl.inject.CompactUpsertTargetDescriptor; @@ -34,23 +37,44 @@ import com.hazelcast.sql.impl.type.QueryDataTypeFamily; import java.util.ArrayList; +import java.util.EnumMap; import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.stream.Stream; +import static com.hazelcast.jet.impl.util.Util.collect; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.COMPACT_FORMAT; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_KEY_COMPACT_TYPE_NAME; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_COMPACT_TYPE_NAME; import static com.hazelcast.jet.sql.impl.connector.keyvalue.KvMetadataResolver.extractFields; +import static com.hazelcast.jet.sql.impl.connector.keyvalue.KvMetadataResolver.getFields; +import static com.hazelcast.jet.sql.impl.connector.keyvalue.KvMetadataResolver.getMetadata; import static com.hazelcast.jet.sql.impl.connector.keyvalue.KvMetadataResolver.maybeAddDefaultField; -final class MetadataCompactResolver implements KvMetadataResolver { - +public final class MetadataCompactResolver implements KvMetadataResolver { static final MetadataCompactResolver INSTANCE = new MetadataCompactResolver(); - private MetadataCompactResolver() { - } + private static final DefaultedMap SQL_TO_COMPACT = new DefaultedMapBuilder<>( + new EnumMap<>(ImmutableMap.builder() + .put(QueryDataTypeFamily.BOOLEAN, FieldKind.NULLABLE_BOOLEAN) + .put(QueryDataTypeFamily.TINYINT, FieldKind.NULLABLE_INT8) + .put(QueryDataTypeFamily.SMALLINT, FieldKind.NULLABLE_INT16) + .put(QueryDataTypeFamily.INTEGER, FieldKind.NULLABLE_INT32) + .put(QueryDataTypeFamily.BIGINT, FieldKind.NULLABLE_INT64) + .put(QueryDataTypeFamily.REAL, FieldKind.NULLABLE_FLOAT32) + .put(QueryDataTypeFamily.DOUBLE, FieldKind.NULLABLE_FLOAT64) + .put(QueryDataTypeFamily.DECIMAL, FieldKind.DECIMAL) + .put(QueryDataTypeFamily.VARCHAR, FieldKind.STRING) + .put(QueryDataTypeFamily.TIME, FieldKind.TIME) + .put(QueryDataTypeFamily.DATE, FieldKind.DATE) + .put(QueryDataTypeFamily.TIMESTAMP, FieldKind.TIMESTAMP) + .put(QueryDataTypeFamily.TIMESTAMP_WITH_TIME_ZONE, FieldKind.TIMESTAMP_WITH_TIMEZONE) + .put(QueryDataTypeFamily.OBJECT, FieldKind.COMPACT) + .build())) + .orElseThrow(type -> new IllegalArgumentException("Compact format does not allow " + type + " data type")); + + private MetadataCompactResolver() { } @Override public Stream supportedFormats() { @@ -67,19 +91,15 @@ public Stream resolveAndValidateFields( if (userFields.isEmpty()) { throw QueryException.error("Column list is required for Compact format"); } + Map fieldsByPath = extractFields(userFields, isKey); - String typeNameProperty = isKey ? OPTION_KEY_COMPACT_TYPE_NAME : OPTION_VALUE_COMPACT_TYPE_NAME; - String typeName = options.get(typeNameProperty); + // Check if the compact type name is specified + getCompactTypeName(fieldsByPath, options, isKey); - if (typeName == null) { - throw QueryException.error("Unable to resolve table metadata. Missing '" + typeNameProperty + "' option"); - } - - Map fields = extractFields(userFields, isKey); - return fields.entrySet().stream() + return fieldsByPath.entrySet().stream() .map(entry -> { QueryPath path = entry.getKey(); - if (path.getPath() == null) { + if (path.isTopLevel()) { throw QueryException.error("Cannot use the '" + path + "' field with Compact serialization"); } QueryDataType type = entry.getValue().type(); @@ -99,8 +119,7 @@ public KvMetadata resolveMetadata( ) { Map fieldsByPath = extractFields(resolvedFields, isKey); - String typeNameProperty = isKey ? OPTION_KEY_COMPACT_TYPE_NAME : OPTION_VALUE_COMPACT_TYPE_NAME; - String typeName = options.get(typeNameProperty); + String typeName = getCompactTypeName(fieldsByPath, options, isKey); List fields = new ArrayList<>(fieldsByPath.size()); for (Entry entry : fieldsByPath.entrySet()) { @@ -112,7 +131,7 @@ public KvMetadata resolveMetadata( } maybeAddDefaultField(isKey, resolvedFields, fields, QueryDataType.OBJECT); - Schema schema = resolveSchema(typeName, fieldsByPath); + Schema schema = resolveSchema(typeName, getFields(fieldsByPath)); return new KvMetadata( fields, @@ -121,49 +140,27 @@ public KvMetadata resolveMetadata( ); } - private Schema resolveSchema(String typeName, Map fields) { - SchemaWriter schemaWriter = new SchemaWriter(typeName); - for (Entry entry : fields.entrySet()) { - String name = entry.getKey().getPath(); - QueryDataType type = entry.getValue().type(); - schemaWriter.addField(new FieldDescriptor(name, resolveToCompactKind(type.getTypeFamily()))); - } - return schemaWriter.build(); + private static Schema resolveSchema(String typeName, Stream fields) { + return collect(new SchemaWriter(typeName), fields, (schema, field) -> + schema.addField(new FieldDescriptor(field.name(), + SQL_TO_COMPACT.getOrDefault(field.type().getTypeFamily()))) + ).build(); } - @SuppressWarnings("checkstyle:ReturnCount") - private static FieldKind resolveToCompactKind(QueryDataTypeFamily type) { - switch (type) { - case BOOLEAN: - return FieldKind.NULLABLE_BOOLEAN; - case TINYINT: - return FieldKind.NULLABLE_INT8; - case SMALLINT: - return FieldKind.NULLABLE_INT16; - case INTEGER: - return FieldKind.NULLABLE_INT32; - case BIGINT: - return FieldKind.NULLABLE_INT64; - case REAL: - return FieldKind.NULLABLE_FLOAT32; - case DOUBLE: - return FieldKind.NULLABLE_FLOAT64; - case DECIMAL: - return FieldKind.DECIMAL; - case VARCHAR: - return FieldKind.STRING; - case TIME: - return FieldKind.TIME; - case DATE: - return FieldKind.DATE; - case TIMESTAMP: - return FieldKind.TIMESTAMP; - case TIMESTAMP_WITH_TIME_ZONE: - return FieldKind.TIMESTAMP_WITH_TIMEZONE; - case OBJECT: - return FieldKind.COMPACT; - default: - throw new IllegalArgumentException("Compact format does not allow " + type + " data type"); + private static String getCompactTypeName( + Map fields, + Map options, + boolean isKey + ) { + return getMetadata(fields).orElseGet(() -> compactTypeName(options, isKey)); + } + + public static String compactTypeName(Map options, boolean isKey) { + String typeNameProperty = isKey ? OPTION_KEY_COMPACT_TYPE_NAME : OPTION_VALUE_COMPACT_TYPE_NAME; + String typeName = options.get(typeNameProperty); + if (typeName == null) { + throw QueryException.error(typeNameProperty + " is required to create Compact-based mapping"); } + return typeName; } } diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/map/MetadataJsonResolver.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/map/MetadataJsonResolver.java index da71f28c4b83..f12c3b0eeed5 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/map/MetadataJsonResolver.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/map/MetadataJsonResolver.java @@ -64,8 +64,8 @@ public Stream resolveAndValidateFields( return extractFields(userFields, isKey).entrySet().stream() .map(entry -> { QueryPath path = entry.getKey(); - if (path.getPath() == null) { - throw QueryException.error("Cannot use the '" + path + "' field with JSON serialization"); + if (path.isTopLevel()) { + throw QueryException.error("Cannot use '" + path + "' field with JSON serialization"); } return entry.getValue(); }); diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/map/MetadataPortableResolver.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/map/MetadataPortableResolver.java index 189fc53a6e0b..bb30d7862f44 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/map/MetadataPortableResolver.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/map/MetadataPortableResolver.java @@ -16,14 +16,17 @@ package com.hazelcast.jet.sql.impl.connector.map; +import com.google.common.collect.ImmutableMap; import com.hazelcast.internal.serialization.InternalSerializationService; -import com.hazelcast.jet.datamodel.Tuple3; +import com.hazelcast.internal.serialization.impl.portable.PortableContext; +import com.hazelcast.internal.util.collection.DefaultedMap; import com.hazelcast.jet.sql.impl.connector.keyvalue.KvMetadata; import com.hazelcast.jet.sql.impl.connector.keyvalue.KvMetadataResolver; import com.hazelcast.jet.sql.impl.inject.PortableUpsertTargetDescriptor; -import com.hazelcast.jet.sql.impl.schema.TypesUtils; import com.hazelcast.nio.serialization.ClassDefinition; import com.hazelcast.nio.serialization.ClassDefinitionBuilder; +import com.hazelcast.nio.serialization.FieldType; +import com.hazelcast.nio.serialization.PortableId; import com.hazelcast.sql.impl.QueryException; import com.hazelcast.sql.impl.extract.GenericQueryTargetDescriptor; import com.hazelcast.sql.impl.extract.QueryPath; @@ -33,15 +36,16 @@ import com.hazelcast.sql.impl.type.QueryDataType; import com.hazelcast.sql.impl.type.QueryDataTypeFamily; -import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.util.ArrayList; -import java.util.Collection; +import java.util.EnumMap; import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.Optional; import java.util.stream.Stream; +import static com.hazelcast.jet.impl.util.Util.reduce; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_KEY_CLASS_ID; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_KEY_CLASS_VERSION; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_KEY_FACTORY_ID; @@ -49,18 +53,37 @@ import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_CLASS_VERSION; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_FACTORY_ID; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.PORTABLE_FORMAT; -import static com.hazelcast.jet.sql.impl.connector.SqlConnectorUtil.asInt; import static com.hazelcast.jet.sql.impl.connector.keyvalue.KvMetadataResolver.extractFields; +import static com.hazelcast.jet.sql.impl.connector.keyvalue.KvMetadataResolver.getFields; +import static com.hazelcast.jet.sql.impl.connector.keyvalue.KvMetadataResolver.getMetadata; import static com.hazelcast.jet.sql.impl.connector.keyvalue.KvMetadataResolver.maybeAddDefaultField; import static com.hazelcast.sql.impl.extract.QueryPath.KEY; import static com.hazelcast.sql.impl.extract.QueryPath.VALUE; -final class MetadataPortableResolver implements KvMetadataResolver { +public final class MetadataPortableResolver implements KvMetadataResolver { + public static final DefaultedMap PORTABLE_TO_SQL = new DefaultedMap<>( + new EnumMap<>(ImmutableMap.builder() + .put(FieldType.BOOLEAN, QueryDataType.BOOLEAN) + .put(FieldType.BYTE, QueryDataType.TINYINT) + .put(FieldType.SHORT, QueryDataType.SMALLINT) + .put(FieldType.INT, QueryDataType.INT) + .put(FieldType.LONG, QueryDataType.BIGINT) + .put(FieldType.FLOAT, QueryDataType.REAL) + .put(FieldType.DOUBLE, QueryDataType.DOUBLE) + .put(FieldType.DECIMAL, QueryDataType.DECIMAL) + .put(FieldType.CHAR, QueryDataType.VARCHAR_CHARACTER) + .put(FieldType.UTF, QueryDataType.VARCHAR) + .put(FieldType.TIME, QueryDataType.TIME) + .put(FieldType.DATE, QueryDataType.DATE) + .put(FieldType.TIMESTAMP, QueryDataType.TIMESTAMP) + .put(FieldType.TIMESTAMP_WITH_TIMEZONE, QueryDataType.TIMESTAMP_WITH_TZ_OFFSET_DATE_TIME) + .put(FieldType.PORTABLE, QueryDataType.OBJECT) + .build()), + QueryDataType.OBJECT); static final MetadataPortableResolver INSTANCE = new MetadataPortableResolver(); - private MetadataPortableResolver() { - } + private MetadataPortableResolver() { } @Override public Stream supportedFormats() { @@ -74,29 +97,28 @@ public Stream resolveAndValidateFields( Map options, InternalSerializationService serializationService ) { - Map userFieldsByPath = extractFields(userFields, isKey); - ClassDefinition classDefinition = findClassDefinition(isKey, options, serializationService); + Map fieldsByPath = extractFields(userFields, isKey); + + PortableId portableId = getPortableId(fieldsByPath, options, isKey); + ClassDefinition classDefinition = serializationService.getPortableContext() + .lookupClassDefinition(portableId); return userFields.isEmpty() - ? resolveFields(isKey, classDefinition, serializationService) - : resolveAndValidateFields(isKey, userFieldsByPath, classDefinition); + ? resolveFields(isKey, classDefinition) + : resolveAndValidateFields(isKey, fieldsByPath, classDefinition); } - Stream resolveFields( - boolean isKey, - @Nullable ClassDefinition clazz, - InternalSerializationService ss - ) { - if (clazz == null || clazz.getFieldCount() == 0) { + private static Stream resolveFields(boolean isKey, ClassDefinition classDefinition) { + if (classDefinition == null || classDefinition.getFieldCount() == 0) { // ClassDefinition does not exist, or it is empty, map the whole value String name = isKey ? KEY : VALUE; return Stream.of(new MappingField(name, QueryDataType.OBJECT, name)); } - return clazz.getFieldNames().stream() + return classDefinition.getFieldNames().stream() .map(name -> { QueryPath path = new QueryPath(name, isKey); - QueryDataType type = TypesUtils.resolvePortableFieldType(clazz.getFieldType(name)); + QueryDataType type = PORTABLE_TO_SQL.getOrDefault(classDefinition.getFieldType(name)); return new MappingField(name, type, path.toString()); }); @@ -104,12 +126,12 @@ Stream resolveFields( private static Stream resolveAndValidateFields( boolean isKey, - Map userFieldsByPath, - @Nullable ClassDefinition clazz + Map fieldsByPath, + @Nullable ClassDefinition classDefinition ) { - if (clazz == null) { + if (classDefinition == null) { // ClassDefinition does not exist, make sure there are no OBJECT fields - return userFieldsByPath.values().stream() + return fieldsByPath.values().stream() .peek(mappingField -> { QueryDataType type = mappingField.type(); if (type.getTypeFamily().equals(QueryDataTypeFamily.OBJECT)) { @@ -118,16 +140,16 @@ private static Stream resolveAndValidateFields( }); } - for (String name : clazz.getFieldNames()) { + for (String name : classDefinition.getFieldNames()) { final QueryPath path = new QueryPath(name, isKey); - final QueryDataType type = TypesUtils.resolvePortableFieldType(clazz.getFieldType(name)); + final QueryDataType type = PORTABLE_TO_SQL.getOrDefault(classDefinition.getFieldType(name)); - MappingField userField = userFieldsByPath.get(path); + MappingField userField = fieldsByPath.get(path); if (userField != null && !type.getTypeFamily().equals(userField.type().getTypeFamily())) { throw QueryException.error("Mismatch between declared and resolved type: " + userField.name()); } } - return userFieldsByPath.values().stream(); + return fieldsByPath.values().stream(); } @Override @@ -138,17 +160,11 @@ public KvMetadata resolveMetadata( InternalSerializationService serializationService ) { Map fieldsByPath = extractFields(resolvedFields, isKey); - ClassDefinition clazz = resolveClassDefinition(isKey, options, fieldsByPath.values(), serializationService); - return resolveMetadata(isKey, resolvedFields, fieldsByPath, clazz); - } + PortableId portableId = getPortableId(fieldsByPath, options, isKey); + ClassDefinition classDefinition = resolveClassDefinition(portableId, getFields(fieldsByPath), + serializationService.getPortableContext()); - private static KvMetadata resolveMetadata( - boolean isKey, - List resolvedFields, - Map fieldsByPath, - @Nonnull ClassDefinition clazz - ) { List fields = new ArrayList<>(); for (Entry entry : fieldsByPath.entrySet()) { QueryPath path = entry.getKey(); @@ -162,98 +178,90 @@ private static KvMetadata resolveMetadata( return new KvMetadata( fields, GenericQueryTargetDescriptor.DEFAULT, - new PortableUpsertTargetDescriptor(clazz) + new PortableUpsertTargetDescriptor(classDefinition) ); } - @Nullable - private static ClassDefinition findClassDefinition( - boolean isKey, - Map options, - InternalSerializationService serializationService - ) { - Tuple3 settings = settings(isKey, options); - //noinspection ConstantConditions - return serializationService - .getPortableContext() - .lookupClassDefinition(settings.f0(), settings.f1(), settings.f2()); - } - - @Nonnull + @SuppressWarnings("ReturnCount") private static ClassDefinition resolveClassDefinition( - boolean isKey, - Map options, - Collection fields, - InternalSerializationService serializationService + PortableId portableId, + Stream fields, + PortableContext context ) { - Tuple3 settings = settings(isKey, options); - //noinspection ConstantConditions - ClassDefinition classDefinition = serializationService - .getPortableContext() - .lookupClassDefinition(settings.f0(), settings.f1(), settings.f2()); + ClassDefinition classDefinition = context.lookupClassDefinition(portableId); if (classDefinition != null) { return classDefinition; } - ClassDefinitionBuilder classDefinitionBuilder = new ClassDefinitionBuilder(settings.f0(), settings.f1(), settings.f2()); - for (MappingField field : fields) { - String name = field.name(); - QueryDataType type = field.type(); - switch (type.getTypeFamily()) { + return reduce(new ClassDefinitionBuilder(portableId), fields, (schema, field) -> { + switch (field.type().getTypeFamily()) { case BOOLEAN: - classDefinitionBuilder.addBooleanField(name); - break; + return schema.addBooleanField(field.name()); case TINYINT: - classDefinitionBuilder.addByteField(name); - break; + return schema.addByteField(field.name()); case SMALLINT: - classDefinitionBuilder.addShortField(name); - break; + return schema.addShortField(field.name()); case INTEGER: - classDefinitionBuilder.addIntField(name); - break; + return schema.addIntField(field.name()); case BIGINT: - classDefinitionBuilder.addLongField(name); - break; + return schema.addLongField(field.name()); case REAL: - classDefinitionBuilder.addFloatField(name); - break; + return schema.addFloatField(field.name()); case DOUBLE: - classDefinitionBuilder.addDoubleField(name); - break; + return schema.addDoubleField(field.name()); case DECIMAL: - classDefinitionBuilder.addDecimalField(name); - break; + return schema.addDecimalField(field.name()); case VARCHAR: - classDefinitionBuilder.addStringField(name); - break; + return schema.addStringField(field.name()); case TIME: - classDefinitionBuilder.addTimeField(name); - break; + return schema.addTimeField(field.name()); case DATE: - classDefinitionBuilder.addDateField(name); - break; + return schema.addDateField(field.name()); case TIMESTAMP: - classDefinitionBuilder.addTimestampField(name); - break; + return schema.addTimestampField(field.name()); case TIMESTAMP_WITH_TIME_ZONE: - classDefinitionBuilder.addTimestampWithTimezoneField(name); - break; + return schema.addTimestampWithTimezoneField(field.name()); default: // validated earlier, skip whole __key & this + return schema; } - } - return classDefinitionBuilder.build(); + }).build(); + } + + private static PortableId getPortableId( + Map fields, + Map options, + boolean isKey + ) { + return getMetadata(fields) + .map(PortableId::new) + .orElseGet(() -> portableId(options, isKey)); } - private static Tuple3 settings(boolean isKey, Map options) { + public static PortableId portableId(Map options, boolean isKey) { String factoryIdProperty = isKey ? OPTION_KEY_FACTORY_ID : OPTION_VALUE_FACTORY_ID; String classIdProperty = isKey ? OPTION_KEY_CLASS_ID : OPTION_VALUE_CLASS_ID; - String classVersionProperty = isKey ? OPTION_KEY_CLASS_VERSION : OPTION_VALUE_CLASS_VERSION; - return Tuple3.tuple3( - asInt(options, factoryIdProperty, null), - asInt(options, classIdProperty, null), - asInt(options, classVersionProperty, 0) - ); + String versionProperty = isKey ? OPTION_KEY_CLASS_VERSION : OPTION_VALUE_CLASS_VERSION; + + PortableId portableId = portableId(options, factoryIdProperty, classIdProperty, versionProperty); + if (portableId == null) { + throw QueryException.error(String.format("%s Portable ID (%s, %s and optional %s)" + + " is required to create Portable-based mapping", isKey ? "Key" : "Value", + factoryIdProperty, classIdProperty, versionProperty)); + } + return portableId; + } + + public static PortableId portableId( + Map options, + String factoryIdProperty, + String classIdProperty, + String versionProperty + ) { + Integer factoryId = Optional.ofNullable(options.get(factoryIdProperty)).map(Integer::parseInt).orElse(null); + Integer classId = Optional.ofNullable(options.get(classIdProperty)).map(Integer::parseInt).orElse(null); + int version = Optional.ofNullable(options.get(versionProperty)).map(Integer::parseInt).orElse(0); + + return factoryId != null && classId != null ? new PortableId(factoryId, classId, version) : null; } } diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/map/QueryUtil.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/map/QueryUtil.java index 617202123a1a..ea58436123eb 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/map/QueryUtil.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/connector/map/QueryUtil.java @@ -78,11 +78,11 @@ static Predicate toPredicate( EntryObject object; if (rightPath.isKey()) { - object = rightPath.isTop() + object = rightPath.isTopLevel() ? entryObject.key() : entryObject.key().get(rightPath.getPath()); } else { - object = rightPath.isTop() + object = rightPath.isTopLevel() ? entryObject.get(rightPath.toString()) : entryObject.get(QueryPath.VALUE).get(rightPath.getPath()); } diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/expression/UdtObjectToJsonFunction.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/expression/UdtObjectToJsonFunction.java index b3fc9042151b..498df3eee1a8 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/expression/UdtObjectToJsonFunction.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/expression/UdtObjectToJsonFunction.java @@ -94,12 +94,12 @@ private void convert( for (final QueryDataType.QueryDataTypeField field : dataType.getObjectFields()) { final Object fieldValue = extractors.extract(source, field.getName(), false); - if (!field.getDataType().isCustomType() || fieldValue == null) { + if (!field.getType().isCustomType() || fieldValue == null) { values.put(field.getName(), fieldValue); } else { final Map subFieldValue = new HashMap<>(); values.put(field.getName(), subFieldValue); - convert(fieldValue, subFieldValue, field.getDataType(), seenObjects, extractors); + convert(fieldValue, subFieldValue, field.getType(), seenObjects, extractors); } } } diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/extract/AvroQueryTarget.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/extract/AvroQueryTarget.java index 5d7242836845..c75dbde66a43 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/extract/AvroQueryTarget.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/extract/AvroQueryTarget.java @@ -27,7 +27,6 @@ @NotThreadSafe public class AvroQueryTarget implements QueryTarget { - private GenericRecord record; @Override @@ -49,7 +48,7 @@ private QueryExtractor createFieldExtractor(String path, QueryDataType type) { return () -> type.convert(extractValue(record, path)); } - private static Object extractValue(GenericRecord record, String path) { + public static Object extractValue(GenericRecord record, String path) { if (!record.hasField(path)) { return null; } diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/inject/AvroUpsertTarget.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/inject/AvroUpsertTarget.java index 00bedf41b5b1..0f08e23439d5 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/inject/AvroUpsertTarget.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/inject/AvroUpsertTarget.java @@ -19,6 +19,7 @@ import com.google.common.collect.ImmutableMap; import com.hazelcast.internal.util.collection.DefaultedMap; import com.hazelcast.sql.impl.QueryException; +import com.hazelcast.sql.impl.expression.RowValue; import com.hazelcast.sql.impl.type.QueryDataType; import org.apache.avro.Schema; import org.apache.avro.generic.GenericData.Record; @@ -84,7 +85,11 @@ public UpsertInjector createInjector(@Nullable String path, QueryDataType type) if (path == null) { return FAILING_TOP_LEVEL_INJECTOR; } + Injector injector = createInjector(schema, path, type); + return value -> injector.set(record, value); + } + private Injector createInjector(Schema schema, String path, QueryDataType type) { Schema fieldSchema = unwrapNullableType(schema.getField(path).schema()); Schema.Type fieldSchemaType = fieldSchema.getType(); switch (fieldSchemaType) { @@ -95,7 +100,7 @@ public UpsertInjector createInjector(@Nullable String path, QueryDataType type) case DOUBLE: case STRING: QueryDataType targetType = AVRO_TO_SQL.getOrDefault(fieldSchemaType); - return value -> { + return (record, value) -> { try { record.set(path, targetType.convert(value)); } catch (QueryException e) { @@ -103,13 +108,28 @@ public UpsertInjector createInjector(@Nullable String path, QueryDataType type) + " (field=" + path + ")"); } }; + case RECORD: + List injectors = type.getObjectFields().stream() + .map(field -> createInjector(fieldSchema, field.getName(), field.getType())) + .collect(toList()); + return (record, value) -> { + if (value == null) { + record.set(path, null); + return; + } + GenericRecordBuilder nestedRecord = new GenericRecordBuilder(fieldSchema); + for (int i = 0; i < injectors.size(); i++) { + injectors.get(i).set(nestedRecord, ((RowValue) value).getValues().get(i)); + } + record.set(path, nestedRecord.build()); + }; case UNION: Predicate hasType = fieldSchema.getTypes().stream() .map(Schema::getType).collect(toImmutableEnumSet())::contains; DefaultedMap, List> availableTargets = CONVERSION_PREFS.mapKeysAndValues(identity(), targets -> targets.stream() .filter(hasType).map(AVRO_TO_SQL::getOrDefault).collect(toList())); - return value -> { + return (record, value) -> { if (value == null) { record.set(path, null); return; @@ -124,7 +144,7 @@ public UpsertInjector createInjector(@Nullable String path, QueryDataType type) + value.getClass().getSimpleName() + ") (field=" + path + ")"); }; case NULL: - return value -> { + return (record, value) -> { if (value != null) { throw QueryException.error("Cannot convert " + value + " to NULL (field=" + path + ")"); } @@ -146,4 +166,9 @@ public Object conclude() { this.record = null; return record; } + + @FunctionalInterface + private interface Injector { + void set(GenericRecordBuilder record, Object value); + } } diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/inject/AvroUpsertTargetDescriptor.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/inject/AvroUpsertTargetDescriptor.java index 134e2c1ad9c7..64053cffba32 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/inject/AvroUpsertTargetDescriptor.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/inject/AvroUpsertTargetDescriptor.java @@ -41,6 +41,11 @@ public UpsertTarget create(InternalSerializationService serializationService) { return new AvroUpsertTarget(schema); } + @Override + public Schema getSchema() { + return schema; + } + @Override public void writeData(ObjectDataOutput out) throws IOException { if (serializedSchema == null) { diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/inject/HazelcastObjectUpsertTarget.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/inject/HazelcastObjectUpsertTarget.java index b5aa4144e80f..e5823d53a9b1 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/inject/HazelcastObjectUpsertTarget.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/inject/HazelcastObjectUpsertTarget.java @@ -30,13 +30,12 @@ class HazelcastObjectUpsertTarget implements UpsertTarget { private Object object; @Override - public UpsertInjector createInjector(@Nullable final String path, final QueryDataType queryDataType) { - final TypeKind typeKind = TypeKind.of(queryDataType.getObjectTypeKind()); - switch (typeKind) { - case JAVA: - return value -> this.object = convertRowToJavaType(value, queryDataType); - default: - throw QueryException.error("TypeKind " + typeKind + " does not support top-level custom types"); + public UpsertInjector createInjector(@Nullable String path, QueryDataType type) { + TypeKind typeKind = type.getObjectTypeKind(); + if (typeKind == TypeKind.JAVA) { + return value -> object = convertRowToJavaType(value, type); + } else { + throw QueryException.error("TypeKind " + typeKind + " does not support top-level custom types"); } } diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/inject/PortableUpsertTarget.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/inject/PortableUpsertTarget.java index 11a6ee726c23..5d1222966415 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/inject/PortableUpsertTarget.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/inject/PortableUpsertTarget.java @@ -23,6 +23,7 @@ import com.hazelcast.nio.serialization.genericrecord.GenericRecord; import com.hazelcast.sql.impl.QueryException; import com.hazelcast.sql.impl.expression.RowValue; +import com.hazelcast.sql.impl.schema.type.TypeKind; import com.hazelcast.sql.impl.type.QueryDataType; import javax.annotation.Nonnull; @@ -63,7 +64,7 @@ public UpsertInjector createInjector(@Nullable String path, QueryDataType type) int fieldIndex = classDefinition.hasField(path) ? classDefinition.getField(path).getIndex() : -1; - if (type.isCustomType() && type.getObjectTypeKind() == QueryDataType.OBJECT_TYPE_KIND_PORTABLE) { + if (type.isCustomType() && type.getObjectTypeKind() == TypeKind.PORTABLE) { dataTypeMap.put(fieldIndex, type); } diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/inject/UpsertTargetDescriptor.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/inject/UpsertTargetDescriptor.java index 260f19d68e28..30d95ae07ce1 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/inject/UpsertTargetDescriptor.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/inject/UpsertTargetDescriptor.java @@ -22,4 +22,8 @@ public interface UpsertTargetDescriptor extends DataSerializable { UpsertTarget create(InternalSerializationService serializationService); + + default Object getSchema() { + return null; + } } diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/inject/UpsertTargetUtils.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/inject/UpsertTargetUtils.java index 1a459350a6e6..afbea64e9e56 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/inject/UpsertTargetUtils.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/inject/UpsertTargetUtils.java @@ -17,20 +17,20 @@ package com.hazelcast.jet.sql.impl.inject; import com.hazelcast.internal.serialization.impl.portable.FieldDefinitionImpl; -import com.hazelcast.jet.datamodel.Tuple3; import com.hazelcast.jet.impl.util.ReflectionUtils; -import com.hazelcast.jet.sql.impl.schema.TypesUtils; import com.hazelcast.jet.sql.impl.type.converter.ToConverter; import com.hazelcast.jet.sql.impl.type.converter.ToConverters; import com.hazelcast.nio.serialization.ClassDefinition; import com.hazelcast.nio.serialization.ClassDefinitionBuilder; import com.hazelcast.nio.serialization.FieldDefinition; import com.hazelcast.nio.serialization.FieldType; +import com.hazelcast.nio.serialization.PortableId; import com.hazelcast.nio.serialization.genericrecord.GenericRecord; import com.hazelcast.nio.serialization.genericrecord.GenericRecordBuilder; import com.hazelcast.sql.impl.QueryException; import com.hazelcast.sql.impl.expression.RowValue; import com.hazelcast.sql.impl.type.QueryDataType; +import com.hazelcast.sql.impl.type.QueryDataType.QueryDataTypeField; import com.hazelcast.sql.impl.type.QueryDataTypeFamily; import java.lang.reflect.Field; @@ -64,20 +64,20 @@ public static Object convertRowToJavaType(final Object value, final QueryDataTyp ); for (int i = 0; i < type.getObjectFields().size(); i++) { - final QueryDataType.QueryDataTypeField typeField = type.getObjectFields().get(i); + final QueryDataTypeField typeField = type.getObjectFields().get(i); final boolean isRowValueField = rowValue.getValues().get(i) instanceof RowValue; final Object fieldValue = isRowValueField - ? convertRowToJavaType(rowValue.getValues().get(i), typeField.getDataType()) + ? convertRowToJavaType(rowValue.getValues().get(i), typeField.getType()) : rowValue.getValues().get(i); Method setter = ReflectionUtils.findPropertySetter(targetClass, typeField.getName()); - ToConverter toConverter = ToConverters.getToConverter(typeField.getDataType()); + ToConverter toConverter = ToConverters.getToConverter(typeField.getType()); if (setter != null) { if (fieldValue == null && setter.getParameterTypes()[0].isPrimitive()) { throw QueryException.error("Cannot pass NULL to a method with a primitive argument: " + setter); } try { - if (typeField.getDataType().getTypeFamily().equals(QueryDataTypeFamily.OBJECT)) { + if (typeField.getType().getTypeFamily().equals(QueryDataTypeFamily.OBJECT)) { setter.invoke(result, fieldValue); } else { setter.invoke(result, toConverter.convert(fieldValue)); @@ -97,7 +97,7 @@ public static Object convertRowToJavaType(final Object value, final QueryDataTyp if (fieldValue == null && field.getType().isPrimitive()) { throw QueryException.error("Cannot set NULL to a primitive field: " + field); } - if (typeField.getDataType().getTypeFamily().equals(QueryDataTypeFamily.OBJECT)) { + if (typeField.getType().getTypeFamily().equals(QueryDataTypeFamily.OBJECT)) { field.set(result, fieldValue); } else { field.set(result, toConverter.convert(fieldValue)); @@ -110,10 +110,10 @@ public static Object convertRowToJavaType(final Object value, final QueryDataTyp return result; } - public static GenericRecord convertRowToCompactType(RowValue rowValue, QueryDataType targetDataType) { - final GenericRecordBuilder recordBuilder = GenericRecordBuilder.compact(targetDataType.getObjectTypeMetadata()); + public static GenericRecord convertRowToCompactType(RowValue rowValue, QueryDataType targetType) { + final GenericRecordBuilder recordBuilder = GenericRecordBuilder.compact(targetType.getObjectTypeMetadata()); - setCompactFields(rowValue, targetDataType, recordBuilder); + setCompactFields(rowValue, targetType, recordBuilder); return recordBuilder.build(); } @@ -183,7 +183,7 @@ private static void setPortableFields( break; case PORTABLE: if (value instanceof RowValue) { - final QueryDataType fieldQDT = queryDataType.getObjectFields().get(i).getDataType(); + final QueryDataType fieldQDT = queryDataType.getObjectFields().get(i).getType(); final ClassDefinition fieldClassDefinition = toPortableClassDefinition(fieldQDT); final GenericRecordBuilder fieldBuilder = GenericRecordBuilder.portable(fieldClassDefinition); @@ -203,15 +203,15 @@ private static void setPortableFields( } public static ClassDefinition toPortableClassDefinition(final QueryDataType queryDataType) { - final Tuple3 ids = TypesUtils - .decodePortableId(queryDataType.getObjectTypeMetadata()); - final ClassDefinitionBuilder builder = new ClassDefinitionBuilder(ids.f0(), ids.f1(), ids.f2()); + final PortableId portableId = new PortableId(queryDataType.getObjectTypeMetadata()); + final ClassDefinitionBuilder builder = new ClassDefinitionBuilder(portableId); for (int i = 0; i < queryDataType.getObjectFields().size(); i++) { - final String name = queryDataType.getObjectFields().get(i).getName(); - final QueryDataType type = queryDataType.getObjectFields().get(i).getDataType(); + final QueryDataTypeField field = queryDataType.getObjectFields().get(i); + final String name = field.getName(); + final QueryDataType fieldType = field.getType(); - switch (type.getTypeFamily()) { + switch (fieldType.getTypeFamily()) { case BOOLEAN: builder.addBooleanField(name); break; @@ -252,21 +252,13 @@ public static ClassDefinition toPortableClassDefinition(final QueryDataType quer builder.addTimestampWithTimezoneField(name); break; case OBJECT: - if (type.isCustomType()) { - final Tuple3 portableFieldIds = - TypesUtils.decodePortableId(type.getObjectTypeMetadata()); - builder.addField(new FieldDefinitionImpl( - i, - name, - FieldType.PORTABLE, - portableFieldIds.f0(), - portableFieldIds.f1(), - portableFieldIds.f2() - )); + if (fieldType.isCustomType()) { + final PortableId fieldId = new PortableId(fieldType.getObjectTypeMetadata()); + builder.addField(new FieldDefinitionImpl(i, name, FieldType.PORTABLE, fieldId)); } break; default: - throw QueryException.error("Unsupported Nested Fields Portable data type: " + type); + throw QueryException.error("Unsupported Nested Fields Portable data type: " + fieldType); } } @@ -275,13 +267,13 @@ public static ClassDefinition toPortableClassDefinition(final QueryDataType quer private static void setCompactFields( final RowValue rowValue, - final QueryDataType targetDataType, + final QueryDataType targetType, final GenericRecordBuilder recordBuilder ) { - for (int i = 0; i < targetDataType.getObjectFields().size(); i++) { - final QueryDataType.QueryDataTypeField field = targetDataType.getObjectFields().get(i); + for (int i = 0; i < targetType.getObjectFields().size(); i++) { + final QueryDataTypeField field = targetType.getObjectFields().get(i); final Object fieldValue = rowValue.getValues().get(i); - switch (field.getDataType().getTypeFamily()) { + switch (field.getType().getTypeFamily()) { case VARCHAR: recordBuilder.setString(field.getName(), (String) fieldValue); break; @@ -326,8 +318,8 @@ private static void setCompactFields( recordBuilder.setGenericRecord(field.getName(), null); } else { final GenericRecordBuilder nestedRecordBuilder = GenericRecordBuilder - .compact(field.getDataType().getObjectTypeMetadata()); - setCompactFields((RowValue) fieldValue, field.getDataType(), nestedRecordBuilder); + .compact(field.getType().getObjectTypeMetadata()); + setCompactFields((RowValue) fieldValue, field.getType(), nestedRecordBuilder); recordBuilder.setGenericRecord(field.getName(), nestedRecordBuilder.build()); } break; @@ -337,7 +329,7 @@ private static void setCompactFields( case JSON: case ROW: default: - throw QueryException.error("Unsupported upsert type: " + field.getDataType()); + throw QueryException.error("Unsupported upsert type: " + field.getType()); } } } diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/opt/OptUtils.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/opt/OptUtils.java index 40838252d6cc..d94944d639ee 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/opt/OptUtils.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/opt/OptUtils.java @@ -28,7 +28,6 @@ import com.hazelcast.jet.sql.impl.schema.JetTable; import com.hazelcast.jet.sql.impl.validate.types.HazelcastJsonType; import com.hazelcast.jet.sql.impl.validate.types.HazelcastObjectType; -import com.hazelcast.jet.sql.impl.validate.types.HazelcastObjectTypeReference; import com.hazelcast.jet.sql.impl.validate.types.HazelcastTypeUtils; import com.hazelcast.sql.impl.QueryParameterMetadata; import com.hazelcast.sql.impl.expression.Expression; @@ -37,6 +36,7 @@ import com.hazelcast.sql.impl.schema.Table; import com.hazelcast.sql.impl.schema.TableField; import com.hazelcast.sql.impl.type.QueryDataType; +import com.hazelcast.sql.impl.type.QueryDataType.QueryDataTypeField; import org.apache.calcite.plan.ConventionTraitDef; import org.apache.calcite.plan.HazelcastRelOptCluster; import org.apache.calcite.plan.RelOptCluster; @@ -350,50 +350,42 @@ public static RelDataType convert(TableField field, RelDataTypeFactory typeFacto } } - private static RelDataType convertCustomType(QueryDataType fieldType, RelDataTypeFactory typeFactory) { - final Map dataTypeMap = new HashMap<>(); - convertCustomTypeRecursively(fieldType, typeFactory, dataTypeMap); - return dataTypeMap.get(fieldType.getObjectTypeName()); + private static RelDataType convertCustomType(QueryDataType type, RelDataTypeFactory typeFactory) { + Map typeMap = new HashMap<>(); + RelDataType converted = convertCustomType(type, typeFactory, typeMap); + typeMap.values().forEach(HazelcastObjectType::finalizeFields); + return converted; } - private static void convertCustomTypeRecursively( + private static RelDataType convertCustomType( QueryDataType type, RelDataTypeFactory typeFactory, - Map typeMap + Map typeMap ) { - if (typeMap.get(type.getObjectTypeName()) != null) { - return; + HazelcastObjectType converted = typeMap.get(type.getObjectTypeName()); + if (converted != null) { + return converted; } - final List fields = new ArrayList<>(); - final HazelcastObjectTypeReference typeRef = new HazelcastObjectTypeReference(); - typeMap.put(type.getObjectTypeName(), typeRef); + converted = new HazelcastObjectType(type.getObjectTypeName()); + typeMap.put(type.getObjectTypeName(), converted); for (int i = 0; i < type.getObjectFields().size(); i++) { - final String fieldName = type.getObjectFields().get(i).getName(); - final QueryDataType fieldType = type.getObjectFields().get(i).getDataType(); - - RelDataType fieldRelDataType; - if (fieldType.isCustomType()) { - fieldRelDataType = typeMap.get(fieldType.getObjectTypeName()); - if (fieldRelDataType == null) { - convertCustomTypeRecursively(fieldType, typeFactory, typeMap); - fieldRelDataType = typeMap.get(fieldType.getObjectTypeName()); - } - } else { - fieldRelDataType = typeFactory.createTypeWithNullability( - typeFactory.createSqlType(HazelcastTypeUtils.toCalciteType(fieldType)), - true - ); - } - - fields.add(new HazelcastObjectType.Field(fieldName, i, fieldRelDataType)); + QueryDataTypeField field = type.getObjectFields().get(i); + QueryDataType fieldType = field.getType(); + + RelDataType convertedFieldType = fieldType.isCustomType() + ? convertCustomType(fieldType, typeFactory, typeMap) + : typeFactory.createTypeWithNullability( + typeFactory.createSqlType(HazelcastTypeUtils.toCalciteType(fieldType)), + true + ); + converted.addField(new HazelcastObjectType.Field(field.getName(), i, convertedFieldType)); } - typeRef.setOriginal(new HazelcastObjectType(type.getObjectTypeName(), fields)); + return converted; } - private static RelDataType convertOtherType(QueryDataType fieldType) { switch (fieldType.getTypeFamily()) { case JSON: diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/parse/SqlCreateType.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/parse/SqlCreateType.java index db8ae523e73e..9a958f028767 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/parse/SqlCreateType.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/parse/SqlCreateType.java @@ -139,7 +139,7 @@ public static String unparse(Type type) { SqlCreateType t = new SqlCreateType( identifier(CATALOG, SCHEMA_NAME_PUBLIC, type.name()), nodeList(type.getFields(), f -> new SqlTypeColumn( - identifier(f.getName()), new SqlDataType(f.getQueryDataType(), SqlParserPos.ZERO), SqlParserPos.ZERO)), + identifier(f.getName()), new SqlDataType(f.getType(), SqlParserPos.ZERO), SqlParserPos.ZERO)), reconstructOptions(type.options()), true, false, SqlParserPos.ZERO); diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/parse/SqlExtendedInsert.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/parse/SqlExtendedInsert.java index 75457a319999..f09c9e4cfdb8 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/parse/SqlExtendedInsert.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/parse/SqlExtendedInsert.java @@ -21,6 +21,7 @@ import com.hazelcast.sql.impl.extract.QueryPath; import com.hazelcast.sql.impl.schema.TableField; import com.hazelcast.sql.impl.schema.map.MapTableField; +import com.hazelcast.sql.impl.schema.type.TypeKind; import com.hazelcast.sql.impl.type.QueryDataType; import com.hazelcast.sql.impl.type.QueryDataTypeFamily; import org.apache.calcite.rel.type.RelDataType; @@ -149,6 +150,6 @@ public SqlLiteral symbol(SqlParserPos pos) { private boolean objectTypeSupportsTopLevelUpserts(QueryDataType dataType) { // Only Java Types support top level upserts. - return dataType.isCustomType() && dataType.getObjectTypeKind().equals(QueryDataType.OBJECT_TYPE_KIND_JAVA); + return dataType.isCustomType() && dataType.getObjectTypeKind() == TypeKind.JAVA; } } diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/schema/RelationsStorage.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/schema/RelationsStorage.java index 9f8c23b6db3b..47f1c2c26ba6 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/schema/RelationsStorage.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/schema/RelationsStorage.java @@ -58,6 +58,11 @@ Mapping removeMapping(String name) { return (Mapping) storage().remove(name); } + public Mapping getMapping(String name) { + Object object = storage().get(name); + return object instanceof Mapping ? (Mapping) object : null; + } + public Collection getAllTypes() { return storage().values().stream() .filter(o -> o instanceof Type) diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/schema/TypeDefinitionColumn.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/schema/TypeDefinitionColumn.java index f647ce9a9976..5180abb1df21 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/schema/TypeDefinitionColumn.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/schema/TypeDefinitionColumn.java @@ -20,18 +20,18 @@ public class TypeDefinitionColumn { private final String name; - private final QueryDataType dataType; + private final QueryDataType type; - public TypeDefinitionColumn(String name, QueryDataType dataType) { + public TypeDefinitionColumn(String name, QueryDataType type) { this.name = name; - this.dataType = dataType; + this.type = type; } public String name() { return name; } - public QueryDataType dataType() { - return dataType; + public QueryDataType type() { + return type; } } diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/schema/TypeUtils.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/schema/TypeUtils.java new file mode 100644 index 000000000000..593a68f08f9f --- /dev/null +++ b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/schema/TypeUtils.java @@ -0,0 +1,363 @@ +/* + * Copyright 2023 Hazelcast Inc. + * + * Licensed under the Hazelcast Community License (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://hazelcast.com/hazelcast-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.hazelcast.jet.sql.impl.schema; + +import com.hazelcast.internal.serialization.InternalSerializationService; +import com.hazelcast.internal.serialization.impl.portable.PortableContext; +import com.hazelcast.nio.serialization.ClassDefinition; +import com.hazelcast.nio.serialization.FieldDefinition; +import com.hazelcast.nio.serialization.FieldType; +import com.hazelcast.nio.serialization.PortableId; +import com.hazelcast.sql.impl.FieldsUtil; +import com.hazelcast.sql.impl.QueryException; +import com.hazelcast.sql.impl.extract.QueryPath; +import com.hazelcast.sql.impl.schema.MappingField; +import com.hazelcast.sql.impl.schema.type.Type; +import com.hazelcast.sql.impl.schema.type.Type.TypeField; +import com.hazelcast.sql.impl.schema.type.TypeKind; +import com.hazelcast.sql.impl.type.QueryDataType; +import com.hazelcast.sql.impl.type.QueryDataType.QueryDataTypeField; +import com.hazelcast.sql.impl.type.QueryDataTypeUtils; +import org.apache.avro.Schema; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.SortedMap; +import java.util.function.Supplier; + +import static com.hazelcast.jet.sql.impl.connector.SqlConnector.AVRO_FORMAT; +import static com.hazelcast.jet.sql.impl.connector.SqlConnector.COMPACT_FORMAT; +import static com.hazelcast.jet.sql.impl.connector.SqlConnector.JAVA_FORMAT; +import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_TYPE_AVRO_SCHEMA; +import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_TYPE_COMPACT_TYPE_NAME; +import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_TYPE_JAVA_CLASS; +import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_TYPE_PORTABLE_CLASS_ID; +import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_TYPE_PORTABLE_CLASS_VERSION; +import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_TYPE_PORTABLE_FACTORY_ID; +import static com.hazelcast.jet.sql.impl.connector.SqlConnector.PORTABLE_FORMAT; +import static com.hazelcast.jet.sql.impl.connector.file.AvroResolver.AVRO_TO_SQL; +import static com.hazelcast.jet.sql.impl.connector.file.AvroResolver.unwrapNullableType; +import static com.hazelcast.jet.sql.impl.connector.keyvalue.KvMetadataAvroResolver.inlineSchema; +import static com.hazelcast.jet.sql.impl.connector.keyvalue.KvMetadataJavaResolver.loadClass; +import static com.hazelcast.jet.sql.impl.connector.map.MetadataCompactResolver.compactTypeName; +import static com.hazelcast.jet.sql.impl.connector.map.MetadataPortableResolver.PORTABLE_TO_SQL; +import static com.hazelcast.jet.sql.impl.connector.map.MetadataPortableResolver.portableId; +import static java.util.stream.Collectors.toList; + +public final class TypeUtils { + private TypeUtils() { } + + public static FieldEnricher getFieldEnricher( + String format, + InternalSerializationService serializationService, + RelationsStorage relationsStorage + ) { + switch (format) { + case PORTABLE_FORMAT: + return new PortableEnricher(relationsStorage, serializationService); + case COMPACT_FORMAT: + return new CompactEnricher(relationsStorage); + case JAVA_FORMAT: + return new JavaEnricher(relationsStorage); + case AVRO_FORMAT: + return new AvroEnricher(relationsStorage); + default: + throw QueryException.error("Unsupported type format: " + format); + } + } + + private static class PortableEnricher extends FieldEnricher { + private final PortableContext context; + + PortableEnricher(RelationsStorage relationsStorage, InternalSerializationService serializationService) { + super(TypeKind.PORTABLE, relationsStorage); + context = serializationService.getPortableContext(); + } + + @Override + protected String getTypeMetadata(PortableId portableId) { + return portableId.toString(); + } + + @Override + protected ClassDefinition getSchema(PortableId portableId) { + return context.lookupClassDefinition(portableId); + } + + @Override + protected List resolveFields(ClassDefinition classDef) { + if (classDef == null) { + throw QueryException.error("Either a column list must be provided or the class " + + "definition must be registered to create Portable-based types"); + } + return classDef.getFieldNames().stream().map(name -> { + FieldDefinition field = classDef.getField(name); + if (field.getType().equals(FieldType.PORTABLE)) { + throw QueryException.error("Column list is required to create nested fields"); + } + return new TypeField(name, PORTABLE_TO_SQL.getOrDefault(field.getType())); + }).collect(toList()); + } + + @Override + protected PortableId getFieldSchemaId(ClassDefinition classDef, String fieldName, String fieldTypeName) { + if (classDef == null) { + throw QueryException.error("Either a portable ID must be provided or the " + + "class definition must be registered to create nested fields"); + } + return classDef.getField(fieldName).getPortableId(); + } + + @Override + protected PortableId getSchemaId(Map mappingOptions, boolean isKey) { + return portableId(mappingOptions, isKey); + } + + @Override + protected PortableId getSchemaId(Map typeOptions) { + return portableId(typeOptions, OPTION_TYPE_PORTABLE_FACTORY_ID, + OPTION_TYPE_PORTABLE_CLASS_ID, OPTION_TYPE_PORTABLE_CLASS_VERSION); + } + } + + private static class CompactEnricher extends FieldEnricher { + CompactEnricher(RelationsStorage relationsStorage) { + super(TypeKind.COMPACT, relationsStorage); + } + + @Override + protected String getTypeMetadata(String compactTypeName) { + return compactTypeName; + } + + @Override + protected Void getSchema(String schemaId) { + return null; + } + + @Override + protected List resolveFields(Void schema) { + throw QueryException.error("Column list is required to create Compact-based types"); + } + + @Override + protected String getFieldSchemaId(Void schema, String fieldName, String fieldTypeName) { + return fieldTypeName + "CompactType"; + } + + @Override + protected String getSchemaId(Map mappingOptions, boolean isKey) { + return compactTypeName(mappingOptions, isKey); + } + + @Override + protected String getSchemaId(Map typeOptions) { + return typeOptions.get(OPTION_TYPE_COMPACT_TYPE_NAME); + } + } + + private static class JavaEnricher extends FieldEnricher, SortedMap>> { + JavaEnricher(RelationsStorage relationsStorage) { + super(TypeKind.JAVA, relationsStorage); + } + + @Override + protected String getTypeMetadata(Class typeClass) { + return typeClass.getName(); + } + + @Override + protected SortedMap> getSchema(Class typeClass) { + return FieldsUtil.resolveClass(typeClass); + } + + @Override + protected List resolveFields(SortedMap> classFields) { + return classFields.entrySet().stream().map(e -> { + if (isUserClass(e.getValue())) { + throw QueryException.error("Column list is required to create nested fields"); + } + return new TypeField(e.getKey(), QueryDataTypeUtils.resolveTypeForClass(e.getValue())); + }).collect(toList()); + } + + @Override + protected Class getFieldSchemaId(SortedMap> classFields, + String fieldName, String fieldTypeName) { + return classFields.get(fieldName); + } + + @Override + protected Class getSchemaId(Map mappingOptions, boolean isKey) { + return loadClass(mappingOptions, isKey); + } + + @Override + protected Class getSchemaId(Map typeOptions) { + String className = typeOptions.get(OPTION_TYPE_JAVA_CLASS); + return className != null ? loadClass(className) : null; + } + + private static boolean isUserClass(Class clazz) { + return !clazz.isPrimitive() && !clazz.getPackage().getName().startsWith("java."); + } + } + + private static class AvroEnricher extends FieldEnricher { + AvroEnricher(RelationsStorage relationsStorage) { + super(TypeKind.AVRO, relationsStorage); + } + + @Override + protected String getTypeMetadata(Schema schema) { + // AvroUpsertTarget has already a reference to the schema, and Avro schemas + // are self-contained. That is, it is not possible to have a partial schema + // or override some parts of a schema. + return null; + } + + @Override + protected Schema getSchema(Schema schema) { + return schema; + } + + @Override + protected List resolveFields(Schema schema) { + if (schema == null) { + throw QueryException.error( + "Either a column list or an inline schema is required to create Avro-based types"); + } + return schema.getFields().stream().map(field -> { + Schema fieldSchema = unwrapNullableType(field.schema()); + if (fieldSchema.getType() == Schema.Type.RECORD) { + throw QueryException.error("Column list is required to create nested fields"); + } + return new TypeField(field.name(), AVRO_TO_SQL.getOrDefault(fieldSchema.getType())); + }).collect(toList()); + } + + @Override + protected Schema getFieldSchemaId(Schema schema, String fieldName, String fieldTypeName) { + return schema != null ? unwrapNullableType(schema.getField(fieldName).schema()) : null; + } + + @Override + protected Schema getSchemaId(Map mappingOptions, boolean isKey) { + return inlineSchema(mappingOptions, isKey); + } + + @Override + protected Schema getSchemaId(Map typeOptions) { + String json = typeOptions.get(OPTION_TYPE_AVRO_SCHEMA); + return json != null ? new Schema.Parser().parse(json) : null; + } + } + + /** + * Adds {@linkplain QueryDataType#getObjectTypeMetadata() metadata}, i.e. schema ID, + * to custom types.
    + *
  1. If the type has a well-defined† metadata in options, this self-reported metadata + * is used. († For example, in Portable, {@code factoryId}, {@code classId}, and + * {@code version} must be provided together.) + *
  2. If the mapping has only {@code __key} or {@code this} fields, and both the mapping + * and type define metadata, the one that is defined by the type is used. In other + * words, type options override mapping options. In this case, the mapping does not + * need to specify metadata since it will be ignored. + *
  3. If the type does not define metadata, it is resolved from the parent, which might + * be another type or the mapping: if the type belongs to explicit {@code __key} or + * {@code this} fields, the metadata will be inherited from the mapping; + * otherwise, it will be accessed as a field from the parent schema. + *
+ * + * @param type of schema identifier + * @param type of schema + */ + public abstract static class FieldEnricher { + private final TypeKind typeKind; + private final RelationsStorage relationsStorage; + + FieldEnricher(TypeKind typeKind, RelationsStorage relationsStorage) { + this.typeKind = typeKind; + this.relationsStorage = relationsStorage; + } + + public void enrich(MappingField field, Map mappingOptions, boolean isKey) { + String typeName = field.type().getObjectTypeName(); + field.setType(createFieldType( + field.name().equals(isKey ? QueryPath.KEY : QueryPath.VALUE) + ? () -> getSchemaId(mappingOptions, isKey) + : () -> getFieldSchemaId(getSchema(getSchemaId(mappingOptions, isKey)), + plainExternalName(field), typeName), + typeName, new HashMap<>())); + } + + protected QueryDataType createFieldType(Supplier schemaIdSupplier, String typeName, + Map seen) { + QueryDataType convertedType = seen.get(typeName); + if (convertedType != null) { + return convertedType; + } + + Type type = relationsStorage.getType(typeName); + if (type == null) { + throw QueryException.error("Encountered type '" + typeName + "', which doesn't exist"); + } + + ID schemaId = getSchemaId(type.options()); + if (schemaId == null) { + schemaId = schemaIdSupplier.get(); + } + + S schema = getSchema(schemaId); + if (type.getFields().isEmpty()) { + type.setFields(resolveFields(schema)); + relationsStorage.put(typeName, type); + } + + convertedType = new QueryDataType(typeName, typeKind, getTypeMetadata(schemaId)); + seen.put(typeName, convertedType); + + for (TypeField field : type.getFields()) { + QueryDataType fieldType = field.getType(); + String fieldTypeName = fieldType.getObjectTypeName(); + + if (fieldType.isCustomType()) { + fieldType = createFieldType(() -> getFieldSchemaId(schema, field.getName(), fieldTypeName), + fieldTypeName, seen); + } + convertedType.getObjectFields().add(new QueryDataTypeField(field.getName(), fieldType)); + } + + return convertedType; + } + + protected abstract String getTypeMetadata(ID schemaId); + protected abstract S getSchema(ID schemaId); + protected abstract List resolveFields(S schema); + protected abstract ID getFieldSchemaId(S schema, String fieldName, String fieldTypeName); + protected abstract ID getSchemaId(Map mappingOptions, boolean isKey); + protected abstract ID getSchemaId(Map typeOptions); + + /** + * Returns the external name of the field without {@code __key.} or {@code this.} prefix. + */ + public static String plainExternalName(MappingField field) { + return QueryPath.create(field.externalName()).getPath(); + } + } +} diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/schema/TypesUtils.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/schema/TypesUtils.java deleted file mode 100644 index b06966251ddf..000000000000 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/schema/TypesUtils.java +++ /dev/null @@ -1,290 +0,0 @@ -/* - * Copyright 2023 Hazelcast Inc. - * - * Licensed under the Hazelcast Community License (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://hazelcast.com/hazelcast-community-license - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.jet.sql.impl.schema; - -import com.hazelcast.jet.datamodel.Tuple3; -import com.hazelcast.jet.sql.impl.connector.SqlConnector; -import com.hazelcast.nio.serialization.ClassDefinition; -import com.hazelcast.nio.serialization.FieldDefinition; -import com.hazelcast.nio.serialization.FieldType; -import com.hazelcast.sql.impl.FieldsUtil; -import com.hazelcast.sql.impl.QueryException; -import com.hazelcast.sql.impl.schema.MappingField; -import com.hazelcast.sql.impl.schema.type.Type; -import com.hazelcast.sql.impl.schema.type.TypeKind; -import com.hazelcast.sql.impl.type.QueryDataType; -import com.hazelcast.sql.impl.type.QueryDataTypeUtils; - -import javax.annotation.Nonnull; -import javax.annotation.Nullable; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -public final class TypesUtils { - private TypesUtils() { - } - - public static QueryDataType convertTypeToQueryDataType(final Type rootType, final RelationsStorage relationsStorage) { - return convertTypeToQueryDataTypeInt(rootType.name(), rootType, relationsStorage, new HashMap<>()); - } - - public static Type convertPortableClassToType( - final String name, - final ClassDefinition classDef, - final TableResolverImpl tableResolver - ) { - final Type type = new Type(); - type.setName(name); - type.setKind(TypeKind.PORTABLE); - type.setPortableFactoryId(classDef.getFactoryId()); - type.setPortableClassId(classDef.getClassId()); - type.setPortableVersion(classDef.getVersion()); - - final List fields = new ArrayList<>(); - for (int i = 0; i < classDef.getFieldCount(); i++) { - final FieldDefinition portableField = classDef.getField(i); - final Type.TypeField typeField = new Type.TypeField(); - typeField.setName(portableField.getName()); - - final QueryDataType queryDataType; - if (portableField.getType().equals(FieldType.PORTABLE)) { - queryDataType = toQueryDataTypeRef(tableResolver.getTypes() - .stream() - .filter(t -> t.getKind().equals(TypeKind.PORTABLE)) - .filter(t -> t.getPortableFactoryId().equals(portableField.getFactoryId())) - .filter(t -> t.getPortableClassId().equals(portableField.getClassId())) - .filter(t -> t.getPortableVersion().equals(portableField.getVersion())) - .findFirst() - .orElseThrow(() -> QueryException.error("Type with Portable IDs " + encodePortableId( - portableField.getFactoryId(), - portableField.getClassId(), - portableField.getVersion() - ) + " does not exist."))); - - } else { - queryDataType = resolvePortableFieldType(portableField.getType()); - } - - typeField.setQueryDataType(queryDataType); - fields.add(typeField); - } - type.setFields(fields); - - return type; - } - - @SuppressWarnings("checkstyle:ReturnCount") - public static QueryDataType resolvePortableFieldType(FieldType fieldType) { - switch (fieldType) { - case BOOLEAN: - return QueryDataType.BOOLEAN; - case BYTE: - return QueryDataType.TINYINT; - case SHORT: - return QueryDataType.SMALLINT; - case INT: - return QueryDataType.INT; - case LONG: - return QueryDataType.BIGINT; - case FLOAT: - return QueryDataType.REAL; - case DOUBLE: - return QueryDataType.DOUBLE; - case DECIMAL: - return QueryDataType.DECIMAL; - case CHAR: - return QueryDataType.VARCHAR_CHARACTER; - case UTF: - return QueryDataType.VARCHAR; - case TIME: - return QueryDataType.TIME; - case DATE: - return QueryDataType.DATE; - case TIMESTAMP: - return QueryDataType.TIMESTAMP; - case TIMESTAMP_WITH_TIMEZONE: - return QueryDataType.TIMESTAMP_WITH_TZ_OFFSET_DATE_TIME; - case PORTABLE: - default: - return QueryDataType.OBJECT; - } - } - - public static QueryDataType toQueryDataTypeRef(final Type type) { - final QueryDataType queryDataType; - switch (type.getKind()) { - case JAVA: - queryDataType = new QueryDataType(type.name(), QueryDataType.OBJECT_TYPE_KIND_JAVA); - queryDataType.setObjectTypeMetadata(type.getJavaClassName()); - return queryDataType; - case PORTABLE: - queryDataType = new QueryDataType(type.name(), QueryDataType.OBJECT_TYPE_KIND_PORTABLE); - queryDataType.setObjectTypeMetadata(encodePortableId( - type.getPortableFactoryId(), - type.getPortableClassId(), - type.getPortableVersion() - )); - return queryDataType; - case COMPACT: - queryDataType = new QueryDataType(type.name(), QueryDataType.OBJECT_TYPE_KIND_COMPACT); - queryDataType.setObjectTypeMetadata(type.getCompactTypeName()); - return queryDataType; - default: - throw new UnsupportedOperationException("Not implemented yet."); - } - } - - public static String encodePortableId(final int factoryId, final int classId, final int version) { - return factoryId + ":" + classId + ":" + version; - } - - public static Tuple3 decodePortableId(final String encoded) { - final String[] components = encoded.split(":"); - assert components.length == 3 : "Number of Portable ID components should always be 3"; - - return Tuple3.tuple3( - Integer.parseInt(components[0]), - Integer.parseInt(components[1]), - Integer.parseInt(components[2]) - ); - } - - public static Type convertJavaClassToType( - final String name, - final List columns, - final Class typeClass - ) { - final Map userColumnsMap = columns.stream() - .collect(Collectors.toMap(TypeDefinitionColumn::name, TypeDefinitionColumn::dataType)); - - final Type type = new Type(); - type.setName(name); - type.setKind(TypeKind.JAVA); - type.setJavaClassName(typeClass.getName()); - - final List fields = new ArrayList<>(); - for (final Map.Entry> entry : FieldsUtil.resolveClass(typeClass).entrySet()) { - final QueryDataType queryDataType; - if (isUserClass(entry.getValue())) { - if (entry.getValue().getName().equals(type.getJavaClassName())) { - queryDataType = toQueryDataTypeRef(type); - } else { - queryDataType = userColumnsMap.get(entry.getKey()) != null - ? userColumnsMap.get(entry.getKey()) - : QueryDataType.OBJECT; - } - } else { - queryDataType = QueryDataTypeUtils.resolveTypeForClass(entry.getValue()); - } - - fields.add(new Type.TypeField(entry.getKey(), queryDataType)); - } - type.setFields(fields); - - return type; - } - - public static void enrichMappingFieldType( - final TypeKind mappingTypeKind, - final MappingField field, - final RelationsStorage relationsStorage - ) { - if (!field.type().isCustomType()) { - return; - } - final Type type = relationsStorage.getType(field.type().getObjectTypeName()); - if (type == null) { - throw QueryException.error("Non existing type found in the mapping: " - + field.type().getObjectTypeName()); - } - - if (!mappingTypeKind.equals(type.getKind())) { - throw QueryException.error("Can not use Type " + type.name() + "[" - + type.getKind() + "] with " + mappingTypeKind + " mapping."); - } - - final QueryDataType resolved = convertTypeToQueryDataType(type, relationsStorage); - field.setType(resolved); - } - - public static TypeKind formatToTypeKind(String format) { - if (format == null) { - return TypeKind.NONE; - } - - switch (format) { - case SqlConnector.JAVA_FORMAT: - return TypeKind.JAVA; - case SqlConnector.PORTABLE_FORMAT: - return TypeKind.PORTABLE; - case SqlConnector.COMPACT_FORMAT: - return TypeKind.COMPACT; - default: - return TypeKind.NONE; - } - } - - /** - * If `type` is null, `typeName` will be used to look it up from the storage. - */ - private static QueryDataType convertTypeToQueryDataTypeInt( - @Nonnull final String typeName, - @Nullable Type type, - @Nonnull final RelationsStorage relationsStorage, - @Nonnull final Map seen - ) { - QueryDataType convertedType = seen.get(typeName); - if (convertedType != null) { - return convertedType; - } - - if (type == null) { - type = relationsStorage.getType(typeName); - } - - if (type == null) { - throw QueryException.error("Encountered type '" + typeName + "', which doesn't exist"); - } - - // At this point the `convertedType` lacks fields. We put it to the `seen` map for the purpose of resolving - // cyclic references, we'll add the fields later below. - convertedType = toQueryDataTypeRef(type); - seen.putIfAbsent(type.name(), convertedType); - - for (Type.TypeField field : type.getFields()) { - QueryDataType queryDataType; - if (field.getQueryDataType().isCustomType()) { - queryDataType = convertTypeToQueryDataTypeInt(field.getQueryDataType().getObjectTypeName(), - null, relationsStorage, seen); - } else { - queryDataType = field.getQueryDataType(); - } - - convertedType.getObjectFields().add( - new QueryDataType.QueryDataTypeField(field.getName(), queryDataType)); - } - - return convertedType; - } - - private static boolean isUserClass(Class clazz) { - return !clazz.isPrimitive() && !clazz.getPackage().getName().startsWith("java."); - } -} diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/type/converter/ToConverters.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/type/converter/ToConverters.java index 0d58bc26a0ff..05c71f90079d 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/type/converter/ToConverters.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/type/converter/ToConverters.java @@ -17,6 +17,7 @@ package com.hazelcast.jet.sql.impl.type.converter; import com.hazelcast.sql.impl.type.QueryDataType; +import com.hazelcast.sql.impl.type.QueryDataTypeFamily; import javax.annotation.Nonnull; import java.math.BigDecimal; @@ -30,14 +31,16 @@ import java.util.Objects; public final class ToConverters { - private static final Map CONVERTERS = prepareConverters(); - private ToConverters() { - } + private ToConverters() { } @Nonnull public static ToConverter getToConverter(QueryDataType type) { + if (type.getTypeFamily() == QueryDataTypeFamily.OBJECT) { + // User-defined types are subject to the same conversion rules as ordinary OBJECT. + type = QueryDataType.OBJECT; + } return Objects.requireNonNull(CONVERTERS.get(type), "missing converter for " + type); } diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/validate/HazelcastSqlValidator.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/validate/HazelcastSqlValidator.java index 22142937ab65..3e858c33aee6 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/validate/HazelcastSqlValidator.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/validate/HazelcastSqlValidator.java @@ -82,8 +82,6 @@ import static com.hazelcast.jet.sql.impl.connector.SqlConnectorUtil.getJetSqlConnector; import static com.hazelcast.jet.sql.impl.validate.ValidatorResource.RESOURCE; -import static com.hazelcast.jet.sql.impl.validate.types.HazelcastTypeUtils.extractHzObjectType; -import static com.hazelcast.jet.sql.impl.validate.types.HazelcastTypeUtils.isHzObjectType; import static org.apache.calcite.sql.JoinType.FULL; import static org.apache.calcite.sql.SqlKind.AS; import static org.apache.calcite.sql.SqlKind.COLLECTION_TABLE; @@ -232,8 +230,8 @@ private boolean containsCycles(final HazelcastObjectType type, final Set for (final RelDataTypeField field : type.getFieldList()) { final RelDataType fieldType = field.getType(); - if (isHzObjectType(fieldType) - && containsCycles(extractHzObjectType(fieldType), discovered)) { + if (fieldType instanceof HazelcastObjectType + && containsCycles((HazelcastObjectType) fieldType, discovered)) { return true; } } @@ -355,11 +353,11 @@ private void validateUpsertRowType(SqlIdentifier table) { for (final RelDataTypeField field : rowType.getFieldList()) { final RelDataType fieldType = field.getType(); - if (!isHzObjectType(fieldType)) { + if (!(fieldType instanceof HazelcastObjectType)) { continue; } - if (containsCycles(extractHzObjectType(fieldType), new HashSet<>())) { + if (containsCycles((HazelcastObjectType) fieldType, new HashSet<>())) { throw QueryException.error("Upserts are not supported for cyclic data type columns"); } } diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/validate/operators/special/HazelcastUdtObjectToJsonFunction.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/validate/operators/special/HazelcastUdtObjectToJsonFunction.java index 06c14aa20ed2..2783ca14d974 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/validate/operators/special/HazelcastUdtObjectToJsonFunction.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/validate/operators/special/HazelcastUdtObjectToJsonFunction.java @@ -21,7 +21,6 @@ import com.hazelcast.jet.sql.impl.validate.operators.typeinference.ReplaceUnknownOperandTypeInference; import com.hazelcast.jet.sql.impl.validate.types.HazelcastJsonType; import com.hazelcast.jet.sql.impl.validate.types.HazelcastObjectType; -import com.hazelcast.jet.sql.impl.validate.types.HazelcastObjectTypeReference; import org.apache.calcite.sql.SqlFunctionCategory; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlOperandCountRange; @@ -43,8 +42,7 @@ public HazelcastUdtObjectToJsonFunction() { @Override protected boolean checkOperandTypes(final HazelcastCallBinding callBinding, final boolean throwOnFailure) { - return callBinding.getOperandType(0) instanceof HazelcastObjectTypeReference || - callBinding.getOperandType(0) instanceof HazelcastObjectType; + return callBinding.getOperandType(0) instanceof HazelcastObjectType; } @Override diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/validate/types/HazelcastObjectType.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/validate/types/HazelcastObjectType.java index bb473c5a1b4b..4a7f2c35fe4c 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/validate/types/HazelcastObjectType.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/validate/types/HazelcastObjectType.java @@ -17,29 +17,38 @@ package com.hazelcast.jet.sql.impl.validate.types; import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeFactoryImpl; import org.apache.calcite.rel.type.RelDataTypeField; import org.apache.calcite.rel.type.RelDataTypeFieldImpl; import org.apache.calcite.rel.type.RelDataTypeImpl; import org.apache.calcite.sql.type.SqlTypeName; import org.checkerframework.checker.nullness.qual.Nullable; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.Iterator; import java.util.List; +import java.util.Set; + +import static java.util.stream.Collectors.toUnmodifiableList; public class HazelcastObjectType extends RelDataTypeImpl { - private final boolean nullable; private final String name; + private final boolean nullable; + /** Modifiable list of fields to support cyclic types. */ + private List fields = new ArrayList<>(); + /** Cached list of field names. */ + private List fieldNames; - public HazelcastObjectType(final String name, final List fields) { - super(fields); - this.name = name; - this.digest = "HZ_OBJECT(" + name + ")"; - this.nullable = true; + /** Not usable until {@link #finalizeFields()} is called. */ + public HazelcastObjectType(String name) { + this(name, true); } - public HazelcastObjectType(final String name, final List fields, final boolean nullable) { - super(fields); + /** Not usable until {@link #finalizeFields()} is called. */ + public HazelcastObjectType(String name, boolean nullable) { + super(null); this.name = name; - this.digest = "HZ_OBJECT(" + name + ")"; this.nullable = nullable; } @@ -48,10 +57,8 @@ public String getTypeName() { } @Override - protected void generateTypeString(final StringBuilder sb, final boolean withDetail) { - sb.append("HZ_OBJECT(") - .append(name) - .append(")"); + public boolean isNullable() { + return nullable; } @Override @@ -59,33 +66,101 @@ public SqlTypeName getSqlTypeName() { return SqlTypeName.OTHER; } + /** + * @implNote This is called by {@link #computeDigest} and {@link #toString}, with + * and without detail respectively. {@link #getFullTypeString}, {@link #equals} and + * {@link #hashCode} uses the {@link #digest} generated via this method. + *

+ * A previous implementation used to ignore {@code withDetail} and only incorporated + * {@link #name} into the digest. This resulted in conflicts in {@link + * RelDataTypeFactoryImpl#DATATYPE_CACHE} in the test environment. However, the + * current state of user-defined types (UDTs) does not allow having different set of + * fields for the same UDT in different mappings. If this is allowed in the future, + * this implementation will ensure that there are no collisions in the cache. + */ @Override - public @Nullable RelDataTypeField getField(final String fieldName, final boolean caseSensitive, final boolean elideRecord) { - assert fieldList != null; - RelDataTypeField fieldType = null; - for (final RelDataTypeField field : fieldList) { - if (fieldName.equals(field.getName())) { - fieldType = new RelDataTypeFieldImpl(fieldName, field.getIndex(), field.getType()); - break; + protected void generateTypeString(StringBuilder sb, boolean withDetail) { + if (withDetail) { + generateFullTypeString(this, sb, new HashSet<>()); + } else { + sb.append(name); + } + } + + private static void generateFullTypeString(HazelcastObjectType type, StringBuilder sb, Set seen) { + sb.append(type.name); + if (seen.contains(type.name)) { + return; + } + seen.add(type.name); + sb.append('('); + for (Iterator it = type.fields.iterator(); it.hasNext();) { + RelDataTypeField field = it.next(); + sb.append(field.getName()).append(':'); + if (field.getType() instanceof HazelcastObjectType) { + generateFullTypeString((HazelcastObjectType) field.getType(), sb, seen); + } else { + sb.append(field.getType().getFullTypeString()); + } + if (it.hasNext()) { + sb.append(", "); } } + sb.append(')'); + } - return fieldType; + public void addField(Field field) { + fields.add(field); } - @Override - protected void computeDigest() { + public void finalizeFields() { + fields = List.of(fields.toArray(Field[]::new)); super.computeDigest(); } - public static class Field extends RelDataTypeFieldImpl { - public Field(final String name, final int index, final RelDataType type) { - super(name, index, type); + /** + * @implNote {@code caseSensitive} is ignored since {@link #equals} compares the + * {@link #digest} fields in a case-sensitive manner. + */ + @Override + @Nullable + public RelDataTypeField getField(String fieldName, boolean caseSensitive, boolean elideRecord) { + assert !elideRecord; + for (RelDataTypeField field : fields) { + if (fieldName.equals(field.getName())) { + return new RelDataTypeFieldImpl(fieldName, field.getIndex(), field.getType()); + } } + return null; } @Override - public boolean isNullable() { - return nullable; + @SuppressWarnings("unchecked") + public List getFieldList() { + return (List) (List) fields; + } + + @Override + public List getFieldNames() { + if (fieldNames == null) { + fieldNames = fields.stream().map(Field::getName).collect(toUnmodifiableList()); + } + return fieldNames; + } + + @Override + public int getFieldCount() { + return fields.size(); + } + + @Override + public boolean isStruct() { + return true; + } + + public static class Field extends RelDataTypeFieldImpl { + public Field(String name, int index, RelDataType type) { + super(name, index, type); + } } } diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/validate/types/HazelcastObjectTypeReference.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/validate/types/HazelcastObjectTypeReference.java deleted file mode 100644 index 5949d749c3b3..000000000000 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/validate/types/HazelcastObjectTypeReference.java +++ /dev/null @@ -1,208 +0,0 @@ -/* - * Copyright 2023 Hazelcast Inc. - * - * Licensed under the Hazelcast Community License (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://hazelcast.com/hazelcast-community-license - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.jet.sql.impl.validate.types; - -import org.apache.calcite.rel.type.RelDataType; -import org.apache.calcite.rel.type.RelDataTypeComparability; -import org.apache.calcite.rel.type.RelDataTypeFamily; -import org.apache.calcite.rel.type.RelDataTypeField; -import org.apache.calcite.rel.type.RelDataTypePrecedenceList; -import org.apache.calcite.rel.type.StructKind; -import org.apache.calcite.sql.SqlCollation; -import org.apache.calcite.sql.SqlIdentifier; -import org.apache.calcite.sql.SqlIntervalQualifier; -import org.apache.calcite.sql.type.SqlTypeName; -import org.checkerframework.checker.nullness.qual.Nullable; - -import java.nio.charset.Charset; -import java.util.List; - -/** - * Temporary reference type used to support CRec (Circularly-recurrent) types e.g. - * Class A { B b; } - * Class B { C c; } - * Class C { A a; } - * In this hierarchy instead of supplying actual type for columns b, c, a, this type will be inserted and later on - * the original's will be set to respective HazelcastObjectTypes to fix the references. - */ -public class HazelcastObjectTypeReference implements RelDataType { - private HazelcastObjectType original; - - public RelDataType getOriginal() { - return original; - } - - public void setOriginal(final HazelcastObjectType original) { - this.original = original; - } - - @Override - public boolean isStruct() { - assert original != null; - return original.isStruct(); - } - - @Override - public List getFieldList() { - assert original != null; - return original.getFieldList(); - } - - @Override - public List getFieldNames() { - assert original != null; - return original.getFieldNames(); - } - - @Override - public int getFieldCount() { - assert original != null; - return original.getFieldCount(); - } - - @Override - public StructKind getStructKind() { - assert original != null; - return original.getStructKind(); - } - - @Override - public @Nullable RelDataTypeField getField(final String fieldName, final boolean caseSensitive, final boolean elideRecord) { - assert original != null; - return original.getField(fieldName, caseSensitive, elideRecord); - } - - @Override - public boolean isNullable() { - assert original != null; - return original.isNullable(); - } - - @Override - public @Nullable RelDataType getComponentType() { - assert original != null; - return original.getComponentType(); - } - - @Override - public @Nullable RelDataType getKeyType() { - assert original != null; - return original.getKeyType(); - } - - @Override - public @Nullable RelDataType getValueType() { - assert original != null; - return original.getValueType(); - } - - @Override - public @Nullable Charset getCharset() { - assert original != null; - return original.getCharset(); - } - - @Override - public @Nullable SqlCollation getCollation() { - assert original != null; - return original.getCollation(); - } - - @Override - public @Nullable SqlIntervalQualifier getIntervalQualifier() { - assert original != null; - return original.getIntervalQualifier(); - } - - @Override - public int getPrecision() { - assert original != null; - return original.getPrecision(); - } - - @Override - public int getScale() { - assert original != null; - return original.getScale(); - } - - @Override - public SqlTypeName getSqlTypeName() { - assert original != null; - return original.getSqlTypeName(); - } - - @Override - public @Nullable SqlIdentifier getSqlIdentifier() { - assert original != null; - return original.getSqlIdentifier(); - } - - @Override - public String getFullTypeString() { - assert original != null; - return original.getFullTypeString(); - } - - @Override - public RelDataTypeFamily getFamily() { - assert original != null; - return original.getFamily(); - } - - @Override - public RelDataTypePrecedenceList getPrecedenceList() { - assert original != null; - return original.getPrecedenceList(); - } - - @Override - public RelDataTypeComparability getComparability() { - assert original != null; - return original.getComparability(); - } - - @Override - public boolean isDynamicStruct() { - assert original != null; - return original.isDynamicStruct(); - } - - @Override - public boolean equals(@Nullable Object obj) { - assert original != null; - return obj != null && obj.equals(original); - } - - @Override - public int hashCode() { - assert original != null; - return original.hashCode(); - } - - @Override - public String toString() { - assert original != null; - return original.toString(); - } - - @Override - public boolean equalsSansFieldNames(@Nullable final RelDataType that) { - assert original != null; - return original.equalsSansFieldNames(that); - } -} diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/validate/types/HazelcastTypeCoercion.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/validate/types/HazelcastTypeCoercion.java index 28c33735f49b..736d93790d63 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/validate/types/HazelcastTypeCoercion.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/validate/types/HazelcastTypeCoercion.java @@ -241,9 +241,9 @@ private boolean customTypesCoercion( return customTypesCoercion(source, target, ((SqlCall) rowElement).operand(0), scope); } - if (HazelcastTypeUtils.isHzObjectType(source)) { - return HazelcastTypeUtils.extractHzObjectType(source).getTypeName() - .equals(HazelcastTypeUtils.extractHzObjectType(target).getTypeName()); + if (source instanceof HazelcastObjectType) { + return ((HazelcastObjectType) source).getTypeName() + .equals(((HazelcastObjectType) target).getTypeName()); } assert rowElement instanceof SqlCall : "Row Element must be an SqlCall"; diff --git a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/validate/types/HazelcastTypeUtils.java b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/validate/types/HazelcastTypeUtils.java index 0b00dd722eb8..756d4fb7ece5 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/validate/types/HazelcastTypeUtils.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/jet/sql/impl/validate/types/HazelcastTypeUtils.java @@ -132,7 +132,7 @@ public static QueryDataType toHazelcastType(RelDataType relDataType) { } private static QueryDataType convertHazelcastObjectType(final RelDataType relDataType) { - final HazelcastObjectType hazelcastObjectType = extractHzObjectType(relDataType); + final HazelcastObjectType hazelcastObjectType = (HazelcastObjectType) relDataType; final Map typeMap = new HashMap<>(); traverseHzObjectType(hazelcastObjectType, typeMap); @@ -149,8 +149,8 @@ private static void traverseHzObjectType(final HazelcastObjectType source, Map create(int index, QueryDataType type) { // like QueryDataType.VARCHAR_CHARACTER, are canonicalized to values of // some other canonical type, like QueryDataType.VARCHAR. That kind of // changes the observed type of a column to a canonical one. - if (type.getTypeFamily().equals(QueryDataTypeFamily.OBJECT)) { + if (type.getTypeFamily() == QueryDataTypeFamily.OBJECT) { return new ColumnExpression<>(index, type); } else { Class canonicalClass = type.getConverter().getNormalizedValueClass(); @@ -67,14 +67,13 @@ public Object evalTop(Row row, ExpressionEvalContext context) { // if the column expression is the top expression. Object res = row.get(index, false); if (res instanceof LazyTarget) { - assert type.equals(QueryDataType.OBJECT); + assert type.getTypeFamily() == QueryDataTypeFamily.OBJECT; LazyTarget lazyTarget = (LazyTarget) res; res = lazyTarget.getDeserialized() != null ? lazyTarget.getDeserialized() : lazyTarget.getSerialized(); } return res; } - @SuppressWarnings("unchecked") @Override public T eval(Row row, ExpressionEvalContext context) { // Lazy deserialization is disabled by default, and it has to be requested explicitly. @@ -86,7 +85,7 @@ public T eval(Row row, ExpressionEvalContext context, boolean useLazyDeserializa Object res = row.get(index, useLazyDeserialization); if (res instanceof LazyTarget) { - assert type.equals(QueryDataType.OBJECT); + assert type.getTypeFamily() == QueryDataTypeFamily.OBJECT; res = ((LazyTarget) res).deserialize(context.getSerializationService()); } diff --git a/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/expression/Expression.java b/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/expression/Expression.java index b8cb95509cea..ec0190eba116 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/expression/Expression.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/expression/Expression.java @@ -22,6 +22,7 @@ import com.hazelcast.sql.impl.row.Row; import com.hazelcast.sql.impl.type.QueryDataType; +import javax.annotation.concurrent.ThreadSafe; import java.io.Serializable; /** @@ -31,6 +32,7 @@ * * @param the return type of this expression. */ +@ThreadSafe public interface Expression extends IdentifiedDataSerializable, Serializable { /** * Evaluates this expression, guaranteeing that this is a top-level call. diff --git a/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/expression/FieldAccessExpression.java b/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/expression/FieldAccessExpression.java index b1bb52935021..8921429669e5 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/expression/FieldAccessExpression.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/expression/FieldAccessExpression.java @@ -17,6 +17,7 @@ package com.hazelcast.sql.impl.expression; import com.hazelcast.jet.sql.impl.JetSqlSerializerHook; +import com.hazelcast.jet.sql.impl.extract.AvroQueryTarget; import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.query.impl.getters.EvictableGetterCache; @@ -25,23 +26,22 @@ import com.hazelcast.sql.impl.QueryException; import com.hazelcast.sql.impl.row.Row; import com.hazelcast.sql.impl.type.QueryDataType; +import org.apache.avro.generic.GenericRecord; import java.io.IOException; -import java.util.concurrent.atomic.AtomicReference; /** * An expression backing the DOT operator for extracting field from a struct type. *

- * {@code ref.field} - extracts `field` from `ref`. + * {@code ref.field} extracts {@code field} from {@code ref}. */ public class FieldAccessExpression implements Expression { - // FAE can be potentially used for many sub-classes of the base class, but it will always use same getter + // FAE can be potentially used for many subclasses of the base class, but it will always use same getter. private static final int MAX_CLASS_COUNT = 10; private static final int MAX_GETTER_PER_CLASS_COUNT = 1; - // single instance for all calls to eval, used only during execution on particular node - // atomic reference due to serialization constraints - private final AtomicReference getterCache = new AtomicReference<>(); + // Single instance for all calls to eval, used only during execution on particular node. + private transient volatile GetterCache getterCache; private QueryDataType type; private String name; @@ -76,33 +76,31 @@ public T eval(Row row, ExpressionEvalContext context) { @Override public T eval(final Row row, final ExpressionEvalContext context, boolean useLazyDeserialization) { // Use lazy deserialization for nested queries. Only the last access should be eager. - final Object res = ref.eval(row, context, true); - if (res == null) { + final Object result = ref.eval(row, context, true); + if (result == null) { return null; } - if (isPrimitive(res.getClass())) { + if (isPrimitive(result.getClass())) { throw QueryException.error("Field Access expression can not be applied to primitive types"); } - if (getterCache.get() == null) { - getterCache.compareAndSet(null, new EvictableGetterCache( + if (getterCache == null) { + getterCache = new EvictableGetterCache( MAX_CLASS_COUNT, MAX_GETTER_PER_CLASS_COUNT, GetterCache.EVICTABLE_CACHE_EVICTION_PERCENTAGE, false - )); + ); } - // defensive check, should never happen - final GetterCache cache = getterCache.get(); - assert cache != null : "GetterCache should never be null"; - - final Extractors extractors = Extractors.newBuilder(context.getSerializationService()) - .setGetterCacheSupplier(() -> cache) - .build(); try { - return (T) type.convert(extractors.extract(res, name, useLazyDeserialization)); + Object value = result instanceof GenericRecord + ? AvroQueryTarget.extractValue((GenericRecord) result, name) + : Extractors.newBuilder(context.getSerializationService()) + .setGetterCacheSupplier(() -> getterCache) + .build().extract(result, name, useLazyDeserialization); + return (T) type.convert(value); } catch (Exception e) { throw QueryException.error("Failed to extract field"); } diff --git a/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/expression/predicate/ComparisonPredicate.java b/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/expression/predicate/ComparisonPredicate.java index 19691361406e..b751fc9d2d79 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/expression/predicate/ComparisonPredicate.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/expression/predicate/ComparisonPredicate.java @@ -51,7 +51,7 @@ private ComparisonPredicate(Expression left, Expression right, ComparisonM } public static ComparisonPredicate create(Expression left, Expression right, ComparisonMode comparisonMode) { - assert left.getType().equals(right.getType()); + assert left.getType().getTypeFamily() == right.getType().getTypeFamily(); return new ComparisonPredicate(left, right, comparisonMode); } diff --git a/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/extract/QueryPath.java b/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/extract/QueryPath.java index d953dac7a2d6..ff9163248b03 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/extract/QueryPath.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/extract/QueryPath.java @@ -68,7 +68,7 @@ public boolean isKey() { return key; } - public boolean isTop() { + public boolean isTopLevel() { return path == null; } diff --git a/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/schema/MappingField.java b/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/schema/MappingField.java index eb8e5ee73d18..285f192a9eb8 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/schema/MappingField.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/schema/MappingField.java @@ -89,8 +89,10 @@ public MappingField setType(QueryDataType type) { } /** - * The external name of a field. For example, in case of IMap or Kafka, - * it always starts with `__key` or `this`. + * Returns what is specified after {@code EXTERNAL NAME} keyword in the corresponding + * SQL mapping column. The interpretation of external name is up to the connector. + * For example, in IMap and Kafka, the external name is fully-qualified: it starts with + * {@code __key.} or {@code this.}. */ public String externalName() { return (String) properties.get(EXTERNAL_NAME); diff --git a/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/schema/TableField.java b/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/schema/TableField.java index 58b06d36cc2f..ab16a85a2a23 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/schema/TableField.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/schema/TableField.java @@ -18,6 +18,8 @@ import com.hazelcast.sql.impl.type.QueryDataType; +import java.util.Objects; + /** * Base class for all table fields. Different backends may have additional * metadata associated with the field. @@ -51,23 +53,15 @@ public boolean equals(Object o) { if (this == o) { return true; } - if (o == null || getClass() != o.getClass()) { return false; } - - TableField field = (TableField) o; - - return name.equals(field.name) && type.equals(field.type) && hidden == field.hidden; + TableField that = (TableField) o; + return name.equals(that.name) && type.equals(that.type) && hidden == that.hidden; } @Override public int hashCode() { - int result = name.hashCode(); - - result = 31 * result + type.hashCode(); - result = 31 * result + (hidden ? 1 : 0); - - return result; + return Objects.hash(name, type, hidden); } } diff --git a/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/schema/type/Type.java b/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/schema/type/Type.java index dd4ebc6fb034..683ae5750c44 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/schema/type/Type.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/schema/type/Type.java @@ -16,12 +16,14 @@ package com.hazelcast.sql.impl.schema.type; -import com.google.common.collect.ImmutableMap; +import com.hazelcast.internal.serialization.impl.SerializationUtil; import com.hazelcast.jet.sql.impl.parse.SqlCreateType; +import com.hazelcast.jet.sql.impl.schema.TypeDefinitionColumn; import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; import com.hazelcast.sql.impl.SqlDataSerializerHook; +import com.hazelcast.sql.impl.schema.Mapping; import com.hazelcast.sql.impl.schema.SqlCatalogObject; import com.hazelcast.sql.impl.type.QueryDataType; import com.hazelcast.sql.impl.type.QueryDataTypeFamily; @@ -32,27 +34,27 @@ import javax.annotation.Nonnull; import java.io.IOException; import java.io.Serializable; -import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; +import static java.util.stream.Collectors.toList; + /** * A class stored in the SQL catalog to represent a type created using the * CREATE TYPE command. - *

- * It can represent a java class, or a portable/compact type, see {@link #kind}. */ public class Type implements Serializable, SqlCatalogObject { private String name; - private TypeKind kind = TypeKind.JAVA; - private String javaClassName; - private String compactTypeName; - private Integer portableFactoryId; - private Integer portableClassId; - private Integer portableVersion; private List fields; + private Map options; + + public Type() { } - public Type() { + public Type(String name, List columns, Map options) { + this.name = name; + this.fields = columns.stream().map(column -> new TypeField(column.name(), column.type())).collect(toList()); + this.options = options; } @Override @@ -60,88 +62,16 @@ public String name() { return name; } - public void setName(final String name) { - this.name = name; - } - - public TypeKind getKind() { - return kind; - } - - public void setKind(final TypeKind kind) { - this.kind = kind; - } - - public String getJavaClassName() { - return javaClassName; - } - - public void setJavaClassName(final String javaClassName) { - this.javaClassName = javaClassName; - } - public List getFields() { return fields; } - public void setFields(final List fields) { + public void setFields(List fields) { this.fields = fields; } - public Integer getPortableFactoryId() { - return portableFactoryId; - } - - public void setPortableFactoryId(final Integer portableFactoryId) { - this.portableFactoryId = portableFactoryId; - } - - public Integer getPortableClassId() { - return portableClassId; - } - - public void setPortableClassId(final Integer portableClassId) { - this.portableClassId = portableClassId; - } - - public Integer getPortableVersion() { - return portableVersion; - } - - public void setPortableVersion(final Integer portableVersion) { - this.portableVersion = portableVersion; - } - - public String getCompactTypeName() { - return compactTypeName; - } - - public void setCompactTypeName(final String compactTypeName) { - this.compactTypeName = compactTypeName; - } - public Map options() { - if (javaClassName != null) { - return ImmutableMap.of( - "format", "java", - "javaClass", javaClassName); - } - - if (compactTypeName != null) { - return ImmutableMap.of( - "format", "compact", - "compactTypeName", compactTypeName); - } - - if (portableFactoryId != null) { - return ImmutableMap.of( - "format", "portable", - "portableFactoryId", String.valueOf(portableFactoryId), - "portableClassId", String.valueOf(portableClassId), - "portableClassVersion", String.valueOf(portableVersion != null ? portableVersion : 0)); - } - - throw new AssertionError("unexpected state"); + return Collections.unmodifiableMap(options); } @Override @@ -151,56 +81,17 @@ public String unparse() { } @Override - public void writeData(final ObjectDataOutput out) throws IOException { + public void writeData(ObjectDataOutput out) throws IOException { out.writeString(name); - out.writeInt(kind.ordinal()); - switch (kind) { - case JAVA: - out.writeString(javaClassName); - break; - case PORTABLE: - out.writeInt(portableFactoryId); - out.writeInt(portableClassId); - out.writeInt(portableVersion); - break; - case COMPACT: - out.writeString(compactTypeName); - break; - default: - throw new UnsupportedOperationException("Unsupported Type Kind: " + kind); - } - - out.writeInt(fields.size()); - for (final TypeField field : fields) { - out.writeObject(field); - } + SerializationUtil.writeList(fields, out); + SerializationUtil.writeMap(options, out); } @Override - public void readData(final ObjectDataInput in) throws IOException { - this.name = in.readString(); - this.kind = TypeKind.values()[in.readInt()]; - switch (kind) { - case JAVA: - this.javaClassName = in.readString(); - break; - case PORTABLE: - this.portableFactoryId = in.readInt(); - this.portableClassId = in.readInt(); - this.portableVersion = in.readInt(); - break; - case COMPACT: - this.compactTypeName = in.readString(); - break; - default: - throw new UnsupportedOperationException("Unsupported Type Kind: " + kind); - } - - final int size = in.readInt(); - this.fields = new ArrayList<>(); - for (int i = 0; i < size; i++) { - this.fields.add(in.readObject()); - } + public void readData(ObjectDataInput in) throws IOException { + name = in.readString(); + fields = SerializationUtil.readList(in); + options = SerializationUtil.readMap(in); } @Override @@ -210,49 +101,43 @@ public int getClassId() { public static class TypeField implements IdentifiedDataSerializable, Serializable { private String name; - private QueryDataType queryDataType; - - public TypeField() { - } - - public TypeField(final String name, final QueryDataType queryDataType) { + /** + * A predefined type or a custom type without kind and metadata. + *

+ * Type kind indicates the serialization format, which is inherited from + * mapping. Type metadata stores schema information, which is reconstructed + * from {@link Mapping} and {@link Type} options each time a new mapping is created. + */ + private QueryDataType type; + + public TypeField() { } + + public TypeField(String name, QueryDataType type) { this.name = name; - this.queryDataType = queryDataType; + this.type = type; } public String getName() { return name; } - public void setName(final String name) { - this.name = name; - } - - public QueryDataType getQueryDataType() { - return queryDataType; - } - - public void setQueryDataType(final QueryDataType queryDataType) { - this.queryDataType = queryDataType; + public QueryDataType getType() { + return type; } @Override - public void writeData(final ObjectDataOutput out) throws IOException { + public void writeData(ObjectDataOutput out) throws IOException { out.writeString(name); - out.writeInt(queryDataType == null ? -1 : queryDataType.getConverter().getId()); - out.writeString(queryDataType == null ? "" : queryDataType.getObjectTypeName()); + out.writeInt(type.getConverter().getId()); + out.writeString(type.getObjectTypeName()); } @Override - public void readData(final ObjectDataInput in) throws IOException { - this.name = in.readString(); - final int converterId = in.readInt(); - final String typeName = in.readString(); - final Converter converter = Converters.getConverter(converterId); - - // TODO: is this the correct type kind? (NONE). Maybe worth writing it too. - this.queryDataType = converter.getTypeFamily().equals(QueryDataTypeFamily.OBJECT) - && ((typeName != null && !typeName.isEmpty())) + public void readData(ObjectDataInput in) throws IOException { + name = in.readString(); + Converter converter = Converters.getConverter(in.readInt()); + String typeName = in.readString(); + type = converter.getTypeFamily() == QueryDataTypeFamily.OBJECT && typeName != null ? new QueryDataType(typeName) : QueryDataTypeUtils.resolveTypeForClass(converter.getValueClass()); } diff --git a/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/schema/type/TypeKind.java b/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/schema/type/TypeKind.java index 9ead92fb12fd..84c432abb917 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/schema/type/TypeKind.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/schema/type/TypeKind.java @@ -20,7 +20,8 @@ public enum TypeKind { NONE(0), JAVA(1), PORTABLE(2), - COMPACT(3); + COMPACT(3), + AVRO(4); private final int value; diff --git a/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/type/QueryDataType.java b/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/type/QueryDataType.java index 1c5a4adb3bd2..a7234ee59717 100644 --- a/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/type/QueryDataType.java +++ b/hazelcast-sql/src/main/java/com/hazelcast/sql/impl/type/QueryDataType.java @@ -55,6 +55,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; /** * Data type represents a type of concrete expression which is based on some basic data type. @@ -62,13 +63,8 @@ * Java serialization is needed for Jet. */ public class QueryDataType implements IdentifiedDataSerializable, Serializable { - public static final int MAX_DECIMAL_PRECISION = 76; public static final int MAX_DECIMAL_SCALE = 38; - public static final int OBJECT_TYPE_KIND_NONE = TypeKind.NONE.value(); - public static final int OBJECT_TYPE_KIND_JAVA = TypeKind.JAVA.value(); - public static final int OBJECT_TYPE_KIND_PORTABLE = TypeKind.PORTABLE.value(); - public static final int OBJECT_TYPE_KIND_COMPACT = TypeKind.COMPACT.value(); public static final QueryDataType VARCHAR = new QueryDataType(StringConverter.INSTANCE); public static final QueryDataType VARCHAR_CHARACTER = new QueryDataType(CharacterConverter.INSTANCE); @@ -105,26 +101,23 @@ public class QueryDataType implements IdentifiedDataSerializable, Serializable { public static final QueryDataType ROW = new QueryDataType(RowConverter.INSTANCE); private Converter converter; - // never empty for custom types (nested types) - private String objectTypeName = ""; - private Integer objectTypeKind = OBJECT_TYPE_KIND_NONE; - private List objectFields = new ArrayList<>(); - private String objectTypeMetadata = ""; - - public QueryDataType() { - // No-op. - } + // nonnull for custom types (nested types) + private String objectTypeName; + private TypeKind objectTypeKind = TypeKind.NONE; + private String objectTypeMetadata; + private final List objectFields = new ArrayList<>(); + + public QueryDataType() { } public QueryDataType(String objectTypeName) { - this.converter = ObjectConverter.INSTANCE; - this.objectTypeKind = OBJECT_TYPE_KIND_NONE; - this.objectTypeName = objectTypeName; + this(objectTypeName, TypeKind.NONE, null); } - public QueryDataType(String objectTypeName, int typeKind) { - this.converter = ObjectConverter.INSTANCE; + public QueryDataType(String objectTypeName, TypeKind typeKind, String typeMetadata) { + converter = ObjectConverter.INSTANCE; this.objectTypeName = objectTypeName; - this.objectTypeKind = typeKind; + objectTypeKind = typeKind; + objectTypeMetadata = typeMetadata; } QueryDataType(Converter converter) { @@ -143,10 +136,6 @@ public String getObjectTypeMetadata() { return objectTypeMetadata; } - public void setObjectTypeMetadata(final String objectTypeMetadata) { - this.objectTypeMetadata = objectTypeMetadata; - } - public QueryDataTypeFamily getTypeFamily() { return converter.getTypeFamily(); } @@ -155,14 +144,10 @@ public Converter getConverter() { return converter; } - public Integer getObjectTypeKind() { + public TypeKind getObjectTypeKind() { return objectTypeKind; } - public void setObjectTypeKind(final Integer objectTypeKind) { - this.objectTypeKind = objectTypeKind; - } - /** * Normalize the given value to a value returned by this instance. If the value doesn't match * the type expected by the converter, an exception is thrown. @@ -222,49 +207,55 @@ public int getClassId() { return SqlDataSerializerHook.QUERY_DATA_TYPE; } + /** + * @implNote Collects all distinct custom types into a type map beforehand + * to avoid infinite recursion. Then, it writes each type with its direct children, + * i.e. each subtree, in an arbitrary order. {@link #readData} first creates a type + * map that initially contains only this {@code QueryDataType}, i.e. the root. Then, + * it reads all subtrees by creating a type only if it is not created before using + * the type map. Even though subtrees don't lie in a particular order, the children + * of all subtrees will eventually be populated, including the root. + */ @Override public void writeData(ObjectDataOutput out) throws IOException { out.writeInt(converter.getId()); - // TODO this needs to be backwards-compatible, it's stored as a part of MappingField in the catalog - if (!converter.getTypeFamily().equals(QueryDataTypeFamily.OBJECT)) { + if (converter.getTypeFamily() != QueryDataTypeFamily.OBJECT) { return; } writeObjectTypeMetadata(this, out); - if (!isCustomType()) { return; } - final Map nestedTypes = new HashMap<>(); - collectNestedTypes(this, nestedTypes); + Map typeMap = new HashMap<>(); + collectCustomTypes(this, typeMap); - out.writeInt(nestedTypes.size()); - for (final QueryDataType nestedType : nestedTypes.values()) { + out.writeInt(typeMap.size()); + for (QueryDataType nestedType : typeMap.values()) { writeObjectTypeMetadata(nestedType, out); out.writeInt(nestedType.getObjectFields().size()); - for (final QueryDataTypeField field : nestedType.getObjectFields()) { + for (QueryDataTypeField field : nestedType.getObjectFields()) { out.writeString(field.name); - out.writeInt(field.dataType.converter.getId()); - writeObjectTypeMetadata(field.dataType, out); + out.writeInt(field.type.converter.getId()); + writeObjectTypeMetadata(field.type, out); } } } - private static void writeObjectTypeMetadata(final QueryDataType queryDataType, ObjectDataOutput out) - throws IOException { - out.writeInt(queryDataType.objectTypeKind); - out.writeString(queryDataType.objectTypeName); - out.writeString(queryDataType.objectTypeMetadata); + private static void writeObjectTypeMetadata(QueryDataType type, ObjectDataOutput out) throws IOException { + out.writeInt(type.objectTypeKind.value()); + out.writeString(type.objectTypeName); + out.writeString(type.objectTypeMetadata); } - private void collectNestedTypes(final QueryDataType dataType, final Map collected) { - collected.putIfAbsent(dataType.objectTypeName, dataType); + private void collectCustomTypes(QueryDataType type, Map typeMap) { + typeMap.put(type.objectTypeName, type); - for (final QueryDataTypeField field : dataType.objectFields) { - if (field.getDataType().isCustomType()) { - if (!collected.containsKey(field.dataType.objectTypeName)) { - collectNestedTypes(field.dataType, collected); + for (QueryDataTypeField field : type.objectFields) { + if (field.getType().isCustomType()) { + if (!typeMap.containsKey(field.type.objectTypeName)) { + collectCustomTypes(field.type, typeMap); } } } @@ -272,82 +263,74 @@ private void collectNestedTypes(final QueryDataType dataType, final Map(); - final int typeMapSize = in.readInt(); - final Map nestedTypes = new HashMap<>(); - nestedTypes.put(objectTypeName, this); + Map typeMap = new HashMap<>(); + typeMap.put(objectTypeName, this); + int typeMapSize = in.readInt(); for (int i = 0; i < typeMapSize; i++) { - final Integer currentTypeKind = in.readInt(); - final String currentTypeName = in.readString(); - final String currentTypeMetadata = in.readString(); - - final int fieldsSize = in.readInt(); - final QueryDataType currentType = nestedTypes.computeIfAbsent(currentTypeName, s -> { - // TODO: simplify? - final QueryDataType fieldType = new QueryDataType(currentTypeName); - fieldType.setObjectTypeMetadata(currentTypeMetadata); - fieldType.setObjectTypeKind(currentTypeKind); - return fieldType; - }); - - for (int j = 0; j < fieldsSize; j++) { - final String fieldName = in.readString(); - final int fieldConverterId = in.readInt(); - final int fieldTypeKind = in.readInt(); - final String fieldTypeName = in.readString(); - final String fieldTypeMetadata = in.readString(); - - if (fieldConverterId == QueryDataType.OBJECT.getConverter().getId() - && (fieldTypeName != null && !fieldTypeName.isEmpty())) { - currentType.getObjectFields().add(new QueryDataTypeField(fieldName, - nestedTypes.computeIfAbsent(fieldTypeName, s -> { - // TODO: simplify? - final QueryDataType fieldType = new QueryDataType(fieldTypeName); - fieldType.setObjectTypeMetadata(fieldTypeMetadata); - fieldType.setObjectTypeKind(fieldTypeKind); - return fieldType; - }))); - } else { - final QueryDataType fieldDataType = QueryDataTypeUtils - .resolveTypeForClass(Converters.getConverter(fieldConverterId).getValueClass()); - currentType.getObjectFields().add(new QueryDataTypeField(fieldName, fieldDataType)); - } + QueryDataType type = readNestedType(OBJECT.getConverter(), in, typeMap); + int fields = in.readInt(); + for (int j = 0; j < fields; j++) { + String fieldName = in.readString(); + Converter converter = Converters.getConverter(in.readInt()); + QueryDataType nestedType = readNestedType(converter, in, typeMap); + type.getObjectFields().add(new QueryDataTypeField(fieldName, nestedType)); } } } + private static void readObjectTypeMetadata(QueryDataType type, ObjectDataInput in) throws IOException { + type.objectTypeKind = TypeKind.of(in.readInt()); + type.objectTypeName = in.readString(); + type.objectTypeMetadata = in.readString(); + } + + private static QueryDataType readNestedType(Converter converter, ObjectDataInput in, + Map typeMap) throws IOException { + QueryDataType type = new QueryDataType(converter); + readObjectTypeMetadata(type, in); + + return type.objectTypeName == null + ? QueryDataTypeUtils.resolveTypeForClass(converter.getValueClass()) + : typeMap.computeIfAbsent(type.objectTypeName, k -> type); + } + + public boolean isCustomType() { + return converter.getTypeFamily() == QueryDataTypeFamily.OBJECT && objectTypeName != null; + } + @Override public int hashCode() { - return 251 * converter.getId(); + return !isCustomType() + ? converter.getId() + : Objects.hash(objectTypeName, objectTypeKind, objectTypeMetadata); } @Override public boolean equals(Object o) { - // TODO: proper equals if (this == o) { return true; } - if (o == null || getClass() != o.getClass()) { return false; } - - QueryDataType type = (QueryDataType) o; - - return converter.getId() == type.converter.getId(); + QueryDataType that = (QueryDataType) o; + return !isCustomType() + ? converter.getId() == that.converter.getId() + : objectTypeName.equals(that.objectTypeName) + && objectTypeKind == that.objectTypeKind + && Objects.equals(objectTypeMetadata, that.objectTypeMetadata) + && objectFields.equals(that.objectFields); } @Override @@ -355,48 +338,35 @@ public String toString() { return getClass().getSimpleName() + " {family=" + getTypeFamily() + "}"; } - public boolean isCustomType() { - return this.converter.getTypeFamily().equals(QueryDataTypeFamily.OBJECT) && (!this.objectTypeName.isEmpty()); - } - public static class QueryDataTypeField implements IdentifiedDataSerializable, Serializable { private String name; - private QueryDataType dataType; + private QueryDataType type; - public QueryDataTypeField() { - } + public QueryDataTypeField() { } - public QueryDataTypeField(final String name, final QueryDataType dataType) { + public QueryDataTypeField(String name, QueryDataType type) { this.name = name; - this.dataType = dataType; + this.type = type; } public String getName() { return name; } - public void setName(final String name) { - this.name = name; - } - - public QueryDataType getDataType() { - return dataType; - } - - public void setDataType(final QueryDataType dataType) { - this.dataType = dataType; + public QueryDataType getType() { + return type; } @Override - public void writeData(final ObjectDataOutput out) throws IOException { + public void writeData(ObjectDataOutput out) throws IOException { out.writeString(name); - out.writeObject(dataType); + out.writeObject(type); } @Override - public void readData(final ObjectDataInput in) throws IOException { - this.name = in.readString(); - this.dataType = in.readObject(); + public void readData(ObjectDataInput in) throws IOException { + name = in.readString(); + type = in.readObject(); } @Override @@ -408,5 +378,22 @@ public int getFactoryId() { public int getClassId() { return SqlDataSerializerHook.QUERY_DATA_TYPE_FIELD; } + + @Override + public int hashCode() { + return Objects.hash(name, type); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + QueryDataTypeField that = (QueryDataTypeField) o; + return name.equals(that.name) && type.equals(that.type); + } } } diff --git a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/SqlTestSupport.java b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/SqlTestSupport.java index dd0aa55ee956..bce8a83a5df8 100644 --- a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/SqlTestSupport.java +++ b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/SqlTestSupport.java @@ -28,6 +28,9 @@ import com.hazelcast.jet.core.test.TestSupport; import com.hazelcast.jet.impl.util.Util; import com.hazelcast.jet.sql.impl.JetSqlSerializerHook; +import com.hazelcast.jet.sql.impl.connector.SqlConnector; +import com.hazelcast.jet.sql.impl.connector.file.FileSqlConnector; +import com.hazelcast.jet.sql.impl.connector.kafka.KafkaSqlConnector; import com.hazelcast.jet.sql.impl.connector.map.IMapSqlConnector; import com.hazelcast.logging.ILogger; import com.hazelcast.logging.Logger; @@ -46,9 +49,14 @@ import com.hazelcast.sql.impl.expression.ExpressionEvalContext; import com.hazelcast.sql.impl.plan.cache.PlanCache; import com.hazelcast.sql.impl.row.JetSqlRow; +import com.hazelcast.sql.impl.schema.Mapping; +import com.hazelcast.sql.impl.schema.MappingField; +import com.hazelcast.sql.impl.type.QueryDataType; +import com.hazelcast.sql.impl.type.QueryDataType.QueryDataTypeField; import com.hazelcast.test.Accessors; import com.hazelcast.test.annotation.ParallelJVMTest; import com.hazelcast.test.annotation.QuickTest; +import org.apache.avro.Schema; import org.junit.After; import org.junit.experimental.categories.Category; @@ -63,9 +71,11 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.Deque; import java.util.HashMap; import java.util.Iterator; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -88,8 +98,8 @@ import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_FACTORY_ID; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_FORMAT; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.PORTABLE_FORMAT; +import static com.hazelcast.jet.sql.impl.schema.TypeUtils.FieldEnricher.plainExternalName; import static com.hazelcast.sql.impl.ResultIterator.HasNextResult.YES; -import static java.lang.String.format; import static java.util.Arrays.asList; import static java.util.Collections.emptyList; import static java.util.stream.Collectors.joining; @@ -102,7 +112,6 @@ @Category({QuickTest.class, ParallelJVMTest.class}) public abstract class SqlTestSupport extends SimpleTestInClusterSupport { - private static final ILogger SUPPORT_LOGGER = Logger.getLogger(SqlTestSupport.class); @After @@ -119,52 +128,6 @@ public void tearDown() { } } - public static void setupCompactTypesForNestedQuery(HazelcastInstance instance) { - instance.getSql().execute("CREATE TYPE Office (" - + "id BIGINT, " - + "name VARCHAR " - + ") OPTIONS ('format'='compact', 'compactTypeName'='OfficeCompactType')"); - - instance.getSql().execute("CREATE TYPE Organization (" - + "id BIGINT, " - + "name VARCHAR, " - + "office Office" - + ") OPTIONS ('format'='compact', 'compactTypeName'='OrganizationCompactType')"); - - instance.getSql().execute( - "CREATE MAPPING test (" - + "__key BIGINT," - + "id BIGINT, " - + "name VARCHAR, " - + "organization Organization" - + ")" - + "TYPE IMap " - + "OPTIONS (" - + "'keyFormat'='bigint'," - + "'valueFormat'='compact'," - + "'valueCompactTypeName'='UserCompactType'" - + ")"); - } - - public static void setupPortableTypesForNestedQuery(HazelcastInstance instance) { - instance.getSql().execute("CREATE TYPE Office OPTIONS " - + "('format'='portable', 'portableFactoryId'='1', 'portableClassId'='3', 'portableClassVersion'='0')"); - instance.getSql().execute("CREATE TYPE Organization OPTIONS " - + "('format'='portable', 'portableFactoryId'='1', 'portableClassId'='2', 'portableClassVersion'='0')"); - - instance.getSql().execute("CREATE MAPPING test (" - + "__key BIGINT, " - + "id BIGINT, " - + "name VARCHAR, " - + "organization Organization " - + ") TYPE IMap " - + "OPTIONS (" - + "'keyFormat'='bigint', " - + "'valueFormat'='portable', " - + "'valuePortableFactoryId'='1', " - + "'valuePortableClassId'='1')"); - } - /** * Execute a query and assert that it eventually returns the expected entries. * @@ -531,14 +494,6 @@ public static void createMapping(String name, Class keyClass, Class valueC createMapping(instance(), name, keyClass, valueClass); } - public static void createType(String name, Class clazz) { - createType(instance(), name, clazz); - } - - public static void createType(HazelcastInstance instance, String name, Class clazz) { - instance.getSql().execute(format("CREATE TYPE \"%s\" OPTIONS ('format'='java', 'javaClass'='%s')", name, clazz.getName())); - } - /** * Create an IMap mapping with the given {@code name} that uses * java serialization for both key and value with the given classes. @@ -830,49 +785,187 @@ protected static Object[] row(Object... values) { return values; } - public static class SqlMapping { - public final String name; + /** + * Inserts the specified record into the given mapping by converting values + * into string and using row syntax {@code (...)} for nested fields. + */ + public static void insertLiterals(HazelcastInstance instance, String mapping, Object... values) { + instance.getSql().execute("INSERT INTO " + mapping + " VALUES (" + + Arrays.stream(values).map(SqlTestSupport::toSQL).collect(joining(", ")) + ")"); + } + + /** + * Inserts the specified record into the given mapping by passing values + * as dynamic parameters ({@code ?}). + */ + public static void insertParams(HazelcastInstance instance, String mapping, Object... values) { + instance.getSql().execute("INSERT INTO " + mapping + " VALUES (" + + String.join(", ", Collections.nCopies(values.length, "?")) + ")", values); + } + + public static String toSQL(Object value) { + if (value instanceof Object[]) { + return "(" + Arrays.stream((Object[]) value).map(SqlTestSupport::toSQL).collect(joining(", ")) + ")"; + } + return value == null || value instanceof Boolean || value instanceof Number + ? String.valueOf(value) : "'" + value + "'"; + } + + public static class SqlMapping extends SqlStructure { + private static final Map, String> TYPES = Map.of( + FileSqlConnector.class, FileSqlConnector.TYPE_NAME, + IMapSqlConnector.class, IMapSqlConnector.TYPE_NAME, + KafkaSqlConnector.class, KafkaSqlConnector.TYPE_NAME + ); + public String externalName; public final String type; + + public SqlMapping(String name, Class connector) { + super(name); + type = "TYPE " + TYPES.get(connector); + } + + public SqlMapping(String name, String dataConnection) { + super(name); + type = "DATA CONNECTION " + dataConnection; + } + + public SqlMapping externalName(String externalName) { + this.externalName = externalName; + return this; + } + + @Override + protected void create(HazelcastInstance instance, boolean replace, boolean ifNotExists) { + instance.getSql().execute("CREATE " + (replace ? "OR REPLACE " : "") + "MAPPING " + + (ifNotExists ? "IF NOT EXISTS " : "") + name + " " + + (externalName != null ? "EXTERNAL NAME " + externalName + " " : "") + + (fields.isEmpty() ? "" : "(" + String.join(", ", fields) + ") ") + + type + " OPTIONS (" + options.entrySet().stream() + .map(e -> "'" + e.getKey() + "'='" + e.getValue() + "'").collect(joining(", ")) + + ")"); + } + + public Type toTypeTree() { + return new Type(sqlServiceImpl(instance()).getOptimizer().relationsStorage().getMapping(name)); + } + } + + public static class SqlType extends SqlStructure { + public SqlType(String name) { + super(name); + } + + @Override + protected void create(HazelcastInstance instance, boolean replace, boolean ifNotExists) { + instance.getSql().execute("CREATE " + (replace ? "OR REPLACE " : "") + "TYPE " + + (ifNotExists ? "IF NOT EXISTS " : "") + name + + (fields.isEmpty() ? "" : " (" + String.join(", ", fields) + ")") + + (options.isEmpty() ? "" : " OPTIONS (" + options.entrySet().stream() + .map(e -> "'" + e.getKey() + "'='" + e.getValue() + "'").collect(joining(", ")) + + ")")); + } + } + + private abstract static class SqlStructure> { + public final String name; public final List fields = new ArrayList<>(); - public final Map options = new HashMap<>(); + public final Map options = new LinkedHashMap<>(); - public SqlMapping(String name, String type) { + SqlStructure(String name) { this.name = name; - this.type = type; } - public SqlMapping fields(String... fields) { + public T fields(String... fields) { this.fields.addAll(asList(fields)); - return this; + return me(); + } + + public T fieldsIf(boolean condition, String... fields) { + return condition ? fields(fields) : me(); } - public SqlMapping options(Object... options) { + public T options(Object... options) { for (int i = 0; i < options.length / 2; i++) { this.options.put(options[2 * i], options[2 * i + 1]); } - return this; + return me(); + } + + public T optionsIf(boolean condition, Object... options) { + return condition ? options(options) : me(); } - public SqlMapping optionsIf(boolean condition, Object... options) { - return condition ? options(options) : this; + @SuppressWarnings("unchecked") + private T me() { + return (T) this; } public void create() { - create(instance(), false); + create(instance()); + } + + public void create(HazelcastInstance instance) { + create(instance, false, false); } public void createOrReplace() { - create(instance(), true); + createOrReplace(instance()); + } + + public void createOrReplace(HazelcastInstance instance) { + create(instance, true, false); + } + + public void createIfNotExists() { + createIfNotExists(instance()); } - protected void create(HazelcastInstance instance, boolean replace) { - instance.getSql().execute("CREATE " + (replace ? "OR REPLACE " : "") + "MAPPING " + name - + (fields.isEmpty() ? " " : "(" + String.join(",", fields) + ") ") - + "TYPE " + type + " " - + "OPTIONS (" + options.entrySet().stream() - .map(e -> "'" + e.getKey() + "'='" + e.getValue() + "'").collect(joining(",")) - + ")" - ); + public void createIfNotExists(HazelcastInstance instance) { + create(instance, false, true); + } + + protected abstract void create(HazelcastInstance instance, boolean replace, boolean ifNotExists); + } + + public static class Type { + public final String name; + public final Field[] fields; + + public Type(Mapping mapping) { + name = mapping.name(); + fields = mapping.fields().stream().map(Field::new).toArray(Field[]::new); + } + + public Type(QueryDataType type) { + name = type.getObjectTypeName(); + fields = type.getObjectFields().stream().map(Field::new).toArray(Field[]::new); + } + + public Type(Schema schema) { + name = schema.getName(); + fields = schema.getType() == Schema.Type.RECORD + ? schema.getFields().stream().map(Field::new).toArray(Field[]::new) : null; + } + + public static class Field { + public final String name; + public final Type type; + + public Field(MappingField field) { + name = plainExternalName(field); + type = new Type(field.type()); + } + + public Field(QueryDataTypeField field) { + name = field.getName(); + type = new Type(field.getType()); + } + + public Field(Schema.Field field) { + name = field.name(); + type = new Type(field.schema()); + } } } diff --git a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/file/FileUtil.java b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/file/FileUtil.java index c96be35540ff..70a27a4ec5e9 100644 --- a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/file/FileUtil.java +++ b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/file/FileUtil.java @@ -16,7 +16,6 @@ package com.hazelcast.jet.sql.impl.connector.file; -import com.hazelcast.jet.impl.util.ExceptionUtil; import org.apache.avro.SchemaBuilder; import org.apache.avro.file.DataFileWriter; import org.apache.avro.generic.GenericData.Record; @@ -35,6 +34,7 @@ import java.nio.file.Files; import static com.hazelcast.internal.util.ExceptionUtil.sneakyThrow; +import static com.hazelcast.jet.impl.util.Util.reduce; final class FileUtil { @@ -74,9 +74,8 @@ final class FileUtil { .set("object", new GenericRecordBuilder(SchemaBuilder.record("object").fields().endRecord()).build()) .build(); - static final Record AVRO_NULLABLE_RECORD = - AVRO_RECORD.getSchema().getFields().stream().collect( - () -> new GenericRecordBuilder(SchemaBuilder.record("name") + static final Record AVRO_NULLABLE_RECORD = reduce( + new GenericRecordBuilder(SchemaBuilder.record("name") .fields() .name("string").type().nullable().stringType().noDefault() .name("boolean").type().nullable().booleanType().noDefault() @@ -94,16 +93,15 @@ final class FileUtil { .name("null").type().nullable().record("nul").fields().endRecord().noDefault() .name("object").type().nullable().record("object").fields().endRecord().noDefault() .endRecord()), - (builder, field) -> builder.set(field, AVRO_RECORD.get(field.pos())), - ExceptionUtil::combinerUnsupported - ).build(); + AVRO_RECORD.getSchema().getFields().stream(), + (record, field) -> record.set(field, AVRO_RECORD.get(field.pos())) + ).build(); - static final Record AVRO_NULL_RECORD = - AVRO_NULLABLE_RECORD.getSchema().getFields().stream().collect( - () -> new GenericRecordBuilder(AVRO_NULLABLE_RECORD.getSchema()), - (builder, field) -> builder.set(field, null), - ExceptionUtil::combinerUnsupported - ).build(); + static final Record AVRO_NULL_RECORD = reduce( + new GenericRecordBuilder(AVRO_NULLABLE_RECORD.getSchema()), + AVRO_NULLABLE_RECORD.getSchema().getFields().stream(), + (record, field) -> record.set(field, null) + ).build(); private static final Record PARQUET_RECORD = new GenericRecordBuilder(SchemaBuilder.record("name") diff --git a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/file/SqlAvroTest.java b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/file/SqlAvroTest.java index a6abeec769e0..ba1b9fa0c674 100644 --- a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/file/SqlAvroTest.java +++ b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/file/SqlAvroTest.java @@ -55,7 +55,7 @@ public static void setUpClass() { } private static SqlMapping fileMapping(String name, File file) { - return new SqlMapping(name, FileSqlConnector.TYPE_NAME).options( + return new SqlMapping(name, FileSqlConnector.class).options( OPTION_FORMAT, AVRO_FORMAT, OPTION_PATH, file.getParent(), OPTION_GLOB, file.getName() diff --git a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/infoschema/SqlInfoSchemaTest.java b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/infoschema/SqlInfoSchemaTest.java index d204363c7550..d85d4ebf7237 100644 --- a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/infoschema/SqlInfoSchemaTest.java +++ b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/infoschema/SqlInfoSchemaTest.java @@ -41,7 +41,6 @@ * Tests for the {@code information_schema}. */ public class SqlInfoSchemaTest extends SqlTestSupport { - private static final String LE = System.lineSeparator(); private static SqlService sqlService; @@ -53,7 +52,7 @@ public class SqlInfoSchemaTest extends SqlTestSupport { private final String mappingExternalName = "my_map"; @BeforeClass - public static void setUpClass() { + public static void initialize() { Config config = smallInstanceConfig() .setProperty(SQL_CUSTOM_TYPES_ENABLED.getName(), "true"); initialize(1, config); @@ -61,36 +60,30 @@ public static void setUpClass() { } @Before - public void setUp() { - sqlService.execute( - "CREATE MAPPING " + mappingName + " EXTERNAL NAME " + mappingExternalName + "(" - + "__key INT" - + ", __value VARCHAR EXTERNAL NAME \"this.value\"" - + ") TYPE " + IMapSqlConnector.TYPE_NAME + "\n" - + "OPTIONS (\n" - + '\'' + OPTION_KEY_FORMAT + "'='int'\n" - + ", '" + OPTION_VALUE_FORMAT + "'='" + JAVA_FORMAT + "'\n" - + ", '" + OPTION_VALUE_CLASS + "'='" + Value.class.getName() + "'\n" - + ")"); + public void setup() { + new SqlMapping(mappingName, IMapSqlConnector.class) + .externalName(mappingExternalName) + .fields("__key INT", + "__value VARCHAR EXTERNAL NAME \"this.value\"") + .options(OPTION_KEY_FORMAT, "int", + OPTION_VALUE_FORMAT, JAVA_FORMAT, + OPTION_VALUE_CLASS, Value.class.getName()) + .create(); + sqlService.execute("CREATE VIEW " + viewName + " AS SELECT * FROM " + mappingName); - sqlService.execute("CREATE TYPE " + firstTypeName + "(" - + "id BIGINT, " - + "name VARCHAR," - + "created TIMESTAMP WITH TIME ZONE," - + "balance DOUBLE" - + ") OPTIONS (" - + "'format'='compact'," - + "'compactTypeName'='" + firstTypeName + "'" - + ")"); - sqlService.execute("CREATE TYPE " + secondTypeName + "(" - + "id BIGINT, " - + "name VARCHAR, " - + "other " + firstTypeName - + ") OPTIONS (" - + "'format'='compact'," - + "'compactTypeName'='" + firstTypeName + "'" - + ")"); + new SqlType(firstTypeName) + .fields("id BIGINT", + "name VARCHAR", + "created TIMESTAMP WITH TIME ZONE", + "balance DOUBLE") + .create(); + + new SqlType(secondTypeName) + .fields("id BIGINT", + "name VARCHAR", + "other " + firstTypeName) + .create(); } @Test diff --git a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/kafka/KafkaSqlTestSupport.java b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/kafka/KafkaSqlTestSupport.java index 4a44e60c9fcd..085dcd7c7cb2 100644 --- a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/kafka/KafkaSqlTestSupport.java +++ b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/kafka/KafkaSqlTestSupport.java @@ -16,11 +16,15 @@ package com.hazelcast.jet.sql.impl.connector.kafka; +import com.hazelcast.config.Config; import com.hazelcast.jet.kafka.impl.KafkaTestSupport; import com.hazelcast.jet.sql.SqlTestSupport; import com.hazelcast.sql.SqlResult; import com.hazelcast.sql.SqlService; import io.confluent.kafka.schemaregistry.rest.SchemaRegistryConfig; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.GenericRecordBuilder; import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.StringDeserializer; @@ -32,6 +36,7 @@ import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_KEY_FORMAT; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_FORMAT; +import static com.hazelcast.jet.sql.impl.connector.file.AvroResolver.unwrapNullableType; import static org.assertj.core.api.Assertions.assertThat; public abstract class KafkaSqlTestSupport extends SqlTestSupport { @@ -40,7 +45,11 @@ public abstract class KafkaSqlTestSupport extends SqlTestSupport { @BeforeClass public static void setup() throws Exception { - initialize(1, null); + setup(1, null); + } + + protected static void setup(int memberCount, Config config) throws Exception { + initialize(memberCount, config); sqlService = instance().getSql(); kafkaTestSupport = KafkaTestSupport.create(); @@ -50,10 +59,10 @@ public static void setup() throws Exception { protected static void createSchemaRegistry() throws Exception { Properties properties = new Properties(); properties.put("listeners", "http://0.0.0.0:0"); - properties.put(SchemaRegistryConfig.KAFKASTORE_BOOTSTRAP_SERVERS_CONFIG, kafkaTestSupport.getBrokerConnectionString()); - //When Kafka is under load the schema registry may give - //io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException: Register operation timed out; error code: 50002 - //Because the default timeout is 500 ms. Use a bigger timeout value to avoid it + properties.put(SchemaRegistryConfig.KAFKASTORE_BOOTSTRAP_SERVERS_CONFIG, + kafkaTestSupport.getBrokerConnectionString()); + // We increase the timeout (default is 500 ms) because when Kafka is under load, + // the schema registry may give "RestClientException: Register operation timed out". properties.put(SchemaRegistryConfig.KAFKASTORE_TIMEOUT_CONFIG, "5000"); SchemaRegistryConfig config = new SchemaRegistryConfig(properties); kafkaTestSupport.createSchemaRegistry(config); @@ -73,6 +82,26 @@ protected static String createRandomTopic(int partitionCount) { return topicName; } + public static GenericRecord createRecord(Schema schema, Type.Field[] fields, Object[] values) { + GenericRecordBuilder record = new GenericRecordBuilder(schema); + for (int i = 0; i < fields.length; i++) { + Schema.Field field = schema.getField(fields[i].name); + if (values[i] == null) { + record.set(field, null); + } else { + Schema fieldSchema = unwrapNullableType(field.schema()); + record.set(field, fieldSchema.getType() == Schema.Type.RECORD + ? createRecord(fieldSchema, fields[i].type.fields, (Object[]) values[i]) + : values[i]); + } + } + return record.build(); + } + + public static GenericRecord createRecord(Schema schema, Object... values) { + return createRecord(schema, new Type(schema).fields, values); + } + protected static void createSqlKafkaDataConnection(String dlName, boolean isShared, String options) { try (SqlResult result = instance().getSql().execute( "CREATE DATA CONNECTION " + dlName + " TYPE Kafka " @@ -117,25 +146,6 @@ protected static String defaultNotSharedOptions() { kafkaTestSupport.getBrokerConnectionString()); } - protected static String constructDataConnectionOptions( - Class keySerializerClazz, - Class keyDeserializerClazz, - Class valueSerializerClazz, - Class valueDeserializerClazz) { - return String.format("OPTIONS ( " + - "'bootstrap.servers' = '%s', " + - "'key.serializer' = '%s', " + - "'key.deserializer' = '%s', " + - "'value.serializer' = '%s', " + - "'value.deserializer' = '%s', " + - "'auto.offset.reset' = 'earliest') ", - kafkaTestSupport.getBrokerConnectionString(), - keySerializerClazz.getCanonicalName(), - keyDeserializerClazz.getCanonicalName(), - valueSerializerClazz.getCanonicalName(), - valueDeserializerClazz.getCanonicalName()); - } - protected static String constructMappingOptions(String keyFormat, String valueFormat) { return "OPTIONS ('" + OPTION_KEY_FORMAT + "'='" + keyFormat + "'" + ", '" + OPTION_VALUE_FORMAT + "'='" + valueFormat + "')"; diff --git a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/kafka/PropertiesResolverTest.java b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/kafka/PropertiesResolverTest.java index 63d1c27272ee..2a2c4252861f 100644 --- a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/kafka/PropertiesResolverTest.java +++ b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/kafka/PropertiesResolverTest.java @@ -22,6 +22,8 @@ import io.confluent.kafka.serializers.KafkaAvroSerializer; import junitparams.JUnitParamsRunner; import junitparams.Parameters; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.serialization.DoubleDeserializer; @@ -40,12 +42,15 @@ import org.junit.runner.RunWith; import java.util.Map; +import java.util.Properties; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.AVRO_FORMAT; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.JAVA_FORMAT; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.JSON_FLAT_FORMAT; +import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_KEY_AVRO_SCHEMA; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_KEY_CLASS; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_KEY_FORMAT; +import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_AVRO_SCHEMA; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_CLASS; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_FORMAT; import static com.hazelcast.jet.sql.impl.connector.kafka.PropertiesResolver.KEY_DESERIALIZER; @@ -59,16 +64,17 @@ public class PropertiesResolverTest { private static final String UNKNOWN_FORMAT = "unknown"; + private static final Schema DUMMY_SCHEMA = SchemaBuilder.record("jet.sql").fields().endRecord(); @Test public void test_consumerProperties_absentFormat() { - assertThat(PropertiesResolver.resolveConsumerProperties(emptyMap())) + assertThat(resolveConsumerProperties(emptyMap())) .containsExactlyEntriesOf(Map.of(KEY_DESERIALIZER, ByteArrayDeserializer.class.getCanonicalName())); } @Test public void test_producerProperties_absentFormat() { - assertThat(PropertiesResolver.resolveProducerProperties(emptyMap())) + assertThat(resolveProducerProperties(emptyMap())) .containsExactlyEntriesOf(Map.of(KEY_SERIALIZER, ByteArraySerializer.class.getCanonicalName())); } @@ -76,28 +82,28 @@ public void test_producerProperties_absentFormat() { public void when_consumerProperties_formatIsUnknown_then_itIsIgnored() { // key Map keyOptions = Map.of(OPTION_KEY_FORMAT, UNKNOWN_FORMAT); - assertThat(PropertiesResolver.resolveConsumerProperties(keyOptions)).isEmpty(); + assertThat(resolveConsumerProperties(keyOptions)).isEmpty(); // value Map valueOptions = Map.of( OPTION_KEY_FORMAT, UNKNOWN_FORMAT, OPTION_VALUE_FORMAT, UNKNOWN_FORMAT ); - assertThat(PropertiesResolver.resolveConsumerProperties(valueOptions)).isEmpty(); + assertThat(resolveConsumerProperties(valueOptions)).isEmpty(); } @Test public void when_producerProperties_formatIsUnknown_then_itIsIgnored() { // key Map keyOptions = Map.of(OPTION_KEY_FORMAT, UNKNOWN_FORMAT); - assertThat(PropertiesResolver.resolveProducerProperties(keyOptions)).isEmpty(); + assertThat(resolveProducerProperties(keyOptions)).isEmpty(); // value Map valueOptions = Map.of( OPTION_KEY_FORMAT, UNKNOWN_FORMAT, OPTION_VALUE_FORMAT, UNKNOWN_FORMAT ); - assertThat(PropertiesResolver.resolveProducerProperties(valueOptions)).isEmpty(); + assertThat(resolveProducerProperties(valueOptions)).isEmpty(); } @SuppressWarnings("unused") @@ -121,13 +127,13 @@ private Object[] consumerValues() { @Parameters(method = "consumerValues") public void test_consumerProperties_java(String clazz, String deserializer) { // key - assertThat(PropertiesResolver.resolveConsumerProperties(Map.of( + assertThat(resolveConsumerProperties(Map.of( OPTION_KEY_FORMAT, JAVA_FORMAT, OPTION_KEY_CLASS, clazz ))).containsExactlyEntriesOf(Map.of(KEY_DESERIALIZER, deserializer)); // value - assertThat(PropertiesResolver.resolveConsumerProperties(Map.of( + assertThat(resolveConsumerProperties(Map.of( OPTION_KEY_FORMAT, UNKNOWN_FORMAT, OPTION_VALUE_FORMAT, JAVA_FORMAT, OPTION_VALUE_CLASS, clazz) @@ -155,13 +161,13 @@ private Object[] producerValues() { @Parameters(method = "producerValues") public void test_producerProperties_java(String clazz, String serializer) { // key - assertThat(PropertiesResolver.resolveProducerProperties(Map.of( + assertThat(resolveProducerProperties(Map.of( OPTION_KEY_FORMAT, JAVA_FORMAT, OPTION_KEY_CLASS, clazz ))).containsExactlyEntriesOf(Map.of(KEY_SERIALIZER, serializer)); // value - assertThat(PropertiesResolver.resolveProducerProperties(Map.of( + assertThat(resolveProducerProperties(Map.of( OPTION_KEY_FORMAT, UNKNOWN_FORMAT, OPTION_VALUE_FORMAT, JAVA_FORMAT, OPTION_VALUE_CLASS, clazz) @@ -195,7 +201,7 @@ public void when_consumerProperties_javaPropertyIsDefined_then_itsNotOverwritten KEY_DESERIALIZER, "deserializer" ); - assertThat(PropertiesResolver.resolveConsumerProperties(keyOptions)) + assertThat(resolveConsumerProperties(keyOptions)) .containsExactlyEntriesOf(Map.of(KEY_DESERIALIZER, "deserializer")); // value @@ -206,7 +212,7 @@ public void when_consumerProperties_javaPropertyIsDefined_then_itsNotOverwritten VALUE_DESERIALIZER, "deserializer" ); - assertThat(PropertiesResolver.resolveConsumerProperties(valueOptions)) + assertThat(resolveConsumerProperties(valueOptions)) .containsExactlyEntriesOf(Map.of(VALUE_DESERIALIZER, "deserializer")); } @@ -220,7 +226,7 @@ public void when_producerProperties_javaPropertyIsDefined_then_itsNotOverwritten KEY_SERIALIZER, "serializer" ); - assertThat(PropertiesResolver.resolveProducerProperties(keyOptions)) + assertThat(resolveProducerProperties(keyOptions)) .containsExactlyEntriesOf(Map.of(KEY_SERIALIZER, "serializer")); // value @@ -231,27 +237,34 @@ public void when_producerProperties_javaPropertyIsDefined_then_itsNotOverwritten VALUE_SERIALIZER, "serializer" ); - assertThat(PropertiesResolver.resolveProducerProperties(valueOptions)) + assertThat(resolveProducerProperties(valueOptions)) .containsExactlyEntriesOf(Map.of(VALUE_SERIALIZER, "serializer")); } @Test public void test_consumerProperties_avro() { // key - assertThat(PropertiesResolver.resolveConsumerProperties(Map.of(OPTION_KEY_FORMAT, AVRO_FORMAT))) - .containsExactlyEntriesOf(Map.of(KEY_DESERIALIZER, HazelcastKafkaAvroDeserializer.class.getCanonicalName())); + assertThat(PropertiesResolver.resolveConsumerProperties(Map.of( + OPTION_KEY_FORMAT, AVRO_FORMAT + ), DUMMY_SCHEMA, null)).containsExactlyInAnyOrderEntriesOf(Map.of( + KEY_DESERIALIZER, HazelcastKafkaAvroDeserializer.class.getCanonicalName(), + OPTION_KEY_AVRO_SCHEMA, DUMMY_SCHEMA + )); // value assertThat(PropertiesResolver.resolveConsumerProperties(Map.of( OPTION_KEY_FORMAT, UNKNOWN_FORMAT, OPTION_VALUE_FORMAT, AVRO_FORMAT - ))).containsExactlyEntriesOf(Map.of(VALUE_DESERIALIZER, HazelcastKafkaAvroDeserializer.class.getCanonicalName())); + ), null, DUMMY_SCHEMA)).containsExactlyInAnyOrderEntriesOf(Map.of( + VALUE_DESERIALIZER, HazelcastKafkaAvroDeserializer.class.getCanonicalName(), + OPTION_VALUE_AVRO_SCHEMA, DUMMY_SCHEMA + )); } @Test public void test_consumerProperties_avro_schemaRegistry() { // key - assertThat(PropertiesResolver.resolveConsumerProperties(Map.of( + assertThat(resolveConsumerProperties(Map.of( OPTION_KEY_FORMAT, AVRO_FORMAT, "schema.registry.url", "http://localhost:8081" ))).containsExactlyInAnyOrderEntriesOf(Map.of( @@ -260,7 +273,7 @@ public void test_consumerProperties_avro_schemaRegistry() { )); // value - assertThat(PropertiesResolver.resolveConsumerProperties(Map.of( + assertThat(resolveConsumerProperties(Map.of( OPTION_KEY_FORMAT, UNKNOWN_FORMAT, OPTION_VALUE_FORMAT, AVRO_FORMAT, "schema.registry.url", "http://localhost:8081" @@ -273,20 +286,27 @@ public void test_consumerProperties_avro_schemaRegistry() { @Test public void test_producerProperties_avro() { // key - assertThat(PropertiesResolver.resolveProducerProperties(Map.of(OPTION_KEY_FORMAT, AVRO_FORMAT))) - .containsExactlyEntriesOf(Map.of(KEY_SERIALIZER, HazelcastKafkaAvroSerializer.class.getCanonicalName())); + assertThat(PropertiesResolver.resolveProducerProperties(Map.of( + OPTION_KEY_FORMAT, AVRO_FORMAT + ), DUMMY_SCHEMA, null)).containsExactlyInAnyOrderEntriesOf(Map.of( + KEY_SERIALIZER, HazelcastKafkaAvroSerializer.class.getCanonicalName(), + OPTION_KEY_AVRO_SCHEMA, DUMMY_SCHEMA + )); // value assertThat(PropertiesResolver.resolveProducerProperties(Map.of( OPTION_KEY_FORMAT, UNKNOWN_FORMAT, OPTION_VALUE_FORMAT, AVRO_FORMAT - ))).containsExactlyEntriesOf(Map.of(VALUE_SERIALIZER, HazelcastKafkaAvroSerializer.class.getCanonicalName())); + ), null, DUMMY_SCHEMA)).containsExactlyInAnyOrderEntriesOf(Map.of( + VALUE_SERIALIZER, HazelcastKafkaAvroSerializer.class.getCanonicalName(), + OPTION_VALUE_AVRO_SCHEMA, DUMMY_SCHEMA + )); } @Test public void test_producerProperties_avro_schemaRegistry() { // key - assertThat(PropertiesResolver.resolveProducerProperties(Map.of( + assertThat(resolveProducerProperties(Map.of( OPTION_KEY_FORMAT, AVRO_FORMAT, "schema.registry.url", "http://localhost:8081" ))).containsExactlyInAnyOrderEntriesOf(Map.of( @@ -295,7 +315,7 @@ public void test_producerProperties_avro_schemaRegistry() { )); // value - assertThat(PropertiesResolver.resolveProducerProperties(Map.of( + assertThat(resolveProducerProperties(Map.of( OPTION_KEY_FORMAT, UNKNOWN_FORMAT, OPTION_VALUE_FORMAT, AVRO_FORMAT, "schema.registry.url", "http://localhost:8081" @@ -308,55 +328,55 @@ public void test_producerProperties_avro_schemaRegistry() { @Test public void when_consumerProperties_avroPropertyIsDefined_then_itsNotOverwritten() { // key - Map keyOptions = Map.of( + assertThat(PropertiesResolver.resolveConsumerProperties(Map.of( OPTION_KEY_FORMAT, AVRO_FORMAT, KEY_DESERIALIZER, "deserializer" - ); - - assertThat(PropertiesResolver.resolveConsumerProperties(keyOptions)) - .containsExactlyEntriesOf(Map.of(KEY_DESERIALIZER, "deserializer")); + ), DUMMY_SCHEMA, null)).containsExactlyInAnyOrderEntriesOf(Map.of( + KEY_DESERIALIZER, "deserializer", + OPTION_KEY_AVRO_SCHEMA, DUMMY_SCHEMA + )); // value - Map valueOptions = Map.of( + assertThat(PropertiesResolver.resolveConsumerProperties(Map.of( OPTION_KEY_FORMAT, UNKNOWN_FORMAT, OPTION_VALUE_FORMAT, AVRO_FORMAT, VALUE_DESERIALIZER, "deserializer" - ); - - assertThat(PropertiesResolver.resolveConsumerProperties(valueOptions)) - .containsExactlyEntriesOf(Map.of(VALUE_DESERIALIZER, "deserializer")); + ), null, DUMMY_SCHEMA)).containsExactlyInAnyOrderEntriesOf(Map.of( + VALUE_DESERIALIZER, "deserializer", + OPTION_VALUE_AVRO_SCHEMA, DUMMY_SCHEMA + )); } @Test public void when_producerProperties_avroPropertyIsDefined_then_itsNotOverwritten() { // key - Map keyOptions = Map.of( + assertThat(PropertiesResolver.resolveProducerProperties(Map.of( OPTION_KEY_FORMAT, AVRO_FORMAT, KEY_SERIALIZER, "serializer" - ); - - assertThat(PropertiesResolver.resolveProducerProperties(keyOptions)) - .containsExactlyEntriesOf(Map.of(KEY_SERIALIZER, "serializer")); + ), DUMMY_SCHEMA, null)).containsExactlyInAnyOrderEntriesOf(Map.of( + KEY_SERIALIZER, "serializer", + OPTION_KEY_AVRO_SCHEMA, DUMMY_SCHEMA + )); // value - Map valueOptions = Map.of( + assertThat(PropertiesResolver.resolveProducerProperties(Map.of( OPTION_KEY_FORMAT, UNKNOWN_FORMAT, OPTION_VALUE_FORMAT, AVRO_FORMAT, VALUE_SERIALIZER, "serializer" - ); - - assertThat(PropertiesResolver.resolveProducerProperties(valueOptions)) - .containsExactlyEntriesOf(Map.of(VALUE_SERIALIZER, "serializer")); + ), null, DUMMY_SCHEMA)).containsExactlyInAnyOrderEntriesOf(Map.of( + VALUE_SERIALIZER, "serializer", + OPTION_VALUE_AVRO_SCHEMA, DUMMY_SCHEMA + )); } @Test public void test_consumerProperties_json() { // key - assertThat(PropertiesResolver.resolveConsumerProperties(Map.of(OPTION_KEY_FORMAT, JSON_FLAT_FORMAT))) + assertThat(resolveConsumerProperties(Map.of(OPTION_KEY_FORMAT, JSON_FLAT_FORMAT))) .containsExactlyEntriesOf(Map.of(KEY_DESERIALIZER, ByteArrayDeserializer.class.getCanonicalName())); // value - assertThat(PropertiesResolver.resolveConsumerProperties(Map.of( + assertThat(resolveConsumerProperties(Map.of( OPTION_KEY_FORMAT, UNKNOWN_FORMAT, OPTION_VALUE_FORMAT, JSON_FLAT_FORMAT ))).containsExactlyEntriesOf(Map.of(VALUE_DESERIALIZER, ByteArrayDeserializer.class.getCanonicalName())); @@ -365,11 +385,11 @@ public void test_consumerProperties_json() { @Test public void test_producerProperties_json() { // key - assertThat(PropertiesResolver.resolveProducerProperties(Map.of(OPTION_KEY_FORMAT, JSON_FLAT_FORMAT))) + assertThat(resolveProducerProperties(Map.of(OPTION_KEY_FORMAT, JSON_FLAT_FORMAT))) .containsExactlyEntriesOf(Map.of(KEY_SERIALIZER, ByteArraySerializer.class.getCanonicalName())); // value - assertThat(PropertiesResolver.resolveProducerProperties(Map.of( + assertThat(resolveProducerProperties(Map.of( OPTION_KEY_FORMAT, UNKNOWN_FORMAT, OPTION_VALUE_FORMAT, JSON_FLAT_FORMAT ))).containsExactlyEntriesOf(Map.of(VALUE_SERIALIZER, ByteArraySerializer.class.getCanonicalName())); @@ -383,7 +403,7 @@ public void when_consumerProperties_jsonPropertyIsDefined_then_itsNotOverwritten KEY_DESERIALIZER, "deserializer" ); - assertThat(PropertiesResolver.resolveConsumerProperties(keyOptions)) + assertThat(resolveConsumerProperties(keyOptions)) .containsExactlyEntriesOf(Map.of(KEY_DESERIALIZER, "deserializer")); // value @@ -393,7 +413,7 @@ public void when_consumerProperties_jsonPropertyIsDefined_then_itsNotOverwritten VALUE_DESERIALIZER, "deserializer" ); - assertThat(PropertiesResolver.resolveConsumerProperties(valueOptions)) + assertThat(resolveConsumerProperties(valueOptions)) .containsExactlyEntriesOf(Map.of(VALUE_DESERIALIZER, "deserializer")); } @@ -405,7 +425,7 @@ public void when_producerProperties_jsonPropertyIsDefined_then_itsNotOverwritten KEY_SERIALIZER, "serializer" ); - assertThat(PropertiesResolver.resolveProducerProperties(keyOptions)) + assertThat(resolveProducerProperties(keyOptions)) .containsExactlyEntriesOf(Map.of(KEY_SERIALIZER, "serializer")); // value @@ -415,7 +435,15 @@ public void when_producerProperties_jsonPropertyIsDefined_then_itsNotOverwritten VALUE_SERIALIZER, "serializer" ); - assertThat(PropertiesResolver.resolveProducerProperties(valueOptions)) + assertThat(resolveProducerProperties(valueOptions)) .containsExactlyEntriesOf(Map.of(VALUE_SERIALIZER, "serializer")); } + + private static Properties resolveConsumerProperties(Map options) { + return PropertiesResolver.resolveConsumerProperties(options, null, null); + } + + private static Properties resolveProducerProperties(Map options) { + return PropertiesResolver.resolveProducerProperties(options, null, null); + } } diff --git a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/kafka/SqlAvroSchemaEvolutionTest.java b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/kafka/SqlAvroSchemaEvolutionTest.java index 6a405d795ec6..96d3b015d6fa 100644 --- a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/kafka/SqlAvroSchemaEvolutionTest.java +++ b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/kafka/SqlAvroSchemaEvolutionTest.java @@ -16,13 +16,11 @@ package com.hazelcast.jet.sql.impl.connector.kafka; -import com.google.common.collect.Lists; import com.hazelcast.test.HazelcastParametrizedRunner; import com.hazelcast.test.HazelcastSerialParametersRunnerFactory; import io.confluent.kafka.schemaregistry.exceptions.SchemaRegistryException; import org.apache.avro.Schema; import org.apache.avro.SchemaBuilder; -import org.apache.avro.generic.GenericRecordBuilder; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; @@ -32,7 +30,6 @@ import org.junit.runners.Parameterized.UseParametersRunnerFactory; import java.util.Arrays; -import java.util.List; import java.util.Map; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.AVRO_FORMAT; @@ -41,6 +38,7 @@ import static com.hazelcast.jet.sql.impl.connector.kafka.SqlAvroTest.ID_SCHEMA; import static com.hazelcast.jet.sql.impl.connector.kafka.SqlAvroTest.NAME_SCHEMA; import static java.util.Arrays.asList; +import static java.util.Arrays.copyOf; import static java.util.stream.Collectors.toList; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.Assert.assertEquals; @@ -61,9 +59,9 @@ public class SqlAvroSchemaEvolutionTest extends KafkaSqlTestSupport { @Parameters(name = "{0}, updateMapping=[{1}]") public static Iterable parameters() { - return Lists.cartesianProduct( + return parameters( asList("TopicNameStrategy", "TopicRecordNameStrategy", "RecordNameStrategy"), - asList(false, true)).stream().map(List::toArray).collect(toList()); + asList(false, true)); } @Parameter(0) @@ -106,7 +104,7 @@ public void before() throws Exception { } private SqlMapping kafkaMapping() { - return new SqlMapping(name, KafkaSqlConnector.TYPE_NAME).options( + return new SqlMapping(name, KafkaSqlConnector.class).options( OPTION_KEY_FORMAT, AVRO_FORMAT, OPTION_VALUE_FORMAT, AVRO_FORMAT, "bootstrap.servers", kafkaTestSupport.getBrokerConnectionString(), @@ -167,7 +165,7 @@ public void test_useLatestSchema() throws SchemaRegistryException { if (topicNameStrategy && !updateMapping) { // insert record against mapping's schema - assertThatThrownBy(() -> sqlService.execute("INSERT INTO " + name + " VALUES (29, 'Bob')")) + assertThatThrownBy(() -> insertRecord(29, "Bob")) .hasMessageContaining("Error serializing Avro message"); } else { insertAndAssertRecords(); @@ -208,7 +206,7 @@ public void test_useSpecificSchema() throws SchemaRegistryException { private int insertInitialRecordAndAlterSchema() throws SchemaRegistryException { // insert initial record - sqlService.execute("INSERT INTO " + name + " VALUES (13, 'Alice')"); + insertRecord(13, "Alice"); assertEquals(1, kafkaTestSupport.getLatestSchemaVersion(valueSubjectName)); // alter schema externally @@ -228,21 +226,17 @@ private void insertAndAssertRecords() throws SchemaRegistryException { int fields = updateMapping ? 3 : 2; // insert record against mapping's schema - sqlService.execute("INSERT INTO " + name + " VALUES (29, 'Bob'" + (fields == 3 ? ", 123456789)" : ")")); + insertRecord(copyOf(row(29, "Bob", 123456789L), fields)); // insert record against old schema externally - kafkaTestSupport.produce(name, - new GenericRecordBuilder(ID_SCHEMA).set("id", 31).build(), - new GenericRecordBuilder(NAME_SCHEMA).set("name", "Carol").build()); + kafkaTestSupport.produce(name, createRecord(ID_SCHEMA, 31), createRecord(NAME_SCHEMA, "Carol")); // insert record against new schema externally - kafkaTestSupport.produce(name, - new GenericRecordBuilder(ID_SCHEMA).set("id", 47).build(), - new GenericRecordBuilder(topicNameStrategy ? NAME_SSN_SCHEMA : NAME_SSN_SCHEMA2) - .set("name", "Dave").set("ssn", 123456789L).build()); + kafkaTestSupport.produce(name, createRecord(ID_SCHEMA, 47), + createRecord(topicNameStrategy ? NAME_SSN_SCHEMA : NAME_SSN_SCHEMA2, "Dave", 123456789L)); // insert record against mapping's schema again - sqlService.execute("INSERT INTO " + name + " VALUES (53, 'Erin'" + (fields == 3 ? ", 987654321)" : ")")); + insertRecord(copyOf(row(53, "Erin", 987654321L), fields)); if (topicNameStrategy) { assertEquals(2, kafkaTestSupport.getLatestSchemaVersion(valueSubjectName)); @@ -261,7 +255,11 @@ private void insertAndAssertRecords() throws SchemaRegistryException { }; assertRowsEventuallyInAnyOrder( "SELECT * FROM " + name, - Arrays.stream(records).map(record -> new Row(Arrays.copyOf(record, fields))).collect(toList()) + Arrays.stream(records).map(record -> new Row(copyOf(record, fields))).collect(toList()) ); } + + private void insertRecord(Object... values) { + insertLiterals(instance(), name, values); + } } diff --git a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/kafka/SqlAvroTest.java b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/kafka/SqlAvroTest.java index 3469812ba820..158080dc89dc 100644 --- a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/kafka/SqlAvroTest.java +++ b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/kafka/SqlAvroTest.java @@ -18,15 +18,13 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; +import com.hazelcast.core.HazelcastInstance; import com.hazelcast.internal.nio.Bits; import com.hazelcast.jet.datamodel.Tuple2; import com.hazelcast.jet.datamodel.Tuple4; -import com.hazelcast.jet.impl.util.ExceptionUtil; import com.hazelcast.jet.kafka.HazelcastKafkaAvroDeserializer; import com.hazelcast.jet.kafka.HazelcastKafkaAvroSerializer; import com.hazelcast.jet.sql.impl.connector.test.TestAllTypesSqlConnector; -import com.hazelcast.sql.SqlResult; -import com.hazelcast.sql.SqlRow; import com.hazelcast.sql.impl.type.QueryDataType; import com.hazelcast.sql.impl.type.QueryDataTypeFamily; import com.hazelcast.test.HazelcastParametrizedRunner; @@ -35,8 +33,6 @@ import io.confluent.kafka.serializers.KafkaAvroSerializer; import org.apache.avro.Schema; import org.apache.avro.SchemaBuilder; -import org.apache.avro.SchemaBuilder.FieldAssembler; -import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecordBuilder; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; @@ -59,13 +55,11 @@ import java.time.LocalTime; import java.time.OffsetDateTime; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.function.BiConsumer; import java.util.function.Consumer; -import java.util.function.UnaryOperator; -import java.util.stream.IntStream; import java.util.stream.Stream; import static com.hazelcast.jet.datamodel.Tuple2.tuple2; @@ -75,21 +69,20 @@ import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_KEY_AVRO_SCHEMA; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_KEY_CLASS; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_KEY_FORMAT; +import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_AVRO_RECORD_NAME; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_AVRO_SCHEMA; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_FORMAT; import static com.hazelcast.jet.sql.impl.connector.kafka.SqlAvroSchemaEvolutionTest.NAME_SSN_SCHEMA; import static com.hazelcast.jet.sql.impl.connector.keyvalue.KvMetadataAvroResolver.Schemas.OBJECT_SCHEMA; +import static com.hazelcast.jet.sql.impl.connector.keyvalue.KvMetadataAvroResolver.optional; +import static com.hazelcast.spi.properties.ClusterProperty.SQL_CUSTOM_TYPES_ENABLED; import static java.time.ZoneOffset.UTC; -import static java.util.Arrays.asList; import static java.util.Arrays.copyOfRange; import static java.util.Collections.emptyMap; import static java.util.concurrent.TimeUnit.SECONDS; -import static java.util.stream.Collectors.joining; import static java.util.stream.Collectors.toList; -import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; import static org.junit.Assume.assumeFalse; import static org.junit.Assume.assumeTrue; @@ -127,19 +120,20 @@ public class SqlAvroTest extends KafkaSqlTestSupport { @Parameters(name = "useSchemaRegistry=[{0}]") public static Iterable parameters() { - return asList(false, true); + return List.of(false, true); } @Parameter public boolean useSchemaRegistry; - private SqlMapping mapping; + private Type mapping; private Schema keySchema; private Schema valueSchema; private Map clientProperties; @BeforeClass - public static void initialize() throws Exception { + public static void setup() throws Exception { + setup(1, smallInstanceConfig().setProperty(SQL_CUSTOM_TYPES_ENABLED.getName(), "true")); createSchemaRegistry(); } @@ -153,19 +147,52 @@ private SqlMapping kafkaMapping(String name, Schema keySchema, Schema valueSchem clientProperties = useSchemaRegistry ? ImmutableMap.of("schema.registry.url", kafkaTestSupport.getSchemaRegistryURI().toString()) : ImmutableMap.of(OPTION_KEY_AVRO_SCHEMA, keySchema.toString(), - OPTION_VALUE_AVRO_SCHEMA, valueSchema.toString()); + OPTION_VALUE_AVRO_SCHEMA, valueSchema.toString()); kafkaTestSupport.setProducerProperties(name, clientProperties); - return mapping = new SqlMapping(name, KafkaSqlConnector.TYPE_NAME) + return new KafkaMapping(name) .options(OPTION_KEY_FORMAT, AVRO_FORMAT, - OPTION_VALUE_FORMAT, AVRO_FORMAT, - "bootstrap.servers", kafkaTestSupport.getBrokerConnectionString(), - "auto.offset.reset", "earliest") + OPTION_VALUE_FORMAT, AVRO_FORMAT, + "bootstrap.servers", kafkaTestSupport.getBrokerConnectionString(), + "auto.offset.reset", "earliest") .optionsIf(useSchemaRegistry, - "schema.registry.url", kafkaTestSupport.getSchemaRegistryURI()) + "schema.registry.url", kafkaTestSupport.getSchemaRegistryURI()) .optionsIf(!useSchemaRegistry, - OPTION_KEY_AVRO_SCHEMA, keySchema, - OPTION_VALUE_AVRO_SCHEMA, valueSchema); + OPTION_KEY_AVRO_SCHEMA, keySchema, + OPTION_VALUE_AVRO_SCHEMA, valueSchema); + } + + @Test + public void test_nestedField() { + String name = createRandomTopic(); + new SqlType("Organization").fields("id INT", "name VARCHAR").create(); + + Schema orgSchema = SchemaBuilder.record("Organization").fields() + .optionalInt("id") + .optionalString("name") + .endRecord(); + Schema personSchema = SchemaBuilder.record("Person").fields() + .optionalString("name") + .name("organization").type().optional().type(orgSchema) + .endRecord(); + + kafkaMapping(name, ID_SCHEMA, personSchema) + .fields("id INT EXTERNAL NAME \"__key.id\"", + "name VARCHAR", + "organization Organization") + .optionsIf(useSchemaRegistry, + OPTION_VALUE_AVRO_RECORD_NAME, "Person") + .create(); + + insertAndAssertRecord( + row(1, "Alice", row(1, "Umbrella Corporation")), + row(1, "Alice", row(1, "Umbrella Corporation")), + row(1, "Alice", createRecord(orgSchema, 1, "Umbrella Corporation"))); + + assertRowsEventuallyInAnyOrder( + "SELECT name, (organization).name FROM " + name, + List.of(new Row("Alice", "Umbrella Corporation")) + ); } @Test @@ -204,42 +231,74 @@ public void when_schemaHasMissingField_then_fail() { } @Test - public void test_mappingHasMissingOptionalField() { + public void test_mappingAndTypeHasMissingOptionalField() { assumeFalse(useSchemaRegistry); String name = createRandomTopic(); - kafkaMapping(name, ID_SCHEMA, NAME_SSN_SCHEMA) + new SqlType("Parent").fields("name VARCHAR", "phone VARCHAR").create(); + + Schema parentSchema = SchemaBuilder.record("Parent").fields() + .requiredString("name") + .name("address").type().stringType().stringDefault("") + .requiredString("phone") + .endRecord(); + Schema studentSchema = SchemaBuilder.record("Student").fields() + .requiredString("name") + .optionalLong("ssn") + .name("parent").type(parentSchema).noDefault() + .endRecord(); + + kafkaMapping(name, ID_SCHEMA, studentSchema) .fields("id INT EXTERNAL NAME \"__key.id\"", - "name VARCHAR") + "name VARCHAR", + "parent Parent") .create(); - insertAndAssertRecord(1, "Alice"); + insertAndAssertRecord( + row(1, "Alice", row("Bob", "(111) 111-1111")), + row(1, "Alice", row("Bob", "(111) 111-1111")), + row(1, "Alice", createRecord(parentSchema, "Bob", "", "(111) 111-1111"))); - kafkaTestSupport.produce(name, createRecord(ID_SCHEMA, 2), - createRecord(NAME_SSN_SCHEMA, "Bob", 123456789L)); + kafkaTestSupport.produce(name, createRecord(ID_SCHEMA, 3), + createRecord(studentSchema, "Dave", 123456789L, + row("Erin", "Insignificant St. 34", "(999) 999-9999"))); assertRowsEventuallyInAnyOrder( - "SELECT * FROM " + name, - asList( - new Row(1, "Alice"), - new Row(2, "Bob") + "SELECT name, (parent).name, (parent).phone FROM " + name, + List.of( + new Row("Alice", "Bob", "(111) 111-1111"), + new Row("Dave", "Erin", "(999) 999-9999") ) ); } @Test - public void when_mappingHasMissingMandatoryField_then_fail() { + public void when_mappingOrTypeHasMissingMandatoryField_then_fail() { assumeFalse(useSchemaRegistry); - Schema schema = SchemaBuilder.record("jet.sql").fields() + new SqlType("Parent").fields("name VARCHAR").create(); + + Schema parentSchema = SchemaBuilder.record("Parent").fields() + .requiredString("name") + .requiredString("phone") + .endRecord(); + Schema studentSchema = SchemaBuilder.record("Student").fields() .requiredString("name") - .requiredLong("ssn") + .name("parent").type(parentSchema).noDefault() .endRecord(); assertThatThrownBy(() -> - kafkaMapping("kafka", ID_SCHEMA, schema) + kafkaMapping("kafka", ID_SCHEMA, studentSchema) .fields("id INT EXTERNAL NAME \"__key.id\"", "name VARCHAR") .create()) - .hasMessage("Mandatory field 'ssn' is not mapped to any column"); + .hasMessage("Mandatory field 'parent' is not mapped to any column"); + + assertThatThrownBy(() -> + kafkaMapping("kafka", ID_SCHEMA, studentSchema) + .fields("id INT EXTERNAL NAME \"__key.id\"", + "name VARCHAR", + "parent Parent") + .create()) + .hasMessage("Mandatory field 'phone' is not mapped to any column"); } @Test @@ -329,7 +388,7 @@ public void test_nonInclusiveUnion() { assertRowsEventuallyInAnyOrder( "SELECT * FROM " + name, - asList( + List.of( new Row(1, null), new Row(2, true), new Row(3, true), @@ -342,37 +401,6 @@ public void test_nonInclusiveUnion() { "Not in union [\"null\",\"boolean\",\"int\"]: " + Long.MAX_VALUE + " (Long) (field=info)"); } - @Test - public void when_createAvroMapping_then_keyAndValueAreAvailableInInfoSchema() { - assumeFalse(useSchemaRegistry); - String name = createRandomTopic(); - kafkaMapping(name, ID_SCHEMA, NAME_SCHEMA) - .fields("id BIGINT EXTERNAL NAME \"__key.id\"", "name VARCHAR") - .create(); - - String query = "SELECT * FROM information_schema.mappings"; - try (SqlResult result = sqlService.execute(query)) { - SqlRow row = result.stream().findFirst().orElseThrow(); - assertNotNull(row); - - // Destructure the row: it must contain "catalog", "schema", "name", "external name", "type", "options" - assertEquals("hazelcast", row.getObject(0)); - assertEquals("public", row.getObject(1)); - assertEquals(name, row.getObject(2)); - assertEquals('"' + name + '"', row.getObject(3)); - assertEquals(KafkaSqlConnector.TYPE_NAME, row.getObject(4)); - - String options = row.getObject(5); - assertNotNull(options); - // Ensure that the options contain avro schema key and value - assertThat(options) - .contains("keyFormat\":\"avro\"") - .contains("valueFormat\":\"avro\"") - .contains("keyAvroSchema\":") - .contains("valueAvroSchema\":"); - } - } - @Test public void test_allConversions() { assumeFalse(useSchemaRegistry); @@ -406,7 +434,7 @@ public void test_allConversions() { List.of(tuple2(QueryDataTypeFamily.REAL, 1F), tuple2(QueryDataTypeFamily.DOUBLE, 1D)), List.of(tuple2(Schema.Type.STRING, "1.0")))); - conversions.addAll(asList( + conversions.addAll(List.of( tuple4(QueryDataTypeFamily.TIME, LocalTime.of(12, 23, 34), Schema.Type.STRING, "12:23:34"), tuple4(QueryDataTypeFamily.DATE, LocalDate.of(2020, 4, 15), Schema.Type.STRING, "2020-04-15"), tuple4(QueryDataTypeFamily.TIMESTAMP, LocalDateTime.of(2020, 4, 15, 12, 23, 34, 1_000_000), @@ -456,7 +484,7 @@ public void test_allConversions() { Schema.Type.NULL, Schema.Type.BYTES ).forEach(type -> schemaFieldTypes.add(Schema.create(type))); - schemaFieldTypes.addAll(asList( + schemaFieldTypes.addAll(List.of( OBJECT_SCHEMA, // Schema.Type.UNION SchemaBuilder.array().items(Schema.create(Schema.Type.INT)), SchemaBuilder.map().values(Schema.create(Schema.Type.INT)), @@ -473,8 +501,8 @@ public void test_allConversions() { for (Schema fieldSchema : schemaFieldTypes) { Schema.Type schemaFieldType = fieldSchema.getType(); - Schema valueSchema = optionalField("info", fieldSchema) - .apply(SchemaBuilder.record("jet.sql").fields()) + Schema valueSchema = SchemaBuilder.record("jet.sql").fields() + .name("info").type(optional(fieldSchema)).withDefault(null) .endRecord(); Consumer createKafkaMapping = name -> kafkaMapping(name, ID_SCHEMA, valueSchema) @@ -551,7 +579,7 @@ public void test_schemaEvolution() { // assert both - initial & evolved - records are correctly read Runnable assertRecords = () -> assertRowsEventuallyInAnyOrder( "SELECT * FROM " + name, - asList( + List.of( new Row(13, "Alice", null), new Row(69, "Bob", 123456789L) ) @@ -685,14 +713,34 @@ public void test_explicitKeyAndValueSerializers() { .fields("key_name VARCHAR EXTERNAL NAME \"__key.name\"", "value_name VARCHAR EXTERNAL NAME \"this.name\"") .options("key.serializer", serializerClass.getCanonicalName(), - "key.deserializer", deserializerClass.getCanonicalName(), - "value.serializer", serializerClass.getCanonicalName(), - "value.deserializer", deserializerClass.getCanonicalName()) + "key.deserializer", deserializerClass.getCanonicalName(), + "value.serializer", serializerClass.getCanonicalName(), + "value.deserializer", deserializerClass.getCanonicalName()) .create(); insertAndAssertRecord("Alice", "Bob"); } + @Test + public void test_keyAndValueSchemaAvailableInInformationSchema() { + assumeFalse(useSchemaRegistry); + String name = createRandomTopic(); + kafkaMapping(name, ID_SCHEMA, NAME_SCHEMA) + .fields("id BIGINT EXTERNAL NAME \"__key.id\"", + "name VARCHAR") + .create(); + + String options = sqlService.execute("SELECT mapping_options FROM information_schema.mappings") + .iterator().next().getObject(0); + BiConsumer assertContainsSchema = (option, schema) -> assertContains( + options, + String.format("\"%s\":\"%s\"", option, schema.toString().replace("\"", "\\\"")) + ); + + assertContainsSchema.accept(OPTION_KEY_AVRO_SCHEMA, keySchema); + assertContainsSchema.accept(OPTION_VALUE_AVRO_SCHEMA, valueSchema); + } + @Test public void test_schemaIdForTwoQueriesIsEqual() { assumeTrue(useSchemaRegistry); @@ -701,7 +749,7 @@ public void test_schemaIdForTwoQueriesIsEqual() { .fields("__key INT", "field1 VARCHAR") .options(OPTION_KEY_FORMAT, JAVA_FORMAT, - OPTION_KEY_CLASS, Integer.class.getCanonicalName()) + OPTION_KEY_CLASS, Integer.class.getCanonicalName()) .create(); insertRecord(42, "foo"); @@ -732,8 +780,7 @@ private static String createRandomTopic() { } private void insertRecord(Object... values) { - sqlService.execute("INSERT INTO " + mapping.name + " VALUES (" + - Arrays.stream(values).map(SqlAvroTest::toSQL).collect(joining(", ")) + ")"); + insertLiterals(instance(), mapping.name, values); } private void insertAndAssertRecord(Object... values) { @@ -748,14 +795,13 @@ private void insertAndAssertRecord(@Nonnull Object[] insertValues, @Nonnull Obje @Nonnull Object[] selectValues) { insertRecord(insertValues); - String[] fields = getExternalFields(); kafkaTestSupport.assertTopicContentsEventually( mapping.name, Map.of( - createRecord(keySchema, copyOfRange(fields, 0, 1), + createRecord(keySchema, copyOfRange(mapping.fields, 0, 1), copyOfRange(avroValues, 0, 1)), - createRecord(valueSchema, copyOfRange(fields, 1, fields.length), - copyOfRange(avroValues, 1, fields.length)) + createRecord(valueSchema, copyOfRange(mapping.fields, 1, mapping.fields.length), + copyOfRange(avroValues, 1, mapping.fields.length)) ), useSchemaRegistry ? KafkaAvroDeserializer.class : HazelcastKafkaAvroDeserializer.class, useSchemaRegistry ? KafkaAvroDeserializer.class : HazelcastKafkaAvroDeserializer.class, @@ -767,29 +813,6 @@ private void insertAndAssertRecord(@Nonnull Object[] insertValues, @Nonnull Obje ); } - private String[] getExternalFields() { - return mapping.fields.stream().map(field -> field.endsWith("\"") - ? field.substring(field.lastIndexOf('.') + 1, field.length() - 1) - : field.substring(0, field.indexOf(' '))).toArray(String[]::new); - } - - private static String toSQL(Object value) { - return value == null || value instanceof Boolean || value instanceof Number - ? String.valueOf(value) : "'" + value + "'"; - } - - private static GenericRecord createRecord(Schema schema, String[] fields, Object[] values) { - return IntStream.range(0, fields.length).collect(() -> new GenericRecordBuilder(schema), - (record, i) -> record.set(fields[i], values[i]), - ExceptionUtil::combinerUnsupported).build(); - } - - private static GenericRecord createRecord(Schema schema, Object... values) { - return createRecord(schema, - schema.getFields().stream().map(Schema.Field::name).toArray(String[]::new), - values); - } - @SuppressWarnings("unchecked") private static List> cartesian(List> list1, List> list2) { @@ -798,9 +821,16 @@ private static List> cartesian(List> optionalField(String name, Schema schema) { - return schema.isNullable() - ? builder -> builder.name(name).type(schema).withDefault(null) - : builder -> builder.name(name).type().optional().type(schema); + /** Generates a type tree on creation. */ + private class KafkaMapping extends SqlMapping { + KafkaMapping(String name) { + super(name, KafkaSqlConnector.class); + } + + @Override + protected void create(HazelcastInstance instance, boolean replace, boolean ifNotExists) { + super.create(instance, replace, ifNotExists); + mapping = toTypeTree(); + } } } diff --git a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/kafka/SqlJsonTest.java b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/kafka/SqlJsonTest.java index 82451cc85480..c3dbbe832496 100644 --- a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/kafka/SqlJsonTest.java +++ b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/kafka/SqlJsonTest.java @@ -16,143 +16,94 @@ package com.hazelcast.jet.sql.impl.connector.kafka; -import com.google.common.collect.ImmutableMap; import com.hazelcast.core.HazelcastJsonValue; -import com.hazelcast.jet.kafka.impl.KafkaTestSupport; -import com.hazelcast.jet.sql.SqlTestSupport; import com.hazelcast.jet.sql.impl.connector.test.TestAllTypesSqlConnector; -import com.hazelcast.sql.HazelcastSqlException; -import com.hazelcast.sql.SqlService; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.serialization.StringDeserializer; -import org.junit.AfterClass; -import org.junit.BeforeClass; import org.junit.Test; -import java.io.IOException; import java.math.BigDecimal; import java.time.LocalDate; import java.time.LocalDateTime; import java.time.LocalTime; import java.time.OffsetDateTime; +import java.util.List; import java.util.Map; -import static com.hazelcast.jet.core.TestUtil.createMap; +import static com.hazelcast.jet.pipeline.file.JsonFileFormat.FORMAT_JSON; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.JSON_FLAT_FORMAT; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_KEY_FORMAT; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_FORMAT; -import static java.lang.String.format; import static java.time.ZoneOffset.UTC; -import static java.util.Arrays.asList; -import static java.util.Collections.singletonList; import static org.assertj.core.api.Assertions.assertThatThrownBy; -public class SqlJsonTest extends SqlTestSupport { - +public class SqlJsonTest extends KafkaSqlTestSupport { private static final int INITIAL_PARTITION_COUNT = 4; - private static KafkaTestSupport kafkaTestSupport; - - private static SqlService sqlService; - - @BeforeClass - public static void setUpClass() throws IOException { - initialize(1, null); - sqlService = instance().getSql(); - - kafkaTestSupport = KafkaTestSupport.create(); - kafkaTestSupport.createKafkaCluster(); - } - - @AfterClass - public static void tearDownClass() { - kafkaTestSupport.shutdownKafkaCluster(); + private static SqlMapping kafkaMapping(String name) { + return new SqlMapping(name, KafkaSqlConnector.class) + .options(OPTION_KEY_FORMAT, JSON_FLAT_FORMAT, + OPTION_VALUE_FORMAT, JSON_FLAT_FORMAT, + "bootstrap.servers", kafkaTestSupport.getBrokerConnectionString(), + "auto.offset.reset", "earliest"); } @Test public void test_nulls() { String name = createRandomTopic(); - sqlService.execute("CREATE MAPPING " + name + " (" - + "id INT EXTERNAL NAME \"__key.id\"" - + ", name VARCHAR EXTERNAL NAME \"this.name\"" - + ") TYPE " + KafkaSqlConnector.TYPE_NAME + ' ' - + "OPTIONS ( " - + '\'' + OPTION_KEY_FORMAT + "'='" + JSON_FLAT_FORMAT + '\'' - + ", '" + OPTION_VALUE_FORMAT + "'='" + JSON_FLAT_FORMAT + '\'' - + ", 'bootstrap.servers'='" + kafkaTestSupport.getBrokerConnectionString() + '\'' - + ", 'auto.offset.reset'='earliest'" - + ")" - ); + kafkaMapping(name) + .fields("id INT EXTERNAL NAME \"__key.id\"", + "name VARCHAR EXTERNAL NAME \"this.name\"") + .create(); assertTopicEventually( name, "INSERT INTO " + name + " VALUES (null, null)", - createMap("{\"id\":null}", "{\"name\":null}") + Map.of("{\"id\":null}", "{\"name\":null}") ); assertRowsEventuallyInAnyOrder( "SELECT * FROM " + name, - singletonList(new Row(null, null)) + List.of(new Row(null, null)) ); } @Test public void test_fieldsMapping() { String name = createRandomTopic(); - sqlService.execute("CREATE MAPPING " + name + " (" - + "key_name VARCHAR EXTERNAL NAME \"__key.name\"" - + ", value_name VARCHAR EXTERNAL NAME \"this.name\"" - + ") TYPE " + KafkaSqlConnector.TYPE_NAME + ' ' - + "OPTIONS ( " - + '\'' + OPTION_KEY_FORMAT + "'='" + JSON_FLAT_FORMAT + '\'' - + ", '" + OPTION_VALUE_FORMAT + "'='" + JSON_FLAT_FORMAT + '\'' - + ", 'bootstrap.servers'='" + kafkaTestSupport.getBrokerConnectionString() + '\'' - + ", 'auto.offset.reset'='earliest'" - + ")" - ); + kafkaMapping(name) + .fields("key_name VARCHAR EXTERNAL NAME \"__key.name\"", + "value_name VARCHAR EXTERNAL NAME \"this.name\"") + .create(); assertTopicEventually( name, "INSERT INTO " + name + " (value_name, key_name) VALUES ('Bob', 'Alice')", - createMap("{\"name\":\"Alice\"}", "{\"name\":\"Bob\"}") + Map.of("{\"name\":\"Alice\"}", "{\"name\":\"Bob\"}") ); assertRowsEventuallyInAnyOrder( "SELECT * FROM " + name, - singletonList(new Row("Alice", "Bob")) + List.of(new Row("Alice", "Bob")) ); } @Test public void test_schemaEvolution() { String name = createRandomTopic(); - sqlService.execute("CREATE MAPPING " + name + " (" - + "id INT EXTERNAL NAME \"__key.id\"" - + ", name VARCHAR" - + ") TYPE " + KafkaSqlConnector.TYPE_NAME + ' ' - + "OPTIONS ( " - + '\'' + OPTION_KEY_FORMAT + "'='" + JSON_FLAT_FORMAT + '\'' - + ", '" + OPTION_VALUE_FORMAT + "'='" + JSON_FLAT_FORMAT + '\'' - + ", 'bootstrap.servers'='" + kafkaTestSupport.getBrokerConnectionString() + '\'' - + ", 'auto.offset.reset'='earliest'" - + ")" - ); + kafkaMapping(name) + .fields("id INT EXTERNAL NAME \"__key.id\"", + "name VARCHAR") + .create(); // insert initial record sqlService.execute("INSERT INTO " + name + " VALUES (13, 'Alice')"); // alter schema - sqlService.execute("CREATE OR REPLACE MAPPING " + name + " (" - + "id INT EXTERNAL NAME \"__key.id\"" - + ", name VARCHAR" - + ", ssn BIGINT" - + ") TYPE " + KafkaSqlConnector.TYPE_NAME + ' ' - + "OPTIONS ( " - + '\'' + OPTION_KEY_FORMAT + "'='" + JSON_FLAT_FORMAT + '\'' - + ", '" + OPTION_VALUE_FORMAT + "'='" + JSON_FLAT_FORMAT + '\'' - + ", 'bootstrap.servers'='" + kafkaTestSupport.getBrokerConnectionString() + '\'' - + ", 'auto.offset.reset'='earliest'" - + ")" - ); + kafkaMapping(name) + .fields("id INT EXTERNAL NAME \"__key.id\"", + "name VARCHAR", + "ssn BIGINT") + .createOrReplace(); // insert record against new schema sqlService.execute("INSERT INTO " + name + " VALUES (69, 'Bob', 123456789)"); @@ -160,7 +111,7 @@ public void test_schemaEvolution() { // assert both - initial & evolved - records are correctly read assertRowsEventuallyInAnyOrder( "SELECT * FROM " + name, - asList( + List.of( new Row(13, "Alice", null), new Row(69, "Bob", 123456789L) ) @@ -173,37 +124,30 @@ public void test_allTypes() { TestAllTypesSqlConnector.create(sqlService, from); String to = createRandomTopic(); - sqlService.execute("CREATE MAPPING " + to + " (" - + "id VARCHAR EXTERNAL NAME \"__key.id\"" - + ", string VARCHAR" - + ", \"boolean\" BOOLEAN" - + ", byte TINYINT" - + ", short SMALLINT" - + ", \"int\" INT" - + ", long BIGINT" - + ", \"float\" REAL" - + ", \"double\" DOUBLE" - + ", \"decimal\" DECIMAL" - + ", \"time\" TIME" - + ", \"date\" DATE" - + ", \"timestamp\" TIMESTAMP" - + ", timestampTz TIMESTAMP WITH TIME ZONE" - + ", map OBJECT" - + ", object OBJECT" - + ") TYPE " + KafkaSqlConnector.TYPE_NAME + ' ' - + "OPTIONS ( " - + '\'' + OPTION_KEY_FORMAT + "'='" + JSON_FLAT_FORMAT + '\'' - + ", '" + OPTION_VALUE_FORMAT + "'='" + JSON_FLAT_FORMAT + '\'' - + ", 'bootstrap.servers'='" + kafkaTestSupport.getBrokerConnectionString() + '\'' - + ", 'auto.offset.reset'='earliest'" - + ")" - ); + kafkaMapping(to) + .fields("id VARCHAR EXTERNAL NAME \"__key.id\"", + "string VARCHAR", + "\"boolean\" BOOLEAN", + "byte TINYINT", + "short SMALLINT", + "\"int\" INT", + "long BIGINT", + "\"float\" REAL", + "\"double\" DOUBLE", + "\"decimal\" DECIMAL", + "\"time\" TIME", + "\"date\" DATE", + "\"timestamp\" TIMESTAMP", + "timestampTz TIMESTAMP WITH TIME ZONE", + "map OBJECT", + "object OBJECT") + .create(); sqlService.execute("INSERT INTO " + to + " SELECT '1', f.* FROM " + from + " f"); assertRowsEventuallyInAnyOrder( "SELECT * FROM " + to, - singletonList(new Row( + List.of(new Row( "1", "string", true, @@ -218,7 +162,7 @@ public void test_allTypes() { LocalDate.of(2020, 4, 15), LocalDateTime.of(2020, 4, 15, 12, 23, 34, 1_000_000), OffsetDateTime.of(2020, 4, 15, 12, 23, 34, 200_000_000, UTC), - ImmutableMap.of("42", 43), // JSON serializer stores maps as JSON objects, the key is converted to a string + Map.of("42", 43), // JSON serializer stores maps as JSON objects, the key is converted to a string null )) ); @@ -226,114 +170,83 @@ public void test_allTypes() { @Test public void when_createMappingNoColumns_then_fail() { - assertThatThrownBy(() -> - sqlService.execute("CREATE MAPPING kafka " - + "TYPE " + KafkaSqlConnector.TYPE_NAME + ' ' - + "OPTIONS ('valueFormat'='" + JSON_FLAT_FORMAT + "')")) + assertThatThrownBy(() -> kafkaMapping("kafka").create()) .hasMessage("Column list is required for JSON format"); } - @Test - public void when_explicitTopLevelField_then_fail_key() { - when_explicitTopLevelField_then_fail("__key", "this"); - } - - @Test - public void when_explicitTopLevelField_then_fail_this() { - when_explicitTopLevelField_then_fail("this", "__key"); - } - @Test public void test_jsonType() { String name = createRandomTopic(); - - String createSql = format("CREATE MAPPING %s TYPE %s ", name, KafkaSqlConnector.TYPE_NAME) - + "OPTIONS ( " - + format("'%s' = 'json'", OPTION_KEY_FORMAT) - + format(", '%s' = 'json'", OPTION_VALUE_FORMAT) - + format(", 'bootstrap.servers' = '%s'", kafkaTestSupport.getBrokerConnectionString()) - + ", 'auto.offset.reset' = 'earliest'" - + ")"; - - sqlService.execute(createSql); + kafkaMapping(name) + .options(OPTION_KEY_FORMAT, FORMAT_JSON, + OPTION_VALUE_FORMAT, FORMAT_JSON) + .create(); assertTopicEventually( name, "INSERT INTO " + name + " VALUES ('[1,2,3]', '[4,5,6]')", - createMap("[1,2,3]", "[4,5,6]") + Map.of("[1,2,3]", "[4,5,6]") ); assertRowsEventuallyInAnyOrder( "SELECT * FROM " + name, - singletonList(new Row( + List.of(new Row( new HazelcastJsonValue("[1,2,3]"), new HazelcastJsonValue("[4,5,6]") )) ); } + @Test + public void when_explicitTopLevelField_then_fail_key() { + when_explicitTopLevelField_then_fail("__key", "this"); + } + + @Test + public void when_explicitTopLevelField_then_fail_this() { + when_explicitTopLevelField_then_fail("this", "__key"); + } + private void when_explicitTopLevelField_then_fail(String field, String otherField) { assertThatThrownBy(() -> - sqlService.execute("CREATE MAPPING kafka (" - + field + " VARCHAR" - + ", f VARCHAR EXTERNAL NAME \"" + otherField + ".f\"" - + ") TYPE " + KafkaSqlConnector.TYPE_NAME + ' ' - + "OPTIONS (" - + '\'' + OPTION_KEY_FORMAT + "'='" + JSON_FLAT_FORMAT + '\'' - + ", '" + OPTION_VALUE_FORMAT + "'='" + JSON_FLAT_FORMAT + '\'' - + ", 'bootstrap.servers'='" + kafkaTestSupport.getBrokerConnectionString() + '\'' - + ", 'auto.offset.reset'='earliest'" - + ")")) - .isInstanceOf(HazelcastSqlException.class) - .hasMessage("Cannot use the '" + field + "' field with JSON serialization"); + kafkaMapping("kafka") + .fields(field + " VARCHAR", + "f VARCHAR EXTERNAL NAME \"" + otherField + ".f\"") + .create()) + .hasMessage("Cannot use '" + field + "' field with JSON serialization"); } @Test public void test_writingToTopLevel() { String mapName = randomName(); - sqlService.execute("CREATE MAPPING " + mapName + "(" - + "id INT EXTERNAL NAME \"__key.id\"" - + ", name VARCHAR" - + ") TYPE " + KafkaSqlConnector.TYPE_NAME + ' ' - + "OPTIONS (" - + '\'' + OPTION_KEY_FORMAT + "'='" + JSON_FLAT_FORMAT + '\'' - + ", '" + OPTION_VALUE_FORMAT + "'='" + JSON_FLAT_FORMAT + '\'' - + ", 'bootstrap.servers'='" + kafkaTestSupport.getBrokerConnectionString() + '\'' - + ", 'auto.offset.reset'='earliest'" - + ")" - ); + kafkaMapping(mapName) + .fields("id INT EXTERNAL NAME \"__key.id\"", + "name VARCHAR") + .create(); assertThatThrownBy(() -> - sqlService.execute("INSERT INTO " + mapName + "(__key, name) VALUES('{\"id\":1}', null)")) - .isInstanceOf(HazelcastSqlException.class) + sqlService.execute("INSERT INTO " + mapName + "(__key, name) VALUES ('{\"id\":1}', null)")) .hasMessageContaining("Writing to top-level fields of type OBJECT not supported"); assertThatThrownBy(() -> - sqlService.execute("INSERT INTO " + mapName + "(id, this) VALUES(1, '{\"name\":\"foo\"}')")) - .isInstanceOf(HazelcastSqlException.class) + sqlService.execute("INSERT INTO " + mapName + "(id, this) VALUES (1, '{\"name\":\"foo\"}')")) .hasMessageContaining("Writing to top-level fields of type OBJECT not supported"); } @Test public void test_topLevelFieldExtraction() { String name = createRandomTopic(); - sqlService.execute("CREATE MAPPING " + name + " (" - + "id INT EXTERNAL NAME \"__key.id\"" - + ", name VARCHAR" - + ") TYPE " + KafkaSqlConnector.TYPE_NAME + ' ' - + "OPTIONS ( " - + '\'' + OPTION_KEY_FORMAT + "'='" + JSON_FLAT_FORMAT + '\'' - + ", '" + OPTION_VALUE_FORMAT + "'='" + JSON_FLAT_FORMAT + '\'' - + ", 'bootstrap.servers'='" + kafkaTestSupport.getBrokerConnectionString() + '\'' - + ", 'auto.offset.reset'='earliest'" - + ")" - ); + kafkaMapping(name) + .fields("id INT EXTERNAL NAME \"__key.id\"", + "name VARCHAR") + .create(); + sqlService.execute("INSERT INTO " + name + " VALUES (1, 'Alice')"); assertRowsEventuallyInAnyOrder( "SELECT __key, this FROM " + name, - singletonList(new Row( - ImmutableMap.of("id", 1), - ImmutableMap.of("name", "Alice") + List.of(new Row( + Map.of("id", 1), + Map.of("name", "Alice") )) ); } @@ -341,37 +254,28 @@ public void test_topLevelFieldExtraction() { @Test public void test_explicitKeyAndValueSerializers() { String name = createRandomTopic(); - sqlService.execute("CREATE MAPPING " + name + " (" - + "key_name VARCHAR EXTERNAL NAME \"__key.name\"" - + ", value_name VARCHAR EXTERNAL NAME \"this.name\"" - + ") TYPE " + KafkaSqlConnector.TYPE_NAME + ' ' - + "OPTIONS ( " - + '\'' + OPTION_KEY_FORMAT + "'='" + JSON_FLAT_FORMAT + '\'' - + ", '" + OPTION_VALUE_FORMAT + "'='" + JSON_FLAT_FORMAT + '\'' - + ", 'key.serializer'='" + ByteArraySerializer.class.getCanonicalName() + '\'' - + ", 'key.deserializer'='" + ByteArrayDeserializer.class.getCanonicalName() + '\'' - + ", 'value.serializer'='" + ByteArraySerializer.class.getCanonicalName() + '\'' - + ", 'value.deserializer'='" + ByteArrayDeserializer.class.getCanonicalName() + '\'' - + ", 'bootstrap.servers'='" + kafkaTestSupport.getBrokerConnectionString() + '\'' - + ", 'auto.offset.reset'='earliest'" - + ")" - ); + kafkaMapping(name) + .fields("key_name VARCHAR EXTERNAL NAME \"__key.name\"", + "value_name VARCHAR EXTERNAL NAME \"this.name\"") + .options("key.serializer", ByteArraySerializer.class.getCanonicalName(), + "key.deserializer", ByteArrayDeserializer.class.getCanonicalName(), + "value.serializer", ByteArraySerializer.class.getCanonicalName(), + "value.deserializer", ByteArrayDeserializer.class.getCanonicalName()) + .create(); assertTopicEventually( name, "INSERT INTO " + name + " (value_name, key_name) VALUES ('Bob', 'Alice')", - createMap("{\"name\":\"Alice\"}", "{\"name\":\"Bob\"}") + Map.of("{\"name\":\"Alice\"}", "{\"name\":\"Bob\"}") ); assertRowsEventuallyInAnyOrder( "SELECT * FROM " + name, - singletonList(new Row("Alice", "Bob")) + List.of(new Row("Alice", "Bob")) ); } private static String createRandomTopic() { - String topicName = "t_" + randomString().replace('-', '_'); - kafkaTestSupport.createTopic(topicName, INITIAL_PARTITION_COUNT); - return topicName; + return createRandomTopic(INITIAL_PARTITION_COUNT); } private static void assertTopicEventually(String name, String sql, Map expected) { diff --git a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/kafka/SqlPlanCacheTest.java b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/kafka/SqlPlanCacheTest.java index 7fcc421bfe43..a7cf7d165c07 100644 --- a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/kafka/SqlPlanCacheTest.java +++ b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/kafka/SqlPlanCacheTest.java @@ -16,38 +16,21 @@ package com.hazelcast.jet.sql.impl.connector.kafka; -import com.hazelcast.jet.kafka.impl.KafkaTestSupport; -import com.hazelcast.jet.sql.SqlTestSupport; -import com.hazelcast.sql.SqlService; -import org.junit.AfterClass; -import org.junit.BeforeClass; import org.junit.Test; -import java.io.IOException; - import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_FORMAT; import static org.assertj.core.api.Assertions.assertThat; -public class SqlPlanCacheTest extends SqlTestSupport { - +public class SqlPlanCacheTest extends KafkaSqlTestSupport { private static final int INITIAL_PARTITION_COUNT = 4; - private static KafkaTestSupport kafkaTestSupport; - - private static SqlService sqlService; - - @BeforeClass - public static void setUpClass() throws IOException { - initialize(1, null); - sqlService = instance().getSql(); - - kafkaTestSupport = KafkaTestSupport.create(); - kafkaTestSupport.createKafkaCluster(); - } - - @AfterClass - public static void tearDownClass() { - kafkaTestSupport.shutdownKafkaCluster(); + private static void createMapping(String tableName, String topicName, String valueFormat, String offset) { + new SqlMapping(tableName, KafkaSqlConnector.class) + .externalName(topicName) + .options(OPTION_VALUE_FORMAT, valueFormat, + "bootstrap.servers", kafkaTestSupport.getBrokerConnectionString(), + "auto.offset.reset", offset) + .createOrReplace(); } @Test @@ -113,24 +96,6 @@ public void test_dmlCaching() { } private static String createRandomTopic() { - String topicName = randomName(); - kafkaTestSupport.createTopic(topicName, INITIAL_PARTITION_COUNT); - return topicName; - } - - private static void createMapping( - String tableName, - String topicName, - String valueFormat, - String offset - ) { - sqlService.execute("CREATE OR REPLACE MAPPING " + tableName + " EXTERNAL NAME " + topicName + ' ' - + "TYPE " + KafkaSqlConnector.TYPE_NAME + ' ' - + "OPTIONS ( " - + '\'' + OPTION_VALUE_FORMAT + "'='" + valueFormat + '\'' - + ", 'bootstrap.servers'='" + kafkaTestSupport.getBrokerConnectionString() + '\'' - + ", 'auto.offset.reset'='" + offset + '\'' - + ")" - ); + return createRandomTopic(INITIAL_PARTITION_COUNT); } } diff --git a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/kafka/SqlPojoTest.java b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/kafka/SqlPojoTest.java index d6e00e34cdc0..e07d7be42388 100644 --- a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/kafka/SqlPojoTest.java +++ b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/kafka/SqlPojoTest.java @@ -16,9 +16,6 @@ package com.hazelcast.jet.sql.impl.connector.kafka; -import com.hazelcast.config.Config; -import com.hazelcast.jet.kafka.impl.KafkaTestSupport; -import com.hazelcast.jet.sql.SqlTestSupport; import com.hazelcast.jet.sql.impl.connector.kafka.model.AllCanonicalTypesValue; import com.hazelcast.jet.sql.impl.connector.kafka.model.AllCanonicalTypesValueDeserializer; import com.hazelcast.jet.sql.impl.connector.kafka.model.AllCanonicalTypesValueSerializer; @@ -27,148 +24,114 @@ import com.hazelcast.jet.sql.impl.connector.kafka.model.Person; import com.hazelcast.jet.sql.impl.connector.kafka.model.PersonId; import com.hazelcast.jet.sql.impl.connector.test.TestAllTypesSqlConnector; -import com.hazelcast.spi.properties.ClusterProperty; -import com.hazelcast.sql.HazelcastSqlException; -import com.hazelcast.sql.SqlService; +import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.IntegerSerializer; -import org.junit.AfterClass; +import org.apache.kafka.common.serialization.Serializer; import org.junit.BeforeClass; import org.junit.Test; -import java.io.IOException; import java.io.Serializable; import java.math.BigDecimal; import java.time.LocalDate; import java.time.LocalDateTime; import java.time.LocalTime; import java.time.OffsetDateTime; +import java.util.List; import java.util.Map; -import static com.hazelcast.jet.core.TestUtil.createMap; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.JAVA_FORMAT; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_KEY_CLASS; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_KEY_FORMAT; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_CLASS; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_FORMAT; +import static com.hazelcast.spi.properties.ClusterProperty.SQL_CUSTOM_TYPES_ENABLED; import static java.time.ZoneOffset.UTC; -import static java.util.Collections.singletonList; import static org.assertj.core.api.Assertions.assertThatThrownBy; -public class SqlPojoTest extends SqlTestSupport { - +public class SqlPojoTest extends KafkaSqlTestSupport { private static final int INITIAL_PARTITION_COUNT = 4; - private static KafkaTestSupport kafkaTestSupport; - - private static SqlService sqlService; - @BeforeClass - public static void setUpClass() throws IOException { - Config config = smallInstanceConfig(); - config.setProperty(ClusterProperty.SQL_CUSTOM_TYPES_ENABLED.getName(), "true"); - initialize(1, config); - sqlService = instance().getSql(); - - kafkaTestSupport = KafkaTestSupport.create(); - kafkaTestSupport.createKafkaCluster(); + public static void setup() throws Exception { + setup(1, smallInstanceConfig().setProperty(SQL_CUSTOM_TYPES_ENABLED.getName(), "true")); } - @AfterClass - public static void tearDownClass() { - kafkaTestSupport.shutdownKafkaCluster(); + private static SqlMapping kafkaMapping(String name, Class keyClass, Class valueClass, + Class> keySerializerClass, + Class> keyDeserializerClass, + Class> valueSerializerClass, + Class> valueDeserializerClass) { + return new SqlMapping(name, KafkaSqlConnector.class) + .options(OPTION_KEY_FORMAT, JAVA_FORMAT, + OPTION_KEY_CLASS, keyClass.getName(), + OPTION_VALUE_FORMAT, JAVA_FORMAT, + OPTION_VALUE_CLASS, valueClass.getName(), + "bootstrap.servers", kafkaTestSupport.getBrokerConnectionString(), + "key.serializer", keySerializerClass.getCanonicalName(), + "key.deserializer", keyDeserializerClass.getCanonicalName(), + "value.serializer", valueSerializerClass.getCanonicalName(), + "value.deserializer", valueDeserializerClass.getCanonicalName(), + "auto.offset.reset", "earliest"); } @Test public void test_nulls() { String name = createRandomTopic(); - sqlService.execute("CREATE MAPPING " + name + ' ' - + "TYPE " + KafkaSqlConnector.TYPE_NAME + ' ' - + "OPTIONS ( " - + '\'' + OPTION_KEY_FORMAT + "'='" + JAVA_FORMAT + '\'' - + ", '" + OPTION_KEY_CLASS + "'='" + PersonId.class.getName() + '\'' - + ", '" + OPTION_VALUE_FORMAT + "'='" + JAVA_FORMAT + '\'' - + ", '" + OPTION_VALUE_CLASS + "'='" + Person.class.getName() + '\'' - + ", 'bootstrap.servers'='" + kafkaTestSupport.getBrokerConnectionString() + '\'' - + ", 'key.serializer'='" + JavaSerializer.class.getCanonicalName() + '\'' - + ", 'key.deserializer'='" + JavaDeserializer.class.getCanonicalName() + '\'' - + ", 'value.serializer'='" + JavaSerializer.class.getCanonicalName() + '\'' - + ", 'value.deserializer'='" + JavaDeserializer.class.getCanonicalName() + '\'' - + ", 'auto.offset.reset'='earliest'" - + ")" - ); + kafkaMapping(name, PersonId.class, Person.class, + JavaSerializer.class, JavaDeserializer.class, + JavaSerializer.class, JavaDeserializer.class) + .create(); assertTopicEventually( name, "INSERT INTO " + name + " VALUES (null, null)", - createMap(new PersonId(), new Person()) + Map.of(new PersonId(), new Person()) ); assertRowsEventuallyInAnyOrder( "SELECT * FROM " + name, - singletonList(new Row(null, null)) + List.of(new Row(null, null)) ); } @Test public void test_fieldsShadowing() { String name = createRandomTopic(); - sqlService.execute("CREATE MAPPING " + name + ' ' - + "TYPE " + KafkaSqlConnector.TYPE_NAME + ' ' - + "OPTIONS ( " - + '\'' + OPTION_KEY_FORMAT + "'='" + JAVA_FORMAT + '\'' - + ", '" + OPTION_KEY_CLASS + "'='" + PersonId.class.getName() + '\'' - + ", '" + OPTION_VALUE_FORMAT + "'='" + JAVA_FORMAT + '\'' - + ", '" + OPTION_VALUE_CLASS + "'='" + Person.class.getName() + '\'' - + ", 'bootstrap.servers'='" + kafkaTestSupport.getBrokerConnectionString() + '\'' - + ", 'key.serializer'='" + JavaSerializer.class.getCanonicalName() + '\'' - + ", 'key.deserializer'='" + JavaDeserializer.class.getCanonicalName() + '\'' - + ", 'value.serializer'='" + JavaSerializer.class.getCanonicalName() + '\'' - + ", 'value.deserializer'='" + JavaDeserializer.class.getCanonicalName() + '\'' - + ", 'auto.offset.reset'='earliest'" - + ")" - ); + kafkaMapping(name, PersonId.class, Person.class, + JavaSerializer.class, JavaDeserializer.class, + JavaSerializer.class, JavaDeserializer.class) + .create(); assertTopicEventually( name, "INSERT INTO " + name + " VALUES (1, 'Alice')", - createMap(new PersonId(1), new Person(null, "Alice")) + Map.of(new PersonId(1), new Person(null, "Alice")) ); assertRowsEventuallyInAnyOrder( "SELECT * FROM " + name, - singletonList(new Row(1, "Alice")) + List.of(new Row(1, "Alice")) ); } @Test public void test_fieldsMapping() { String name = createRandomTopic(); - sqlService.execute("CREATE MAPPING " + name + " (" - + "key_id INT EXTERNAL NAME \"__key.id\"" - + ", value_id INT EXTERNAL NAME \"this.id\"" - + ", name VARCHAR" - + ") TYPE " + KafkaSqlConnector.TYPE_NAME + ' ' - + "OPTIONS ( " - + '\'' + OPTION_KEY_FORMAT + "'='" + JAVA_FORMAT + '\'' - + ", '" + OPTION_KEY_CLASS + "'='" + PersonId.class.getName() + '\'' - + ", '" + OPTION_VALUE_FORMAT + "'='" + JAVA_FORMAT + '\'' - + ", '" + OPTION_VALUE_CLASS + "'='" + Person.class.getName() + '\'' - + ", 'bootstrap.servers'='" + kafkaTestSupport.getBrokerConnectionString() + '\'' - + ", 'key.serializer'='" + JavaSerializer.class.getCanonicalName() + '\'' - + ", 'key.deserializer'='" + JavaDeserializer.class.getCanonicalName() + '\'' - + ", 'value.serializer'='" + JavaSerializer.class.getCanonicalName() + '\'' - + ", 'value.deserializer'='" + JavaDeserializer.class.getCanonicalName() + '\'' - + ", 'auto.offset.reset'='earliest'" - + ")" - ); + kafkaMapping(name, PersonId.class, Person.class, + JavaSerializer.class, JavaDeserializer.class, + JavaSerializer.class, JavaDeserializer.class) + .fields("key_id INT EXTERNAL NAME \"__key.id\"", + "value_id INT EXTERNAL NAME \"this.id\"", + "name VARCHAR") + .create(); assertTopicEventually( name, "INSERT INTO " + name + " (value_id, key_id) VALUES (2, 1)", - createMap(new PersonId(1), new Person(2, null)) + Map.of(new PersonId(1), new Person(2, null)) ); assertRowsEventuallyInAnyOrder( "SELECT key_id, value_id FROM " + name, - singletonList(new Row(1, 2)) + List.of(new Row(1, 2)) ); } @@ -178,21 +141,10 @@ public void test_allTypes() { TestAllTypesSqlConnector.create(sqlService, from); String to = createRandomTopic(); - sqlService.execute("CREATE MAPPING " + to + ' ' - + " TYPE " + KafkaSqlConnector.TYPE_NAME + ' ' - + "OPTIONS ( " - + '\'' + OPTION_KEY_FORMAT + "'='" + JAVA_FORMAT + '\'' - + ", '" + OPTION_KEY_CLASS + "'='" + PersonId.class.getName() + '\'' - + ", '" + OPTION_VALUE_FORMAT + "'='" + JAVA_FORMAT + '\'' - + ", '" + OPTION_VALUE_CLASS + "'='" + AllCanonicalTypesValue.class.getName() + '\'' - + ", 'bootstrap.servers'='" + kafkaTestSupport.getBrokerConnectionString() + '\'' - + ", 'key.serializer'='" + JavaSerializer.class.getCanonicalName() + '\'' - + ", 'key.deserializer'='" + JavaDeserializer.class.getCanonicalName() + '\'' - + ", 'value.serializer'='" + AllCanonicalTypesValueSerializer.class.getCanonicalName() + '\'' - + ", 'value.deserializer'='" + AllCanonicalTypesValueDeserializer.class.getCanonicalName() + '\'' - + ", 'auto.offset.reset'='earliest'" - + ")" - ); + kafkaMapping(to, PersonId.class, AllCanonicalTypesValue.class, + JavaSerializer.class, JavaDeserializer.class, + AllCanonicalTypesValueSerializer.class, AllCanonicalTypesValueDeserializer.class) + .create(); sqlService.execute("INSERT INTO " + to + "(" + "id" @@ -235,7 +187,7 @@ public void test_allTypes() { + ", timestampTz" + ", object" + " FROM " + to, - singletonList(new Row( + List.of(new Row( 1, "string", true, @@ -267,97 +219,65 @@ public void test_writingToTopLevelWhileNestedFieldMapped_implicit() { public void test_writingToTopLevel(boolean explicit) { String topicName = createRandomTopic(); - sqlService.execute("CREATE MAPPING " + topicName + "(" - + "__key INT" - + (explicit ? ", this OBJECT" : "") - + ", name VARCHAR" - + ") TYPE " + KafkaSqlConnector.TYPE_NAME + "\n" - + "OPTIONS (\n" - + '\'' + OPTION_KEY_FORMAT + "'='" + JAVA_FORMAT + "'\n" - + ", '" + OPTION_KEY_CLASS + "'='" + Integer.class.getName() + "'\n" - + ", '" + OPTION_VALUE_FORMAT + "'='" + JAVA_FORMAT + "'\n" - + ", '" + OPTION_VALUE_CLASS + "'='" + Person.class.getName() + "'\n" - + ", 'bootstrap.servers'='" + kafkaTestSupport.getBrokerConnectionString() + '\'' - + ", 'key.serializer'='" + IntegerSerializer.class.getCanonicalName() + '\'' - + ", 'key.deserializer'='" + IntegerDeserializer.class.getCanonicalName() + '\'' - + ", 'value.serializer'='" + JavaSerializer.class.getCanonicalName() + '\'' - + ", 'value.deserializer'='" + JavaDeserializer.class.getCanonicalName() + '\'' - + ", 'auto.offset.reset'='earliest'" - + ")" - ); + kafkaMapping(topicName, Integer.class, Person.class, + IntegerSerializer.class, IntegerDeserializer.class, + JavaSerializer.class, JavaDeserializer.class) + .fields("__key INT") + .fieldsIf(explicit, "this OBJECT") + .fields("name VARCHAR") + .create(); if (explicit) { assertThatThrownBy(() -> sqlService.execute("INSERT INTO " + topicName + " VALUES(1, null, 'foo')")) - .isInstanceOf(HazelcastSqlException.class) .hasMessageContaining("Writing to top-level fields of type OBJECT not supported"); } assertThatThrownBy(() -> sqlService.execute("INSERT INTO " + topicName + "(__key, this) VALUES(1, null)")) - .isInstanceOf(HazelcastSqlException.class) .hasMessageContaining("Writing to top-level fields of type OBJECT not supported"); sqlService.execute("INSERT INTO " + topicName + (explicit ? "(__key, name)" : "") + " VALUES (1, 'foo')"); assertRowsEventuallyInAnyOrder("SELECT __key, this, name FROM " + topicName, - singletonList(new Row(1, new Person(null, "foo"), "foo"))); + List.of(new Row(1, new Person(null, "foo"), "foo"))); } @Test public void test_topLevelFieldExtraction() { String name = createRandomTopic(); - sqlService.execute("CREATE MAPPING " + name + ' ' - + "TYPE " + KafkaSqlConnector.TYPE_NAME + ' ' - + "OPTIONS ( " - + '\'' + OPTION_KEY_FORMAT + "'='" + JAVA_FORMAT + '\'' - + ", '" + OPTION_KEY_CLASS + "'='" + PersonId.class.getName() + '\'' - + ", '" + OPTION_VALUE_FORMAT + "'='" + JAVA_FORMAT + '\'' - + ", '" + OPTION_VALUE_CLASS + "'='" + Person.class.getName() + '\'' - + ", 'bootstrap.servers'='" + kafkaTestSupport.getBrokerConnectionString() + '\'' - + ", 'key.serializer'='" + JavaSerializer.class.getCanonicalName() + '\'' - + ", 'key.deserializer'='" + JavaDeserializer.class.getCanonicalName() + '\'' - + ", 'value.serializer'='" + JavaSerializer.class.getCanonicalName() + '\'' - + ", 'value.deserializer'='" + JavaDeserializer.class.getCanonicalName() + '\'' - + ", 'auto.offset.reset'='earliest'" - + ")" - ); + kafkaMapping(name, PersonId.class, Person.class, + JavaSerializer.class, JavaDeserializer.class, + JavaSerializer.class, JavaDeserializer.class) + .create(); + sqlService.execute("INSERT INTO " + name + " VALUES (1, 'Alice')"); assertRowsEventuallyInAnyOrder( "SELECT __key, this FROM " + name, - singletonList(new Row(new PersonId(1), new Person(null, "Alice"))) + List.of(new Row(new PersonId(1), new Person(null, "Alice"))) ); } @Test public void test_customType() { - sqlService.execute("CREATE TYPE person_type OPTIONS ('format'='java', 'javaClass'='" + Person.class.getName() + "')"); - sqlService.execute("CREATE MAPPING m (outerField INT, person person_type) TYPE " + KafkaSqlConnector.TYPE_NAME + " OPTIONS (" - + "'bootstrap.servers'='" + kafkaTestSupport.getBrokerConnectionString() + "'," - + "'auto.offset.reset'='earliest'," - + "'valueFormat'='java'," - + "'valueJavaClass'='" + ClzWithPerson.class.getName() + "'," - + "'value.serializer'='" + JavaSerializer.class.getName() + "'," - + "'value.deserializer'='" + JavaDeserializer.class.getName() + "'" - + ")" - ); + new SqlType("person_type").create(); + kafkaMapping("m", Integer.class, ClzWithPerson.class, + IntegerSerializer.class, IntegerDeserializer.class, + JavaSerializer.class, JavaDeserializer.class) + .fields("__key INT", + "outerField INT", + "person person_type") + .create(); - sqlService.execute("insert into m values (1, (2, 'foo'))"); + sqlService.execute("insert into m values (0, 1, (2, 'foo'))"); assertRowsEventuallyInAnyOrder("select outerField, (person).id, (person).name from m", rows(3, 1, 2, "foo")); } - public static class ClzWithPerson implements Serializable { - public int outerField; - public Person person; - } - private static String createRandomTopic() { - String topicName = "t_" + randomString().replace('-', '_'); - kafkaTestSupport.createTopic(topicName, INITIAL_PARTITION_COUNT); - return topicName; + return createRandomTopic(INITIAL_PARTITION_COUNT); } private static void assertTopicEventually(String name, String sql, Map expected) { @@ -370,4 +290,10 @@ private static void assertTopicEventually(String name, String sql, Map sqlService.execute("CREATE MAPPING map (" - + "id INT EXTERNAL NAME __key" - + ", this VARCHAR" - + ") TYPE " + KafkaSqlConnector.TYPE_NAME + ' ' - + "OPTIONS (" - + '\'' + OPTION_KEY_FORMAT + "'='int'" - + ", '" + OPTION_VALUE_FORMAT + "'='varchar'" - + ", 'bootstrap.servers'='" + kafkaTestSupport.getBrokerConnectionString() + '\'' - + ", 'auto.offset.reset'='earliest'" - + ")" - )).hasMessageContaining("Cannot rename field: '__key'"); - - assertThatThrownBy(() -> sqlService.execute("CREATE MAPPING map (" - + "__key INT EXTERNAL NAME renamed" - + ", this VARCHAR" - + ") TYPE " + KafkaSqlConnector.TYPE_NAME + ' ' - + "OPTIONS (" - + '\'' + OPTION_KEY_FORMAT + "'='int'" - + ", '" + OPTION_VALUE_FORMAT + "'='varchar'" - + ", 'bootstrap.servers'='" + kafkaTestSupport.getBrokerConnectionString() + '\'' - + ", 'auto.offset.reset'='earliest'" - + ")" - )).hasMessageContaining("Cannot rename field: '__key'"); + assertThatThrownBy(() -> + kafkaMapping("map", true) + .fields("id INT EXTERNAL NAME __key", + "this VARCHAR") + .create()) + .hasMessageContaining("Cannot rename field: '__key'"); + + assertThatThrownBy(() -> + kafkaMapping("map", true) + .fields("__key INT EXTERNAL NAME renamed", + "this VARCHAR") + .create()) + .hasMessageContaining("Cannot rename field: '__key'"); } @Test public void test_renameThis() { - assertThatThrownBy(() -> sqlService.execute("CREATE MAPPING map (" - + "__key INT" - + ", name VARCHAR EXTERNAL NAME this" - + ") TYPE " + KafkaSqlConnector.TYPE_NAME + ' ' - + "OPTIONS (" - + '\'' + OPTION_KEY_FORMAT + "'='int'" - + ", '" + OPTION_VALUE_FORMAT + "'='varchar'" - + ", 'bootstrap.servers'='" + kafkaTestSupport.getBrokerConnectionString() + '\'' - + ", 'auto.offset.reset'='earliest'" - + ")" - )).hasMessageContaining("Cannot rename field: 'this'"); - - assertThatThrownBy(() -> sqlService.execute("CREATE MAPPING map (" - + "__key INT" - + ", this VARCHAR EXTERNAL NAME renamed" - + ") TYPE " + KafkaSqlConnector.TYPE_NAME + ' ' - + "OPTIONS (" - + '\'' + OPTION_KEY_FORMAT + "'='int'" - + ", '" + OPTION_VALUE_FORMAT + "'='varchar'" - + ", 'bootstrap.servers'='" + kafkaTestSupport.getBrokerConnectionString() + '\'' - + ", 'auto.offset.reset'='earliest'" - + ")" - )).hasMessageContaining("Cannot rename field: 'this'"); + assertThatThrownBy(() -> + kafkaMapping("map", true) + .fields("__key INT", + "name VARCHAR EXTERNAL NAME this") + .create()) + .hasMessageContaining("Cannot rename field: 'this'"); + + assertThatThrownBy(() -> + kafkaMapping("map", true) + .fields("__key INT", + "this VARCHAR EXTERNAL NAME renamed") + .create()) + .hasMessageContaining("Cannot rename field: 'this'"); } @Test @@ -382,17 +271,9 @@ public void test_objectAndMappingNameDifferent() { String topicName = createRandomTopic(); String tableName = randomName(); - sqlService.execute("CREATE MAPPING " + tableName + " EXTERNAL NAME " + topicName + ' ' - + "TYPE " + KafkaSqlConnector.TYPE_NAME + ' ' - + "OPTIONS ( " - + '\'' + OPTION_KEY_FORMAT + "'='" + JAVA_FORMAT + '\'' - + ", '" + OPTION_KEY_CLASS + "'='" + Integer.class.getName() + '\'' - + ", '" + OPTION_VALUE_FORMAT + "'='" + JAVA_FORMAT + '\'' - + ", '" + OPTION_VALUE_CLASS + "'='" + String.class.getName() + '\'' - + ", 'bootstrap.servers'='" + kafkaTestSupport.getBrokerConnectionString() + '\'' - + ", 'auto.offset.reset'='earliest'" - + ")" - ); + kafkaMapping(tableName, false) + .externalName(topicName) + .create(); kafkaTestSupport.produce(topicName, 1, "Alice"); kafkaTestSupport.produce(topicName, 2, "Bob"); @@ -413,20 +294,10 @@ public void test_objectAndMappingNameDifferent() { @Test public void test_explicitKeyAndThis() { String topicName = createRandomTopic(); - - sqlService.execute("CREATE MAPPING " + topicName + '(' - + "__key INT," - + "this VARCHAR" + - ") TYPE " + KafkaSqlConnector.TYPE_NAME + ' ' - + "OPTIONS ( " - + '\'' + OPTION_KEY_FORMAT + "'='" + JAVA_FORMAT + '\'' - + ", '" + OPTION_KEY_CLASS + "'='" + Integer.class.getName() + '\'' - + ", '" + OPTION_VALUE_FORMAT + "'='" + JAVA_FORMAT + '\'' - + ", '" + OPTION_VALUE_CLASS + "'='" + String.class.getName() + '\'' - + ", 'bootstrap.servers'='" + kafkaTestSupport.getBrokerConnectionString() + '\'' - + ", 'auto.offset.reset'='earliest'" - + ")" - ); + kafkaMapping(topicName, false) + .fields("__key INT", + "this VARCHAR") + .create(); kafkaTestSupport.produce(topicName, 1, "Alice"); kafkaTestSupport.produce(topicName, 2, "Bob"); @@ -447,20 +318,10 @@ public void test_explicitKeyAndThis() { @Test public void test_explicitKeyAndThisWithExternalNames() { String topicName = createRandomTopic(); - - sqlService.execute("CREATE MAPPING " + topicName + '(' - + "__key INT EXTERNAL NAME __key," - + "this VARCHAR EXTERNAL NAME this" + - ") TYPE " + KafkaSqlConnector.TYPE_NAME + ' ' - + "OPTIONS ( " - + '\'' + OPTION_KEY_FORMAT + "'='" + JAVA_FORMAT + '\'' - + ", '" + OPTION_KEY_CLASS + "'='" + Integer.class.getName() + '\'' - + ", '" + OPTION_VALUE_FORMAT + "'='" + JAVA_FORMAT + '\'' - + ", '" + OPTION_VALUE_CLASS + "'='" + String.class.getName() + '\'' - + ", 'bootstrap.servers'='" + kafkaTestSupport.getBrokerConnectionString() + '\'' - + ", 'auto.offset.reset'='earliest'" - + ")" - ); + kafkaMapping(topicName, false) + .fields("__key INT EXTERNAL NAME __key", + "this VARCHAR EXTERNAL NAME this") + .create(); kafkaTestSupport.produce(topicName, 1, "Alice"); kafkaTestSupport.produce(topicName, 2, "Bob"); @@ -481,21 +342,12 @@ public void test_explicitKeyAndThisWithExternalNames() { @Test public void test_explicitKeyAndValueSerializers() { String name = createRandomTopic(); - sqlService.execute("CREATE MAPPING " + name + ' ' - + "TYPE " + KafkaSqlConnector.TYPE_NAME + ' ' - + "OPTIONS ( " - + '\'' + OPTION_KEY_FORMAT + "'='" + JAVA_FORMAT + '\'' - + ", '" + OPTION_KEY_CLASS + "'='" + Integer.class.getName() + '\'' - + ", '" + OPTION_VALUE_FORMAT + "'='" + JAVA_FORMAT + '\'' - + ", '" + OPTION_VALUE_CLASS + "'='" + String.class.getName() + '\'' - + ", 'key.serializer'='" + IntegerSerializer.class.getCanonicalName() + '\'' - + ", 'key.deserializer'='" + IntegerDeserializer.class.getCanonicalName() + '\'' - + ", 'value.serializer'='" + StringSerializer.class.getCanonicalName() + '\'' - + ", 'value.deserializer'='" + StringDeserializer.class.getCanonicalName() + '\'' - + ", 'bootstrap.servers'='" + kafkaTestSupport.getBrokerConnectionString() + '\'' - + ", 'auto.offset.reset'='earliest'" - + ")" - ); + kafkaMapping(name, false) + .options("key.serializer", IntegerSerializer.class.getCanonicalName(), + "key.deserializer", IntegerDeserializer.class.getCanonicalName(), + "value.serializer", StringSerializer.class.getCanonicalName(), + "value.deserializer", StringDeserializer.class.getCanonicalName()) + .create(); assertTopicEventually( name, @@ -511,15 +363,12 @@ public void test_explicitKeyAndValueSerializers() { @Test public void test_noKeyFormat() { String topicName = createRandomTopic(); - sqlService.execute("CREATE MAPPING " + topicName + ' ' - + "TYPE " + KafkaSqlConnector.TYPE_NAME + ' ' - + "OPTIONS (" - + '\'' + OPTION_VALUE_FORMAT + "'='" + JAVA_FORMAT + "'," - + '\'' + OPTION_VALUE_CLASS + "'='" + Integer.class.getName() + "'" - + ", 'bootstrap.servers'='" + kafkaTestSupport.getBrokerConnectionString() + '\'' - + ", 'auto.offset.reset'='earliest'" - + ")" - ); + new SqlMapping(topicName, KafkaSqlConnector.class) + .options(OPTION_VALUE_FORMAT, JAVA_FORMAT, + OPTION_VALUE_CLASS, Integer.class.getName(), + "bootstrap.servers", kafkaTestSupport.getBrokerConnectionString(), + "auto.offset.reset", "earliest") + .create(); sqlService.execute("INSERT INTO " + topicName + " VALUES(42)"); @@ -528,13 +377,11 @@ public void test_noKeyFormat() { @Test public void test_noValueFormat() { - assertThatThrownBy( - () -> sqlService.execute("CREATE MAPPING kafka " - + "TYPE " + KafkaSqlConnector.TYPE_NAME + " " - + "OPTIONS (" - + '\'' + OPTION_KEY_FORMAT + "'='" + JAVA_FORMAT + "'," - + '\'' + OPTION_KEY_CLASS + "'='" + String.class.getName() + "'" - + ")")) + assertThatThrownBy(() -> + new SqlMapping("kafka", KafkaSqlConnector.class) + .options(OPTION_KEY_FORMAT, JAVA_FORMAT, + OPTION_KEY_CLASS, String.class.getName()) + .create()) .hasMessage("Missing 'valueFormat' option"); } @@ -549,44 +396,31 @@ public void test_multipleFieldsForPrimitive_value() { } private void test_multipleFieldsForPrimitive(String fieldName) { - assertThatThrownBy( - () -> sqlService.execute("CREATE MAPPING kafka (" - + fieldName + " INT" - + ", field INT EXTERNAL NAME \"" + fieldName + ".field\"" - + ") TYPE " + KafkaSqlConnector.TYPE_NAME - + " OPTIONS (" - + '\'' + OPTION_KEY_FORMAT + "'='" + JAVA_FORMAT + '\'' - + ", '" + OPTION_KEY_CLASS + "'='" + Integer.class.getName() + '\'' - + ", '" + OPTION_VALUE_FORMAT + "'='" + JAVA_FORMAT + '\'' - + ", '" + OPTION_VALUE_CLASS + "'='" + Integer.class.getName() + '\'' - + ")") - ).hasMessage("The field '" + fieldName + "' is of type INTEGER, you can't map '" + fieldName + ".field' too"); + assertThatThrownBy(() -> + new SqlMapping("kafka", KafkaSqlConnector.class) + .fields(fieldName + " INT", + "field INT EXTERNAL NAME \"" + fieldName + ".field\"") + .options(OPTION_KEY_FORMAT, JAVA_FORMAT, + OPTION_KEY_CLASS, Integer.class.getName(), + OPTION_VALUE_FORMAT, JAVA_FORMAT, + OPTION_VALUE_CLASS, Integer.class.getName()) + .create()) + .hasMessage("The field '" + fieldName + "' is of type INTEGER, you can't map '" + + fieldName + ".field' too"); } // test for https://github.com/hazelcast/hazelcast/issues/21455 @Test public void test_nonExistentTopic() { String name = "nonExistentTopic"; - sqlService.execute("CREATE MAPPING " + name + ' ' - + "TYPE " + KafkaSqlConnector.TYPE_NAME + ' ' - + "OPTIONS ( " - + '\'' + OPTION_KEY_FORMAT + "'='" + JAVA_FORMAT + '\'' - + ", '" + OPTION_KEY_CLASS + "'='" + Integer.class.getName() + '\'' - + ", '" + OPTION_VALUE_FORMAT + "'='" + JAVA_FORMAT + '\'' - + ", '" + OPTION_VALUE_CLASS + "'='" + String.class.getName() + '\'' - + ", 'bootstrap.servers'='" + kafkaTestSupport.getBrokerConnectionString() + '\'' - + ", 'auto.offset.reset'='earliest'" - + ")" - ); + createMapping(name, false); ResultIterator result = (ResultIterator) sqlService.execute("select * from " + name).iterator(); result.hasNext(500, TimeUnit.MILLISECONDS); } private static String createRandomTopic() { - String topicName = randomName(); - kafkaTestSupport.createTopic(topicName, INITIAL_PARTITION_COUNT); - return topicName; + return createRandomTopic(INITIAL_PARTITION_COUNT); } private static void assertTopicEventually(String name, String sql, Map expected) { diff --git a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/keyvalue/KvMetadataAvroResolverTest.java b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/keyvalue/KvMetadataAvroResolverTest.java index 8c5aebd3b145..6c0dc0848649 100644 --- a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/keyvalue/KvMetadataAvroResolverTest.java +++ b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/keyvalue/KvMetadataAvroResolverTest.java @@ -29,13 +29,12 @@ import org.junit.Test; import org.junit.runner.RunWith; +import java.util.List; import java.util.stream.Stream; import static com.hazelcast.jet.sql.impl.connector.keyvalue.KvMetadataAvroResolver.INSTANCE; import static com.hazelcast.jet.sql.impl.connector.keyvalue.KvMetadataAvroResolver.Schemas.OBJECT_SCHEMA; -import static java.util.Arrays.asList; import static java.util.Collections.emptyMap; -import static java.util.Collections.singletonList; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -50,7 +49,7 @@ public class KvMetadataAvroResolverTest { public void test_resolveFields(boolean key, String prefix) { Stream fields = INSTANCE.resolveAndValidateFields( key, - singletonList(field("field", QueryDataType.INT, prefix + ".field")), + List.of(field("field", QueryDataType.INT, prefix + ".field")), emptyMap(), null ); @@ -66,19 +65,18 @@ public void test_resolveFields(boolean key, String prefix) { public void when_noKeyOrThisPrefixInExternalName_then_usesValue(boolean key) { KvMetadata metadata = INSTANCE.resolveMetadata( key, - singletonList(field("field", QueryDataType.INT, "extField")), + List.of(field("field", QueryDataType.INT, "extField")), emptyMap(), null ); - assertThat(metadata.getFields()).containsExactly( - key - ? new MapTableField[]{ - new MapTableField("__key", QueryDataType.OBJECT, true, QueryPath.KEY_PATH) - } - : new MapTableField[]{ - new MapTableField("field", QueryDataType.INT, false, new QueryPath("extField", false)), - new MapTableField("this", QueryDataType.OBJECT, true, QueryPath.VALUE_PATH) - }); + assertThat(metadata.getFields()).containsExactly(key + ? new MapTableField[]{ + new MapTableField("__key", QueryDataType.OBJECT, true, QueryPath.KEY_PATH) + } + : new MapTableField[]{ + new MapTableField("field", QueryDataType.INT, false, new QueryPath("extField", false)), + new MapTableField("this", QueryDataType.OBJECT, true, QueryPath.VALUE_PATH) + }); } @Test @@ -89,7 +87,7 @@ public void when_noKeyOrThisPrefixInExternalName_then_usesValue(boolean key) { public void when_duplicateExternalName_then_throws(boolean key, String prefix) { assertThatThrownBy(() -> INSTANCE.resolveAndValidateFields( key, - asList( + List.of( field("field1", QueryDataType.INT, prefix + ".field"), field("field2", QueryDataType.VARCHAR, prefix + ".field") ), @@ -108,7 +106,7 @@ public void when_duplicateExternalName_then_throws(boolean key, String prefix) { public void test_resolveMetadata(boolean key, String prefix) { KvMetadata metadata = INSTANCE.resolveMetadata( key, - asList( + List.of( field("string", QueryDataType.VARCHAR, prefix + ".string"), field("boolean", QueryDataType.BOOLEAN, prefix + ".boolean"), field("byte", QueryDataType.TINYINT, prefix + ".byte"), diff --git a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/map/MetadataCompactResolverTest.java b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/map/MetadataCompactResolverTest.java index 89796e134a2e..39e788b9ba72 100644 --- a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/map/MetadataCompactResolverTest.java +++ b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/map/MetadataCompactResolverTest.java @@ -16,7 +16,6 @@ package com.hazelcast.jet.sql.impl.connector.map; -import com.google.common.collect.ImmutableMap; import com.hazelcast.config.SerializationConfig; import com.hazelcast.internal.serialization.InternalSerializationService; import com.hazelcast.internal.serialization.impl.DefaultSerializationServiceBuilder; @@ -38,18 +37,16 @@ import org.junit.Test; import org.junit.runner.RunWith; -import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.stream.Collectors; import java.util.stream.Stream; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_KEY_COMPACT_TYPE_NAME; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_COMPACT_TYPE_NAME; import static com.hazelcast.jet.sql.impl.connector.map.MetadataCompactResolver.INSTANCE; -import static java.util.Arrays.asList; import static java.util.Collections.emptyList; -import static java.util.Collections.singletonList; +import static java.util.Collections.emptyMap; +import static java.util.stream.Collectors.toList; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.Assert.assertEquals; @@ -66,7 +63,7 @@ public void test_columnListIsRequiredForCompact(boolean key) { InternalSerializationService ss = createSerializationService(); Map options = - ImmutableMap.of((key ? OPTION_KEY_COMPACT_TYPE_NAME : OPTION_VALUE_COMPACT_TYPE_NAME), "testAll"); + Map.of(key ? OPTION_KEY_COMPACT_TYPE_NAME : OPTION_VALUE_COMPACT_TYPE_NAME, "testAll"); // TODO: fix compact nested types support? assertThatThrownBy(() -> INSTANCE.resolveAndValidateFields(key, emptyList(), options, ss)) @@ -82,12 +79,12 @@ public void test_objectIsForbiddenForCompact(boolean key, String prefix) { InternalSerializationService ss = createSerializationService(); Map options = - ImmutableMap.of((key ? OPTION_KEY_COMPACT_TYPE_NAME : OPTION_VALUE_COMPACT_TYPE_NAME), "testAll"); + Map.of(key ? OPTION_KEY_COMPACT_TYPE_NAME : OPTION_VALUE_COMPACT_TYPE_NAME, "testAll"); - List fields = asList(field("object", QueryDataType.OBJECT, prefix + ".object")); + List fields = List.of(field("object", QueryDataType.OBJECT, prefix + ".object")); // TODO: fix compact nested types support? - assertThatThrownBy(() -> INSTANCE.resolveAndValidateFields(key, fields, options, ss).collect(Collectors.toList())) + assertThatThrownBy(() -> INSTANCE.resolveAndValidateFields(key, fields, options, ss).collect(toList())) .isInstanceOf(QueryException.class) .hasMessageContaining("Cannot derive Compact type for '" + QueryDataTypeFamily.OBJECT + "'"); } @@ -100,9 +97,9 @@ public void test_objectIsForbiddenForCompact(boolean key, String prefix) { public void test_resolveFields(boolean key, String prefix) { InternalSerializationService ss = createSerializationService(); Map options = - ImmutableMap.of((key ? OPTION_KEY_COMPACT_TYPE_NAME : OPTION_VALUE_COMPACT_TYPE_NAME), "testAll"); + Map.of(key ? OPTION_KEY_COMPACT_TYPE_NAME : OPTION_VALUE_COMPACT_TYPE_NAME, "testAll"); - List fields = asList( + List fields = List.of( field("string", QueryDataType.VARCHAR, prefix + ".string"), field("character", QueryDataType.VARCHAR_CHARACTER, prefix + ".character"), field("boolean", QueryDataType.BOOLEAN, prefix + ".boolean"), @@ -132,12 +129,11 @@ public void test_resolveFields(boolean key, String prefix) { public void test_typeNameIsRequiredToResolveFields(boolean key, String prefix) { InternalSerializationService ss = createSerializationService(); - Map options = Collections.emptyMap(); + Map options = emptyMap(); - // TODO: fix compact nested types support? assertThatThrownBy(() -> INSTANCE.resolveAndValidateFields(key, - singletonList(field("field", QueryDataType.INT, prefix + ".field")), options, ss)) - .hasMessageMatching("Unable to resolve table metadata\\. Missing '(key|value)CompactTypeName' option"); + List.of(field("field", QueryDataType.INT, prefix + ".field")), options, ss) + ).hasMessageMatching("(key|value)CompactTypeName is required to create Compact-based mapping"); } @Test @@ -149,11 +145,11 @@ public void when_duplicateExternalName_then_throws(boolean key, String prefix) { InternalSerializationService ss = createSerializationService(); Map options = - ImmutableMap.of((key ? OPTION_KEY_COMPACT_TYPE_NAME : OPTION_VALUE_COMPACT_TYPE_NAME), "testAll"); + Map.of(key ? OPTION_KEY_COMPACT_TYPE_NAME : OPTION_VALUE_COMPACT_TYPE_NAME, "testAll"); assertThatThrownBy(() -> MetadataPortableResolver.INSTANCE.resolveAndValidateFields( key, - asList( + List.of( field("field1", QueryDataType.INT, prefix + ".field"), field("field2", QueryDataType.VARCHAR, prefix + ".field") ), @@ -171,7 +167,7 @@ public void when_duplicateExternalName_then_throws(boolean key, String prefix) { public void test_resolveMetadata(boolean key, String prefix) { KvMetadata metadata = INSTANCE.resolveMetadata( key, - asList( + List.of( field("boolean", QueryDataType.BOOLEAN, prefix + ".boolean"), field("byte", QueryDataType.TINYINT, prefix + ".byte"), field("short", QueryDataType.SMALLINT, prefix + ".short"), @@ -186,7 +182,7 @@ public void test_resolveMetadata(boolean key, String prefix) { field("timestamp", QueryDataType.TIMESTAMP, prefix + ".timestamp"), field("timestampTz", QueryDataType.TIMESTAMP_WITH_TZ_OFFSET_DATE_TIME, prefix + ".timestampTz") ), - ImmutableMap.of((key ? OPTION_KEY_COMPACT_TYPE_NAME : OPTION_VALUE_COMPACT_TYPE_NAME), "test"), + Map.of(key ? OPTION_KEY_COMPACT_TYPE_NAME : OPTION_VALUE_COMPACT_TYPE_NAME, "test"), createSerializationService() ); @@ -230,9 +226,10 @@ public void test_resolveMetadata(boolean key, String prefix) { } private static InternalSerializationService createSerializationService() { - SerializationConfig serializationConfig = new SerializationConfig(); - return new DefaultSerializationServiceBuilder().setSchemaService(CompactTestUtil.createInMemorySchemaService()) - .setConfig(serializationConfig).build(); + return new DefaultSerializationServiceBuilder() + .setSchemaService(CompactTestUtil.createInMemorySchemaService()) + .setConfig(new SerializationConfig()) + .build(); } private static MappingField field(String name, QueryDataType type, String externalName) { diff --git a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/map/SqlJsonTest.java b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/map/SqlJsonTest.java index 3e0853bbb3bc..32301cf1f89a 100644 --- a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/map/SqlJsonTest.java +++ b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/map/SqlJsonTest.java @@ -19,7 +19,6 @@ import com.hazelcast.core.HazelcastJsonValue; import com.hazelcast.jet.sql.SqlTestSupport; import com.hazelcast.jet.sql.impl.connector.test.TestAllTypesSqlConnector; -import com.hazelcast.sql.HazelcastSqlException; import com.hazelcast.sql.SqlService; import org.junit.BeforeClass; import org.junit.Test; @@ -29,106 +28,94 @@ import java.time.LocalDateTime; import java.time.LocalTime; import java.time.OffsetDateTime; +import java.util.List; +import java.util.Map; -import static com.hazelcast.jet.core.TestUtil.createMap; +import static com.hazelcast.jet.pipeline.file.JsonFileFormat.FORMAT_JSON; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.JAVA_FORMAT; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.JSON_FLAT_FORMAT; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_KEY_CLASS; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_KEY_FORMAT; import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_FORMAT; -import static java.lang.String.format; import static java.time.ZoneOffset.UTC; -import static java.util.Arrays.asList; -import static java.util.Collections.singletonList; import static org.assertj.core.api.Assertions.assertThatThrownBy; public class SqlJsonTest extends SqlTestSupport { - private static SqlService sqlService; @BeforeClass - public static void setUpClass() { + public static void setup() { initialize(1, null); sqlService = instance().getSql(); } + private static SqlMapping jsonMapping(String name) { + return new SqlMapping(name, IMapSqlConnector.class) + .options(OPTION_KEY_FORMAT, JSON_FLAT_FORMAT, + OPTION_VALUE_FORMAT, JSON_FLAT_FORMAT); + } + @Test public void test_nulls() { String name = randomName(); - sqlService.execute("CREATE MAPPING " + name + " (" - + "id INT EXTERNAL NAME \"__key.id\"" - + ", name VARCHAR EXTERNAL NAME \"this.name\"" - + ") TYPE " + IMapSqlConnector.TYPE_NAME + ' ' - + "OPTIONS (" - + '\'' + OPTION_KEY_FORMAT + "'='" + JSON_FLAT_FORMAT + '\'' - + ", '" + OPTION_VALUE_FORMAT + "'='" + JSON_FLAT_FORMAT + '\'' - + ")" - ); + jsonMapping(name) + .fields("id INT EXTERNAL NAME \"__key.id\"", + "name VARCHAR EXTERNAL NAME \"this.name\"") + .create(); assertMapEventually( name, "SINK INTO " + name + " VALUES (null, null)", - createMap(new HazelcastJsonValue("{\"id\":null}"), new HazelcastJsonValue("{\"name\":null}")) + Map.of(new HazelcastJsonValue("{\"id\":null}"), new HazelcastJsonValue("{\"name\":null}")) ); assertRowsAnyOrder( "SELECT * FROM " + name, - singletonList(new Row(null, null)) + List.of(new Row(null, null)) ); } @Test public void test_fieldsMapping() { String name = randomName(); - sqlService.execute("CREATE MAPPING " + name + " (" - + "key_name VARCHAR EXTERNAL NAME \"__key.name\"" - + ", value_name VARCHAR EXTERNAL NAME \"this.name\"" - + ") TYPE " + IMapSqlConnector.TYPE_NAME + ' ' - + "OPTIONS (" - + '\'' + OPTION_KEY_FORMAT + "'='" + JSON_FLAT_FORMAT + '\'' - + ", '" + OPTION_VALUE_FORMAT + "'='" + JSON_FLAT_FORMAT + '\'' - + ")" - ); + jsonMapping(name) + .fields("key_name VARCHAR EXTERNAL NAME \"__key.name\"", + "value_name VARCHAR EXTERNAL NAME \"this.name\"") + .create(); assertMapEventually( name, "SINK INTO " + name + " (value_name, key_name) VALUES ('Bob', 'Alice')", - createMap(new HazelcastJsonValue("{\"name\":\"Alice\"}"), new HazelcastJsonValue("{\"name\":\"Bob\"}")) + Map.of(new HazelcastJsonValue("{\"name\":\"Alice\"}"), new HazelcastJsonValue("{\"name\":\"Bob\"}")) ); assertRowsAnyOrder( "SELECT * FROM " + name, - singletonList(new Row("Alice", "Bob")) + List.of(new Row("Alice", "Bob")) ); } @Test public void test_schemaEvolution() { String name = randomName(); - sqlService.execute("CREATE MAPPING " + name + " (" - + "__key INT" - + ", name VARCHAR" - + ") TYPE " + IMapSqlConnector.TYPE_NAME + ' ' - + "OPTIONS (" - + '\'' + OPTION_KEY_FORMAT + "'='" + JAVA_FORMAT + '\'' - + ", '" + OPTION_KEY_CLASS + "'='" + Integer.class.getName() + '\'' - + ", '" + OPTION_VALUE_FORMAT + "'='" + JSON_FLAT_FORMAT + '\'' - + ")" - ); + new SqlMapping(name, IMapSqlConnector.class) + .fields("__key INT", + "name VARCHAR") + .options(OPTION_KEY_FORMAT, JAVA_FORMAT, + OPTION_KEY_CLASS, Integer.class.getName(), + OPTION_VALUE_FORMAT, JSON_FLAT_FORMAT) + .create(); // insert initial record sqlService.execute("SINK INTO " + name + " VALUES (13, 'Alice')"); // alter schema - sqlService.execute("CREATE OR REPLACE MAPPING " + name + " (" - + "__key INT" - + ", name VARCHAR" - + ", ssn BIGINT" - + ") TYPE " + IMapSqlConnector.TYPE_NAME + ' ' - + "OPTIONS (" - + '\'' + OPTION_KEY_FORMAT + "'='" + JAVA_FORMAT + '\'' - + ", '" + OPTION_KEY_CLASS + "'='" + Integer.class.getName() + '\'' - + ", '" + OPTION_VALUE_FORMAT + "'='" + JSON_FLAT_FORMAT + '\'' - + ")" - ); + new SqlMapping(name, IMapSqlConnector.class) + .fields("__key INT", + "name VARCHAR", + "ssn BIGINT") + .options(OPTION_KEY_FORMAT, JAVA_FORMAT, + OPTION_KEY_CLASS, Integer.class.getName(), + OPTION_VALUE_FORMAT, JSON_FLAT_FORMAT) + .createOrReplace(); // insert record against new schema sqlService.execute("SINK INTO " + name + " VALUES (69, 'Bob', 123456789)"); @@ -136,7 +123,7 @@ public void test_schemaEvolution() { // assert both - initial & evolved - records are correctly read assertRowsAnyOrder( "SELECT * FROM " + name, - asList( + List.of( new Row(13, "Alice", null), new Row(69, "Bob", 123456789L) ) @@ -149,35 +136,30 @@ public void test_allTypes() { TestAllTypesSqlConnector.create(sqlService, from); String to = randomName(); - sqlService.execute("CREATE MAPPING " + to + " (" - + "id VARCHAR EXTERNAL NAME \"__key.id\"" - + ", string VARCHAR" - + ", \"boolean\" BOOLEAN" - + ", byte TINYINT" - + ", short SMALLINT" - + ", \"int\" INT" - + ", long BIGINT" - + ", \"float\" REAL" - + ", \"double\" DOUBLE" - + ", \"decimal\" DECIMAL" - + ", \"time\" TIME" - + ", \"date\" DATE" - + ", \"timestamp\" TIMESTAMP" - + ", timestampTz TIMESTAMP WITH TIME ZONE" - + ", object OBJECT" - + ", map OBJECT" - + ") TYPE " + IMapSqlConnector.TYPE_NAME + ' ' - + "OPTIONS (" - + '\'' + OPTION_KEY_FORMAT + "'='" + JSON_FLAT_FORMAT + '\'' - + ", '" + OPTION_VALUE_FORMAT + "'='" + JSON_FLAT_FORMAT + '\'' - + ")" - ); + jsonMapping(to) + .fields("id VARCHAR EXTERNAL NAME \"__key.id\"", + "string VARCHAR", + "\"boolean\" BOOLEAN", + "byte TINYINT", + "short SMALLINT", + "\"int\" INT", + "long BIGINT", + "\"float\" REAL", + "\"double\" DOUBLE", + "\"decimal\" DECIMAL", + "\"time\" TIME", + "\"date\" DATE", + "\"timestamp\" TIMESTAMP", + "timestampTz TIMESTAMP WITH TIME ZONE", + "object OBJECT", + "map OBJECT") + .create(); sqlService.execute("SINK INTO " + to + " SELECT '1', f.* FROM " + from + " f"); assertRowsAnyOrder( "SELECT * FROM " + to, - singletonList(new Row( + List.of(new Row( "1", "string", true, @@ -211,60 +193,43 @@ public void when_explicitTopLevelField_then_fail_this() { private void when_explicitTopLevelField_then_fail(String field, String otherField) { String name = randomName(); assertThatThrownBy(() -> - sqlService.execute("CREATE MAPPING " + name + " (" - + field + " VARCHAR" - + ", f VARCHAR EXTERNAL NAME \"" + otherField + ".f\"" - + ") TYPE " + IMapSqlConnector.TYPE_NAME + ' ' - + "OPTIONS (" - + '\'' + OPTION_KEY_FORMAT + "'='" + JSON_FLAT_FORMAT + '\'' - + ", '" + OPTION_VALUE_FORMAT + "'='" + JSON_FLAT_FORMAT + '\'' - + ")" - )) - .isInstanceOf(HazelcastSqlException.class) - .hasMessage("Cannot use the '" + field + "' field with JSON serialization"); + jsonMapping(name) + .fields(field + " VARCHAR", + "f VARCHAR EXTERNAL NAME \"" + otherField + ".f\"") + .create()) + .hasMessage("Cannot use '" + field + "' field with JSON serialization"); } @Test public void test_writingToTopLevel() { String mapName = randomName(); - sqlService.execute("CREATE MAPPING " + mapName + "(" - + "id INT EXTERNAL NAME \"__key.id\"" - + ", name VARCHAR" - + ") TYPE " + IMapSqlConnector.TYPE_NAME + ' ' - + "OPTIONS (" - + '\'' + OPTION_KEY_FORMAT + "'='" + JSON_FLAT_FORMAT + '\'' - + ", '" + OPTION_VALUE_FORMAT + "'='" + JSON_FLAT_FORMAT + '\'' - + ")" - ); + jsonMapping(mapName) + .fields("id INT EXTERNAL NAME \"__key.id\"", + "name VARCHAR") + .create(); assertThatThrownBy(() -> - sqlService.execute("SINK INTO " + mapName + "(__key, name) VALUES('{\"id\":1}', null)")) - .isInstanceOf(HazelcastSqlException.class) + sqlService.execute("SINK INTO " + mapName + "(__key, name) VALUES ('{\"id\":1}', null)")) .hasMessageContaining("Writing to top-level fields of type OBJECT not supported"); assertThatThrownBy(() -> - sqlService.execute("SINK INTO " + mapName + "(id, this) VALUES(1, '{\"name\":\"foo\"}')")) - .isInstanceOf(HazelcastSqlException.class) + sqlService.execute("SINK INTO " + mapName + "(id, this) VALUES (1, '{\"name\":\"foo\"}')")) .hasMessageContaining("Writing to top-level fields of type OBJECT not supported"); } @Test public void test_topLevelFieldExtraction() { String name = randomName(); - sqlService.execute("CREATE MAPPING " + name + " (" - + "id INT EXTERNAL NAME \"__key.id\"" - + ", name VARCHAR EXTERNAL NAME \"this.name\"" - + ") TYPE " + IMapSqlConnector.TYPE_NAME + ' ' - + "OPTIONS (" - + '\'' + OPTION_KEY_FORMAT + "'='" + JSON_FLAT_FORMAT + '\'' - + ", '" + OPTION_VALUE_FORMAT + "'='" + JSON_FLAT_FORMAT + '\'' - + ")" - ); + jsonMapping(name) + .fields("id INT EXTERNAL NAME \"__key.id\"", + "name VARCHAR EXTERNAL NAME \"this.name\"") + .create(); + sqlService.execute("SINK INTO " + name + " VALUES (1, 'Alice')"); assertRowsAnyOrder( "SELECT __key, this FROM " + name, - singletonList(new Row( + List.of(new Row( new HazelcastJsonValue("{\"id\":1}"), new HazelcastJsonValue("{\"name\":\"Alice\"}") )) @@ -274,18 +239,16 @@ public void test_topLevelFieldExtraction() { @Test public void test_jsonType() { String name = randomName(); - String createSql = format("CREATE MAPPING %s TYPE %s ", name, IMapSqlConnector.TYPE_NAME) - + "OPTIONS ( " - + format("'%s' = 'json'", OPTION_KEY_FORMAT) - + format(", '%s' = 'json'", OPTION_VALUE_FORMAT) - + ")"; - - sqlService.execute(createSql); + new SqlMapping(name, IMapSqlConnector.class) + .options(OPTION_KEY_FORMAT, FORMAT_JSON, + OPTION_VALUE_FORMAT, FORMAT_JSON) + .create(); sqlService.execute("SINK INTO " + name + " VALUES (CAST('[1,2,3]' AS JSON), CAST('[4,5,6]' AS JSON))"); + assertRowsAnyOrder( "SELECT __key, this FROM " + name, - singletonList(new Row( + List.of(new Row( new HazelcastJsonValue("[1,2,3]"), new HazelcastJsonValue("[4,5,6]") )) diff --git a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/map/SqlPojoTest.java b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/map/SqlPojoTest.java index 778ff33a285d..42c541d3613d 100644 --- a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/map/SqlPojoTest.java +++ b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/connector/map/SqlPojoTest.java @@ -24,7 +24,6 @@ import com.hazelcast.jet.sql.impl.connector.map.model.PersonId; import com.hazelcast.jet.sql.impl.connector.test.TestAllTypesSqlConnector; import com.hazelcast.map.IMap; -import com.hazelcast.sql.HazelcastSqlException; import com.hazelcast.sql.SqlService; import com.hazelcast.sql.impl.schema.Mapping; import org.junit.BeforeClass; @@ -40,6 +39,7 @@ import java.util.ArrayList; import java.util.Date; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Objects; @@ -53,26 +53,31 @@ import static java.time.Instant.ofEpochMilli; import static java.time.ZoneId.systemDefault; import static java.time.ZoneOffset.UTC; -import static java.util.Arrays.asList; -import static java.util.Collections.singletonList; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.Assert.assertEquals; public class SqlPojoTest extends SqlTestSupport { - private static SqlService sqlService; @BeforeClass - public static void setUpClass() { + public static void setup() { initialize(1, null); sqlService = instance().getSql(); } + private static SqlMapping javaMapping(String name, Class keyClass, Class valueClass) { + return new SqlMapping(name, IMapSqlConnector.class) + .options(OPTION_KEY_FORMAT, JAVA_FORMAT, + OPTION_KEY_CLASS, keyClass.getName(), + OPTION_VALUE_FORMAT, JAVA_FORMAT, + OPTION_VALUE_CLASS, valueClass.getName()); + } + @Test public void test_nulls() { String name = randomName(); - createMapping(name, PersonId.class, Person.class); + javaMapping(name, PersonId.class, Person.class).create(); assertMapEventually( name, @@ -81,14 +86,14 @@ public void test_nulls() { ); assertRowsAnyOrder( "SELECT * FROM " + name, - singletonList(new Row(1, null)) + List.of(new Row(1, null)) ); } @Test public void when_nullIntoPrimitive_then_fails() { String name = randomName(); - createMapping(name, PersonId.class, Person.class); + javaMapping(name, PersonId.class, Person.class).create(); assertThatThrownBy(() -> sqlService.execute("SINK INTO " + name + " VALUES (null, 'Alice')")) .hasMessageContaining("Cannot pass NULL to a method with a primitive argument"); @@ -112,10 +117,8 @@ public void when_wrongClass_then_canNotQueryTable() { createBrokenMapping(badName); assertThatThrownBy(() -> sqlService.execute("SELECT * FROM " + badName)) - .isInstanceOf(HazelcastSqlException.class) - .hasMessage("Mapping '%s' is invalid: " + - "com.hazelcast.sql.impl.QueryException: Unable to load class: 'com.hazelcast.NoSuchClass'", - badName); + .hasMessage("Mapping '%s' is invalid: com.hazelcast.sql.impl.QueryException: " + + "Unable to load class 'com.hazelcast.NoSuchClass'", badName); } @Test @@ -123,7 +126,7 @@ public void when_wrongClass_then_canQueryOtherTables() { String badName = randomName(); String goodName = randomName(); createBrokenMapping(badName); - createMapping(goodName, PersonId.class, Person.class); + javaMapping(goodName, PersonId.class, Person.class).create(); assertThat(sqlService.execute("SELECT * FROM " + goodName)).hasSize(0); } @@ -132,7 +135,7 @@ public void when_wrongClass_then_canQueryOtherTables() { * Simulates creation of a mapping for class that was later unloaded. */ private static void createBrokenMapping(String name) { - createMapping(name, PersonId.class, Person.class); + javaMapping(name, PersonId.class, Person.class).create(); IMap catalog = instance().getMap(SQL_CATALOG_MAP_NAME); Mapping m = catalog.get(name); @@ -151,35 +154,27 @@ private static void createBrokenMapping(String name) { @Test public void test_fieldsShadowing() { String name = randomName(); - createMapping(name, PersonId.class, Person.class); + javaMapping(name, PersonId.class, Person.class).create(); assertMapEventually( name, "SINK INTO " + name + " (id, name) VALUES (1, 'Alice')", createMap(new PersonId(1), new Person(null, "Alice")) ); - assertRowsAnyOrder( "SELECT * FROM " + name, - singletonList(new Row(1, "Alice")) + List.of(new Row(1, "Alice")) ); } @Test public void test_fieldsMapping() { String name = randomName(); - sqlService.execute("CREATE MAPPING " + name + " (" - + "key_id INT EXTERNAL NAME \"__key.id\"" - + ", value_id INT EXTERNAL NAME \"this.id\"" - + ", name VARCHAR" - + ") TYPE " + IMapSqlConnector.TYPE_NAME + ' ' - + "OPTIONS (" - + '\'' + OPTION_KEY_FORMAT + "'='" + JAVA_FORMAT + '\'' - + ", '" + OPTION_KEY_CLASS + "'='" + PersonId.class.getName() + '\'' - + ", '" + OPTION_VALUE_FORMAT + "'='" + JAVA_FORMAT + '\'' - + ", '" + OPTION_VALUE_CLASS + "'='" + Person.class.getName() + '\'' - + ")" - ); + javaMapping(name, PersonId.class, Person.class) + .fields("key_id INT EXTERNAL NAME \"__key.id\"", + "value_id INT EXTERNAL NAME \"this.id\"", + "name VARCHAR") + .create(); assertMapEventually( name, @@ -188,28 +183,20 @@ public void test_fieldsMapping() { ); assertRowsAnyOrder( "SELECT key_id, value_id, name FROM " + name, - singletonList(new Row(1, 2, "Alice")) + List.of(new Row(1, 2, "Alice")) ); } @Test public void test_schemaEvolution() { String name = randomName(); - createMapping(name, PersonId.class, Person.class); + javaMapping(name, PersonId.class, Person.class).create(); // insert initial record sqlService.execute("SINK INTO " + name + " VALUES (1, 'Alice')"); // alter schema - sqlService.execute("CREATE OR REPLACE MAPPING " + name + ' ' - + "TYPE " + IMapSqlConnector.TYPE_NAME + ' ' - + "OPTIONS (" - + '\'' + OPTION_KEY_FORMAT + "'='" + JAVA_FORMAT + '\'' - + ", '" + OPTION_KEY_CLASS + "'='" + PersonId.class.getName() + '\'' - + ", '" + OPTION_VALUE_FORMAT + "'='" + JAVA_FORMAT + '\'' - + ", '" + OPTION_VALUE_CLASS + "'='" + InsuredPerson.class.getName() + '\'' - + ")" - ); + javaMapping(name, PersonId.class, InsuredPerson.class).createOrReplace(); // insert record against new schema sqlService.execute("SINK INTO " + name + " (id, name, ssn) VALUES (2, 'Bob', 123456789)"); @@ -217,7 +204,7 @@ public void test_schemaEvolution() { // assert both - initial & evolved - records are correctly read assertRowsAnyOrder( "SELECT id, name, ssn FROM " + name, - asList( + List.of( new Row(1, "Alice", null), new Row(2, "Bob", 123456789L) ) @@ -231,19 +218,11 @@ public void test_fieldsExtensions() { Map map = instance().getMap(name); map.put(new PersonId(1), new InsuredPerson(1, "Alice", 123456789L)); - sqlService.execute("CREATE MAPPING " + name + " (" - + "id INT EXTERNAL NAME \"__key.id\"," - + "name VARCHAR," - // the "ssn" field isn't defined in the `Person` class, but in the subclass - + "ssn BIGINT" - + ") TYPE " + IMapSqlConnector.TYPE_NAME + ' ' - + "OPTIONS (" - + '\'' + OPTION_KEY_FORMAT + "'='" + JAVA_FORMAT + '\'' - + ", '" + OPTION_KEY_CLASS + "'='" + PersonId.class.getName() + '\'' - + ", '" + OPTION_VALUE_FORMAT + "'='" + JAVA_FORMAT + '\'' - + ", '" + OPTION_VALUE_CLASS + "'='" + Person.class.getName() + '\'' - + ")" - ); + javaMapping(name, PersonId.class, Person.class) + .fields("id INT EXTERNAL NAME \"__key.id\"", + "name VARCHAR", + "ssn BIGINT" /* defined in the subclass */) + .create(); assertMapEventually( name, @@ -255,7 +234,7 @@ public void test_fieldsExtensions() { ); assertRowsAnyOrder( "SELECT * FROM " + name, - asList( + List.of( new Row(1, "Alice", 123456789L), new Row(2, "Bob", null) ) @@ -268,7 +247,7 @@ public void test_allTypes() { TestAllTypesSqlConnector.create(sqlService, from); String to = randomName(); - createMapping(to, BigInteger.class, AllTypesValue.class); + javaMapping(to, BigInteger.class, AllTypesValue.class).create(); assertMapEventually( to, @@ -346,7 +325,7 @@ public void test_allTypes() { + ", map" + ", object " + "FROM " + to, - singletonList(new Row( + List.of(new Row( BigDecimal.valueOf(1), "string", "s", @@ -375,14 +354,9 @@ public void test_allTypes() { @Test public void when_fieldWithInitialValueUnmapped_then_initialValuePreserved() { String mapName = randomName(); - sqlService.execute("CREATE MAPPING " + mapName + "(__key INT)" - + " TYPE " + IMapSqlConnector.TYPE_NAME + "\n" - + "OPTIONS (\n" - + '\'' + OPTION_KEY_FORMAT + "'='" + JAVA_FORMAT + "',\n" - + '\'' + OPTION_KEY_CLASS + "'='" + Integer.class.getName() + "',\n" - + '\'' + OPTION_VALUE_FORMAT + "'='" + JAVA_FORMAT + "',\n" - + '\'' + OPTION_VALUE_CLASS + "'='" + ClassInitialValue.class.getName() + "'\n" - + ")"); + javaMapping(mapName, Integer.class, ClassInitialValue.class) + .fields("__key INT") + .create(); sqlService.execute("SINK INTO " + mapName + "(__key) VALUES (1)"); ClassInitialValue val = instance().getMap(mapName).get(1); @@ -397,17 +371,23 @@ public void when_fieldWithInitialValueNotUsed_then_valueOverwritten() { // are always overwritten: if they're not present, we'll write null. We don't support DEFAULT values yet, but // it behaves as if the DEFAULT was null. String mapName = randomName(); - createMapping(mapName, Integer.class, ClassInitialValue.class); + javaMapping(mapName, Integer.class, ClassInitialValue.class).create(); sqlService.execute("SINK INTO " + mapName + "(__key) VALUES (1)"); - assertRowsAnyOrder("SELECT * FROM " + mapName, singletonList(new Row(1, null))); + assertRowsAnyOrder( + "SELECT * FROM " + mapName, + List.of(new Row(1, null)) + ); } @Test public void when_fieldWithInitialValueAssignedNull_then_isNull() { String mapName = randomName(); - createMapping(mapName, Integer.class, ClassInitialValue.class); + javaMapping(mapName, Integer.class, ClassInitialValue.class).create(); sqlService.execute("SINK INTO " + mapName + "(__key, field) VALUES (1, null)"); - assertRowsAnyOrder("SELECT * FROM " + mapName, singletonList(new Row(1, null))); + assertRowsAnyOrder( + "SELECT * FROM " + mapName, + List.of(new Row(1, null)) + ); } @Test @@ -422,46 +402,37 @@ public void test_writingToTopLevelWhileNestedFieldMapped_implicit() { private void test_writingToTopLevel(boolean explicit) { String mapName = randomName(); - sqlService.execute("CREATE MAPPING " + mapName + "(" - + "__key INT" - + (explicit ? ", this OBJECT" : "") - + ", name VARCHAR" - + ") TYPE " + IMapSqlConnector.TYPE_NAME + "\n" - + "OPTIONS (\n" - + '\'' + OPTION_KEY_FORMAT + "'='" + JAVA_FORMAT + "'\n" - + ", '" + OPTION_KEY_CLASS + "'='" + Integer.class.getName() + "'\n" - + ", '" + OPTION_VALUE_FORMAT + "'='" + JAVA_FORMAT + "'\n" - + ", '" + OPTION_VALUE_CLASS + "'='" + Person.class.getName() + "'\n" - + ")" - ); + javaMapping(mapName, Integer.class, Person.class) + .fields("__key INT") + .fieldsIf(explicit, "this OBJECT") + .fields("name VARCHAR") + .create(); if (explicit) { - assertThatThrownBy(() -> - sqlService.execute("SINK INTO " + mapName + " VALUES(1, null, 'foo')")) - .isInstanceOf(HazelcastSqlException.class) + assertThatThrownBy(() -> sqlService.execute("SINK INTO " + mapName + " VALUES(1, null, 'foo')")) .hasMessageContaining("Writing to top-level fields of type OBJECT not supported"); } - assertThatThrownBy(() -> - sqlService.execute("SINK INTO " + mapName + "(__key, this) VALUES(1, null)")) - .isInstanceOf(HazelcastSqlException.class) + assertThatThrownBy(() -> sqlService.execute("SINK INTO " + mapName + "(__key, this) VALUES(1, null)")) .hasMessageContaining("Writing to top-level fields of type OBJECT not supported"); sqlService.execute("SINK INTO " + mapName + (explicit ? "(__key, name)" : "") + " VALUES (1, 'foo')"); - assertRowsAnyOrder("SELECT __key, this, name FROM " + mapName, - singletonList(new Row(1, new Person(null, "foo"), "foo"))); + assertRowsAnyOrder( + "SELECT __key, this, name FROM " + mapName, + List.of(new Row(1, new Person(null, "foo"), "foo")) + ); } @Test public void test_topLevelFieldExtraction() { String name = randomName(); - createMapping(name, PersonId.class, Person.class); + javaMapping(name, PersonId.class, Person.class).create(); sqlService.execute("SINK INTO " + name + " (id, name) VALUES (1, 'Alice')"); assertRowsAnyOrder( "SELECT __key, this FROM " + name, - singletonList(new Row(new PersonId(1), new Person(null, "Alice"))) + List.of(new Row(new PersonId(1), new Person(null, "Alice"))) ); } @@ -469,43 +440,43 @@ public void test_topLevelFieldExtraction() { public void test_nestedField() { String mapName = randomName(); assertThatThrownBy(() -> - sqlService.execute("CREATE MAPPING " + mapName + "(" - + "__key INT," - + "petName VARCHAR," - + "\"owner.name\" VARCHAR) " - + "TYPE " + IMapSqlConnector.TYPE_NAME - )) - .isInstanceOf(HazelcastSqlException.class) + new SqlMapping(mapName, IMapSqlConnector.class) + .fields("__key INT", + "petName VARCHAR", + "\"owner.name\" VARCHAR") + .create()) .hasMessageContaining("Invalid external name: this.owner.name"); } @Test public void when_noFieldsResolved_then_wholeValueMapped() { String name = randomName(); - createMapping(name, Object.class, Object.class); + javaMapping(name, Object.class, Object.class).create(); Person key = new Person(1, "foo"); Person value = new Person(2, "bar"); instance().getMap(name).put(key, value); - assertRowsAnyOrder("SELECT __key, this FROM " + name, - singletonList(new Row(key, value))); + assertRowsAnyOrder( + "SELECT __key, this FROM " + name, + List.of(new Row(key, value)) + ); } @Test public void when_keyHasKeyField_then_fieldIsSkipped() { String name = randomName(); - createMapping(name, ClassWithKey.class, Integer.class); + javaMapping(name, ClassWithKey.class, Integer.class).create(); instance().getMap(name).put(new ClassWithKey(), 0); assertRowsAnyOrder( "SELECT * FROM " + name, - singletonList(new Row(0)) + List.of(new Row(0)) ); assertRowsAnyOrder( "SELECT __key, this FROM " + name, - singletonList(new Row(new ClassWithKey(), 0)) + List.of(new Row(new ClassWithKey(), 0)) ); } @@ -514,21 +485,20 @@ public void test_classWithMapField() { final String name = randomName(); final ClassWithMapField obj = new ClassWithMapField(100L, "k", "v"); - createMapping(name, Long.class, ClassWithMapField.class); + javaMapping(name, Long.class, ClassWithMapField.class).create(); instance().getSql().execute("SINK INTO " + name + " VALUES (?, ?, ?)", 1L, obj.id, obj.props); - assertRowsAnyOrder("SELECT * FROM " + name, singletonList( - new Row(1L, obj.id, obj.props) - )); + assertRowsAnyOrder( + "SELECT * FROM " + name, + List.of(new Row(1L, obj.id, obj.props)) + ); } public static class ClassInitialValue implements Serializable { - public Integer field = 42; } public static class ClassWithKey implements Serializable { - public int __key; @Override @@ -553,10 +523,10 @@ public static class ClassWithMapField implements Serializable { private Long id; private Map props; - public ClassWithMapField() { - } + @SuppressWarnings("unused") + public ClassWithMapField() { } - public ClassWithMapField(final Long id, String... values) { + public ClassWithMapField(Long id, String... values) { this.id = id; this.props = new HashMap<>(); for (int i = 0; i < values.length; i += 2) { @@ -568,7 +538,7 @@ public Long getId() { return id; } - public void setId(final Long id) { + public void setId(Long id) { this.id = id; } @@ -576,7 +546,7 @@ public Map getProps() { return props; } - public void setProps(final Map props) { + public void setProps(Map props) { this.props = props; } } diff --git a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/schema/GetDdlTest.java b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/schema/GetDdlTest.java index 4c1e2f013ecb..3d5eaa816fe9 100644 --- a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/schema/GetDdlTest.java +++ b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/schema/GetDdlTest.java @@ -16,8 +16,6 @@ package com.hazelcast.jet.sql.impl.schema; -import com.google.common.collect.ImmutableList; -import com.hazelcast.config.Config; import com.hazelcast.jet.sql.SqlTestSupport; import com.hazelcast.map.IMap; import com.hazelcast.sql.SqlResult; @@ -25,8 +23,8 @@ import org.junit.BeforeClass; import org.junit.Test; -import java.util.Arrays; import java.util.Collections; +import java.util.List; import static com.hazelcast.spi.properties.ClusterProperty.SQL_CUSTOM_TYPES_ENABLED; import static java.util.Collections.emptyList; @@ -38,17 +36,14 @@ public class GetDdlTest extends SqlTestSupport { @BeforeClass public static void beforeClass() throws Exception { - Config config = smallInstanceConfig() - .setProperty(SQL_CUSTOM_TYPES_ENABLED.getName(), "true"); - - initialize(1, config); + initialize(1, smallInstanceConfig().setProperty(SQL_CUSTOM_TYPES_ENABLED.getName(), "true")); } @Test public void when_queryMappingFromRelationNamespace_then_success() { createMapping("a", int.class, int.class); - assertRowsAnyOrder("SELECT GET_DDL('relation', 'a')", ImmutableList.of( + assertRowsAnyOrder("SELECT GET_DDL('relation', 'a')", List.of( new Row("CREATE OR REPLACE EXTERNAL MAPPING \"hazelcast\".\"public\".\"a\" EXTERNAL NAME \"a\" (" + LE + " \"__key\" INTEGER EXTERNAL NAME \"__key\"," + LE + " \"this\" INTEGER EXTERNAL NAME \"this\"" + LE + @@ -69,7 +64,7 @@ public void when_queryViewFromRelationNamespace_then_success() { createMapping("a", int.class, int.class); instance().getSql().execute("CREATE VIEW v AS SELECT * FROM a"); - assertRowsAnyOrder("SELECT GET_DDL('relation', 'v')", ImmutableList.of( + assertRowsAnyOrder("SELECT GET_DDL('relation', 'v')", List.of( new Row("CREATE OR REPLACE VIEW \"hazelcast\".\"public\".\"v\" AS" + LE + "SELECT \"a\".\"__key\", \"a\".\"this\"" + LE + "FROM \"hazelcast\".\"public\".\"a\" AS \"a\"")) @@ -83,14 +78,19 @@ public void when_queryTypeFromRelationNamespace_then_success() { " \"b\" INTEGER" + LE + ")" + LE + "OPTIONS (" + LE + - " 'format'='portable'," + LE + - " 'portableFactoryId'='1'," + LE + - " 'portableClassId'='3'," + LE + - " 'portableClassVersion'='0'" + LE + + " 'typeClass'='foo'" + LE + ")"; instance().getSql().execute(createTypeQuery); - assertRowsAnyOrder("SELECT GET_DDL('relation', 't')", ImmutableList.of(new Row(createTypeQuery))); + assertRowsAnyOrder("SELECT GET_DDL('relation', 't')", List.of(new Row(createTypeQuery))); + } + + @Test + public void when_queryTypeFromRelationNamespace_withoutFieldsAndOptions_then_success() { + String createTypeQuery = "CREATE OR REPLACE TYPE \"hazelcast\".\"public\".\"t\""; + + instance().getSql().execute(createTypeQuery); + assertRowsAnyOrder("SELECT GET_DDL('relation', 't')", List.of(new Row(createTypeQuery))); } @Test @@ -99,7 +99,7 @@ public void when_queryDataConnectionFromDataConnectionNamespace_then_success() { + "TYPE \"dummy\"" + LE + "SHARED"; instance().getSql().execute(createDataConnectionQuery); - assertRowsAnyOrder("SELECT GET_DDL('dataconnection', 'dl')", ImmutableList.of(new Row(createDataConnectionQuery))); + assertRowsAnyOrder("SELECT GET_DDL('dataconnection', 'dl')", List.of(new Row(createDataConnectionQuery))); } @Test @@ -113,7 +113,7 @@ public void when_queryDataConnectionWithByKeyPlan_then_success() { + "TYPE \"dummy\"" + LE + "SHARED"; assertRowsAnyOrder("SELECT __key, GET_DDL('dataconnection', this) FROM a WHERE __key = 1", - ImmutableList.of(new Row(1, ddl)) + List.of(new Row(1, ddl)) ); } @@ -156,13 +156,13 @@ public void when_queryDdlWithAnotherOperator_then_success() { createMapping("a", int.class, int.class); assertRowsAnyOrder("SELECT SUBSTRING(GET_DDL('relation', 'a') FROM 1 FOR 6)", - ImmutableList.of(new Row("CREATE")) + List.of(new Row("CREATE")) ); assertRowsAnyOrder( "SELECT SUBSTRING(GET_DDL('relation', 'a') FROM 1 FOR 6) " + "|| SUBSTRING(GET_DDL('relation', 'a') FROM 1 FOR 3)", - ImmutableList.of(new Row("CREATECRE")) + List.of(new Row("CREATECRE")) ); } @@ -172,7 +172,7 @@ public void when_queryDdlWithOtherRels_then_success() { assertRowsAnyOrder("SELECT SUBSTRING(GET_DDL('relation', 'a') FROM 1 FOR 6)" + "UNION ALL SELECT SUBSTRING(GET_DDL('relation', 'a') FROM 1 FOR 6)", - Arrays.asList(new Row("CREATE"), new Row("CREATE"))); + List.of(new Row("CREATE"), new Row("CREATE"))); } @Test @@ -181,7 +181,7 @@ public void when_queryDdlWithInput_then_success() { instance().getMap("a").put(1, "a"); assertRowsAnyOrder("SELECT GET_DDL('relation', this) FROM a", - ImmutableList.of(new Row( + List.of(new Row( "CREATE OR REPLACE EXTERNAL MAPPING \"hazelcast\".\"public\".\"a\" EXTERNAL NAME \"a\" (" + LE + " \"__key\" INTEGER EXTERNAL NAME \"__key\"," + LE + " \"this\" VARCHAR EXTERNAL NAME \"this\"" + LE + diff --git a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/type/BasicNestedFieldsTest.java b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/type/BasicNestedFieldsTest.java index 49e4f3f126c1..092bdcfc62ce 100644 --- a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/type/BasicNestedFieldsTest.java +++ b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/type/BasicNestedFieldsTest.java @@ -18,10 +18,11 @@ import com.hazelcast.config.Config; import com.hazelcast.core.HazelcastInstance; +import com.hazelcast.internal.serialization.impl.compact.DeserializedGenericRecord; import com.hazelcast.jet.sql.SqlTestSupport; +import com.hazelcast.jet.sql.impl.connector.map.IMapSqlConnector; import com.hazelcast.jet.sql.impl.connector.map.model.AllTypesValue; import com.hazelcast.map.IMap; -import com.hazelcast.sql.HazelcastSqlException; import com.hazelcast.sql.SqlResult; import com.hazelcast.test.HazelcastParametrizedRunner; import com.hazelcast.test.HazelcastSerialParametersRunnerFactory; @@ -44,9 +45,15 @@ import java.util.Date; import java.util.Objects; +import static com.hazelcast.jet.sql.impl.connector.SqlConnector.JAVA_FORMAT; +import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_KEY_FORMAT; +import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_TYPE_COMPACT_TYPE_NAME; +import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_TYPE_JAVA_CLASS; +import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_CLASS; +import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_FORMAT; +import static com.hazelcast.jet.sql.impl.type.CompactNestedFieldsTest.createCompactMapping; import static com.hazelcast.spi.properties.ClusterProperty.SQL_CUSTOM_TYPES_ENABLED; import static com.hazelcast.sql.SqlColumnType.OBJECT; -import static java.lang.String.format; import static java.time.Instant.ofEpochMilli; import static java.time.ZoneId.systemDefault; import static java.time.ZoneOffset.UTC; @@ -77,16 +84,57 @@ private HazelcastInstance testInstance() { return useClient ? client() : instance(); } + static void createJavaMapping(HazelcastInstance instance, String name, Class valueClass, String... valueFields) { + new SqlMapping(name, IMapSqlConnector.class) + .fields("__key BIGINT") + .fields(valueFields) + // com.hazelcast.jet.sql.impl.connector.keyvalue.JavaClassNameResolver.CLASS_NAMES_BY_FORMAT + .options(OPTION_KEY_FORMAT, "bigint", + OPTION_VALUE_FORMAT, JAVA_FORMAT, + OPTION_VALUE_CLASS, valueClass.getName()) + .create(instance); + } + + private void createJavaMapping(String name, Class valueClass, String... valueFields) { + createJavaMapping(testInstance(), name, valueClass, valueFields); + } + + private void createType(String name, String... fields) { + new SqlType(name) + .fields(fields) + .create(testInstance()); + } + + private SqlResult execute(String sql, Object... args) { + return testInstance().getSql().execute(sql, args); + } + + private User initDefault() { + createType("UserType", "id BIGINT", "name VARCHAR", "organization OrganizationType"); + createType("OrganizationType", "id BIGINT", "name VARCHAR", "office OfficeType"); + createType("OfficeType", "id BIGINT", "name VARCHAR"); + + final IMap testMap = testInstance().getMap("test"); + createJavaMapping("test", User.class, "this UserType"); + + final Office office = new Office(3L, "office1"); + final Organization organization = new Organization(2L, "organization1", office); + final User user = new User(1L, "user1", organization); + testMap.put(1L, user); + + return user; + } + @Test public void test_simpleNestedColumnSelect() { initDefault(); - final String sql = "SELECT " + "test.this.name AS user_name, " + "test.this.organization.name AS org_name, " + "test.this.organization.office.name AS office_name, " + "test.this.organization.office.id AS office_id " + "FROM test"; + assertRowsAnyOrder(testInstance(), sql, rows(4, "user1", "organization1", "office1", 3L)); } @@ -95,8 +143,9 @@ public void test_complexProjections() { initDefault(); final String sql = "SELECT " + "ABS((this).id) * 2 AS C1, " - + "FLOOR(CAST(((this).organization).id AS REAL) * 5.0 / 2.0) AS c2" - + " FROM test"; + + "FLOOR(CAST(((this).organization).id AS REAL) * 5.0 / 2.0) AS c2 " + + "FROM test"; + assertRowsAnyOrder(testInstance(), sql, rows(2, 2L, 5.0f)); } @@ -110,7 +159,8 @@ public void test_wholeObjectSelect() { + "test.this.organization, " + "test.this.organization.office " + "FROM test"; - SqlResult res = testInstance().getSql().execute(sql); + + SqlResult res = execute(sql); assertEquals(OBJECT, res.getRowMetadata().getColumn(0).getType()); assertRowsAnyOrder(testInstance(), sql, rows(2, organization, office)); } @@ -133,12 +183,6 @@ public void test_objectComparison() { @Test public void test_fullInsert() { initDefault(); - execute("CREATE OR REPLACE MAPPING test (__key BIGINT, this UserType) TYPE IMap OPTIONS (" - + "'keyFormat'='bigint'," - + "'valueFormat'='java'," - + format("'valueJavaClass'='%s'", User.class.getName()) - + ")"); - final Office office = new Office(5L, "office2"); final Organization organization = new Organization(4L, "organization2", office); final User user = new User(2L, "user1", organization); @@ -146,24 +190,25 @@ public void test_fullInsert() { execute("INSERT INTO test (__key, this) VALUES (?, ?)", 2L, new User(2L, "user2", user.organization)); - assertRowsAnyOrder(testInstance(), "SELECT test.this.organization, test.this.organization.office FROM test WHERE __key = 2", + assertRowsAnyOrder(testInstance(), + "SELECT test.this.organization, test.this.organization.office FROM test WHERE __key = 2", rows(2, organization, office)); } @Test public void test_update() { final User oldUser = initDefault(); - final User newUser = new User(1L, "new-name", oldUser.organization); execute("UPDATE test SET this = ? WHERE __key = 1", newUser); - assertRowsAnyOrder(testInstance(), "SELECT test.this.id, test.this.name, test.this.organization FROM test WHERE __key = 1", + assertRowsAnyOrder(testInstance(), + "SELECT test.this.id, test.this.name, test.this.organization FROM test WHERE __key = 1", rows(3, 1L, "new-name", oldUser.organization)); } @Test public void test_selfRefType() { - createJavaType("SelfRefType", SelfRef.class, "id BIGINT", "other SelfRefType"); + createType("SelfRefType", "id BIGINT", "name VARCHAR", "other SelfRefType"); final SelfRef first = new SelfRef(1L, "first"); final SelfRef second = new SelfRef(2L, "second"); @@ -175,7 +220,7 @@ public void test_selfRefType() { third.other = fourth; fourth.other = first; - createJavaMapping("test", SelfRef.class, "this SelfRefType"); + createJavaMapping("test", SelfRef.class, "this SelfRefType"); testInstance().getMap("test").put(1L, first); assertRowsAnyOrder(testInstance(), "SELECT " @@ -196,9 +241,9 @@ public void test_selfRefType() { @Test public void test_circularlyRecurrentTypes() { - createJavaType("AType", A.class, "name VARCHAR", "b BType"); - createJavaType("BType", B.class, "name VARCHAR", "c CType"); - createJavaType("CType", C.class, "name VARCHAR", "a AType"); + createType("AType", "name VARCHAR", "b BType"); + createType("BType", "name VARCHAR", "c CType"); + createType("CType", "name VARCHAR", "a AType"); final A a = new A("a"); final B b = new B("b"); @@ -218,8 +263,8 @@ public void test_circularlyRecurrentTypes() { @Test public void test_deepInsert() { initDefault(); - execute("INSERT INTO test VALUES (2, " + - "(2, 'user2', (2, 'organization2', (2, 'office2'))))"); + execute("INSERT INTO test VALUES (2, (2, 'user2', (2, 'organization2', (2, 'office2'))))"); + assertRowsAnyOrder(testInstance(), "SELECT " + "test.this.name, " + "test.this.organization.name, " @@ -249,43 +294,39 @@ public void test_deepUpdate() { @Test public void test_mixedModeQuerying() { - createType("NestedType", NestedPOJO.class); + createType("NestedType"); createJavaMapping("test", RegularPOJO.class, "name VARCHAR", "child NestedType"); testInstance().getMap("test") .put(1L, new RegularPOJO("parentPojo", new NestedPOJO(1L, "childPojo"))); - assertRowsAnyOrder(testInstance(), "SELECT name, (child).name FROM test", rows(2, - "parentPojo", - "childPojo" - )); + assertRowsAnyOrder(testInstance(), "SELECT name, (child).name FROM test", + rows(2, "parentPojo", "childPojo")); - assertRowsAnyOrder(testInstance(), "SELECT child FROM test", rows(1, new NestedPOJO(1L, "childPojo"))); + assertRowsAnyOrder(testInstance(), "SELECT child FROM test", + rows(1, new NestedPOJO(1L, "childPojo"))); } @Test public void test_mixedModeAliasQuerying() { - createType("NestedType", NestedPOJO.class); - execute(format("CREATE MAPPING test (" - + "__key BIGINT," - + "parentName VARCHAR EXTERNAL NAME \"name\"," - + "childObj NestedType EXTERNAL NAME \"child\"" - + ")" - + "TYPE IMap " - + "OPTIONS ('keyFormat'='bigint', 'valueFormat'='java', 'valueJavaClass'='%s')", RegularPOJO.class.getName())); + createType("NestedType"); + createJavaMapping("test", RegularPOJO.class, + "parentName VARCHAR EXTERNAL NAME \"name\"", + "childObj NestedType EXTERNAL NAME \"child\""); testInstance().getMap("test") .put(1L, new RegularPOJO("parentPojo", new NestedPOJO(1L, "childPojo"))); - assertRowsAnyOrder(testInstance(), "SELECT parentName, (childObj).name FROM (SELECT * FROM test)", rows(2, - "parentPojo", "childPojo" - )); - assertRowsAnyOrder(testInstance(), "SELECT childObj FROM test", rows(1, new NestedPOJO(1L, "childPojo"))); + assertRowsAnyOrder(testInstance(), "SELECT parentName, (childObj).name FROM (SELECT * FROM test)", + rows(2, "parentPojo", "childPojo")); + + assertRowsAnyOrder(testInstance(), "SELECT childObj FROM test", + rows(1, new NestedPOJO(1L, "childPojo"))); } @Test public void test_mixedModeUpsert() { - createType("NestedType", NestedPOJO.class); + createType("NestedType"); createJavaMapping("test", RegularPOJO.class, "name VARCHAR", "child NestedType"); execute("INSERT INTO test (__key, name, child) " @@ -300,7 +341,7 @@ public void test_mixedModeUpsert() { @Test public void test_typeCoercionUpserts() { - createType("AllTypesValue", AllTypesValue.class); + createType("AllTypesValue"); createJavaMapping("test", AllTypesParent.class, "name VARCHAR", "child AllTypesValue"); final String allTypesValueRowLiteral = "(" @@ -327,8 +368,7 @@ public void test_typeCoercionUpserts() { + "'1970-01-01T00:00:00Z'" + ")"; - execute("INSERT INTO test (__key, name, child) VALUES (1, 'parent', " - + allTypesValueRowLiteral + ")"); + execute("INSERT INTO test (__key, name, child) VALUES (1, 'parent', " + allTypesValueRowLiteral + ")"); assertRowsAnyOrder(testInstance(), "SELECT " + "test.child.bigDecimal," @@ -383,9 +423,11 @@ public void test_typeCoercionUpserts() { @Test public void test_compoundAliases() { initDefault(); - assertRowsAnyOrder(testInstance(), "SELECT ((org).office).name FROM " + - "(SELECT (this).organization as org FROM (SELECT * FROM test))", + + assertRowsAnyOrder(testInstance(), + "SELECT ((org).office).name FROM (SELECT (this).organization as org FROM (SELECT * FROM test))", rows(1, "office1")); + assertRowsAnyOrder(testInstance(), "SELECT (((this).organization).office).name FROM (SELECT * FROM test)", rows(1, "office1")); } @@ -393,6 +435,7 @@ public void test_compoundAliases() { @Test public void test_newDotOperatorSyntax() { initDefault(); + assertRowsAnyOrder(testInstance(), "SELECT (((this).organization).office).name FROM test", rows(1, "office1")); } @@ -400,15 +443,12 @@ public void test_newDotOperatorSyntax() { @Test public void test_joins() { initDefault(); - execute("CREATE MAPPING test2 (__key BIGINT, this UserType) " - + "TYPE IMap OPTIONS (" - + "'keyFormat'='bigint', " - + "'valueFormat'='java', " - + "'valueJavaClass'='" + User.class.getName() + "')"); + createJavaMapping("test2", User.class, "this UserType"); execute("INSERT INTO test2 VALUES (1, (1, 'user2', (1, 'organization2', (1, 'office2'))))"); - assertRowsAnyOrder(testInstance(), "SELECT (((t1.this).organization).office).name, (((t2.this).organization).office).name " + assertRowsAnyOrder(testInstance(), + "SELECT (((t1.this).organization).office).name, (((t2.this).organization).office).name " + "FROM test AS t1 JOIN test2 AS t2 ON t1.__key = t2.__key", rows(2, "office1", "office2")); @@ -420,8 +460,10 @@ public void test_joins() { + "FROM (SELECT t2.this FROM test AS t1 JOIN test2 AS t2 ON t1.__key = t2.__key)", rows(1, "office2")); - assertRowsAnyOrder(testInstance(), "SELECT (((this1).organization).office).name, (((this2).organization).office).name " - + "FROM (SELECT t1.this as this1, t2.this AS this2 FROM test AS t1 JOIN test2 AS t2 ON t1.__key = t2.__key)", + assertRowsAnyOrder(testInstance(), + "SELECT (((this1).organization).office).name, (((this2).organization).office).name " + + "FROM (SELECT t1.this as this1, t2.this AS this2 " + + " FROM test AS t1 JOIN test2 AS t2 ON t1.__key = t2.__key)", rows(2, "office1", "office2")); } @@ -431,7 +473,8 @@ public void test_joinsOnNestedFields() { createJavaMapping("test2", User.class, "this UserType"); execute("INSERT INTO test2 VALUES (1, (1, 'user2', (1, 'organization2', (1, 'office2'))))"); - assertRowsAnyOrder(testInstance(), "SELECT t1.this.organization.office.name, t2.this.organization.office.name " + assertRowsAnyOrder(testInstance(), + "SELECT t1.this.organization.office.name, t2.this.organization.office.name " + "FROM test AS t1 JOIN test2 AS t2 " + "ON ABS(t1.this.id) = t2.this.id AND t1.this.id = t2.this.id", rows(2, "office1", "office2")); @@ -440,120 +483,54 @@ public void test_joinsOnNestedFields() { @Test public void test_missingType() { // we create UserType, that has OrganizationType field, but we don't create OrganizationType - testInstance().getSql().execute(format("CREATE TYPE UserType (id BIGINT, name VARCHAR, organization OrganizationType) " - + "OPTIONS ('format'='java', 'javaClass'='%s')", User.class.getName())); + createType("UserType", "id BIGINT", "name VARCHAR", "organization OrganizationType"); - assertThatThrownBy(() -> execute("CREATE MAPPING test (__key BIGINT, this UserType) " - + "TYPE IMap OPTIONS (" - + "'keyFormat'='bigint', " - + "'valueFormat'='java', " - + "'valueJavaClass'='" + User.class.getName() + "')")) - .hasMessage("Encountered type 'OrganizationType', which doesn't exist") - .isInstanceOf(HazelcastSqlException.class); + assertThatThrownBy(() -> createJavaMapping("test", User.class, "this UserType")) + .hasMessage("Encountered type 'OrganizationType', which doesn't exist"); } @Test public void test_nullValueInRow() { - instance().getSql().execute("CREATE TYPE Office (" - + "id BIGINT, " - + "name VARCHAR " - + ") OPTIONS ('format'='compact', 'compactTypeName'='OfficeCompactType')"); - - instance().getSql().execute("CREATE TYPE Organization (" - + "id BIGINT, " - + "name VARCHAR, " - + "office Office" - + ") OPTIONS ('format'='compact', 'compactTypeName'='OrganizationCompactType')"); - - instance().getSql().execute( - "CREATE MAPPING test (" - + "__key BIGINT," - + "organization Organization" - + ")" - + "TYPE IMap " - + "OPTIONS (" - + "'keyFormat'='bigint'," - + "'valueFormat'='compact'," - + "'valueCompactTypeName'='UserCompactType'" - + ")"); - - instance().getSql().execute("INSERT INTO test VALUES (1, (2, 'orgName', null))"); - assertRowsAnyOrder("SELECT (organization).office FROM test", rows(1, new Object[]{null})); - } + createType("Office", "id BIGINT", "name VARCHAR"); + createType("Organization", "id BIGINT", "name VARCHAR", "office Office"); - private User initDefault() { - testInstance().getSql().execute(format("CREATE TYPE UserType (id BIGINT, name VARCHAR, organization OrganizationType) " - + "OPTIONS ('format'='java', 'javaClass'='%s')", User.class.getName())); - testInstance().getSql().execute(format("CREATE TYPE OrganizationType (id BIGINT, name VARCHAR, office OfficeType) " - + "OPTIONS ('format'='java', 'javaClass'='%s')", Organization.class.getName())); - testInstance().getSql().execute(format("CREATE TYPE OfficeType (id BIGINT, name VARCHAR) " - + "OPTIONS ('format'='java', 'javaClass'='%s')", Office.class.getName())); - final IMap testMap = testInstance().getMap("test"); - execute("CREATE MAPPING test (__key BIGINT, this UserType) " - + "TYPE IMap OPTIONS (" - + "'keyFormat'='bigint', " - + "'valueFormat'='java', " - + "'valueJavaClass'='" + User.class.getName() + "')"); + createCompactMapping(testInstance(), "test", "UserCompactType", "organization Organization"); - final Office office = new Office(3L, "office1"); - final Organization organization = new Organization(2L, "organization1", office); - final User user = new User(1L, "user1", organization); - testMap.put(1L, user); - - return user; - } - - private void createJavaType(String name, Class typeClass, String... columns) { - createJavaType(testInstance(), name, typeClass, columns); + execute("INSERT INTO test VALUES (1, (2, 'orgName', null))"); + assertRowsAnyOrder("SELECT (organization).office FROM test", rows(1, new Object[]{null})); } - static void createJavaType(HazelcastInstance inst, String name, Class typeClass, String... columns) { - final String sql = "CREATE TYPE " + name - + "(" + String.join(",", columns) + ")" - + "OPTIONS (" - + "'format'='java'," - + format("'javaClass'='%s'", typeClass.getName()) - + ")"; + @Test + public void test_customOptions() { + new SqlType("Organization") + .fields("name VARCHAR", "governmentFunded BOOLEAN") + .options(OPTION_TYPE_JAVA_CLASS, NonprofitOrganization.class.getName(), + OPTION_TYPE_COMPACT_TYPE_NAME, "NonprofitOrganization") + .create(testInstance()); - execute(inst, sql); - } + createJavaMapping("Users", User.class, "name VARCHAR", "organization Organization"); - private void createJavaMapping(String name, Class javaClass, String... columns) { - createJavaMapping(testInstance(), name, javaClass, columns); - } + execute("INSERT INTO Users VALUES (1, 'Alice', ('Doctors Without Borders', true))"); + assertRowsAnyOrder("SELECT name, (organization).name, (organization).governmentFunded FROM Users", + rows(3, "Alice", "Doctors Without Borders", true)); - static void createJavaMapping(HazelcastInstance inst, String name, Class javaClass, String... columns) { - final String sql = "CREATE MAPPING " + name - + "(" - + "__key BIGINT," - + String.join(",", columns) + - ")" - + " TYPE IMap " - + "OPTIONS (" - + "'keyFormat'='bigint'," - + "'valueFormat'='java'," - + format("'valueJavaClass'='%s'", javaClass.getName()) - + ")"; + createCompactMapping(testInstance(), "Users2", "Users", "name VARCHAR", "organization Organization"); - execute(inst, sql); - } - - void execute(String sql, Object... args) { - execute(testInstance(), sql, args); - } - - static void execute(HazelcastInstance inst, String sql, Object... args) { - inst.getSql().execute(sql, args); + execute("INSERT INTO Users2 VALUES (1, 'Alice', ('Doctors Without Borders', true))"); + SqlResult result = execute("SELECT this FROM Users2"); + DeserializedGenericRecord record = result.iterator().next().getObject(0); + assertEquals("Users", record.getSchema().getTypeName()); + assertEquals("NonprofitOrganization", + ((DeserializedGenericRecord) record.getObject("organization")).getSchema().getTypeName()); } public static class A implements Serializable { public String name; public B b; - public A() { - } + public A() { } - public A(final String name) { + public A(String name) { this.name = name; } } @@ -562,10 +539,9 @@ public static class B implements Serializable { public String name; public C c; - public B() { - } + public B() { } - public B(final String name) { + public B(String name) { this.name = name; } } @@ -574,10 +550,9 @@ public static class C implements Serializable { public String name; public A a; - public C() { - } + public C() { } - public C(final String name) { + public C(String name) { this.name = name; } } @@ -587,10 +562,10 @@ public static class SelfRef implements Serializable { public String name; public SelfRef other; - public SelfRef() { - } + @SuppressWarnings("unused") + public SelfRef() { } - public SelfRef(final Long id, final String name) { + public SelfRef(Long id, String name) { this.id = id; this.name = name; } @@ -601,10 +576,9 @@ public static class User implements Serializable { private String name; private Organization organization; - public User() { - } + public User() { } - public User(final Long id, final String name, final Organization organization) { + public User(Long id, String name, Organization organization) { this.id = id; this.name = name; this.organization = organization; @@ -614,7 +588,7 @@ public Long getId() { return id; } - public void setId(final Long id) { + public void setId(Long id) { this.id = id; } @@ -622,7 +596,7 @@ public String getName() { return name; } - public void setName(final String name) { + public void setName(String name) { this.name = name; } @@ -630,19 +604,20 @@ public Organization getOrganization() { return organization; } - public void setOrganization(final Organization organization) { + @SuppressWarnings("unused") + public void setOrganization(Organization organization) { this.organization = organization; } @Override - public boolean equals(final Object o) { + public boolean equals(Object o) { if (this == o) { return true; } if (o == null || getClass() != o.getClass()) { return false; } - final User user = (User) o; + User user = (User) o; return Objects.equals(id, user.id) && Objects.equals(name, user.name) && Objects.equals(organization, user.organization); @@ -664,14 +639,13 @@ public String toString() { } public static class Organization implements Serializable, Comparable { - private Long id; - private String name; - private Office office; + protected Long id; + protected String name; + protected Office office; - public Organization() { - } + public Organization() { } - public Organization(final Long id, final String name, final Office office) { + public Organization(Long id, String name, Office office) { this.id = id; this.name = name; this.office = office; @@ -681,7 +655,7 @@ public Long getId() { return id; } - public void setId(final Long id) { + public void setId(Long id) { this.id = id; } @@ -689,7 +663,7 @@ public String getName() { return name; } - public void setName(final String name) { + public void setName(String name) { this.name = name; } @@ -697,19 +671,20 @@ public Office getOffice() { return office; } - public void setOffice(final Office office) { + @SuppressWarnings("unused") + public void setOffice(Office office) { this.office = office; } @Override - public boolean equals(final Object o) { + public boolean equals(Object o) { if (this == o) { return true; } if (o == null || getClass() != o.getClass()) { return false; } - final Organization that = (Organization) o; + Organization that = (Organization) o; return Objects.equals(id, that.id) && Objects.equals(name, that.name) && Objects.equals(office, that.office); @@ -735,14 +710,59 @@ public String toString() { } } + @SuppressWarnings("unused") + public static class NonprofitOrganization extends Organization { + private Boolean governmentFunded; + + public NonprofitOrganization() { } + + public Boolean isGovernmentFunded() { + return governmentFunded; + } + + public void setGovernmentFunded(Boolean governmentFunded) { + this.governmentFunded = governmentFunded; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + NonprofitOrganization that = (NonprofitOrganization) o; + return Objects.equals(id, that.id) + && Objects.equals(name, that.name) + && Objects.equals(office, that.office) + && Objects.equals(governmentFunded, that.governmentFunded); + } + + @Override + public int hashCode() { + return Objects.hash(id, name, office, governmentFunded); + } + + @Override + public String toString() { + return "Organization{" + + "id=" + id + + ", name='" + name + '\'' + + ", office=" + office + + ", governmentFunded=" + governmentFunded + + '}'; + } + } + public static class Office implements Serializable, Comparable { private Long id; private String name; - public Office() { - } + @SuppressWarnings("unused") + public Office() { } - public Office(final Long id, final String name) { + public Office(Long id, String name) { this.id = id; this.name = name; } @@ -751,7 +771,7 @@ public Long getId() { return id; } - public void setId(final Long id) { + public void setId(Long id) { this.id = id; } @@ -759,19 +779,19 @@ public String getName() { return name; } - public void setName(final String name) { + public void setName(String name) { this.name = name; } @Override - public boolean equals(final Object o) { + public boolean equals(Object o) { if (this == o) { return true; } if (o == null || getClass() != o.getClass()) { return false; } - final Office office = (Office) o; + Office office = (Office) o; return Objects.equals(id, office.id) && Objects.equals(name, office.name); } @@ -798,10 +818,10 @@ public static class RegularPOJO implements Serializable { private String name; private NestedPOJO child; - public RegularPOJO() { - } + @SuppressWarnings("unused") + public RegularPOJO() { } - public RegularPOJO(final String name, final NestedPOJO child) { + public RegularPOJO(String name, NestedPOJO child) { this.name = name; this.child = child; } @@ -810,7 +830,7 @@ public String getName() { return name; } - public void setName(final String name) { + public void setName(String name) { this.name = name; } @@ -818,21 +838,19 @@ public NestedPOJO getChild() { return child; } - public void setChild(final NestedPOJO child) { + public void setChild(NestedPOJO child) { this.child = child; } @Override - public boolean equals(final Object o) { + public boolean equals(Object o) { if (this == o) { return true; } - if (o == null || getClass() != o.getClass()) { return false; } - - final RegularPOJO that = (RegularPOJO) o; + RegularPOJO that = (RegularPOJO) o; return Objects.equals(name, that.name) && Objects.equals(child, that.child); } @@ -846,10 +864,10 @@ public static class NestedPOJO implements Serializable { private Long id; private String name; - public NestedPOJO() { - } + @SuppressWarnings("unused") + public NestedPOJO() { } - public NestedPOJO(final Long id, final String name) { + public NestedPOJO(Long id, String name) { this.id = id; this.name = name; } @@ -858,7 +876,7 @@ public Long getId() { return id; } - public void setId(final Long id) { + public void setId(Long id) { this.id = id; } @@ -866,21 +884,19 @@ public String getName() { return name; } - public void setName(final String name) { + public void setName(String name) { this.name = name; } @Override - public boolean equals(final Object o) { + public boolean equals(Object o) { if (this == o) { return true; } - if (o == null || getClass() != o.getClass()) { return false; } - - final NestedPOJO that = (NestedPOJO) o; + NestedPOJO that = (NestedPOJO) o; return Objects.equals(id, that.id) && Objects.equals(name, that.name); } @@ -894,19 +910,14 @@ public static class AllTypesParent implements Serializable { private String name; private AllTypesValue child; - public AllTypesParent() { - } - - public AllTypesParent(final String name, final AllTypesValue child) { - this.name = name; - this.child = child; - } + @SuppressWarnings("unused") + public AllTypesParent() { } public String getName() { return name; } - public void setName(final String name) { + public void setName(String name) { this.name = name; } @@ -914,19 +925,19 @@ public AllTypesValue getChild() { return child; } - public void setChild(final AllTypesValue child) { + public void setChild(AllTypesValue child) { this.child = child; } @Override - public boolean equals(final Object o) { + public boolean equals(Object o) { if (this == o) { return true; } if (o == null || getClass() != o.getClass()) { return false; } - final AllTypesParent that = (AllTypesParent) o; + AllTypesParent that = (AllTypesParent) o; return Objects.equals(name, that.name) && Objects.equals(child, that.child); } diff --git a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/type/CompactNestedFieldsTest.java b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/type/CompactNestedFieldsTest.java index 66fda752d47c..a74d7a035e40 100644 --- a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/type/CompactNestedFieldsTest.java +++ b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/type/CompactNestedFieldsTest.java @@ -17,8 +17,10 @@ package com.hazelcast.jet.sql.impl.type; import com.hazelcast.config.Config; +import com.hazelcast.core.HazelcastInstance; import com.hazelcast.internal.serialization.impl.compact.DeserializedGenericRecord; import com.hazelcast.jet.sql.SqlTestSupport; +import com.hazelcast.jet.sql.impl.connector.map.IMapSqlConnector; import com.hazelcast.sql.SqlResult; import com.hazelcast.sql.SqlRow; import com.hazelcast.test.HazelcastSerialClassRunner; @@ -28,6 +30,10 @@ import java.util.ArrayList; +import static com.hazelcast.jet.sql.impl.connector.SqlConnector.COMPACT_FORMAT; +import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_KEY_FORMAT; +import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_COMPACT_TYPE_NAME; +import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_FORMAT; import static com.hazelcast.spi.properties.ClusterProperty.SQL_CUSTOM_TYPES_ENABLED; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.Assert.assertEquals; @@ -44,50 +50,55 @@ public static void beforeClass() { initializeWithClient(3, config, null); } + static void setupCompactTypesForNestedQuery(HazelcastInstance instance) { + new SqlType("Office").fields("id BIGINT", "name VARCHAR").create(instance); + new SqlType("Organization").fields("id BIGINT", "name VARCHAR", "office Office").create(instance); + + createCompactMapping(instance, "test", "UserCompactType", + "id BIGINT", "name VARCHAR", "organization Organization"); + } + + static void createCompactMapping(HazelcastInstance instance, String name, String valueCompactTypeName, + String... valueFields) { + new SqlMapping(name, IMapSqlConnector.class) + .fields("__key BIGINT") + .fields(valueFields) + .options(OPTION_KEY_FORMAT, "bigint", + OPTION_VALUE_FORMAT, COMPACT_FORMAT, + OPTION_VALUE_COMPACT_TYPE_NAME, valueCompactTypeName) + .create(instance); + } + + private static void createType(String name, String... fields) { + new SqlType(name) + .fields(fields) + .create(client()); + } + + private static SqlResult execute(String sql) { + return client().getSql().execute(sql); + } + @Test public void test_basicQuerying() { setupCompactTypesForNestedQuery(client()); - client().getSql().execute("INSERT INTO test VALUES (1, 1, 'user1', (1, 'organization1', (1, 'office1')))"); + execute("INSERT INTO test VALUES (1, 1, 'user1', (1, 'organization1', (1, 'office1')))"); assertRowsAnyOrder("SELECT (organization).office.name FROM test", rows(1, "office1")); } @Test public void test_nestedCompactsAreReturnedAsDeserialized() { - client().getSql().execute("CREATE TYPE Office (" - + "id BIGINT, " - + "name VARCHAR " - + ") OPTIONS ('format'='compact', 'compactTypeName'='OfficeCompactType')"); - - client().getSql().execute("CREATE TYPE Organization (" - + "id BIGINT, " - + "name VARCHAR, " - + "office Office" - + ") OPTIONS ('format'='compact', 'compactTypeName'='OrganizationCompactType')"); - - client().getSql().execute("CREATE TYPE OrganizationAndLong (" - + "id BIGINT, " - + "l BIGINT, " - + "organization Organization" - + ") OPTIONS ('format'='compact', 'compactTypeName'='OrganizationAndLongCompactType')"); - - client().getSql().execute( - "CREATE MAPPING test (" - + "__key BIGINT," - + "id BIGINT, " - + "name VARCHAR, " - + "organizationAndLong OrganizationAndLong" - + ")" - + "TYPE IMap " - + "OPTIONS (" - + "'keyFormat'='bigint'," - + "'valueFormat'='compact'," - + "'valueCompactTypeName'='UserCompactType'" - + ")"); - - - client().getSql().execute("INSERT INTO test VALUES (1, 1, 'user1', (1, 1, (1, 'organization1', (1, 'office1'))))"); - - SqlResult result = client().getSql().execute("SELECT (organizationAndLong).organization.office FROM test"); + createType("Office", "id BIGINT", "name VARCHAR"); + createType("Organization", "id BIGINT", "name VARCHAR", "office Office"); + createType("OrganizationAndLong", "id BIGINT", "l BIGINT", "organization Organization"); + + createCompactMapping(client(), "test", "UserCompactType", + "id BIGINT", "name VARCHAR", "organizationAndLong OrganizationAndLong"); + + + execute("INSERT INTO test VALUES (1, 1, 'user1', (1, 1, (1, 'organization1', (1, 'office1'))))"); + + SqlResult result = execute("SELECT (organizationAndLong).organization.office FROM test"); ArrayList rows = new ArrayList<>(); for (SqlRow row : result) { rows.add(row); @@ -99,8 +110,8 @@ public void test_nestedCompactsAreReturnedAsDeserialized() { @Test public void test_emptyColumnList() { - assertThatThrownBy(() -> client().getSql() - .execute("CREATE TYPE Office OPTIONS ('format'='compact', 'compactTypeName'='OfficeCompactType')")) - .hasMessageContaining("Column list is required to create Compact-based Types"); + createType("Office"); + assertThatThrownBy(() -> createCompactMapping(client(), "test", "OfficesCompactType", "office Office")) + .hasMessageContaining("Column list is required to create Compact-based types"); } } diff --git a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/type/NestedTypesDDLTest.java b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/type/NestedTypesDDLTest.java index 06ae295bd1af..086e538369e0 100644 --- a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/type/NestedTypesDDLTest.java +++ b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/type/NestedTypesDDLTest.java @@ -16,34 +16,26 @@ package com.hazelcast.jet.sql.impl.type; -import com.hazelcast.config.Config; import com.hazelcast.jet.sql.SqlTestSupport; import com.hazelcast.jet.sql.impl.schema.RelationsStorage; -import com.hazelcast.jet.sql.impl.connector.map.IMapSqlConnector; -import com.hazelcast.sql.HazelcastSqlException; -import com.hazelcast.sql.SqlResult; +import com.hazelcast.jet.sql.impl.type.BasicNestedFieldsTest.RegularPOJO; import com.hazelcast.test.HazelcastSerialClassRunner; import org.junit.BeforeClass; import org.junit.Test; import org.junit.runner.RunWith; -import java.io.Serializable; - -import static com.hazelcast.jet.sql.impl.connector.SqlConnector.JAVA_FORMAT; -import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_KEY_CLASS; -import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_KEY_FORMAT; -import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_CLASS_ID; -import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_CLASS_VERSION; -import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_FACTORY_ID; -import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_FORMAT; -import static com.hazelcast.jet.sql.impl.connector.SqlConnector.PORTABLE_FORMAT; +import static com.hazelcast.jet.sql.impl.type.BasicNestedFieldsTest.createJavaMapping; +import static com.hazelcast.jet.sql.impl.type.CompactNestedFieldsTest.createCompactMapping; import static com.hazelcast.spi.properties.ClusterProperty.SQL_CUSTOM_TYPES_ENABLED; -import static java.lang.String.format; -import static org.assertj.core.api.Assertions.assertThat; +import static com.hazelcast.sql.impl.type.QueryDataType.INT; +import static com.hazelcast.sql.impl.type.QueryDataType.VARCHAR; +import static org.assertj.core.api.Assertions.assertThatCode; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; @RunWith(HazelcastSerialClassRunner.class) public class NestedTypesDDLTest extends SqlTestSupport { @@ -51,130 +43,84 @@ public class NestedTypesDDLTest extends SqlTestSupport { @BeforeClass public static void beforeClass() { - Config config = smallInstanceConfig() - .setProperty(SQL_CUSTOM_TYPES_ENABLED.getName(), "true"); - - initialize(2, config); + initialize(2, smallInstanceConfig().setProperty(SQL_CUSTOM_TYPES_ENABLED.getName(), "true")); storage = sqlServiceImpl(instance()).getOptimizer().relationsStorage(); } @Test public void test_createTypeIsNotDuplicatedByDefault() { - execute(format("CREATE TYPE FirstType OPTIONS ('format'='java','javaClass'='%s')", FirstType.class.getName())); - assertThatThrownBy(() -> instance().getSql() - .execute(format("CREATE TYPE FirstType OPTIONS ('format'='java','javaClass'='%s')", SecondType.class.getName()))) - .isInstanceOf(HazelcastSqlException.class) + new SqlType("FirstType").create(); + assertThatThrownBy(() -> new SqlType("FirstType").create()) .hasMessage("Type already exists: FirstType"); - assertEquals(FirstType.class.getName(), storage.getType("FirstType").getJavaClassName()); } @Test public void test_replaceType() { - execute(format("CREATE TYPE FirstType OPTIONS ('format'='java','javaClass'='%s')", FirstType.class.getName())); - execute(format("CREATE OR REPLACE TYPE FirstType OPTIONS ('format'='java','javaClass'='%s')", SecondType.class.getName())); + new SqlType("FirstType").fields("a INT").create(); + new SqlType("FirstType").fields("a VARCHAR").createOrReplace(); - assertEquals(SecondType.class.getName(), storage.getType("FirstType").getJavaClassName()); + assertEquals(VARCHAR, storage.getType("FirstType").getFields().get(0).getType()); } @Test public void test_createIfNotExists() { - execute(format("CREATE TYPE FirstType OPTIONS ('format'='java','javaClass'='%s')", FirstType.class.getName())); - execute(format("CREATE TYPE IF NOT EXISTS FirstType OPTIONS ('format'='java','javaClass'='%s')", SecondType.class.getName())); + new SqlType("FirstType").fields("a INT").create(); + new SqlType("FirstType").fields("a VARCHAR").createIfNotExists(); - assertEquals(FirstType.class.getName(), storage.getType("FirstType").getJavaClassName()); + assertEquals(INT, storage.getType("FirstType").getFields().get(0).getType()); } @Test public void test_showTypes() { - execute(format("CREATE TYPE FirstType OPTIONS ('format'='java','javaClass'='%s')", FirstType.class.getName())); - execute(format("CREATE TYPE SecondType OPTIONS ('format'='java','javaClass'='%s')", SecondType.class.getName())); + new SqlType("FirstType").create(); + new SqlType("SecondType").create(); + assertRowsAnyOrder("SHOW TYPES", rows(1, "FirstType", "SecondType")); } @Test public void test_dropNonexistentType() { assertThatThrownBy(() -> execute("DROP TYPE Foo")) - .isInstanceOf(HazelcastSqlException.class) .hasMessage("Type does not exist: Foo"); execute("DROP TYPE IF EXISTS Foo"); } @Test - public void test_createTwoTypesForSameJavaClass() { - execute(format("CREATE TYPE FirstType OPTIONS ('format'='java','javaClass'='%s')", FirstType.class.getName())); - execute(format("CREATE TYPE SecondType OPTIONS ('format'='java','javaClass'='%s')", FirstType.class.getName())); - } - - @Test - public void test_createTwoTypesForSamePortableClass() { - execute("CREATE TYPE FirstType(a INT, b INT) OPTIONS ('format'='portable','portableFactoryId'='123','portableClassId'='456')"); - execute("CREATE TYPE SecondType(c VARCHAR, d VARCHAR) OPTIONS ('format'='portable','portableFactoryId'='123','portableClassId'='456')"); - - try (SqlResult result = instance().getSql().execute("CREATE OR REPLACE MAPPING " + "m(" + - "e varchar" + - ")" + " TYPE " + IMapSqlConnector.TYPE_NAME + " " - + "OPTIONS (" - + '\'' + OPTION_KEY_FORMAT + "'='" + JAVA_FORMAT + '\'' - + ", '" + OPTION_KEY_CLASS + "'='" + Long.class.getName() + '\'' - + ", '" + OPTION_VALUE_FORMAT + "'='" + PORTABLE_FORMAT + '\'' - + ", '" + OPTION_VALUE_FACTORY_ID + "'='" + 123 + '\'' - + ", '" + OPTION_VALUE_CLASS_ID + "'='" + 456 + '\'' - + ", '" + OPTION_VALUE_CLASS_VERSION + "'='" + 0 + '\'' - + ")" - )) { - assertThat(result.updateCount()).isEqualTo(0); - } - } - - @Test - public void test_createTwoTypesForSameCompactClass() { - execute("CREATE TYPE FirstType(a int, b varchar) OPTIONS ('format'='compact','compactTypeName'='foo')"); - execute("CREATE TYPE SecondType(a int, b varchar) OPTIONS ('format'='compact','compactTypeName'='foo')"); - } - - @Test - public void when_javaClassUnknown_then_fail() { - assertThatThrownBy(() -> - execute("CREATE TYPE FirstType OPTIONS ('format'='java','javaClass'='foo')")) - .hasRootCauseMessage("foo") - .hasRootCauseInstanceOf(ClassNotFoundException.class); + public void when_compactTypeNoColumns_then_fail() { + new SqlType("FirstType").create(); + assertThatThrownBy(() -> createCompactMapping(instance(), "test", "foo", "bar FirstType")) + .hasMessage("Column list is required to create Compact-based types"); } @Test - public void when_portableClassDefNotKnown_then_requireFields() { - assertThatThrownBy(() -> - execute("CREATE TYPE FirstType OPTIONS ('format'='portable','portableFactoryId'='123','portableClassId'='456')")) - .hasMessage("The given FactoryID/ClassID/Version combination not known to the member. You need to provide column list for this type"); - } + public void test_typeFieldResolution() { + new SqlType("NestedType").create(); + assertTrue(storage.getType("NestedType").getFields().isEmpty()); + assertThatThrownBy(() -> createCompactMapping(instance(), "test", "foo", "bar NestedType")) + .hasMessage("Column list is required to create Compact-based types"); - @Test - public void when_compactTypeNoColumns_then_fail() { - assertThatThrownBy(() -> - execute("CREATE TYPE FirstType OPTIONS ('format'='compact','compactTypeName'='foo')")) - .hasMessage("Column list is required to create Compact-based Types"); + createJavaMapping(instance(), "test2", RegularPOJO.class, "name VARCHAR", "child NestedType"); + assertFalse(storage.getType("NestedType").getFields().isEmpty()); + assertThatCode(() -> createCompactMapping(instance(), "test", "foo", "bar NestedType")) + .doesNotThrowAnyException(); } @Test public void test_failOnDuplicateColumnName() { - assertThatThrownBy(() -> execute("CREATE TYPE TestType (id BIGINT, id BIGINT) " - + "OPTIONS ('format'='compact', 'compactTypeName'='TestType')")) - .isInstanceOf(HazelcastSqlException.class) + assertThatThrownBy(() -> new SqlType("TestType").fields("id BIGINT", "id BIGINT").create()) .hasMessageContaining("Column 'id' specified more than once"); } @Test public void test_failOnReplaceAndIfNotExists() { - assertThatThrownBy(() -> execute("CREATE OR REPLACE TYPE IF NOT EXISTS TestType (id BIGINT, name VARCHAR) " - + "OPTIONS ('format'='compact', 'compactTypeName'='TestType')")) - .isInstanceOf(HazelcastSqlException.class) + assertThatThrownBy(() -> execute("CREATE OR REPLACE TYPE IF NOT EXISTS TestType (id BIGINT, name VARCHAR)")) .hasMessageContaining("OR REPLACE in conjunction with IF NOT EXISTS not supported"); } @Test public void test_fullyQualifiedTypeName() { - execute(format("CREATE TYPE hazelcast.public.FirstType OPTIONS ('format'='java','javaClass'='%s')", - FirstType.class.getName())); + new SqlType("hazelcast.public.FirstType").create(); assertNotNull(storage.getType("FirstType")); execute("DROP TYPE hazelcast.public.FirstType"); @@ -183,65 +129,17 @@ public void test_fullyQualifiedTypeName() { @Test public void test_failOnNonPublicSchemaType() { - assertThatThrownBy(() -> execute("CREATE TYPE information_schema.TestType (id BIGINT, name VARCHAR) " - + "OPTIONS ('format'='compact', 'compactTypeName'='TestType')")) - .isInstanceOf(HazelcastSqlException.class) + assertThatThrownBy(() -> new SqlType("information_schema.TestType").create()) .hasMessageContaining("The type must be created in the \"public\" schema"); assertNull(storage.getType("TestType")); - execute(format("CREATE TYPE hazelcast.public.TestType OPTIONS ('format'='java','javaClass'='%s')", - FirstType.class.getName())); + new SqlType("hazelcast.public.TestType").create(); assertThatThrownBy(() -> execute("DROP TYPE information_schema.TestType")) - .isInstanceOf(HazelcastSqlException.class) .hasMessageContaining("Type does not exist: information_schema.TestType"); assertNotNull(storage.getType("TestType")); } - @Test - public void test_failOnDuplicateOptions() { - assertThatThrownBy(() -> execute("CREATE TYPE TestType (id BIGINT, name VARCHAR) " - + "OPTIONS ('format'='compact', 'compactTypeName'='TestType', 'compactTypeName'='TestType2')")) - .isInstanceOf(HazelcastSqlException.class) - .hasMessageContaining("Option 'compactTypeName' specified more than once"); - } - void execute(String sql) { instance().getSql().execute(sql); } - - public static class FirstType implements Serializable { - private String name; - - public FirstType() { } - - public FirstType(final String name) { - this.name = name; - } - - public String getName() { - return name; - } - - public void setName(final String name) { - this.name = name; - } - } - - public static class SecondType implements Serializable { - private String name; - - public SecondType() { } - - public SecondType(final String name) { - this.name = name; - } - - public String getName() { - return name; - } - - public void setName(final String name) { - this.name = name; - } - } } diff --git a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/type/PortableNestedFieldsTest.java b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/type/PortableNestedFieldsTest.java index ec54b3f1ca17..b4ac5e73e4da 100644 --- a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/type/PortableNestedFieldsTest.java +++ b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/type/PortableNestedFieldsTest.java @@ -18,11 +18,12 @@ import com.hazelcast.config.Config; import com.hazelcast.config.SerializationConfig; +import com.hazelcast.core.HazelcastInstance; import com.hazelcast.internal.serialization.impl.portable.DeserializedPortableGenericRecord; import com.hazelcast.jet.sql.SqlTestSupport; +import com.hazelcast.jet.sql.impl.connector.map.IMapSqlConnector; import com.hazelcast.nio.serialization.ClassDefinition; import com.hazelcast.nio.serialization.ClassDefinitionBuilder; -import com.hazelcast.sql.HazelcastSqlException; import com.hazelcast.sql.SqlResult; import com.hazelcast.sql.SqlRow; import com.hazelcast.test.HazelcastSerialClassRunner; @@ -32,12 +33,18 @@ import java.util.ArrayList; +import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_KEY_FORMAT; +import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_CLASS_ID; +import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_FACTORY_ID; +import static com.hazelcast.jet.sql.impl.connector.SqlConnector.OPTION_VALUE_FORMAT; +import static com.hazelcast.jet.sql.impl.connector.SqlConnector.PORTABLE_FORMAT; import static com.hazelcast.spi.properties.ClusterProperty.SQL_CUSTOM_TYPES_ENABLED; -import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assertions.assertThatCode; import static org.junit.Assert.assertEquals; @RunWith(HazelcastSerialClassRunner.class) public class PortableNestedFieldsTest extends SqlTestSupport { + @BeforeClass public static void beforeClass() { Config config = smallInstanceConfig() @@ -70,7 +77,6 @@ public static void beforeClass() { .addStringField("name") .addPortableField("organization", organizationType) .build(); - serializationConfig.addClassDefinition(userType); ClassDefinition userType2 = new ClassDefinitionBuilder(1, 5) @@ -78,15 +84,38 @@ public static void beforeClass() { .addStringField("name") .addPortableField("organizationAndLong", organizationAndLongType) .build(); - serializationConfig.addClassDefinition(userType2); + initialize(2, config); } + static void setupPortableTypesForNestedQuery(HazelcastInstance instance) { + new SqlType("Office").create(instance); + new SqlType("Organization").fields("id BIGINT", "name VARCHAR", "office Office").create(instance); + + createPortableMapping(instance, "test", 1, 1, "id BIGINT", "name VARCHAR", "organization Organization"); + } + + static void createPortableMapping(HazelcastInstance instance, String name, int factoryId, int classId, + String... valueFields) { + new SqlMapping(name, IMapSqlConnector.class) + .fields("__key BIGINT") + .fields(valueFields) + .options(OPTION_KEY_FORMAT, "bigint", + OPTION_VALUE_FORMAT, PORTABLE_FORMAT, + OPTION_VALUE_FACTORY_ID, factoryId, + OPTION_VALUE_CLASS_ID, classId) + .create(instance); + } + + private static SqlResult execute(String sql) { + return instance().getSql().execute(sql); + } + @Test public void test_basicQuerying() { setupPortableTypesForNestedQuery(instance()); - instance().getSql().execute("INSERT INTO test VALUES (1, 1, 'user1', (1, 'organization1', (1, 'office1')))"); + execute("INSERT INTO test VALUES (1, 1, 'user1', (1, 'organization1', (1, 'office1')))"); assertRowsAnyOrder("SELECT (organization).name FROM test", rows(1, "organization1")); assertRowsAnyOrder("SELECT (organization).office.name FROM test", rows(1, "office1")); @@ -94,28 +123,16 @@ public void test_basicQuerying() { @Test public void test_nestedPortablesAreReturnedAsDeserialized() { - instance().getSql().execute("CREATE TYPE Office OPTIONS " - + "('format'='portable', 'portableFactoryId'='1', 'portableClassId'='3', 'portableClassVersion'='0')"); - instance().getSql().execute("CREATE TYPE Organization OPTIONS " - + "('format'='portable', 'portableFactoryId'='1', 'portableClassId'='2', 'portableClassVersion'='0')"); - instance().getSql().execute("CREATE TYPE OrganizationAndLong OPTIONS " - + "('format'='portable', 'portableFactoryId'='1', 'portableClassId'='4', 'portableClassVersion'='0')"); - - instance().getSql().execute("CREATE MAPPING test (" - + "__key BIGINT, " - + "id BIGINT, " - + "name VARCHAR, " - + "organizationAndLong OrganizationAndLong " - + ") TYPE IMap " - + "OPTIONS (" - + "'keyFormat'='bigint', " - + "'valueFormat'='portable', " - + "'valuePortableFactoryId'='1', " - + "'valuePortableClassId'='5')"); - - instance().getSql().execute("INSERT INTO test VALUES (1, 1, 'user1', (1, 1, (1, 'organization1', (1, 'office1'))))"); - - SqlResult result = instance().getSql().execute("SELECT (organizationAndLong).organization.office FROM test"); + new SqlType("Office").create(); + new SqlType("Organization").fields("id BIGINT", "name VARCHAR", "office Office").create(); + new SqlType("OrganizationAndLong").fields("id BIGINT", "l BIGINT", "organization Organization").create(); + + createPortableMapping(instance(), "test", 1, 5, + "id BIGINT", "name VARCHAR", "organizationAndLong OrganizationAndLong"); + + execute("INSERT INTO test VALUES (1, 1, 'user1', (1, 1, (1, 'organization1', (1, 'office1'))))"); + + SqlResult result = execute("SELECT (organizationAndLong).organization.office FROM test"); ArrayList rows = new ArrayList<>(); for (SqlRow row : result) { rows.add(row); @@ -124,19 +141,9 @@ public void test_nestedPortablesAreReturnedAsDeserialized() { assertInstanceOf(DeserializedPortableGenericRecord.class, rows.get(0).getObject(0)); } - @Test - public void when_unknownClassDef_noColumns_then_fail() { - assertThatThrownBy(() -> instance().getSql().execute("CREATE TYPE Foo " + - "OPTIONS('format'='portable', 'portableFactoryId'='42', 'portableClassId'='43')")) - .isInstanceOf(HazelcastSqlException.class) - .hasMessage("The given FactoryID/ClassID/Version combination not known to the member. You need" + - " to provide column list for this type"); - } - @Test public void test_unknownClassDef_givenColumns() { - instance().getSql().execute("CREATE TYPE Foo (column1 INT, column2 VARCHAR) " + - "OPTIONS('format'='portable', 'portableFactoryId'='44', 'portableClassId'='45')"); - // we test that the above command doesn't fail. + assertThatCode(() -> new SqlType("Foo").fields("column1 INT", "column2 VARCHAR").create()) + .doesNotThrowAnyException(); } } diff --git a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/type/RecurrentStructuresNestedFieldsTest.java b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/type/RecurrentStructuresNestedFieldsTest.java index f6f778f38d17..348db07f03e8 100644 --- a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/type/RecurrentStructuresNestedFieldsTest.java +++ b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/type/RecurrentStructuresNestedFieldsTest.java @@ -17,7 +17,6 @@ package com.hazelcast.jet.sql.impl.type; import com.hazelcast.config.Config; -import com.hazelcast.core.HazelcastException; import com.hazelcast.jet.sql.SqlTestSupport; import com.hazelcast.test.HazelcastSerialClassRunner; import org.junit.BeforeClass; @@ -25,6 +24,7 @@ import org.junit.runner.RunWith; import java.io.Serializable; +import java.util.function.Consumer; import static com.hazelcast.spi.properties.ClusterProperty.SQL_CUSTOM_TYPES_ENABLED; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -40,57 +40,47 @@ public static void beforeClass() { initializeWithClient(2, config, null); } + private static void createJavaMapping(String name, Class valueClass, String... valueFields) { + BasicNestedFieldsTest.createJavaMapping(client(), name, valueClass, valueFields); + } + + private static void createType(String name, String... fields) { + new SqlType(name) + .fields(fields) + .create(client()); + } + @Test public void test_cyclicTypeUpsertsValidationError() { - createJavaType("FCA", FullyConnectedA.class, "name VARCHAR", "b FCB", "c FCC"); - createJavaType("FCB", FullyConnectedB.class, "name VARCHAR", "a FCA", "c FCC"); - createJavaType("FCC", FullyConnectedC.class, "name VARCHAR", "a FCA", "b FCB"); + createType("FCA", "name VARCHAR", "b FCB", "c FCC"); + createType("FCB", "name VARCHAR", "a FCA", "c FCC"); + createType("FCC", "name VARCHAR", "a FCA", "b FCB"); createJavaMapping("tableA", FullyConnectedA.class, "this FCA"); createJavaMapping("tableB", FullyConnectedB.class, "this FCB"); createJavaMapping("tableC", FullyConnectedC.class, "this FCC"); - createJavaType("DualGraph", DualPathGraph.class, - "name VARCHAR", "\"left\" DualGraph", "\"right\" DualGraph"); + createType("DualGraph", "name VARCHAR", "\"left\" DualGraph", "\"right\" DualGraph"); createJavaMapping("tableD", DualPathGraph.class, "this DualGraph"); - assertThatThrownBy(() -> client().getSql().execute("INSERT INTO tableA VALUES (1, ?)")) - .isInstanceOf(HazelcastException.class) - .hasMessageContaining("Upserts are not supported for cyclic data type columns"); - - assertThatThrownBy(() -> client().getSql().execute("INSERT INTO tableB VALUES (1, ?)")) - .isInstanceOf(HazelcastException.class) - .hasMessageContaining("Upserts are not supported for cyclic data type columns"); - - assertThatThrownBy(() -> client().getSql().execute("INSERT INTO tableC VALUES (1, ?)")) - .isInstanceOf(HazelcastException.class) - .hasMessageContaining("Upserts are not supported for cyclic data type columns"); - - assertThatThrownBy(() -> client().getSql().execute("INSERT INTO tableD VALUES (1, ?)")) - .isInstanceOf(HazelcastException.class) + Consumer assertNotSupported = sql -> assertThatThrownBy(() -> client().getSql().execute(sql)) .hasMessageContaining("Upserts are not supported for cyclic data type columns"); - assertThatThrownBy(() -> client().getSql().execute("UPDATE tableA SET this = ? WHERE __key = 1")) - .isInstanceOf(HazelcastException.class) - .hasMessageContaining("Upserts are not supported for cyclic data type columns"); - - assertThatThrownBy(() -> client().getSql().execute("UPDATE tableB SET this = ? WHERE __key = 1")) - .isInstanceOf(HazelcastException.class) - .hasMessageContaining("Upserts are not supported for cyclic data type columns"); - - assertThatThrownBy(() -> client().getSql().execute("UPDATE tableC SET this = ? WHERE __key = 1")) - .isInstanceOf(HazelcastException.class) - .hasMessageContaining("Upserts are not supported for cyclic data type columns"); + assertNotSupported.accept("INSERT INTO tableA VALUES (1, ?)"); + assertNotSupported.accept("INSERT INTO tableB VALUES (1, ?)"); + assertNotSupported.accept("INSERT INTO tableC VALUES (1, ?)"); + assertNotSupported.accept("INSERT INTO tableD VALUES (1, ?)"); - assertThatThrownBy(() -> client().getSql().execute("UPDATE tableD SET this = ? WHERE __key = 1")) - .isInstanceOf(HazelcastException.class) - .hasMessageContaining("Upserts are not supported for cyclic data type columns"); + assertNotSupported.accept("UPDATE tableA SET this = ? WHERE __key = 1"); + assertNotSupported.accept("UPDATE tableB SET this = ? WHERE __key = 1"); + assertNotSupported.accept("UPDATE tableC SET this = ? WHERE __key = 1"); + assertNotSupported.accept("UPDATE tableD SET this = ? WHERE __key = 1"); } @Test public void test_fullyConnectedGraph() { - createJavaType("FCA", FullyConnectedA.class, "name VARCHAR", "b FCB", "c FCC"); - createJavaType("FCB", FullyConnectedB.class, "name VARCHAR", "a FCA", "c FCC"); - createJavaType("FCC", FullyConnectedC.class, "name VARCHAR", "a FCA", "b FCB"); + createType("FCA", "name VARCHAR", "b FCB", "c FCC"); + createType("FCB", "name VARCHAR", "a FCA", "c FCC"); + createType("FCC", "name VARCHAR", "a FCA", "b FCB"); final FullyConnectedA a = new FullyConnectedA("A1"); final FullyConnectedB b = new FullyConnectedB("B1"); @@ -122,9 +112,9 @@ public void test_fullyConnectedGraph() { @Test public void test_sameTypesDifferentInstances() { - createJavaType("FCA", FullyConnectedA.class, "name VARCHAR", "b FCB", "c FCC"); - createJavaType("FCB", FullyConnectedB.class, "name VARCHAR", "a FCA", "c FCC"); - createJavaType("FCC", FullyConnectedC.class, "name VARCHAR", "a FCA", "b FCB"); + createType("FCA", "name VARCHAR", "b FCB", "c FCC"); + createType("FCB", "name VARCHAR", "a FCA", "c FCC"); + createType("FCC", "name VARCHAR", "a FCA", "b FCB"); // A1 -> B1 -> C1 -> A2 -> B2 -> C2 -> final FullyConnectedA a1 = new FullyConnectedA("A1"); @@ -169,8 +159,7 @@ public void test_treeLikeCyclicGraph() { | \ \ [A1][A4] [A3] */ - createJavaType("DualGraph", DualPathGraph.class, - "name VARCHAR", "\"left\" DualGraph", "\"right\" DualGraph"); + createType("DualGraph", "name VARCHAR", "\"left\" DualGraph", "\"right\" DualGraph"); DualPathGraph a1 = new DualPathGraph("A1"); DualPathGraph a2 = new DualPathGraph("A2"); DualPathGraph a3 = new DualPathGraph("A3"); @@ -204,22 +193,15 @@ public void test_treeLikeCyclicGraph() { } - private void createJavaType(String name, Class typeClass, String... columns) { - BasicNestedFieldsTest.createJavaType(client(), name, typeClass, columns); - } - - private void createJavaMapping(String name, Class javaClass, String... columns) { - BasicNestedFieldsTest.createJavaMapping(client(), name, javaClass, columns); - } - public static class FullyConnectedA implements Serializable { private String name; private FullyConnectedB b; private FullyConnectedC c; + @SuppressWarnings("unused") public FullyConnectedA() { } - public FullyConnectedA(final String name) { + public FullyConnectedA(String name) { this.name = name; } @@ -227,7 +209,7 @@ public String getName() { return name; } - public void setName(final String name) { + public void setName(String name) { this.name = name; } @@ -235,7 +217,7 @@ public FullyConnectedB getB() { return b; } - public void setB(final FullyConnectedB b) { + public void setB(FullyConnectedB b) { this.b = b; } @@ -243,7 +225,7 @@ public FullyConnectedC getC() { return c; } - public void setC(final FullyConnectedC c) { + public void setC(FullyConnectedC c) { this.c = c; } } @@ -253,9 +235,10 @@ public static class FullyConnectedB implements Serializable { private FullyConnectedA a; private FullyConnectedC c; + @SuppressWarnings("unused") public FullyConnectedB() { } - public FullyConnectedB(final String name) { + public FullyConnectedB(String name) { this.name = name; } @@ -263,7 +246,7 @@ public String getName() { return name; } - public void setName(final String name) { + public void setName(String name) { this.name = name; } @@ -271,7 +254,7 @@ public FullyConnectedA getA() { return a; } - public void setA(final FullyConnectedA a) { + public void setA(FullyConnectedA a) { this.a = a; } @@ -279,7 +262,7 @@ public FullyConnectedC getC() { return c; } - public void setC(final FullyConnectedC c) { + public void setC(FullyConnectedC c) { this.c = c; } } @@ -289,13 +272,14 @@ public static class FullyConnectedC implements Serializable { private FullyConnectedA a; private FullyConnectedB b; + @SuppressWarnings("unused") public FullyConnectedC() { } public String getName() { return name; } - public void setName(final String name) { + public void setName(String name) { this.name = name; } @@ -303,7 +287,7 @@ public FullyConnectedA getA() { return a; } - public void setA(final FullyConnectedA a) { + public void setA(FullyConnectedA a) { this.a = a; } @@ -311,11 +295,11 @@ public FullyConnectedB getB() { return b; } - public void setB(final FullyConnectedB b) { + public void setB(FullyConnectedB b) { this.b = b; } - public FullyConnectedC(final String name) { + public FullyConnectedC(String name) { this.name = name; } } @@ -325,9 +309,10 @@ public static class DualPathGraph implements Serializable { private DualPathGraph left; private DualPathGraph right; + @SuppressWarnings("unused") public DualPathGraph() { } - public DualPathGraph(final String name) { + public DualPathGraph(String name) { this.name = name; } @@ -335,7 +320,7 @@ public String getName() { return name; } - public void setName(final String name) { + public void setName(String name) { this.name = name; } @@ -343,7 +328,7 @@ public DualPathGraph getLeft() { return left; } - public void setLeft(final DualPathGraph left) { + public void setLeft(DualPathGraph left) { this.left = left; } @@ -351,7 +336,7 @@ public DualPathGraph getRight() { return right; } - public void setRight(final DualPathGraph right) { + public void setRight(DualPathGraph right) { this.right = right; } } diff --git a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/type/SqlLazyDeserializationTest.java b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/type/SqlLazyDeserializationTest.java index ab9334633dd1..db410aa60e21 100644 --- a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/type/SqlLazyDeserializationTest.java +++ b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/type/SqlLazyDeserializationTest.java @@ -54,9 +54,9 @@ import java.util.concurrent.CopyOnWriteArrayList; import static com.hazelcast.jet.sql.SqlTestSupport.assertRowsAnyOrder; -import static com.hazelcast.jet.sql.SqlTestSupport.setupCompactTypesForNestedQuery; import static com.hazelcast.jet.sql.SqlTestSupport.rows; -import static com.hazelcast.jet.sql.SqlTestSupport.setupPortableTypesForNestedQuery; +import static com.hazelcast.jet.sql.impl.type.CompactNestedFieldsTest.setupCompactTypesForNestedQuery; +import static com.hazelcast.jet.sql.impl.type.PortableNestedFieldsTest.setupPortableTypesForNestedQuery; import static com.hazelcast.spi.properties.ClusterProperty.SQL_CUSTOM_TYPES_ENABLED; import static org.junit.Assert.assertEquals; import static org.mockito.ArgumentMatchers.any; @@ -73,7 +73,6 @@ @RunWith(HazelcastSerialClassRunner.class) @Category({QuickTest.class, ParallelJVMTest.class}) public class SqlLazyDeserializationTest { - private final SqlLazyDeserializationTestInstanceFactory mockInstanceFactory = new SqlLazyDeserializationTestInstanceFactory(); diff --git a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/type/UdtObjectToJsonFunctionTest.java b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/type/UdtObjectToJsonFunctionTest.java index a6412ef4ea2f..149ade44dd70 100644 --- a/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/type/UdtObjectToJsonFunctionTest.java +++ b/hazelcast-sql/src/test/java/com/hazelcast/jet/sql/impl/type/UdtObjectToJsonFunctionTest.java @@ -23,7 +23,6 @@ import com.hazelcast.map.IMap; import com.hazelcast.nio.serialization.ClassDefinition; import com.hazelcast.nio.serialization.ClassDefinitionBuilder; -import com.hazelcast.sql.HazelcastSqlException; import com.hazelcast.sql.impl.SqlErrorCode; import com.hazelcast.test.HazelcastSerialClassRunner; import org.junit.BeforeClass; @@ -36,13 +35,14 @@ import static com.hazelcast.jet.sql.impl.type.BasicNestedFieldsTest.Office; import static com.hazelcast.jet.sql.impl.type.BasicNestedFieldsTest.Organization; import static com.hazelcast.jet.sql.impl.type.BasicNestedFieldsTest.createJavaMapping; -import static com.hazelcast.jet.sql.impl.type.BasicNestedFieldsTest.createJavaType; +import static com.hazelcast.jet.sql.impl.type.CompactNestedFieldsTest.setupCompactTypesForNestedQuery; +import static com.hazelcast.jet.sql.impl.type.PortableNestedFieldsTest.setupPortableTypesForNestedQuery; import static com.hazelcast.spi.properties.ClusterProperty.SQL_CUSTOM_TYPES_ENABLED; -import static java.lang.String.format; import static org.assertj.core.api.Assertions.assertThatThrownBy; @RunWith(HazelcastSerialClassRunner.class) public class UdtObjectToJsonFunctionTest extends SqlJsonTestSupport { + @BeforeClass public static void beforeClass() { Config config = smallInstanceConfig() @@ -73,19 +73,44 @@ public static void beforeClass() { initializeWithClient(3, config, null); } + private static void createType(String name, String... fields) { + new SqlType(name) + .fields(fields) + .create(client()); + } + + private static void execute(String sql, Object... args) { + client().getSql().execute(sql, args); + } + + private void initDefault() { + createType("UserType", "id BIGINT", "name VARCHAR", "organization OrganizationType"); + createType("OrganizationType", "id BIGINT", "name VARCHAR", "office OfficeType"); + createType("OfficeType", "id BIGINT", "name VARCHAR"); + + final IMap testMap = client().getMap("test"); + createJavaMapping(client(), "test", User.class, "this UserType"); + + final Office office = new Office(3L, "office1"); + final Organization organization = new Organization(2L, "organization1", office); + final User user = new User(1L, "user1", organization); + testMap.put(1L, user); + } + @Test public void test_nonCyclic() { initDefault(); - assertJsonRowsAnyOrder("SELECT CAST(this AS JSON) FROM test", rows(1, json( - "{\"organization\":{\"name\":\"organization1\",\"id\":2,\"office\":{\"name\":\"office1\",\"id\":3}},\"id\":1,\"name\":\"user1\"}"))); + assertJsonRowsAnyOrder("SELECT CAST(this AS JSON) FROM test", rows(1, + json("{\"organization\":{\"name\":\"organization1\",\"id\":2,\"office\":" + + "{\"name\":\"office1\",\"id\":3}},\"id\":1,\"name\":\"user1\"}"))); } @Test public void test_failOnCycles() { - createJavaType(client(), "AType", A.class, "name VARCHAR", "b BType"); - createJavaType(client(), "BType", B.class, "name VARCHAR", "c CType"); - createJavaType(client(), "CType", C.class, "name VARCHAR", "a AType"); + createType("AType", "name VARCHAR", "b BType"); + createType("BType", "name VARCHAR", "c CType"); + createType("CType", "name VARCHAR", "a AType"); final A a = new A("a"); final B b = new B("b"); @@ -99,9 +124,9 @@ public void test_failOnCycles() { IMap map = client().getMap("test"); map.put(1L, a); - assertThatThrownBy(() -> assertRowsAnyOrder(client(), "SELECT CAST(this AS JSON) FROM test", rows(1, "a"))) + assertThatThrownBy(() -> assertRowsAnyOrder(client(), "SELECT CAST(this AS JSON) FROM test", + rows(1, "a"))) .hasMessageEndingWith("Cycle detected in row value") - .isInstanceOf(HazelcastSqlException.class) .hasFieldOrPropertyWithValue("code", SqlErrorCode.DATA_EXCEPTION); } @@ -112,36 +137,34 @@ public void test_returnNullOnNull() { final User user = new User(2L, "user2", null); client().getMap("test").put(2L, user); - assertRowsAnyOrder("SELECT CAST((this).organization AS JSON) FROM test WHERE __key = 2", rows(1, new Object[] {null})); + assertRowsAnyOrder("SELECT CAST((this).organization AS JSON) FROM test WHERE __key = 2", + rows(1, new Object[] {null})); } @Test public void test_castRowAsJsonShouldFail() { - assertThatThrownBy(() -> assertRowsAnyOrder(client(), "SELECT CAST(v AS JSON) FROM (SELECT (42, 'foo') v)", rows(1, ""))) + assertThatThrownBy(() -> assertRowsAnyOrder(client(), "SELECT CAST(v AS JSON) FROM (SELECT (42, 'foo') v)", + rows(1, ""))) .hasMessageEndingWith("CAST function cannot convert value of type ROW to type JSON") - .isInstanceOf(HazelcastSqlException.class) .hasFieldOrPropertyWithValue("code", SqlErrorCode.PARSING); } @Test public void test_disabledQueries() { - assertThatThrownBy(() -> client().getSql().execute("SELECT CAST(? as JSON)", new User())) - .isInstanceOf(HazelcastSqlException.class) + assertThatThrownBy(() -> execute("SELECT CAST(? as JSON)", new User())) .hasMessageContaining("Cannot convert OBJECT to JSON"); - assertThatThrownBy(() -> client().getSql().execute("SELECT CAST((2, 'user', null) as JSON)")) - .isInstanceOf(HazelcastSqlException.class) + assertThatThrownBy(() -> execute("SELECT CAST((2, 'user', null) as JSON)")) .hasMessageContaining("Cannot convert ROW to JSON"); - assertThatThrownBy(() -> instance().getSql().execute("SELECT CAST(CAST((2, 'user', null) AS UserType) as JSON)")) - .isInstanceOf(HazelcastSqlException.class) + assertThatThrownBy(() -> execute("SELECT CAST(CAST((2, 'user', null) AS UserType) as JSON)")) .hasMessageContaining("Complex type specifications are not supported"); } @Test public void test_compact() { setupCompactTypesForNestedQuery(client()); - client().getSql().execute("INSERT INTO test VALUES (1, 1, 'user1', (10, 'organization1', (100, 'office1')))"); + execute("INSERT INTO test VALUES (1, 1, 'user1', (10, 'organization1', (100, 'office1')))"); assertJsonRowsAnyOrder("SELECT CAST(organization AS JSON) FROM test", rows(1, json("{\"name\":\"organization1\",\"id\":10,\"office\":{\"name\":\"office1\",\"id\":100}}"))); @@ -150,29 +173,9 @@ public void test_compact() { @Test public void test_portable() { setupPortableTypesForNestedQuery(client()); - client().getSql().execute("INSERT INTO test VALUES (1, 1, 'user1', (10, 'organization1', (100, 'office1')))"); + execute("INSERT INTO test VALUES (1, 1, 'user1', (10, 'organization1', (100, 'office1')))"); assertJsonRowsAnyOrder("SELECT CAST(organization AS JSON) FROM test", rows(1, json("{\"name\":\"organization1\",\"id\":10,\"office\":{\"name\":\"office1\",\"id\":100}}"))); } - - private void initDefault() { - client().getSql().execute(format("CREATE TYPE UserType (id BIGINT, name VARCHAR, organization OrganizationType) " - + "OPTIONS ('format'='java', 'javaClass'='%s')", User.class.getName())); - client().getSql().execute(format("CREATE TYPE OrganizationType (id BIGINT, name VARCHAR, office OfficeType) " - + "OPTIONS ('format'='java', 'javaClass'='%s')", Organization.class.getName())); - client().getSql().execute(format("CREATE TYPE OfficeType (id BIGINT, name VARCHAR) " - + "OPTIONS ('format'='java', 'javaClass'='%s')", Office.class.getName())); - final IMap testMap = client().getMap("test"); - client().getSql().execute("CREATE MAPPING test (__key BIGINT, this UserType) " - + "TYPE IMap OPTIONS (" - + "'keyFormat'='bigint', " - + "'valueFormat'='java', " - + "'valueJavaClass'='" + User.class.getName() + "')"); - - final Office office = new Office(3L, "office1"); - final Organization organization = new Organization(2L, "organization1", office); - final User user = new User(1L, "user1", organization); - testMap.put(1L, user); - } } diff --git a/hazelcast-sql/src/test/java/com/hazelcast/sql/impl/extract/QueryPathTest.java b/hazelcast-sql/src/test/java/com/hazelcast/sql/impl/extract/QueryPathTest.java index 90a61869f65e..2f814b746829 100644 --- a/hazelcast-sql/src/test/java/com/hazelcast/sql/impl/extract/QueryPathTest.java +++ b/hazelcast-sql/src/test/java/com/hazelcast/sql/impl/extract/QueryPathTest.java @@ -75,6 +75,6 @@ private void checkPath(String originalPath, boolean expectedKey, String expected assertEquals(expectedKey, path.isKey()); assertEquals(expectedPath, path.getPath()); - assertEquals(expectedPath == null, path.isTop()); + assertEquals(expectedPath == null, path.isTopLevel()); } } diff --git a/hazelcast/src/main/java/com/hazelcast/internal/serialization/impl/portable/ClassDefinitionImpl.java b/hazelcast/src/main/java/com/hazelcast/internal/serialization/impl/portable/ClassDefinitionImpl.java index 7cd16ae1c0b9..75a3db002c3f 100644 --- a/hazelcast/src/main/java/com/hazelcast/internal/serialization/impl/portable/ClassDefinitionImpl.java +++ b/hazelcast/src/main/java/com/hazelcast/internal/serialization/impl/portable/ClassDefinitionImpl.java @@ -23,6 +23,7 @@ import com.hazelcast.nio.serialization.DataSerializable; import com.hazelcast.nio.serialization.FieldDefinition; import com.hazelcast.nio.serialization.FieldType; +import com.hazelcast.nio.serialization.PortableId; import java.io.IOException; import java.util.Collections; @@ -34,21 +35,15 @@ @SerializableByConvention(PUBLIC_API) public class ClassDefinitionImpl implements ClassDefinition, DataSerializable { - - private int factoryId; - private int classId; - private int version; + private PortableId portableId; private Map fieldDefinitionsMap; @SuppressWarnings("unused") - private ClassDefinitionImpl() { - } + private ClassDefinitionImpl() { } - public ClassDefinitionImpl(int factoryId, int classId, int version) { - this.factoryId = factoryId; - this.classId = classId; - this.version = version; - this.fieldDefinitionsMap = new LinkedHashMap<>(); + public ClassDefinitionImpl(PortableId portableId) { + this.portableId = portableId; + fieldDefinitionsMap = new LinkedHashMap<>(); } public void addFieldDef(FieldDefinitionImpl fd) { @@ -108,38 +103,38 @@ public int getFieldCount() { @Override public final int getFactoryId() { - return factoryId; + return portableId.getFactoryId(); } @Override public final int getClassId() { - return classId; + return portableId.getClassId(); } @Override public final int getVersion() { - return version; + return portableId.getVersion(); + } + + @Override + public PortableId getPortableId() { + return portableId; } void setVersionIfNotSet(int version) { - if (getVersion() < 0) { - this.version = version; - } + portableId.setVersionIfNotSet(version); } @Override public void writeData(ObjectDataOutput out) throws IOException { - out.writeInt(factoryId); - out.writeInt(classId); - out.writeInt(version); + portableId.writeData(out); out.writeObject(fieldDefinitionsMap); } @Override public void readData(ObjectDataInput in) throws IOException { - factoryId = in.readInt(); - classId = in.readInt(); - version = in.readInt(); + portableId = new PortableId(); + portableId.readData(in); fieldDefinitionsMap = in.readObject(); } @@ -152,31 +147,21 @@ public boolean equals(Object o) { return false; } ClassDefinitionImpl that = (ClassDefinitionImpl) o; - if (factoryId != that.factoryId) { - return false; - } - if (classId != that.classId) { - return false; - } - if (version != that.version) { - return false; - } - return fieldDefinitionsMap.equals(that.fieldDefinitionsMap); + return portableId.equals(that.portableId) + && fieldDefinitionsMap.equals(that.fieldDefinitionsMap); } @Override public int hashCode() { - int result = classId; - result = 31 * result + version; - return result; + return portableId.hashCode(); } @Override public String toString() { return "ClassDefinition{" - + "factoryId=" + factoryId - + ", classId=" + classId - + ", version=" + version + + "factoryId=" + portableId.getFactoryId() + + ", classId=" + portableId.getClassId() + + ", version=" + portableId.getVersion() + ", fieldDefinitions=" + fieldDefinitionsMap.values() + '}'; } diff --git a/hazelcast/src/main/java/com/hazelcast/internal/serialization/impl/portable/FieldDefinitionImpl.java b/hazelcast/src/main/java/com/hazelcast/internal/serialization/impl/portable/FieldDefinitionImpl.java index 0324225128d8..cadf1c4f3bd9 100644 --- a/hazelcast/src/main/java/com/hazelcast/internal/serialization/impl/portable/FieldDefinitionImpl.java +++ b/hazelcast/src/main/java/com/hazelcast/internal/serialization/impl/portable/FieldDefinitionImpl.java @@ -22,36 +22,36 @@ import com.hazelcast.nio.serialization.DataSerializable; import com.hazelcast.nio.serialization.FieldDefinition; import com.hazelcast.nio.serialization.FieldType; +import com.hazelcast.nio.serialization.PortableId; import java.io.IOException; +import java.util.Objects; import static com.hazelcast.internal.serialization.SerializableByConvention.Reason.PUBLIC_API; @SerializableByConvention(PUBLIC_API) public class FieldDefinitionImpl implements FieldDefinition, DataSerializable { - private int index; private String fieldName; private FieldType type; - private int factoryId; - private int classId; - private int version; + private PortableId portableId; @SuppressWarnings("unused") - private FieldDefinitionImpl() { - } + private FieldDefinitionImpl() { } public FieldDefinitionImpl(int index, String fieldName, FieldType type, int version) { this(index, fieldName, type, 0, 0, version); } public FieldDefinitionImpl(int index, String fieldName, FieldType type, int factoryId, int classId, int version) { - this.type = type; - this.fieldName = fieldName; + this(index, fieldName, type, new PortableId(factoryId, classId, version)); + } + + public FieldDefinitionImpl(int index, String fieldName, FieldType type, PortableId portableId) { this.index = index; - this.factoryId = factoryId; - this.classId = classId; - this.version = version; + this.fieldName = fieldName; + this.type = type; + this.portableId = portableId; } @Override @@ -71,17 +71,17 @@ public int getIndex() { @Override public int getFactoryId() { - return factoryId; + return portableId.getFactoryId(); } @Override public int getClassId() { - return classId; + return portableId.getClassId(); } @Override public int getVersion() { - return version; + return portableId.getVersion(); } @Override @@ -89,9 +89,7 @@ public void writeData(ObjectDataOutput out) throws IOException { out.writeInt(index); out.writeString(fieldName); out.writeByte(type.getId()); - out.writeInt(factoryId); - out.writeInt(classId); - out.writeInt(version); + portableId.writeData(out); } @Override @@ -99,9 +97,8 @@ public void readData(ObjectDataInput in) throws IOException { index = in.readInt(); fieldName = in.readString(); type = FieldType.get(in.readByte()); - factoryId = in.readInt(); - classId = in.readInt(); - version = in.readInt(); + portableId = new PortableId(); + portableId.readData(in); } @Override @@ -113,35 +110,16 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) { return false; } - FieldDefinitionImpl that = (FieldDefinitionImpl) o; - if (index != that.index) { - return false; - } - if (factoryId != that.factoryId) { - return false; - } - if (classId != that.classId) { - return false; - } - if (version != that.version) { - return false; - } - if (fieldName != null ? !fieldName.equals(that.fieldName) : that.fieldName != null) { - return false; - } - return type == that.type; + return index == that.index + && Objects.equals(fieldName, that.fieldName) + && type == that.type + && portableId.equals(that.portableId); } @Override public int hashCode() { - int result = index; - result = 31 * result + (fieldName != null ? fieldName.hashCode() : 0); - result = 31 * result + (type != null ? type.hashCode() : 0); - result = 31 * result + factoryId; - result = 31 * result + classId; - result = 31 * result + version; - return result; + return Objects.hash(index, fieldName, type, portableId); } @Override @@ -150,9 +128,9 @@ public String toString() { + "index=" + index + ", fieldName='" + fieldName + '\'' + ", type=" + type - + ", factoryId=" + factoryId - + ", classId=" + classId - + ", version=" + version + + ", factoryId=" + portableId.getFactoryId() + + ", classId=" + portableId.getClassId() + + ", version=" + portableId.getVersion() + '}'; } } diff --git a/hazelcast/src/main/java/com/hazelcast/internal/serialization/impl/portable/PortableContext.java b/hazelcast/src/main/java/com/hazelcast/internal/serialization/impl/portable/PortableContext.java index 46912aaebb14..c7e81355a066 100644 --- a/hazelcast/src/main/java/com/hazelcast/internal/serialization/impl/portable/PortableContext.java +++ b/hazelcast/src/main/java/com/hazelcast/internal/serialization/impl/portable/PortableContext.java @@ -21,6 +21,7 @@ import com.hazelcast.nio.serialization.ClassDefinition; import com.hazelcast.nio.serialization.FieldDefinition; import com.hazelcast.nio.serialization.Portable; +import com.hazelcast.nio.serialization.PortableId; import java.io.IOException; import java.nio.ByteOrder; @@ -35,6 +36,8 @@ public interface PortableContext { ClassDefinition lookupClassDefinition(int factoryId, int classId, int version); + ClassDefinition lookupClassDefinition(PortableId portableId); + ClassDefinition lookupClassDefinition(Data data) throws IOException; ClassDefinition registerClassDefinition(ClassDefinition cd); diff --git a/hazelcast/src/main/java/com/hazelcast/internal/serialization/impl/portable/PortableContextImpl.java b/hazelcast/src/main/java/com/hazelcast/internal/serialization/impl/portable/PortableContextImpl.java index 0842b5a23169..b816f55fa4e1 100644 --- a/hazelcast/src/main/java/com/hazelcast/internal/serialization/impl/portable/PortableContextImpl.java +++ b/hazelcast/src/main/java/com/hazelcast/internal/serialization/impl/portable/PortableContextImpl.java @@ -29,6 +29,7 @@ import com.hazelcast.nio.serialization.FieldType; import com.hazelcast.nio.serialization.HazelcastSerializationException; import com.hazelcast.nio.serialization.Portable; +import com.hazelcast.nio.serialization.PortableId; import java.io.IOException; import java.nio.ByteOrder; @@ -69,6 +70,11 @@ public ClassDefinition lookupClassDefinition(int factoryId, int classId, int ver return getClassDefContext(factoryId).lookup(classId, version); } + @Override + public ClassDefinition lookupClassDefinition(PortableId portableId) { + return lookupClassDefinition(portableId.getFactoryId(), portableId.getClassId(), portableId.getVersion()); + } + @Override public ClassDefinition lookupClassDefinition(Data data) throws IOException { if (!data.isPortable()) { @@ -197,8 +203,7 @@ public FieldDefinition getFieldDefinition(ClassDefinition classDef, String name) if (fd == null) { throw new IllegalArgumentException("Unknown field: " + name); } - currentClassDef = lookupClassDefinition(fd.getFactoryId(), fd.getClassId(), - fd.getVersion()); + currentClassDef = lookupClassDefinition(fd.getPortableId()); if (currentClassDef == null) { throw new IllegalArgumentException("Not a registered Portable field: " + fd); } diff --git a/hazelcast/src/main/java/com/hazelcast/internal/util/collection/DefaultedMap.java b/hazelcast/src/main/java/com/hazelcast/internal/util/collection/DefaultedMap.java index 14fe43129434..c4dbff9596d6 100644 --- a/hazelcast/src/main/java/com/hazelcast/internal/util/collection/DefaultedMap.java +++ b/hazelcast/src/main/java/com/hazelcast/internal/util/collection/DefaultedMap.java @@ -24,38 +24,82 @@ import java.util.function.BiFunction; import java.util.function.Function; +import static com.hazelcast.internal.util.ExceptionUtil.rethrow; import static java.util.stream.Collectors.toMap; /** - * A map decorator that overrides {@link #get(Object)} to return a default value - * if the delegated map does not contain the specified key. It differs from - * {@link org.apache.commons.collections.map.DefaultedMap} by
    + * A map decorator that introduces {@link #getOrDefault(Object)} to return a + * default value if the delegated map does not contain the specified key. It + * differs from {@link org.apache.commons.collections.map.DefaultedMap} by
      *
    1. forwarding operations that have a default implementation in {@link Map} * to the delegated map, which can increase performance if the delegated * map has a custom implementation, and *
    2. using {@link #getOrDefault(K, V)} to implement {@link #getOrDefault(K)}. + * + * @see + * DefaultedMap vs. switch */ public class DefaultedMap implements Map { private final Map map; - private final V defaultValue; + private final V missingValuePlaceholder; + private final Function missingValueComputer; - public DefaultedMap(Map map, V defaultValue) { + private DefaultedMap(Map map, V missingValuePlaceholder, Function missingValueComputer) { this.map = map; - this.defaultValue = defaultValue; + this.missingValuePlaceholder = missingValuePlaceholder; + this.missingValueComputer = missingValueComputer; } - public V getOrDefault(Object key) { - return map.getOrDefault(key, defaultValue); + public DefaultedMap(Map map, V defaultValue) { + this(map, defaultValue, null); } - public V getDefaultValue() { - return defaultValue; + public V getOrDefault(Object key) { + V value = map.getOrDefault(key, missingValuePlaceholder); + if (value == missingValuePlaceholder && missingValueComputer != null) { + return missingValueComputer.apply((K) key); + } + return value; } public DefaultedMap mapKeysAndValues(Function keyMapper, Function valueMapper) { + assert missingValueComputer == null; return new DefaultedMap<>(map.entrySet().stream() .collect(toMap(e -> keyMapper.apply(e.getKey()), e -> valueMapper.apply(e.getValue()))), - valueMapper.apply(defaultValue)); + valueMapper.apply(missingValuePlaceholder)); + } + + public static class DefaultedMapBuilder { + private final Map map; + private V missingValuePlaceholder; + + public DefaultedMapBuilder(Map map) { + this.map = map; + } + + /** + * The missing value placeholder is passed to {@link #getOrDefault(Object, Object)} to + * avoid querying the map twice. It is null by default and needs to be changed if the + * map contains null values. + */ + public DefaultedMapBuilder missingValuePlaceholder(V missingValuePlaceholder) { + this.missingValuePlaceholder = missingValuePlaceholder; + return this; + } + + public DefaultedMap orElse(V defaultValue) { + return new DefaultedMap<>(map, defaultValue); + } + + public DefaultedMap orElseGet(Function missingValueComputer) { + return new DefaultedMap<>(map, missingValuePlaceholder, missingValueComputer); + } + + public DefaultedMap orElseThrow(Function keyNotFoundException) { + return new DefaultedMap<>(map, missingValuePlaceholder, key -> { + throw rethrow(keyNotFoundException.apply(key)); + }); + } } @Override diff --git a/hazelcast/src/main/java/com/hazelcast/jet/impl/util/ExceptionUtil.java b/hazelcast/src/main/java/com/hazelcast/jet/impl/util/ExceptionUtil.java index 33d96cea317c..7c02af91add6 100644 --- a/hazelcast/src/main/java/com/hazelcast/jet/impl/util/ExceptionUtil.java +++ b/hazelcast/src/main/java/com/hazelcast/jet/impl/util/ExceptionUtil.java @@ -49,9 +49,6 @@ import java.util.List; import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutionException; -import java.util.function.BiConsumer; -import java.util.function.Supplier; -import java.util.stream.Stream; import static com.hazelcast.client.impl.protocol.ClientProtocolErrorCodes.JET_EXCEPTIONS_RANGE_START; import static com.hazelcast.jet.datamodel.Tuple3.tuple3; @@ -149,14 +146,6 @@ public static RuntimeException rethrow(@Nonnull final Throwable t) { throw peeledAndUnchecked(t); } - /** - * A {@linkplain Stream#collect(Supplier, BiConsumer, BiConsumer) combiner} that throws an - * {@link UnsupportedOperationException}. It is useful when parallel stream is not supported. - */ - public static void combinerUnsupported(T ignored, T ignored2) { - throw new UnsupportedOperationException("parallelStream() is not supported"); - } - @Nonnull public static String stackTraceToString(Throwable t) { StringWriter sw = new StringWriter(); diff --git a/hazelcast/src/main/java/com/hazelcast/jet/impl/util/Util.java b/hazelcast/src/main/java/com/hazelcast/jet/impl/util/Util.java index ecd153982cf1..ce9b6a7f181a 100644 --- a/hazelcast/src/main/java/com/hazelcast/jet/impl/util/Util.java +++ b/hazelcast/src/main/java/com/hazelcast/jet/impl/util/Util.java @@ -75,6 +75,9 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLongArray; +import java.util.function.BiConsumer; +import java.util.function.BiFunction; +import java.util.function.BinaryOperator; import java.util.function.Function; import java.util.function.Predicate; import java.util.function.Supplier; @@ -381,6 +384,33 @@ public static int arrayIndexOf(int needle, int[] haystack) { return -1; } + /** + * An alternative to {@link Stream#reduce(Object, BiFunction, BinaryOperator) + * Stream.reduce(identity, accumulator, combiner)}, which is not parallelizable. + * It eliminates the need for a combiner by processing elements in the order + * they appear in the stream. + */ + public static T reduce(T identity, Stream elements, BiFunction accumulator) { + T result = identity; + for (R element : (Iterable) elements::iterator) { + result = accumulator.apply(result, element); + } + return result; + } + + /** + * An alternative to {@link Stream#collect(Supplier, BiConsumer, BiConsumer) + * Stream.collect(supplier, accumulator, combiner)}, which is not parallelizable. + * It eliminates the need for a combiner by processing elements in the order + * they appear in the stream. + */ + public static T collect(T container, Stream elements, BiConsumer accumulator) { + for (R element : (Iterable) elements::iterator) { + accumulator.accept(container, element); + } + return container; + } + /** * Returns a future which is already completed with the supplied exception. */ diff --git a/hazelcast/src/main/java/com/hazelcast/nio/serialization/ClassDefinition.java b/hazelcast/src/main/java/com/hazelcast/nio/serialization/ClassDefinition.java index 877112ae7145..a44dac90cae2 100644 --- a/hazelcast/src/main/java/com/hazelcast/nio/serialization/ClassDefinition.java +++ b/hazelcast/src/main/java/com/hazelcast/nio/serialization/ClassDefinition.java @@ -22,7 +22,7 @@ * Defines a class schema for {@link Portable} classes. It allows querying of field names, types, class IDs etc. * It can be created manually using {@link ClassDefinitionBuilder} or on demand during deserialization. * - * @see com.hazelcast.nio.serialization.Portable + * @see Portable * @see ClassDefinitionBuilder */ public interface ClassDefinition { @@ -104,4 +104,12 @@ public interface ClassDefinition { * @return version */ int getVersion(); + + /** + * @return portable ID of this class + * @since 5.4 + */ + default PortableId getPortableId() { + return new PortableId(getFactoryId(), getClassId(), getVersion()); + } } diff --git a/hazelcast/src/main/java/com/hazelcast/nio/serialization/ClassDefinitionBuilder.java b/hazelcast/src/main/java/com/hazelcast/nio/serialization/ClassDefinitionBuilder.java index e56132af73ef..3e9f9ed5f8ba 100644 --- a/hazelcast/src/main/java/com/hazelcast/nio/serialization/ClassDefinitionBuilder.java +++ b/hazelcast/src/main/java/com/hazelcast/nio/serialization/ClassDefinitionBuilder.java @@ -16,6 +16,7 @@ package com.hazelcast.nio.serialization; +import com.hazelcast.config.SerializationConfig; import com.hazelcast.internal.serialization.impl.portable.ClassDefinitionImpl; import com.hazelcast.internal.serialization.impl.portable.FieldDefinitionImpl; import com.hazelcast.spi.annotation.PrivateApi; @@ -30,15 +31,12 @@ * ClassDefinitionBuilder is used to build and register ClassDefinitions manually. * * @see ClassDefinition - * @see com.hazelcast.nio.serialization.Portable - * @see com.hazelcast.config.SerializationConfig#addClassDefinition(ClassDefinition) + * @see Portable + * @see SerializationConfig#addClassDefinition(ClassDefinition) */ public final class ClassDefinitionBuilder { - - private final int factoryId; - private final int classId; - private final int version; - private final List fieldDefinitions = new ArrayList(); + private final PortableId portableId; + private final List fieldDefinitions = new ArrayList<>(); private final Set addedFieldNames = new HashSet<>(); private int index; private boolean done; @@ -47,27 +45,25 @@ public final class ClassDefinitionBuilder { * IMPORTANT: It uses a default portableVersion (0) for non-versioned classes. * Make sure to specify the portableVersion in the constructor if you override the default portableVersion * in the SerializationService - * - * @param factoryId factoryId to use - * @param classId classId to use */ public ClassDefinitionBuilder(int factoryId, int classId) { - this.factoryId = factoryId; - this.classId = classId; - this.version = 0; + this(factoryId, classId, 0); } /** * IMPORTANT: Make sure that the version matches the portableVersion in the SerializationService - * - * @param factoryId factoryId to use - * @param classId classId to use - * @param version portableVersion to use */ public ClassDefinitionBuilder(int factoryId, int classId, int version) { - this.factoryId = factoryId; - this.classId = classId; - this.version = version; + this(new PortableId(factoryId, classId, version)); + } + + /** + * IMPORTANT: Make sure that the version matches the portableVersion in the SerializationService + * + * @since 5.4 + */ + public ClassDefinitionBuilder(PortableId portableId) { + this.portableId = portableId; } /** @@ -369,19 +365,19 @@ public ClassDefinitionBuilder addStringArrayField(@Nonnull String fieldName) { * if this method is called after {@link ClassDefinitionBuilder#build()} */ @Nonnull - public ClassDefinitionBuilder addPortableField(@Nonnull String fieldName, ClassDefinition def) { - if (def.getClassId() == 0) { + public ClassDefinitionBuilder addPortableField(@Nonnull String fieldName, ClassDefinition classDefinition) { + if (classDefinition.getClassId() == 0) { throw new IllegalArgumentException("Portable class ID cannot be zero!"); } check(fieldName); - fieldDefinitions.add(new FieldDefinitionImpl(index++, fieldName, - FieldType.PORTABLE, def.getFactoryId(), def.getClassId(), def.getVersion())); + fieldDefinitions.add(new FieldDefinitionImpl(index++, fieldName, FieldType.PORTABLE, + classDefinition.getPortableId())); return this; } /** - * @param fieldName name of the field that will be add to this class definition - * @param classDefinition class definition of the nested portable that will be add to this class definition + * @param fieldName name of the field that will be added to this class definition + * @param classDefinition class definition of the nested portable that will be added to this class definition * @return itself for chaining * @throws HazelcastSerializationException if a field with same name already exists or * if this method is called after {@link ClassDefinitionBuilder#build()} @@ -393,7 +389,7 @@ public ClassDefinitionBuilder addPortableArrayField(@Nonnull String fieldName, C } check(fieldName); fieldDefinitions.add(new FieldDefinitionImpl(index++, fieldName, FieldType.PORTABLE_ARRAY, - classDefinition.getFactoryId(), classDefinition.getClassId(), classDefinition.getVersion())); + classDefinition.getPortableId())); return this; } @@ -469,7 +465,7 @@ public ClassDefinitionBuilder addTimestampWithTimezoneArrayField(@Nonnull String private ClassDefinitionBuilder addField(@Nonnull String fieldName, FieldType fieldType) { check(fieldName); - fieldDefinitions.add(new FieldDefinitionImpl(index++, fieldName, fieldType, version)); + fieldDefinitions.add(new FieldDefinitionImpl(index++, fieldName, fieldType, portableId.getVersion())); return this; } @@ -489,7 +485,7 @@ public void addField(FieldDefinitionImpl fieldDefinition) { @Nonnull public ClassDefinition build() { done = true; - final ClassDefinitionImpl cd = new ClassDefinitionImpl(factoryId, classId, version); + final ClassDefinitionImpl cd = new ClassDefinitionImpl(portableId); for (FieldDefinitionImpl fd : fieldDefinitions) { cd.addFieldDef(fd); } @@ -501,19 +497,24 @@ private void check(@Nonnull String fieldName) { throw new HazelcastSerializationException("Field with field name : " + fieldName + " already exists"); } if (done) { - throw new HazelcastSerializationException("ClassDefinition is already built for " + classId); + throw new HazelcastSerializationException("ClassDefinition is already built for " + portableId.getClassId()); } } public int getFactoryId() { - return factoryId; + return portableId.getFactoryId(); } public int getClassId() { - return classId; + return portableId.getClassId(); } public int getVersion() { - return version; + return portableId.getVersion(); + } + + /** @since 5.4 */ + public PortableId getPortableId() { + return portableId; } } diff --git a/hazelcast/src/main/java/com/hazelcast/nio/serialization/FieldDefinition.java b/hazelcast/src/main/java/com/hazelcast/nio/serialization/FieldDefinition.java index b0085a7167a8..e5a91e27f903 100644 --- a/hazelcast/src/main/java/com/hazelcast/nio/serialization/FieldDefinition.java +++ b/hazelcast/src/main/java/com/hazelcast/nio/serialization/FieldDefinition.java @@ -37,17 +37,25 @@ public interface FieldDefinition { int getIndex(); /** - * @return class ID of this field's class + * @return factory ID of this field's class */ - int getClassId(); + int getFactoryId(); /** - * @return factory ID of this field's class + * @return class ID of this field's class */ - int getFactoryId(); + int getClassId(); /** * @return version of this field's class */ int getVersion(); + + /** + * @return portable ID of this field's class + * @since 5.4 + */ + default PortableId getPortableId() { + return new PortableId(getFactoryId(), getClassId(), getVersion()); + } } diff --git a/hazelcast/src/main/java/com/hazelcast/nio/serialization/PortableId.java b/hazelcast/src/main/java/com/hazelcast/nio/serialization/PortableId.java new file mode 100644 index 000000000000..3df69b2bbffe --- /dev/null +++ b/hazelcast/src/main/java/com/hazelcast/nio/serialization/PortableId.java @@ -0,0 +1,118 @@ +/* + * Copyright (c) 2008-2023, Hazelcast, Inc. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.hazelcast.nio.serialization; + +import com.hazelcast.internal.serialization.SerializableByConvention; +import com.hazelcast.nio.ObjectDataInput; +import com.hazelcast.nio.ObjectDataOutput; +import com.hazelcast.spi.annotation.PrivateApi; + +import java.io.IOException; +import java.util.Objects; + +import static com.hazelcast.internal.serialization.SerializableByConvention.Reason.PUBLIC_API; + +/** + * Uniquely defines a {@link Portable} class. + * + * @see ClassDefinition#getPortableId() + * @see FieldDefinition#getPortableId() + * @see ClassDefinitionBuilder#ClassDefinitionBuilder(PortableId) new ClassDefinitionBuilder(PortableId) + * + * @since 5.4 + */ +@SerializableByConvention(PUBLIC_API) +public class PortableId implements DataSerializable { + private int factoryId; + private int classId; + private int version; + + @PrivateApi + public PortableId() { } + + public PortableId(int factoryId, int classId, int version) { + this.factoryId = factoryId; + this.classId = classId; + this.version = version; + } + + public PortableId(String portableId) { + String[] components = portableId.split(":"); + assert components.length == 3 : "Number of Portable ID components should always be 3"; + + this.factoryId = Integer.parseInt(components[0]); + this.classId = Integer.parseInt(components[1]); + this.version = Integer.parseInt(components[2]); + } + + public int getFactoryId() { + return factoryId; + } + + public int getClassId() { + return classId; + } + + public int getVersion() { + return version; + } + + @PrivateApi + public void setVersionIfNotSet(int version) { + if (this.version < 0) { + this.version = version; + } + } + + @Override + public String toString() { + return factoryId + ":" + classId + ":" + version; + } + + @Override + public void writeData(ObjectDataOutput out) throws IOException { + out.writeInt(factoryId); + out.writeInt(classId); + out.writeInt(version); + } + + @Override + public void readData(ObjectDataInput in) throws IOException { + factoryId = in.readInt(); + classId = in.readInt(); + version = in.readInt(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PortableId that = (PortableId) o; + return factoryId == that.factoryId + && classId == that.classId + && version == that.version; + } + + @Override + public int hashCode() { + return Objects.hash(factoryId, classId, version); + } +} diff --git a/hazelcast/src/test/java/com/hazelcast/test/HazelcastTestSupport.java b/hazelcast/src/test/java/com/hazelcast/test/HazelcastTestSupport.java index ab3abafcec62..914ee0440a7f 100644 --- a/hazelcast/src/test/java/com/hazelcast/test/HazelcastTestSupport.java +++ b/hazelcast/src/test/java/com/hazelcast/test/HazelcastTestSupport.java @@ -16,6 +16,7 @@ package com.hazelcast.test; +import com.google.common.collect.Lists; import com.hazelcast.client.HazelcastClient; import com.hazelcast.cluster.Address; import com.hazelcast.cluster.Cluster; @@ -97,6 +98,7 @@ import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.NANOSECONDS; import static java.util.concurrent.TimeUnit.SECONDS; +import static java.util.stream.Collectors.toList; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -110,20 +112,22 @@ /** * Base class for Hazelcast tests which provides a big number of convenient test methods. *

      - * Has built-in support for {@link TestHazelcastInstanceFactory}, {@link JitterRule} and {@link DumpBuildInfoOnFailureRule}. + * Has built-in support for {@link TestHazelcastInstanceFactory}, {@link JitterRule} and + * {@link DumpBuildInfoOnFailureRule}. *

      - * Tests which are be extended in Hazelcast Enterprise, should use {@link #getConfig()} instead of {@code new Config()}, - * so the Enterprise test can override this method to return a config with a {@link com.hazelcast.config.NativeMemoryConfig}. + * Tests which are be extended in Hazelcast Enterprise, should use {@link #getConfig()} + * instead of {@code new Config()}, so the Enterprise test can override this method to + * return a config with a {@link com.hazelcast.config.NativeMemoryConfig}. */ public abstract class HazelcastTestSupport { - public static final String JAVA_VENDOR = System.getProperty("java.vendor"); public static final int ASSERT_TRUE_EVENTUALLY_TIMEOUT; public static final int ASSERT_COMPLETES_STALL_TOLERANCE; public static final String PERSISTENT_MEMORY_DIRECTORIES; - private static final String COMPAT_HZ_INSTANCE_FACTORY = "com.hazelcast.test.CompatibilityTestHazelcastInstanceFactory"; + private static final String COMPAT_HZ_INSTANCE_FACTORY = + "com.hazelcast.test.CompatibilityTestHazelcastInstanceFactory"; private static final boolean EXPECT_DIFFERENT_HASHCODES = (new Object().hashCode() != new Object().hashCode()); private static final ILogger LOGGER = Logger.getLogger(HazelcastTestSupport.class); @@ -152,31 +156,6 @@ public abstract class HazelcastTestSupport { ClusterProperty.METRICS_DEBUG.setSystemProperty("true"); } - protected static boolean containsIn(T item1, Collection collection, Comparator comparator) { - for (T item2 : collection) { - if (comparator.compare(item1, item2) == 0) { - return true; - } - } - return false; - } - - protected static void assertCollection(Collection expected, Collection actual) { - assertEquals(String.format("Expected collection: `%s`, actual collection: `%s`", expected, actual), - expected.size(), actual.size()); - assertContainsAll(expected, actual); - } - - protected static void assertCollection(Collection expected, Collection actual, Comparator comparator) { - assertEquals(String.format("Expected collection: `%s`, actual collection: `%s`", expected, actual), - expected.size(), actual.size()); - for (T item : expected) { - if (!containsIn(item, actual, comparator)) { - throw new AssertionError("Actual collection does not contain the item " + item); - } - } - } - @After public final void shutdownNodeFactory() { LOGGER.info("Shutting down node factory as @After action"); @@ -362,6 +341,22 @@ public static void interruptCurrentThread(final int delayMillis) { }).start(); } + protected static boolean containsIn(T item1, Collection collection, Comparator comparator) { + for (T item2 : collection) { + if (comparator.compare(item1, item2) == 0) { + return true; + } + } + return false; + } + + /** + * Returns the cartesian product of the specified lists. + */ + public static List parameters(List... lists) { + return Lists.cartesianProduct(lists).stream().map(List::toArray).collect(toList()); + } + // ########################### // ########## sleep ########## // ########################### @@ -413,7 +408,8 @@ public static void sleepAtMostSeconds(long referenceTime, int seconds) { } } else { long absSleepTime = Math.abs(sleepTime); - LOGGER.warning("There is no time left to sleep. We are beyond the desired end of sleep by " + absSleepTime + "ms"); + LOGGER.warning("There is no time left to sleep. We are beyond the desired end of sleep by " + + absSleepTime + "ms"); } } @@ -422,8 +418,9 @@ public static void sleepAtMostSeconds(long referenceTime, int seconds) { *

      * If stop is changed to true while sleeping, the calls returns before waiting the full sleeping period. *

      - * This method is very useful for stress tests that run for a certain amount of time. But if one of the stress tests - * runs into a failure, the test should be aborted immediately. This is done by letting the thread set stop to true. + * This method is very useful for stress tests that run for a certain amount of time. But if one of the + * stress tests runs into a failure, the test should be aborted immediately. This is done by letting the + * thread set stop to true. * * @param stop an {@link AtomicBoolean} to stop the sleep method * @param durationSeconds sleep duration in seconds @@ -681,7 +678,8 @@ private static void checkMemberCount(boolean generateOwnedKey, Cluster cluster) } Set members = cluster.getMembers(); if (members.size() < 2) { - throw new UnsupportedOperationException("Cluster has only one member, you can not generate a `not owned key`"); + throw new UnsupportedOperationException( + "Cluster has only one member, you can not generate a `not owned key`"); } } @@ -884,19 +882,22 @@ private static String formatAssertMessage(Object expected, Object actual) { assertMessage.append(" but was: "); formatClassAndValue(assertMessage, actual, actualString); } else { - assertMessage.append("expected: <").append(expectedString).append("> but was: <").append(actualString).append(">"); + assertMessage.append("expected: <").append(expectedString) + .append("> but was: <").append(actualString).append(">"); } return assertMessage.toString(); } private static void formatClassAndValue(StringBuilder message, Object value, String valueString) { - message.append((value == null) ? "null" : value.getClass().getName()).append("<").append(valueString).append(">"); + message.append((value == null) ? "null" : value.getClass().getName()) + .append("<").append(valueString).append(">"); } @SuppressWarnings("unchecked") public static E assertInstanceOf(Class expected, Object actual) { assertNotNull(actual); - assertTrue(actual + " is not an instanceof " + expected.getName(), expected.isAssignableFrom(actual.getClass())); + assertTrue(actual + " is not an instanceof " + expected.getName(), + expected.isAssignableFrom(actual.getClass())); return (E) actual; } @@ -926,6 +927,22 @@ public static void assertJoinable(long timeoutSeconds, Thread... threads) { } } + protected static void assertCollection(Collection expected, Collection actual) { + assertEquals(String.format("Expected collection: `%s`, actual collection: `%s`", expected, actual), + expected.size(), actual.size()); + assertContainsAll(expected, actual); + } + + protected static void assertCollection(Collection expected, Collection actual, Comparator comparator) { + assertEquals(String.format("Expected collection: `%s`, actual collection: `%s`", expected, actual), + expected.size(), actual.size()); + for (T item : expected) { + if (!containsIn(item, actual, comparator)) { + throw new AssertionError("Actual collection does not contain the item " + item); + } + } + } + public static void assertIterableEquals(Iterable actual, Object... expected) { List actualList = new ArrayList<>(); for (Object object : actual) { @@ -950,16 +967,18 @@ public static void assertSizeEventually(int expectedSize, Collection collecti assertSizeEventually(expectedSize, collection, ASSERT_TRUE_EVENTUALLY_TIMEOUT); } - public static void assertSizeEventually(final int expectedSize, final Collection collection, long timeoutSeconds) { - assertTrueEventually(() -> assertEquals("the size of the collection is not correct: found-content:" + collection, expectedSize, - collection.size()), timeoutSeconds); + public static void assertSizeEventually(final int expectedSize, final Collection collection, + long timeoutSeconds) { + assertTrueEventually(() -> assertEquals("the size of the collection is not correct: found-content:" + + collection, expectedSize, collection.size()), timeoutSeconds); } public static void assertSizeEventually(int expectedSize, Supplier> collectionSupplier) { assertSizeEventually(expectedSize, collectionSupplier, ASSERT_TRUE_EVENTUALLY_TIMEOUT); } - public static void assertSizeEventually(int expectedSize, Supplier> collectionSupplier, long timeoutSeconds) { + public static void assertSizeEventually(int expectedSize, Supplier> collectionSupplier, + long timeoutSeconds) { assertTrueEventually(() -> assertEquals("the size of the collection is not correct: found-content:" + collectionSupplier.get(), expectedSize, collectionSupplier.get().size()), timeoutSeconds); @@ -970,7 +989,8 @@ public static void assertSizeEventually(int expectedSize, Map map) { } public static void assertSizeEventually(final int expectedSize, final Map map, long timeoutSeconds) { - assertTrueEventually(() -> assertEquals("the size of the map is not correct", expectedSize, map.size()), timeoutSeconds); + assertTrueEventually(() -> assertEquals("the size of the map is not correct", expectedSize, map.size()), + timeoutSeconds); } public static void assertEqualsEventually(final FutureTask task, final E expected) { @@ -999,8 +1019,7 @@ public static void assertClusterSize(int expectedSize, HazelcastInstance... inst } private static int getClusterSize(HazelcastInstance instance) { - Set members = instance.getCluster().getMembers(); - return members == null ? 0 : members.size(); + return instance.getCluster().getMembers().size(); } public static void assertClusterSizeEventually(int expectedSize, HazelcastInstance... instances) { @@ -1036,7 +1055,8 @@ public static void assertMasterAddress(Address masterAddress, HazelcastInstance. } } - public static void assertMasterAddressEventually(final Address masterAddress, final HazelcastInstance... instances) { + public static void assertMasterAddressEventually(final Address masterAddress, + final HazelcastInstance... instances) { assertTrueEventually(() -> { for (HazelcastInstance instance : instances) { assertMasterAddress(masterAddress, instance); @@ -1051,7 +1071,8 @@ public static void assertClusterState(ClusterState expectedState, HazelcastInsta } } - public static void assertClusterStateEventually(final ClusterState expectedState, final HazelcastInstance... instances) { + public static void assertClusterStateEventually(final ClusterState expectedState, + final HazelcastInstance... instances) { assertTrueEventually(() -> assertClusterState(expectedState, instances)); } @@ -1079,25 +1100,25 @@ public static void assertOpenEventually(String message, CountDownLatch latch, lo assertOpenEventually(message, new CountdownLatchAdapter(latch), timeoutSeconds); } - public static void assertOpenEventually(String message, Latch latch, long timeoutSeconds) { + private static void assertOpenEventually(String message, Latch latch, long timeoutSeconds) { try { boolean completed = latch.await(timeoutSeconds, SECONDS); if (message == null) { - assertTrue(format("CountDownLatch failed to complete within %d seconds, count left: %d", timeoutSeconds, - latch.getCount()), completed); + assertTrue(format("CountDownLatch failed to complete within %d seconds, count left: %d", + timeoutSeconds, latch.getCount()), completed); } else { - assertTrue(format("%s, failed to complete within %d seconds, count left: %d", message, timeoutSeconds, - latch.getCount()), completed); + assertTrue(format("%s, failed to complete within %d seconds, count left: %d", message, + timeoutSeconds, latch.getCount()), completed); } } catch (InterruptedException e) { throw new RuntimeException(e); } } - public static void assertCountEventually(final String message, final int expectedCount, final CountDownLatch latch, - long timeoutInSeconds) { + public static void assertCountEventually(final String message, final int expectedCount, + final CountDownLatch latch, long timeoutInSeconds) { assertTrueEventually(() -> { - for (int i = 0; i < 2; i++) { // recheck to see if hasn't changed + for (int i = 0; i < 2; i++) { // recheck to see if it hasn't changed if (latch.getCount() != expectedCount) { throw new AssertionError("Latch count has not been met. " + message); } @@ -1106,10 +1127,10 @@ public static void assertCountEventually(final String message, final int expecte }, timeoutInSeconds); } - public static void assertAtomicEventually(final String message, final int expectedValue, final AtomicInteger atomic, - int timeoutInSeconds) { + public static void assertAtomicEventually(final String message, final int expectedValue, + final AtomicInteger atomic, int timeoutInSeconds) { assertTrueEventually(() -> { - for (int i = 0; i < 2; i++) { // recheck to see if hasn't changed + for (int i = 0; i < 2; i++) { // recheck to see if it hasn't changed if (atomic.get() != expectedValue) { throw new AssertionError("Atomic value has not been met. " + message); } @@ -1118,10 +1139,10 @@ public static void assertAtomicEventually(final String message, final int expect }, timeoutInSeconds); } - public static void assertAtomicEventually(final String message, final boolean expectedValue, final AtomicBoolean atomic, - int timeoutInSeconds) { + public static void assertAtomicEventually(final String message, final boolean expectedValue, + final AtomicBoolean atomic, int timeoutInSeconds) { assertTrueEventually(() -> { - for (int i = 0; i < 2; i++) { // recheck to see if hasn't changed + for (int i = 0; i < 2; i++) { // recheck to see if it hasn't changed if (atomic.get() != expectedValue) { throw new AssertionError("Atomic value has not been met. " + message); } @@ -1141,7 +1162,7 @@ public static void assertTrueAllTheTime(AssertTask task, long durationSeconds) { } catch (Exception e) { throw rethrow(e); } - // Don't wait if there is not next iteration + // Don't wait if there is no next iteration if ((i + 1) <= durationSeconds) { sleepSeconds(1); } @@ -1240,8 +1261,8 @@ public static void assertCompletesEventually(String message, ProgressCheckerTask // if the task exceeded stall tolerance, we fail and log the history of the progress changes if (toleranceExceeded && !progressMade) { - StringBuilder sb = new StringBuilder("Stall tolerance " + stallToleranceSeconds + " seconds has been " - + "exceeded without completing the task. Track of progress:\n"); + StringBuilder sb = new StringBuilder("Stall tolerance " + stallToleranceSeconds + + " seconds has been exceeded without completing the task. Track of progress:\n"); for (TaskProgress historicProgress : progresses) { long elapsedMillis = historicProgress.timestamp() - taskStartTimestamp; String elapsedMillisPadded = String.format("%1$5s", elapsedMillis); @@ -1252,7 +1273,8 @@ public static void assertCompletesEventually(String message, ProgressCheckerTask } LOGGER.severe(sb.toString()); fail("Stall tolerance " + stallToleranceSeconds - + " seconds has been exceeded without completing the task. " + (message != null ? message : "")); + + " seconds has been exceeded without completing the task. " + + (message != null ? message : "")); } sleepMillis(sleepMillis); @@ -1267,7 +1289,6 @@ public static void assertTrueEventually(AssertTask task) { } public static void assertFalseEventually(AssertTask task, long timeoutSeconds) { - AssertionError error = null; // we are going to check five times a second int sleepMillis = 200; long iterations = timeoutSeconds * 5; @@ -1306,7 +1327,7 @@ public static void assertTrueDelayed(int delaySeconds, AssertTask task) { /** * This method executes the normal assertEquals with expected and actual values. - * In addition it formats the given string with those values to provide a good assert message. + * In addition, it formats the given string with those values to provide a good assert message. * * @param message assert message which is formatted with expected and actual values * @param expected expected value which is used for assert @@ -1318,7 +1339,7 @@ public static void assertEqualsStringFormat(String message, Object expected, Obj /** * This method executes the normal assertNotEquals with expected and actual values. - * In addition it formats the given string with those values to provide a good assert message. + * In addition, it formats the given string with those values to provide a good assert message. * * @param message assert message which is formatted with expected and actual values * @param expected expected value which is used for assert @@ -1332,12 +1353,14 @@ public static void assertNotEqualsStringFormat(String message, Object expected, * Assert that {@code actualValue >= lowerBound && actualValue <= upperBound}. */ public static void assertBetween(String label, long actualValue, long lowerBound, long upperBound) { - assertTrue(format("Expected '%s' to be between %d and %d, but was %d", label, lowerBound, upperBound, actualValue), + assertTrue(format("Expected '%s' to be between %d and %d, but was %d", + label, lowerBound, upperBound, actualValue), actualValue >= lowerBound && actualValue <= upperBound); } public static void assertGreaterOrEquals(String label, long actualValue, long lowerBound) { - assertTrue(format("Expected '%s' to be greater than or equal to %d, but was %d", label, lowerBound, actualValue), + assertTrue(format("Expected '%s' to be greater than or equal to %d, but was %d", + label, lowerBound, actualValue), actualValue >= lowerBound); } @@ -1399,7 +1422,8 @@ private static OperationParkerImpl getOperationParkingService(HazelcastInstance } public static void assertThatIsNotMultithreadedTest() { - assertFalse("Test cannot run with parallel runner", Thread.currentThread() instanceof MultithreadedTestRunnerThread); + assertFalse("Test cannot run with parallel runner", + Thread.currentThread() instanceof MultithreadedTestRunnerThread); } // ################################### @@ -1545,16 +1569,16 @@ public static void assumeNoArm64Architecture() { } /** - * Throws {@link AssumptionViolatedException} if two new Objects have the same hashCode (e.g. when running tests - * with static hashCode ({@code -XX:hashCode=2}). + * Throws {@link AssumptionViolatedException} if two new Objects have the same hashCode, + * e.g. when running tests with static hashCode ({@code -XX:hashCode=2}). */ public static void assumeDifferentHashCodes() { assumeTrue("Hash codes are equal for different objects", EXPECT_DIFFERENT_HASHCODES); } /** - * Throws {@link AssumptionViolatedException} if the given {@link InternalSerializationService} is not configured - * with the assumed {@link ByteOrder}. + * Throws {@link AssumptionViolatedException} if the given {@link InternalSerializationService} + * is not configured with the assumed {@link ByteOrder}. * * @param serializationService the {@link InternalSerializationService} to check * @param assumedByteOrder the assumed {@link ByteOrder} @@ -1569,7 +1593,7 @@ public static void assumeConfiguredByteOrder(InternalSerializationService serial /** * Walk the stack trace and execute the provided {@code BiConsumer} on each {@code StackTraceElement} * encountered while walking the stack trace. - * + *

      * The {@code BiConsumer} expects {@code StackTraceElement, List} arguments; any * result from the {@code BiConsumer} should be added to the {@code results} list which is * returned as the result of this method. diff --git a/hazelcast/src/test/java/com/hazelcast/test/MobyNamingRule.java b/hazelcast/src/test/java/com/hazelcast/test/MobyNamingRule.java index 1c2fadcb7f0f..501d34942349 100644 --- a/hazelcast/src/test/java/com/hazelcast/test/MobyNamingRule.java +++ b/hazelcast/src/test/java/com/hazelcast/test/MobyNamingRule.java @@ -21,7 +21,7 @@ import static com.hazelcast.instance.impl.MobyNames.MOBY_NAMING_PREFIX; -class MobyNamingRule extends TestWatcher { +public class MobyNamingRule extends TestWatcher { @Override protected void starting(Description description) { String className = description.getTestClass().getSimpleName();