[ 
https://issues.apache.org/jira/browse/FLINK-28358?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17563073#comment-17563073
 ] 

PengfeiChang edited comment on FLINK-28358 at 7/6/22 9:30 AM:
--------------------------------------------------------------

i found  the reason why the test was removed,have two points:
h2. one

run sql  “SELECT id1, id2, comment1, comment2 FROM T, LATERAL 
TABLE(jdbcLookup(id1, id2)) AS S(l_id1, comment1, comment2, l_id2)”

jdbcLookup is used as udf like 1.14 ,will throw exception ,as follows:

 
{code:java}
org.apache.flink.table.api.ValidationException: SQL validation failed. An error 
occurred in the type inference logic of function 'jdbcLookup'.
    at 
org.apache.flink.table.planner.calcite.FlinkPlannerImpl.org$apache$flink$table$planner$calcite$FlinkPlannerImpl$$validate(FlinkPlannerImpl.scala:184)
    at 
org.apache.flink.table.planner.calcite.FlinkPlannerImpl.validate(FlinkPlannerImpl.scala:109)
    at 
org.apache.flink.table.planner.operations.SqlToOperationConverter.convert(SqlToOperationConverter.java:237)
    at 
org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:105)
    at 
org.apache.flink.table.api.internal.TableEnvironmentImpl.executeSql(TableEnvironmentImpl.java:695)
    at 
org.apache.flink.connector.jdbc.table.JdbcLookupTableITCase.useLegacyTableFactory(JdbcLookupTableITCase.java:174)
    at 
org.apache.flink.connector.jdbc.table.JdbcLookupTableITCase.testLookup(JdbcLookupTableITCase.java:92)
    at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
    at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
    at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
    at java.lang.reflect.Method.invoke(Method.java:498)
    at 
org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59)
    at 
org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
    at 
org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
    at 
org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
    at 
org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
    at 
org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
    at 
org.apache.flink.util.TestNameProvider$1.evaluate(TestNameProvider.java:45)
    at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:61)
    at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
    at 
org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100)
    at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366)
    at 
org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103)
    at 
org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63)
    at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331)
    at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79)
    at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329)
    at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66)
    at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293)
    at org.junit.runners.ParentRunner.run(ParentRunner.java:413)
    at org.junit.runners.Suite.runChild(Suite.java:128)
    at org.junit.runners.Suite.runChild(Suite.java:27)
    at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331)
    at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79)
    at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329)
    at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66)
    at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293)
    at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:54)
    at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:54)
    at org.junit.rules.RunRules.evaluate(RunRules.java:20)
    at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
    at org.junit.runners.ParentRunner.run(ParentRunner.java:413)
    at org.junit.runner.JUnitCore.run(JUnitCore.java:137)
    at 
com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69)
    at 
com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38)
    at 
com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11)
    at 
com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35)
    at 
com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:235)
    at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:54)
Caused by: org.apache.flink.table.api.ValidationException: An error occurred in 
the type inference logic of function 'jdbcLookup'.
    at 
org.apache.flink.table.planner.catalog.FunctionCatalogOperatorTable.convertToBridgingSqlFunction(FunctionCatalogOperatorTable.java:158)
    at 
org.apache.flink.table.planner.catalog.FunctionCatalogOperatorTable.convertToSqlFunction(FunctionCatalogOperatorTable.java:142)
    at 
org.apache.flink.table.planner.catalog.FunctionCatalogOperatorTable.lambda$lookupOperatorOverloads$0(FunctionCatalogOperatorTable.java:99)
    at java.util.Optional.flatMap(Optional.java:241)
    at 
org.apache.flink.table.planner.catalog.FunctionCatalogOperatorTable.lookupOperatorOverloads(FunctionCatalogOperatorTable.java:99)
    at 
org.apache.calcite.sql.util.ChainedSqlOperatorTable.lookupOperatorOverloads(ChainedSqlOperatorTable.java:67)
    at 
org.apache.calcite.sql.validate.SqlValidatorImpl.performUnconditionalRewrites(SqlValidatorImpl.java:1183)
    at 
org.apache.calcite.sql.validate.SqlValidatorImpl.performUnconditionalRewrites(SqlValidatorImpl.java:1169)
    at 
