diff --git a/fdb-relational-core/src/main/antlr/RelationalParser.g4 b/fdb-relational-core/src/main/antlr/RelationalParser.g4 index 3bbbd614b5..8460fdd8e4 100644 --- a/fdb-relational-core/src/main/antlr/RelationalParser.g4 +++ b/fdb-relational-core/src/main/antlr/RelationalParser.g4 @@ -341,7 +341,7 @@ selectStatement ; query - : ctes? queryExpressionBody continuation? + : ctes? queryExpressionBody ; ctes @@ -369,10 +369,6 @@ tableFunctionName : fullId ; -continuation - : WITH CONTINUATION continuationAtom - ; - // done queryExpressionBody : queryTerm #queryTermDefault // done @@ -402,7 +398,6 @@ updateStatement SET updatedElement (',' updatedElement)* (WHERE whereExpr)? (RETURNING selectElements)? - (WITH CONTINUATION continuationAtom)? queryOptions? ; diff --git a/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/metadata/TypeMetadataEnricher.java b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/metadata/TypeMetadataEnricher.java new file mode 100644 index 0000000000..268c702434 --- /dev/null +++ b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/metadata/TypeMetadataEnricher.java @@ -0,0 +1,226 @@ +/* + * TypeMetadataEnricher.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.record.RecordMetaData; +import com.apple.foundationdb.record.query.plan.cascades.typing.Type; +import com.apple.foundationdb.relational.api.exceptions.ErrorCode; +import com.apple.foundationdb.relational.api.exceptions.RelationalException; +import com.apple.foundationdb.relational.api.metadata.DataType; +import com.apple.foundationdb.relational.util.Assert; +import com.google.common.collect.ImmutableList; +import com.google.protobuf.Descriptors; +import com.google.protobuf.Descriptors.Descriptor; + +import javax.annotation.Nonnull; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Utility class for enriching DataType structures with metadata from semantic analysis and protobuf descriptors. + * + *

This class handles the merging of type information from multiple sources: + *

+ * + *

The planner's Type.Record loses struct type names during optimization (they become null), + * but semantic analysis preserves them. This utility merges both sources to create complete + * type metadata for result sets. + */ +@API(API.Status.EXPERIMENTAL) +public final class TypeMetadataEnricher { + + private TypeMetadataEnricher() { + // Utility class - prevent instantiation + } + + /** + * Merge semantic type structure (preserving struct type names) with planner field names. + * + *

This method combines: + *

+ * + * @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 + * @throws RelationalException if type structures don't match + */ + @Nonnull + public static 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. + * + * @param descriptor The protobuf descriptor to cache + * @param cache The cache map to populate + */ + private static 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. + * + * @param descriptor The protobuf descriptor + * @return A signature string representing the structure + */ + @Nonnull + private static String createStructuralSignature(@Nonnull final Descriptor descriptor) { + return descriptor.getFields().stream() + .map(Descriptors.FieldDescriptor::getName) + .collect(java.util.stream.Collectors.joining(",")); + } + + /** + * Create a structural signature for a DataType.StructType based on field names only. + * + * @param structType The struct type + * @return A signature string representing the structure + */ + @Nonnull + private static String createStructuralSignature(@Nonnull final DataType.StructType structType) { + return structType.getFields().stream() + .map(DataType.StructType.Field::getName) + .collect(java.util.stream.Collectors.joining(",")); + } + + /** + * Recursively enrich a struct type with proper names from the descriptor cache. + * + * @param structType The struct type to enrich + * @param descriptorCache Cache of descriptors keyed by structural signature + * @return Enriched struct type with proper names from descriptors + */ + @Nonnull + private static 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. + * + * @param field The field to enrich + * @param descriptorCache Cache of descriptors keyed by structural signature + * @return Enriched field with proper type metadata + */ + @Nonnull + private static 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. + * + *

