ferenc-csaky commented on code in PR #27340:
URL: https://github.com/apache/flink/pull/27340#discussion_r2836272677


##########
flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/table/SavepointTypeInformationFactoryTest.java:
##########
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.flink.state.table;
+
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.types.Row;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.configuration.ExecutionOptions.RUNTIME_MODE;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Unit tests for the SavepointTypeInformationFactory. */
+public class SavepointTypeInformationFactoryTest {
+
+    public static class TestLongTypeInformationFactory implements 
SavepointTypeInformationFactory {
+        private static volatile boolean wasCalled = false;
+
+        public static boolean wasFactoryCalled() {
+            return wasCalled;
+        }
+
+        public static void resetCallTracker() {
+            wasCalled = false;
+        }
+
+        @Override
+        public TypeInformation<?> getTypeInformation() {
+            wasCalled = true;
+            return TypeInformation.of(Long.class);
+        }
+    }
+
+    public static class TestStringTypeInformationFactory
+            implements SavepointTypeInformationFactory {
+        @Override
+        public TypeInformation<?> getTypeInformation() {
+            return TypeInformation.of(String.class);
+        }
+    }
+
+    @Test
+    public void testSavepointTypeInformationFactoryEndToEnd() throws Exception 
{
+        TestLongTypeInformationFactory.resetCallTracker();
+
+        Configuration config = new Configuration();
+        config.set(RUNTIME_MODE, RuntimeExecutionMode.BATCH);
+        StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment(config);
+        StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+        final String sql =
+                "CREATE TABLE state_table (\n"
+                        + "  k bigint,\n"
+                        + "  KeyedPrimitiveValue bigint,\n"
+                        + "  PRIMARY KEY (k) NOT ENFORCED\n"
+                        + ")\n"
+                        + "with (\n"
+                        + "  'connector' = 'savepoint',\n"
+                        + "  'state.path' = 
'src/test/resources/table-state',\n"
+                        + "  'operator.uid' = 'keyed-state-process-uid',\n"
+                        + "  'fields.KeyedPrimitiveValue.value-type-factory' = 
'"
+                        + TestLongTypeInformationFactory.class.getName()
+                        + "'\n"
+                        + ")";
+
+        tEnv.executeSql(sql);
+        Table table = tEnv.sqlQuery("SELECT k, KeyedPrimitiveValue FROM 
state_table");
+        List<Row> result = tEnv.toDataStream(table).executeAndCollect(100);
+
+        assertThat(TestLongTypeInformationFactory.wasFactoryCalled())
+                .as(
+                        "Factory getTypeInformation() method must be called - 
this proves factory is used instead of metadata inference")
+                .isTrue();
+
+        assertThat(result.size()).isEqualTo(10);
+
+        Set<Long> keys =
+                result.stream().map(r -> (Long) 
r.getField("k")).collect(Collectors.toSet());
+        assertThat(keys).hasSize(10);
+        assertThat(keys).containsExactlyInAnyOrder(0L, 1L, 2L, 3L, 4L, 5L, 6L, 
7L, 8L, 9L);
+
+        Set<Long> primitiveValues =
+                result.stream()
+                        .map(r -> (Long) r.getField("KeyedPrimitiveValue"))
+                        .collect(Collectors.toSet());
+        assertThat(primitiveValues).hasSize(1);
+        assertThat(primitiveValues.iterator().next()).isEqualTo(1L);

Review Comment:
   It seems we can simplify this 
to:`assertThat(primitiveValues).containsExactly(1L)`



##########
flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/table/SavepointTypeInformationFactoryTest.java:
##########
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.flink.state.table;
+
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.types.Row;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.configuration.ExecutionOptions.RUNTIME_MODE;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Unit tests for the SavepointTypeInformationFactory. */
+public class SavepointTypeInformationFactoryTest {
+
+    public static class TestLongTypeInformationFactory implements 
SavepointTypeInformationFactory {
+        private static volatile boolean wasCalled = false;
+
+        public static boolean wasFactoryCalled() {
+            return wasCalled;
+        }
+
+        public static void resetCallTracker() {
+            wasCalled = false;
+        }
+
+        @Override
+        public TypeInformation<?> getTypeInformation() {
+            wasCalled = true;
+            return TypeInformation.of(Long.class);
+        }
+    }
+
+    public static class TestStringTypeInformationFactory

Review Comment:
   Any reason to not define the inner test classes `private` instead?



##########
flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/table/SavepointTypeInformationFactoryTest.java:
##########
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.flink.state.table;
+
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.types.Row;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.configuration.ExecutionOptions.RUNTIME_MODE;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Unit tests for the SavepointTypeInformationFactory. */
+public class SavepointTypeInformationFactoryTest {
+
+    public static class TestLongTypeInformationFactory implements 
SavepointTypeInformationFactory {
+        private static volatile boolean wasCalled = false;
+
+        public static boolean wasFactoryCalled() {
+            return wasCalled;
+        }
+
+        public static void resetCallTracker() {
+            wasCalled = false;
+        }
+
+        @Override
+        public TypeInformation<?> getTypeInformation() {
+            wasCalled = true;
+            return TypeInformation.of(Long.class);
+        }
+    }
+
+    public static class TestStringTypeInformationFactory
+            implements SavepointTypeInformationFactory {
+        @Override
+        public TypeInformation<?> getTypeInformation() {
+            return TypeInformation.of(String.class);
+        }
+    }
+
+    @Test
+    public void testSavepointTypeInformationFactoryEndToEnd() throws Exception 
{
+        TestLongTypeInformationFactory.resetCallTracker();
+
+        Configuration config = new Configuration();
+        config.set(RUNTIME_MODE, RuntimeExecutionMode.BATCH);
+        StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment(config);
+        StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+        final String sql =
+                "CREATE TABLE state_table (\n"
+                        + "  k bigint,\n"
+                        + "  KeyedPrimitiveValue bigint,\n"
+                        + "  PRIMARY KEY (k) NOT ENFORCED\n"
+                        + ")\n"
+                        + "with (\n"
+                        + "  'connector' = 'savepoint',\n"
+                        + "  'state.path' = 
'src/test/resources/table-state',\n"
+                        + "  'operator.uid' = 'keyed-state-process-uid',\n"
+                        + "  'fields.KeyedPrimitiveValue.value-type-factory' = 
'"
+                        + TestLongTypeInformationFactory.class.getName()
+                        + "'\n"
+                        + ")";
+
+        tEnv.executeSql(sql);
+        Table table = tEnv.sqlQuery("SELECT k, KeyedPrimitiveValue FROM 
state_table");
+        List<Row> result = tEnv.toDataStream(table).executeAndCollect(100);
+
+        assertThat(TestLongTypeInformationFactory.wasFactoryCalled())
+                .as(
+                        "Factory getTypeInformation() method must be called - 
this proves factory is used instead of metadata inference")
+                .isTrue();
+
+        assertThat(result.size()).isEqualTo(10);

Review Comment:
   nit: use `hasSize` instead



##########
flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/table/SavepointDynamicTableSourceFactory.java:
##########
@@ -324,82 +281,25 @@ private SavepointConnectorOptions.StateType 
inferStateType(LogicalType logicalTy
         }
     }
 
-    @Nullable
-    private String inferStateMapKeyFormat(String columnName, LogicalType 
logicalType) {
-        return logicalType.is(LogicalTypeRoot.MAP)
-                ? inferStateValueFormat(columnName, ((MapType) 
logicalType).getKeyType())
-                : null;
-    }
-
-    private String inferStateValueFormat(String columnName, LogicalType 
logicalType) {
-        switch (logicalType.getTypeRoot()) {
-            case CHAR:
-            case VARCHAR:
-                return String.class.getName();
-
-            case BOOLEAN:
-                return Boolean.class.getName();
-
-            case BINARY:
-            case VARBINARY:
-                return byte[].class.getName();
-
-            case DECIMAL:
-                return BigDecimal.class.getName();
-
-            case TINYINT:
-                return Byte.class.getName();
-
-            case SMALLINT:
-                return Short.class.getName();
-
-            case INTEGER:
-                return Integer.class.getName();
-
-            case BIGINT:
-                return Long.class.getName();
-
-            case FLOAT:
-                return Float.class.getName();
-
-            case DOUBLE:
-                return Double.class.getName();
-
-            case DATE:
-                return Integer.class.getName();
-
-            case INTERVAL_YEAR_MONTH:
-            case INTERVAL_DAY_TIME:
-                return Long.class.getName();
-
-            case ARRAY:
-                return inferStateValueFormat(
-                        columnName, ((ArrayType) 
logicalType).getElementType());
-
-            case MAP:
-                return inferStateValueFormat(columnName, ((MapType) 
logicalType).getValueType());
-
-            case NULL:
-                return null;
-
-            case ROW:
-            case MULTISET:
-            case TIME_WITHOUT_TIME_ZONE:
-            case TIMESTAMP_WITHOUT_TIME_ZONE:
-            case TIMESTAMP_WITH_TIME_ZONE:
-            case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
-            case DISTINCT_TYPE:
-            case STRUCTURED_TYPE:
-            case RAW:
-            case SYMBOL:
-            case UNRESOLVED:
-            case DESCRIPTOR:
-            default:
-                throw new UnsupportedOperationException(
-                        String.format(
-                                "Unable to infer state format for SQL type: %s 
in column: %s. "
-                                        + "Please override the type with the 
following config parameter: %s.%s.%s",
-                                logicalType, columnName, FIELDS, columnName, 
VALUE_CLASS));
+    /**
+     * Preloads all state metadata for an operator in a single I/O operation.
+     *
+     * @param savepointPath Path to the savepoint
+     * @param operatorIdentifier Operator UID or hash
+     * @return Map from state name to StateMetaInfoSnapshot
+     */
+    private Map<String, StateMetaInfoSnapshot> preloadStateMetadata(
+            String savepointPath, OperatorIdentifier operatorIdentifier) {
+        try {
+            return SavepointLoader.loadOperatorStateMetadata(savepointPath, 
operatorIdentifier);
+        } catch (Exception e) {
+            throw new RuntimeException(
+                    String.format(
+                            "Failed to load state metadata from savepoint '%s' 
for operator '%s'. "
+                                    + "Ensure the savepoint path is valid and 
the operator exists in the savepoint. "
+                                    + "Original error: %s",

Review Comment:
   Any reason to include the cause error message explicitly? It is passed as a 
cause, so it will be listed in the stack trace anyways. This can be useful if 
sometimes only the message gets surfaced, I'm just not sure that's actually 
happening.



##########
flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/table/SavepointTypeInfoResolver.java:
##########
@@ -0,0 +1,493 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.flink.state.table;
+
+import org.apache.flink.api.common.serialization.SerializerConfig;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.utils.TypeUtils;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot;
+import org.apache.flink.api.common.typeutils.base.MapSerializer;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.metainfo.StateMetaInfoSnapshot;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigDecimal;
+import java.util.Map;
+import java.util.Optional;
+
+import static org.apache.flink.state.table.SavepointConnectorOptions.FIELDS;
+import static 
org.apache.flink.state.table.SavepointConnectorOptions.VALUE_CLASS;
+
+/** Resolver for TypeInformation from savepoint metadata and configuration. */
+public class SavepointTypeInfoResolver {

Review Comment:
   Any reason to not make this class and its inner `enum` package-private? I 
guess these are meaningful outside this package, nor something any user should 
tinker with.



##########
flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/table/SavepointDynamicTableSourceFactory.java:
##########
@@ -234,6 +154,83 @@ public DynamicTableSource createDynamicTableSource(Context 
context) {
                 rowType);
     }
 
+    private StateValueColumnConfiguration createStateColumnConfiguration(
+            int columnIndex,
+            RowType rowType,
+            Configuration options,
+            Set<ConfigOption<?>> optionalOptions,
+            SavepointTypeInfoResolver typeResolver) {
+
+        RowType.RowField valueRowField = rowType.getFields().get(columnIndex);
+
+        ConfigOption<String> stateNameOption =
+                key(String.format("%s.%s.%s", FIELDS, valueRowField.getName(), 
STATE_NAME))

Review Comment:
   I may add a separate key builder method, cause I think we duplicate this 
enough now that something like this would worth adding:
   ```java
   private static ConfigOptions.OptionBuilder optionOf(String rowField, String 
optionName) {
       return ConfigOptions.key(String.format("%s.%s.%s", FIELDS, rowField, 
optionName));
   }
   ```



##########
flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/table/SavepointTypeInformationFactoryTest.java:
##########
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.flink.state.table;
+
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.types.Row;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.configuration.ExecutionOptions.RUNTIME_MODE;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Unit tests for the SavepointTypeInformationFactory. */
+public class SavepointTypeInformationFactoryTest {

Review Comment:
   Since JUnit5 both the class itself and the test methods should be 
package-private as a rule of thumb.



##########
flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/table/SavepointTypeInfoResolver.java:
##########
@@ -0,0 +1,493 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.flink.state.table;
+
+import org.apache.flink.api.common.serialization.SerializerConfig;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.utils.TypeUtils;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot;
+import org.apache.flink.api.common.typeutils.base.MapSerializer;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.metainfo.StateMetaInfoSnapshot;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigDecimal;
+import java.util.Map;
+import java.util.Optional;
+
+import static org.apache.flink.state.table.SavepointConnectorOptions.FIELDS;
+import static 
org.apache.flink.state.table.SavepointConnectorOptions.VALUE_CLASS;
+
+/** Resolver for TypeInformation from savepoint metadata and configuration. */
+public class SavepointTypeInfoResolver {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(SavepointTypeInfoResolver.class);
+
+    /** Context for type inference to determine what aspect of the type we 
need. */
+    public enum InferenceContext {
+        /** Inferring the key type of keyed state (always primitive). */
+        KEY,
+        /** Inferring the key type of a MAP state. */
+        MAP_KEY,
+        /** Inferring the value type (behavior depends on logical type). */
+        VALUE
+    }
+
+    private final Map<String, StateMetaInfoSnapshot> preloadedStateMetadata;
+    private final SerializerConfig serializerConfig;
+
+    public SavepointTypeInfoResolver(
+            Map<String, StateMetaInfoSnapshot> preloadedStateMetadata,
+            SerializerConfig serializerConfig) {
+        this.preloadedStateMetadata = preloadedStateMetadata;
+        this.serializerConfig = serializerConfig;
+    }
+
+    /**
+     * Resolves TypeInformation for keyed state keys (primitive types only).
+     *
+     * <p>This is a simplified version of type resolution specifically for key 
types, which are
+     * always primitive and don't require complex metadata inference.
+     *
+     * @param options Configuration containing table options
+     * @param classOption Config option for explicit class specification
+     * @param typeInfoFactoryOption Config option for type factory 
specification
+     * @param rowField The row field containing name and LogicalType
+     * @return The resolved TypeInformation for the key
+     * @throws IllegalArgumentException If both class and factory options are 
specified
+     * @throws RuntimeException If type instantiation fails
+     */
+    public TypeInformation<?> resolveKeyType(
+            Configuration options,
+            ConfigOption<String> classOption,
+            ConfigOption<String> typeInfoFactoryOption,
+            RowType.RowField rowField) {
+        try {
+            // Priority 1: Explicit configuration (backward compatibility)
+            TypeInformation<?> explicitTypeInfo =
+                    getExplicitTypeInfo(options, classOption, 
typeInfoFactoryOption);
+            if (explicitTypeInfo != null) {
+                return explicitTypeInfo;
+            }
+
+            // Priority 2: Simple primitive type inference from LogicalType
+            LogicalType logicalType = rowField.getType();
+            String columnName = rowField.getName();
+            return TypeInformation.of(getPrimitiveClass(logicalType, 
columnName));
+        } catch (ReflectiveOperationException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    /**
+     * Resolves TypeSerializer for a table field using a three-tier priority 
system with direct
+     * serializer extraction for metadata inference.
+     *
+     * <h3>Three-Tier Priority System (Serializer-First)</h3>
+     *
+     * <ol>
+     *   <li><strong>Priority 1: Explicit Configuration</strong> (Highest 
priority) <br>
+     *       Uses user-specified class name or type factory from table 
options, then converts to
+     *       serializer.
+     *   <li><strong>Priority 2: Metadata Inference</strong> <br>
+     *       Directly extracts serializers from preloaded savepoint metadata 
(NO TypeInformation
+     *       conversion).
+     *   <li><strong>Priority 3: LogicalType Fallback</strong> (Lowest 
priority) <br>
+     *       Infers TypeInformation from table schema's LogicalType, then 
converts to serializer.
+     * </ol>
+     *
+     * <p>This approach eliminates TypeInformation extraction complexity for 
metadata inference,
+     * making it work with ANY serializer type (Avro, custom types, etc.).
+     *
+     * @param options Configuration containing table options
+     * @param classOption Config option for explicit class specification
+     * @param typeInfoFactoryOption Config option for type factory 
specification
+     * @param rowField The table field containing name and LogicalType
+     * @param inferStateType Whether to enable automatic type inference. If 
false, returns null when
+     *     no explicit configuration is provided.
+     * @param context The inference context determining what type aspect to 
extract.
+     * @return The resolved TypeSerializer, or null if inferStateType is false 
and no explicit
+     *     configuration is provided.
+     * @throws IllegalArgumentException If both class and factory options are 
specified
+     * @throws RuntimeException If serializer creation fails
+     */
+    public TypeSerializer<?> resolveSerializer(
+            Configuration options,
+            ConfigOption<String> classOption,
+            ConfigOption<String> typeInfoFactoryOption,
+            RowType.RowField rowField,
+            boolean inferStateType,
+            InferenceContext context) {
+        try {
+            // Priority 1: Explicit configuration (backward compatibility)
+            TypeInformation<?> explicitTypeInfo =
+                    getExplicitTypeInfo(options, classOption, 
typeInfoFactoryOption);
+            if (explicitTypeInfo != null) {
+                return explicitTypeInfo.createSerializer(serializerConfig);
+            }
+
+            if (inferStateType) {

Review Comment:
   I'd invert this check and return `null` early, so we can spare an 
indentation level for this non-trivial block.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to