-
Notifications
You must be signed in to change notification settings - Fork 2.4k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
support create table like in flink catalog and watermark in windows #12116
base: main
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
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://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.iceberg.flink; | ||
|
||
import org.apache.flink.configuration.ConfigOption; | ||
import org.apache.flink.configuration.ConfigOptions; | ||
|
||
public class FlinkCreateTableOptions { | ||
|
||
private FlinkCreateTableOptions() {} | ||
|
||
public static final ConfigOption<String> CATALOG_NAME = | ||
ConfigOptions.key("catalog-name") | ||
.stringType() | ||
.noDefaultValue() | ||
.withDescription("Catalog name"); | ||
|
||
public static final ConfigOption<String> CATALOG_TYPE = | ||
ConfigOptions.key(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE) | ||
.stringType() | ||
.noDefaultValue() | ||
.withDescription("Catalog type, the optional types are: custom, hadoop, hive."); | ||
|
||
public static final ConfigOption<String> CATALOG_DATABASE = | ||
ConfigOptions.key("catalog-database") | ||
.stringType() | ||
.defaultValue(FlinkCatalogFactory.DEFAULT_DATABASE_NAME) | ||
.withDescription("Database name managed in the iceberg catalog."); | ||
|
||
public static final ConfigOption<String> CATALOG_TABLE = | ||
ConfigOptions.key("catalog-table") | ||
.stringType() | ||
.noDefaultValue() | ||
.withDescription("Table name managed in the underlying iceberg catalog and database."); | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -35,6 +35,7 @@ | |
import org.apache.flink.table.connector.source.abilities.SupportsFilterPushDown; | ||
import org.apache.flink.table.connector.source.abilities.SupportsLimitPushDown; | ||
import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown; | ||
import org.apache.flink.table.connector.source.abilities.SupportsSourceWatermark; | ||
import org.apache.flink.table.data.RowData; | ||
import org.apache.flink.table.expressions.ResolvedExpression; | ||
import org.apache.flink.table.types.DataType; | ||
|
@@ -53,7 +54,8 @@ public class IcebergTableSource | |
implements ScanTableSource, | ||
SupportsProjectionPushDown, | ||
SupportsFilterPushDown, | ||
SupportsLimitPushDown { | ||
SupportsLimitPushDown, | ||
SupportsSourceWatermark { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think we have 2 feature in a single PR:
Could we separate out these features to different PRs? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. These features were driven by mainly a use case, where an iceberg table is needed to be used in Flink window functions. This needs an incoming table to have MILLISECOND precision timestamp column and also watermark to be defined on source table. As iceberg only supports MICROSECOND timestamp columns, we need to have a table with computed columns and we can create these only in Flink Catalog. Iceberg catalog doesn't support creating tables with computed columns. i am happy to split them into 2 separate PR's . As Watermark support is just making Source to implement interface and falling back to #9346 for core logic, i didn't have a test case. I can add a validation on if There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Please separate out the features to 2 PR |
||
|
||
private int[] projectedFields; | ||
private Long limit; | ||
|
@@ -175,6 +177,14 @@ public Result applyFilters(List<ResolvedExpression> flinkFilters) { | |
return Result.of(acceptedFilters, flinkFilters); | ||
} | ||
|
||
@Override | ||
public void applySourceWatermark() { | ||
if (!readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE)) { | ||
throw new UnsupportedOperationException( | ||
"Source watermarks are supported only in flip-27 iceberg source implementation"); | ||
} | ||
} | ||
|
||
@Override | ||
public boolean supportsNestedProjection() { | ||
// TODO: support nested projection | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -256,43 +256,6 @@ public void testCatalogDatabaseConflictWithFlinkDatabase() { | |
.hasMessageStartingWith("Could not execute CreateTable in path"); | ||
} | ||
|
||
@TestTemplate | ||
public void testConnectorTableInIcebergCatalog() { | ||
// Create the catalog properties | ||
Map<String, String> catalogProps = Maps.newHashMap(); | ||
Comment on lines
-259
to
-262
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why is this test removed? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is testing the check mentioned in #12116 (comment) Fail creating a table in Iceberg Catalog if connector=iceberg is specified in the Option. As the check is been deleted, i removed this test case. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think that this is still a valid check in most cases. Only not valid when the table is created with "CREATE TABLE.. LIKE" and only if the source table is an iceberg table. |
||
catalogProps.put("type", "iceberg"); | ||
if (isHiveCatalog()) { | ||
catalogProps.put("catalog-type", "hive"); | ||
catalogProps.put(CatalogProperties.URI, CatalogTestBase.getURI(hiveConf)); | ||
} else { | ||
catalogProps.put("catalog-type", "hadoop"); | ||
} | ||
catalogProps.put(CatalogProperties.WAREHOUSE_LOCATION, createWarehouse()); | ||
|
||
// Create the table properties | ||
Map<String, String> tableProps = createTableProps(); | ||
|
||
// Create a connector table in an iceberg catalog. | ||
sql("CREATE CATALOG `test_catalog` WITH %s", toWithClause(catalogProps)); | ||
try { | ||
assertThatThrownBy( | ||
() -> | ||
sql( | ||
"CREATE TABLE `test_catalog`.`%s`.`%s` (id BIGINT, data STRING) WITH %s", | ||
FlinkCatalogFactory.DEFAULT_DATABASE_NAME, | ||
TABLE_NAME, | ||
toWithClause(tableProps))) | ||
.cause() | ||
.isInstanceOf(IllegalArgumentException.class) | ||
.hasMessage( | ||
"Cannot create the table with 'connector'='iceberg' table property in an iceberg catalog, " | ||
+ "Please create table with 'connector'='iceberg' property in a non-iceberg catalog or " | ||
+ "create table without 'connector'='iceberg' related properties in an iceberg table."); | ||
} finally { | ||
sql("DROP CATALOG IF EXISTS `test_catalog`"); | ||
} | ||
} | ||
|
||
private Map<String, String> createTableProps() { | ||
Map<String, String> tableProps = Maps.newHashMap(properties); | ||
tableProps.put("catalog-name", catalogName); | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why do we remove this check?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Tables can be created using LIKE in
This check basically fails, if we try to create table using LIKE in Iceberg catalog, basically case#2 if we have connector=iceberg in options . For example, DDL like below,
In order to support Case#1 without user setting any extra Options using WITH clause, we need to add connector in getTable,
iceberg/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java
Line 344 in 52bfbdc
This check was added in very old PR,
#2666
#2666 (comment) where Flink SQL didn't support CREATE TABLE A LIKE B , where A and B are in different Catalogs.
So, in this case by removing this check, we are ignoring connector option being passed, so following DDL can create table table_like in Flink catalog backed by
iceberg_catalog.db.table
. As we know source table is an Iceberg table, adding connector=iceberg would be redundant.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
What happens when the source table is not an Iceberg table?
I'm trying to understand here, where we get the missing information in this case, and wether we have a way to check that we actually get the missing information. If we can create such a check, then we can still throw an exception when we don't get this information from any source