You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@seatunnel.apache.org by GitBox <gi...@apache.org> on 2022/08/20 15:12:41 UTC

[GitHub] [incubator-seatunnel] CalvinKirs opened a new pull request, #2485: [New-Engine]Support Hdfs Storage and async storage

CalvinKirs opened a new pull request, #2485:
URL: https://github.com/apache/incubator-seatunnel/pull/2485

   [New-Engine]Support Hdfs Storage and async storage
   Support config storage namespace
   Support windows local file storage
   Support async submit the task


-- 
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: commits-unsubscribe@seatunnel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-seatunnel] ashulin commented on a diff in pull request #2485: [New-Engine]Support Hdfs Storage and async storage

Posted by GitBox <gi...@apache.org>.
ashulin commented on code in PR #2485:
URL: https://github.com/apache/incubator-seatunnel/pull/2485#discussion_r950862538


##########
seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-api/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/common/ProtoStuffSerializer.java:
##########
@@ -48,12 +43,13 @@ private static <T> Schema<T> getSchema(Class<T> clazz) {
     @Override
     public <T> byte[] serialize(T obj) {
         Class<T> clazz = (Class<T>) obj.getClass();
+        LinkedBuffer buffer = LinkedBuffer.allocate(LinkedBuffer.DEFAULT_BUFFER_SIZE);

Review Comment:
   I don't quite understand the change here, could you please explain it?



-- 
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: commits-unsubscribe@seatunnel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-seatunnel] CalvinKirs commented on a diff in pull request #2485: [New-Engine]Support Hdfs Storage and async storage

Posted by GitBox <gi...@apache.org>.
CalvinKirs commented on code in PR #2485:
URL: https://github.com/apache/incubator-seatunnel/pull/2485#discussion_r950941282


##########
seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-hdfs/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/hdfs/HdfsStorage.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.seatunnel.engine.checkpoint.storage.hdfs;
+
+import static org.apache.seatunnel.engine.checkpoint.storage.constants.StorageConstants.STORAGE_NAME_SPACE;
+
+import org.apache.seatunnel.engine.checkpoint.storage.PipelineState;
+import org.apache.seatunnel.engine.checkpoint.storage.api.AbstractCheckpointStorage;
+import org.apache.seatunnel.engine.checkpoint.storage.exception.CheckpointStorageException;
+
+import lombok.extern.slf4j.Slf4j;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+@Slf4j
+public class HdfsStorage extends AbstractCheckpointStorage {
+
+    public FileSystem fs;
+    private static final String HADOOP_SECURITY_AUTHENTICATION_KEY = "hadoop.security.authentication";
+
+    private static final String KERBEROS_KEY = "kerberos";
+
+    public HdfsStorage(Map<String, String> configuration) throws CheckpointStorageException {
+        this.initStorage(configuration);
+    }
+
+    @Override
+    public void initStorage(Map<String, String> configuration) throws CheckpointStorageException {
+        Configuration hadoopConf = new Configuration();
+        if (configuration.containsKey(HdfsConstants.HDFS_DEF_FS_NAME)) {
+            hadoopConf.set(HdfsConstants.HDFS_DEF_FS_NAME, configuration.get(HdfsConstants.HDFS_DEF_FS_NAME));
+        }
+        if (StringUtils.isNotBlank(configuration.get(STORAGE_NAME_SPACE))) {
+            setStorageNameSpace(configuration.get(STORAGE_NAME_SPACE));
+        }
+        // todo support other config configurations
+        if (configuration.containsKey(HdfsConstants.KERBEROS_PRINCIPAL) && configuration.containsKey(HdfsConstants.KERBEROS_KEYTAB_FILE_PATH)) {
+            String kerberosPrincipal = configuration.get(HdfsConstants.KERBEROS_PRINCIPAL);
+            String kerberosKeytabFilePath = configuration.get(HdfsConstants.KERBEROS_KEYTAB_FILE_PATH);
+            if (StringUtils.isNotBlank(kerberosPrincipal) && StringUtils.isNotBlank(kerberosKeytabFilePath)) {
+                hadoopConf.set(HADOOP_SECURITY_AUTHENTICATION_KEY, KERBEROS_KEY);
+                authenticateKerberos(kerberosPrincipal, kerberosKeytabFilePath, hadoopConf);
+            }
+        }
+        JobConf jobConf = new JobConf(hadoopConf);
+        try {
+            fs = FileSystem.get(jobConf);
+        } catch (IOException e) {
+            throw new CheckpointStorageException("Failed to get file system", e);
+        }
+
+    }
+
+    @Override
+    public String storeCheckPoint(PipelineState state) throws CheckpointStorageException {
+        byte[] datas;
+        try {
+            datas = serializeCheckPointData(state);
+        } catch (IOException e) {
+            throw new CheckpointStorageException("Failed to serialize checkpoint data", e);
+        }
+        //Consider file paths for different operating systems
+        String fileName = getStorageParentDirectory() + state.getJobId() + "/" + getCheckPointName(state);
+        try (FSDataOutputStream out = fs.create(new Path(fileName))) {
+            out.write(datas);
+        } catch (IOException e) {
+            throw new CheckpointStorageException("Failed to write checkpoint data", e);
+        }
+        return fileName;
+    }
+
+    @Override
+    public List<PipelineState> getAllCheckpoints(String jobId) throws CheckpointStorageException {
+        String path = getStorageParentDirectory() + jobId;
+        List<String> fileNames = getFileNames(path);
+        if (fileNames.isEmpty()) {
+            throw new CheckpointStorageException("No checkpoint found for job, job id is: " + jobId);
+        }
+        List<PipelineState> states = new ArrayList<>();
+        fileNames.forEach(file -> {
+            try {
+                states.add(readPipelineState(file, jobId));
+            } catch (CheckpointStorageException e) {
+                log.error("Failed to read checkpoint data from file: " + file, e);
+            }
+        });
+        if (states.isEmpty()) {
+            throw new CheckpointStorageException("No checkpoint found for job, job id is: " + jobId);
+        }
+        return states;
+    }
+
+    @Override
+    public List<PipelineState> getLatestCheckpoint(String jobId) throws CheckpointStorageException {
+        String path = getStorageParentDirectory() + jobId;
+        List<String> fileNames = getFileNames(path);
+        if (fileNames.isEmpty()) {
+            throw new CheckpointStorageException("No checkpoint found for job, job id is: " + jobId);
+        }
+        Set<String> latestPipelineNames = getLatestPipelineNames(fileNames);
+        List<PipelineState> latestPipelineStates = new ArrayList<>();
+        latestPipelineNames.forEach(fileName -> {
+            try {
+                latestPipelineStates.add(readPipelineState(fileName, jobId));
+            } catch (CheckpointStorageException e) {
+                log.error("Failed to read pipeline state for file: {}", fileName, e);

Review Comment:
   No. This method is what it is, if you read its interface documentation.



-- 
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: commits-unsubscribe@seatunnel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-seatunnel] CalvinKirs commented on a diff in pull request #2485: [New-Engine]Support Hdfs Storage and async storage

Posted by GitBox <gi...@apache.org>.
CalvinKirs commented on code in PR #2485:
URL: https://github.com/apache/incubator-seatunnel/pull/2485#discussion_r950944127


##########
seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-hdfs/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/hdfs/HdfsStorage.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.seatunnel.engine.checkpoint.storage.hdfs;
+
+import static org.apache.seatunnel.engine.checkpoint.storage.constants.StorageConstants.STORAGE_NAME_SPACE;
+
+import org.apache.seatunnel.engine.checkpoint.storage.PipelineState;
+import org.apache.seatunnel.engine.checkpoint.storage.api.AbstractCheckpointStorage;
+import org.apache.seatunnel.engine.checkpoint.storage.exception.CheckpointStorageException;
+
+import lombok.extern.slf4j.Slf4j;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+@Slf4j
+public class HdfsStorage extends AbstractCheckpointStorage {
+
+    public FileSystem fs;
+    private static final String HADOOP_SECURITY_AUTHENTICATION_KEY = "hadoop.security.authentication";
+
+    private static final String KERBEROS_KEY = "kerberos";
+
+    public HdfsStorage(Map<String, String> configuration) throws CheckpointStorageException {
+        this.initStorage(configuration);
+    }
+
+    @Override
+    public void initStorage(Map<String, String> configuration) throws CheckpointStorageException {
+        Configuration hadoopConf = new Configuration();
+        if (configuration.containsKey(HdfsConstants.HDFS_DEF_FS_NAME)) {
+            hadoopConf.set(HdfsConstants.HDFS_DEF_FS_NAME, configuration.get(HdfsConstants.HDFS_DEF_FS_NAME));
+        }
+        if (StringUtils.isNotBlank(configuration.get(STORAGE_NAME_SPACE))) {
+            setStorageNameSpace(configuration.get(STORAGE_NAME_SPACE));
+        }
+        // todo support other config configurations
+        if (configuration.containsKey(HdfsConstants.KERBEROS_PRINCIPAL) && configuration.containsKey(HdfsConstants.KERBEROS_KEYTAB_FILE_PATH)) {
+            String kerberosPrincipal = configuration.get(HdfsConstants.KERBEROS_PRINCIPAL);
+            String kerberosKeytabFilePath = configuration.get(HdfsConstants.KERBEROS_KEYTAB_FILE_PATH);
+            if (StringUtils.isNotBlank(kerberosPrincipal) && StringUtils.isNotBlank(kerberosKeytabFilePath)) {
+                hadoopConf.set(HADOOP_SECURITY_AUTHENTICATION_KEY, KERBEROS_KEY);
+                authenticateKerberos(kerberosPrincipal, kerberosKeytabFilePath, hadoopConf);
+            }
+        }
+        JobConf jobConf = new JobConf(hadoopConf);
+        try {
+            fs = FileSystem.get(jobConf);
+        } catch (IOException e) {
+            throw new CheckpointStorageException("Failed to get file system", e);
+        }
+
+    }
+
+    @Override
+    public String storeCheckPoint(PipelineState state) throws CheckpointStorageException {
+        byte[] datas;
+        try {
+            datas = serializeCheckPointData(state);
+        } catch (IOException e) {
+            throw new CheckpointStorageException("Failed to serialize checkpoint data", e);
+        }
+        //Consider file paths for different operating systems
+        String fileName = getStorageParentDirectory() + state.getJobId() + "/" + getCheckPointName(state);
+        try (FSDataOutputStream out = fs.create(new Path(fileName))) {
+            out.write(datas);
+        } catch (IOException e) {
+            throw new CheckpointStorageException("Failed to write checkpoint data", e);

Review Comment:
   > Write to temporary file(xxx.tmp) first and then rename?
   could you tell me why? I don't think it's good way.



##########
seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-hdfs/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/hdfs/HdfsStorage.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.seatunnel.engine.checkpoint.storage.hdfs;
+
+import static org.apache.seatunnel.engine.checkpoint.storage.constants.StorageConstants.STORAGE_NAME_SPACE;
+
+import org.apache.seatunnel.engine.checkpoint.storage.PipelineState;
+import org.apache.seatunnel.engine.checkpoint.storage.api.AbstractCheckpointStorage;
+import org.apache.seatunnel.engine.checkpoint.storage.exception.CheckpointStorageException;
+
+import lombok.extern.slf4j.Slf4j;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+@Slf4j
+public class HdfsStorage extends AbstractCheckpointStorage {
+
+    public FileSystem fs;
+    private static final String HADOOP_SECURITY_AUTHENTICATION_KEY = "hadoop.security.authentication";
+
+    private static final String KERBEROS_KEY = "kerberos";
+
+    public HdfsStorage(Map<String, String> configuration) throws CheckpointStorageException {
+        this.initStorage(configuration);
+    }
+
+    @Override
+    public void initStorage(Map<String, String> configuration) throws CheckpointStorageException {
+        Configuration hadoopConf = new Configuration();
+        if (configuration.containsKey(HdfsConstants.HDFS_DEF_FS_NAME)) {
+            hadoopConf.set(HdfsConstants.HDFS_DEF_FS_NAME, configuration.get(HdfsConstants.HDFS_DEF_FS_NAME));
+        }
+        if (StringUtils.isNotBlank(configuration.get(STORAGE_NAME_SPACE))) {
+            setStorageNameSpace(configuration.get(STORAGE_NAME_SPACE));
+        }
+        // todo support other config configurations
+        if (configuration.containsKey(HdfsConstants.KERBEROS_PRINCIPAL) && configuration.containsKey(HdfsConstants.KERBEROS_KEYTAB_FILE_PATH)) {
+            String kerberosPrincipal = configuration.get(HdfsConstants.KERBEROS_PRINCIPAL);
+            String kerberosKeytabFilePath = configuration.get(HdfsConstants.KERBEROS_KEYTAB_FILE_PATH);
+            if (StringUtils.isNotBlank(kerberosPrincipal) && StringUtils.isNotBlank(kerberosKeytabFilePath)) {
+                hadoopConf.set(HADOOP_SECURITY_AUTHENTICATION_KEY, KERBEROS_KEY);
+                authenticateKerberos(kerberosPrincipal, kerberosKeytabFilePath, hadoopConf);
+            }
+        }
+        JobConf jobConf = new JobConf(hadoopConf);
+        try {
+            fs = FileSystem.get(jobConf);
+        } catch (IOException e) {
+            throw new CheckpointStorageException("Failed to get file system", e);
+        }
+
+    }
+
+    @Override
+    public String storeCheckPoint(PipelineState state) throws CheckpointStorageException {
+        byte[] datas;
+        try {
+            datas = serializeCheckPointData(state);
+        } catch (IOException e) {
+            throw new CheckpointStorageException("Failed to serialize checkpoint data", e);
+        }
+        //Consider file paths for different operating systems
+        String fileName = getStorageParentDirectory() + state.getJobId() + "/" + getCheckPointName(state);
+        try (FSDataOutputStream out = fs.create(new Path(fileName))) {
+            out.write(datas);
+        } catch (IOException e) {
+            throw new CheckpointStorageException("Failed to write checkpoint data", e);

Review Comment:
   > Write to temporary file(xxx.tmp) first and then rename?
   
   could you tell me why? I don't think it's good way.



-- 
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: commits-unsubscribe@seatunnel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-seatunnel] CalvinKirs commented on a diff in pull request #2485: [New-Engine]Support Hdfs Storage and async storage

Posted by GitBox <gi...@apache.org>.
CalvinKirs commented on code in PR #2485:
URL: https://github.com/apache/incubator-seatunnel/pull/2485#discussion_r950941744


##########
seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-api/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/common/StorageThreadFactory.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.seatunnel.engine.checkpoint.storage.common;
+
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class StorageThreadFactory implements ThreadFactory {

Review Comment:
   For me, I prefer Java native.



-- 
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: commits-unsubscribe@seatunnel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-seatunnel] CalvinKirs commented on a diff in pull request #2485: [New-Engine]Support Hdfs Storage and async storage

Posted by GitBox <gi...@apache.org>.
CalvinKirs commented on code in PR #2485:
URL: https://github.com/apache/incubator-seatunnel/pull/2485#discussion_r950991556


##########
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:
   Good catch~



-- 
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: commits-unsubscribe@seatunnel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-seatunnel] Hisoka-X merged pull request #2485: [New-Engine]Support Hdfs Storage and async storage

Posted by GitBox <gi...@apache.org>.
Hisoka-X merged PR #2485:
URL: https://github.com/apache/incubator-seatunnel/pull/2485


-- 
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: commits-unsubscribe@seatunnel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-seatunnel] CalvinKirs commented on a diff in pull request #2485: [New-Engine]Support Hdfs Storage and async storage

Posted by GitBox <gi...@apache.org>.
CalvinKirs commented on code in PR #2485:
URL: https://github.com/apache/incubator-seatunnel/pull/2485#discussion_r953320957


##########
seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-api/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/api/AbstractCheckpointStorage.java:
##########
@@ -23,36 +23,58 @@
 import org.apache.seatunnel.engine.checkpoint.storage.PipelineState;
 import org.apache.seatunnel.engine.checkpoint.storage.common.ProtoStuffSerializer;
 import org.apache.seatunnel.engine.checkpoint.storage.common.Serializer;
+import org.apache.seatunnel.engine.checkpoint.storage.common.StorageThreadFactory;
 import org.apache.seatunnel.engine.checkpoint.storage.exception.CheckpointStorageException;
 
+import lombok.extern.slf4j.Slf4j;
+
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
 
+@Slf4j
 public abstract class AbstractCheckpointStorage implements CheckpointStorage {
 
     /**
      * serializer,default is protostuff,if necessary, consider other serialization methods, temporarily hard-coding
      */
     private final Serializer serializer = new ProtoStuffSerializer();
 
+    public static final String DEFAULT_CHECKPOINT_FILE_PATH_SPLIT = "/";
+
     /**
      * storage root directory
+     * if not set, use default value
      */
-    private static final String CHECKPOINT_DEFAULT_FILE_DIR = "/tmp/seatunnel/checkpoint/";
+    private String storageNameSpace = "/seatunnel/checkpoint/";

Review Comment:
   This seems redundant, for filesystems, which are usually configured by the user, and if the default directory is used, the naming `${seatunnel installation directory}` will be meaningless,



-- 
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: commits-unsubscribe@seatunnel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-seatunnel] CalvinKirs commented on a diff in pull request #2485: [New-Engine]Support Hdfs Storage and async storage

Posted by GitBox <gi...@apache.org>.
CalvinKirs commented on code in PR #2485:
URL: https://github.com/apache/incubator-seatunnel/pull/2485#discussion_r953270298


##########
seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-api/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/api/AbstractCheckpointStorage.java:
##########
@@ -79,4 +96,73 @@ 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 filePipelineId = fileName.split(FILE_NAME_SPLIT)[FILE_NAME_PIPELINE_ID_INDEX];
+            int fileVersion = Integer.parseInt(fileName.split(FILE_NAME_SPLIT)[FILE_SORT_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 -> {
+            int fileVersion = Integer.parseInt(fileName.split(FILE_NAME_SPLIT)[FILE_SORT_ID_INDEX]);
+            String filePipelineId = fileName.split(FILE_NAME_SPLIT)[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) {
+        if (null == this.executor || this.executor.isShutdown()) {
+            this.executor = Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors() * 2 + 1, new StorageThreadFactory());

Review Comment:
   It's great if so, I think we couldn't need `Map<String,String> config` I will update this in next PR



-- 
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: commits-unsubscribe@seatunnel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-seatunnel] CalvinKirs commented on a diff in pull request #2485: [New-Engine]Support Hdfs Storage and async storage

Posted by GitBox <gi...@apache.org>.
CalvinKirs commented on code in PR #2485:
URL: https://github.com/apache/incubator-seatunnel/pull/2485#discussion_r950940716


##########
seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-api/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/api/CheckpointStorage.java:
##########
@@ -35,6 +35,14 @@ public interface CheckpointStorage {
      */
     String storeCheckPoint(PipelineState state) throws CheckpointStorageException;
 
+    /**
+     * async save checkpoint to storage
+     *
+     * @param state PipelineState
+     * @throws CheckpointStorageException if save checkpoint failed
+     */
+    void asyncStoreCheckPoint(PipelineState state) throws CheckpointStorageException;

Review Comment:
   No



-- 
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: commits-unsubscribe@seatunnel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-seatunnel] hailin0 commented on a diff in pull request #2485: [New-Engine]Support Hdfs Storage and async storage

Posted by GitBox <gi...@apache.org>.
hailin0 commented on code in PR #2485:
URL: https://github.com/apache/incubator-seatunnel/pull/2485#discussion_r954422178


##########
seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-api/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/api/AbstractCheckpointStorage.java:
##########
@@ -23,36 +23,58 @@
 import org.apache.seatunnel.engine.checkpoint.storage.PipelineState;
 import org.apache.seatunnel.engine.checkpoint.storage.common.ProtoStuffSerializer;
 import org.apache.seatunnel.engine.checkpoint.storage.common.Serializer;
+import org.apache.seatunnel.engine.checkpoint.storage.common.StorageThreadFactory;
 import org.apache.seatunnel.engine.checkpoint.storage.exception.CheckpointStorageException;
 
+import lombok.extern.slf4j.Slf4j;
+
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
 
+@Slf4j
 public abstract class AbstractCheckpointStorage implements CheckpointStorage {
 
     /**
      * serializer,default is protostuff,if necessary, consider other serialization methods, temporarily hard-coding
      */
     private final Serializer serializer = new ProtoStuffSerializer();
 
+    public static final String DEFAULT_CHECKPOINT_FILE_PATH_SPLIT = "/";
+
     /**
      * storage root directory
+     * if not set, use default value
      */
-    private static final String CHECKPOINT_DEFAULT_FILE_DIR = "/tmp/seatunnel/checkpoint/";
+    private String storageNameSpace = "/seatunnel/checkpoint/";

Review Comment:
   👌



-- 
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: commits-unsubscribe@seatunnel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-seatunnel] CalvinKirs commented on a diff in pull request #2485: [New-Engine]Support Hdfs Storage and async storage

Posted by GitBox <gi...@apache.org>.
CalvinKirs commented on code in PR #2485:
URL: https://github.com/apache/incubator-seatunnel/pull/2485#discussion_r950949571


##########
seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-api/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/api/AbstractCheckpointStorage.java:
##########
@@ -79,4 +96,73 @@ 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 filePipelineId = fileName.split(FILE_NAME_SPLIT)[FILE_NAME_PIPELINE_ID_INDEX];
+            int fileVersion = Integer.parseInt(fileName.split(FILE_NAME_SPLIT)[FILE_SORT_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 -> {
+            int fileVersion = Integer.parseInt(fileName.split(FILE_NAME_SPLIT)[FILE_SORT_ID_INDEX]);
+            String filePipelineId = fileName.split(FILE_NAME_SPLIT)[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) {
+        if (null == this.executor || this.executor.isShutdown()) {
+            this.executor = Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors() * 2 + 1, new StorageThreadFactory());

Review Comment:
   We currently do not have a unified Config acquisition. We  use `System.getProperties` get,WDYT?



-- 
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: commits-unsubscribe@seatunnel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-seatunnel] CalvinKirs commented on a diff in pull request #2485: [New-Engine]Support Hdfs Storage and async storage

Posted by GitBox <gi...@apache.org>.
CalvinKirs commented on code in PR #2485:
URL: https://github.com/apache/incubator-seatunnel/pull/2485#discussion_r950940482


##########
seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-api/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/common/ProtoStuffSerializer.java:
##########
@@ -48,12 +43,13 @@ private static <T> Schema<T> getSchema(Class<T> clazz) {
     @Override
     public <T> byte[] serialize(T obj) {
         Class<T> clazz = (Class<T>) obj.getClass();
+        LinkedBuffer buffer = LinkedBuffer.allocate(LinkedBuffer.DEFAULT_BUFFER_SIZE);

Review Comment:
   Just serialized buffers.he cannot share.



-- 
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: commits-unsubscribe@seatunnel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-seatunnel] EricJoy2048 commented on a diff in pull request #2485: [New-Engine]Support Hdfs Storage and async storage

Posted by GitBox <gi...@apache.org>.
EricJoy2048 commented on code in PR #2485:
URL: https://github.com/apache/incubator-seatunnel/pull/2485#discussion_r953249998


##########
seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-api/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/api/AbstractCheckpointStorage.java:
##########
@@ -79,4 +96,73 @@ 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 filePipelineId = fileName.split(FILE_NAME_SPLIT)[FILE_NAME_PIPELINE_ID_INDEX];
+            int fileVersion = Integer.parseInt(fileName.split(FILE_NAME_SPLIT)[FILE_SORT_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 -> {
+            int fileVersion = Integer.parseInt(fileName.split(FILE_NAME_SPLIT)[FILE_SORT_ID_INDEX]);
+            String filePipelineId = fileName.split(FILE_NAME_SPLIT)[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) {
+        if (null == this.executor || this.executor.isShutdown()) {
+            this.executor = Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors() * 2 + 1, new StorageThreadFactory());

Review Comment:
   > We currently do not have a unified Config acquisition. We use `System.getProperties` get,WDYT?
   
   Actually, we already have config , look at `EngineConfig` class



-- 
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: commits-unsubscribe@seatunnel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-seatunnel] hailin0 commented on a diff in pull request #2485: [New-Engine]Support Hdfs Storage and async storage

Posted by GitBox <gi...@apache.org>.
hailin0 commented on code in PR #2485:
URL: https://github.com/apache/incubator-seatunnel/pull/2485#discussion_r953289946


##########
seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-hdfs/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/hdfs/HdfsStorage.java:
##########
@@ -0,0 +1,262 @@
+/*
+ * 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.seatunnel.engine.checkpoint.storage.hdfs;
+
+import static org.apache.seatunnel.engine.checkpoint.storage.constants.StorageConstants.STORAGE_NAME_SPACE;
+
+import org.apache.seatunnel.engine.checkpoint.storage.PipelineState;
+import org.apache.seatunnel.engine.checkpoint.storage.api.AbstractCheckpointStorage;
+import org.apache.seatunnel.engine.checkpoint.storage.exception.CheckpointStorageException;
+
+import lombok.extern.slf4j.Slf4j;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+@Slf4j
+public class HdfsStorage extends AbstractCheckpointStorage {
+
+    public FileSystem fs;
+    private static final String HADOOP_SECURITY_AUTHENTICATION_KEY = "hadoop.security.authentication";
+
+    private static final String KERBEROS_KEY = "kerberos";
+
+    private static final String STORAGE_TMP_SUFFIX = "tmp";
+
+    public HdfsStorage(Map<String, String> configuration) throws CheckpointStorageException {
+        this.initStorage(configuration);
+    }
+
+    @Override
+    public void initStorage(Map<String, String> configuration) throws CheckpointStorageException {
+        Configuration hadoopConf = new Configuration();
+        if (configuration.containsKey(HdfsConstants.HDFS_DEF_FS_NAME)) {
+            hadoopConf.set(HdfsConstants.HDFS_DEF_FS_NAME, configuration.get(HdfsConstants.HDFS_DEF_FS_NAME));
+        }
+        if (StringUtils.isNotBlank(configuration.get(STORAGE_NAME_SPACE))) {
+            setStorageNameSpace(configuration.get(STORAGE_NAME_SPACE));
+        }
+        // todo support other config configurations
+        if (configuration.containsKey(HdfsConstants.KERBEROS_PRINCIPAL) && configuration.containsKey(HdfsConstants.KERBEROS_KEYTAB_FILE_PATH)) {
+            String kerberosPrincipal = configuration.get(HdfsConstants.KERBEROS_PRINCIPAL);
+            String kerberosKeytabFilePath = configuration.get(HdfsConstants.KERBEROS_KEYTAB_FILE_PATH);
+            if (StringUtils.isNotBlank(kerberosPrincipal) && StringUtils.isNotBlank(kerberosKeytabFilePath)) {
+                hadoopConf.set(HADOOP_SECURITY_AUTHENTICATION_KEY, KERBEROS_KEY);
+                authenticateKerberos(kerberosPrincipal, kerberosKeytabFilePath, hadoopConf);
+            }
+        }
+        JobConf jobConf = new JobConf(hadoopConf);
+        try {
+            fs = FileSystem.get(jobConf);
+        } catch (IOException e) {
+            throw new CheckpointStorageException("Failed to get file system", e);
+        }
+
+    }
+
+    @Override
+    public String storeCheckPoint(PipelineState state) throws CheckpointStorageException {
+        byte[] datas;
+        try {
+            datas = serializeCheckPointData(state);
+        } catch (IOException e) {
+            throw new CheckpointStorageException("Failed to serialize checkpoint data,state is :" + state, e);
+        }
+        Path filePath = new Path(getStorageParentDirectory() + state.getJobId() + "/" + getCheckPointName(state));
+
+        Path tmpFilePath = new Path(getStorageParentDirectory() + state.getJobId() + "/" + getCheckPointName(state) + STORAGE_TMP_SUFFIX);
+        try (FSDataOutputStream out = fs.create(tmpFilePath)) {
+            out.write(datas);
+            out.hsync();
+            out.hflush();

Review Comment:
   hsync includes hflush capabilities?



##########
seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-api/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/api/AbstractCheckpointStorage.java:
##########
@@ -23,36 +23,58 @@
 import org.apache.seatunnel.engine.checkpoint.storage.PipelineState;
 import org.apache.seatunnel.engine.checkpoint.storage.common.ProtoStuffSerializer;
 import org.apache.seatunnel.engine.checkpoint.storage.common.Serializer;
+import org.apache.seatunnel.engine.checkpoint.storage.common.StorageThreadFactory;
 import org.apache.seatunnel.engine.checkpoint.storage.exception.CheckpointStorageException;
 
+import lombok.extern.slf4j.Slf4j;
+
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
 
+@Slf4j
 public abstract class AbstractCheckpointStorage implements CheckpointStorage {
 
     /**
      * serializer,default is protostuff,if necessary, consider other serialization methods, temporarily hard-coding
      */
     private final Serializer serializer = new ProtoStuffSerializer();
 
+    public static final String DEFAULT_CHECKPOINT_FILE_PATH_SPLIT = "/";
+
     /**
      * storage root directory
+     * if not set, use default value
      */
-    private static final String CHECKPOINT_DEFAULT_FILE_DIR = "/tmp/seatunnel/checkpoint/";
+    private String storageNameSpace = "/seatunnel/checkpoint/";

Review Comment:
   Use `${seatunnel installation directory}/checkpoint` ?



-- 
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: commits-unsubscribe@seatunnel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-seatunnel] CalvinKirs commented on a diff in pull request #2485: [New-Engine]Support Hdfs Storage and async storage

Posted by GitBox <gi...@apache.org>.
CalvinKirs commented on code in PR #2485:
URL: https://github.com/apache/incubator-seatunnel/pull/2485#discussion_r953321127


##########
seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-hdfs/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/hdfs/HdfsStorage.java:
##########
@@ -0,0 +1,262 @@
+/*
+ * 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.seatunnel.engine.checkpoint.storage.hdfs;
+
+import static org.apache.seatunnel.engine.checkpoint.storage.constants.StorageConstants.STORAGE_NAME_SPACE;
+
+import org.apache.seatunnel.engine.checkpoint.storage.PipelineState;
+import org.apache.seatunnel.engine.checkpoint.storage.api.AbstractCheckpointStorage;
+import org.apache.seatunnel.engine.checkpoint.storage.exception.CheckpointStorageException;
+
+import lombok.extern.slf4j.Slf4j;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+@Slf4j
+public class HdfsStorage extends AbstractCheckpointStorage {
+
+    public FileSystem fs;
+    private static final String HADOOP_SECURITY_AUTHENTICATION_KEY = "hadoop.security.authentication";
+
+    private static final String KERBEROS_KEY = "kerberos";
+
+    private static final String STORAGE_TMP_SUFFIX = "tmp";
+
+    public HdfsStorage(Map<String, String> configuration) throws CheckpointStorageException {
+        this.initStorage(configuration);
+    }
+
+    @Override
+    public void initStorage(Map<String, String> configuration) throws CheckpointStorageException {
+        Configuration hadoopConf = new Configuration();
+        if (configuration.containsKey(HdfsConstants.HDFS_DEF_FS_NAME)) {
+            hadoopConf.set(HdfsConstants.HDFS_DEF_FS_NAME, configuration.get(HdfsConstants.HDFS_DEF_FS_NAME));
+        }
+        if (StringUtils.isNotBlank(configuration.get(STORAGE_NAME_SPACE))) {
+            setStorageNameSpace(configuration.get(STORAGE_NAME_SPACE));
+        }
+        // todo support other config configurations
+        if (configuration.containsKey(HdfsConstants.KERBEROS_PRINCIPAL) && configuration.containsKey(HdfsConstants.KERBEROS_KEYTAB_FILE_PATH)) {
+            String kerberosPrincipal = configuration.get(HdfsConstants.KERBEROS_PRINCIPAL);
+            String kerberosKeytabFilePath = configuration.get(HdfsConstants.KERBEROS_KEYTAB_FILE_PATH);
+            if (StringUtils.isNotBlank(kerberosPrincipal) && StringUtils.isNotBlank(kerberosKeytabFilePath)) {
+                hadoopConf.set(HADOOP_SECURITY_AUTHENTICATION_KEY, KERBEROS_KEY);
+                authenticateKerberos(kerberosPrincipal, kerberosKeytabFilePath, hadoopConf);
+            }
+        }
+        JobConf jobConf = new JobConf(hadoopConf);
+        try {
+            fs = FileSystem.get(jobConf);
+        } catch (IOException e) {
+            throw new CheckpointStorageException("Failed to get file system", e);
+        }
+
+    }
+
+    @Override
+    public String storeCheckPoint(PipelineState state) throws CheckpointStorageException {
+        byte[] datas;
+        try {
+            datas = serializeCheckPointData(state);
+        } catch (IOException e) {
+            throw new CheckpointStorageException("Failed to serialize checkpoint data,state is :" + state, e);
+        }
+        Path filePath = new Path(getStorageParentDirectory() + state.getJobId() + "/" + getCheckPointName(state));
+
+        Path tmpFilePath = new Path(getStorageParentDirectory() + state.getJobId() + "/" + getCheckPointName(state) + STORAGE_TMP_SUFFIX);
+        try (FSDataOutputStream out = fs.create(tmpFilePath)) {
+            out.write(datas);
+            out.hsync();
+            out.hflush();

Review Comment:
   thx~ you are right



-- 
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: commits-unsubscribe@seatunnel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-seatunnel] CalvinKirs commented on a diff in pull request #2485: [New-Engine]Support Hdfs Storage and async storage

Posted by GitBox <gi...@apache.org>.
CalvinKirs commented on code in PR #2485:
URL: https://github.com/apache/incubator-seatunnel/pull/2485#discussion_r950940170


##########
seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-hdfs/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/hdfs/HdfsStorage.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.seatunnel.engine.checkpoint.storage.hdfs;
+
+import static org.apache.seatunnel.engine.checkpoint.storage.constants.StorageConstants.STORAGE_NAME_SPACE;
+
+import org.apache.seatunnel.engine.checkpoint.storage.PipelineState;
+import org.apache.seatunnel.engine.checkpoint.storage.api.AbstractCheckpointStorage;
+import org.apache.seatunnel.engine.checkpoint.storage.exception.CheckpointStorageException;
+
+import lombok.extern.slf4j.Slf4j;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+@Slf4j
+public class HdfsStorage extends AbstractCheckpointStorage {
+
+    public FileSystem fs;
+    private static final String HADOOP_SECURITY_AUTHENTICATION_KEY = "hadoop.security.authentication";
+
+    private static final String KERBEROS_KEY = "kerberos";
+
+    public HdfsStorage(Map<String, String> configuration) throws CheckpointStorageException {
+        this.initStorage(configuration);
+    }
+
+    @Override
+    public void initStorage(Map<String, String> configuration) throws CheckpointStorageException {
+        Configuration hadoopConf = new Configuration();
+        if (configuration.containsKey(HdfsConstants.HDFS_DEF_FS_NAME)) {
+            hadoopConf.set(HdfsConstants.HDFS_DEF_FS_NAME, configuration.get(HdfsConstants.HDFS_DEF_FS_NAME));
+        }
+        if (StringUtils.isNotBlank(configuration.get(STORAGE_NAME_SPACE))) {
+            setStorageNameSpace(configuration.get(STORAGE_NAME_SPACE));
+        }
+        // todo support other config configurations
+        if (configuration.containsKey(HdfsConstants.KERBEROS_PRINCIPAL) && configuration.containsKey(HdfsConstants.KERBEROS_KEYTAB_FILE_PATH)) {
+            String kerberosPrincipal = configuration.get(HdfsConstants.KERBEROS_PRINCIPAL);
+            String kerberosKeytabFilePath = configuration.get(HdfsConstants.KERBEROS_KEYTAB_FILE_PATH);
+            if (StringUtils.isNotBlank(kerberosPrincipal) && StringUtils.isNotBlank(kerberosKeytabFilePath)) {
+                hadoopConf.set(HADOOP_SECURITY_AUTHENTICATION_KEY, KERBEROS_KEY);
+                authenticateKerberos(kerberosPrincipal, kerberosKeytabFilePath, hadoopConf);
+            }
+        }
+        JobConf jobConf = new JobConf(hadoopConf);
+        try {
+            fs = FileSystem.get(jobConf);
+        } catch (IOException e) {
+            throw new CheckpointStorageException("Failed to get file system", e);
+        }
+
+    }
+
+    @Override
+    public String storeCheckPoint(PipelineState state) throws CheckpointStorageException {
+        byte[] datas;
+        try {
+            datas = serializeCheckPointData(state);
+        } catch (IOException e) {
+            throw new CheckpointStorageException("Failed to serialize checkpoint data", e);
+        }
+        //Consider file paths for different operating systems
+        String fileName = getStorageParentDirectory() + state.getJobId() + "/" + getCheckPointName(state);
+        try (FSDataOutputStream out = fs.create(new Path(fileName))) {
+            out.write(datas);
+        } catch (IOException e) {
+            throw new CheckpointStorageException("Failed to write checkpoint data", e);
+        }
+        return fileName;
+    }
+
+    @Override
+    public List<PipelineState> getAllCheckpoints(String jobId) throws CheckpointStorageException {
+        String path = getStorageParentDirectory() + jobId;
+        List<String> fileNames = getFileNames(path);
+        if (fileNames.isEmpty()) {
+            throw new CheckpointStorageException("No checkpoint found for job, job id is: " + jobId);
+        }
+        List<PipelineState> states = new ArrayList<>();
+        fileNames.forEach(file -> {
+            try {
+                states.add(readPipelineState(file, jobId));
+            } catch (CheckpointStorageException e) {
+                log.error("Failed to read checkpoint data from file: " + file, e);
+            }
+        });
+        if (states.isEmpty()) {
+            throw new CheckpointStorageException("No checkpoint found for job, job id is: " + jobId);
+        }
+        return states;
+    }
+
+    @Override
+    public List<PipelineState> getLatestCheckpoint(String jobId) throws CheckpointStorageException {
+        String path = getStorageParentDirectory() + jobId;
+        List<String> fileNames = getFileNames(path);
+        if (fileNames.isEmpty()) {
+            throw new CheckpointStorageException("No checkpoint found for job, job id is: " + jobId);
+        }
+        Set<String> latestPipelineNames = getLatestPipelineNames(fileNames);
+        List<PipelineState> latestPipelineStates = new ArrayList<>();
+        latestPipelineNames.forEach(fileName -> {
+            try {
+                latestPipelineStates.add(readPipelineState(fileName, jobId));
+            } catch (CheckpointStorageException e) {
+                log.error("Failed to read pipeline state for file: {}", fileName, e);
+            }
+        });
+
+        if (latestPipelineStates.isEmpty()) {
+            throw new CheckpointStorageException("No checkpoint found for job, job id is: " + jobId);
+        }
+        return latestPipelineStates;
+    }
+
+    @Override
+    public PipelineState getLatestCheckpointByJobIdAndPipelineId(String jobId, String pipelineId) throws CheckpointStorageException {
+        String path = getStorageParentDirectory() + jobId;
+        List<String> fileNames = getFileNames(path);
+        if (fileNames.isEmpty()) {
+            throw new CheckpointStorageException("No checkpoint found for job, job id is: " + jobId);
+        }
+
+        String latestFileName = getLatestCheckpointFileNameByJobIdAndPipelineId(fileNames, pipelineId);
+        if (latestFileName == null) {
+            throw new CheckpointStorageException("No checkpoint found for job, job id is: " + jobId + ", pipeline id is: " + pipelineId);
+        }
+        return readPipelineState(latestFileName, jobId);
+    }
+
+    @Override
+    public List<PipelineState> getCheckpointsByJobIdAndPipelineId(String jobId, String pipelineId) throws CheckpointStorageException {
+        String path = getStorageParentDirectory() + jobId;
+        List<String> fileNames = getFileNames(path);
+        if (fileNames.isEmpty()) {
+            throw new CheckpointStorageException("No checkpoint found for job, job id is: " + jobId);
+        }
+
+        List<PipelineState> pipelineStates = new ArrayList<>();
+        fileNames.forEach(file -> {
+            String filePipelineId = file.split(FILE_NAME_SPLIT)[FILE_NAME_PIPELINE_ID_INDEX];
+            if (pipelineId.equals(filePipelineId)) {
+                try {
+                    pipelineStates.add(readPipelineState(file, jobId));
+                } catch (Exception e) {
+                    log.error("Failed to read checkpoint data from file " + file, e);
+                }
+            }
+        });
+        return pipelineStates;
+    }
+
+    @Override
+    public void deleteCheckpoint(String jobId) {
+        String jobPath = getStorageParentDirectory() + jobId;
+        try {
+            fs.delete(new Path(jobPath), true);
+        } catch (IOException e) {
+            log.error("Failed to delete checkpoint for job {}", jobId, e);

Review Comment:
   No,For me this is a step that needs to be logged but doesn't need to throw an exception.



-- 
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: commits-unsubscribe@seatunnel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-seatunnel] CalvinKirs commented on a diff in pull request #2485: [New-Engine]Support Hdfs Storage and async storage

Posted by GitBox <gi...@apache.org>.
CalvinKirs commented on code in PR #2485:
URL: https://github.com/apache/incubator-seatunnel/pull/2485#discussion_r950948213


##########
seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-api/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/api/AbstractCheckpointStorage.java:
##########
@@ -79,4 +96,73 @@ 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 filePipelineId = fileName.split(FILE_NAME_SPLIT)[FILE_NAME_PIPELINE_ID_INDEX];
+            int fileVersion = Integer.parseInt(fileName.split(FILE_NAME_SPLIT)[FILE_SORT_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 -> {
+            int fileVersion = Integer.parseInt(fileName.split(FILE_NAME_SPLIT)[FILE_SORT_ID_INDEX]);
+            String filePipelineId = fileName.split(FILE_NAME_SPLIT)[FILE_NAME_PIPELINE_ID_INDEX];

Review Comment:
   good catch.



-- 
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: commits-unsubscribe@seatunnel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-seatunnel] Hisoka-X commented on a diff in pull request #2485: [New-Engine]Support Hdfs Storage and async storage

Posted by GitBox <gi...@apache.org>.
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: commits-unsubscribe@seatunnel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-seatunnel] hailin0 commented on a diff in pull request #2485: [New-Engine]Support Hdfs Storage and async storage

Posted by GitBox <gi...@apache.org>.
hailin0 commented on code in PR #2485:
URL: https://github.com/apache/incubator-seatunnel/pull/2485#discussion_r950843413


##########
seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-api/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/api/AbstractCheckpointStorage.java:
##########
@@ -79,4 +96,73 @@ 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 filePipelineId = fileName.split(FILE_NAME_SPLIT)[FILE_NAME_PIPELINE_ID_INDEX];
+            int fileVersion = Integer.parseInt(fileName.split(FILE_NAME_SPLIT)[FILE_SORT_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 -> {
+            int fileVersion = Integer.parseInt(fileName.split(FILE_NAME_SPLIT)[FILE_SORT_ID_INDEX]);
+            String filePipelineId = fileName.split(FILE_NAME_SPLIT)[FILE_NAME_PIPELINE_ID_INDEX];

Review Comment:
   `fileName` only need to be split once
   
   example
   ```java
   String[] fileNameSegments = fileName.split(FILE_NAME_SPLIT);
   ```



##########
seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-api/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/common/StorageThreadFactory.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.seatunnel.engine.checkpoint.storage.common;
+
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class StorageThreadFactory implements ThreadFactory {

Review Comment:
   Use guava `ThreadFactoryBuilder` create ThreadFactory?



##########
seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-api/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/api/CheckpointStorage.java:
##########
@@ -35,6 +35,14 @@ public interface CheckpointStorage {
      */
     String storeCheckPoint(PipelineState state) throws CheckpointStorageException;
 
+    /**
+     * async save checkpoint to storage
+     *
+     * @param state PipelineState
+     * @throws CheckpointStorageException if save checkpoint failed
+     */
+    void asyncStoreCheckPoint(PipelineState state) throws CheckpointStorageException;

Review Comment:
   Need to return `Future`?



##########
seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-hdfs/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/hdfs/HdfsStorage.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.seatunnel.engine.checkpoint.storage.hdfs;
+
+import static org.apache.seatunnel.engine.checkpoint.storage.constants.StorageConstants.STORAGE_NAME_SPACE;
+
+import org.apache.seatunnel.engine.checkpoint.storage.PipelineState;
+import org.apache.seatunnel.engine.checkpoint.storage.api.AbstractCheckpointStorage;
+import org.apache.seatunnel.engine.checkpoint.storage.exception.CheckpointStorageException;
+
+import lombok.extern.slf4j.Slf4j;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+@Slf4j
+public class HdfsStorage extends AbstractCheckpointStorage {
+
+    public FileSystem fs;
+    private static final String HADOOP_SECURITY_AUTHENTICATION_KEY = "hadoop.security.authentication";
+
+    private static final String KERBEROS_KEY = "kerberos";
+
+    public HdfsStorage(Map<String, String> configuration) throws CheckpointStorageException {
+        this.initStorage(configuration);
+    }
+
+    @Override
+    public void initStorage(Map<String, String> configuration) throws CheckpointStorageException {
+        Configuration hadoopConf = new Configuration();
+        if (configuration.containsKey(HdfsConstants.HDFS_DEF_FS_NAME)) {
+            hadoopConf.set(HdfsConstants.HDFS_DEF_FS_NAME, configuration.get(HdfsConstants.HDFS_DEF_FS_NAME));
+        }
+        if (StringUtils.isNotBlank(configuration.get(STORAGE_NAME_SPACE))) {
+            setStorageNameSpace(configuration.get(STORAGE_NAME_SPACE));
+        }
+        // todo support other config configurations
+        if (configuration.containsKey(HdfsConstants.KERBEROS_PRINCIPAL) && configuration.containsKey(HdfsConstants.KERBEROS_KEYTAB_FILE_PATH)) {
+            String kerberosPrincipal = configuration.get(HdfsConstants.KERBEROS_PRINCIPAL);
+            String kerberosKeytabFilePath = configuration.get(HdfsConstants.KERBEROS_KEYTAB_FILE_PATH);
+            if (StringUtils.isNotBlank(kerberosPrincipal) && StringUtils.isNotBlank(kerberosKeytabFilePath)) {
+                hadoopConf.set(HADOOP_SECURITY_AUTHENTICATION_KEY, KERBEROS_KEY);
+                authenticateKerberos(kerberosPrincipal, kerberosKeytabFilePath, hadoopConf);
+            }
+        }
+        JobConf jobConf = new JobConf(hadoopConf);
+        try {
+            fs = FileSystem.get(jobConf);
+        } catch (IOException e) {
+            throw new CheckpointStorageException("Failed to get file system", e);
+        }
+
+    }
+
+    @Override
+    public String storeCheckPoint(PipelineState state) throws CheckpointStorageException {
+        byte[] datas;
+        try {
+            datas = serializeCheckPointData(state);
+        } catch (IOException e) {
+            throw new CheckpointStorageException("Failed to serialize checkpoint data", e);
+        }
+        //Consider file paths for different operating systems
+        String fileName = getStorageParentDirectory() + state.getJobId() + "/" + getCheckPointName(state);
+        try (FSDataOutputStream out = fs.create(new Path(fileName))) {
+            out.write(datas);
+        } catch (IOException e) {
+            throw new CheckpointStorageException("Failed to write checkpoint data", e);
+        }
+        return fileName;
+    }
+
+    @Override
+    public List<PipelineState> getAllCheckpoints(String jobId) throws CheckpointStorageException {
+        String path = getStorageParentDirectory() + jobId;
+        List<String> fileNames = getFileNames(path);
+        if (fileNames.isEmpty()) {
+            throw new CheckpointStorageException("No checkpoint found for job, job id is: " + jobId);
+        }
+        List<PipelineState> states = new ArrayList<>();
+        fileNames.forEach(file -> {
+            try {
+                states.add(readPipelineState(file, jobId));
+            } catch (CheckpointStorageException e) {
+                log.error("Failed to read checkpoint data from file: " + file, e);
+            }
+        });
+        if (states.isEmpty()) {
+            throw new CheckpointStorageException("No checkpoint found for job, job id is: " + jobId);
+        }
+        return states;
+    }
+
+    @Override
+    public List<PipelineState> getLatestCheckpoint(String jobId) throws CheckpointStorageException {
+        String path = getStorageParentDirectory() + jobId;
+        List<String> fileNames = getFileNames(path);
+        if (fileNames.isEmpty()) {
+            throw new CheckpointStorageException("No checkpoint found for job, job id is: " + jobId);
+        }
+        Set<String> latestPipelineNames = getLatestPipelineNames(fileNames);
+        List<PipelineState> latestPipelineStates = new ArrayList<>();
+        latestPipelineNames.forEach(fileName -> {
+            try {
+                latestPipelineStates.add(readPipelineState(fileName, jobId));
+            } catch (CheckpointStorageException e) {
+                log.error("Failed to read pipeline state for file: {}", fileName, e);
+            }
+        });
+
+        if (latestPipelineStates.isEmpty()) {
+            throw new CheckpointStorageException("No checkpoint found for job, job id is: " + jobId);
+        }
+        return latestPipelineStates;
+    }
+
+    @Override
+    public PipelineState getLatestCheckpointByJobIdAndPipelineId(String jobId, String pipelineId) throws CheckpointStorageException {
+        String path = getStorageParentDirectory() + jobId;
+        List<String> fileNames = getFileNames(path);
+        if (fileNames.isEmpty()) {
+            throw new CheckpointStorageException("No checkpoint found for job, job id is: " + jobId);
+        }
+
+        String latestFileName = getLatestCheckpointFileNameByJobIdAndPipelineId(fileNames, pipelineId);
+        if (latestFileName == null) {
+            throw new CheckpointStorageException("No checkpoint found for job, job id is: " + jobId + ", pipeline id is: " + pipelineId);
+        }
+        return readPipelineState(latestFileName, jobId);
+    }
+
+    @Override
+    public List<PipelineState> getCheckpointsByJobIdAndPipelineId(String jobId, String pipelineId) throws CheckpointStorageException {
+        String path = getStorageParentDirectory() + jobId;
+        List<String> fileNames = getFileNames(path);
+        if (fileNames.isEmpty()) {
+            throw new CheckpointStorageException("No checkpoint found for job, job id is: " + jobId);
+        }
+
+        List<PipelineState> pipelineStates = new ArrayList<>();
+        fileNames.forEach(file -> {
+            String filePipelineId = file.split(FILE_NAME_SPLIT)[FILE_NAME_PIPELINE_ID_INDEX];
+            if (pipelineId.equals(filePipelineId)) {
+                try {
+                    pipelineStates.add(readPipelineState(file, jobId));
+                } catch (Exception e) {
+                    log.error("Failed to read checkpoint data from file " + file, e);

Review Comment:
   Should throws exception?



##########
seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-local-file/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/localfile/LocalFileStorage.java:
##########
@@ -138,32 +156,32 @@ public List<PipelineState> getLatestCheckpoint(String jobId) throws CheckpointSt
     @Override
     public PipelineState getLatestCheckpointByJobIdAndPipelineId(String jobId, String pipelineId) throws CheckpointStorageException {
 
-        Collection<File> fileList = FileUtils.listFiles(new File(getStorageParentDirectory() + jobId), FILE_EXTENSIONS, false);
+        String parentPath = getStorageParentDirectory() + jobId;
+        Collection<File> fileList = FileUtils.listFiles(new File(parentPath), FILE_EXTENSIONS, false);
         if (fileList.isEmpty()) {
             throw new CheckpointStorageException("No checkpoint found for job " + jobId);
         }
+        List<String> fileNames = fileList.stream().map(File::getName).collect(Collectors.toList());
+
+        String latestFileName = getLatestCheckpointFileNameByJobIdAndPipelineId(fileNames, pipelineId);
 
-        AtomicReference<File> latestFile = new AtomicReference<>();
-        AtomicInteger latestVersion = new AtomicInteger();
+        AtomicReference<PipelineState> latestFile = new AtomicReference<>(null);
         fileList.forEach(file -> {
-            int fileVersion = Integer.parseInt(file.getName().split(FILE_NAME_SPLIT)[FILE_SORT_ID_INDEX]);
-            String filePipelineId = file.getName().split(FILE_NAME_SPLIT)[FILE_NAME_PIPELINE_ID_INDEX];
-            if (pipelineId.equals(filePipelineId) && fileVersion > latestVersion.get()) {
-                latestVersion.set(fileVersion);
-                latestFile.set(file);
+            String fileName = file.getName();
+            if (fileName.equals(latestFileName)) {
+                try {
+                    byte[] data = FileUtils.readFileToByteArray(file);
+                    latestFile.set(deserializeCheckPointData(data));
+                } catch (IOException e) {
+                    e.printStackTrace();

Review Comment:
   Should throws exception?



##########
seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-hdfs/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/hdfs/HdfsStorage.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.seatunnel.engine.checkpoint.storage.hdfs;
+
+import static org.apache.seatunnel.engine.checkpoint.storage.constants.StorageConstants.STORAGE_NAME_SPACE;
+
+import org.apache.seatunnel.engine.checkpoint.storage.PipelineState;
+import org.apache.seatunnel.engine.checkpoint.storage.api.AbstractCheckpointStorage;
+import org.apache.seatunnel.engine.checkpoint.storage.exception.CheckpointStorageException;
+
+import lombok.extern.slf4j.Slf4j;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+@Slf4j
+public class HdfsStorage extends AbstractCheckpointStorage {
+
+    public FileSystem fs;
+    private static final String HADOOP_SECURITY_AUTHENTICATION_KEY = "hadoop.security.authentication";
+
+    private static final String KERBEROS_KEY = "kerberos";
+
+    public HdfsStorage(Map<String, String> configuration) throws CheckpointStorageException {
+        this.initStorage(configuration);
+    }
+
+    @Override
+    public void initStorage(Map<String, String> configuration) throws CheckpointStorageException {
+        Configuration hadoopConf = new Configuration();
+        if (configuration.containsKey(HdfsConstants.HDFS_DEF_FS_NAME)) {
+            hadoopConf.set(HdfsConstants.HDFS_DEF_FS_NAME, configuration.get(HdfsConstants.HDFS_DEF_FS_NAME));
+        }
+        if (StringUtils.isNotBlank(configuration.get(STORAGE_NAME_SPACE))) {
+            setStorageNameSpace(configuration.get(STORAGE_NAME_SPACE));
+        }
+        // todo support other config configurations
+        if (configuration.containsKey(HdfsConstants.KERBEROS_PRINCIPAL) && configuration.containsKey(HdfsConstants.KERBEROS_KEYTAB_FILE_PATH)) {
+            String kerberosPrincipal = configuration.get(HdfsConstants.KERBEROS_PRINCIPAL);
+            String kerberosKeytabFilePath = configuration.get(HdfsConstants.KERBEROS_KEYTAB_FILE_PATH);
+            if (StringUtils.isNotBlank(kerberosPrincipal) && StringUtils.isNotBlank(kerberosKeytabFilePath)) {
+                hadoopConf.set(HADOOP_SECURITY_AUTHENTICATION_KEY, KERBEROS_KEY);
+                authenticateKerberos(kerberosPrincipal, kerberosKeytabFilePath, hadoopConf);
+            }
+        }
+        JobConf jobConf = new JobConf(hadoopConf);
+        try {
+            fs = FileSystem.get(jobConf);
+        } catch (IOException e) {
+            throw new CheckpointStorageException("Failed to get file system", e);
+        }
+
+    }
+
+    @Override
+    public String storeCheckPoint(PipelineState state) throws CheckpointStorageException {
+        byte[] datas;
+        try {
+            datas = serializeCheckPointData(state);
+        } catch (IOException e) {
+            throw new CheckpointStorageException("Failed to serialize checkpoint data", e);
+        }
+        //Consider file paths for different operating systems
+        String fileName = getStorageParentDirectory() + state.getJobId() + "/" + getCheckPointName(state);
+        try (FSDataOutputStream out = fs.create(new Path(fileName))) {
+            out.write(datas);
+        } catch (IOException e) {
+            throw new CheckpointStorageException("Failed to write checkpoint data", e);

Review Comment:
   Write to temporary file(xxx.tmp) first and then rename?



##########
seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-api/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/api/AbstractCheckpointStorage.java:
##########
@@ -79,4 +96,73 @@ 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 filePipelineId = fileName.split(FILE_NAME_SPLIT)[FILE_NAME_PIPELINE_ID_INDEX];
+            int fileVersion = Integer.parseInt(fileName.split(FILE_NAME_SPLIT)[FILE_SORT_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 -> {
+            int fileVersion = Integer.parseInt(fileName.split(FILE_NAME_SPLIT)[FILE_SORT_ID_INDEX]);
+            String filePipelineId = fileName.split(FILE_NAME_SPLIT)[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) {
+        if (null == this.executor || this.executor.isShutdown()) {
+            this.executor = Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors() * 2 + 1, new StorageThreadFactory());

Review Comment:
   Does `coreSize` need to define a configuration variable?



##########
seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-local-file/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/localfile/LocalFileStorage.java:
##########
@@ -107,27 +132,20 @@ public List<PipelineState> getLatestCheckpoint(String jobId) throws CheckpointSt
         if (fileList.isEmpty()) {
             throw new CheckpointStorageException("No checkpoint found for job " + jobId);
         }
-        Map<String, File> latestPipelineMap = new HashMap<>();
+        List<String> fileNames = fileList.stream().map(File::getName).collect(Collectors.toList());
+        Set<String> latestPipelines = getLatestPipelineNames(fileNames);
+        List<PipelineState> latestPipelineFiles = new ArrayList<>(latestPipelines.size());
         fileList.forEach(file -> {
-            String filePipelineId = file.getName().split(FILE_NAME_SPLIT)[FILE_NAME_PIPELINE_ID_INDEX];
-            int fileVersion = Integer.parseInt(file.getName().split(FILE_NAME_SPLIT)[FILE_SORT_ID_INDEX]);
-            if (latestPipelineMap.containsKey(filePipelineId)) {
-                int oldVersion = Integer.parseInt(latestPipelineMap.get(filePipelineId).getName().split(FILE_NAME_SPLIT)[FILE_SORT_ID_INDEX]);
-                if (fileVersion > oldVersion) {
-                    latestPipelineMap.put(filePipelineId, file);
+            String fileName = file.getName();
+            if (latestPipelines.contains(fileName)) {
+                try {
+                    byte[] data = FileUtils.readFileToByteArray(file);
+                    latestPipelineFiles.add(deserializeCheckPointData(data));
+                } catch (IOException e) {
+                    log.error("Failed to read checkpoint data from file " + file.getAbsolutePath(), e);

Review Comment:
   Should throws exception?



##########
seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-hdfs/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/hdfs/HdfsStorage.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.seatunnel.engine.checkpoint.storage.hdfs;
+
+import static org.apache.seatunnel.engine.checkpoint.storage.constants.StorageConstants.STORAGE_NAME_SPACE;
+
+import org.apache.seatunnel.engine.checkpoint.storage.PipelineState;
+import org.apache.seatunnel.engine.checkpoint.storage.api.AbstractCheckpointStorage;
+import org.apache.seatunnel.engine.checkpoint.storage.exception.CheckpointStorageException;
+
+import lombok.extern.slf4j.Slf4j;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+@Slf4j
+public class HdfsStorage extends AbstractCheckpointStorage {
+
+    public FileSystem fs;
+    private static final String HADOOP_SECURITY_AUTHENTICATION_KEY = "hadoop.security.authentication";
+
+    private static final String KERBEROS_KEY = "kerberos";
+
+    public HdfsStorage(Map<String, String> configuration) throws CheckpointStorageException {
+        this.initStorage(configuration);
+    }
+
+    @Override
+    public void initStorage(Map<String, String> configuration) throws CheckpointStorageException {
+        Configuration hadoopConf = new Configuration();
+        if (configuration.containsKey(HdfsConstants.HDFS_DEF_FS_NAME)) {
+            hadoopConf.set(HdfsConstants.HDFS_DEF_FS_NAME, configuration.get(HdfsConstants.HDFS_DEF_FS_NAME));
+        }
+        if (StringUtils.isNotBlank(configuration.get(STORAGE_NAME_SPACE))) {
+            setStorageNameSpace(configuration.get(STORAGE_NAME_SPACE));
+        }
+        // todo support other config configurations
+        if (configuration.containsKey(HdfsConstants.KERBEROS_PRINCIPAL) && configuration.containsKey(HdfsConstants.KERBEROS_KEYTAB_FILE_PATH)) {
+            String kerberosPrincipal = configuration.get(HdfsConstants.KERBEROS_PRINCIPAL);
+            String kerberosKeytabFilePath = configuration.get(HdfsConstants.KERBEROS_KEYTAB_FILE_PATH);
+            if (StringUtils.isNotBlank(kerberosPrincipal) && StringUtils.isNotBlank(kerberosKeytabFilePath)) {
+                hadoopConf.set(HADOOP_SECURITY_AUTHENTICATION_KEY, KERBEROS_KEY);
+                authenticateKerberos(kerberosPrincipal, kerberosKeytabFilePath, hadoopConf);
+            }
+        }
+        JobConf jobConf = new JobConf(hadoopConf);
+        try {
+            fs = FileSystem.get(jobConf);
+        } catch (IOException e) {
+            throw new CheckpointStorageException("Failed to get file system", e);
+        }
+
+    }
+
+    @Override
+    public String storeCheckPoint(PipelineState state) throws CheckpointStorageException {
+        byte[] datas;
+        try {
+            datas = serializeCheckPointData(state);
+        } catch (IOException e) {
+            throw new CheckpointStorageException("Failed to serialize checkpoint data", e);
+        }
+        //Consider file paths for different operating systems
+        String fileName = getStorageParentDirectory() + state.getJobId() + "/" + getCheckPointName(state);
+        try (FSDataOutputStream out = fs.create(new Path(fileName))) {
+            out.write(datas);
+        } catch (IOException e) {
+            throw new CheckpointStorageException("Failed to write checkpoint data", e);

Review Comment:
   Log fileName or add fileName into exception message?



##########
seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-hdfs/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/hdfs/HdfsStorage.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.seatunnel.engine.checkpoint.storage.hdfs;
+
+import static org.apache.seatunnel.engine.checkpoint.storage.constants.StorageConstants.STORAGE_NAME_SPACE;
+
+import org.apache.seatunnel.engine.checkpoint.storage.PipelineState;
+import org.apache.seatunnel.engine.checkpoint.storage.api.AbstractCheckpointStorage;
+import org.apache.seatunnel.engine.checkpoint.storage.exception.CheckpointStorageException;
+
+import lombok.extern.slf4j.Slf4j;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+@Slf4j
+public class HdfsStorage extends AbstractCheckpointStorage {
+
+    public FileSystem fs;
+    private static final String HADOOP_SECURITY_AUTHENTICATION_KEY = "hadoop.security.authentication";
+
+    private static final String KERBEROS_KEY = "kerberos";
+
+    public HdfsStorage(Map<String, String> configuration) throws CheckpointStorageException {
+        this.initStorage(configuration);
+    }
+
+    @Override
+    public void initStorage(Map<String, String> configuration) throws CheckpointStorageException {
+        Configuration hadoopConf = new Configuration();
+        if (configuration.containsKey(HdfsConstants.HDFS_DEF_FS_NAME)) {
+            hadoopConf.set(HdfsConstants.HDFS_DEF_FS_NAME, configuration.get(HdfsConstants.HDFS_DEF_FS_NAME));
+        }
+        if (StringUtils.isNotBlank(configuration.get(STORAGE_NAME_SPACE))) {
+            setStorageNameSpace(configuration.get(STORAGE_NAME_SPACE));
+        }
+        // todo support other config configurations
+        if (configuration.containsKey(HdfsConstants.KERBEROS_PRINCIPAL) && configuration.containsKey(HdfsConstants.KERBEROS_KEYTAB_FILE_PATH)) {
+            String kerberosPrincipal = configuration.get(HdfsConstants.KERBEROS_PRINCIPAL);
+            String kerberosKeytabFilePath = configuration.get(HdfsConstants.KERBEROS_KEYTAB_FILE_PATH);
+            if (StringUtils.isNotBlank(kerberosPrincipal) && StringUtils.isNotBlank(kerberosKeytabFilePath)) {
+                hadoopConf.set(HADOOP_SECURITY_AUTHENTICATION_KEY, KERBEROS_KEY);
+                authenticateKerberos(kerberosPrincipal, kerberosKeytabFilePath, hadoopConf);
+            }
+        }
+        JobConf jobConf = new JobConf(hadoopConf);
+        try {
+            fs = FileSystem.get(jobConf);
+        } catch (IOException e) {
+            throw new CheckpointStorageException("Failed to get file system", e);
+        }
+
+    }
+
+    @Override
+    public String storeCheckPoint(PipelineState state) throws CheckpointStorageException {
+        byte[] datas;
+        try {
+            datas = serializeCheckPointData(state);
+        } catch (IOException e) {
+            throw new CheckpointStorageException("Failed to serialize checkpoint data", e);
+        }
+        //Consider file paths for different operating systems
+        String fileName = getStorageParentDirectory() + state.getJobId() + "/" + getCheckPointName(state);
+        try (FSDataOutputStream out = fs.create(new Path(fileName))) {
+            out.write(datas);
+        } catch (IOException e) {
+            throw new CheckpointStorageException("Failed to write checkpoint data", e);
+        }
+        return fileName;
+    }
+
+    @Override
+    public List<PipelineState> getAllCheckpoints(String jobId) throws CheckpointStorageException {
+        String path = getStorageParentDirectory() + jobId;
+        List<String> fileNames = getFileNames(path);
+        if (fileNames.isEmpty()) {
+            throw new CheckpointStorageException("No checkpoint found for job, job id is: " + jobId);
+        }
+        List<PipelineState> states = new ArrayList<>();
+        fileNames.forEach(file -> {
+            try {
+                states.add(readPipelineState(file, jobId));
+            } catch (CheckpointStorageException e) {
+                log.error("Failed to read checkpoint data from file: " + file, e);
+            }
+        });
+        if (states.isEmpty()) {
+            throw new CheckpointStorageException("No checkpoint found for job, job id is: " + jobId);
+        }
+        return states;
+    }
+
+    @Override
+    public List<PipelineState> getLatestCheckpoint(String jobId) throws CheckpointStorageException {
+        String path = getStorageParentDirectory() + jobId;
+        List<String> fileNames = getFileNames(path);
+        if (fileNames.isEmpty()) {
+            throw new CheckpointStorageException("No checkpoint found for job, job id is: " + jobId);
+        }
+        Set<String> latestPipelineNames = getLatestPipelineNames(fileNames);
+        List<PipelineState> latestPipelineStates = new ArrayList<>();
+        latestPipelineNames.forEach(fileName -> {
+            try {
+                latestPipelineStates.add(readPipelineState(fileName, jobId));
+            } catch (CheckpointStorageException e) {
+                log.error("Failed to read pipeline state for file: {}", fileName, e);
+            }
+        });
+
+        if (latestPipelineStates.isEmpty()) {
+            throw new CheckpointStorageException("No checkpoint found for job, job id is: " + jobId);
+        }
+        return latestPipelineStates;
+    }
+
+    @Override
+    public PipelineState getLatestCheckpointByJobIdAndPipelineId(String jobId, String pipelineId) throws CheckpointStorageException {
+        String path = getStorageParentDirectory() + jobId;
+        List<String> fileNames = getFileNames(path);
+        if (fileNames.isEmpty()) {
+            throw new CheckpointStorageException("No checkpoint found for job, job id is: " + jobId);
+        }
+
+        String latestFileName = getLatestCheckpointFileNameByJobIdAndPipelineId(fileNames, pipelineId);
+        if (latestFileName == null) {
+            throw new CheckpointStorageException("No checkpoint found for job, job id is: " + jobId + ", pipeline id is: " + pipelineId);
+        }
+        return readPipelineState(latestFileName, jobId);
+    }
+
+    @Override
+    public List<PipelineState> getCheckpointsByJobIdAndPipelineId(String jobId, String pipelineId) throws CheckpointStorageException {
+        String path = getStorageParentDirectory() + jobId;
+        List<String> fileNames = getFileNames(path);
+        if (fileNames.isEmpty()) {
+            throw new CheckpointStorageException("No checkpoint found for job, job id is: " + jobId);
+        }
+
+        List<PipelineState> pipelineStates = new ArrayList<>();
+        fileNames.forEach(file -> {
+            String filePipelineId = file.split(FILE_NAME_SPLIT)[FILE_NAME_PIPELINE_ID_INDEX];
+            if (pipelineId.equals(filePipelineId)) {
+                try {
+                    pipelineStates.add(readPipelineState(file, jobId));
+                } catch (Exception e) {
+                    log.error("Failed to read checkpoint data from file " + file, e);
+                }
+            }
+        });
+        return pipelineStates;
+    }
+
+    @Override
+    public void deleteCheckpoint(String jobId) {
+        String jobPath = getStorageParentDirectory() + jobId;
+        try {
+            fs.delete(new Path(jobPath), true);
+        } catch (IOException e) {
+            log.error("Failed to delete checkpoint for job {}", jobId, e);

Review Comment:
   Should throws exception?



##########
seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-hdfs/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/hdfs/HdfsStorage.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.seatunnel.engine.checkpoint.storage.hdfs;
+
+import static org.apache.seatunnel.engine.checkpoint.storage.constants.StorageConstants.STORAGE_NAME_SPACE;
+
+import org.apache.seatunnel.engine.checkpoint.storage.PipelineState;
+import org.apache.seatunnel.engine.checkpoint.storage.api.AbstractCheckpointStorage;
+import org.apache.seatunnel.engine.checkpoint.storage.exception.CheckpointStorageException;
+
+import lombok.extern.slf4j.Slf4j;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+@Slf4j
+public class HdfsStorage extends AbstractCheckpointStorage {
+
+    public FileSystem fs;
+    private static final String HADOOP_SECURITY_AUTHENTICATION_KEY = "hadoop.security.authentication";
+
+    private static final String KERBEROS_KEY = "kerberos";
+
+    public HdfsStorage(Map<String, String> configuration) throws CheckpointStorageException {
+        this.initStorage(configuration);
+    }
+
+    @Override
+    public void initStorage(Map<String, String> configuration) throws CheckpointStorageException {
+        Configuration hadoopConf = new Configuration();
+        if (configuration.containsKey(HdfsConstants.HDFS_DEF_FS_NAME)) {
+            hadoopConf.set(HdfsConstants.HDFS_DEF_FS_NAME, configuration.get(HdfsConstants.HDFS_DEF_FS_NAME));
+        }
+        if (StringUtils.isNotBlank(configuration.get(STORAGE_NAME_SPACE))) {
+            setStorageNameSpace(configuration.get(STORAGE_NAME_SPACE));
+        }
+        // todo support other config configurations
+        if (configuration.containsKey(HdfsConstants.KERBEROS_PRINCIPAL) && configuration.containsKey(HdfsConstants.KERBEROS_KEYTAB_FILE_PATH)) {
+            String kerberosPrincipal = configuration.get(HdfsConstants.KERBEROS_PRINCIPAL);
+            String kerberosKeytabFilePath = configuration.get(HdfsConstants.KERBEROS_KEYTAB_FILE_PATH);
+            if (StringUtils.isNotBlank(kerberosPrincipal) && StringUtils.isNotBlank(kerberosKeytabFilePath)) {
+                hadoopConf.set(HADOOP_SECURITY_AUTHENTICATION_KEY, KERBEROS_KEY);
+                authenticateKerberos(kerberosPrincipal, kerberosKeytabFilePath, hadoopConf);
+            }
+        }
+        JobConf jobConf = new JobConf(hadoopConf);
+        try {
+            fs = FileSystem.get(jobConf);
+        } catch (IOException e) {
+            throw new CheckpointStorageException("Failed to get file system", e);
+        }
+
+    }
+
+    @Override
+    public String storeCheckPoint(PipelineState state) throws CheckpointStorageException {
+        byte[] datas;
+        try {
+            datas = serializeCheckPointData(state);
+        } catch (IOException e) {
+            throw new CheckpointStorageException("Failed to serialize checkpoint data", e);
+        }
+        //Consider file paths for different operating systems
+        String fileName = getStorageParentDirectory() + state.getJobId() + "/" + getCheckPointName(state);
+        try (FSDataOutputStream out = fs.create(new Path(fileName))) {
+            out.write(datas);
+        } catch (IOException e) {
+            throw new CheckpointStorageException("Failed to write checkpoint data", e);
+        }
+        return fileName;
+    }
+
+    @Override
+    public List<PipelineState> getAllCheckpoints(String jobId) throws CheckpointStorageException {
+        String path = getStorageParentDirectory() + jobId;
+        List<String> fileNames = getFileNames(path);
+        if (fileNames.isEmpty()) {
+            throw new CheckpointStorageException("No checkpoint found for job, job id is: " + jobId);
+        }
+        List<PipelineState> states = new ArrayList<>();
+        fileNames.forEach(file -> {
+            try {
+                states.add(readPipelineState(file, jobId));
+            } catch (CheckpointStorageException e) {
+                log.error("Failed to read checkpoint data from file: " + file, e);
+            }
+        });
+        if (states.isEmpty()) {
+            throw new CheckpointStorageException("No checkpoint found for job, job id is: " + jobId);
+        }
+        return states;
+    }
+
+    @Override
+    public List<PipelineState> getLatestCheckpoint(String jobId) throws CheckpointStorageException {
+        String path = getStorageParentDirectory() + jobId;
+        List<String> fileNames = getFileNames(path);
+        if (fileNames.isEmpty()) {
+            throw new CheckpointStorageException("No checkpoint found for job, job id is: " + jobId);
+        }
+        Set<String> latestPipelineNames = getLatestPipelineNames(fileNames);
+        List<PipelineState> latestPipelineStates = new ArrayList<>();
+        latestPipelineNames.forEach(fileName -> {
+            try {
+                latestPipelineStates.add(readPipelineState(fileName, jobId));
+            } catch (CheckpointStorageException e) {
+                log.error("Failed to read pipeline state for file: {}", fileName, e);

Review Comment:
   Should throws exception?



##########
seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-hdfs/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/hdfs/HdfsStorage.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.seatunnel.engine.checkpoint.storage.hdfs;
+
+import static org.apache.seatunnel.engine.checkpoint.storage.constants.StorageConstants.STORAGE_NAME_SPACE;
+
+import org.apache.seatunnel.engine.checkpoint.storage.PipelineState;
+import org.apache.seatunnel.engine.checkpoint.storage.api.AbstractCheckpointStorage;
+import org.apache.seatunnel.engine.checkpoint.storage.exception.CheckpointStorageException;
+
+import lombok.extern.slf4j.Slf4j;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+@Slf4j
+public class HdfsStorage extends AbstractCheckpointStorage {
+
+    public FileSystem fs;
+    private static final String HADOOP_SECURITY_AUTHENTICATION_KEY = "hadoop.security.authentication";
+
+    private static final String KERBEROS_KEY = "kerberos";
+
+    public HdfsStorage(Map<String, String> configuration) throws CheckpointStorageException {
+        this.initStorage(configuration);
+    }
+
+    @Override
+    public void initStorage(Map<String, String> configuration) throws CheckpointStorageException {
+        Configuration hadoopConf = new Configuration();
+        if (configuration.containsKey(HdfsConstants.HDFS_DEF_FS_NAME)) {
+            hadoopConf.set(HdfsConstants.HDFS_DEF_FS_NAME, configuration.get(HdfsConstants.HDFS_DEF_FS_NAME));
+        }
+        if (StringUtils.isNotBlank(configuration.get(STORAGE_NAME_SPACE))) {
+            setStorageNameSpace(configuration.get(STORAGE_NAME_SPACE));
+        }
+        // todo support other config configurations
+        if (configuration.containsKey(HdfsConstants.KERBEROS_PRINCIPAL) && configuration.containsKey(HdfsConstants.KERBEROS_KEYTAB_FILE_PATH)) {
+            String kerberosPrincipal = configuration.get(HdfsConstants.KERBEROS_PRINCIPAL);
+            String kerberosKeytabFilePath = configuration.get(HdfsConstants.KERBEROS_KEYTAB_FILE_PATH);
+            if (StringUtils.isNotBlank(kerberosPrincipal) && StringUtils.isNotBlank(kerberosKeytabFilePath)) {
+                hadoopConf.set(HADOOP_SECURITY_AUTHENTICATION_KEY, KERBEROS_KEY);
+                authenticateKerberos(kerberosPrincipal, kerberosKeytabFilePath, hadoopConf);
+            }
+        }
+        JobConf jobConf = new JobConf(hadoopConf);
+        try {
+            fs = FileSystem.get(jobConf);
+        } catch (IOException e) {
+            throw new CheckpointStorageException("Failed to get file system", e);
+        }
+
+    }
+
+    @Override
+    public String storeCheckPoint(PipelineState state) throws CheckpointStorageException {
+        byte[] datas;
+        try {
+            datas = serializeCheckPointData(state);
+        } catch (IOException e) {
+            throw new CheckpointStorageException("Failed to serialize checkpoint data", e);
+        }
+        //Consider file paths for different operating systems
+        String fileName = getStorageParentDirectory() + state.getJobId() + "/" + getCheckPointName(state);
+        try (FSDataOutputStream out = fs.create(new Path(fileName))) {
+            out.write(datas);
+        } catch (IOException e) {
+            throw new CheckpointStorageException("Failed to write checkpoint data", e);
+        }
+        return fileName;
+    }
+
+    @Override
+    public List<PipelineState> getAllCheckpoints(String jobId) throws CheckpointStorageException {
+        String path = getStorageParentDirectory() + jobId;
+        List<String> fileNames = getFileNames(path);
+        if (fileNames.isEmpty()) {
+            throw new CheckpointStorageException("No checkpoint found for job, job id is: " + jobId);
+        }
+        List<PipelineState> states = new ArrayList<>();
+        fileNames.forEach(file -> {
+            try {
+                states.add(readPipelineState(file, jobId));
+            } catch (CheckpointStorageException e) {
+                log.error("Failed to read checkpoint data from file: " + file, e);

Review Comment:
   Should throws exception?



-- 
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: commits-unsubscribe@seatunnel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org