org.apache.calcite.sql.validate.SqlValidatorImpl.performUnconditionalRewrites(SqlValidatorImpl.java:1169)
    at 
org.apache.calcite.sql.validate.SqlValidatorImpl.performUnconditionalRewrites(SqlValidatorImpl.java:1169)
    at 
org.apache.calcite.sql.validate.SqlValidatorImpl.performUnconditionalRewrites(SqlValidatorImpl.java:1169)
    at 
org.apache.calcite.sql.validate.SqlValidatorImpl.performUnconditionalRewrites(SqlValidatorImpl.java:1169)
    at 
org.apache.calcite.sql.validate.SqlValidatorImpl.validateScopedExpression(SqlValidatorImpl.java:945)
    at 
org.apache.calcite.sql.validate.SqlValidatorImpl.validate(SqlValidatorImpl.java:704)
    at 
org.apache.flink.table.planner.calcite.FlinkPlannerImpl.org$apache$flink$table$planner$calcite$FlinkPlannerImpl$$validate(FlinkPlannerImpl.scala:180)
    ... 48 more
Caused by: org.apache.flink.table.api.ValidationException: Could not extract a 
valid type inference for function class 
'org.apache.flink.connector.jdbc.table.JdbcRowDataLookupFunction'. Please check 
for implementation mistakes and/or provide a corresponding hint.
    at 
org.apache.flink.table.types.extraction.ExtractionUtils.extractionError(ExtractionUtils.java:362)
    at 
org.apache.flink.table.types.extraction.TypeInferenceExtractor.extractTypeInference(TypeInferenceExtractor.java:150)
    at 
org.apache.flink.table.types.extraction.TypeInferenceExtractor.forTableFunction(TypeInferenceExtractor.java:113)
    at 
org.apache.flink.table.functions.TableFunction.getTypeInference(TableFunction.java:208)
    at 
org.apache.flink.table.planner.catalog.FunctionCatalogOperatorTable.convertToBridgingSqlFunction(FunctionCatalogOperatorTable.java:155)
    ... 62 more
Caused by: org.apache.flink.table.api.ValidationException: Error in extracting 
a signature to output mapping.
    at 
org.apache.flink.table.types.extraction.ExtractionUtils.extractionError(ExtractionUtils.java:362)
    at 
org.apache.flink.table.types.extraction.FunctionMappingExtractor.extractOutputMapping(FunctionMappingExtractor.java:117)
    at 
org.apache.flink.table.types.extraction.TypeInferenceExtractor.extractTypeInferenceOrError(TypeInferenceExtractor.java:161)
    at 
org.apache.flink.table.types.extraction.TypeInferenceExtractor.extractTypeInference(TypeInferenceExtractor.java:148)
    ... 65 more
Caused by: org.apache.flink.table.api.ValidationException: Unable to extract a 
type inference from method:
public void 
org.apache.flink.connector.jdbc.table.JdbcRowDataLookupFunction.eval(java.lang.Object[])
    at 
org.apache.flink.table.types.extraction.ExtractionUtils.extractionError(ExtractionUtils.java:362)
    at 
org.apache.flink.table.types.extraction.FunctionMappingExtractor.extractResultMappings(FunctionMappingExtractor.java:183)
    at 
org.apache.flink.table.types.extraction.FunctionMappingExtractor.extractOutputMapping(FunctionMappingExtractor.java:114)
    ... 67 more
Caused by: org.apache.flink.table.api.ValidationException: Could not extract a 
data type from 'interface org.apache.flink.table.data.RowData' in generic class 
'org.apache.flink.table.functions.TableFunction' in class 
org.apache.flink.connector.jdbc.table.JdbcRowDataLookupFunction. Please pass 
the required data type manually or allow RAW types.
    at 
org.apache.flink.table.types.extraction.ExtractionUtils.extractionError(ExtractionUtils.java:362)
    at 
org.apache.flink.table.types.extraction.DataTypeExtractor.extractDataTypeOrRawWithTemplate(DataTypeExtractor.java:240)
    at 
org.apache.flink.table.types.extraction.DataTypeExtractor.extractDataTypeOrRaw(DataTypeExtractor.java:218)
    at 
