Hisoka-X commented on code in PR #2485:
URL:
https://github.com/apache/incubator-seatunnel/pull/2485#discussion_r950981512
##########
seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-api/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/api/AbstractCheckpointStorage.java:
##########
@@ -79,4 +96,75 @@ public byte[] serializeCheckPointData(PipelineState state)
throws IOException {
public PipelineState deserializeCheckPointData(byte[] data) throws
IOException {
return serializer.deserialize(data, PipelineState.class);
}
+
+ public void setStorageNameSpace(String storageNameSpace) {
+ if (storageNameSpace != null) {
+ this.storageNameSpace = storageNameSpace;
+ }
+ }
+
+ public Set<String> getLatestPipelineNames(List<String> fileNames) {
+ Map<String, String> latestPipelineMap = new HashMap<>();
+ fileNames.forEach(fileName -> {
+ String[] fileNameSegments = fileName.split(FILE_NAME_SPLIT);
+ int fileVersion =
Integer.parseInt(fileNameSegments[FILE_SORT_ID_INDEX]);
+ String filePipelineId =
fileNameSegments[FILE_NAME_PIPELINE_ID_INDEX];
+ if (latestPipelineMap.containsKey(filePipelineId)) {
+ int oldVersion =
Integer.parseInt(latestPipelineMap.get(filePipelineId).split(FILE_NAME_SPLIT)[FILE_SORT_ID_INDEX]);
+ if (fileVersion > oldVersion) {
+ latestPipelineMap.put(filePipelineId, fileName);
+ }
+ } else {
+ latestPipelineMap.put(filePipelineId, fileName);
+ }
+ });
+ Set<String> latestPipelines = new HashSet<>(latestPipelineMap.size());
+ latestPipelineMap.forEach((pipelineId, fileName) ->
latestPipelines.add(fileName));
+ return latestPipelines;
+ }
+
+ /**
+ * get latest checkpoint file name
+ *
+ * @param fileNames file names
+ * @return latest checkpoint file name
+ */
+ public String getLatestCheckpointFileNameByJobIdAndPipelineId(List<String>
fileNames, String pipelineId) {
+ AtomicReference<String> latestFileName = new AtomicReference<>();
+ AtomicInteger latestVersion = new AtomicInteger();
+ fileNames.forEach(fileName -> {
+ String[] fileNameSegments = fileName.split(FILE_NAME_SPLIT);
+ int fileVersion =
Integer.parseInt(fileNameSegments[FILE_SORT_ID_INDEX]);
+ String filePipelineId =
fileNameSegments[FILE_NAME_PIPELINE_ID_INDEX];
+ if (pipelineId.equals(filePipelineId) && fileVersion >
latestVersion.get()) {
+ latestVersion.set(fileVersion);
+ latestFileName.set(fileName);
+ }
+ });
+ return latestFileName.get();
+ }
+
+ /**
+ * get the latest checkpoint file name
+ *
+ * @param fileName file names. note: file name cannot contain parent path
+ * @return latest checkpoint file name
+ */
+ public String getPipelineIdByFileName(String fileName) {
+ return fileName.split(FILE_NAME_SPLIT)[FILE_NAME_PIPELINE_ID_INDEX];
+ }
+
+ @Override
+ public void asyncStoreCheckPoint(PipelineState state) {
Review Comment:
Does this method have multi-thread access?If so, the executor maybe create
not only one time.
--
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]