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:
+ *
+ *
Field names from the planner's Type.Record (which handles aliases, star expansion, etc.)
+ *
Type structure from semantic DataTypes (which preserves struct type names like "STRUCT_1")
+ *
Additional enrichment from RecordMetaData descriptors for nested types
+ *
+ *
+ *
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:
+ *
+ *
Field names and field count from planner Type.Record
+ * (planner handles aliases, star expansion, and "_0" naming for unnamed expressions)
+ *
Type structure (especially nested struct type names) from semantic DataTypes
+ * (semantic analysis preserves "STRUCT_1", "STRUCT_2" which planner loses)
+ *
Additionally enrich nested structs with RecordMetaData descriptor names
+ *
+ *
+ * @param plannerType The Type.Record from the physical plan (has correct field names)
+ * @param semanticTypes The semantic DataTypes captured before planning (have struct type names)
+ * @param recordMetaData Schema metadata for enriching nested types
+ * @return Merged DataType.StructType with planner names and semantic type structure
+ * @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