org.apache.flink.table.types.extraction.DataTypeExtractor.extractDataTypeWithClassContext(DataTypeExtractor.java:194)
    at 
org.apache.flink.table.types.extraction.DataTypeExtractor.extractFromGeneric(DataTypeExtractor.java:124)
    at 
org.apache.flink.table.types.extraction.FunctionMappingExtractor.lambda$createGenericResultExtraction$13(FunctionMappingExtractor.java:478)
    at 
org.apache.flink.table.types.extraction.FunctionMappingExtractor.putExtractedResultMappings(FunctionMappingExtractor.java:319)
    at 
org.apache.flink.table.types.extraction.FunctionMappingExtractor.collectMethodMappings(FunctionMappingExtractor.java:269)
    at 
org.apache.flink.table.types.extraction.FunctionMappingExtractor.extractResultMappings(FunctionMappingExtractor.java:169)
    ... 68 more
Caused by: org.apache.flink.table.api.ValidationException: Cannot extract a 
data type from an internal 'org.apache.flink.table.data.RowData' class without 
further information. Please use annotations to define the full logical type.
    at 
org.apache.flink.table.types.extraction.ExtractionUtils.extractionError(ExtractionUtils.java:362)
    at 
org.apache.flink.table.types.extraction.ExtractionUtils.extractionError(ExtractionUtils.java:357)
    at 
org.apache.flink.table.types.extraction.DataTypeExtractor.checkForCommonErrors(DataTypeExtractor.java:367)
    at 
org.apache.flink.table.types.extraction.DataTypeExtractor.extractDataTypeOrError(DataTypeExtractor.java:272)
    at 
org.apache.flink.table.types.extraction.DataTypeExtractor.extractDataTypeOrRawWithTemplate(DataTypeExtractor.java:232)
    ... 75 more {code}
 

 

_1.15 uses JdbcRowDataLookupFunction to_ replace {_}of 
'{_}JdbcTableSource.Builder'{_}, whether it is not recommended to use 
JdbcRowDataLookupFunction as udf directly in 1.15 later, I am a little confused 
here{_}

 

 
h2. two

 

run sql "

SELECT source.id1, source.id2, L.comment1, L.comment2 FROM T AS source 

JOIN lookup for system_time as of source.proctime AS L 

ON source.id1 = L.id1 and source.id2 = L.id2

"

when debug in local ,throw out "The system time period specification expects 
Timestamp type but is 'TIMESTAMP_WITH_LOCAL_TIME_ZONE' " exception

 

finally,lead to this test of 
"org.apache.flink.connector.jdbc.table.JdbcLookupTableITCase.testLookup" all 
case is failed,so this test is removed

 

 

 

 


was (Author: JIRAUSER292000):
i found  the reason why the test was removed,have two points:
h2. one

jdbcLookup is used as udf like 1.14  “SELECT id1, id2, comment1, comment2 FROM 
T, LATERAL TABLE(jdbcLookup(id1, id2)) AS S(l_id1, comment1, comment2, 
l_id2)”,will throw exception ,as follows:

 
{code:java}
org.apache.flink.table.api.ValidationException: SQL validation failed. An error 
occurred in the type inference logic of function 'jdbcLookup'.
    at 
org.apache.flink.table.planner.calcite.FlinkPlannerImpl.org$apache$flink$table$planner$calcite$FlinkPlannerImpl$$validate(FlinkPlannerImpl.scala:184)
    at 
org.apache.flink.table.planner.calcite.FlinkPlannerImpl.validate(FlinkPlannerImpl.scala:109)
    at 
org.apache.flink.table.planner.operations.SqlToOperationConverter.convert(SqlToOperationConverter.java:237)
    at 
org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:105)
    at 
org.apache.flink.table.api.internal.TableEnvironmentImpl.executeSql(TableEnvironmentImpl.java:695)
    at 
org.apache.flink.connector.jdbc.table.JdbcLookupTableITCase.useLegacyTableFactory(JdbcLookupTableITCase.java:174)
    at 
org.apache.flink.connector.jdbc.table.JdbcLookupTableITCase.testLookup(JdbcLookupTableITCase.java:92)
    at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
    at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
    at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
    at java.lang.reflect.Method.invoke(Method.java:498)
    at 