For struct types, looks up matching descriptors and enriches the struct name. + * For array types, recursively enriches the element type. + * For primitive types, returns as-is. + * + * @param dataType The data type to enrich + * @param descriptorCache Cache of descriptors keyed by structural signature + * @return Enriched data type with proper metadata + */ + @Nonnull + private static 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; + } + } +} diff --git a/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/AstNormalizer.java b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/AstNormalizer.java index 57f7c67233..e61db33914 100644 --- a/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/AstNormalizer.java +++ b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/AstNormalizer.java @@ -271,17 +271,9 @@ public Object visitQuery(@Nonnull RelationalParser.QueryContext ctx) { visit(ctx.ctes()); } ctx.queryExpressionBody().accept(this); - if (ctx.continuation() != null) { - ctx.continuation().accept(this); - } return null; } - @Override - public Object visitContinuation(@Nonnull RelationalParser.ContinuationContext ctx) { - return ctx.continuationAtom().accept(this); - } - @Override public RelationalExpression visitQueryOptions(@Nonnull RelationalParser.QueryOptionsContext ctx) { for (final var opt : ctx.queryOption()) { 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 7db30bce17..44329aae11 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.ProtoUtils; @@ -42,6 +43,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; @@ -62,6 +64,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; @@ -326,12 +329,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..39fceae5f0 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,7 +73,7 @@ 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.metadata.TypeMetadataEnricher; import com.apple.foundationdb.relational.recordlayer.util.ExceptionUtil; import com.apple.foundationdb.relational.util.Assert; import com.google.common.base.Suppliers; @@ -128,6 +128,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 +143,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 +154,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 +178,11 @@ public QueryPlanConstraint getContinuationConstraint() { return continuationConstraint; } + @Nonnull + public List getSemanticFieldTypes() { + return semanticFieldTypes; + } + @Nonnull @Override public Type getResultType() { @@ -192,7 +207,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,10 +420,12 @@ 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 = TypeMetadataEnricher.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)); }); @@ -476,9 +494,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 +515,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 +569,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 +638,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 +687,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/BaseVisitor.java b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/visitors/BaseVisitor.java index c318a443b3..ecf6590442 100644 --- a/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/visitors/BaseVisitor.java +++ b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/visitors/BaseVisitor.java @@ -624,12 +624,6 @@ public Identifier visitTableFunctionName(final RelationalParser.TableFunctionNam return identifierVisitor.visitTableFunctionName(ctx); } - @Nonnull - @Override - public Expression visitContinuation(RelationalParser.ContinuationContext ctx) { - return expressionVisitor.visitContinuation(ctx); - } - @Nonnull @Override public Expression visitContinuationAtom(@Nonnull RelationalParser.ContinuationAtomContext ctx) { diff --git a/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/visitors/DelegatingVisitor.java b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/visitors/DelegatingVisitor.java index 4273b248ac..d82f3613e1 100644 --- a/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/visitors/DelegatingVisitor.java +++ b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/visitors/DelegatingVisitor.java @@ -476,12 +476,6 @@ public Identifier visitTableFunctionName(@Nonnull RelationalParser.TableFunction return getDelegate().visitTableFunctionName(ctx); } - @Nonnull - @Override - public Expression visitContinuation(@Nonnull RelationalParser.ContinuationContext ctx) { - return getDelegate().visitContinuation(ctx); - } - @Nonnull @Override public Expression visitContinuationAtom(@Nonnull RelationalParser.ContinuationAtomContext ctx) { 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..8178bd893a 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 @@ -121,12 +121,6 @@ public Expression visitNamedFunctionArg(@Nonnull final RelationalParser.NamedFun return expression.toNamedArgument(name); } - @Nonnull - @Override - public Expression visitContinuation(@Nonnull RelationalParser.ContinuationContext ctx) { - return visitContinuationAtom(ctx.continuationAtom()); - } - @Nonnull @Override public Expression visitContinuationAtom(@Nonnull RelationalParser.ContinuationAtomContext ctx) { 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 4989946f6e..8bd0334b41 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 @@ -31,10 +31,10 @@ import com.apple.foundationdb.record.query.plan.cascades.predicates.CompatibleTypeEvolutionPredicate; import com.apple.foundationdb.record.query.plan.cascades.typing.Type; import com.apple.foundationdb.record.query.plan.cascades.values.FieldValue; -import com.apple.foundationdb.record.query.plan.cascades.values.LiteralValue; 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; @@ -58,7 +58,6 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.Streams; -import com.google.protobuf.ByteString; import org.antlr.v4.runtime.ParserRuleContext; import javax.annotation.Nonnull; @@ -81,29 +80,51 @@ 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 @Override public LogicalOperator visitQuery(@Nonnull RelationalParser.QueryContext ctx) { - if (ctx.continuation() != null) { - final var continuationExpression = visitContinuation(ctx.continuation()); - final var continuationValue = Assert.castUnchecked(continuationExpression.getUnderlying(), LiteralValue.class); - final var continuationBytes = Assert.castUnchecked(continuationValue.getLiteralValue(), ByteString.class); - getDelegate().getPlanGenerationContext().setContinuation(continuationBytes.toByteArray()); - } if (ctx.ctes() != null) { final var currentPlanFragment = getDelegate().pushPlanFragment(); visitCtes(ctx.ctes()).forEach(currentPlanFragment::addOperator); @@ -556,7 +577,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/main/java/com/apple/foundationdb/relational/recordlayer/query/visitors/TypedVisitor.java b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/visitors/TypedVisitor.java index 92a0ca8db9..1730c86fef 100644 --- a/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/visitors/TypedVisitor.java +++ b/fdb-relational-core/src/main/java/com/apple/foundationdb/relational/recordlayer/query/visitors/TypedVisitor.java @@ -266,10 +266,6 @@ public interface TypedVisitor extends RelationalParserVisitor { @Override Identifier visitTableFunctionName(RelationalParser.TableFunctionNameContext ctx); - @Nonnull - @Override - Expression visitContinuation(RelationalParser.ContinuationContext ctx); - @Nonnull @Override Expression visitContinuationAtom(@Nonnull RelationalParser.ContinuationAtomContext ctx); diff --git a/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/CursorTest.java b/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/CursorTest.java index 33f44a70a0..b4968acdda 100644 --- a/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/CursorTest.java +++ b/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/CursorTest.java @@ -176,7 +176,7 @@ public void continuationWithReturnRowLimit() throws SQLException, RelationalExce } catch (SQLException e) { throw new RuntimeException(e); } - try (final var preparedStatement = conn.prepareStatement("select * from RESTAURANT with continuation ?param")) { + try (final var preparedStatement = conn.prepareStatement("EXECUTE CONTINUATION ?param")) { preparedStatement.setBytes("param", continuation.serialize()); try (final var resultSet = preparedStatement.executeQuery()) { Assertions.assertThrows(SQLException.class, resultSet::getContinuation); @@ -218,7 +218,7 @@ public void continuationWithScanRowLimit() throws SQLException, RelationalExcept // 2. Further count the rows in other execution without limits and see if total number of rows is 10 try (final var conn = DriverManager.getConnection(database.getConnectionUri().toString()).unwrap(RelationalConnection.class)) { conn.setSchema(database.getSchemaName()); - try (final var preparedStatement = conn.prepareStatement("select * from RESTAURANT with continuation ?param")) { + try (final var preparedStatement = conn.prepareStatement("EXECUTE CONTINUATION ?param")) { preparedStatement.setBytes("param", continuation.serialize()); try (final var resultSet = preparedStatement.executeQuery()) { Assertions.assertThrows(SQLException.class, resultSet::getContinuation); diff --git a/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/PlanGenerationStackTest.java b/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/PlanGenerationStackTest.java index 6314eb76bb..d2cb126822 100644 --- a/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/PlanGenerationStackTest.java +++ b/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/PlanGenerationStackTest.java @@ -143,11 +143,11 @@ public Stream provideArguments(final ExtensionContext conte Arguments.of(57, "select * from restaurant where (((42 + 3) - 2) + 6 is not null OR ((42 + 3) - 2) + 6 > rest_no) OR (name = 'foo')", null), Arguments.of(58, "select * from restaurant where rest_no is null", null), Arguments.of(59, "select * from restaurant where rest_no is not null", null), - Arguments.of(60, "select * from restaurant with continuation b64'abc'", null), + Arguments.of(60, "select * from restaurant with continuation b64'abc'", "syntax error"), Arguments.of(61, "select * from restaurant USE INDEX (record_name_idx) where rest_no > 10 ", null), Arguments.of(62, "select * from restaurant USE INDEX (record_name_idx, reviewer_name_idx) where rest_no > 10 ", "Unknown index(es) REVIEWER_NAME_IDX"), Arguments.of(63, "select * from restaurant USE INDEX (record_name_idx), USE INDEX (reviewer_name_idx) where rest_no > 10 ", "Unknown index(es) REVIEWER_NAME_IDX"), - Arguments.of(64, "select * from restaurant with continuation", "syntax error[[]]select * from restaurant with continuation[[]] ^^"), + Arguments.of(64, "select * from restaurant with continuation", "syntax error"), Arguments.of(65, "select X.rest_no from (select rest_no from restaurant where 42 >= rest_no OR 42 > rest_no) X", null), Arguments.of( 66, "select X.UNKNOWN from (select rest_no from restaurant where 42 >= rest_no OR 42 > rest_no) X", "Attempting to query non existing column 'X.UNKNOWN'"), Arguments.of(67, "select X.rest_no from (select Y.rest_no from (select rest_no from restaurant where 42 >= rest_no OR 42 > rest_no) Y where 42 >= Y.rest_no OR 42 > Y.rest_no) X", null), diff --git a/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/SimpleDirectAccessInsertionTests.java b/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/SimpleDirectAccessInsertionTests.java index e5478dd1ce..089acf0dca 100644 --- a/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/SimpleDirectAccessInsertionTests.java +++ b/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/SimpleDirectAccessInsertionTests.java @@ -40,7 +40,6 @@ import java.sql.DriverManager; import java.sql.SQLException; -import java.util.Base64; import java.util.Collections; import java.util.List; @@ -70,8 +69,9 @@ void useScanContinuationInQueryShouldNotWork() throws Exception { rrs.next(); continuation1 = rrs.getContinuation(); } - String continuationString = Base64.getEncoder().encodeToString(continuation1.serialize()); - org.junit.jupiter.api.Assertions.assertThrows(ContextualSQLException.class, () -> s.executeQuery("SELECT * FROM RESTAURANT_REVIEWER LIMIT 1 WITH CONTINUATION B64'" + continuationString + "'"), "Continuation binding does not match query"); + // Try to use scan continuation with EXECUTE CONTINUATION - should fail + String continuationString = java.util.Base64.getEncoder().encodeToString(continuation1.serialize()); + org.junit.jupiter.api.Assertions.assertThrows(ContextualSQLException.class, () -> s.executeQuery("EXECUTE CONTINUATION B64'" + continuationString + "'"), "Continuation binding does not match query"); } // get @@ -81,8 +81,9 @@ void useScanContinuationInQueryShouldNotWork() throws Exception { rrs.next(); continuation1 = rrs.getContinuation(); } - String continuationString = Base64.getEncoder().encodeToString(continuation1.serialize()); - org.junit.jupiter.api.Assertions.assertThrows(ContextualSQLException.class, () -> s.executeQuery("SELECT * FROM RESTAURANT_REVIEWER LIMIT 1 WITH CONTINUATION B64'" + continuationString + "'"), "Continuation binding does not match query"); + // Try to use get continuation with EXECUTE CONTINUATION - should fail + String continuationString = java.util.Base64.getEncoder().encodeToString(continuation1.serialize()); + org.junit.jupiter.api.Assertions.assertThrows(ContextualSQLException.class, () -> s.executeQuery("EXECUTE CONTINUATION B64'" + continuationString + "'"), "Continuation binding does not match query"); } } } 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..15ba16fd2e 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,10 +20,12 @@ 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.utils.SimpleDatabaseRule; @@ -32,9 +34,12 @@ 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; @@ -82,6 +87,11 @@ void setUp() throws SQLException { .addStruct("ST1", EmbeddedRelationalStruct.newBuilder().addString("A", "Hello").build()) .build(); statement.executeInsert("T", m); + m = EmbeddedRelationalStruct.newBuilder() + .addString("NAME", "test_record_2") + .addStruct("ST1", EmbeddedRelationalStruct.newBuilder().addString("A", "World").build()) + .build(); + statement.executeInsert("T", m); m = EmbeddedRelationalStruct.newBuilder() .addString("T_NAME", "nt_record") @@ -94,7 +104,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 +129,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 @@ -121,39 +157,155 @@ void canReadSingleStruct() throws Exception { Assertions.assertEquals("Hello", struct.getString("A"), "Incorrect value for nested struct!"); //check that the JDBC attributes methods work properly - Assertions.assertArrayEquals(struct.getAttributes(), new Object[]{"Hello"}, "Incorrect attributes!"); + Assertions.assertArrayEquals(new Object[]{"Hello"}, struct.getAttributes(), "Incorrect attributes!"); } } + /** + * Helper method to test struct type metadata preservation across query execution and continuations. + * + * @param query The SQL query to execute + * @param assertOnMetaData Consumer to assert on the result set metadata + * @param numBaseQueryRuns Number of times to run the base query (tests PhysicalQueryPlan.withExecutionContext when > 1) + * @param numContinuationRuns Number of times to run the continuation (tests ContinuedPhysicalQueryPlan.withExecutionContext when > 1) + */ + private void canReadStructTypeName(String query, + ThrowingConsumer assertOnMetaData, + int numBaseQueryRuns, + int numContinuationRuns) throws Throwable { + // Only set maxRows if we're testing continuations + if (numContinuationRuns > 0) { + statement.setMaxRows(1); + } + + Continuation continuation = null; + + // Run base query the specified number of times + for (int i = 0; i < numBaseQueryRuns; i++) { + try (final RelationalResultSet resultSet = statement.executeQuery(query)) { + Assertions.assertTrue(resultSet.next(), "Did not find a record on base query run " + (i + 1)); + assertOnMetaData.accept(resultSet); + if (i == 0 && numContinuationRuns > 0) { + continuation = resultSet.getContinuation(); + } + } + } + + // Run continuation the specified number of times + for (int i = 0; i < numContinuationRuns; i++) { + 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 on continuation run " + (i + 1)); + assertOnMetaData.accept(resultSet.unwrap(RelationalResultSet.class)); + } + } + } + } + + private void canReadStructTypeName(String query, ThrowingConsumer assertOnMetaData) throws Throwable { + canReadStructTypeName(query, assertOnMetaData, 1, 1); + } + @Test - void canReadProjectedStructTypeNameInNestedStar() throws Exception { - try (final RelationalResultSet resultSet = statement.executeQuery("SELECT (*) FROM T")) { - Assertions.assertTrue(resultSet.next(), "Did not find a record!"); + 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 @@ -255,4 +407,51 @@ void canReadRepeatedStructWithArray() throws SQLException { } } } + + @Test + void structTypeMetadataPreservedAcrossPlanCache() throws Throwable { + canReadStructTypeName("SELECT * FROM T WHERE NAME = 'test_record_1'", resultSet -> { + RelationalStruct struct = resultSet.getStruct("ST1"); + Assertions.assertEquals("STRUCT_1", struct.getMetaData().getTypeName(), + "Struct type name should be preserved across plan cache"); + }, 2, 0); + } + + @Test + void nestedStructTypeMetadataPreservedAcrossPlanCache() throws Throwable { + canReadStructTypeName("SELECT * FROM NT WHERE T_NAME = 'nt_record'", resultSet -> { + RelationalStruct struct = resultSet.getStruct("ST1"); + RelationalStruct nestedStruct = struct.getStruct("D"); + Assertions.assertEquals("STRUCT_1", nestedStruct.getMetaData().getTypeName(), + "Nested struct type name should be preserved across plan cache"); + }, 2, 0); + } + + @Test + void arrayStructTypeMetadataPreservedAcrossPlanCache() throws Throwable { + canReadStructTypeName("SELECT * FROM AT WHERE A_NAME = 'a_test_rec'", resultSet -> { + RelationalArray array = resultSet.getArray("ST2"); + Assertions.assertEquals("STRUCT_3", array.getMetaData().getElementStructMetaData().getTypeName(), + "Array element struct type name should be preserved across plan cache"); + }, 2, 0); + } + + @Test + void structTypeMetadataPreservedInContinuationAcrossPlanCache() throws Throwable { + canReadStructTypeName("SELECT * FROM T", resultSet -> { + RelationalStruct struct = resultSet.getStruct("ST1"); + Assertions.assertEquals("STRUCT_1", struct.getMetaData().getTypeName(), + "Struct type name should be preserved in continuation across plan cache"); + }, 1, 2); + } + + @Test + void nestedStructTypeMetadataPreservedInContinuationAcrossPlanCache() throws Throwable { + canReadStructTypeName("SELECT * FROM NT", resultSet -> { + RelationalStruct struct = resultSet.getStruct("ST1"); + RelationalStruct nestedStruct = struct.getStruct("D"); + Assertions.assertEquals("STRUCT_1", nestedStruct.getMetaData().getTypeName(), + "Nested struct type name should be preserved in continuation across plan cache"); + }, 1, 2); + } } diff --git a/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/AstNormalizerTests.java b/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/AstNormalizerTests.java index 4680349d60..da5b77b23e 100644 --- a/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/AstNormalizerTests.java +++ b/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/AstNormalizerTests.java @@ -521,26 +521,6 @@ void stripBase64Literal() throws RelationalException { Map.of(constantId(1), ByteString.copyFrom(Hex.decodeHex("cafe")))); } - @Test - void continuationIsStripped() throws Exception { - validate(List.of("select * from t1 with continuation b64'yv4='", - "select * from t1 with continuation x'cafe'", - "select * from t1"), - "select * from \"T1\" "); - } - - @Test - void parseContinuation() throws Exception { - final var expectedContinuationStr = "FBUCFA=="; - validate(List.of("select * from t1 with continuation b64'" + expectedContinuationStr + "'", - "select * from t1 with continuation b64'" + expectedContinuationStr + "'"), - PreparedParams.empty(), - "select * from \"T1\" ", - List.of(Map.of(), Map.of()), - expectedContinuationStr, - -1); - } - @Test void parseInPredicateAllConstants() throws Exception { // although these queries have different number of arguments in their in-predicate @@ -886,27 +866,6 @@ void stripBase64LiteralWithPreparedParameters() throws RelationalException { constantId(5), ByteString.copyFrom(Hex.decodeHex("0B0C")))); } - @Test - void parseContinuationWithPreparedParameters() throws Exception { - final var expectedContinuationStr = "FBUCFA=="; - final var expectedContinuation = Base64.getDecoder().decode(expectedContinuationStr); - validate(List.of("select * from t1 with continuation ?", - "select * from t1 with continuation ? "), - PreparedParams.ofUnnamed(Map.of(1, expectedContinuation)), - "select * from \"T1\" ", - List.of(Map.of(), Map.of()), - expectedContinuationStr, - -1); - - validate(List.of("select * from t1 with continuation ?param", - "select * from t1 with continuation ?param "), - PreparedParams.ofNamed(Map.of("param", expectedContinuation)), - "select * from \"T1\" ", - List.of(Map.of(), Map.of()), - expectedContinuationStr, - -1); - } - @Test void parseInPredicateAllConstantsWithPreparedParameters() throws Exception { // although these queries have different number of arguments in their in-predicate diff --git a/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/ExecutePropertyTests.java b/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/ExecutePropertyTests.java index a3a531d8b1..195993ef93 100644 --- a/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/ExecutePropertyTests.java +++ b/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/ExecutePropertyTests.java @@ -103,8 +103,11 @@ void hitLimitEveryRow(Options.Name optionName, Object optionValue, int expectedR try (var conn = driver.connect(database.getConnectionUri(), Options.builder().withOption(optionName, optionValue).build())) { conn.setSchema("TEST_SCHEMA"); while (!continuation.atEnd()) { - try (var ps = conn.prepareStatement("SELECT * FROM FOO WITH CONTINUATION ?")) { - ps.setBytes(1, continuation.serialize()); + String query = continuation.atBeginning() ? "SELECT * FROM FOO" : "EXECUTE CONTINUATION ?"; + try (var ps = conn.prepareStatement(query)) { + if (!continuation.atBeginning()) { + ps.setBytes(1, continuation.serialize()); + } try (final RelationalResultSet rs = ps.executeQuery()) { for (int currentRowCount = 0; currentRowCount < expectedRowCountPerQuery; currentRowCount++) { if (nextCorrectResult == 17L) { diff --git a/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/ExplainTests.java b/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/ExplainTests.java index 69adc7b138..4e4df8120f 100644 --- a/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/ExplainTests.java +++ b/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/ExplainTests.java @@ -125,7 +125,7 @@ void explainWithContinuationSerializedPlanTest() throws Exception { continuation = consumeResultAndGetContinuation(ps, 2); } - try (RelationalPreparedStatement ps = connection.prepareStatement("EXPLAIN SELECT * FROM RestaurantComplexRecord WITH CONTINUATION ?cont")) { + try (RelationalPreparedStatement ps = connection.prepareStatement("EXPLAIN EXECUTE CONTINUATION ?cont")) { ps.setObject("cont", continuation.serialize()); try (final RelationalResultSet resultSet = ps.executeQuery()) { final var assertResult = ResultSetAssert.assertThat(resultSet); @@ -146,38 +146,6 @@ void explainWithContinuationSerializedPlanTest() throws Exception { } } - @Test - void explainWithContinuationSerializedPlanWithDifferentQueryTest() throws Exception { - try (var ddl = Ddl.builder().database(URI.create("/TEST/QT")).relationalExtension(relationalExtension).schemaTemplate(schemaTemplate).build()) { - executeInsert(ddl); - Continuation continuation; - try (final var connection = ddl.setSchemaAndGetConnection()) { - try (RelationalPreparedStatement ps = ddl.setSchemaAndGetConnection().prepareStatement("SELECT * FROM RestaurantComplexRecord")) { - ps.setMaxRows(2); - continuation = consumeResultAndGetContinuation(ps, 2); - } - - try (RelationalPreparedStatement ps = connection.prepareStatement("EXPLAIN SELECT rest_no FROM RestaurantComplexRecord WITH CONTINUATION ?cont")) { - ps.setObject("cont", continuation.serialize()); - try (final RelationalResultSet resultSet = ps.executeQuery()) { - final var assertResult = ResultSetAssert.assertThat(resultSet); - assertResult.hasNextRow() - .hasColumn("PLAN", "COVERING(RECORD_NAME_IDX <,> -> [NAME: KEY[0], REST_NO: KEY[2]]) | MAP (_.REST_NO AS REST_NO)") - .hasColumn("PLAN_HASH", 4759756); - final var continuationInfo = resultSet.getStruct(5); - org.junit.jupiter.api.Assertions.assertNotNull(continuationInfo); - final var assertStruct = RelationalStructAssert.assertThat(continuationInfo); - assertStruct.hasValue("EXECUTION_STATE", new byte[]{10, 5, 0, 21, 1, 21, 11, 17, -84, -51, 115, -104, -35, 66, 0, 94}); - assertStruct.hasValue("VERSION", 1); - assertStruct.hasValue("PLAN_HASH_MODE", "VC0"); - assertStruct.hasValue("PLAN_HASH", -1635569052); - assertStruct.hasValue("SERIALIZED_PLAN_COMPLEXITY", 1); - } - } - } - } - } - @Test void explainExecuteStatementTest() throws Exception { try (var ddl = Ddl.builder().database(URI.create("/TEST/QT")).relationalExtension(relationalExtension).schemaTemplate(schemaTemplate).build()) { diff --git a/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/GroupByQueryTests.java b/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/GroupByQueryTests.java index bbaef024ce..2cd1e84930 100644 --- a/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/GroupByQueryTests.java +++ b/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/GroupByQueryTests.java @@ -35,7 +35,6 @@ import org.junit.jupiter.api.extension.RegisterExtension; import java.net.URI; -import java.util.Base64; import static com.apple.foundationdb.relational.recordlayer.query.QueryTestUtils.insertT1Record; import static com.apple.foundationdb.relational.recordlayer.query.QueryTestUtils.insertT1RecordColAIsNull; @@ -149,13 +148,15 @@ void groupByWithRowLimit() throws Exception { .hasNoNextRow(); continuation = resultSet.getContinuation(); } - String postfix = " WITH CONTINUATION B64'" + Base64.getEncoder().encodeToString(continuation.serialize()) + "'"; - Assertions.assertTrue(statement.execute(query + postfix), "Did not return a result set from a select statement!"); - try (final RelationalResultSet resultSet = statement.getResultSet()) { - ResultSetAssert.assertThat(resultSet).hasNextRow() - .isRowExactly(9.5) - .hasNoNextRow(); - continuation = resultSet.getContinuation(); + try (var ps = conn.prepareStatement("EXECUTE CONTINUATION ?continuation")) { + ps.setBytes("continuation", continuation.serialize()); + Assertions.assertTrue(ps.execute(), "Did not return a result set from a select statement!"); + try (final RelationalResultSet resultSet = ps.getResultSet()) { + ResultSetAssert.assertThat(resultSet).hasNextRow() + .isRowExactly(9.5) + .hasNoNextRow(); + continuation = resultSet.getContinuation(); + } } } } diff --git a/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/PreparedStatementTests.java b/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/PreparedStatementTests.java index 657f586dc9..6487a2afc9 100644 --- a/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/PreparedStatementTests.java +++ b/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/PreparedStatementTests.java @@ -345,7 +345,8 @@ void continuation() throws Exception { statement.execute("INSERT INTO RestaurantComplexRecord(rest_no) VALUES (10), (11), (12), (13), (14)"); } Continuation continuation; - try (var ps = ddl.setSchemaAndGetConnection().prepareStatement("SELECT * FROM RestaurantComplexRecord")) { + final var connection = ddl.setSchemaAndGetConnection(); + try (var ps = connection.prepareStatement("SELECT * FROM RestaurantComplexRecord")) { ps.setMaxRows(2); try (final RelationalResultSet resultSet = ps.executeQuery()) { ResultSetAssert.assertThat(resultSet) @@ -355,7 +356,7 @@ void continuation() throws Exception { continuation = resultSet.getContinuation(); } } - try (var ps = ddl.setSchemaAndGetConnection().prepareStatement("SELECT * FROM RestaurantComplexRecord WITH CONTINUATION ?continuation")) { + try (var ps = connection.prepareStatement("EXECUTE CONTINUATION ?continuation")) { ps.setMaxRows(2); ps.setBytes("continuation", continuation.serialize()); try (final RelationalResultSet resultSet = ps.executeQuery()) { @@ -374,7 +375,7 @@ void continuation() throws Exception { } // Same but with logs - try (var ps = ddl.setSchemaAndGetConnection().prepareStatement("SELECT * FROM RestaurantComplexRecord OPTIONS(LOG QUERY)")) { + try (var ps = connection.prepareStatement("SELECT * FROM RestaurantComplexRecord OPTIONS(LOG QUERY)")) { ps.setMaxRows(2); try (final RelationalResultSet resultSet = ps.executeQuery()) { ResultSetAssert.assertThat(resultSet) @@ -385,7 +386,7 @@ void continuation() throws Exception { } } Assertions.assertThat(logAppender.getLastLogEventMessage()).contains("planCache=\"hit\""); - try (var ps = ddl.setSchemaAndGetConnection().prepareStatement("SELECT * FROM RestaurantComplexRecord OPTIONS(LOG QUERY) WITH CONTINUATION ?continuation")) { + try (var ps = connection.prepareStatement("EXECUTE CONTINUATION ?continuation OPTIONS(LOG QUERY)")) { ps.setMaxRows(2); ps.setBytes("continuation", continuation.serialize()); try (final RelationalResultSet resultSet = ps.executeQuery()) { @@ -395,14 +396,15 @@ void continuation() throws Exception { .hasNoNextRow(); continuation = resultSet.getContinuation(); } - Assertions.assertThat(logAppender.getLastLogEventMessage()).contains("planCache=\"hit\""); + // With EXECUTE CONTINUATION, the plan is embedded in the continuation, so cache is skipped + Assertions.assertThat(logAppender.getLastLogEventMessage()).contains("planCache=\"skip\""); ps.setBytes("continuation", continuation.serialize()); try (final RelationalResultSet resultSet = ps.executeQuery()) { ResultSetAssert.assertThat(resultSet) .hasNextRow().hasColumn("REST_NO", 14L) .hasNoNextRow(); } - Assertions.assertThat(logAppender.getLastLogEventMessage()).contains("planCache=\"hit\""); + Assertions.assertThat(logAppender.getLastLogEventMessage()).contains("planCache=\"skip\""); } } } diff --git a/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/QueryWithContinuationTest.java b/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/QueryWithContinuationTest.java index 2e5a0f972a..53f75a069d 100644 --- a/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/QueryWithContinuationTest.java +++ b/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/QueryWithContinuationTest.java @@ -27,7 +27,6 @@ import com.apple.foundationdb.relational.api.RelationalPreparedStatement; import com.apple.foundationdb.relational.api.RelationalResultSet; import com.apple.foundationdb.relational.api.RelationalStatement; -import com.apple.foundationdb.relational.api.exceptions.RelationalException; import com.apple.foundationdb.relational.api.metrics.RelationalMetric; import com.apple.foundationdb.relational.continuation.ContinuationProto; import com.apple.foundationdb.relational.recordlayer.ContinuationImpl; @@ -44,7 +43,6 @@ import java.net.URI; import java.sql.SQLException; -import java.util.Base64; import java.util.Objects; public class QueryWithContinuationTest { @@ -80,7 +78,7 @@ void preparedStatement() throws Exception { continuation = assertResult(ps, 10L, 11L); assertContinuation(continuation, false, false); } - try (RelationalPreparedStatement ps = ddl.setSchemaAndGetConnection().prepareStatement("SELECT * FROM RestaurantComplexRecord WITH CONTINUATION ?continuation")) { + try (RelationalPreparedStatement ps = ddl.setSchemaAndGetConnection().prepareStatement("EXECUTE CONTINUATION ?continuation")) { ps.setMaxRows(2); ps.setBytes("continuation", continuation.serialize()); continuation = assertResult(ps, 12L, 13L); @@ -128,7 +126,7 @@ void preparedStatementWithLimit() throws Exception { continuation = assertResult(ps, 10L, 11L); assertContinuation(continuation, false, false); } - try (RelationalPreparedStatement ps = ddl.setSchemaAndGetConnection().prepareStatement("SELECT * FROM RestaurantComplexRecord WITH CONTINUATION ?continuation")) { + try (RelationalPreparedStatement ps = ddl.setSchemaAndGetConnection().prepareStatement("EXECUTE CONTINUATION ?continuation")) { ps.setBytes("continuation", continuation.serialize()); ps.setMaxRows(2); continuation = assertResult(ps, 12L, 13L); @@ -146,22 +144,23 @@ void preparedStatementWithParam() throws Exception { try (var ddl = Ddl.builder().database(URI.create("/TEST/QT")).relationalExtension(relationalExtension).schemaTemplate(schemaTemplate).build()) { executeInsert(ddl); Continuation continuation; - try (RelationalPreparedStatement ps = ddl.setSchemaAndGetConnection().prepareStatement("SELECT * FROM RestaurantComplexRecord WHERE REST_NO > ?p")) { - ps.setMaxRows(2); - ps.setInt("p", 9); - continuation = assertResult(ps, 10L, 11L); - assertContinuation(continuation, false, false); - } - try (RelationalPreparedStatement ps = ddl.setSchemaAndGetConnection().prepareStatement("SELECT * FROM RestaurantComplexRecord WHERE REST_NO > ?p WITH CONTINUATION ?continuation")) { - ps.setBytes("continuation", continuation.serialize()); - ps.setMaxRows(2); - ps.setInt("p", 9); - continuation = assertResult(ps, 12L, 13L); - assertContinuation(continuation, false, false); + try (final var connection = ddl.setSchemaAndGetConnection()) { + try (var statement = connection.prepareStatement("SELECT * FROM RestaurantComplexRecord WHERE REST_NO > ?p")) { + statement.setMaxRows(2); + statement.setInt("p", 9); + continuation = assertResult(statement, 10L, 11L); + assertContinuation(continuation, false, false); + } + try (var statement = connection.prepareStatement("EXECUTE CONTINUATION ?continuation")) { + statement.setBytes("continuation", continuation.serialize()); + statement.setMaxRows(2); + continuation = assertResult(statement, 12L, 13L); + assertContinuation(continuation, false, false); - ps.setBytes("continuation", continuation.serialize()); - continuation = assertResult(ps, 14L); - assertContinuation(continuation, false, true); + statement.setBytes("continuation", continuation.serialize()); + continuation = assertResult(statement, 14L); + assertContinuation(continuation, false, true); + } } } } @@ -171,21 +170,22 @@ void preparedStatementWithLiteral() throws Exception { try (var ddl = Ddl.builder().database(URI.create("/TEST/QT")).relationalExtension(relationalExtension).schemaTemplate(schemaTemplate).build()) { executeInsert(ddl); Continuation continuation; - try (RelationalPreparedStatement ps = ddl.setSchemaAndGetConnection().prepareStatement("SELECT * FROM RestaurantComplexRecord WHERE REST_NO > 9")) { - ps.setMaxRows(2); - continuation = assertResult(ps, 10L, 11L); - assertContinuation(continuation, false, false); - } - try (RelationalPreparedStatement ps = ddl.setSchemaAndGetConnection().prepareStatement("SELECT * FROM RestaurantComplexRecord WHERE REST_NO > 9 WITH CONTINUATION ?continuation")) { - ps.setMaxRows(2); - ps.setBytes("continuation", continuation.serialize()); - ps.setInt("l", 2); - continuation = assertResult(ps, 12L, 13L); - assertContinuation(continuation, false, false); + try (final var connection = ddl.setSchemaAndGetConnection()) { + try (var statement = connection.prepareStatement("SELECT * FROM RestaurantComplexRecord WHERE REST_NO > 9")) { + statement.setMaxRows(2); + continuation = assertResult(statement, 10L, 11L); + assertContinuation(continuation, false, false); + } + try (var statement = connection.prepareStatement("EXECUTE CONTINUATION ?continuation")) { + statement.setMaxRows(2); + statement.setBytes("continuation", continuation.serialize()); + continuation = assertResult(statement, 12L, 13L); + assertContinuation(continuation, false, false); - ps.setBytes("continuation", continuation.serialize()); - continuation = assertResult(ps, 14L); - assertContinuation(continuation, false, true); + statement.setBytes("continuation", continuation.serialize()); + continuation = assertResult(statement, 14L); + assertContinuation(continuation, false, true); + } } } } @@ -195,16 +195,18 @@ void preparedStatementWithDifferentLimit() throws Exception { try (var ddl = Ddl.builder().database(URI.create("/TEST/QT")).relationalExtension(relationalExtension).schemaTemplate(schemaTemplate).build()) { executeInsert(ddl); Continuation continuation; - try (RelationalPreparedStatement ps = ddl.setSchemaAndGetConnection().prepareStatement("SELECT * FROM RestaurantComplexRecord WHERE REST_NO > 9")) { - ps.setMaxRows(2); - continuation = assertResult(ps, 10L, 11L); - assertContinuation(continuation, false, false); - } - try (RelationalPreparedStatement ps = ddl.setSchemaAndGetConnection().prepareStatement("SELECT * FROM RestaurantComplexRecord WHERE REST_NO > 9 WITH CONTINUATION ?continuation")) { - ps.setMaxRows(4); - ps.setBytes("continuation", continuation.serialize()); - continuation = assertResult(ps, 12L, 13L, 14L); - assertContinuation(continuation, false, true); + try (final var connection = ddl.setSchemaAndGetConnection()) { + try (var statement = connection.prepareStatement("SELECT * FROM RestaurantComplexRecord WHERE REST_NO > 9")) { + statement.setMaxRows(2); + continuation = assertResult(statement, 10L, 11L); + assertContinuation(continuation, false, false); + } + try (var statement = connection.prepareStatement("EXECUTE CONTINUATION ?continuation")) { + statement.setMaxRows(4); + statement.setBytes("continuation", continuation.serialize()); + continuation = assertResult(statement, 12L, 13L, 14L); + assertContinuation(continuation, false, true); + } } } } @@ -214,146 +216,18 @@ void preparedStatementWithDifferentLimitParam() throws Exception { try (var ddl = Ddl.builder().database(URI.create("/TEST/QT")).relationalExtension(relationalExtension).schemaTemplate(schemaTemplate).build()) { executeInsert(ddl); Continuation continuation; - try (RelationalPreparedStatement ps = ddl.setSchemaAndGetConnection().prepareStatement("SELECT * FROM RestaurantComplexRecord WHERE REST_NO > 9")) { - ps.setMaxRows(2); - continuation = assertResult(ps, 10L, 11L); - assertContinuation(continuation, false, false); - } - try (RelationalPreparedStatement ps = ddl.setSchemaAndGetConnection().prepareStatement("SELECT * FROM RestaurantComplexRecord WHERE REST_NO > 9 WITH CONTINUATION ?continuation")) { - ps.setBytes("continuation", continuation.serialize()); - ps.setMaxRows(4); - continuation = assertResult(ps, 12L, 13L, 14L); - assertContinuation(continuation, false, true); - } - } - } - - @Test - void preparedStatementInitialContEmpty() throws Exception { - try (var ddl = Ddl.builder().database(URI.create("/TEST/QT")).relationalExtension(relationalExtension).schemaTemplate(schemaTemplate).build()) { - executeInsert(ddl); - Continuation continuation; - try (RelationalPreparedStatement ps = ddl.setSchemaAndGetConnection().prepareStatement("SELECT * FROM RestaurantComplexRecord WHERE REST_NO > ?p WITH CONTINUATION ?continuation")) { - ps.setMaxRows(2); - ps.setBytes("continuation", new byte[0]); - ps.setInt("p", 9); - continuation = assertResult(ps, 10L, 11L); - assertContinuation(continuation, false, false); - - ps.setBytes("continuation", continuation.serialize()); - continuation = assertResult(ps, 12L, 13L); - assertContinuation(continuation, false, false); - - ps.setBytes("continuation", continuation.serialize()); - continuation = assertResult(ps, 14L); - assertContinuation(continuation, false, true); - } - } - } - - @Test - void preparedStatementWithParamChangedFails() throws Exception { - try (var ddl = Ddl.builder().database(URI.create("/TEST/QT")).relationalExtension(relationalExtension).schemaTemplate(schemaTemplate).build()) { - executeInsert(ddl); - Continuation continuation; - try (RelationalPreparedStatement ps = ddl.setSchemaAndGetConnection().prepareStatement("SELECT * FROM RestaurantComplexRecord WHERE REST_NO > ?p")) { - ps.setMaxRows(2); - ps.setInt("p", 9); - continuation = assertResult(ps, 10L, 11L); - assertContinuation(continuation, false, false); - } - try (RelationalPreparedStatement ps = ddl.setSchemaAndGetConnection().prepareStatement("SELECT * FROM RestaurantComplexRecord WHERE REST_NO > ?p WITH CONTINUATION ?continuation")) { - ps.setMaxRows(2); - ps.setBytes("continuation", continuation.serialize()); - ps.setInt("p", 10); - Assertions.assertThatThrownBy(ps::executeQuery) - .hasCauseInstanceOf(RelationalException.class) - .hasMessageContaining("Continuation binding does not match query"); - } - } - } - - @Test - void preparedStatementWithLiteralChangedFails() throws Exception { - try (var ddl = Ddl.builder().database(URI.create("/TEST/QT")).relationalExtension(relationalExtension).schemaTemplate(schemaTemplate).build()) { - executeInsert(ddl); - Continuation continuation; - try (RelationalPreparedStatement ps = ddl.setSchemaAndGetConnection().prepareStatement("SELECT * FROM RestaurantComplexRecord WHERE REST_NO > 9")) { - ps.setMaxRows(2); - continuation = assertResult(ps, 10L, 11L); - assertContinuation(continuation, false, false); - } - try (RelationalPreparedStatement ps = ddl.setSchemaAndGetConnection().prepareStatement("SELECT * FROM RestaurantComplexRecord WHERE REST_NO > 10 WITH CONTINUATION ?continuation")) { - ps.setMaxRows(2); - ps.setBytes("continuation", continuation.serialize()); - Assertions.assertThatThrownBy(ps::executeQuery) - .hasCauseInstanceOf(RelationalException.class) - .hasMessageContaining("Continuation binding does not match query"); - } - } - } - - @Test - void preparedStatementWithLiteralChangedToParamFails() throws Exception { - try (var ddl = Ddl.builder().database(URI.create("/TEST/QT")).relationalExtension(relationalExtension).schemaTemplate(schemaTemplate).build()) { - executeInsert(ddl); - Continuation continuation; - try (RelationalPreparedStatement ps = ddl.setSchemaAndGetConnection().prepareStatement("SELECT * FROM RestaurantComplexRecord WHERE REST_NO > 9")) { - ps.setMaxRows(2); - continuation = assertResult(ps, 10L, 11L); - assertContinuation(continuation, false, false); - } - try (RelationalPreparedStatement ps = ddl.setSchemaAndGetConnection().prepareStatement("SELECT * FROM RestaurantComplexRecord WHERE REST_NO > ?p WITH CONTINUATION ?continuation")) { - ps.setMaxRows(2); - ps.setBytes("continuation", continuation.serialize()); - ps.setInt("p", 9); - Assertions.assertThatThrownBy(ps::executeQuery) - .hasCauseInstanceOf(RelationalException.class) - .hasMessageContaining("Continuation binding does not match query"); - } - } - } - - @Test - void preparedStatementWithParamNameChangedFails() throws Exception { - try (var ddl = Ddl.builder().database(URI.create("/TEST/QT")).relationalExtension(relationalExtension).schemaTemplate(schemaTemplate).build()) { - executeInsert(ddl); - Continuation continuation; - try (RelationalPreparedStatement ps = ddl.setSchemaAndGetConnection().prepareStatement("SELECT * FROM RestaurantComplexRecord WHERE REST_NO > ?p")) { - ps.setMaxRows(2); - ps.setInt("p", 9); - continuation = assertResult(ps, 10L, 11L); - assertContinuation(continuation, false, false); - } - try (RelationalPreparedStatement ps = ddl.setSchemaAndGetConnection().prepareStatement("SELECT * FROM RestaurantComplexRecord WHERE REST_NO > ?otherName WITH CONTINUATION ?continuation")) { - ps.setMaxRows(2); - ps.setBytes("continuation", continuation.serialize()); - ps.setInt("otherName", 9); - Assertions.assertThatThrownBy(() -> ps.executeQuery()) - .hasCauseInstanceOf(RelationalException.class) - .hasMessageContaining("Continuation binding does not match query"); - } - } - } - - @Test - void preparedStatementWithPlanChangedFails() throws Exception { - try (var ddl = Ddl.builder().database(URI.create("/TEST/QT")).relationalExtension(relationalExtension).schemaTemplate(schemaTemplate).build()) { - executeInsert(ddl); - Continuation continuation; - try (RelationalPreparedStatement ps = ddl.setSchemaAndGetConnection().prepareStatement("SELECT * FROM RestaurantComplexRecord WHERE REST_NO > ?p")) { - ps.setMaxRows(2); - ps.setInt("p", 9); - continuation = assertResult(ps, 10L, 11L); - assertContinuation(continuation, false, false); - } - try (RelationalPreparedStatement ps = ddl.setSchemaAndGetConnection().prepareStatement("SELECT REST_NO FROM RestaurantComplexRecord WHERE REST_NO > ?p WITH CONTINUATION ?continuation")) { - ps.setMaxRows(2); - ps.setBytes("continuation", continuation.serialize()); - ps.setInt("p", 9); - Assertions.assertThatThrownBy(() -> ps.executeQuery()) - .hasCauseInstanceOf(RelationalException.class) - .hasMessageContaining("Continuation plan does not match query"); + try (final var connection = ddl.setSchemaAndGetConnection()) { + try (var statement = connection.prepareStatement("SELECT * FROM RestaurantComplexRecord WHERE REST_NO > 9")) { + statement.setMaxRows(2); + continuation = assertResult(statement, 10L, 11L); + assertContinuation(continuation, false, false); + } + try (var statement = connection.prepareStatement("EXECUTE CONTINUATION ?continuation")) { + statement.setBytes("continuation", continuation.serialize()); + statement.setMaxRows(4); + continuation = assertResult(statement, 12L, 13L, 14L); + assertContinuation(continuation, false, true); + } } } } @@ -363,36 +237,39 @@ void standardStatement() throws Exception { try (var ddl = Ddl.builder().database(URI.create("/TEST/QT")).relationalExtension(relationalExtension).schemaTemplate(schemaTemplate).build()) { executeInsert(ddl); Continuation continuation; - try (RelationalStatement statement = ddl.setSchemaAndGetConnection().createStatement()) { - statement.setMaxRows(2); - try (final RelationalResultSet resultSet = statement.executeQuery("SELECT * FROM RestaurantComplexRecord")) { - ResultSetAssert.assertThat(resultSet) - .hasNextRow().hasColumn("REST_NO", 10L) - .hasNextRow().hasColumn("REST_NO", 11L) - .hasNoNextRow(); - continuation = resultSet.getContinuation(); - assertContinuation(continuation, false, false); - } - } - try (RelationalStatement statement = ddl.setSchemaAndGetConnection().createStatement()) { - String continuationString = Base64.getEncoder().encodeToString(continuation.serialize()); - statement.setMaxRows(2); - try (final RelationalResultSet resultSet = statement.executeQuery("SELECT * FROM RestaurantComplexRecord WITH CONTINUATION B64'" + continuationString + "'")) { - ResultSetAssert.assertThat(resultSet) - .hasNextRow().hasColumn("REST_NO", 12L) - .hasNextRow().hasColumn("REST_NO", 13L) - .hasNoNextRow(); - continuation = resultSet.getContinuation(); - assertContinuation(continuation, false, false); + try (final var connection = ddl.setSchemaAndGetConnection()) { + try (RelationalStatement statement = connection.createStatement()) { + statement.setMaxRows(2); + try (final RelationalResultSet resultSet = statement.executeQuery("SELECT * FROM RestaurantComplexRecord")) { + ResultSetAssert.assertThat(resultSet) + .hasNextRow().hasColumn("REST_NO", 10L) + .hasNextRow().hasColumn("REST_NO", 11L) + .hasNoNextRow(); + continuation = resultSet.getContinuation(); + assertContinuation(continuation, false, false); + } } - continuationString = Base64.getEncoder().encodeToString(continuation.serialize()); - statement.setMaxRows(2); - try (final RelationalResultSet resultSet = statement.executeQuery("SELECT * FROM RestaurantComplexRecord WITH CONTINUATION B64'" + continuationString + "'")) { - ResultSetAssert.assertThat(resultSet) - .hasNextRow().hasColumn("REST_NO", 14L) - .hasNoNextRow(); - continuation = resultSet.getContinuation(); - assertContinuation(continuation, false, true); + try (var statement = connection.prepareStatement("EXECUTE CONTINUATION ?continuation")) { + statement.setMaxRows(2); + statement.setBytes("continuation", continuation.serialize()); + try (final RelationalResultSet resultSet = statement.executeQuery()) { + ResultSetAssert.assertThat(resultSet) + .hasNextRow().hasColumn("REST_NO", 12L) + .hasNextRow().hasColumn("REST_NO", 13L) + .hasNoNextRow(); + continuation = resultSet.getContinuation(); + assertContinuation(continuation, false, false); + } + + statement.setBytes("continuation", continuation.serialize()); + statement.setMaxRows(2); + try (final RelationalResultSet resultSet = statement.executeQuery()) { + ResultSetAssert.assertThat(resultSet) + .hasNextRow().hasColumn("REST_NO", 14L) + .hasNoNextRow(); + continuation = resultSet.getContinuation(); + assertContinuation(continuation, false, true); + } } } } @@ -427,9 +304,9 @@ void standardStatementWithDifferentPlanHashModes() throws Exception { PlanHashable.PlanHashMode.VL0.name() + "," + PlanHashable.PlanHashMode.VC0.name()); connection.setOption(Options.Name.CURRENT_PLAN_HASH_MODE, PlanHashable.PlanHashMode.VC0.name()); try (RelationalStatement statement = connection.createStatement()) { - String continuationString = Base64.getEncoder().encodeToString(continuation.serialize()); + String continuationString = java.util.Base64.getEncoder().encodeToString(continuation.serialize()); statement.setMaxRows(2); - try (final RelationalResultSet resultSet = statement.executeQuery("SELECT * FROM RestaurantComplexRecord WITH CONTINUATION B64'" + continuationString + "'")) { + try (final RelationalResultSet resultSet = statement.executeQuery("EXECUTE CONTINUATION B64'" + continuationString + "'")) { ResultSetAssert.assertThat(resultSet) .hasNextRow().hasColumn("REST_NO", 12L) .hasNextRow().hasColumn("REST_NO", 13L) @@ -442,16 +319,16 @@ void standardStatementWithDifferentPlanHashModes() throws Exception { Assertions.assertThat(metricCollector.hasCounter(RelationalMetric.RelationalCount.CONTINUATION_ACCEPTED)).isTrue(); Assertions.assertThat(metricCollector.getCountsForCounter(RelationalMetric.RelationalCount.CONTINUATION_ACCEPTED)).isEqualTo(1L); Assertions.assertThat(metricCollector.hasCounter(RelationalMetric.RelationalCount.CONTINUATION_DOWN_LEVEL)).isTrue(); - Assertions.assertThat(metricCollector.getCountsForCounter(RelationalMetric.RelationalCount.CONTINUATION_DOWN_LEVEL)).isEqualTo(2L); + Assertions.assertThat(metricCollector.getCountsForCounter(RelationalMetric.RelationalCount.CONTINUATION_DOWN_LEVEL)).isEqualTo(1L); } } connection.setOption(Options.Name.VALID_PLAN_HASH_MODES, PlanHashable.PlanHashMode.VC0.name()); connection.setOption(Options.Name.CURRENT_PLAN_HASH_MODE, PlanHashable.PlanHashMode.VC0.name()); try (RelationalStatement statement = connection.createStatement()) { - String continuationString = Base64.getEncoder().encodeToString(continuation.serialize()); + String continuationString = java.util.Base64.getEncoder().encodeToString(continuation.serialize()); statement.setMaxRows(2); - try (final RelationalResultSet resultSet = statement.executeQuery("SELECT * FROM RestaurantComplexRecord WITH CONTINUATION B64'" + continuationString + "'")) { + try (final RelationalResultSet resultSet = statement.executeQuery("EXECUTE CONTINUATION B64'" + continuationString + "'")) { ResultSetAssert.assertThat(resultSet) .hasNextRow().hasColumn("REST_NO", 14L) .hasNoNextRow(); @@ -462,7 +339,7 @@ void standardStatementWithDifferentPlanHashModes() throws Exception { final var metricCollector = Objects.requireNonNull(embeddedRelationalConnection.getMetricCollector()); Assertions.assertThat(metricCollector.hasCounter(RelationalMetric.RelationalCount.CONTINUATION_ACCEPTED)).isTrue(); Assertions.assertThat(metricCollector.getCountsForCounter(RelationalMetric.RelationalCount.CONTINUATION_ACCEPTED)).isEqualTo(1L); - Assertions.assertThat(metricCollector.hasCounter(RelationalMetric.RelationalCount.CONTINUATION_DOWN_LEVEL)).isTrue(); + // No down-level in this execution (continuation was created with VC0, executed with VC0) } } } @@ -474,90 +351,40 @@ void standardStatementWithLiterals() throws Exception { try (var ddl = Ddl.builder().database(URI.create("/TEST/QT")).relationalExtension(relationalExtension).schemaTemplate(schemaTemplate).build()) { executeInsert(ddl); Continuation continuation; - try (RelationalStatement statement = ddl.setSchemaAndGetConnection().createStatement()) { - statement.setMaxRows(2); - try (final RelationalResultSet resultSet = statement.executeQuery("SELECT * FROM RestaurantComplexRecord WHERE REST_NO > 9")) { - ResultSetAssert.assertThat(resultSet) - .hasNextRow().hasColumn("REST_NO", 10L) - .hasNextRow().hasColumn("REST_NO", 11L) - .hasNoNextRow(); - continuation = resultSet.getContinuation(); - assertContinuation(continuation, false, false); - } - } - try (RelationalStatement statement = ddl.setSchemaAndGetConnection().createStatement()) { - String continuationString = Base64.getEncoder().encodeToString(continuation.serialize()); - statement.setMaxRows(2); - try (final RelationalResultSet resultSet = statement.executeQuery("SELECT * FROM RestaurantComplexRecord WHERE REST_NO > 9 WITH CONTINUATION B64'" + continuationString + "'")) { - ResultSetAssert.assertThat(resultSet) - .hasNextRow().hasColumn("REST_NO", 12L) - .hasNextRow().hasColumn("REST_NO", 13L) - .hasNoNextRow(); - continuation = resultSet.getContinuation(); - assertContinuation(continuation, false, false); - } - continuationString = Base64.getEncoder().encodeToString(continuation.serialize()); - statement.setMaxRows(2); - try (final RelationalResultSet resultSet = statement.executeQuery("SELECT * FROM RestaurantComplexRecord WHERE REST_NO > 9 WITH CONTINUATION B64'" + continuationString + "'")) { - ResultSetAssert.assertThat(resultSet) - .hasNextRow().hasColumn("REST_NO", 14L) - .hasNoNextRow(); - continuation = resultSet.getContinuation(); - assertContinuation(continuation, false, true); - } - } - } - } - - @Test - void standardStatementWithDifferentLiteralFails() throws Exception { - try (var ddl = Ddl.builder().database(URI.create("/TEST/QT")).relationalExtension(relationalExtension).schemaTemplate(schemaTemplate).build()) { - executeInsert(ddl); - Continuation continuation; - try (RelationalStatement statement = ddl.setSchemaAndGetConnection().createStatement()) { - statement.setMaxRows(2); - try (final RelationalResultSet resultSet = statement.executeQuery("SELECT * FROM RestaurantComplexRecord WHERE REST_NO > 9")) { - ResultSetAssert.assertThat(resultSet) - .hasNextRow().hasColumn("REST_NO", 10L) - .hasNextRow().hasColumn("REST_NO", 11L) - .hasNoNextRow(); - continuation = resultSet.getContinuation(); - assertContinuation(continuation, false, false); + try (final var connection = ddl.setSchemaAndGetConnection()) { + try (RelationalStatement statement = connection.createStatement()) { + statement.setMaxRows(2); + try (final RelationalResultSet resultSet = statement.executeQuery("SELECT * FROM RestaurantComplexRecord WHERE REST_NO > 9")) { + ResultSetAssert.assertThat(resultSet) + .hasNextRow().hasColumn("REST_NO", 10L) + .hasNextRow().hasColumn("REST_NO", 11L) + .hasNoNextRow(); + continuation = resultSet.getContinuation(); + assertContinuation(continuation, false, false); + } } - } - try (RelationalStatement statement = ddl.setSchemaAndGetConnection().createStatement()) { - String continuationString = Base64.getEncoder().encodeToString(continuation.serialize()); - statement.setMaxRows(2); - Assertions.assertThatThrownBy(() -> statement.executeQuery("SELECT REST_NO FROM RestaurantComplexRecord WHERE REST_NO > 10 WITH CONTINUATION B64'" + continuationString + "'")) - .hasCauseInstanceOf(RelationalException.class) - .hasMessageContaining("Continuation binding does not match query"); - } - } - } - - @Test - void standardStatementWithDifferentPlanFails() throws Exception { - try (var ddl = Ddl.builder().database(URI.create("/TEST/QT")).relationalExtension(relationalExtension).schemaTemplate(schemaTemplate).build()) { - executeInsert(ddl); - Continuation continuation; - try (RelationalStatement statement = ddl.setSchemaAndGetConnection().createStatement()) { - statement.setMaxRows(2); - try (final RelationalResultSet resultSet = statement.executeQuery("SELECT * FROM RestaurantComplexRecord")) { - ResultSetAssert.assertThat(resultSet) - .hasNextRow().hasColumn("REST_NO", 10L) - .hasNextRow().hasColumn("REST_NO", 11L) - .hasNoNextRow(); - continuation = resultSet.getContinuation(); - assertContinuation(continuation, false, false); + try (var statement = connection.prepareStatement("EXECUTE CONTINUATION ?continuation")) { + statement.setMaxRows(2); + statement.setBytes("continuation", continuation.serialize()); + try (final RelationalResultSet resultSet = statement.executeQuery()) { + ResultSetAssert.assertThat(resultSet) + .hasNextRow().hasColumn("REST_NO", 12L) + .hasNextRow().hasColumn("REST_NO", 13L) + .hasNoNextRow(); + continuation = resultSet.getContinuation(); + assertContinuation(continuation, false, false); + } + statement.setMaxRows(2); + statement.setBytes("continuation", continuation.serialize()); + try (final RelationalResultSet resultSet = statement.executeQuery()) { + ResultSetAssert.assertThat(resultSet) + .hasNextRow().hasColumn("REST_NO", 14L) + .hasNoNextRow(); + continuation = resultSet.getContinuation(); + assertContinuation(continuation, false, true); + } } } - try (RelationalStatement statement = ddl.setSchemaAndGetConnection().createStatement()) { - String continuationString = Base64.getEncoder().encodeToString(continuation.serialize()); - statement.setMaxRows(2); - Assertions.assertThatThrownBy(() -> statement.executeQuery("SELECT REST_NO FROM RestaurantComplexRecord WITH CONTINUATION B64'" + continuationString + "'")) - .hasCauseInstanceOf(RelationalException.class) - .hasMessageContaining("Continuation plan does not match query"); - } } } diff --git a/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/StandardQueryTests.java b/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/StandardQueryTests.java index 8e3a61cc03..615bb8fb55 100644 --- a/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/StandardQueryTests.java +++ b/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/StandardQueryTests.java @@ -385,15 +385,14 @@ void selectWithContinuation() throws Exception { RelationalStruct l43 = insertRestaurantComplexRecord(statement, 43L, "rest1"); RelationalStruct l44 = insertRestaurantComplexRecord(statement, 44L, "rest1"); RelationalStruct l45 = insertRestaurantComplexRecord(statement, 45L, "rest2"); - final String initialQuery = "select * from RestaurantComplexRecord where rest_no > 40"; + String query = "select * from RestaurantComplexRecord where rest_no > 40"; Continuation continuation = ContinuationImpl.BEGIN; final List expected = List.of(l42, l43, l44, l45); int i = 0; while (!continuation.atEnd()) { - String query = initialQuery; if (!continuation.atBeginning()) { - query += " WITH CONTINUATION B64'" + Base64.getEncoder().encodeToString(continuation.serialize()) + "'"; + query = "EXECUTE CONTINUATION B64'" + Base64.getEncoder().encodeToString(continuation.serialize()) + "'"; } try (final RelationalResultSet resultSet = statement.executeQuery(query)) { // assert result matches expected @@ -424,7 +423,7 @@ void selectWithContinuationBeginEndShouldFail() throws Exception { .hasErrorCode(ErrorCode.SYNTAX_ERROR); final String end = "select * from RestaurantComplexRecord where rest_no > 40 with continuation b64''"; RelationalAssertions.assertThrowsSqlException(() -> statement.executeQuery(end)) - .hasErrorCode(ErrorCode.INVALID_CONTINUATION); + .hasErrorCode(ErrorCode.SYNTAX_ERROR); } } } diff --git a/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/UpdateTest.java b/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/UpdateTest.java index b6fa92d4fc..a5c11bbd3a 100644 --- a/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/UpdateTest.java +++ b/fdb-relational-core/src/test/java/com/apple/foundationdb/relational/recordlayer/query/UpdateTest.java @@ -179,10 +179,9 @@ public void insertRecords(int numRecords) throws RelationalException, SQLExcepti } private static RelationalPreparedStatement prepareUpdate(RelationalConnection conn, String updateField, Object param, Continuation continuation) throws SQLException { - if (continuation.atBeginning() || !((ContinuationImpl) continuation).hasCompiledStatement()) { - final var statement = conn.prepareStatement("UPDATE RestaurantReviewer SET " + updateField + " = ?param WHERE id >= 0 RETURNING \"new\"." + updateField + ", \"new\".id WITH CONTINUATION ?cont"); + if (continuation.atBeginning()) { + final var statement = conn.prepareStatement("UPDATE RestaurantReviewer SET " + updateField + " = ?param WHERE id >= 0 RETURNING \"new\"." + updateField + ", \"new\".id"); statement.setObject("param", param); - statement.setObject("cont", continuation.serialize()); return statement; } else { final var statement = conn.prepareStatement("EXECUTE CONTINUATION ?cont");