forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 1
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-24586][table] SQL functions should return STRING instead of VA…
…RCHAR(2000) This closes apache#17691.
- Loading branch information
1 parent
1266df4
commit 6f5c649
Showing
11 changed files
with
204 additions
and
69 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
52 changes: 52 additions & 0 deletions
52
...c/main/java/org/apache/flink/table/planner/functions/sql/SqlJsonArrayFunctionWrapper.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,52 @@ | ||
/* | ||
* 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:https://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.table.planner.functions.sql; | ||
|
||
import org.apache.calcite.rel.type.RelDataType; | ||
import org.apache.calcite.sql.SqlOperatorBinding; | ||
import org.apache.calcite.sql.fun.SqlJsonArrayFunction; | ||
import org.apache.calcite.sql.type.SqlReturnTypeInference; | ||
|
||
import static org.apache.flink.table.planner.plan.type.FlinkReturnTypes.VARCHAR_NOT_NULL; | ||
|
||
/** | ||
* This class is a wrapper class for the {@link SqlJsonArrayFunction} but using the {@code | ||
* VARCHAR_NOT_NULL} return type inference. | ||
*/ | ||
class SqlJsonArrayFunctionWrapper extends SqlJsonArrayFunction { | ||
|
||
@Override | ||
public RelDataType inferReturnType(SqlOperatorBinding opBinding) { | ||
RelDataType returnType = VARCHAR_NOT_NULL.inferReturnType(opBinding); | ||
if (returnType == null) { | ||
throw new IllegalArgumentException( | ||
"Cannot infer return type for " | ||
+ opBinding.getOperator() | ||
+ "; operand types: " | ||
+ opBinding.collectOperandTypes()); | ||
} else { | ||
return returnType; | ||
} | ||
} | ||
|
||
@Override | ||
public SqlReturnTypeInference getReturnTypeInference() { | ||
return VARCHAR_NOT_NULL; | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
51 changes: 51 additions & 0 deletions
51
.../main/java/org/apache/flink/table/planner/functions/sql/SqlJsonObjectFunctionWrapper.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,51 @@ | ||
/* | ||
* 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:https://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.table.planner.functions.sql; | ||
|
||
import org.apache.calcite.rel.type.RelDataType; | ||
import org.apache.calcite.sql.SqlOperatorBinding; | ||
import org.apache.calcite.sql.type.SqlReturnTypeInference; | ||
|
||
import static org.apache.flink.table.planner.plan.type.FlinkReturnTypes.VARCHAR_NOT_NULL; | ||
|
||
/** | ||
* This class is a wrapper class for the {@link SqlJsonObjectFunction} but using the {@code | ||
* VARCHAR_NOT_NULL} return type inference. | ||
*/ | ||
class SqlJsonObjectFunctionWrapper extends SqlJsonObjectFunction { | ||
|
||
@Override | ||
public RelDataType inferReturnType(SqlOperatorBinding opBinding) { | ||
RelDataType returnType = VARCHAR_NOT_NULL.inferReturnType(opBinding); | ||
if (returnType == null) { | ||
throw new IllegalArgumentException( | ||
"Cannot infer return type for " | ||
+ opBinding.getOperator() | ||
+ "; operand types: " | ||
+ opBinding.collectOperandTypes()); | ||
} else { | ||
return returnType; | ||
} | ||
} | ||
|
||
@Override | ||
public SqlReturnTypeInference getReturnTypeInference() { | ||
return VARCHAR_NOT_NULL; | ||
} | ||
} |
52 changes: 52 additions & 0 deletions
52
...c/main/java/org/apache/flink/table/planner/functions/sql/SqlJsonQueryFunctionWrapper.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,52 @@ | ||
/* | ||
* 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:https://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.table.planner.functions.sql; | ||
|
||
import org.apache.calcite.rel.type.RelDataType; | ||
import org.apache.calcite.sql.SqlOperatorBinding; | ||
import org.apache.calcite.sql.fun.SqlJsonQueryFunction; | ||
import org.apache.calcite.sql.type.SqlReturnTypeInference; | ||
|
||
import static org.apache.flink.table.planner.plan.type.FlinkReturnTypes.VARCHAR_FORCE_NULLABLE; | ||
|
||
/** | ||
* This class is a wrapper class for the {@link SqlJsonQueryFunction} but using the {@code | ||
* VARCHAR_FORCE_NULLABLE} return type inference. | ||
*/ | ||
class SqlJsonQueryFunctionWrapper extends SqlJsonQueryFunction { | ||
|
||
@Override | ||
public RelDataType inferReturnType(SqlOperatorBinding opBinding) { | ||
RelDataType returnType = VARCHAR_FORCE_NULLABLE.inferReturnType(opBinding); | ||
if (returnType == null) { | ||
throw new IllegalArgumentException( | ||
"Cannot infer return type for " | ||
+ opBinding.getOperator() | ||
+ "; operand types: " | ||
+ opBinding.collectOperandTypes()); | ||
} else { | ||
return returnType; | ||
} | ||
} | ||
|
||
@Override | ||
public SqlReturnTypeInference getReturnTypeInference() { | ||
return VARCHAR_FORCE_NULLABLE; | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.