org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59)
    at 
org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
    at 
org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
    at 
org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
    at 
org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
    at 
org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
    at 
org.apache.flink.util.TestNameProvider$1.evaluate(TestNameProvider.java:45)
    at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:61)
    at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
    at 
org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100)
    at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366)
    at 
org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103)
    at 
org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63)
    at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331)
    at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79)
    at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329)
    at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66)
    at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293)
    at org.junit.runners.ParentRunner.run(ParentRunner.java:413)
    at org.junit.runners.Suite.runChild(Suite.java:128)
    at org.junit.runners.Suite.runChild(Suite.java:27)
    at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331)
    at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79)
    at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329)
    at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66)
    at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293)
    at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:54)
    at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:54)
    at org.junit.rules.RunRules.evaluate(RunRules.java:20)
    at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
    at org.junit.runners.ParentRunner.run(ParentRunner.java:413)
    at org.junit.runner.JUnitCore.run(JUnitCore.java:137)
    at 
com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69)
    at 
com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38)
    at 
com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11)
    at 
com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35)
    at 
com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:235)
    at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:54)
Caused by: org.apache.flink.table.api.ValidationException: An error occurred in 
the type inference logic of function 'jdbcLookup'.
    at 
org.apache.flink.table.planner.catalog.FunctionCatalogOperatorTable.convertToBridgingSqlFunction(FunctionCatalogOperatorTable.java:158)
    at 
org.apache.flink.table.planner.catalog.FunctionCatalogOperatorTable.convertToSqlFunction(FunctionCatalogOperatorTable.java:142)
    at 
org.apache.flink.table.planner.catalog.FunctionCatalogOperatorTable.lambda$lookupOperatorOverloads$0(FunctionCatalogOperatorTable.java:99)
    at java.util.Optional.flatMap(Optional.java:241)
    at 
org.apache.flink.table.planner.catalog.FunctionCatalogOperatorTable.lookupOperatorOverloads(FunctionCatalogOperatorTable.java:99)
    at 
org.apache.calcite.sql.util.ChainedSqlOperatorTable.lookupOperatorOverloads(ChainedSqlOperatorTable.java:67)
    at 
org.apache.calcite.sql.validate.SqlValidatorImpl.performUnconditionalRewrites(SqlValidatorImpl.java:1183)
    at 
org.apache.calcite.sql.validate.SqlValidatorImpl.performUnconditionalRewrites(SqlValidatorImpl.java:1169)
    at 
org.apache.calcite.sql.validate.SqlValidatorImpl.performUnconditionalRewrites(SqlValidatorImpl.java:1169)
    at 
org.apache.calcite.sql.validate.SqlValidatorImpl.performUnconditionalRewrites(SqlValidatorImpl.java:1169)
    at 
org.apache.calcite.sql.validate.SqlValidatorImpl.performUnconditionalRewrites(SqlValidatorImpl.java:1169)
    at 
org.apache.calcite.sql.validate.SqlValidatorImpl.performUnconditionalRewrites(SqlValidatorImpl.java:1169)
    at 
org.apache.calcite.sql.validate.SqlValidatorImpl.validateScopedExpression(SqlValidatorImpl.java:945)
    at 
org.apache.calcite.sql.validate.SqlValidatorImpl.validate(SqlValidatorImpl.java:704)
    at 
org.apache.flink.table.planner.calcite.FlinkPlannerImpl.org$apache$flink$table$planner$calcite$FlinkPlannerImpl$$validate(FlinkPlannerImpl.scala:180)
    ... 48 more
Caused by: org.apache.flink.table.api.ValidationException: Could not extract a 
valid type inference for function class 
'org.apache.flink.connector.jdbc.table.JdbcRowDataLookupFunction'. Please check 
for implementation mistakes and/or provide a corresponding hint.
    at 
org.apache.flink.table.types.extraction.ExtractionUtils.extractionError(ExtractionUtils.java:362)
    at 
org.apache.flink.table.types.extraction.TypeInferenceExtractor.extractTypeInference(TypeInferenceExtractor.java:150)
    at 
org.apache.flink.table.types.extraction.TypeInferenceExtractor.forTableFunction(TypeInferenceExtractor.java:113)
    at 
