-
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
[HUDI-1522] Add a new pipeline for Flink writer #2430
Conversation
+ "the `partitionPath` component of `HoodieKey`. Actual value obtained by invoking .toString(). By default `partitionpath`.") | ||
public String partitionPathField = "partitionpath"; | ||
|
||
@Parameter(names = {"--partition-path-field"}, description = "Key generator class, that implements will extract the key out of incoming record.\n" |
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.
little mistake --partition-path-field
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.
Oops, thanks for the reminder.
Codecov Report
@@ Coverage Diff @@
## master #2430 +/- ##
============================================
+ Coverage 50.18% 50.27% +0.09%
- Complexity 3050 3119 +69
============================================
Files 419 430 +11
Lines 18931 19561 +630
Branches 1948 2003 +55
============================================
+ Hits 9500 9835 +335
- Misses 8656 8923 +267
- Partials 775 803 +28
Flags with carried forward coverage won't be shown. Click here to find out more.
|
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.
@danny0405 thanks for your contribution. Just have a quick look, left some comments.
env.setStateBackend(new FsStateBackend(cfg.flinkCheckPointPath)); | ||
} | ||
|
||
Properties kafkaProps = StreamerUtil.getKafkaProps(cfg); |
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.
And can we extract the same logic both in v1 and v2?
import com.beust.jcommander.IStringConverter; | ||
import com.beust.jcommander.ParameterException; | ||
|
||
/** Converter that converts a string into enum WriteOperationType. */ |
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.
Can we keep the consistency to make the comment of class level align to the existing classes?
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.
There is a OperationConverter
in org.apache.hudi.utilities.deltastreamer
let us use single one.
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.
org.apache.hudi.utilities.deltastreamer
OperationConverter
is in a different module, i would suggest to not add dependency to that. The hoodie-flink
module should not depend on the hudi-utilities
.
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 I mean is that can we find a way to move this class into a common place?
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.
We can, if we fond more usages of this class, in current code base, only 2 places use it, and the class itself is pretty simple, it does not deserve to move the class to common utils.
|
||
/** Utilities for Flink stream read and write. */ |
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.
ditto
public static org.apache.hadoop.conf.Configuration getHadoopConf() { | ||
// create HiveConf from hadoop configuration with hadoop conf directory configured. | ||
org.apache.hadoop.conf.Configuration hadoopConf = null; | ||
for (String possibleHadoopConfPath : HadoopUtils.possibleHadoopConfPaths(new Configuration())) { |
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.
Do we really need to find a possible config file that we may not know its config information? Or only a new Object just enough?
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.
We need to find the correct hadoop configuration to create correct hadoop filesystem, this is also useful for HoodieFlinkEngineContext
.
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 I mean is that can we specify a discovery-order, e.g. first find the specified path, and then find a batch of default paths.
In some scenarios where storage is separated from computing, or data synchronization scenarios, users may not necessarily want to load the local "default" configuration.
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.
The method firstly find the specified path fs.hdfs.hadoopconf
, then directory HADOOP_CONF_DIR
HADOOP_HOME/conf
HADOOP_HOME/etc/hadoop
from the system environment.
Even if storage is separated from computing, the FileSystem
we created is still correct, if we split the hadoop conf files correctly.
In any case, we should not pass an empty hadoop configuration.
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.
The method firstly find the specified path
fs.hdfs.hadoopconf
, then directoryHADOOP_CONF_DIR
HADOOP_HOME/conf
HADOOP_HOME/etc/hadoop
from the system environment.
I have watched the source code before raising this concern.
Even if storage is separated from computing, the
FileSystem
we created is still correct, if we split the hadoop conf files correctly.In any case, we should not pass an empty hadoop configuration.
I mean, do we allow the user's explicit parameter assignment as the highest priority? Greater than the default convention that some users may not know?
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.
I think it is reasonable that the explicit specified path has the highest priority.
* @return A Hadoop configuration instance. | ||
*/ | ||
private static org.apache.hadoop.conf.Configuration getHadoopConfiguration(String hadoopConfDir) { | ||
if (new File(hadoopConfDir).exists()) { |
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.
Can we use Files.exists()
to test?
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.
Do you mean java.nio.file.Files
? I guess new File(hadoopConfDir).exists()
is more simpler.
// Modify to be compatible with old version Avro. | ||
if (genericRecord.getSchema().getField(isDeleteKey) == null) { | ||
return false; | ||
} | ||
Object deleteMarker = genericRecord.get("_hoodie_is_deleted"); |
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 not change the literal to the variable that you defined?
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.
We should fix the logic for Avro version that higher than 1.10, because the higher version throws directly if the field name does not exist in the schema.
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.
I mean you have defined the isDeleteKey
, why do not change the literal in this line?
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.
Okey, got your idea, thanks ~
hudi-flink/pom.xml
Outdated
<dependency> | ||
<groupId>org.mockito</groupId> | ||
<artifactId>mockito-junit-jupiter</artifactId> | ||
<groupId>org.apache.flink</groupId> |
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.
Can you make the changes in this file cleaner?
* | ||
* <p>It has the options for Hoodie table read and write. It also defines some utilities. | ||
*/ | ||
public class HoodieOptions { |
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.
Does HoodieFlinkConfigOptions
sounds better?
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.
Personally i prefer cleaner and shorter names, because options are phrase of Flink connectors, and it is under the package org.apache.hudi.operator
, we can make it more specific if we introduce other config options class.
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.
fine, it's not a key issue, different thought is OK.
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.
+1 for adding Flink
into the class name. We might add more engine support in the future. Use HoodieOptions
make more sense to me if we add this connector under Flink's codebase.
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.
Fine to add Flink
to the class name.
} | ||
}); | ||
checkPropNames.forEach(prop -> | ||
Preconditions.checkState( |
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 can not merge the next two lines into this line?
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.
Personally i prefer shorter line code. Refactoring it if Hoodie thinks long line code is ok.
7c96c0c
to
3e3ef16
Compare
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.
@danny0405 Thanks for your contribution, looks promising! Left some comments
* | ||
* <p>It has the options for Hoodie table read and write. It also defines some utilities. | ||
*/ | ||
public class HoodieOptions { |
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.
+1 for adding Flink
into the class name. We might add more engine support in the future. Use HoodieOptions
make more sense to me if we add this connector under Flink's codebase.
// Write Options | ||
// ------------------------------------------------------------------------ | ||
public static final ConfigOption<String> TABLE_NAME = ConfigOptions | ||
.key("write.table.name") |
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.
Can we reuse the config under HoodieWriteConfiguration
? That would be great if we can use the same set of config for both Spark and Flink.
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.
Agree, thanks.
} | ||
|
||
// Keep for mini-batch write. | ||
private static class BufferSizeEstimator { |
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.
This data also available in the WriteStatus
. We have TotalWriteBytes
and TotalRecords
e.t.c. How are we gonna use this later on?
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.
To test the data buffer we received, the statistics in WriteStatus
are all about written data which is different.
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.
Can we add some docs here. Still a little bit confused about how this will be used.
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.
Where do we use this class?
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.
Removed.
* @throws IOException if error occurs | ||
*/ | ||
@SuppressWarnings("rawtypes") | ||
private HoodieRecord toHoodie(I record) throws IOException { |
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.
nit: toHoodieRecord
?
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.
Agree, thanks ~
} catch (Throwable throwable) { | ||
// when a checkpoint fails, throws directly. | ||
result.completeExceptionally( | ||
new CompletionException( |
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.
How about using a hoodie internal exception here?
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.
No, this is needed by the Flink
coordinator.
implements CoordinatedOperatorFactory<Object>, OneInputStreamOperatorFactory<I, Object> { | ||
private static final long serialVersionUID = 1L; | ||
|
||
private static final long DEFAULT_MAX_BYTES_PER_BATCH = (1 << 21) * 128; // 256MB |
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.
For the later use?
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.
Yes, removed.
*/ | ||
public class OperationConverter implements IStringConverter<WriteOperationType> { | ||
@Override | ||
public WriteOperationType convert(String value) throws ParameterException { |
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.
This is available in the WriteOperationType
. Can we reuse from there?
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.
The class is used by the HoodieFlinkStreamer
.
* Create a key generator class via reflection, passing in any configs needed. | ||
* <p> | ||
* If the class name of key generator is configured through the properties file, i.e., {@code props}, use the corresponding key generator class; otherwise, use the default key generator class | ||
* specified in {@code DataSourceWriteOptions}. |
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.
nit: this linked to Spark code
return this.function.getWriteClient(); | ||
} | ||
|
||
public void checkpointFunction(long checkpointId) throws Exception { |
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.
Where is this method being called? How do we generate the checkpointId?
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.
checkpointId is specified by the test cases, the test case needs to invoke the checkpointFunction
by itself.
/** | ||
* Configurations for Hoodie Flink streamer. | ||
*/ | ||
public class Config extends Configuration { |
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.
IMO, this name is too generic. It would be better to make it more readable. What about renaming to FlinkStreamerConfig
?
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.
Fine with the suggest name.
@Parameter(names = {"--kafka-group-id"}, description = "kafka consumer group id", required = true) | ||
public String kafkaGroupId; | ||
|
||
@Parameter(names = {"--kafka-bootstrap-servers"}, description = "kafka bootstrap.servers", required = true) |
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.
There are many minor issues for the description items of these config options, e.g., describe it with a normal sentence(take care of the grammar?), Upper the first letter? In short, keep consistent.
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.
Okey
Thanks @garyli1019 @yanghua for the review, i have updated based on your comments. |
b46310d
to
9324312
Compare
hudi-flink/src/main/java/org/apache/hudi/operator/StreamWriteOperatorFactory.java
Show resolved
Hide resolved
hudi-flink/src/main/java/org/apache/hudi/schema/FilebasedSchemaProvider.java
Show resolved
Hide resolved
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) { |
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 does the old version Avro mean here? This seems like being handled by the check on return
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.
Hmm, i mean the new version Avro, let me fix the comment.
.intType() | ||
.defaultValue(4) | ||
.withDescription("Parallelism of tasks that do actual write, default is 4"); | ||
|
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.
maybe add another config to specify index type?
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.
We can if we support more index type, currently the new pipeline only supports one. (Although not in this PR, but it is my plan though ~ I don't want to add it then remove ~)
} | ||
|
||
// Keep for mini-batch write. | ||
private static class BufferSizeEstimator { |
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.
Can we add some docs here. Still a little bit confused about how this will be used.
} | ||
|
||
@VisibleForTesting | ||
public String getInFlightInstant() { |
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.
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?
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.
How did the writer know the current instant
Fetch through the HoodieFlinkWriteClient
, actually the HoodieTableMetaClient
.
hudi-flink/src/main/java/org/apache/hudi/operator/FlinkOptions.java
Outdated
Show resolved
Hide resolved
hudi-flink/src/main/java/org/apache/hudi/operator/FlinkOptions.java
Outdated
Show resolved
Hide resolved
hudi-flink/src/main/java/org/apache/hudi/operator/FlinkOptions.java
Outdated
Show resolved
Hide resolved
hudi-flink/src/main/java/org/apache/hudi/operator/StreamWriteFunction.java
Outdated
Show resolved
Hide resolved
LOG.info("No data to write in subtask [{}] for instant [{}]", taskID, currentInstant); | ||
writeStatus = Collections.emptyList(); | ||
} | ||
this.eventGateway.sendEventToCoordinator(new BatchWriteSuccessEvent(this.taskID, currentInstant, writeStatus)); |
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.
Do we need a retry or ack mechanism here? What's the consequence if the coordinator does not receive the event?
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.
Failover when next checkpoint starts. There is a check in the StreamWriteOperatorCoordinator.checkpointCoordinator
.
hudi-flink/src/main/java/org/apache/hudi/operator/StreamWriteFunction.java
Show resolved
Hide resolved
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.
@danny0405 Left some comments.
hudi-flink/src/main/java/org/apache/hudi/operator/event/BatchWriteSuccessEvent.java
Outdated
Show resolved
Hide resolved
* @param instantTime The instant time under which to write the data | ||
* @param writeStatuses The write statues list | ||
*/ | ||
public BatchWriteSuccessEvent( |
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.
unnecessary comment for the constructor? it's easy to understand.
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.
Removed.
@Parameter(names = {"--flink-checkpoint-path"}, description = "Flink checkpoint path.") | ||
public String flinkCheckPointPath; | ||
|
||
@Parameter(names = {"--flink-block-retry-times"}, description = "Times to retry when latest instant has not completed.") |
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.
block
seems hard to understand. Any better word?
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.
Copied from the old code, how about instant-retry-times
?
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.
sounds good
hudi-flink/src/main/java/org/apache/hudi/streamer/HoodieFlinkStreamerV2.java
Outdated
Show resolved
Hide resolved
hudi-flink/src/main/java/org/apache/hudi/streamer/HoodieFlinkStreamer.java
Outdated
Show resolved
Hide resolved
This is the #step 1 of RFC-24: https://cwiki.apache.org/confluence/display/HUDI/RFC+-+24%3A+Hoodie+Flink+Writer+Proposal * The new pipeline removes the InstantGenerateOperator and CommitSink, add StreamWriteOperatorCoordinator to achieve the same function * Add FlinkOptions for configuration of all the Flink operators, this is also useful for the SQL connectors which would be introduced in following contributions * Add UT and IT cases for the new operators
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.
@danny0405 Left some comments.
.key("write.table.type") | ||
.stringType() | ||
.defaultValue("COPY_ON_WRITE") | ||
.withDescription("Type of table to write. COPY_ON_WRITE (or) MERGE_ON_READ"); |
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.
remove .
after write
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.
Replace with ,
instead.
public static final ConfigOption<String> TABLE_TYPE = ConfigOptions | ||
.key("write.table.type") | ||
.stringType() | ||
.defaultValue("COPY_ON_WRITE") |
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.
let us reduce the string literal via HoodieTableType.COPY_ON_WRITE.name()
?
+ "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") |
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.
write.batch.ignore.failed
sounds better? This config option describes a boolean flag, right?
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.
Remove 'batch' because Flink may evolve to streaming write in the near future.
|| WriteOperationType.fromValue(this.config.getString(FlinkOptions.OPERATION)) == WriteOperationType.UPSERT; | ||
GenericRecord gr = (GenericRecord) this.converter.convert(this.avroSchema, record); | ||
HoodieRecordPayload payload = shouldCombine | ||
? StreamerUtil.createPayload(this.config.getString(FlinkOptions.PAYLOAD_CLASS), gr, |
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.
This method createPayload
is too long and hard to read. Can you make it easy to read?
} | ||
} | ||
|
||
private void waitFor(long intervalMillis) { |
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.
waitFor
what? Let us give it a more readable name?
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.
rename to sleepFor
.
doCommit(); | ||
return; | ||
} catch (Throwable throwable) { | ||
String cause = throwable.getCause() == null ? "" : throwable.getCause().toString(); |
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.
Did you check that how many exception categories? If there is not necessary to still retry, may we accept fast failed
strategy?
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.
Not sure how many exception categories there ~, so we'd better keep the re-try because Flink is a streaming system, less failover is better, we can add the check for the exception categories in the future if it is needs, (for example, if there is a exception that does not deserve to retry, just fail fast.)
The user can forbidden the retry by setting the retry times to a negative though ~
@danny0405 sorry for the delay on review, I was super busy this week. The bloom index was merged to master, can we add the bloom index option to this PR as well? |
I'm not planning to using the BloomFilter index in the new pipeline, instead there is a BloomFilter index backed state index in the following PR, which is more suitable for streaming write. |
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.
@danny0405 Kindred thanks for your contribution and patient. I approve this PR now. If there are some minor issues, let's fix them later. @wangxianghu @garyli1019 Do you still have any concern, I will timeout and merge it tomorrow morning!
@danny0405 oh, CI failed... Please fix it before merging, 3ks. |
@danny0405 yes, using bloom index in a streaming fashion is what we are planning to do next as well. No worry for now, we can try this PR out and continue the work on top of this. Really appreciate your work! This PR LGTM. |
What is the purpose of the pull request
This is the #step 1 of RFC-24:
https://cwiki.apache.org/confluence/display/HUDI/RFC+-+24%3A+Hoodie+Flink+Writer+Proposal
in order to add a new pipeline to avoid the single parallelism operators.
Brief change log
StreamWriteOperatorCoordinator to achieve the same function
is also useful for the SQL connectors which would be introduced in
following contributions
Verify this pull request
StreamWriteFunctionTest
to test theStreamWriteFunction
and the related componentsStreamWriteITCase
is a IT to test all the operatorsCommitter checklist
Has a corresponding JIRA in PR title & commit
Commit message is descriptive of the change
CI is green
Necessary doc changes done or have another open PR
For large changes, please consider breaking it into sub-tasks under an umbrella JIRA.