-
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 source watermark for flink sql windows #12191
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 |
---|---|---|
|
@@ -19,11 +19,13 @@ | |
package org.apache.iceberg.flink.source; | ||
|
||
import static org.apache.iceberg.types.Types.NestedField.required; | ||
import static org.assertj.core.api.Assertions.assertThatThrownBy; | ||
|
||
import java.io.IOException; | ||
import java.time.Instant; | ||
import java.time.ZoneId; | ||
import java.util.List; | ||
import java.util.Map; | ||
import org.apache.flink.configuration.Configuration; | ||
import org.apache.flink.table.api.TableEnvironment; | ||
import org.apache.flink.table.api.config.TableConfigOptions; | ||
|
@@ -35,11 +37,15 @@ | |
import org.apache.iceberg.data.GenericRecord; | ||
import org.apache.iceberg.data.Record; | ||
import org.apache.iceberg.flink.FlinkConfigOptions; | ||
import org.apache.iceberg.flink.SqlBase; | ||
import org.apache.iceberg.flink.TestFixtures; | ||
import org.apache.iceberg.flink.TestHelpers; | ||
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; | ||
import org.apache.iceberg.relocated.com.google.common.collect.Lists; | ||
import org.apache.iceberg.relocated.com.google.common.collect.Maps; | ||
import org.apache.iceberg.types.Types; | ||
import org.jetbrains.annotations.NotNull; | ||
import org.junit.jupiter.api.AfterEach; | ||
import org.junit.jupiter.api.BeforeEach; | ||
import org.junit.jupiter.api.Test; | ||
|
||
|
@@ -72,6 +78,11 @@ public void before() throws IOException { | |
tableConf.set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); | ||
} | ||
|
||
@AfterEach | ||
public void after() throws IOException { | ||
CATALOG_EXTENSION.catalog().dropTable(TestFixtures.TABLE_IDENTIFIER); | ||
} | ||
|
||
private Record generateRecord(Instant t1, long t2) { | ||
Record record = GenericRecord.create(SCHEMA_TS); | ||
record.setField("t1", t1.atZone(ZoneId.systemDefault()).toLocalDateTime()); | ||
|
@@ -162,4 +173,74 @@ public void testWatermarkOptionsDescending() throws Exception { | |
expected, | ||
SCHEMA_TS); | ||
} | ||
|
||
@Test | ||
public void testWatermarkInvalidConfig() { | ||
CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA_TS); | ||
|
||
String flinkTable = "`default_catalog`.`default_database`.flink_table"; | ||
try { | ||
SqlHelpers.sql( | ||
getStreamingTableEnv(), | ||
"CREATE TABLE %s " | ||
+ "(t1 TIMESTAMP(6), " | ||
+ "t2 BIGINT," | ||
+ "eventTS AS CAST(t1 AS TIMESTAMP(3)), " | ||
+ "WATERMARK FOR eventTS AS SOURCE_WATERMARK()) WITH %s", | ||
flinkTable, | ||
SqlBase.toWithClause(getConnectorOptions())); | ||
|
||
assertThatThrownBy( | ||
() -> SqlHelpers.sql(getStreamingTableEnv(), "SELECT * FROM %s", flinkTable)) | ||
.isInstanceOf(NullPointerException.class) | ||
.hasMessage("watermark-column needs to be configured to use source watermark."); | ||
} finally { | ||
SqlHelpers.sql(getStreamingTableEnv(), "DROP TABLE IF EXISTS %s", flinkTable); | ||
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 try finally? Can we just do a cleanup in an after method? That would make the tests easier to read |
||
} | ||
} | ||
|
||
@Test | ||
public void testWatermarkValidConfig() throws Exception { | ||
List<Record> expected = generateExpectedRecords(true); | ||
|
||
Map<String, String> connectorOptions = Maps.newHashMap(getConnectorOptions()); | ||
connectorOptions.put("watermark-column", "t1"); | ||
|
||
String flinkTable = "`default_catalog`.`default_database`.flink_table"; | ||
|
||
SqlHelpers.sql( | ||
getStreamingTableEnv(), | ||
"CREATE TABLE %s " | ||
+ "(t1 TIMESTAMP(6), " | ||
+ "t2 BIGINT," | ||
+ "eventTS AS CAST(t1 AS TIMESTAMP(3)), " | ||
+ "WATERMARK FOR eventTS AS SOURCE_WATERMARK()) WITH %s", | ||
flinkTable, | ||
SqlBase.toWithClause(connectorOptions)); | ||
|
||
TestHelpers.assertRecordsWithOrder( | ||
SqlHelpers.sql( | ||
getStreamingTableEnv(), | ||
"SELECT t1, t2 FROM TABLE(TUMBLE(TABLE %s, DESCRIPTOR(eventTS), INTERVAL '1' SECOND))", | ||
flinkTable), | ||
expected, | ||
SCHEMA_TS); | ||
} | ||
|
||
@NotNull | ||
private static Map<String, String> getConnectorOptions() { | ||
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. If we do the PRs in a different order (first merging the optionless part), could we get rid of this method? |
||
return Map.of( | ||
"connector", | ||
"iceberg", | ||
"catalog-type", | ||
"hadoop", | ||
"catalog-name", | ||
"iceberg_catalog", | ||
"catalog-database", | ||
TestFixtures.DATABASE, | ||
"catalog-table", | ||
TestFixtures.TABLE, | ||
"warehouse", | ||
CATALOG_EXTENSION.warehouse()); | ||
} | ||
} |
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 is this change? Did this become a utility method which is shared between tests? Can we find a better place for it then?