org.apache.flink.table.functions.TableFunction.getTypeInference(TableFunction.java:208)
    at 
org.apache.flink.table.planner.catalog.FunctionCatalogOperatorTable.convertToBridgingSqlFunction(FunctionCatalogOperatorTable.java:155)
    ... 62 more
Caused by: org.apache.flink.table.api.ValidationException: Error in extracting 
a signature to output mapping.
    at 
org.apache.flink.table.types.extraction.ExtractionUtils.extractionError(ExtractionUtils.java:362)
    at 
org.apache.flink.table.types.extraction.FunctionMappingExtractor.extractOutputMapping(FunctionMappingExtractor.java:117)
    at 
org.apache.flink.table.types.extraction.TypeInferenceExtractor.extractTypeInferenceOrError(TypeInferenceExtractor.java:161)
    at 
org.apache.flink.table.types.extraction.TypeInferenceExtractor.extractTypeInference(TypeInferenceExtractor.java:148)
    ... 65 more
Caused by: org.apache.flink.table.api.ValidationException: Unable to extract a 
type inference from method:
public void 
org.apache.flink.connector.jdbc.table.JdbcRowDataLookupFunction.eval(java.lang.Object[])
    at 
org.apache.flink.table.types.extraction.ExtractionUtils.extractionError(ExtractionUtils.java:362)
    at 
org.apache.flink.table.types.extraction.FunctionMappingExtractor.extractResultMappings(FunctionMappingExtractor.java:183)
    at 
org.apache.flink.table.types.extraction.FunctionMappingExtractor.extractOutputMapping(FunctionMappingExtractor.java:114)
    ... 67 more
Caused by: org.apache.flink.table.api.ValidationException: Could not extract a 
data type from 'interface org.apache.flink.table.data.RowData' in generic class 
'org.apache.flink.table.functions.TableFunction' in class 
org.apache.flink.connector.jdbc.table.JdbcRowDataLookupFunction. Please pass 
the required data type manually or allow RAW types.
    at 
org.apache.flink.table.types.extraction.ExtractionUtils.extractionError(ExtractionUtils.java:362)
    at 
org.apache.flink.table.types.extraction.DataTypeExtractor.extractDataTypeOrRawWithTemplate(DataTypeExtractor.java:240)
    at 
org.apache.flink.table.types.extraction.DataTypeExtractor.extractDataTypeOrRaw(DataTypeExtractor.java:218)
    at 
org.apache.flink.table.types.extraction.DataTypeExtractor.extractDataTypeWithClassContext(DataTypeExtractor.java:194)
    at 
org.apache.flink.table.types.extraction.DataTypeExtractor.extractFromGeneric(DataTypeExtractor.java:124)
    at 
org.apache.flink.table.types.extraction.FunctionMappingExtractor.lambda$createGenericResultExtraction$13(FunctionMappingExtractor.java:478)
    at 
org.apache.flink.table.types.extraction.FunctionMappingExtractor.putExtractedResultMappings(FunctionMappingExtractor.java:319)
    at 
org.apache.flink.table.types.extraction.FunctionMappingExtractor.collectMethodMappings(FunctionMappingExtractor.java:269)
    at 
org.apache.flink.table.types.extraction.FunctionMappingExtractor.extractResultMappings(FunctionMappingExtractor.java:169)
    ... 68 more
Caused by: org.apache.flink.table.api.ValidationException: Cannot extract a 
data type from an internal 'org.apache.flink.table.data.RowData' class without 
further information. Please use annotations to define the full logical type.
    at 
org.apache.flink.table.types.extraction.ExtractionUtils.extractionError(ExtractionUtils.java:362)
    at 
org.apache.flink.table.types.extraction.ExtractionUtils.extractionError(ExtractionUtils.java:357)
    at 
org.apache.flink.table.types.extraction.DataTypeExtractor.checkForCommonErrors(DataTypeExtractor.java:367)
    at 
org.apache.flink.table.types.extraction.DataTypeExtractor.extractDataTypeOrError(DataTypeExtractor.java:272)
    at 
org.apache.flink.table.types.extraction.DataTypeExtractor.extractDataTypeOrRawWithTemplate(DataTypeExtractor.java:232)
    ... 75 more {code}
 

 

