garyli1019 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r560683106
##########
File path:
hudi-common/src/main/java/org/apache/hudi/common/model/OverwriteWithLatestAvroPayload.java
##########
@@ -79,7 +79,12 @@ public OverwriteWithLatestAvroPayload
preCombine(OverwriteWithLatestAvroPayload
* @returns {@code true} if record represents a delete record. {@code false}
otherwise.
*/
protected boolean isDeleteRecord(GenericRecord genericRecord) {
- Object deleteMarker = genericRecord.get("_hoodie_is_deleted");
+ final String isDeleteKey = "_hoodie_is_deleted";
+ // Modify to be compatible with old version Avro.
+ if (genericRecord.getSchema().getField(isDeleteKey) == null) {
Review comment:
What does the old version Avro mean here? This seems like being handled
by the check on return
##########
File path: hudi-flink/src/main/java/org/apache/hudi/operator/FlinkOptions.java
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.hudi.operator;
+
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.streamer.FlinkStreamerConfig;
+import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
+import org.apache.hudi.keygen.SimpleAvroKeyGenerator;
+import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
+import org.apache.hudi.util.StreamerUtil;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.Configuration;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Hoodie Flink config options.
+ *
+ * <p>It has the options for Hoodie table read and write. It also defines some
utilities.
+ */
+public class FlinkOptions {
+ private FlinkOptions() {
+ }
+
+ // ------------------------------------------------------------------------
+ // Base Options
+ // ------------------------------------------------------------------------
+ public static final ConfigOption<String> PATH = ConfigOptions
+ .key("path")
+ .stringType()
+ .noDefaultValue()
+ .withDescription("Base path for the target hoodie table."
+ + "\nThe path would be created if it does not exist,\n"
+ + "otherwise a Hoodie table expects to be initialized successfully");
+
+ public static final ConfigOption<String> PROPS_FILE_PATH = ConfigOptions
+ .key("properties-file.path")
+ .stringType()
+ .noDefaultValue()
+ .withDescription("Path to properties file on local-fs or dfs, with
configurations for \n"
+ + "hoodie client, schema provider, key generator and data source.
For hoodie client props, sane defaults are\n"
+ + "used, but recommend use to provide basic things like metrics
endpoints, hive configs etc. For sources, refer\n"
+ + "to individual classes, for supported properties");
+
+ // ------------------------------------------------------------------------
+ // Read Options
+ // ------------------------------------------------------------------------
+ public static final ConfigOption<String> READ_SCHEMA_FILE_PATH =
ConfigOptions
+ .key("read.schema.file.path")
+ .stringType()
+ .noDefaultValue()
+ .withDescription("Avro schema file path, the parsed schema is used for
deserializing");
+
+ // ------------------------------------------------------------------------
+ // Write Options
+ // ------------------------------------------------------------------------
+ public static final ConfigOption<String> TABLE_NAME = ConfigOptions
+ .key(HoodieWriteConfig.TABLE_NAME)
+ .stringType()
+ .noDefaultValue()
+ .withDescription("Table name to register to Hive metastore");
+
+ public static final ConfigOption<String> TABLE_TYPE = ConfigOptions
+ .key("write.table.type")
+ .stringType()
+ .defaultValue("COPY_ON_WRITE")
+ .withDescription("Type of table to write. COPY_ON_WRITE (or)
MERGE_ON_READ");
+
+ public static final ConfigOption<String> OPERATION = ConfigOptions
+ .key("write.operation")
+ .stringType()
+ .defaultValue("upsert")
+ .withDescription("The write operation, that this write should do");
+
+ public static final ConfigOption<String> PRECOMBINE_FIELD = ConfigOptions
+ .key("write.precombine.field")
Review comment:
this is also available in `HoodieWriteConfig`, payload, kengen as well.
If the config is commonly used by both engines, I think we could add new items
to `HoodieWriteConfig` as well.
##########
File path: hudi-flink/src/main/java/org/apache/hudi/operator/FlinkOptions.java
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.hudi.operator;
+
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.streamer.FlinkStreamerConfig;
+import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
+import org.apache.hudi.keygen.SimpleAvroKeyGenerator;
+import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
+import org.apache.hudi.util.StreamerUtil;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.Configuration;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Hoodie Flink config options.
+ *
+ * <p>It has the options for Hoodie table read and write. It also defines some
utilities.
+ */
+public class FlinkOptions {
+ private FlinkOptions() {
+ }
+
+ // ------------------------------------------------------------------------
+ // Base Options
+ // ------------------------------------------------------------------------
+ public static final ConfigOption<String> PATH = ConfigOptions
+ .key("path")
+ .stringType()
+ .noDefaultValue()
+ .withDescription("Base path for the target hoodie table."
+ + "\nThe path would be created if it does not exist,\n"
+ + "otherwise a Hoodie table expects to be initialized successfully");
+
+ public static final ConfigOption<String> PROPS_FILE_PATH = ConfigOptions
+ .key("properties-file.path")
+ .stringType()
+ .noDefaultValue()
+ .withDescription("Path to properties file on local-fs or dfs, with
configurations for \n"
+ + "hoodie client, schema provider, key generator and data source.
For hoodie client props, sane defaults are\n"
+ + "used, but recommend use to provide basic things like metrics
endpoints, hive configs etc. For sources, refer\n"
+ + "to individual classes, for supported properties");
+
+ // ------------------------------------------------------------------------
+ // Read Options
+ // ------------------------------------------------------------------------
+ public static final ConfigOption<String> READ_SCHEMA_FILE_PATH =
ConfigOptions
+ .key("read.schema.file.path")
+ .stringType()
+ .noDefaultValue()
+ .withDescription("Avro schema file path, the parsed schema is used for
deserializing");
+
+ // ------------------------------------------------------------------------
+ // Write Options
+ // ------------------------------------------------------------------------
+ public static final ConfigOption<String> TABLE_NAME = ConfigOptions
+ .key(HoodieWriteConfig.TABLE_NAME)
+ .stringType()
+ .noDefaultValue()
+ .withDescription("Table name to register to Hive metastore");
+
+ public static final ConfigOption<String> TABLE_TYPE = ConfigOptions
+ .key("write.table.type")
+ .stringType()
+ .defaultValue("COPY_ON_WRITE")
+ .withDescription("Type of table to write. COPY_ON_WRITE (or)
MERGE_ON_READ");
+
+ public static final ConfigOption<String> OPERATION = ConfigOptions
+ .key("write.operation")
+ .stringType()
+ .defaultValue("upsert")
+ .withDescription("The write operation, that this write should do");
+
+ public static final ConfigOption<String> PRECOMBINE_FIELD = ConfigOptions
+ .key("write.precombine.field")
+ .stringType()
+ .defaultValue("ts")
+ .withDescription("Field used in preCombining before actual write. When
two records have the same\n"
+ + "key value, we will pick the one with the largest value for the
precombine field,\n"
+ + "determined by Object.compareTo(..)");
+
+ public static final ConfigOption<String> PAYLOAD_CLASS = ConfigOptions
+ .key("write.payload.class")
+ .stringType()
+ .defaultValue(OverwriteWithLatestAvroPayload.class.getName())
+ .withDescription("Payload class used. Override this, if you like to roll
your own merge logic, when upserting/inserting.\n"
+ + "This will render any value set for the option in-effective");
+
+ /**
+ * Flag to indicate whether to drop duplicates upon insert.
+ * By default insert will accept duplicates, to gain extra performance.
+ */
+ public static final ConfigOption<Boolean> INSERT_DROP_DUPS = ConfigOptions
+ .key("write.insert.drop.duplicates")
+ .booleanType()
+ .defaultValue(false)
+ .withDescription("Flag to indicate whether to drop duplicates upon
insert.\n"
+ + "By default insert will accept duplicates, to gain extra
performance");
+
+ public static final ConfigOption<Integer> RETRY_TIMES = ConfigOptions
+ .key("write.retry.times")
+ .intType()
+ .defaultValue(3)
+ .withDescription("Flag to indicate how many times streaming job should
retry for a failed checkpoint batch.\n"
+ + "By default 3");
+
+ public static final ConfigOption<Long> RETRY_INTERVAL_MS = ConfigOptions
+ .key("write.retry.interval.ms")
+ .longType()
+ .defaultValue(2000L)
+ .withDescription("Flag to indicate how long (by millisecond) before a
retry should issued for failed checkpoint batch.\n"
+ + "By default 2000 and it will be doubled by every retry");
+
+ public static final ConfigOption<Boolean> IGNORE_FAILED_BATCH = ConfigOptions
+ .key("write.ignore.failed.batch")
+ .booleanType()
+ .defaultValue(true)
+ .withDescription("Flag to indicate whether to ignore any non exception
error (e.g. writestatus error). within a checkpoint batch.\n"
+ + "By default true (in favor of streaming progressing over data
integrity)");
+
+ public static final ConfigOption<String> RECORD_KEY_FIELD = ConfigOptions
+ .key(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY)
+ .stringType()
+ .defaultValue("uuid")
+ .withDescription("Record key field. Value to be used as the `recordKey`
component of `HoodieKey`.\n"
+ + "Actual value will be obtained by invoking .toString() on the
field value. Nested fields can be specified using "
+ + "the dot notation eg: `a.b.c`");
+
+ public static final ConfigOption<String> PARTITION_PATH_FIELD = ConfigOptions
+ .key(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY)
+ .stringType()
+ .defaultValue("partition-path")
+ .withDescription("Partition path field. Value to be used at the
`partitionPath` component of `HoodieKey`.\n"
+ + "Actual value obtained by invoking .toString()");
+
+ public static final ConfigOption<String> KEYGEN_CLASS = ConfigOptions
+ .key("hoodie.datasource.write.keygenerator.class")
+ .stringType()
+ .defaultValue(SimpleAvroKeyGenerator.class.getName())
+ .withDescription("Key generator class, that implements will extract the
key out of incoming record");
+
+ public static final ConfigOption<Integer> WRITE_TASK_PARALLELISM =
ConfigOptions
+ .key("write.task.parallelism")
+ .intType()
+ .defaultValue(4)
+ .withDescription("Parallelism of tasks that do actual write, default is
4");
+
Review comment:
maybe add another config to specify index type?
##########
File path:
hudi-flink/src/main/java/org/apache/hudi/operator/StreamWriteOperatorCoordinator.java
##########
@@ -0,0 +1,419 @@
+/*
+ * 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.hudi.operator;
+
+import org.apache.hudi.client.FlinkTaskContextSupplier;
+import org.apache.hudi.client.HoodieFlinkWriteClient;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.operator.event.BatchWriteSuccessEvent;
+import org.apache.hudi.util.StreamerUtil;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.util.Preconditions;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.jetbrains.annotations.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+/**
+ * {@link OperatorCoordinator} for {@link StreamWriteFunction}.
+ *
+ * <p>This coordinator starts a new instant when a new checkpoint starts. It
commits the instant when all the
+ * operator tasks write the buffer successfully for a round of checkpoint.
+ *
+ * <p>If there is no data for a round of checkpointing, it rolls back the
metadata.
+ *
+ * @see StreamWriteFunction for the work flow and semantics
+ */
+public class StreamWriteOperatorCoordinator
+ implements OperatorCoordinator {
+ private static final Logger LOG =
LoggerFactory.getLogger(StreamWriteOperatorCoordinator.class);
+
+ /**
+ * Config options.
+ */
+ private final Configuration conf;
+
+ /**
+ * Write client.
+ */
+ private transient HoodieFlinkWriteClient writeClient;
+
+ private long inFlightCheckpoint = -1;
+
+ /**
+ * Current REQUESTED instant, for validation.
+ */
+ private String inFlightInstant = "";
+
+ /**
+ * Event buffer for one round of checkpointing. When all the elements are
non-null and have the same
+ * write instant, then the instant succeed and we can commit it.
+ */
+ private transient BatchWriteSuccessEvent[] eventBuffer;
+
+ /**
+ * Task number of the operator.
+ */
+ private final int parallelism;
+
+ /**
+ * Constructs a StreamingSinkOperatorCoordinator.
+ *
+ * @param conf The config options
+ * @param parallelism The operator task number
+ */
+ public StreamWriteOperatorCoordinator(
+ Configuration conf,
+ int parallelism) {
+ this.conf = conf;
+ this.parallelism = parallelism;
+ }
+
+ @Override
+ public void start() throws Exception {
+ // initialize event buffer
+ reset();
+ // writeClient
+ initWriteClient();
+ // init table, create it if not exists.
+ initTable();
+ }
+
+ @Override
+ public void close() {
+ if (writeClient != null) {
+ writeClient.close();
+ }
+ this.eventBuffer = null;
+ }
+
+ @Override
+ public void checkpointCoordinator(long checkpointId,
CompletableFuture<byte[]> result) {
+ try {
+ final String errMsg = "A new checkpoint starts while the last checkpoint
buffer"
+ + " data has not finish writing, roll back the last write and throw";
+ checkAndForceCommit(errMsg);
+ this.inFlightInstant = this.writeClient.startCommit();
+ this.inFlightCheckpoint = checkpointId;
+ LOG.info("Create instant [{}], at checkpoint [{}]",
this.inFlightInstant, checkpointId);
+ result.complete(writeCheckpointBytes());
+ } catch (Throwable throwable) {
+ // when a checkpoint fails, throws directly.
+ result.completeExceptionally(
+ new CompletionException(
+ String.format("Failed to checkpoint Instant %s for source %s",
+ this.inFlightInstant, this.getClass().getSimpleName()),
throwable));
+ }
+ }
+
+ @Override
+ public void notifyCheckpointComplete(long checkpointId) {
+ // start to commit the instant.
+ checkAndCommitWithRetry();
+ }
+
+ @Override
+ public void notifyCheckpointAborted(long checkpointId) {
+ Preconditions.checkState(inFlightCheckpoint == checkpointId,
+ "The aborted checkpoint should always be the last checkpoint");
+ checkAndForceCommit("The last checkpoint was aborted, roll back the last
write and throw");
+ }
+
+ @Override
+ public void resetToCheckpoint(long l, @Nullable byte[] checkpointData)
throws Exception {
+ if (checkpointData != null) {
+ // restore when any checkpoint completed
+ deserializeCheckpointAndRestore(checkpointData);
+ }
+ }
+
+ @Override
+ public void handleEventFromOperator(int i, OperatorEvent operatorEvent) {
+ // no event to handle
+ Preconditions.checkState(operatorEvent instanceof BatchWriteSuccessEvent,
+ "The coordinator can only handle BatchWriteSuccessEvent");
+ BatchWriteSuccessEvent event = (BatchWriteSuccessEvent) operatorEvent;
+
Preconditions.checkState(event.getInstantTime().equals(this.inFlightInstant),
+ String.format("Receive an unexpected event for instant %s from task
%d",
+ event.getInstantTime(), event.getTaskID()));
+ this.eventBuffer[event.getTaskID()] = event;
+ }
+
+ @Override
+ public void subtaskFailed(int i, @Nullable Throwable throwable) {
+ // no operation
+ }
+
+ @Override
+ public void subtaskReset(int i, long l) {
+ // no operation
+ }
+
+ // -------------------------------------------------------------------------
+ // Utilities
+ // -------------------------------------------------------------------------
+
+ @SuppressWarnings("rawtypes")
+ private void initWriteClient() {
+ writeClient = new HoodieFlinkWriteClient(
+ new HoodieFlinkEngineContext(new FlinkTaskContextSupplier(null)),
+ StreamerUtil.getHoodieClientConfig(this.conf),
+ true);
+ }
+
+ private void initTable() throws IOException {
+ final String basePath = this.conf.getString(FlinkOptions.PATH);
+ final org.apache.hadoop.conf.Configuration hadoopConf =
StreamerUtil.getHadoopConf();
+ // Hadoop FileSystem
+ try (FileSystem fs = FSUtils.getFs(basePath, hadoopConf)) {
+ if (!fs.exists(new Path(basePath,
HoodieTableMetaClient.METAFOLDER_NAME))) {
+ HoodieTableMetaClient.initTableType(
+ hadoopConf,
+ basePath,
+
HoodieTableType.valueOf(this.conf.getString(FlinkOptions.TABLE_TYPE)),
+ this.conf.getString(FlinkOptions.TABLE_NAME),
+ "archived",
+ this.conf.getString(FlinkOptions.PAYLOAD_CLASS),
+ 1);
+ LOG.info("Table initialized");
+ } else {
+ LOG.info("Table [{}/{}] already exists, no need to initialize the
table",
+ basePath, this.conf.getString(FlinkOptions.TABLE_NAME));
+ }
+ }
+ }
+
+ static byte[] readBytes(DataInputStream in, int size) throws IOException {
+ byte[] bytes = new byte[size];
+ in.readFully(bytes);
+ return bytes;
+ }
+
+ /**
+ * Serialize the coordinator state. The current implementation may not be
super efficient,
+ * but it should not matter that much because most of the state should be
rather small.
+ * Large states themselves may already be a problem regardless of how the
serialization
+ * is implemented.
+ *
+ * @return A byte array containing the serialized state of the source
coordinator.
+ * @throws IOException When something goes wrong in serialization.
+ */
+ private byte[] writeCheckpointBytes() throws IOException {
+ try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ DataOutputStream out = new DataOutputViewStreamWrapper(baos)) {
+
+ out.writeLong(this.inFlightCheckpoint);
+ byte[] serializedInstant = this.inFlightInstant.getBytes();
+ out.writeInt(serializedInstant.length);
+ out.write(serializedInstant);
+ out.flush();
+ return baos.toByteArray();
+ }
+ }
+
+ /**
+ * Restore the state of this source coordinator from the state bytes.
+ *
+ * @param bytes The checkpoint bytes that was returned from {@link
#writeCheckpointBytes()}
+ * @throws Exception When the deserialization failed.
+ */
+ private void deserializeCheckpointAndRestore(byte[] bytes) throws Exception {
+ try (ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
+ DataInputStream in = new DataInputViewStreamWrapper(bais)) {
+ long checkpointID = in.readLong();
+ int serializedInstantSize = in.readInt();
+ byte[] serializedInstant = readBytes(in, serializedInstantSize);
+ this.inFlightCheckpoint = checkpointID;
+ this.inFlightInstant = new String(serializedInstant);
+ }
+ }
+
+ private void reset() {
+ this.inFlightInstant = "";
+ this.eventBuffer = new BatchWriteSuccessEvent[this.parallelism];
+ }
+
+ private void checkAndForceCommit(String errMsg) {
+ if (!checkReady()) {
+ // forced but still has inflight instant
+ String inflightInstant =
writeClient.getInflightAndRequestedInstant(this.conf.getString(FlinkOptions.TABLE_TYPE));
+ if (inflightInstant != null) {
+ assert inflightInstant.equals(this.inFlightInstant);
+ writeClient.rollback(this.inFlightInstant);
+ throw new HoodieException(errMsg);
+ }
+ if (Arrays.stream(eventBuffer).allMatch(Objects::isNull)) {
+ // The last checkpoint finished successfully.
+ return;
+ }
+ }
+ doCommit();
+ }
+
+ private void checkAndCommitWithRetry() {
+ int retryTimes = this.conf.getInteger(FlinkOptions.RETRY_TIMES);
+ long retryIntervalMillis =
this.conf.getLong(FlinkOptions.RETRY_INTERVAL_MS);
+ int tryTimes = 0;
+ while (tryTimes++ < retryTimes) {
+ try {
+ if (!checkReady()) {
+ // Do not throw if the try times expires but the event buffer are
still not ready,
+ // because we have a force check when next checkpoint starts.
+ waitFor(retryIntervalMillis);
+ continue;
+ }
+ doCommit();
+ return;
+ } catch (Throwable throwable) {
+ String cause = throwable.getCause() == null ? "" :
throwable.getCause().toString();
+ LOG.warn("Try to commit the instant {} failed, with times {} and cause
{}", this.inFlightInstant, tryTimes, cause);
+ if (tryTimes == retryTimes) {
+ throw new HoodieException(throwable);
+ }
+ waitFor(retryIntervalMillis);
+ }
+ }
+ }
+
+ private void waitFor(long intervalMillis) {
+ try {
+ TimeUnit.MILLISECONDS.sleep(intervalMillis);
+ } catch (InterruptedException e) {
+ LOG.error("Thread interrupted while waiting to retry the instant
commits");
+ throw new HoodieException(e);
+ }
+ }
+
+ /** Checks the buffer is ready to commit. */
+ private boolean checkReady() {
+ return Arrays.stream(eventBuffer).allMatch(event ->
+ event != null && event.getInstantTime().equals(this.inFlightInstant));
+ }
+
+ /** Performs the actual commit action. */
+ private void doCommit() {
+ List<WriteStatus> writeResults = Arrays.stream(eventBuffer)
+ .map(BatchWriteSuccessEvent::getWriteStatuses)
+ .flatMap(Collection::stream)
+ .collect(Collectors.toList());
+
+ if (writeResults.size() == 0) {
+ // No data has written, clear the metadata file
+
this.writeClient.deletePendingInstant(this.conf.getString(FlinkOptions.TABLE_TYPE),
this.inFlightInstant);
+ reset();
+ return;
+ }
+
+ // commit or rollback
+ long totalErrorRecords =
writeResults.stream().map(WriteStatus::getTotalErrorRecords).reduce(Long::sum).orElse(0L);
+ long totalRecords =
writeResults.stream().map(WriteStatus::getTotalRecords).reduce(Long::sum).orElse(0L);
+ boolean hasErrors = totalErrorRecords > 0;
+
+ if (!hasErrors || this.conf.getBoolean(FlinkOptions.IGNORE_FAILED_BATCH)) {
+ HashMap<String, String> checkpointCommitMetadata = new HashMap<>();
+ if (hasErrors) {
+ LOG.warn("Some records failed to merge but forcing commit since
commitOnErrors set to true. Errors/Total="
+ + totalErrorRecords + "/" + totalRecords);
+ }
+
+ boolean success = writeClient.commit(this.inFlightInstant, writeResults,
Option.of(checkpointCommitMetadata));
+ if (success) {
+ reset();
+ LOG.info("Commit instant [{}] success!", this.inFlightInstant);
+ } else {
+ throw new HoodieException(String.format("Commit instant [%s] failed!",
this.inFlightInstant));
+ }
+ } else {
+ LOG.error("Error when writing. Errors/Total=" + totalErrorRecords + "/"
+ totalRecords);
+ LOG.error("The first 100 error messages");
+
writeResults.stream().filter(WriteStatus::hasErrors).limit(100).forEach(ws -> {
+ LOG.error("Global error for partition path {} and fileID {}: {}",
+ ws.getGlobalError(), ws.getPartitionPath(), ws.getFileId());
+ if (ws.getErrors().size() > 0) {
+ ws.getErrors().forEach((key, value) -> LOG.trace("Error for key:" +
key + " and value " + value));
+ }
+ });
+ // Rolls back instant
+ writeClient.rollback(this.inFlightInstant);
+ throw new HoodieException(String.format("Commit instant [%s] failed and
rolled back !", this.inFlightInstant));
+ }
+ }
+
+ @VisibleForTesting
+ public BatchWriteSuccessEvent[] getEventBuffer() {
+ return eventBuffer;
+ }
+
+ @VisibleForTesting
+ public String getInFlightInstant() {
Review comment:
is this only visible for testing? IIUC, the actual writer need to know
the current instant and add `_hoodie_commit_time` field to the record. How did
the writer know the current instant?
##########
File path:
hudi-flink/src/main/java/org/apache/hudi/operator/StreamWriteFunction.java
##########
@@ -0,0 +1,342 @@
+/*
+ * 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.hudi.operator;
+
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.client.FlinkTaskContextSupplier;
+import org.apache.hudi.client.HoodieFlinkWriteClient;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.util.ObjectSizeCalculator;
+import org.apache.hudi.keygen.KeyGenerator;
+import org.apache.hudi.operator.event.BatchWriteSuccessEvent;
+import org.apache.hudi.util.StreamerUtil;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.formats.avro.RowDataToAvroConverters;
+import org.apache.flink.runtime.operators.coordination.OperatorEventGateway;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.BiFunction;
+
+/**
+ * Sink function to write the data to the underneath filesystem.
+ *
+ * <p><h2>Work Flow</h2>
+ *
+ * <p>The function firstly buffers the data as a batch of {@link
HoodieRecord}s,
+ * It flushes(write) the records batch when a Flink checkpoint starts. After a
batch has been written successfully,
+ * the function notifies its operator coordinator {@link
StreamWriteOperatorCoordinator} to mark a successful write.
+ *
+ * <p><h2>Exactly-once Semantics</h2>
+ *
+ * <p>The task implements exactly-once semantics by buffering the data between
checkpoints. The operator coordinator
+ * starts a new instant on the time line when a checkpoint triggers, the
coordinator checkpoints always
+ * start before its operator, so when this function starts a checkpoint, a
REQUESTED instant already exists.
+ * The function process thread then block data buffering and the checkpoint
thread starts flushing the existing data buffer.
+ * When the existing data buffer write successfully, the process thread
unblock and start buffering again for the next round checkpoint.
+ * Because any checkpoint failures would trigger the write rollback, it
implements the exactly-once semantics.
+ *
+ * <p><h2>Fault Tolerance</h2>
+ *
+ * <p>The operator coordinator checks the validity for the last instant when
it starts a new one. The operator rolls back
+ * the written data and throws when any error occurs. This means any
checkpoint or task failure would trigger a failover.
+ * The operator coordinator would try several times when committing the
writestatus.
+ *
+ * <p>Note: The function task requires the input stream be partitioned by the
partition fields to avoid different write tasks
+ * write to the same file group that conflict. The general case for partition
path is a datetime field,
+ * so the sink task is very possible to have IO bottleneck, the more flexible
solution is to shuffle the
+ * data by the file group IDs.
+ *
+ * @param <I> Type of the input record
+ * @see StreamWriteOperatorCoordinator
+ */
+public class StreamWriteFunction<K, I, O>
+ extends KeyedProcessFunction<K, I, O>
+ implements CheckpointedFunction {
+
+ private static final long serialVersionUID = 1L;
+
+ private static final Logger LOG =
LoggerFactory.getLogger(StreamWriteFunction.class);
+
+ /**
+ * Write buffer for a checkpoint.
+ */
+ private transient List<HoodieRecord> buffer;
+
+ /**
+ * The buffer lock to control data buffering/flushing.
+ */
+ private transient ReentrantLock bufferLock;
+
+ /**
+ * The condition to decide whether to add new records into the buffer.
+ */
+ private transient Condition addToBufferCondition;
+
+ /**
+ * Flag saying whether there is an on-going checkpoint.
+ */
+ private volatile boolean onCheckpointing = false;
+
+ /**
+ * Config options.
+ */
+ private final Configuration config;
+
+ /**
+ * Id of current subtask.
+ */
+ private int taskID;
+
+ /**
+ * Write Client.
+ */
+ private transient HoodieFlinkWriteClient writeClient;
+
+ private transient BiFunction<List<HoodieRecord>, String, List<WriteStatus>>
writeFunction;
+
+ /**
+ * HoodieKey generator.
+ */
+ private transient KeyGenerator keyGenerator;
+
+ /**
+ * Row type of the input.
+ */
+ private final RowType rowType;
+
+ /**
+ * Avro schema of the input.
+ */
+ private final Schema avroSchema;
+
+ private transient RowDataToAvroConverters.RowDataToAvroConverter converter;
+
+ /**
+ * The REQUESTED instant we write the data.
+ */
+ private volatile String currentInstant;
+
+ /**
+ * Gateway to send operator events to the operator coordinator.
+ */
+ private transient OperatorEventGateway eventGateway;
+
+ /**
+ * Constructs a StreamingSinkFunction.
+ *
+ * @param rowType The input row type
+ * @param config The config options
+ */
+ public StreamWriteFunction(RowType rowType, Configuration config) {
+ this.rowType = rowType;
+ this.avroSchema = org.apache.flink.formats.avro.typeutils
+ .AvroSchemaConverter.convertToSchema(rowType);
+ this.config = config;
+ }
+
+ @Override
+ public void open(Configuration parameters) throws IOException {
+ this.taskID = getRuntimeContext().getIndexOfThisSubtask();
+ this.keyGenerator =
StreamerUtil.createKeyGenerator(HoodieOptions.flatOptions(this.config));
+ this.converter = RowDataToAvroConverters.createConverter(this.rowType);
+ initBuffer();
+ initWriteClient();
+ initWriteFunction();
+ }
+
+ @Override
+ public void initializeState(FunctionInitializationContext context) {
+ // no operation
+ }
+
+ @Override
+ public void snapshotState(FunctionSnapshotContext functionSnapshotContext) {
+ bufferLock.lock();
+ try {
+ // Based on the fact that the coordinator starts the checkpoint first,
+ // it would check the validity.
+ this.onCheckpointing = true;
+ this.currentInstant =
this.writeClient.getInflightAndRequestedInstant(this.config.get(HoodieOptions.TABLE_TYPE));
+ Preconditions.checkNotNull(this.currentInstant,
+ "No inflight instant when flushing data");
+ // wait for the buffer data flush out and request a new instant
+ flushBuffer();
+ // signal the task thread to start buffering
+ addToBufferCondition.signal();
+ } finally {
+ this.onCheckpointing = false;
+ bufferLock.unlock();
+ }
+ }
+
+ @Override
+ public void processElement(I value, KeyedProcessFunction<K, I, O>.Context
ctx, Collector<O> out) throws Exception {
+ bufferLock.lock();
+ try {
+ if (onCheckpointing) {
+ addToBufferCondition.await();
+ }
+ this.buffer.add(toHoodie(value));
+ } finally {
+ bufferLock.unlock();
+ }
+ }
+
+ @Override
+ public void close() {
+ if (this.writeClient != null) {
+ this.writeClient.close();
+ }
+ }
+
+ // -------------------------------------------------------------------------
+ // Getter/Setter
+ // -------------------------------------------------------------------------
+
+ @VisibleForTesting
+ @SuppressWarnings("rawtypes")
+ public List<HoodieRecord> getBuffer() {
+ return buffer;
+ }
+
+ @VisibleForTesting
+ @SuppressWarnings("rawtypes")
+ public HoodieFlinkWriteClient getWriteClient() {
+ return writeClient;
+ }
+
+ public void setOperatorEventGateway(OperatorEventGateway
operatorEventGateway) {
+ this.eventGateway = operatorEventGateway;
+ }
+
+ // -------------------------------------------------------------------------
+ // Utilities
+ // -------------------------------------------------------------------------
+
+ private void initBuffer() {
+ this.buffer = new ArrayList<>();
+ this.bufferLock = new ReentrantLock();
+ this.addToBufferCondition = this.bufferLock.newCondition();
+ }
+
+ private void initWriteClient() {
+ HoodieFlinkEngineContext context =
+ new HoodieFlinkEngineContext(
+ new SerializableConfiguration(StreamerUtil.getHadoopConf()),
+ new FlinkTaskContextSupplier(getRuntimeContext()));
+
+ writeClient = new HoodieFlinkWriteClient<>(context,
StreamerUtil.getHoodieClientConfig(this.config));
+ }
+
+ private void initWriteFunction() {
+ final String writeOperation = this.config.get(HoodieOptions.OPERATION);
+ switch (WriteOperationType.fromValue(writeOperation)) {
+ case INSERT:
+ this.writeFunction = (records, instantTime) ->
this.writeClient.insert(records, instantTime);
+ break;
+ case UPSERT:
+ this.writeFunction = (records, instantTime) ->
this.writeClient.upsert(records, instantTime);
+ break;
+ default:
+ throw new RuntimeException("Unsupported write operation : " +
writeOperation);
+ }
+ }
+
+ // Keep for mini-batch write.
+ private static class BufferSizeEstimator {
Review comment:
Can we add some docs here. Still a little bit confused about how this
will be used.
----------------------------------------------------------------
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.
For queries about this service, please contact Infrastructure at:
[email protected]