stevenzwu commented on code in PR #8555: URL: https://github.com/apache/iceberg/pull/8555#discussion_r1330946559
########## flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java: ########## @@ -0,0 +1,84 @@ +/* + * 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.sink; + +import java.time.Duration; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.util.DateTimeUtil; +import org.apache.iceberg.util.SerializableSupplier; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A table loader that will only reload a table after a certain interval has passed. WARNING: This + * table loader should be used carefully when used with writer tasks. It could result in heavy load + * on a catalog for jobs with many writers. + */ +class CachingTableSupplier implements SerializableSupplier<Table> { + + private static final Logger LOG = LoggerFactory.getLogger(CachingTableSupplier.class); + + private final Table initialTable; + private final TableLoader tableLoader; + private final Duration tableRefreshInterval; + private long nextReloadTimeMs; + private transient Table table; + + CachingTableSupplier(Table initialTable, TableLoader tableLoader, Duration tableRefreshInterval) { Review Comment: should we change `Table ...` to `SerializableTable ...`? technically, only the later is correct ########## flink/v1.17/build.gradle: ########## @@ -125,6 +125,10 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { testImplementation libs.awaitility } + + test { Review Comment: why is this needed? ########## flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java: ########## @@ -0,0 +1,84 @@ +/* + * 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.sink; + +import java.time.Duration; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.util.DateTimeUtil; +import org.apache.iceberg.util.SerializableSupplier; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A table loader that will only reload a table after a certain interval has passed. WARNING: This + * table loader should be used carefully when used with writer tasks. It could result in heavy load + * on a catalog for jobs with many writers. + */ +class CachingTableSupplier implements SerializableSupplier<Table> { + + private static final Logger LOG = LoggerFactory.getLogger(CachingTableSupplier.class); + + private final Table initialTable; + private final TableLoader tableLoader; + private final Duration tableRefreshInterval; + private long nextReloadTimeMs; + private transient Table table; + + CachingTableSupplier(Table initialTable, TableLoader tableLoader, Duration tableRefreshInterval) { + Preconditions.checkArgument(initialTable != null, "initialTable cannot be null"); + Preconditions.checkArgument(tableLoader != null, "tableLoader cannot be null"); + Preconditions.checkArgument( + tableRefreshInterval != null, "tableRefreshInterval cannot be null"); + this.initialTable = initialTable; + this.table = initialTable; + this.tableLoader = tableLoader; + this.tableRefreshInterval = tableRefreshInterval; + this.nextReloadTimeMs = System.currentTimeMillis() + tableRefreshInterval.toMillis(); + } + + @Override + public Table get() { + if (table == null) { + this.table = initialTable; + } + return table; + } + + public void refresh() { Review Comment: if we make this class self-refreshing in the `get()` method, this method can be private. that would match the name `Caching` probably better ########## flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java: ########## @@ -54,16 +59,28 @@ public void open() { this.attemptId = getRuntimeContext().getAttemptNumber(); this.writerMetrics = new IcebergStreamWriterMetrics(super.metrics, fullTableName); - // Initialize the task writer factory. + // Initialize the task writer factory before refreshing the table so that the initial + // schema and partition spec are used. this.taskWriterFactory.initialize(subTaskId, attemptId); + // Refresh the table if needed. + if (tableSupplier instanceof CachingTableSupplier) { Review Comment: once we made `CachingTableSupplier` self refreshing, we won't need this anymore ########## flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java: ########## @@ -369,4 +370,25 @@ public void testOverrideWriteConfigWithUnknownFileFormat() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid file format: UNRECOGNIZED"); } + + @Test + public void testWriteRowWithWriteTableLoader() throws Exception { Review Comment: this method name needs to be updated -- 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] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