_1.15 uses JdbcRowDataLookupFunction to_ replace {_}of 
'{_}JdbcTableSource.Builder'{_}, whether it is not recommended to use 
JdbcRowDataLookupFunction as udf directly in 1.15 later, I am a little confused 
here{_}

 

 
h2. two

 

run sql "

SELECT source.id1, source.id2, L.comment1, L.comment2 FROM T AS source 

JOIN lookup for system_time as of source.proctime AS L 

ON source.id1 = L.id1 and source.id2 = L.id2

"

when debug in local ,throw out "The system time period specification expects 
Timestamp type but is 'TIMESTAMP_WITH_LOCAL_TIME_ZONE' " exception

 

finally,lead to this test of 
"org.apache.flink.connector.jdbc.table.JdbcLookupTableITCase.testLookup" all 
case is failed,so this test is removed

 

 

 

 

> when debug in local ,throw out "The system time period specification expects 
> Timestamp type but is 'TIMESTAMP_WITH_LOCAL_TIME_ZONE' " exception
> -----------------------------------------------------------------------------------------------------------------------------------------------
>
>                 Key: FLINK-28358
>                 URL: https://issues.apache.org/jira/browse/FLINK-28358
>             Project: Flink
>          Issue Type: Bug
>          Components: Connectors / JDBC
>    Affects Versions: 1.14.4
>         Environment: maven:3.2.5 maven:3.6.1  maven:3.3.9 
> openjdk:1.8.0_333
> idea:IntelliJ IDEA 2021.3 (Ultimate Edition)
>            Reporter: PengfeiChang
>            Priority: Minor
>              Labels: debug
>
> h1. subject
> when i debug in local to see the jdbcconnector lookup mechanism and run 
> org.apache.flink.connector.jdbc.table.JdbcLookupTableITCase.testLookup,throw 
> out a exception ,detail as follow:
> {code:java}
> org.apache.flink.table.api.ValidationException: SQL validation failed. From 
> line 1, column 106 to line 1, column 120: The system time period 
> specification expects Timestamp type but is 'TIMESTAMP_WITH_LOCAL_TIME_ZONE'
>       at 
> org.apache.flink.table.planner.calcite.FlinkPlannerImpl.org$apache$flink$table$planner$calcite$FlinkPlannerImpl$$validate(FlinkPlannerImpl.scala:164)
>       at 
> org.apache.flink.table.planner.calcite.FlinkPlannerImpl.validate(FlinkPlannerImpl.scala:107)
>       at 
> org.apache.flink.table.planner.operations.SqlToOperationConverter.convert(SqlToOperationConverter.java:215)
>       at 
> org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:101)
>       at 
> org.apache.flink.table.api.internal.TableEnvironmentImpl.executeSql(TableEnvironmentImpl.java:736)
>       at 
> org.apache.flink.connector.jdbc.table.JdbcLookupTableITCase.useDynamicTableFactory(JdbcLookupTableITCase.java:195)
>       at 
> org.apache.flink.connector.jdbc.table.JdbcLookupTableITCase.testLookup(JdbcLookupTableITCase.java:81)
>       at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>       at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>       at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>       at java.lang.reflect.Method.invoke(Method.java:498)
>       at 
> org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59)
>       at 
> org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
>       at 
> org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
>       at 
> org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
>       at 
> org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
>       at 
> org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
>       at 
> org.apache.flink.util.TestNameProvider$1.evaluate(TestNameProvider.java:45)
>       at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:61)
>       at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
>       at 
> org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100)
>       at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366)
>       at 
> org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103)
>       at 
> org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63)
>       at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331)
>       at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79)
>       at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329)
>       at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66)
>       at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293)
>       at org.junit.runners.ParentRunner.run(ParentRunner.java:413)
>       at org.junit.runners.Suite.runChild(Suite.java:128)
>       at org.junit.runners.Suite.runChild(Suite.java:27)
>       at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331)
>       at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79)
>       at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329)
>       at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66)
>       at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293)
>       at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:54)
>       at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:54)
>       at org.junit.rules.RunRules.evaluate(RunRules.java:20)
>       at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
>       at org.junit.runners.ParentRunner.run(ParentRunner.java:413)
>       at org.junit.runner.JUnitCore.run(JUnitCore.java:137)
>       at 
> com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:68)
>       at 
> com.intellij.rt.execution.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:47)
>       at 
> com.intellij.rt.execution.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:242)
>       at 
> com.intellij.rt.execution.junit.JUnitStarter.main(JUnitStarter.java:70)
> Caused by: org.apache.calcite.runtime.CalciteContextException: From line 1, 
> column 106 to line 1, column 120: The system time period specification 
> expects Timestamp type but is 'TIMESTAMP_WITH_LOCAL_TIME_ZONE'
>       at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
>       at 
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
>       at 
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
>       at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
>       at 
> org.apache.calcite.runtime.Resources$ExInstWithCause.ex(Resources.java:467)
>       at org.apache.calcite.sql.SqlUtil.newContextException(SqlUtil.java:883)
>       at org.apache.calcite.sql.SqlUtil.newContextException(SqlUtil.java:868)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.newValidationError(SqlValidatorImpl.java:5043)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateSnapshot(SqlValidatorImpl.java:4886)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateQuery(SqlValidatorImpl.java:1055)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateFrom(SqlValidatorImpl.java:3205)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateFrom(SqlValidatorImpl.java:3187)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateJoin(SqlValidatorImpl.java:3252)
>       at 
> org.apache.flink.table.planner.calcite.FlinkCalciteSqlValidator.validateJoin(FlinkCalciteSqlValidator.java:117)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateFrom(SqlValidatorImpl.java:3196)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateSelect(SqlValidatorImpl.java:3461)
>       at 
> org.apache.calcite.sql.validate.SelectNamespace.validateImpl(SelectNamespace.java:60)
>       at 
> org.apache.calcite.sql.validate.AbstractNamespace.validate(AbstractNamespace.java:84)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateNamespace(SqlValidatorImpl.java:1067)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateQuery(SqlValidatorImpl.java:1041)
>       at org.apache.calcite.sql.SqlSelect.validate(SqlSelect.java:232)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateScopedExpression(SqlValidatorImpl.java:1016)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validate(SqlValidatorImpl.java:724)
>       at 
> org.apache.flink.table.planner.calcite.FlinkPlannerImpl.org$apache$flink$table$planner$calcite$FlinkPlannerImpl$$validate(FlinkPlannerImpl.scala:159)
>       ... 46 more
> Caused by: org.apache.calcite.sql.validate.SqlValidatorException: The system 
> time period specification expects Timestamp type but is 
> 'TIMESTAMP_WITH_LOCAL_TIME_ZONE'
>       at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
>       at 
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
>       at 
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
>       at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
>       at 
> org.apache.calcite.runtime.Resources$ExInstWithCause.ex(Resources.java:467)
>       at org.apache.calcite.runtime.Resources$ExInst.ex(Resources.java:560)
>       ... 65 more
> Process finished with exit code -1 {code}
> but run test used maven is ok,command as follow:
> {code:java}
> cd flink-connector-jdbc
> mvn test 
> -Dtest=org.apache.flink.connector.jdbc.table.JdbcLookupTableITCase{code}
> log as follow:
> {code:java}
> [INFO]  T E S T S
> [INFO] -------------------------------------------------------
> [INFO] Running org.apache.flink.connector.jdbc.table.JdbcLookupTableITCase
> [INFO] Tests run: 4, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 7.111 
> s - in org.apache.flink.connector.jdbc.table.JdbcLookupTableITCase
> [INFO] 
> [INFO] Results:
> [INFO] 
> [INFO] Tests run: 4, Failures: 0, Errors: 0, Skipped: 0
> [INFO] 
> [INFO] 
> ------------------------------------------------------------------------
> [INFO] BUILD SUCCESS
> [INFO] 
> ------------------------------------------------------------------------
> [INFO] Total time: 34.326 s
> [INFO] Finished at: 2022-07-02T02:31:51+08:00
> [INFO] Final Memory: 54M/706M
> [INFO] 
> ------------------------------------------------------------------------ 
> {code}
>  
>  
> I have commit a pr,feel free to review,link as follows:
> [https://github.com/apache/flink/pull/20135]



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to