diff --git a/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/metadata/RecordLayerSchemaTemplate.java b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/metadata/RecordLayerSchemaTemplate.java index f21fb0198c..fa519e650c 100644 --- a/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/metadata/RecordLayerSchemaTemplate.java +++ b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/metadata/RecordLayerSchemaTemplate.java @@ -102,6 +102,9 @@ public final class RecordLayerSchemaTemplate implements SchemaTemplate { private final boolean intermingleTables; + @Nonnull + private final Map auxiliaryTypes; + private RecordLayerSchemaTemplate(@Nonnull final String name, @Nonnull final Set tables, @Nonnull final Set invokedRoutines, @@ -109,7 +112,8 @@ private RecordLayerSchemaTemplate(@Nonnull final String name, int version, boolean enableLongRows, boolean storeRowVersions, - boolean intermingleTables) { + boolean intermingleTables, + @Nonnull final Map auxiliaryTypes) { this.name = name; this.tables = ImmutableSet.copyOf(tables); this.invokedRoutines = ImmutableSet.copyOf(invokedRoutines); @@ -118,6 +122,7 @@ private RecordLayerSchemaTemplate(@Nonnull final String name, this.enableLongRows = enableLongRows; this.storeRowVersions = storeRowVersions; this.intermingleTables = intermingleTables; + this.auxiliaryTypes = ImmutableMap.copyOf(auxiliaryTypes); this.metaDataSupplier = Suppliers.memoize(this::buildRecordMetadata); this.tableIndexMappingSupplier = Suppliers.memoize(this::computeTableIndexMapping); this.indexesSupplier = Suppliers.memoize(this::computeIndexes); @@ -133,7 +138,8 @@ private RecordLayerSchemaTemplate(@Nonnull final String name, boolean enableLongRows, boolean storeRowVersions, boolean intermingleTables, - @Nonnull final RecordMetaData cachedMetadata) { + @Nonnull final RecordMetaData cachedMetadata, + @Nonnull final Map auxiliaryTypes) { this.name = name; this.version = version; this.tables = ImmutableSet.copyOf(tables); @@ -142,6 +148,7 @@ private RecordLayerSchemaTemplate(@Nonnull final String name, this.enableLongRows = enableLongRows; this.storeRowVersions = storeRowVersions; this.intermingleTables = intermingleTables; + this.auxiliaryTypes = ImmutableMap.copyOf(auxiliaryTypes); this.metaDataSupplier = Suppliers.memoize(() -> cachedMetadata); this.tableIndexMappingSupplier = Suppliers.memoize(this::computeTableIndexMapping); this.indexesSupplier = Suppliers.memoize(this::computeIndexes); @@ -343,6 +350,21 @@ public Optional findViewByName(@Nonnull final String viewName) { return views.stream().filter(view -> view.getName().equals(viewName)).findFirst(); } + /** + * Retrieves an auxiliary type (struct, etc.) by looking up its name. + * + * @param typeName The name of the type. + * @return An {@link Optional} containing the {@link DataType.Named} if it is found, otherwise {@code Empty}. + */ + @Nonnull + public Optional findAuxiliaryType(@Nonnull final String typeName) { + // SQL is case-insensitive, so do case-insensitive lookup + return auxiliaryTypes.entrySet().stream() + .filter(entry -> entry.getKey().equalsIgnoreCase(typeName)) + .map(Map.Entry::getValue) + .findFirst(); + } + @Nonnull private Collection computeTemporaryInvokedRoutines() { return invokedRoutines.stream().filter(RecordLayerInvokedRoutine::isTemporary) @@ -625,10 +647,10 @@ public RecordLayerSchemaTemplate build() { if (cachedMetadata != null) { return new RecordLayerSchemaTemplate(name, new LinkedHashSet<>(tables.values()), - new LinkedHashSet<>(invokedRoutines.values()), new LinkedHashSet<>(views.values()), version, enableLongRows, storeRowVersions, intermingleTables, cachedMetadata); + new LinkedHashSet<>(invokedRoutines.values()), new LinkedHashSet<>(views.values()), version, enableLongRows, storeRowVersions, intermingleTables, cachedMetadata, auxiliaryTypes); } else { return new RecordLayerSchemaTemplate(name, new LinkedHashSet<>(tables.values()), - new LinkedHashSet<>(invokedRoutines.values()), new LinkedHashSet<>(views.values()), version, enableLongRows, storeRowVersions, intermingleTables); + new LinkedHashSet<>(invokedRoutines.values()), new LinkedHashSet<>(views.values()), version, enableLongRows, storeRowVersions, intermingleTables, auxiliaryTypes); } } @@ -756,6 +778,7 @@ public Builder toBuilder() { .setIntermingleTables(intermingleTables) .addTables(getTables()) .addInvokedRoutines(getInvokedRoutines()) - .addViews(getViews()); + .addViews(getViews()) + .addAuxiliaryTypes(auxiliaryTypes.values()); } } diff --git a/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/metadata/StructTypeValidator.java b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/metadata/StructTypeValidator.java new file mode 100644 index 0000000000..da154d26df --- /dev/null +++ b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/metadata/StructTypeValidator.java @@ -0,0 +1,139 @@ +/* + * StructTypeValidator.java + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2021-2025 Apple Inc. and the FoundationDB project authors + * + * 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.apple.foundationdb.relational.recordlayer.metadata; + +import com.apple.foundationdb.annotation.API; +import com.apple.foundationdb.relational.api.exceptions.ErrorCode; +import com.apple.foundationdb.relational.api.metadata.DataType; +import com.apple.foundationdb.relational.util.Assert; + +import javax.annotation.Nonnull; +import java.util.Locale; + +/** + * Utility class for validating struct type compatibility. + * Provides centralized logic for comparing struct types, with support for + * ignoring nullability differences and recursive validation of nested structs. + */ +@API(API.Status.EXPERIMENTAL) +public final class StructTypeValidator { + + private StructTypeValidator() { + // Utility class - prevent instantiation + } + + /** + * Check if two struct types are compatible, ignoring nullability differences. + * Two struct types are considered compatible if: + * - They have the same number of fields + * - Each corresponding field has the same type code (ignoring nullability) + * - If recursive=true, nested struct fields are recursively validated + * + * @param expected The expected struct type + * @param provided The provided struct type + * @param recursive If true, recursively validate nested struct types + * @return true if the struct types are compatible, false otherwise + */ + public static boolean areStructTypesCompatible(@Nonnull DataType.StructType expected, + @Nonnull DataType.StructType provided, + boolean recursive) { + final var expectedFields = expected.getFields(); + final var providedFields = provided.getFields(); + + // Check field count + if (!Integer.valueOf(expectedFields.size()).equals(providedFields.size())) { + return false; + } + + // Check each field type + for (int i = 0; i < expectedFields.size(); i++) { + final var expectedFieldType = expectedFields.get(i).getType(); + final var providedFieldType = providedFields.get(i).getType(); + + // Compare type codes (ignoring nullability) + if (!expectedFieldType.getCode().equals(providedFieldType.getCode())) { + return false; + } + + // Recursively validate nested structs if requested + if (recursive && expectedFieldType instanceof DataType.StructType && providedFieldType instanceof DataType.StructType) { + if (!areStructTypesCompatible((DataType.StructType) expectedFieldType, + (DataType.StructType) providedFieldType, + true)) { + return false; + } + } + } + + return true; + } + + /** + * Validate that two struct types are compatible, throwing an exception if they are not. + * This is a wrapper around {@link #areStructTypesCompatible} that throws an exception + * with a detailed error message if the types are incompatible. + * + * @param expected The expected struct type + * @param provided The provided struct type + * @param structName The name of the struct being validated (for error messages) + * @param recursive If true, recursively validate nested struct types + * @throws com.apple.foundationdb.relational.api.exceptions.RelationalException if the types are incompatible + */ + public static void validateStructTypesCompatible(@Nonnull DataType.StructType expected, + @Nonnull DataType.StructType provided, + @Nonnull String structName, + boolean recursive) { + final var expectedFields = expected.getFields(); + final var providedFields = provided.getFields(); + + // Check field count + if (!Integer.valueOf(expectedFields.size()).equals(providedFields.size())) { + Assert.failUnchecked(ErrorCode.CANNOT_CONVERT_TYPE, + String.format(Locale.ROOT, + "Struct type '%s' has incompatible signatures: expected %d fields but got %d fields", + structName, expectedFields.size(), providedFields.size())); + } + + // Check each field type + for (int i = 0; i < expectedFields.size(); i++) { + final var expectedFieldType = expectedFields.get(i).getType(); + final var providedFieldType = providedFields.get(i).getType(); + + // Compare type codes (ignoring nullability) + if (!expectedFieldType.getCode().equals(providedFieldType.getCode())) { + Assert.failUnchecked(ErrorCode.CANNOT_CONVERT_TYPE, + String.format(Locale.ROOT, + "Struct type '%s' has incompatible field at position %d: expected %s but got %s", + structName, i + 1, expectedFieldType.getCode(), providedFieldType.getCode())); + } + + // Recursively validate nested structs if requested + if (recursive && expectedFieldType instanceof DataType.StructType && providedFieldType instanceof DataType.StructType) { + // StructType extends Named, so we can always get the name + final var expectedStructName = ((DataType.StructType) expectedFieldType).getName(); + validateStructTypesCompatible((DataType.StructType) expectedFieldType, + (DataType.StructType) providedFieldType, + expectedStructName, + true); + } + } + } +} diff --git a/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/MutablePlanGenerationContext.java b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/MutablePlanGenerationContext.java index 232f7c786c..7bad379ce4 100644 --- a/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/MutablePlanGenerationContext.java +++ b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/MutablePlanGenerationContext.java @@ -42,9 +42,11 @@ import com.apple.foundationdb.relational.api.WithMetadata; import com.apple.foundationdb.relational.api.exceptions.RelationalException; import com.apple.foundationdb.relational.recordlayer.metadata.DataTypeUtils; +import com.apple.foundationdb.relational.recordlayer.metadata.StructTypeValidator; import com.apple.foundationdb.relational.util.Assert; import com.apple.foundationdb.relational.util.SpotBugsSuppressWarnings; +import com.apple.foundationdb.relational.api.metadata.DataType; import com.google.common.collect.ImmutableList; import com.google.protobuf.ZeroCopyByteString; @@ -55,8 +57,11 @@ import java.sql.SQLException; import java.sql.Struct; import java.util.ArrayList; +import java.util.HashMap; import java.util.LinkedList; import java.util.List; +import java.util.Locale; +import java.util.Map; import java.util.Optional; import java.util.function.Supplier; @@ -98,6 +103,9 @@ public class MutablePlanGenerationContext implements QueryExecutionContext { @Nonnull private final ImmutableList.Builder equalityConstraints; + @Nonnull + private final Map dynamicStructDefinitions; + private void startStructLiteral() { literalsBuilder.startStructLiteral(); } @@ -282,6 +290,7 @@ public MutablePlanGenerationContext(@Nonnull PreparedParams preparedParams, forExplain = false; setContinuation(null); equalityConstraints = ImmutableList.builder(); + dynamicStructDefinitions = new HashMap<>(); } @Nonnull @@ -493,4 +502,28 @@ private static Type getObjectType(@Nullable final Object object) { } return Type.fromObject(object); } + + /** + * Registers or validates a dynamic struct definition created within the query. + * If this is the first time seeing this struct name, registers it. + * If the struct name was already registered, validates that the new definition matches the previous one. + * + * @param structName The name of the struct type + * @param structType The struct type definition + * @throws com.apple.foundationdb.relational.api.exceptions.RelationalException if a struct with this name + * already exists with an incompatible signature + */ + public void registerOrValidateDynamicStruct(@Nonnull String structName, @Nonnull DataType.StructType structType) { + final var normalizedName = structName.toUpperCase(Locale.ROOT); + final var existing = dynamicStructDefinitions.get(normalizedName); + + if (existing == null) { + // First time seeing this struct name, register it + dynamicStructDefinitions.put(normalizedName, structType); + } else { + // Struct name already exists, validate compatibility using centralized validator + // This now correctly ignores nullability and recursively validates nested structs + StructTypeValidator.validateStructTypesCompatible(existing, structType, structName, true); + } + } } diff --git a/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/PlanGenerator.java b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/PlanGenerator.java index d5adc8953c..66ae9464e9 100644 --- a/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/PlanGenerator.java +++ b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/PlanGenerator.java @@ -34,6 +34,7 @@ import com.apple.foundationdb.record.query.plan.cascades.SemanticException; import com.apple.foundationdb.record.query.plan.cascades.StableSelectorCostModel; import com.apple.foundationdb.record.query.plan.cascades.typing.TypeRepository; +import com.apple.foundationdb.record.query.plan.cascades.typing.Type; import com.apple.foundationdb.record.query.plan.plans.RecordQueryPlan; import com.apple.foundationdb.record.query.plan.serialization.DefaultPlanSerializationRegistry; import com.apple.foundationdb.record.util.pair.NonnullPair; @@ -41,6 +42,7 @@ import com.apple.foundationdb.relational.api.exceptions.ErrorCode; import com.apple.foundationdb.relational.api.exceptions.RelationalException; import com.apple.foundationdb.relational.api.exceptions.UncheckedRelationalException; +import com.apple.foundationdb.relational.api.metadata.DataType; import com.apple.foundationdb.relational.api.metrics.RelationalMetric; import com.apple.foundationdb.relational.continuation.CompiledStatement; import com.apple.foundationdb.relational.continuation.TypedQueryArgument; @@ -61,6 +63,7 @@ import javax.annotation.Nonnull; import java.sql.SQLException; import java.util.Arrays; +import java.util.List; import java.util.Objects; import java.util.Optional; import java.util.Set; @@ -323,12 +326,26 @@ private QueryPlan.PhysicalQueryPlan generatePhysicalPlanForExecuteContinuation(@ planGenerationContext.setContinuation(continuationProto); final var continuationPlanConstraint = QueryPlanConstraint.fromProto(serializationContext, compiledStatement.getPlanConstraint()); + + final Type resultType = recordQueryPlan.getResultType().getInnerType(); + final List semanticFieldTypes; + if (resultType instanceof Type.Record) { + final Type.Record recordType = (Type.Record) resultType; + semanticFieldTypes = recordType.getFields().stream() + .map(field -> com.apple.foundationdb.relational.recordlayer.metadata.DataTypeUtils.toRelationalType(field.getFieldType())) + .collect(java.util.stream.Collectors.toList()); + } else { + // Fallback for non-record types (shouldn't happen for SELECT results) + semanticFieldTypes = java.util.Collections.emptyList(); + } + return new QueryPlan.ContinuedPhysicalQueryPlan(recordQueryPlan, typeRepository, continuationPlanConstraint, planGenerationContext, "EXECUTE CONTINUATION " + ast.getQueryCacheKey().getCanonicalQueryString(), currentPlanHashMode, - serializedPlanHashMode); + serializedPlanHashMode, + semanticFieldTypes); } private void resetTimer() { diff --git a/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/QueryPlan.java b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/QueryPlan.java index dd81fb418c..12ae186053 100644 --- a/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/QueryPlan.java +++ b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/QueryPlan.java @@ -73,15 +73,17 @@ import com.apple.foundationdb.relational.recordlayer.RecordLayerResultSet; import com.apple.foundationdb.relational.recordlayer.RecordLayerSchema; import com.apple.foundationdb.relational.recordlayer.ResumableIterator; -import com.apple.foundationdb.relational.recordlayer.metadata.DataTypeUtils; import com.apple.foundationdb.relational.recordlayer.util.ExceptionUtil; import com.apple.foundationdb.relational.util.Assert; import com.google.common.base.Suppliers; import com.google.common.base.Verify; import com.google.common.collect.ImmutableList; +import com.google.protobuf.Descriptors.Descriptor; import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.Message; +import java.util.HashMap; + import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.sql.SQLException; @@ -89,6 +91,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Objects; import java.util.Optional; @@ -128,6 +131,14 @@ public static class PhysicalQueryPlan extends QueryPlan { @Nonnull private final QueryExecutionContext queryExecutionContext; + /** + * Semantic type structure captured during semantic analysis. + * Preserves struct type names (like "STRUCT_1", "STRUCT_2") that get lost in planner Type conversion. + * Field names come from planner Type.Record - these are merged in executePhysicalPlan(). + */ + @Nonnull + private final List semanticFieldTypes; + public PhysicalQueryPlan(@Nonnull final RecordQueryPlan recordQueryPlan, @Nullable final StatsMaps plannerStatsMaps, @Nonnull final TypeRepository typeRepository, @@ -135,7 +146,8 @@ public PhysicalQueryPlan(@Nonnull final RecordQueryPlan recordQueryPlan, @Nonnull final QueryPlanConstraint continuationConstraint, @Nonnull final QueryExecutionContext queryExecutionContext, @Nonnull final String query, - @Nonnull final PlanHashMode currentPlanHashMode) { + @Nonnull final PlanHashMode currentPlanHashMode, + @Nonnull final List semanticFieldTypes) { super(query); this.recordQueryPlan = recordQueryPlan; this.plannerStatsMaps = plannerStatsMaps; @@ -145,6 +157,7 @@ public PhysicalQueryPlan(@Nonnull final RecordQueryPlan recordQueryPlan, this.queryExecutionContext = queryExecutionContext; this.currentPlanHashMode = currentPlanHashMode; this.planHashSupplier = Suppliers.memoize(() -> recordQueryPlan.planHash(currentPlanHashMode)); + this.semanticFieldTypes = semanticFieldTypes; } @Nonnull @@ -168,6 +181,11 @@ public QueryPlanConstraint getContinuationConstraint() { return continuationConstraint; } + @Nonnull + public List getSemanticFieldTypes() { + return semanticFieldTypes; + } + @Nonnull @Override public Type getResultType() { @@ -192,7 +210,8 @@ public PhysicalQueryPlan withExecutionContext(@Nonnull final QueryExecutionConte return this; } return new PhysicalQueryPlan(recordQueryPlan, plannerStatsMaps, typeRepository, constraint, - continuationConstraint, queryExecutionContext, query, queryExecutionContext.getPlanHashMode()); + continuationConstraint, queryExecutionContext, query, queryExecutionContext.getPlanHashMode(), + semanticFieldTypes); } @Nonnull @@ -404,15 +423,157 @@ private RelationalResultSet executePhysicalPlan(@Nonnull final RecordLayerSchema parsedContinuation.getExecutionState(), executeProperties)); final var currentPlanHashMode = OptionsUtils.getCurrentPlanHashMode(options); - final var dataType = (DataType.StructType) DataTypeUtils.toRelationalType(type); + + final DataType.StructType resultDataType = mergeSemanticTypesWithPlannerNames(type, semanticFieldTypes, fdbRecordStore.getRecordMetaData()); + return executionContext.metricCollector.clock(RelationalMetric.RelationalEvent.CREATE_RESULT_SET_ITERATOR, () -> { final ResumableIterator iterator = RecordLayerIterator.create(cursor, messageFDBQueriedRecord -> new MessageTuple(messageFDBQueriedRecord.getMessage())); - return new RecordLayerResultSet(RelationalStructMetaData.of(dataType), iterator, connection, + return new RecordLayerResultSet(RelationalStructMetaData.of(resultDataType), iterator, connection, (continuation, reason) -> enrichContinuation(continuation, currentPlanHashMode, reason)); }); } + /** + * Merge semantic type structure (preserving struct type names) with planner field names. + * + * - Field names and field count come from planner Type.Record + * (planner handles aliases, star expansion, and "_0" naming for unnamed expressions) + * - Type structure (especially nested struct type names) come from semantic DataTypes + * (semantic analysis preserves "STRUCT_1", "STRUCT_2" which planner loses) + * - Additionally enrich nested structs with RecordMetaData descriptor names + * + * @param plannerType The Type.Record from the physical plan (has correct field names) + * @param semanticTypes The semantic DataTypes captured before planning (have struct type names) + * @param recordMetaData Schema metadata for enriching nested types + * @return Merged DataType.StructType with planner names and semantic type structure + */ + @Nonnull + private DataType.StructType mergeSemanticTypesWithPlannerNames( + @Nonnull final Type plannerType, + @Nonnull final List semanticTypes, + @Nonnull final RecordMetaData recordMetaData) throws RelationalException { + + Assert.that(plannerType instanceof Type.Record, ErrorCode.INTERNAL_ERROR, + "Expected Type.Record but got %s", plannerType.getTypeCode()); + + final Type.Record recordType = (Type.Record) plannerType; + final List plannerFields = recordType.getFields(); + + // Planner and semantic should have same field count + Assert.that(plannerFields.size() == semanticTypes.size(), ErrorCode.INTERNAL_ERROR, + "Field count mismatch: planner has %d fields, semantic has %d", + plannerFields.size(), semanticTypes.size()); + + // Build descriptor cache for enriching nested structs + final Map descriptorCache = new HashMap<>(); + for (var recordTypeEntry : recordMetaData.getRecordTypes().values()) { + cacheDescriptorAndNested(recordTypeEntry.getDescriptor(), descriptorCache); + } + final var fileDescriptor = recordMetaData.getRecordTypes().values().iterator().next() + .getDescriptor().getFile(); + for (var messageType : fileDescriptor.getMessageTypes()) { + cacheDescriptorAndNested(messageType, descriptorCache); + } + + // Merge: field names from planner, types from semantic (enriched) + final ImmutableList.Builder mergedFields = ImmutableList.builder(); + for (int i = 0; i < plannerFields.size(); i++) { + final String fieldName = plannerFields.get(i).getFieldName(); + final DataType enrichedType = enrichDataType(semanticTypes.get(i), descriptorCache); + mergedFields.add(DataType.StructType.Field.from(fieldName, enrichedType, i)); + } + + return DataType.StructType.from("QUERY_RESULT", mergedFields.build(), true); + } + + /** + * Cache a descriptor and all its nested types, keyed by their structural signature. + */ + private void cacheDescriptorAndNested(@Nonnull final Descriptor descriptor, + @Nonnull final Map cache) { + // Create a structural signature for this descriptor (field names and count) + final String signature = createStructuralSignature(descriptor); + cache.put(signature, descriptor); + + // Process nested types + for (var nestedType : descriptor.getNestedTypes()) { + cacheDescriptorAndNested(nestedType, cache); + } + } + + /** + * Create a structural signature for a descriptor based on field names only. + * Field indices can vary between DataType and protobuf representations. + */ + @Nonnull + private String createStructuralSignature(@Nonnull final Descriptor descriptor) { + return descriptor.getFields().stream() + .map(f -> f.getName().toLowerCase(Locale.ROOT)) + .sorted() + .collect(java.util.stream.Collectors.joining(",")); + } + + /** + * Create a structural signature for a DataType.StructType based on field names only. + */ + @Nonnull + private String createStructuralSignature(@Nonnull final DataType.StructType structType) { + return structType.getFields().stream() + .map(f -> f.getName().toLowerCase(Locale.ROOT)) + .sorted() + .collect(java.util.stream.Collectors.joining(",")); + } + + /** + * Recursively enrich a struct type with proper names from the descriptor cache. + */ + @Nonnull + private DataType.StructType enrichStructType(@Nonnull final DataType.StructType structType, + @Nonnull final Map descriptorCache) { + // Enrich each field recursively + final List enrichedFields = structType.getFields().stream() + .map(field -> enrichField(field, descriptorCache)) + .collect(java.util.stream.Collectors.toList()); + + // Try to find a matching descriptor for this struct type + final String signature = createStructuralSignature(structType); + final Descriptor matchedDescriptor = descriptorCache.get(signature); + + // Use the descriptor's name if found, otherwise keep the existing name + final String enrichedName = matchedDescriptor != null ? matchedDescriptor.getName() : structType.getName(); + + return DataType.StructType.from(enrichedName, enrichedFields, structType.isNullable()); + } + + /** + * Enrich a field, recursively enriching any nested struct types. + */ + @Nonnull + private DataType.StructType.Field enrichField(@Nonnull final DataType.StructType.Field field, + @Nonnull final Map descriptorCache) { + final DataType enrichedType = enrichDataType(field.getType(), descriptorCache); + return DataType.StructType.Field.from(field.getName(), enrichedType, field.getIndex()); + } + + /** + * Enrich a DataType, handling structs, arrays, and primitives. + */ + @Nonnull + private DataType enrichDataType(@Nonnull final DataType dataType, + @Nonnull final Map descriptorCache) { + if (dataType instanceof DataType.StructType) { + return enrichStructType((DataType.StructType) dataType, descriptorCache); + } else if (dataType instanceof DataType.ArrayType) { + final DataType.ArrayType arrayType = (DataType.ArrayType) dataType; + final DataType enrichedElementType = enrichDataType(arrayType.getElementType(), descriptorCache); + return DataType.ArrayType.from(enrichedElementType, arrayType.isNullable()); + } else { + // Primitive types don't need enrichment + return dataType; + } + } + @Nonnull private Continuation enrichContinuation(@Nonnull final Continuation continuation, @Nonnull final PlanHashMode currentPlanHashMode, @@ -476,9 +637,10 @@ public ContinuedPhysicalQueryPlan(@Nonnull final RecordQueryPlan recordQueryPlan @Nonnull final QueryExecutionContext queryExecutionParameters, @Nonnull final String query, @Nonnull final PlanHashMode currentPlanHashMode, - @Nonnull final PlanHashMode serializedPlanHashMode) { + @Nonnull final PlanHashMode serializedPlanHashMode, + @Nonnull final List semanticFieldTypes) { super(recordQueryPlan, null, typeRepository, QueryPlanConstraint.noConstraint(), - continuationConstraint, queryExecutionParameters, query, currentPlanHashMode); + continuationConstraint, queryExecutionParameters, query, currentPlanHashMode, semanticFieldTypes); this.serializedPlanHashMode = serializedPlanHashMode; this.serializedPlanHashSupplier = Suppliers.memoize(() -> recordQueryPlan.planHash(serializedPlanHashMode)); } @@ -496,7 +658,8 @@ public PhysicalQueryPlan withExecutionContext(@Nonnull final QueryExecutionConte return this; } return new ContinuedPhysicalQueryPlan(getRecordQueryPlan(), getTypeRepository(), getContinuationConstraint(), - queryExecutionContext, query, queryExecutionContext.getPlanHashMode(), getSerializedPlanHashMode()); + queryExecutionContext, query, queryExecutionContext.getPlanHashMode(), getSerializedPlanHashMode(), + getSemanticFieldTypes()); } @Override @@ -549,18 +712,27 @@ public static class LogicalQueryPlan extends QueryPlan { @Nonnull private final String query; + /** + * Semantic type structure captured during semantic analysis. + * Preserves struct type names - will be merged with planner field names after planning. + */ + @Nonnull + private final List semanticFieldTypes; + @SuppressWarnings("OptionalUsedAsFieldOrParameterType") @Nonnull private Optional optimizedPlan; private LogicalQueryPlan(@Nonnull final RelationalExpression relationalExpression, @Nonnull final MutablePlanGenerationContext context, - @Nonnull final String query) { + @Nonnull final String query, + @Nonnull final List semanticFieldTypes) { super(query); this.relationalExpression = relationalExpression; this.context = context; this.optimizedPlan = Optional.empty(); this.query = query; + this.semanticFieldTypes = semanticFieldTypes; } @Override @@ -609,7 +781,8 @@ public PhysicalQueryPlan optimize(@Nonnull CascadesPlanner planner, @Nonnull Pla optimizedPlan = Optional.of( new PhysicalQueryPlan(minimizedPlan, statsMaps, builder.build(), - constraint, continuationConstraint, context, query, currentPlanHashMode)); + constraint, continuationConstraint, context, query, currentPlanHashMode, + semanticFieldTypes)); return optimizedPlan.get(); }); } @@ -657,8 +830,9 @@ public MutablePlanGenerationContext getGenerationContext() { @Nonnull public static LogicalQueryPlan of(@Nonnull final RelationalExpression relationalExpression, @Nonnull final MutablePlanGenerationContext context, - @Nonnull final String query) { - return new LogicalQueryPlan(relationalExpression, context, query); + @Nonnull final String query, + @Nonnull final List semanticFieldTypes) { + return new LogicalQueryPlan(relationalExpression, context, query, semanticFieldTypes); } @Nonnull diff --git a/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/visitors/ExpressionVisitor.java b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/visitors/ExpressionVisitor.java index 0abe0c59db..180488c411 100644 --- a/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/visitors/ExpressionVisitor.java +++ b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/visitors/ExpressionVisitor.java @@ -41,6 +41,7 @@ import com.apple.foundationdb.relational.recordlayer.metadata.DataTypeUtils; import com.apple.foundationdb.relational.recordlayer.metadata.RecordLayerColumn; import com.apple.foundationdb.relational.recordlayer.metadata.RecordLayerTable; +import com.apple.foundationdb.relational.recordlayer.metadata.StructTypeValidator; import com.apple.foundationdb.relational.recordlayer.query.Expression; import com.apple.foundationdb.relational.recordlayer.query.Expressions; import com.apple.foundationdb.relational.recordlayer.query.Identifier; @@ -805,6 +806,90 @@ public Expression visitRecordConstructor(@Nonnull RelationalParser.RecordConstru final var expressions = parseRecordFieldsUnderReorderings(ctx.expressionWithOptionalName()); if (ctx.ofTypeClause() != null) { final var recordId = visitUid(ctx.ofTypeClause().uid()); + + // When creating a named struct inline (e.g., STRUCT STRUCT_1(...)), validate that the + // constructed type is compatible with any schema-defined type with the same name. + // Look up the type by searching through all tables and their fields + final var schemaTemplate = getDelegate().getSchemaTemplate(); + Optional existingStructTypeMaybe = Optional.empty(); + + // Search through all tables to find a field with this struct type + for (final var table : schemaTemplate.getTables()) { + // Table datatype is always a StructType + final var structType = table.getDatatype(); + for (final var field : structType.getFields()) { + final var fieldType = field.getType(); + if (fieldType instanceof DataType.Named) { + final var namedType = (DataType.Named) fieldType; + if (namedType.getName().equalsIgnoreCase(recordId.getName())) { + // Found the type! Now resolve it to get the actual struct definition + if (namedType instanceof DataType.StructType) { + existingStructTypeMaybe = Optional.of((DataType.StructType) namedType); + break; + } + } + } + } + if (existingStructTypeMaybe.isPresent()) { + break; + } + } + + if (existingStructTypeMaybe.isPresent()) { + final var existingStructType = existingStructTypeMaybe.get(); + final var existingFields = existingStructType.getFields(); + final var providedExpressions = expressions.asList(); + + // Validate field count matches + if (!Integer.valueOf(existingFields.size()).equals(providedExpressions.size())) { + Assert.failUnchecked(ErrorCode.CANNOT_CONVERT_TYPE, + String.format("Cannot create struct '%s': expected %d fields but got %d", + recordId.getName(), existingFields.size(), providedExpressions.size())); + } + + // Validate each field type is compatible + for (int i = 0; i < existingFields.size(); i++) { + final var expectedField = existingFields.get(i); + final var providedExpression = providedExpressions.get(i); + final var expectedDataType = expectedField.getType(); + final var providedDataType = providedExpression.getDataType(); + + // Check if types are compatible + // For now, we check that the type codes match, ignoring nullability differences + // since a non-null value can always be used where a nullable field is expected + if (!expectedDataType.getCode().equals(providedDataType.getCode())) { + Assert.failUnchecked(ErrorCode.CANNOT_CONVERT_TYPE, + String.format("Cannot create struct '%s': field %d has incompatible type (expected %s but got %s)", + recordId.getName(), i + 1, expectedDataType.getCode(), providedDataType.getCode())); + } + // For struct types, we need to do deeper validation + if (expectedDataType instanceof DataType.StructType && providedDataType instanceof DataType.StructType) { + final var expectedStructType = (DataType.StructType) expectedDataType; + final var providedStructType = (DataType.StructType) providedDataType; + // Recursively validate struct fields using centralized validator + if (!StructTypeValidator.areStructTypesCompatible(expectedStructType, providedStructType, true)) { + Assert.failUnchecked(ErrorCode.CANNOT_CONVERT_TYPE, + String.format("Cannot create struct '%s': field %d has incompatible struct type", + recordId.getName(), i + 1)); + } + } + } + } + + // Also validate against dynamic struct definitions within this query + // Build a StructType from the provided expressions to validate consistency + final var providedFields = new ArrayList(); + final var providedExpressionsList = expressions.asList(); + for (int i = 0; i < providedExpressionsList.size(); i++) { + final var expression = providedExpressionsList.get(i); + final var fieldName = expression.getName().map(n -> n.getName()).orElse("_" + i); + providedFields.add(DataType.StructType.Field.from(fieldName, expression.getDataType(), i)); + } + final var providedStructType = DataType.StructType.from(recordId.getName(), providedFields, false); + + // Register or validate this dynamic struct definition + getDelegate().getPlanGenerationContext().registerOrValidateDynamicStruct(recordId.getName(), providedStructType); + final var resultValue = RecordConstructorValue.ofColumnsAndName(expressions.underlyingAsColumns(), recordId.getName()); return Expression.ofUnnamed(resultValue); } diff --git a/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/visitors/QueryVisitor.java b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/visitors/QueryVisitor.java index 9d7a45dd49..cd149395ac 100644 --- a/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/visitors/QueryVisitor.java +++ b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/visitors/QueryVisitor.java @@ -35,6 +35,7 @@ import com.apple.foundationdb.record.query.plan.cascades.values.Value; import com.apple.foundationdb.record.util.pair.NonnullPair; import com.apple.foundationdb.relational.api.exceptions.ErrorCode; +import com.apple.foundationdb.relational.api.metadata.DataType; import com.apple.foundationdb.relational.generated.RelationalLexer; import com.apple.foundationdb.relational.generated.RelationalParser; import com.apple.foundationdb.relational.recordlayer.metadata.RecordLayerTable; @@ -81,18 +82,46 @@ public static QueryVisitor of(@Nonnull BaseVisitor baseVisitor) { return new QueryVisitor(baseVisitor); } + /** + * Capture semantic type structure from expressions. + * + * This preserves struct type names (like "STRUCT_1", "STRUCT_2") that are known during + * semantic analysis but get lost in planner Type conversion (which has null names). + * + * Field names in the returned StructType are TEMPORARY PLACEHOLDERS. The actual field names + * will be taken from the planner Type.Record during result set creation. + * + * @param expressions The expressions from LogicalOperator.getOutput() + * @return List of DataTypes preserving struct type names (field names are placeholders) + */ + @Nonnull + private static List captureSemanticTypeStructure( + @Nonnull Expressions expressions) { + final ImmutableList.Builder types = ImmutableList.builder(); + for (final var expression : expressions) { + types.add(expression.getDataType()); + } + return types.build(); + } + @Nonnull @Override public QueryPlan.LogicalQueryPlan visitSelectStatement(@Nonnull RelationalParser.SelectStatementContext ctx) { final var logicalOperator = parseChild(ctx); - return QueryPlan.LogicalQueryPlan.of(logicalOperator.getQuantifier().getRangesOver().get(), getDelegate().getPlanGenerationContext(), "TODO"); + // Capture semantic type structure (preserves struct type names, field names come from planner later) + final var semanticTypes = captureSemanticTypeStructure(logicalOperator.getOutput()); + return QueryPlan.LogicalQueryPlan.of(logicalOperator.getQuantifier().getRangesOver().get(), + getDelegate().getPlanGenerationContext(), getDelegate().getPlanGenerationContext().getQuery(), semanticTypes); } @Nonnull @Override public QueryPlan.LogicalQueryPlan visitDmlStatement(@Nonnull RelationalParser.DmlStatementContext ctx) { final var logicalOperator = parseChild(ctx); - return QueryPlan.LogicalQueryPlan.of(logicalOperator.getQuantifier().getRangesOver().get(), getDelegate().getPlanGenerationContext(), "TODO"); + // Capture semantic type structure (preserves struct type names, field names come from planner later) + final var semanticTypes = captureSemanticTypeStructure(logicalOperator.getOutput()); + return QueryPlan.LogicalQueryPlan.of(logicalOperator.getQuantifier().getRangesOver().get(), + getDelegate().getPlanGenerationContext(), getDelegate().getPlanGenerationContext().getQuery(), semanticTypes); } @Nonnull @@ -556,7 +585,10 @@ public Object visitExecuteContinuationStatement(@Nonnull RelationalParser.Execut public QueryPlan.LogicalQueryPlan visitFullDescribeStatement(@Nonnull RelationalParser.FullDescribeStatementContext ctx) { getDelegate().getPlanGenerationContext().setForExplain(ctx.EXPLAIN() != null); final var logicalOperator = Assert.castUnchecked(ctx.describeObjectClause().accept(this), LogicalOperator.class); - return QueryPlan.LogicalQueryPlan.of(logicalOperator.getQuantifier().getRangesOver().get(), getDelegate().getPlanGenerationContext(), "TODO"); + // Capture semantic type structure (preserves struct type names, field names come from planner later) + final var semanticTypes = captureSemanticTypeStructure(logicalOperator.getOutput()); + return QueryPlan.LogicalQueryPlan.of(logicalOperator.getQuantifier().getRangesOver().get(), + getDelegate().getPlanGenerationContext(), getDelegate().getPlanGenerationContext().getQuery(), semanticTypes); } @Nonnull diff --git a/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/StructDataMetadataTest.java b/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/StructDataMetadataTest.java index 01f74d76ff..2cbf544a6d 100644 --- a/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/StructDataMetadataTest.java +++ b/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/StructDataMetadataTest.java @@ -20,21 +20,28 @@ package com.apple.foundationdb.relational.recordlayer; +import com.apple.foundationdb.relational.api.Continuation; import com.apple.foundationdb.relational.api.EmbeddedRelationalArray; import com.apple.foundationdb.relational.api.EmbeddedRelationalStruct; import com.apple.foundationdb.relational.api.KeySet; import com.apple.foundationdb.relational.api.Options; +import com.apple.foundationdb.relational.api.RelationalArray; import com.apple.foundationdb.relational.api.RelationalResultSet; import com.apple.foundationdb.relational.api.RelationalStruct; +import com.apple.foundationdb.relational.api.exceptions.ErrorCode; +import com.apple.foundationdb.relational.utils.RelationalAssertions; import com.apple.foundationdb.relational.utils.SimpleDatabaseRule; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Order; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.function.ThrowingConsumer; import java.nio.charset.StandardCharsets; import java.sql.Array; +import java.sql.PreparedStatement; +import java.sql.ResultSet; import java.sql.SQLException; import java.util.Set; @@ -48,7 +55,7 @@ public class StructDataMetadataTest { public static final EmbeddedRelationalExtension relationalExtension = new EmbeddedRelationalExtension(); private static final String TABLE_STRUCTURE = - "CREATE TYPE AS STRUCT struct_1 (a string) " + + "CREATE TYPE AS STRUCT struct_1 (a string, b bigint) " + " CREATE TABLE t (name string, st1 struct_1, PRIMARY KEY(name))" + " CREATE TYPE AS STRUCT struct_2 (c bigint, d struct_1) " + " CREATE TABLE nt (t_name string, st1 struct_2, PRIMARY KEY(t_name))" + @@ -79,7 +86,18 @@ public class StructDataMetadataTest { void setUp() throws SQLException { var m = EmbeddedRelationalStruct.newBuilder() .addString("NAME", "test_record_1") - .addStruct("ST1", EmbeddedRelationalStruct.newBuilder().addString("A", "Hello").build()) + .addStruct("ST1", EmbeddedRelationalStruct.newBuilder() + .addString("A", "Hello") + .addLong("B", 100L) + .build()) + .build(); + statement.executeInsert("T", m); + m = EmbeddedRelationalStruct.newBuilder() + .addString("NAME", "test_record_2") + .addStruct("ST1", EmbeddedRelationalStruct.newBuilder() + .addString("A", "World") + .addLong("B", 200L) + .build()) .build(); statement.executeInsert("T", m); @@ -94,7 +112,18 @@ void setUp() throws SQLException { .build(); statement.executeInsert("NT", m); - final var atBuilder = EmbeddedRelationalStruct.newBuilder(); + m = EmbeddedRelationalStruct.newBuilder() + .addString("T_NAME", "nt_record2") + .addStruct("ST1", EmbeddedRelationalStruct.newBuilder() + .addLong("C", 5678L) + .addStruct("D", EmbeddedRelationalStruct.newBuilder() + .addString("A", "Ciao") + .build()) + .build()) + .build(); + statement.executeInsert("NT", m); + + var atBuilder = EmbeddedRelationalStruct.newBuilder(); m = atBuilder.addString("A_NAME", "a_test_rec") .addArray("ST2", EmbeddedRelationalArray.newBuilder() .addStruct(EmbeddedRelationalStruct.newBuilder() @@ -108,6 +137,21 @@ void setUp() throws SQLException { .build()) .build(); statement.executeInsert("AT", m); + + atBuilder = EmbeddedRelationalStruct.newBuilder(); + m = atBuilder.addString("A_NAME", "another_test_rec") + .addArray("ST2", EmbeddedRelationalArray.newBuilder() + .addStruct(EmbeddedRelationalStruct.newBuilder() + .addBytes("C", "今日は".getBytes(StandardCharsets.UTF_8)) + .addBoolean("D", true) + .build()) + .addStruct(EmbeddedRelationalStruct.newBuilder() + .addBytes("C", "مرحبًا".getBytes(StandardCharsets.UTF_8)) + .addBoolean("D", false) + .build()) + .build()) + .build(); + statement.executeInsert("AT", m); } @Test @@ -119,41 +163,183 @@ void canReadSingleStruct() throws Exception { Assertions.assertNotNull(struct, "No struct found for column!"); Assertions.assertEquals("Hello", struct.getString(1), "Incorrect value for nested struct!"); Assertions.assertEquals("Hello", struct.getString("A"), "Incorrect value for nested struct!"); + Assertions.assertEquals(100L, struct.getLong(2), "Incorrect value for nested struct B field!"); + Assertions.assertEquals(100L, struct.getLong("B"), "Incorrect value for nested struct B field!"); //check that the JDBC attributes methods work properly - Assertions.assertArrayEquals(struct.getAttributes(), new Object[]{"Hello"}, "Incorrect attributes!"); + Assertions.assertArrayEquals(new Object[]{"Hello", 100L}, struct.getAttributes(), "Incorrect attributes!"); } } - @Test - void canReadProjectedStructTypeNameInNestedStar() throws Exception { - try (final RelationalResultSet resultSet = statement.executeQuery("SELECT (*) FROM T")) { + private void canReadStructTypeName(String query, ThrowingConsumer assertOnMetaData) throws Throwable { + Continuation continuation; + statement.setMaxRows(1); + try (final RelationalResultSet resultSet = statement.executeQuery(query)) { Assertions.assertTrue(resultSet.next(), "Did not find a record!"); + assertOnMetaData.accept(resultSet); + continuation = resultSet.getContinuation(); + } + try (final PreparedStatement ps = connection.prepareStatement("EXECUTE CONTINUATION ?")) { + ps.setBytes(1, continuation.serialize()); + try (final ResultSet resultSet = ps.executeQuery()) { + Assertions.assertTrue(resultSet.next(), "Did not find a record!"); + assertOnMetaData.accept(resultSet.unwrap(RelationalResultSet.class)); + } + } + } + + @Test + void canReadProjectedStructTypeNameInNestedStar() throws Throwable { + canReadStructTypeName("SELECT (*) FROM T", resultSet -> { RelationalStruct struct = resultSet.getStruct(1).getStruct("ST1"); Assertions.assertEquals("STRUCT_1", struct.getMetaData().getTypeName()); - } + }); } - // When projecting *, the underlying struct types are lost and replaced with a generic UUID type. - // This test should be replaced with the correct expected behavior once this is fixed. - // When projecting (*), everything works as expected, see `canReadProjectedStructTypeNameInNestedStar`. - // See https://github.com/FoundationDB/fdb-record-layer/issues/3743 @Test - void cannotReadProjectedStructTypeNameInUnnestedStar() throws Exception { - try (final RelationalResultSet resultSet = statement.executeQuery("SELECT * FROM T")) { - Assertions.assertTrue(resultSet.next(), "Did not find a record!"); + void canReadProjectedNestedStructTypeNameInNestedStar() throws Throwable { + canReadStructTypeName("SELECT (*) FROM NT", resultSet -> { + RelationalStruct struct = resultSet.getStruct(1).getStruct("ST1").getStruct("D"); + Assertions.assertEquals("STRUCT_1", struct.getMetaData().getTypeName()); + }); + } + + @Test + void canReadProjectedStructInArrayTypeNameInNestedStar() throws Throwable { + canReadStructTypeName("SELECT (*) FROM AT", resultSet -> { + RelationalArray array = resultSet.getStruct(1).getArray("ST2"); + Assertions.assertEquals("STRUCT", array.getMetaData().getElementTypeName()); + Assertions.assertEquals("STRUCT_3", array.getMetaData().getElementStructMetaData().getTypeName()); + }); + } + + @Test + void canReadProjectedStructTypeNameInUnnestedStar() throws Throwable { + canReadStructTypeName("SELECT * FROM T", resultSet -> { RelationalStruct struct = resultSet.getStruct("ST1"); - Assertions.assertNotEquals("STRUCT_1", struct.getMetaData().getTypeName()); - } + Assertions.assertEquals("STRUCT_1", struct.getMetaData().getTypeName()); + }); } @Test - void canReadProjectedStructTypeNameDirectlyProjected() throws Exception { - try (final RelationalResultSet resultSet = statement.executeQuery("SELECT ST1 FROM T")) { - Assertions.assertTrue(resultSet.next(), "Did not find a record!"); + void canReadProjectedNestedStructTypeNameInUnnestedStar() throws Throwable { + canReadStructTypeName("SELECT * FROM NT", resultSet -> { + RelationalStruct struct = resultSet.getStruct("ST1").getStruct("D"); + Assertions.assertEquals("STRUCT_1", struct.getMetaData().getTypeName()); + }); + } + + @Test + void canReadProjectedStructInArrayTypeNameInUnnestedStar() throws Throwable { + canReadStructTypeName("SELECT * FROM AT", resultSet -> { + RelationalArray array = resultSet.getArray("ST2"); + Assertions.assertEquals("STRUCT", array.getMetaData().getElementTypeName()); + Assertions.assertEquals("STRUCT_3", array.getMetaData().getElementStructMetaData().getTypeName()); + }); + } + + @Test + void canReadProjectedStructTypeNameDirectlyProjected() throws Throwable { + canReadStructTypeName("SELECT ST1 FROM T", resultSet -> { RelationalStruct struct = resultSet.getStruct("ST1"); Assertions.assertEquals("STRUCT_1", struct.getMetaData().getTypeName()); - } + }); + } + + @Test + void canReadProjectedNestedStructTypeNameDirectlyProjected() throws Throwable { + canReadStructTypeName("SELECT ST1 FROM NT", resultSet -> { + RelationalStruct struct = resultSet.getStruct("ST1").getStruct("D"); + Assertions.assertEquals("STRUCT_1", struct.getMetaData().getTypeName()); + }); + } + + @Test + void canReadProjectedStructInArrayTypeNameDirectlyProjected() throws Throwable { + canReadStructTypeName("SELECT * FROM AT", resultSet -> { + RelationalArray array = resultSet.getArray("ST2"); + Assertions.assertEquals("STRUCT", array.getMetaData().getElementTypeName()); + Assertions.assertEquals("STRUCT_3", array.getMetaData().getElementStructMetaData().getTypeName()); + }); + } + + @Test + void canReadProjectedDynamicStruct() throws Throwable { + canReadStructTypeName("SELECT STRUCT STRUCT_6(name, st1.a, st1) FROM T", resultSet -> { + RelationalStruct struct = resultSet.getStruct(1); + Assertions.assertEquals("STRUCT_6", struct.getMetaData().getTypeName()); + Assertions.assertEquals("STRUCT_1", struct.getStruct(3).getMetaData().getTypeName()); + }); + } + + @Test + void canReadProjectedStructWithDynamicStructInside() throws Throwable { + canReadStructTypeName("SELECT STRUCT STRUCT_6(name, STRUCT STRUCT_7(name, st1.a)) FROM T", resultSet -> { + RelationalStruct struct = resultSet.getStruct(1); + Assertions.assertEquals("STRUCT_6", struct.getMetaData().getTypeName()); + Assertions.assertEquals("STRUCT_7", struct.getStruct(2).getMetaData().getTypeName()); + }); + } + + @Test + void canReadAnonymousStructWithDynamicStructInside() throws Throwable { + canReadStructTypeName("SELECT (name, STRUCT STRUCT_7(name, st1.a)) FROM T", resultSet -> { + RelationalStruct struct = resultSet.getStruct(1); + Assertions.assertEquals("STRUCT_7", struct.getStruct(2).getMetaData().getTypeName()); + }); + } + + @Test + void canNameDynamicStructSameAsStaticStructIfSame() throws Throwable { + canReadStructTypeName("SELECT STRUCT STRUCT_1(name, st1.b) FROM T", resultSet -> { + RelationalStruct struct = resultSet.getStruct(1); + Assertions.assertEquals("STRUCT_1", struct.getMetaData().getTypeName()); + }); + } + + @Test + void canNameDynamicStructSameAsStaticStructIfSame2() throws Throwable { + canReadStructTypeName("SELECT STRUCT STRUCT_2(10l, STRUCT STRUCT_1(name, st1.b)) FROM T", resultSet -> { + RelationalStruct struct = resultSet.getStruct(1); + Assertions.assertEquals("STRUCT_2", struct.getMetaData().getTypeName()); + Assertions.assertEquals("STRUCT_1", struct.getStruct(2).getMetaData().getTypeName()); + }); + } + + @Test + void cannotNameDynamicStructSameAsStaticStructIfSignatureIsDifferent() throws Throwable { + RelationalAssertions.assertThrowsSqlException(() -> statement.executeQuery("SELECT STRUCT STRUCT_1(st1.b) FROM T")) + .hasErrorCode(ErrorCode.CANNOT_CONVERT_TYPE); + } + + @Test + void cannotHaveTwoDynamicStructsWithDifferentTypes() throws Throwable { + RelationalAssertions.assertThrowsSqlException(() -> statement.executeQuery("SELECT STRUCT STRUCT_19(st1.b), STRUCT STRUCT_19(name) FROM T")) + .hasErrorCode(ErrorCode.CANNOT_CONVERT_TYPE); + } + + @Test + void cannotHaveTwoDynamicStructsWithDifferentNestedTypes() throws Throwable { + RelationalAssertions.assertThrowsSqlException(() -> statement.executeQuery("SELECT STRUCT STRUCT_19(STRUCT STRUCT_20(st1.b)), STRUCT STRUCT_19(STRUCT STRUCT_20(name)) FROM T")) + .hasErrorCode(ErrorCode.CANNOT_CONVERT_TYPE); + } + + @Test + void cannotHaveTwoDynamicStructsOneWithSameAsTemplateOneDifferent() throws Throwable { + RelationalAssertions.assertThrowsSqlException(() -> statement.executeQuery("SELECT STRUCT STRUCT_1(name, st1.b), STRUCT STRUCT_1(name) FROM T")) + .hasErrorCode(ErrorCode.CANNOT_CONVERT_TYPE); + } + + @Test + void canUseDifferentDynamicStructsWithSameNameInTwoDifferentQueries() throws Throwable { + canReadStructTypeName("SELECT STRUCT STRUCT_18(name, st1.b) FROM T", resultSet -> { + RelationalStruct struct = resultSet.getStruct(1); + Assertions.assertEquals("STRUCT_18", struct.getMetaData().getTypeName()); + }); + canReadStructTypeName("SELECT STRUCT STRUCT_18(st1.b) FROM T", resultSet -> { + RelationalStruct struct = resultSet.getStruct(1); + Assertions.assertEquals("STRUCT_18", struct.getMetaData().getTypeName()); + }); } @Test