-
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-1511] InstantGenerateOperator support multiple parallelism #2434
Conversation
reopen with new changes |
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.
@loukey-lj Thanks for your contribution. Quickly check and left some comments.
@@ -71,6 +71,7 @@ | |||
|
|||
private static final long serialVersionUID = 1L; | |||
private static final Logger LOG = LogManager.getLogger(HoodieTableMetaClient.class); | |||
public static final String INSTANT_GENERATE_FOLDER_NAME = "instant_generate"; |
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.
Renaming to instant_generate_tmp
looks 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.
move it to hudi-flink
module would be 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.
moved to InstantGenerateOperator
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java
Outdated
Show resolved
Hide resolved
@@ -272,7 +273,7 @@ public HoodieWrapperFileSystem getFs() { | |||
|
|||
/** | |||
* Return raw file-system. | |||
* |
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
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.
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.
Or just revert this file's change via git command.
@@ -420,6 +421,12 @@ public static HoodieTableMetaClient initTableAndGetMetaClient(Configuration hado | |||
} | |||
} | |||
|
|||
// Always create instantGenerateFolder which is needed for InstantGenerateOperator | |||
final Path instantGenerateFolder = new Path(basePath, HoodieTableMetaClient.INSTANT_GENERATE_FOLDER_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.
@wangxianghu Do we have a better place to put this change? It's common package.
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.
@wangxianghu Do we have a better place to put this change? It's common package.
How about StreamerUtil
? this operation is only used in HoodieFlinkStreamer
WDYT @loukey-lj
@@ -18,6 +18,18 @@ | |||
|
|||
package org.apache.hudi.operator; | |||
|
|||
import org.apache.flink.api.common.state.ListState; |
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 revert unnecessary changes? Pay attention to the import order.
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.
reverted
TaskContextSupplier taskContextSupplier = new FlinkTaskContextSupplier(null); | ||
if (isMain) { | ||
// retry times | ||
retryTimes = Integer.valueOf(cfg.blockRetryTime); |
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 may need to verify the config opinion to see if it's valuable?
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.
config have default value
// writeClient | ||
writeClient = new HoodieFlinkWriteClient(new HoodieFlinkEngineContext(taskContextSupplier), StreamerUtil.getHoodieClientConfig(cfg), true); | ||
// retry interval | ||
retryInterval = Integer.valueOf(cfg.blockRetryInterval); |
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?
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.
config have default value
// last instant completed, set it empty | ||
latestInstant = ""; | ||
int indexOfThisSubtask = getRuntimeContext().getIndexOfThisSubtask(); | ||
String instantGenerateInfoFileName = indexOfThisSubtask + UNDERLINE + checkpointId + UNDERLINE + batchSize; |
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's use String.format()
?
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
fs.create(path,true); | ||
LOG.info("subtask [{}] at checkpoint [{}] created generate file [{}]",indexOfThisSubtask,checkpointId,instantGenerateInfoFileName); | ||
if (isMain) { | ||
boolean hasData = generateFilePasre(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.
receivedDataInCurrentCP
?
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
|
||
} | ||
|
||
private boolean generateFilePasre(long checkpointId) throws InterruptedException, 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.
checkReceivedData
?
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
Codecov Report
@@ Coverage Diff @@
## master #2434 +/- ##
============================================
- Coverage 53.56% 9.68% -43.88%
+ Complexity 2774 48 -2726
============================================
Files 348 53 -295
Lines 16117 1930 -14187
Branches 1641 230 -1411
============================================
- Hits 8633 187 -8446
+ Misses 6785 1730 -5055
+ Partials 699 13 -686
Flags with carried forward coverage won't be shown. Click here to find out more. |
hudi-flink/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java
Outdated
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.
@loukey-lj Left some comments.
latestInstant = ""; | ||
} | ||
int indexOfThisSubtask = getRuntimeContext().getIndexOfThisSubtask(); | ||
String instantGenerateInfoFileName = String.format("%d_%d_%d",indexOfThisSubtask,checkpointId,batchSize); |
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.
Many lines have the same issue, please change the code style like this: xxx,xxx
-> xxx, xxx
.
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.
formated
return hasData; | ||
} | ||
|
||
private void createinstantGenerateTmpFolder() 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.
-> createInstantGenerateTmpDir
?
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.
ok
} | ||
} | ||
|
||
boolean hasData = false; |
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.
-> receivedData
?
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.
ok
if (context.isRestored()) { | ||
Iterator<String> latestInstantIterator = latestInstantState.get().iterator(); | ||
latestInstantIterator.forEachRemaining(x -> latestInstant = x); | ||
LOG.info("InstantGenerateOperator initializeState get latestInstant [{}]", latestInstant); |
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.
Restoring the latest instant [{}] from the state.
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.
ok
} | ||
} | ||
|
||
@Override | ||
public void open() throws Exception { | ||
super.open(); | ||
isMain = getRuntimeContext().getIndexOfThisSubtask() == 0; |
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.
Wet call getRuntimeContext().getIndexOfThisSubtask()
several times, can we define a variable to store it?
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.
called in open and initializeState method, initializeState method start before open method . So I removed the code in open method.
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.
@loukey-lj Left another comments.
@@ -102,65 +105,76 @@ public void open() throws Exception { | |||
// Hadoop FileSystem | |||
fs = FSUtils.getFs(cfg.targetBasePath, serializableHadoopConf.get()); | |||
|
|||
TaskContextSupplier taskContextSupplier = new FlinkTaskContextSupplier(null); | |||
if (isMain) { | |||
TaskContextSupplier taskContextSupplier = new FlinkTaskContextSupplier(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.
IMO, here we can set the argument of FlinkTaskContextSupplier to the instance of RuntimeContext
, 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.
wdyt about this review suggestion?
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.
done
// init table, create it if not exists. | ||
initTable(); | ||
|
||
// create instantGenerateTmpFolder |
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.
About the comment, we'd better avoid copying the method name, create temp folder for generating instant
seems 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.
changed to 'create instant marker directory'
fileStatuses = fs.listStatus(generatePath, new PathFilter() { | ||
@Override | ||
public boolean accept(Path pathname) { | ||
return pathname.getName().contains(String.format("_%d_", 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.
You have defined the separator by UNDERLINE
, it would be better to use that. wdyt?
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 this suggestion be accepted?
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.
ok
hudi-flink/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java
Outdated
Show resolved
Hide resolved
private boolean checkReceivedData(long checkpointId) throws InterruptedException, IOException { | ||
int numberOfParallelSubtasks = getRuntimeContext().getNumberOfParallelSubtasks(); | ||
FileStatus[] fileStatuses = null; | ||
Path generatePath = new Path(INSTANT_GENERATE_FOLDER_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.
@loukey-lj how about moving this path to .hoodie/.aux
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
private Integer retryTimes; | ||
private Integer retryInterval; | ||
private static final String UNDERLINE = "_"; | ||
private static final String INSTANT_GENERATE_FOLDER_NAME = "instant_generate_tmp"; |
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.
INSTANT_GENERATE_FOLDER_NAME = ".instant_marker"
?
WDYT ? @yanghua
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
…, set batchSize to thread safe
…, set batchSize to thread safe
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java
Outdated
Show resolved
Hide resolved
@@ -272,7 +273,7 @@ public HoodieWrapperFileSystem getFs() { | |||
|
|||
/** | |||
* Return raw file-system. | |||
* |
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.
Or just revert this file's change via git command.
private static final String UNDERLINE = "_"; | ||
private static final String INSTANT_MARKER_FOLDER_NAME = ".instant_marker"; | ||
private transient boolean isMain = false; | ||
private transient AtomicLong batchSize = new AtomicLong(0); |
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 still think cpRecordCounter
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.
agree
@@ -102,65 +105,76 @@ public void open() throws Exception { | |||
// Hadoop FileSystem | |||
fs = FSUtils.getFs(cfg.targetBasePath, serializableHadoopConf.get()); | |||
|
|||
TaskContextSupplier taskContextSupplier = new FlinkTaskContextSupplier(null); | |||
if (isMain) { | |||
TaskContextSupplier taskContextSupplier = new FlinkTaskContextSupplier(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.
wdyt about this review suggestion?
} else { | ||
latestInstantList.set(0, latestInstant); | ||
LOG.info("Records size [{}] checkpointId [{}]", batchSize, 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.
Task instance %d received %d records in checkpoint [%d]
looks 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.
done
fileStatuses = fs.listStatus(generatePath, new PathFilter() { | ||
@Override | ||
public boolean accept(Path pathname) { | ||
return pathname.getName().contains(String.format("_%d_", 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.
Can this suggestion be accepted?
hudi-flink/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java
Outdated
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.
@loukey-lj We are close to merging this PR, and there are several minor issues that need to be fixed.
// no data no new instant | ||
if (!bufferedRecords.isEmpty()) { | ||
latestInstant = startNewInstant(checkpointId); | ||
// no data no new 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.
IMO, we can move boolean receivedDataInCurrentCP = checkReceivedData(checkpointId);
to this line. This way we can delay calling the method.
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.
done
Path path = new Path(new Path(HoodieTableMetaClient.AUXILIARYFOLDER_NAME, INSTANT_MARKER_FOLDER_NAME), instantMarkerFileName); | ||
// mk generate file by each subtask | ||
fs.create(path, true); | ||
LOG.info("Subtask [{}] at checkpoint [{}] created generate file [{}]", indexOfThisSubtask, checkpointId, instantMarkerFileName); |
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.
generate
-> marker
?
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.
done
} | ||
|
||
if (tryTimes >= 5) { | ||
LOG.warn("waiting generate file, checkpointId [{}]", 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.
generate
-> marker
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.
done
} | ||
String instantMarkerFileName = String.format("%d%s%d%s%d", indexOfThisSubtask, DELIMITER, checkpointId, DELIMITER, recordCounter.get()); | ||
Path path = new Path(new Path(HoodieTableMetaClient.AUXILIARYFOLDER_NAME, INSTANT_MARKER_FOLDER_NAME), instantMarkerFileName); | ||
// mk generate file by each subtask |
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.
generate
-> marker
?
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.
done
|
||
if (tryTimes >= 5) { | ||
LOG.warn("waiting marker file, checkpointId [{}]", checkpointId); | ||
tryTimes = 0; |
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 there a bug? when tryTimes >= 5
, only reinit it to 0? Do not break? If the logic is correct, why we need this counter?
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.
@loukey-lj Thanks for your effort! LGTM now. @wangxianghu Do you still have any concerns?
|
Hi @vinothchandar Can we merge this PR, right now? |
@yanghua please wait till I make the release branch. We also need to stabilize CI before landing anymore. it will be very hard to track down otherwise. |
@yanghua should be good to land now if you are happy with it |
ack, thanks. |
InstantGenerateOperator support multiple parallelism.
When InstantGenerateOperator subtask size greater than 1 we can set subtask 0 as a main subtask, only main task create new instant.
The prerequisite of create new instant is exist subtask received data in current checkpoint. Every subtask will create a tmp file,
flie name is make up by checkpointid,subtask index and received records size.
The main subtask will check every subtask file and parse file to make sure is shuold to create new instant.