├── flink-cos-fs-hadoop ├── src │ └── main │ │ ├── resources │ │ └── META-INF │ │ │ └── services │ │ │ └── org.apache.flink.core.fs.FileSystemFactory │ │ └── java │ │ └── org │ │ └── apache │ │ └── flink │ │ └── fs │ │ └── coshadoop │ │ ├── Constants.java │ │ ├── HadoopCOSAccessHelper.java │ │ └── COSNFileSystemFactory.java └── pom.xml ├── LICENSE ├── flink-cos-fs-base ├── src │ └── main │ │ └── java │ │ └── org │ │ └── apache │ │ └── flink │ │ └── fs │ │ └── cos │ │ └── common │ │ ├── utils │ │ ├── RefCounted.java │ │ ├── RefCountedFSOutputStream.java │ │ ├── OffsetTrackOutputStream.java │ │ ├── BackPressuringExecutor.java │ │ ├── RefCountedTmpFileCreator.java │ │ ├── RefCountedFile.java │ │ └── RefCountedBufferingFileStream.java │ │ ├── writer │ │ ├── RecoverableMultipartUpload.java │ │ ├── COSAccessHelper.java │ │ ├── MultipartUploadInfo.java │ │ ├── COSRecoverable.java │ │ ├── COSRecoverableMultipartUploadFactory.java │ │ ├── COSCommitter.java │ │ ├── COSRecoverableSerializer.java │ │ ├── COSRecoverableWriter.java │ │ ├── COSRecoverableFsDataOutputStream.java │ │ └── RecoverableMultipartUploadImpl.java │ │ ├── fswriter │ │ ├── COSPosixRecoverable.java │ │ ├── COSPosixRecoverableSerializer.java │ │ ├── COSPosixRecoverableWriter.java │ │ └── COSPosixRecoverableFsDataOutputStream.java │ │ ├── FlinkCOSFileSystem.java │ │ └── AbstractCOSFileSystemFactory.java └── pom.xml ├── flink-fs-hadoop-shaded └── src │ └── main │ ├── resources │ └── META-INF │ │ └── licenses │ │ ├── LICENSE-stax2api │ │ └── LICENSE-re2j │ └── java │ └── org │ └── apache │ └── hadoop │ ├── util │ ├── NativeCodeLoader.java │ └── VersionInfo.java │ └── conf │ └── StorageSize.java ├── flink-cos-fs-examples ├── src │ └── main │ │ └── java │ │ └── org │ │ └── apache │ │ └── flink │ │ └── fs │ │ └── cos │ │ └── examples │ │ └── streaming │ │ ├── MockSource.java │ │ └── StreamingFileSinkTest.java └── pom.xml ├── .gitignore ├── tools └── maven │ ├── suppressions.xml │ ├── suppressions-core.xml │ ├── suppressions-optimizer.xml │ └── spotbugs-exclude.xml ├── pom.xml └── README.md /flink-cos-fs-hadoop/src/main/resources/META-INF/services/org.apache.flink.core.fs.FileSystemFactory: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | org.apache.flink.fs.coshadoop.COSNFileSystemFactory 17 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | MIT License 2 | 3 | Copyright (c) 2020 腾讯云 4 | 5 | Permission is hereby granted, free of charge, to any person obtaining a copy 6 | of this software and associated documentation files (the "Software"), to deal 7 | in the Software without restriction, including without limitation the rights 8 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 9 | copies of the Software, and to permit persons to whom the Software is 10 | furnished to do so, subject to the following conditions: 11 | 12 | The above copyright notice and this permission notice shall be included in all 13 | copies or substantial portions of the Software. 14 | 15 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 16 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 17 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 18 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 19 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 20 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE 21 | SOFTWARE. 22 | -------------------------------------------------------------------------------- /flink-cos-fs-base/src/main/java/org/apache/flink/fs/cos/common/utils/RefCounted.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.fs.cos.common.utils; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | 23 | /** The interface for ref count IOStream. */ 24 | @Internal 25 | public interface RefCounted { 26 | void retain(); 27 | 28 | boolean release(); 29 | } 30 | -------------------------------------------------------------------------------- /flink-cos-fs-hadoop/src/main/java/org/apache/flink/fs/coshadoop/Constants.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.fs.coshadoop; 20 | 21 | /** 22 | * 用来定义一些需要排除在 shade 以外的常量字符串. 23 | */ 24 | public final class Constants { 25 | public static final String SHADING_VALUE_PREFIX = (new StringBuilder()) 26 | .append("org.").append("apache.").append("hadoop.").append("fs").toString(); 27 | } 28 | -------------------------------------------------------------------------------- /flink-fs-hadoop-shaded/src/main/resources/META-INF/licenses/LICENSE-stax2api: -------------------------------------------------------------------------------- 1 | Copyright woodstox stax2api contributors. 2 | 3 | Redistribution and use in source and binary forms, with or without modification, 4 | are permitted provided that the following conditions are met: 5 | 6 | 1. Redistributions of source code must retain the above copyright notice, 7 | this list of conditions and the following disclaimer. 8 | 9 | 2. Redistributions in binary form must reproduce the above copyright notice, 10 | this list of conditions and the following disclaimer in the documentation 11 | and/or other materials provided with the distribution. 12 | 13 | THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND 14 | ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED 15 | WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. 16 | IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, 17 | INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, 18 | BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, 19 | OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, 20 | WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) 21 | ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 22 | POSSIBILITY OF SUCH DAMAGE. 23 | -------------------------------------------------------------------------------- /flink-cos-fs-base/src/main/java/org/apache/flink/fs/cos/common/utils/RefCountedFSOutputStream.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.fs.cos.common.utils; 20 | 21 | import org.apache.flink.core.fs.FSDataOutputStream; 22 | 23 | import java.io.File; 24 | import java.io.IOException; 25 | 26 | /** A {@link FSDataOutputStream} with the {@link RefCounted} functionality. */ 27 | public abstract class RefCountedFSOutputStream extends FSDataOutputStream implements RefCounted { 28 | public abstract File getInputFile(); 29 | 30 | public abstract boolean isClosed() throws IOException; 31 | } 32 | -------------------------------------------------------------------------------- /flink-cos-fs-examples/src/main/java/org/apache/flink/fs/cos/examples/streaming/MockSource.java: -------------------------------------------------------------------------------- 1 | package org.apache.flink.fs.cos.examples.streaming; 2 | 3 | import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; 4 | 5 | import org.slf4j.Logger; 6 | import org.slf4j.LoggerFactory; 7 | 8 | import java.util.Date; 9 | 10 | /** The MockSource for Flink streaming test. */ 11 | public class MockSource extends RichParallelSourceFunction { 12 | private static final Logger LOG = LoggerFactory.getLogger(MockSource.class); 13 | private volatile boolean runningFlag = false; 14 | 15 | private final long intervalMillis; 16 | private final int messageNumber; 17 | 18 | public MockSource(long intervalMillis, int count) { 19 | this.intervalMillis = intervalMillis; 20 | this.messageNumber = count; 21 | } 22 | 23 | @Override 24 | public void run(SourceContext sourceContext) throws Exception { 25 | this.runningFlag = true; 26 | int messageCount = 1; 27 | while (this.runningFlag && messageCount < this.messageNumber) { 28 | String mockMessage = 29 | String.format( 30 | "Message Number:%d, Time: %s.", 31 | messageCount, new Date(System.currentTimeMillis())); 32 | sourceContext.collect(mockMessage); 33 | Thread.sleep(this.intervalMillis); 34 | messageCount++; 35 | } 36 | } 37 | 38 | @Override 39 | public void cancel() { 40 | this.runningFlag = false; 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /flink-fs-hadoop-shaded/src/main/resources/META-INF/licenses/LICENSE-re2j: -------------------------------------------------------------------------------- 1 | This is a work derived from Russ Cox's RE2 in Go, whose license 2 | http://golang.org/LICENSE is as follows: 3 | 4 | Copyright (c) 2009 The Go Authors. All rights reserved. 5 | 6 | Redistribution and use in source and binary forms, with or without 7 | modification, are permitted provided that the following conditions are 8 | met: 9 | 10 | * Redistributions of source code must retain the above copyright 11 | notice, this list of conditions and the following disclaimer. 12 | 13 | * Redistributions in binary form must reproduce the above copyright 14 | notice, this list of conditions and the following disclaimer in 15 | the documentation and/or other materials provided with the 16 | distribution. 17 | 18 | * Neither the name of Google Inc. nor the names of its contributors 19 | may be used to endorse or promote products derived from this 20 | software without specific prior written permission. 21 | 22 | THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS 23 | "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT 24 | LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR 25 | A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 26 | OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 27 | SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT 28 | LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, 29 | DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY 30 | THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT 31 | (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE 32 | OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 33 | -------------------------------------------------------------------------------- /flink-cos-fs-base/src/main/java/org/apache/flink/fs/cos/common/writer/RecoverableMultipartUpload.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.fs.cos.common.writer; 20 | 21 | import org.apache.flink.core.fs.RecoverableFsDataOutputStream; 22 | import org.apache.flink.core.fs.RecoverableWriter; 23 | import org.apache.flink.fs.cos.common.utils.RefCountedFSOutputStream; 24 | 25 | import javax.annotation.Nullable; 26 | 27 | import java.io.File; 28 | import java.io.IOException; 29 | import java.util.Optional; 30 | 31 | /** the Recoverable MPU interface. */ 32 | public interface RecoverableMultipartUpload { 33 | RecoverableFsDataOutputStream.Committer snapshotAndGetCommitter() throws IOException; 34 | 35 | RecoverableWriter.ResumeRecoverable snapshotAndGetRecoverable( 36 | @Nullable final RefCountedFSOutputStream incompletePartFile) throws IOException; 37 | 38 | void uploadPart(final RefCountedFSOutputStream file) throws IOException; 39 | 40 | Optional getIncompletePart(); 41 | } 42 | -------------------------------------------------------------------------------- /flink-cos-fs-base/src/main/java/org/apache/flink/fs/cos/common/utils/OffsetTrackOutputStream.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.fs.cos.common.utils; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.util.IOUtils; 23 | 24 | import java.io.Closeable; 25 | import java.io.IOException; 26 | import java.io.OutputStream; 27 | 28 | /** The {@link OutputStream} that keeps track of its current length. */ 29 | @Internal 30 | public final class OffsetTrackOutputStream implements Closeable { 31 | private final OutputStream currentOut; 32 | private long position; 33 | 34 | public OffsetTrackOutputStream(OutputStream currentOut, long position) { 35 | this.currentOut = currentOut; 36 | this.position = position; 37 | } 38 | 39 | public long getLength() { 40 | return this.position; 41 | } 42 | 43 | public void write(byte[] b, int off, int len) throws IOException { 44 | this.currentOut.write(b, off, len); 45 | position += len; 46 | } 47 | 48 | public void flush() throws IOException { 49 | this.currentOut.flush(); 50 | } 51 | 52 | @Override 53 | public void close() throws IOException { 54 | IOUtils.closeQuietly(this.currentOut); 55 | } 56 | } 57 | -------------------------------------------------------------------------------- /flink-fs-hadoop-shaded/src/main/java/org/apache/hadoop/util/NativeCodeLoader.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more contributor license 3 | * agreements. See the NOTICE file distributed with this work for additional information regarding 4 | * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the 5 | * "License"); you may not use this file except in compliance with the License. You may obtain a 6 | * copy of the License at 7 | * 8 | *

http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | *

Unless required by applicable law or agreed to in writing, software distributed under the 11 | * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either 12 | * express or implied. See the License for the specific language governing permissions and 13 | * limitations under the License. 14 | */ 15 | package org.apache.hadoop.util; 16 | 17 | // ---------------------------------------------------------------------------- 18 | // ---------------------------------------------------------------------------- 19 | // 20 | // This class is copied from the Hadoop Source Code (Apache License 2.0) 21 | // in order to override default behavior i the presence of shading 22 | // 23 | // ---------------------------------------------------------------------------- 24 | // ---------------------------------------------------------------------------- 25 | 26 | @SuppressWarnings("unused") 27 | public class NativeCodeLoader { 28 | 29 | public static boolean isNativeCodeLoaded() { 30 | return false; 31 | } 32 | 33 | public static boolean buildSupportsSnappy() { 34 | return false; 35 | } 36 | 37 | public static boolean buildSupportsOpenssl() { 38 | return false; 39 | } 40 | 41 | public static boolean buildSupportsIsal() { 42 | return false; 43 | } 44 | 45 | public static boolean buildSupportsZstd() { 46 | return false; 47 | } 48 | 49 | public static String getLibraryName() { 50 | return null; 51 | } 52 | 53 | private NativeCodeLoader() {} 54 | } 55 | -------------------------------------------------------------------------------- /flink-cos-fs-examples/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | flink-cos-fs 7 | com.qcloud.cos 8 | 1.10.0-0.2.5 9 | 10 | 4.0.0 11 | 12 | flink-cos-fs-examples 13 | Flink : FileSystem : COS FS Examples 14 | 15 | 16 | 17 | iainyu 18 | iainyu@tencent.com 19 | 20 | 21 | alantong 22 | alantong@tencent.com 23 | 24 | 25 | 26 | 27 | 8 28 | 8 29 | true 30 | 31 | 32 | 33 | 34 | org.apache.flink 35 | flink-streaming-java_${scala.binary.version} 36 | ${flink.version} 37 | provided 38 | 39 | 40 | 41 | 42 | 43 | 44 | org.apache.maven.plugins 45 | maven-jar-plugin 46 | 2.4 47 | 48 | 49 | StreamingFileSinkTest 50 | package 51 | 52 | jar 53 | 54 | 55 | StreamingFileSinkTest 56 | 57 | 58 | org.apache.flink.fs.cos.examples.streaming.StreamingFileSinkTest 59 | 60 | 61 | 62 | 63 | 64 | 65 | 66 | 67 | -------------------------------------------------------------------------------- /flink-cos-fs-base/src/main/java/org/apache/flink/fs/cos/common/fswriter/COSPosixRecoverable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.fs.cos.common.fswriter; 20 | 21 | import org.apache.flink.core.fs.RecoverableWriter.CommitRecoverable; 22 | import org.apache.flink.core.fs.RecoverableWriter.ResumeRecoverable; 23 | 24 | import org.apache.hadoop.fs.Path; 25 | 26 | import static org.apache.flink.util.Preconditions.checkArgument; 27 | import static org.apache.flink.util.Preconditions.checkNotNull; 28 | 29 | /** 30 | * An implementation of the resume and commit descriptor objects for Hadoop's 31 | * file system abstraction. 32 | */ 33 | class COSPosixRecoverable implements CommitRecoverable, ResumeRecoverable { 34 | 35 | /** The file path for the final result file. */ 36 | private final Path targetFile; 37 | 38 | /** The file path of the staging file. */ 39 | private final Path tempFile; 40 | 41 | /** The position to resume from. */ 42 | private final long offset; 43 | 44 | /** 45 | * Creates a resumable for the given file at the given position. 46 | * 47 | * @param targetFile The file to resume. 48 | * @param offset The position to resume from. 49 | */ 50 | COSPosixRecoverable(Path targetFile, Path tempFile, long offset) { 51 | checkArgument(offset >= 0, "offset must be >= 0"); 52 | this.targetFile = checkNotNull(targetFile, "targetFile"); 53 | this.tempFile = checkNotNull(tempFile, "tempFile"); 54 | this.offset = offset; 55 | } 56 | 57 | public Path targetFile() { 58 | return targetFile; 59 | } 60 | 61 | public Path tempFile() { 62 | return tempFile; 63 | } 64 | 65 | public long offset() { 66 | return offset; 67 | } 68 | 69 | @Override 70 | public String toString() { 71 | return "COSMergeRecoverable " + tempFile + " @ " + offset + " -> " + targetFile; 72 | } 73 | } 74 | -------------------------------------------------------------------------------- /flink-cos-fs-examples/src/main/java/org/apache/flink/fs/cos/examples/streaming/StreamingFileSinkTest.java: -------------------------------------------------------------------------------- 1 | package org.apache.flink.fs.cos.examples.streaming; 2 | 3 | import org.apache.flink.api.common.serialization.SimpleStringEncoder; 4 | import org.apache.flink.api.java.utils.ParameterTool; 5 | import org.apache.flink.core.fs.Path; 6 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 7 | import org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink; 8 | import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.DefaultRollingPolicy; 9 | 10 | import org.slf4j.Logger; 11 | import org.slf4j.LoggerFactory; 12 | 13 | import java.util.concurrent.TimeUnit; 14 | 15 | /** Flink StreamingFileSink Test. */ 16 | public class StreamingFileSinkTest { 17 | private static final Logger LOG = LoggerFactory.getLogger(StreamingFileSinkTest.class); 18 | private static final String JOB_NAME = "flink-cos-streaming-file-sink-test"; 19 | 20 | private static void printUsage() { 21 | System.out.println( 22 | "Use --input to specify the input data file, and --output to specify the output file. " 23 | + "For example: '--output cosn://flink-test-1250000000/streamingFileSinkTest'"); 24 | } 25 | 26 | public static void main(String[] args) throws Exception { 27 | ParameterTool parameterTool = ParameterTool.fromArgs(args); 28 | 29 | if (null == parameterTool.get("output")) { 30 | printUsage(); 31 | return; 32 | } 33 | String outputPath = parameterTool.getRequired("output"); 34 | StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); 35 | StreamingFileSink streamingFileSink = 36 | StreamingFileSink.forRowFormat( 37 | new Path(outputPath), new SimpleStringEncoder("UTF-8")) 38 | .withRollingPolicy( 39 | DefaultRollingPolicy.builder() 40 | .withRolloverInterval(TimeUnit.SECONDS.toMillis(5)) 41 | .withInactivityInterval(TimeUnit.SECONDS.toMillis(5)) 42 | .withMaxPartSize(1024) 43 | .build()) 44 | .build(); 45 | if (parameterTool.get("input") == null) { 46 | // Use the mockSource to generate the test data 47 | System.out.println( 48 | "The '--input' parameter is not specified. " 49 | + "Use the inner MockSource instead to generate test data."); 50 | env.addSource(new MockSource(1000, 100)).addSink(streamingFileSink); 51 | } else { 52 | env.readTextFile(parameterTool.get("input")).addSink(streamingFileSink); 53 | } 54 | 55 | LOG.info("Begin to execute the job: {}.", JOB_NAME); 56 | env.execute(JOB_NAME); 57 | } 58 | } 59 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | ### Java template 2 | # Compiled class file 3 | *.class 4 | 5 | # Log file 6 | *.log 7 | 8 | # BlueJ files 9 | *.ctxt 10 | 11 | # Mobile Tools for Java (J2ME) 12 | .mtj.tmp/ 13 | 14 | # generate target 15 | flink-cos-fs-base/target 16 | flink-cos-fs-examples/target 17 | flink-cos-fs-hadoop/target 18 | flink-fs-hadoop-shaded/target 19 | target 20 | 21 | # Package Files # 22 | *.jar 23 | *.war 24 | *.nar 25 | *.ear 26 | *.zip 27 | *.tar.gz 28 | *.rar 29 | 30 | # virtual machine crash logs, see http://www.java.com/en/download/help/error_hotspot.xml 31 | hs_err_pid* 32 | 33 | ### JetBrains template 34 | # Covers JetBrains IDEs: IntelliJ, RubyMine, PhpStorm, AppCode, PyCharm, CLion, Android Studio, WebStorm and Rider 35 | # Reference: https://intellij-support.jetbrains.com/hc/en-us/articles/206544839 36 | 37 | # User-specific stuff 38 | .idea 39 | .idea/**/workspace.xml 40 | .idea/**/tasks.xml 41 | .idea/**/usage.statistics.xml 42 | .idea/**/dictionaries 43 | .idea/**/shelf 44 | 45 | # Generated files 46 | .idea/**/contentModel.xml 47 | 48 | # Sensitive or high-churn files 49 | .idea/**/dataSources/ 50 | .idea/**/dataSources.ids 51 | .idea/**/dataSources.local.xml 52 | .idea/**/sqlDataSources.xml 53 | .idea/**/dynamic.xml 54 | .idea/**/uiDesigner.xml 55 | .idea/**/dbnavigator.xml 56 | 57 | # Gradle 58 | .idea/**/gradle.xml 59 | .idea/**/libraries 60 | 61 | # Gradle and Maven with auto-import 62 | # When using Gradle or Maven with auto-import, you should exclude module files, 63 | # since they will be recreated, and may cause churn. Uncomment if using 64 | # auto-import. 65 | # .idea/artifacts 66 | # .idea/compiler.xml 67 | # .idea/jarRepositories.xml 68 | # .idea/modules.xml 69 | # .idea/*.iml 70 | # .idea/modules 71 | # *.iml 72 | # *.ipr 73 | 74 | # CMake 75 | cmake-build-*/ 76 | 77 | # Mongo Explorer plugin 78 | .idea/**/mongoSettings.xml 79 | 80 | # File-based project format 81 | *.iws 82 | 83 | # IntelliJ 84 | out/ 85 | 86 | # mpeltonen/sbt-idea plugin 87 | .idea_modules/ 88 | 89 | # JIRA plugin 90 | atlassian-ide-plugin.xml 91 | 92 | # Cursive Clojure plugin 93 | .idea/replstate.xml 94 | 95 | # Crashlytics plugin (for Android Studio and IntelliJ) 96 | com_crashlytics_export_strings.xml 97 | crashlytics.properties 98 | crashlytics-build.properties 99 | fabric.properties 100 | 101 | # Editor-based Rest Client 102 | .idea/httpRequests 103 | 104 | # Android studio 3.1+ serialized cache file 105 | .idea/caches/build_file_checksums.ser 106 | 107 | ### Diff template 108 | *.patch 109 | *.diff 110 | 111 | ### macOS template 112 | # General 113 | .DS_Store 114 | .AppleDouble 115 | .LSOverride 116 | 117 | # Icon must end with two \r 118 | Icon 119 | 120 | # Thumbnails 121 | ._* 122 | 123 | # Files that might appear in the root of a volume 124 | .DocumentRevisions-V100 125 | .fseventsd 126 | .Spotlight-V100 127 | .TemporaryItems 128 | .Trashes 129 | .VolumeIcon.icns 130 | .com.apple.timemachine.donotpresent 131 | 132 | # Directories potentially created on remote AFP share 133 | .AppleDB 134 | .AppleDesktop 135 | Network Trash Folder 136 | Temporary Items 137 | .apdisk 138 | 139 | -------------------------------------------------------------------------------- /flink-cos-fs-base/src/main/java/org/apache/flink/fs/cos/common/writer/COSAccessHelper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.fs.cos.common.writer; 20 | 21 | import com.qcloud.cos.model.CompleteMultipartUploadResult; 22 | import com.qcloud.cos.model.PartETag; 23 | import org.apache.hadoop.fs.FileMetadata; 24 | 25 | import java.io.File; 26 | import java.io.IOException; 27 | import java.util.List; 28 | 29 | /** The COSAccessHelper interface. */ 30 | public interface COSAccessHelper { 31 | /** 32 | * Initialize a multipart upload. 33 | * 34 | * @param key the cos key whose value we want to upload in parts. 35 | * @return The upload id the initiated multi-part upload which will be used during the uploading 36 | * of parts. 37 | * @throws IOException 38 | */ 39 | String startMultipartUpload(String key) throws IOException; 40 | 41 | /** 42 | * Uploads a part and associates it with MPU with the provided. 43 | * 44 | * @param key The key which MPU is associated with 45 | * @param uploadId the upload id of the MPU 46 | * @param partNumber the part number of the part being uploaded. 47 | * @param inputFile the file holding the part to be uploaded 48 | * @return The {@link PartETag} of the attempt to upload the part. 49 | * @throws IOException 50 | */ 51 | PartETag uploadPart(String key, String uploadId, int partNumber, File inputFile, byte[] md5Hash) 52 | throws IOException; 53 | 54 | /** 55 | * @param key The cos key used to identify this part. 56 | * @param inputFile the local file holding the data to be uploaded. 57 | * @throws IOException 58 | */ 59 | void putObject(String key, File inputFile, byte[] md5Hash) throws IOException; 60 | 61 | /** 62 | * @param key The key identifying the object we finished uploading. 63 | * @param uploadId the upload id of the multipart upload. 64 | * @param partETags the list of {@link PartETag} associated with the Multipart Upload. 65 | * @return the Complete Multipart upload result. 66 | * @throws IOException 67 | */ 68 | CompleteMultipartUploadResult commitMultipartUpload( 69 | String key, String uploadId, List partETags) throws IOException; 70 | 71 | boolean deleteObject(String key) throws IOException; 72 | 73 | long getObject(String key, File targetLocation) throws IOException; 74 | 75 | FileMetadata getObjectMetadata(String key) throws IOException; 76 | 77 | boolean isPosixBucket(); 78 | } 79 | -------------------------------------------------------------------------------- /flink-cos-fs-base/src/main/java/org/apache/flink/fs/cos/common/utils/BackPressuringExecutor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.fs.cos.common.utils; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.util.ExceptionUtils; 23 | import org.apache.flink.util.FlinkRuntimeException; 24 | 25 | import java.util.concurrent.Executor; 26 | import java.util.concurrent.Semaphore; 27 | import java.util.concurrent.atomic.AtomicBoolean; 28 | 29 | import static org.apache.flink.util.Preconditions.checkArgument; 30 | import static org.apache.flink.util.Preconditions.checkNotNull; 31 | 32 | /** 33 | * An executor decorate that allows only a certain number of concurrent executions. The {@link 34 | * #execute(Runnable)} method blocks once that number of executions is exceeded. 35 | */ 36 | @Internal 37 | public final class BackPressuringExecutor implements Executor { 38 | private final Executor delegate; 39 | 40 | private final Semaphore permits; 41 | 42 | public BackPressuringExecutor(Executor delegate, int numConcurrentExecutions) { 43 | checkArgument(numConcurrentExecutions > 0, "numConcurrentExecutions must be > 0"); 44 | this.delegate = checkNotNull(delegate, "delegate"); 45 | this.permits = new Semaphore(numConcurrentExecutions, true); 46 | } 47 | 48 | @Override 49 | public void execute(Runnable command) { 50 | try { 51 | permits.acquire(); 52 | } catch (InterruptedException e) { 53 | Thread.currentThread().interrupt(); 54 | throw new FlinkRuntimeException("interrupted:", e); 55 | } 56 | 57 | final SemaphoreReleasingRunnable runnable = 58 | new SemaphoreReleasingRunnable(command, permits); 59 | try { 60 | delegate.execute(runnable); 61 | } catch (Throwable e) { 62 | runnable.release(); 63 | ExceptionUtils.rethrow(e, e.getMessage()); 64 | } 65 | } 66 | 67 | private static class SemaphoreReleasingRunnable implements Runnable { 68 | 69 | private final Runnable delegate; 70 | 71 | private final Semaphore toRelease; 72 | 73 | private final AtomicBoolean released = new AtomicBoolean(); 74 | 75 | SemaphoreReleasingRunnable(Runnable delegate, Semaphore toRelease) { 76 | this.delegate = delegate; 77 | this.toRelease = toRelease; 78 | } 79 | 80 | @Override 81 | public void run() { 82 | try { 83 | delegate.run(); 84 | } finally { 85 | release(); 86 | } 87 | } 88 | 89 | void release() { 90 | if (released.compareAndSet(false, true)) { 91 | toRelease.release(); 92 | } 93 | } 94 | } 95 | } 96 | -------------------------------------------------------------------------------- /flink-cos-fs-base/src/main/java/org/apache/flink/fs/cos/common/writer/MultipartUploadInfo.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.fs.cos.common.writer; 20 | 21 | import com.qcloud.cos.model.PartETag; 22 | 23 | import java.io.File; 24 | import java.util.ArrayList; 25 | import java.util.List; 26 | import java.util.Optional; 27 | 28 | import static org.apache.flink.util.Preconditions.checkArgument; 29 | import static org.apache.flink.util.Preconditions.checkNotNull; 30 | 31 | /** MultipartUploadInfo. */ 32 | public class MultipartUploadInfo { 33 | 34 | private final String objectName; 35 | 36 | private final String uploadId; 37 | 38 | private final List completeParts; 39 | 40 | private final Optional incompletePart; 41 | 42 | /** 43 | * This contains both the parts that are already uploaded but also the ones that are queued to 44 | * be uploaded. 45 | */ 46 | private int numberOfRegisteredParts; 47 | 48 | /** 49 | * This is the total size of the upload, i.e. also with the parts that are queued but not 50 | * uploaded yet. 51 | */ 52 | private long expectedSizeInBytes; 53 | 54 | MultipartUploadInfo( 55 | final String objectName, 56 | final String uploadId, 57 | final List completeParts, 58 | final long numBytes, 59 | final Optional incompletePart) { 60 | 61 | checkArgument(numBytes >= 0L); 62 | 63 | this.objectName = checkNotNull(objectName); 64 | this.uploadId = checkNotNull(uploadId); 65 | this.completeParts = checkNotNull(completeParts); 66 | this.incompletePart = checkNotNull(incompletePart); 67 | 68 | this.numberOfRegisteredParts = completeParts.size(); 69 | this.expectedSizeInBytes = numBytes; 70 | } 71 | 72 | String getObjectName() { 73 | return objectName; 74 | } 75 | 76 | String getUploadId() { 77 | return uploadId; 78 | } 79 | 80 | int getNumberOfRegisteredParts() { 81 | return numberOfRegisteredParts; 82 | } 83 | 84 | long getExpectedSizeInBytes() { 85 | return expectedSizeInBytes; 86 | } 87 | 88 | Optional getIncompletePart() { 89 | return incompletePart; 90 | } 91 | 92 | List getCopyOfEtagsOfCompleteParts() { 93 | return new ArrayList<>(completeParts); 94 | } 95 | 96 | void registerNewPart(long length) { 97 | this.expectedSizeInBytes += length; 98 | this.numberOfRegisteredParts++; 99 | } 100 | 101 | void registerCompletePart(PartETag eTag) { 102 | completeParts.add(eTag); 103 | } 104 | 105 | int getRemainingParts() { 106 | return numberOfRegisteredParts - completeParts.size(); 107 | } 108 | } 109 | -------------------------------------------------------------------------------- /tools/maven/suppressions.xml: -------------------------------------------------------------------------------- 1 | 2 | 20 | 21 | 24 | 25 | 26 | 27 | 28 | 29 | 30 | 31 | 32 | 33 | 34 | 35 | 36 | 39 | 42 | 43 | 46 | 47 | 50 | 51 | 54 | 55 | 58 | 61 | 64 | 65 | 68 | 69 | 72 | 73 | 76 | 77 | -------------------------------------------------------------------------------- /flink-cos-fs-base/src/main/java/org/apache/flink/fs/cos/common/utils/RefCountedTmpFileCreator.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.fs.cos.common.utils; 20 | 21 | import org.apache.flink.util.function.FunctionWithException; 22 | 23 | import java.io.File; 24 | import java.io.IOException; 25 | import java.io.OutputStream; 26 | import java.nio.file.FileAlreadyExistsException; 27 | import java.nio.file.Files; 28 | import java.nio.file.StandardOpenOption; 29 | import java.util.Random; 30 | import java.util.UUID; 31 | import java.util.concurrent.atomic.AtomicInteger; 32 | 33 | import static org.apache.flink.util.Preconditions.checkArgument; 34 | 35 | /** RefCountedTmpFileCreator. */ 36 | public class RefCountedTmpFileCreator 37 | implements FunctionWithException { 38 | 39 | private final File[] tempDirectories; 40 | 41 | private final AtomicInteger next; 42 | 43 | private RefCountedTmpFileCreator(File... tempDirectories) { 44 | checkArgument(tempDirectories.length > 0, "tempDirectories must not be empty"); 45 | for (File f : tempDirectories) { 46 | if (f == null) { 47 | throw new IllegalArgumentException("tempDirectories contains null entries"); 48 | } 49 | } 50 | 51 | this.tempDirectories = tempDirectories.clone(); 52 | this.next = new AtomicInteger(new Random().nextInt(this.tempDirectories.length)); 53 | } 54 | 55 | @Override 56 | public RefCountedFile apply(File file) throws IOException { 57 | final File directory = tempDirectories[nextIndex()]; 58 | 59 | while (true) { 60 | try { 61 | if (file == null) { 62 | final File newFile = new File(directory, ".tmp_" + UUID.randomUUID()); 63 | final OutputStream out = 64 | Files.newOutputStream(newFile.toPath(), StandardOpenOption.CREATE_NEW); 65 | return RefCountedFile.newFile(newFile, out); 66 | } else { 67 | final OutputStream out = 68 | Files.newOutputStream(file.toPath(), StandardOpenOption.APPEND); 69 | return RefCountedFile.restoreFile(file, out, file.length()); 70 | } 71 | } catch (FileAlreadyExistsException ignored) { 72 | // fall through the loop and retry 73 | } 74 | } 75 | } 76 | 77 | private int nextIndex() { 78 | int currIndex, newIndex; 79 | do { 80 | currIndex = next.get(); 81 | newIndex = currIndex + 1; 82 | if (newIndex == tempDirectories.length) { 83 | newIndex = 0; 84 | } 85 | } while (!next.compareAndSet(currIndex, newIndex)); 86 | 87 | return currIndex; 88 | } 89 | 90 | public static RefCountedTmpFileCreator inDirectories(File... tmpDirectories) { 91 | return new RefCountedTmpFileCreator(tmpDirectories); 92 | } 93 | } 94 | -------------------------------------------------------------------------------- /flink-fs-hadoop-shaded/src/main/java/org/apache/hadoop/conf/StorageSize.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.hadoop.conf; 20 | 21 | import java.util.Locale; 22 | 23 | import static org.apache.commons.lang3.StringUtils.isNotBlank; 24 | 25 | /** 26 | * This class is copied from the Hadoop Source Code (Apache License 2.0) in order to override config 27 | * keys to support shading. 28 | */ 29 | public class StorageSize { 30 | private final StorageUnit unit; 31 | private final double value; 32 | 33 | /** 34 | * Constucts a Storage Measure, which contains the value and the unit of measure. 35 | * 36 | * @param unit - Unit of Measure 37 | * @param value - Numeric value. 38 | */ 39 | public StorageSize(StorageUnit unit, double value) { 40 | this.unit = unit; 41 | this.value = value; 42 | } 43 | 44 | private static void checkState(boolean state, String errorString) { 45 | if (!state) { 46 | throw new IllegalStateException(errorString); 47 | } 48 | } 49 | 50 | public static StorageSize parse(String value) { 51 | checkState(isNotBlank(value), "value cannot be blank"); 52 | String sanitizedValue = value.trim().toLowerCase(Locale.ENGLISH); 53 | StorageUnit parsedUnit = null; 54 | for (StorageUnit unit : StorageUnit.values()) { 55 | if (sanitizedValue.endsWith(unit.getShortName()) 56 | || sanitizedValue.endsWith(unit.getLongName()) 57 | || sanitizedValue.endsWith(unit.getSuffixChar())) { 58 | parsedUnit = unit; 59 | break; 60 | } 61 | } 62 | 63 | if (parsedUnit == null) { 64 | throw new IllegalArgumentException( 65 | value 66 | + " is not in expected format." 67 | + "Expected format is . e.g. 1000MB"); 68 | } 69 | 70 | String suffix = ""; 71 | boolean found = false; 72 | 73 | // We are trying to get the longest match first, so the order of 74 | // matching is getLongName, getShortName and then getSuffixChar. 75 | if (!found && sanitizedValue.endsWith(parsedUnit.getLongName())) { 76 | found = true; 77 | suffix = parsedUnit.getLongName(); 78 | } 79 | 80 | if (!found && sanitizedValue.endsWith(parsedUnit.getShortName())) { 81 | found = true; 82 | suffix = parsedUnit.getShortName(); 83 | } 84 | 85 | if (!found && sanitizedValue.endsWith(parsedUnit.getSuffixChar())) { 86 | found = true; 87 | suffix = parsedUnit.getSuffixChar(); 88 | } 89 | 90 | checkState(found, "Something is wrong, we have to find a " + "match. Internal error."); 91 | 92 | String valString = sanitizedValue.substring(0, value.length() - suffix.length()); 93 | return new StorageSize(parsedUnit, Double.parseDouble(valString)); 94 | } 95 | 96 | public StorageUnit getUnit() { 97 | return unit; 98 | } 99 | 100 | public double getValue() { 101 | return value; 102 | } 103 | } 104 | -------------------------------------------------------------------------------- /flink-cos-fs-base/src/main/java/org/apache/flink/fs/cos/common/writer/COSRecoverable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.fs.cos.common.writer; 20 | 21 | import org.apache.flink.core.fs.RecoverableWriter; 22 | 23 | import com.qcloud.cos.model.PartETag; 24 | 25 | import javax.annotation.Nullable; 26 | 27 | import java.util.List; 28 | 29 | import static org.apache.flink.util.Preconditions.checkArgument; 30 | import static org.apache.flink.util.Preconditions.checkNotNull; 31 | 32 | /** The COS Recoverable to implement the recover from the checkpoint. */ 33 | public final class COSRecoverable implements RecoverableWriter.ResumeRecoverable { 34 | 35 | private final String uploadId; 36 | private final String objectName; 37 | private final List partETags; 38 | 39 | private final String lastPartObject; 40 | 41 | private long numBytesInParts; 42 | private long lastPartObjectLength; 43 | 44 | public COSRecoverable( 45 | String uploadId, String objectName, List partETags, long numBytesInParts) { 46 | this(uploadId, objectName, partETags, numBytesInParts, null, -1L); 47 | } 48 | 49 | COSRecoverable( 50 | String uploadId, 51 | String objectName, 52 | List partETags, 53 | long numBytesInParts, 54 | @Nullable String lastPartObject, 55 | long lastPartObjectLength) { 56 | checkArgument(numBytesInParts >= 0L); 57 | checkArgument(lastPartObject == null || lastPartObjectLength > 0L); 58 | 59 | this.uploadId = checkNotNull(uploadId); 60 | this.objectName = checkNotNull(objectName); 61 | this.partETags = checkNotNull(partETags); 62 | this.numBytesInParts = numBytesInParts; 63 | 64 | this.lastPartObject = lastPartObject; 65 | this.lastPartObjectLength = lastPartObjectLength; 66 | } 67 | 68 | public String getUploadId() { 69 | return uploadId; 70 | } 71 | 72 | public String getObjectName() { 73 | return objectName; 74 | } 75 | 76 | public List getPartETags() { 77 | return partETags; 78 | } 79 | 80 | public long getNumBytesInParts() { 81 | return numBytesInParts; 82 | } 83 | 84 | @Nullable 85 | public String getInCompleteObjectName() { 86 | return this.lastPartObject; 87 | } 88 | 89 | public long getInCompleteObjectLength() { 90 | return this.lastPartObjectLength; 91 | } 92 | 93 | @Override 94 | public String toString() { 95 | StringBuilder buf = new StringBuilder(128); 96 | buf.append("COSRecoverable: "); 97 | buf.append("key=").append(objectName); 98 | buf.append(", uploadId=").append(uploadId); 99 | buf.append(", bytesInParts=").append(numBytesInParts); 100 | buf.append(", parts=["); 101 | int num = 0; 102 | for (PartETag part : this.partETags) { 103 | if (0 != num++) { 104 | buf.append(", "); 105 | } 106 | buf.append(part.getPartNumber()).append('=').append(part.getETag()); 107 | } 108 | buf.append("], trailingPart=").append(lastPartObject); 109 | buf.append("trailingPartLen=").append(lastPartObjectLength); 110 | 111 | return buf.toString(); 112 | } 113 | } 114 | -------------------------------------------------------------------------------- /flink-cos-fs-base/src/main/java/org/apache/flink/fs/cos/common/utils/RefCountedFile.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.fs.cos.common.utils; 20 | 21 | import org.apache.flink.annotation.VisibleForTesting; 22 | import org.apache.flink.util.ExceptionUtils; 23 | import org.apache.flink.util.IOUtils; 24 | 25 | import java.io.File; 26 | import java.io.IOException; 27 | import java.io.OutputStream; 28 | import java.nio.file.Files; 29 | import java.util.concurrent.atomic.AtomicInteger; 30 | 31 | import static org.apache.flink.util.Preconditions.checkNotNull; 32 | 33 | /** RefCountedFile. */ 34 | public class RefCountedFile implements RefCounted { 35 | private final File file; 36 | private final OffsetTrackOutputStream outputStream; 37 | private final AtomicInteger references; 38 | private boolean closed; 39 | 40 | public static RefCountedFile newFile(final File file, final OutputStream currentOutputStream) { 41 | return new RefCountedFile(file, currentOutputStream, 0L); 42 | } 43 | 44 | public static RefCountedFile restoreFile( 45 | final File file, final OutputStream currentOutputStream, final long byteInCurrentPart) { 46 | return new RefCountedFile(file, currentOutputStream, byteInCurrentPart); 47 | } 48 | 49 | private RefCountedFile( 50 | final File file, 51 | final OutputStream currentOutputStream, 52 | final long bytesInCurrentPart) { 53 | 54 | this.file = checkNotNull(file); 55 | this.outputStream = new OffsetTrackOutputStream(currentOutputStream, bytesInCurrentPart); 56 | this.references = new AtomicInteger(1); 57 | this.closed = false; 58 | } 59 | 60 | public File getFile() { 61 | return file; 62 | } 63 | 64 | public OffsetTrackOutputStream getOutputStream() { 65 | return outputStream; 66 | } 67 | 68 | public long getLength() { 69 | return this.outputStream.getLength(); 70 | } 71 | 72 | public void write(byte[] b, int off, int len) throws IOException { 73 | this.requiredOpened(); 74 | if (len > 0) { 75 | this.outputStream.write(b, off, len); 76 | } 77 | } 78 | 79 | public void flush() throws IOException { 80 | this.requiredOpened(); 81 | this.outputStream.flush(); 82 | } 83 | 84 | public void closeStream() { 85 | if (!this.closed) { 86 | IOUtils.closeQuietly(this.outputStream); 87 | this.closed = true; 88 | } 89 | } 90 | 91 | private void requiredOpened() throws IOException { 92 | if (this.closed) { 93 | throw new IOException("The stream has been closed."); 94 | } 95 | } 96 | 97 | @Override 98 | public void retain() { 99 | this.references.incrementAndGet(); 100 | } 101 | 102 | @Override 103 | public boolean release() { 104 | if (this.references.decrementAndGet() == 0) { 105 | return this.tryClose(); 106 | } 107 | return false; 108 | } 109 | 110 | @VisibleForTesting 111 | int getReferenceCounter() { 112 | return references.get(); 113 | } 114 | 115 | private boolean tryClose() { 116 | boolean deletedTag = false; 117 | try { 118 | deletedTag = Files.deleteIfExists(file.toPath()); 119 | } catch (Throwable e) { 120 | ExceptionUtils.rethrowIfFatalError(e); 121 | } 122 | 123 | return deletedTag; 124 | } 125 | } 126 | -------------------------------------------------------------------------------- /flink-cos-fs-base/src/main/java/org/apache/flink/fs/cos/common/fswriter/COSPosixRecoverableSerializer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.fs.cos.common.fswriter; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.core.io.SimpleVersionedSerializer; 23 | 24 | import org.apache.hadoop.fs.Path; 25 | import org.slf4j.Logger; 26 | import org.slf4j.LoggerFactory; 27 | 28 | import java.io.IOException; 29 | import java.nio.ByteBuffer; 30 | import java.nio.ByteOrder; 31 | import java.nio.charset.Charset; 32 | import java.nio.charset.StandardCharsets; 33 | 34 | /** 35 | * Simple serializer for the {@link COSPosixRecoverable}. 36 | */ 37 | @Internal 38 | class COSPosixRecoverableSerializer implements SimpleVersionedSerializer { 39 | 40 | static final COSPosixRecoverableSerializer INSTANCE = 41 | new COSPosixRecoverableSerializer(); 42 | 43 | private static final Logger LOG = LoggerFactory.getLogger(COSPosixRecoverableSerializer.class); 44 | 45 | private static final Charset CHARSET = StandardCharsets.UTF_8; 46 | 47 | private static final int MAGIC_NUMBER = 0xd7436c5e; 48 | 49 | /** 50 | * Do not instantiate, use reusable {@link #INSTANCE} instead. 51 | */ 52 | private COSPosixRecoverableSerializer() {} 53 | 54 | @Override 55 | public int getVersion() { 56 | return 1; 57 | } 58 | 59 | @Override 60 | public byte[] serialize(COSPosixRecoverable obj) throws IOException { 61 | LOG.debug("cos merge serialize {}, {}, {}", obj.targetFile().toString(), 62 | obj.tempFile().toString(), obj.offset()); 63 | final byte[] targetFileBytes = obj.targetFile().toString().getBytes(CHARSET); 64 | final byte[] tempFileBytes = obj.tempFile().toString().getBytes(CHARSET); 65 | final byte[] targetBytes = new byte[20 + targetFileBytes.length + tempFileBytes.length]; 66 | 67 | ByteBuffer bb = ByteBuffer.wrap(targetBytes).order(ByteOrder.LITTLE_ENDIAN); 68 | bb.putInt(MAGIC_NUMBER); 69 | bb.putLong(obj.offset()); 70 | bb.putInt(targetFileBytes.length); 71 | bb.putInt(tempFileBytes.length); 72 | bb.put(targetFileBytes); 73 | bb.put(tempFileBytes); 74 | 75 | return targetBytes; 76 | } 77 | 78 | @Override 79 | public COSPosixRecoverable deserialize(int version, byte[] serialized) throws IOException { 80 | LOG.debug("cos merge deserialize"); 81 | switch (version) { 82 | case 1: 83 | return deserializeV1(serialized); 84 | default: 85 | throw new IOException("Unrecognized version or corrupt state: " + version); 86 | } 87 | } 88 | 89 | private static COSPosixRecoverable deserializeV1(byte[] serialized) throws IOException { 90 | LOG.debug("cos merge deserialize v1"); 91 | final ByteBuffer bb = ByteBuffer.wrap(serialized).order(ByteOrder.LITTLE_ENDIAN); 92 | 93 | if (bb.getInt() != MAGIC_NUMBER) { 94 | throw new IOException("Corrupt data: Unexpected magic number."); 95 | } 96 | 97 | final long offset = bb.getLong(); 98 | final byte[] targetFileBytes = new byte[bb.getInt()]; 99 | final byte[] tempFileBytes = new byte[bb.getInt()]; 100 | bb.get(targetFileBytes); 101 | bb.get(tempFileBytes); 102 | 103 | final String targetPath = new String(targetFileBytes, CHARSET); 104 | final String tempPath = new String(tempFileBytes, CHARSET); 105 | 106 | return new COSPosixRecoverable(new Path(targetPath), new Path(tempPath), offset); 107 | 108 | } 109 | } 110 | -------------------------------------------------------------------------------- /flink-fs-hadoop-shaded/src/main/java/org/apache/hadoop/util/VersionInfo.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.hadoop.util; 20 | 21 | import org.apache.hadoop.classification.InterfaceAudience; 22 | import org.apache.hadoop.classification.InterfaceStability; 23 | 24 | import java.util.Properties; 25 | 26 | // ---------------------------------------------------------------------------- 27 | // ---------------------------------------------------------------------------- 28 | // 29 | // This class is copied from the Hadoop Source Code (Apache License 2.0) 30 | // in order to override default behavior i the presence of shading 31 | // 32 | // ---------------------------------------------------------------------------- 33 | // ---------------------------------------------------------------------------- 34 | 35 | @SuppressWarnings("all") 36 | @InterfaceAudience.Public 37 | @InterfaceStability.Stable 38 | public class VersionInfo { 39 | 40 | private final Properties info; 41 | 42 | protected VersionInfo(String component) { 43 | info = new Properties(); 44 | 45 | if ("common".equals(component)) { 46 | info.setProperty("version", "3.1.0"); 47 | info.setProperty("revision", "16b70619a24cdcf5d3b0fcf4b58ca77238ccbe6d"); 48 | info.setProperty("branch", "branch-3.1.0"); 49 | info.setProperty("user", "wtan"); 50 | info.setProperty("date", "2018-04-03T04:00Z"); 51 | info.setProperty("url", "git@github.com:hortonworks/hadoop-common-trunk.git"); 52 | info.setProperty("srcChecksum", "14182d20c972b3e2105580a1ad6990"); 53 | info.setProperty("protocVersion", "2.5.0"); 54 | } 55 | } 56 | 57 | protected String _getVersion() { 58 | return info.getProperty("version", "Unknown"); 59 | } 60 | 61 | protected String _getRevision() { 62 | return info.getProperty("revision", "Unknown"); 63 | } 64 | 65 | protected String _getBranch() { 66 | return info.getProperty("branch", "Unknown"); 67 | } 68 | 69 | protected String _getDate() { 70 | return info.getProperty("date", "Unknown"); 71 | } 72 | 73 | protected String _getUser() { 74 | return info.getProperty("user", "Unknown"); 75 | } 76 | 77 | protected String _getUrl() { 78 | return info.getProperty("url", "Unknown"); 79 | } 80 | 81 | protected String _getSrcChecksum() { 82 | return info.getProperty("srcChecksum", "Unknown"); 83 | } 84 | 85 | protected String _getBuildVersion(){ 86 | return _getVersion() + 87 | " from " + _getRevision() + 88 | " by " + _getUser() + 89 | " source checksum " + _getSrcChecksum(); 90 | } 91 | 92 | protected String _getProtocVersion() { 93 | return info.getProperty("protocVersion", "Unknown"); 94 | } 95 | 96 | private static final VersionInfo COMMON_VERSION_INFO = new VersionInfo("common"); 97 | 98 | public static String getVersion() { 99 | return COMMON_VERSION_INFO._getVersion(); 100 | } 101 | 102 | public static String getRevision() { 103 | return COMMON_VERSION_INFO._getRevision(); 104 | } 105 | 106 | public static String getBranch() { 107 | return COMMON_VERSION_INFO._getBranch(); 108 | } 109 | 110 | public static String getDate() { 111 | return COMMON_VERSION_INFO._getDate(); 112 | } 113 | 114 | public static String getUser() { 115 | return COMMON_VERSION_INFO._getUser(); 116 | } 117 | 118 | public static String getUrl() { 119 | return COMMON_VERSION_INFO._getUrl(); 120 | } 121 | 122 | public static String getSrcChecksum() { 123 | return COMMON_VERSION_INFO._getSrcChecksum(); 124 | } 125 | 126 | public static String getBuildVersion(){ 127 | return COMMON_VERSION_INFO._getBuildVersion(); 128 | } 129 | 130 | public static String getProtocVersion(){ 131 | return COMMON_VERSION_INFO._getProtocVersion(); 132 | } 133 | } 134 | -------------------------------------------------------------------------------- /flink-cos-fs-hadoop/src/main/java/org/apache/flink/fs/coshadoop/HadoopCOSAccessHelper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.fs.coshadoop; 20 | 21 | import org.apache.flink.fs.cos.common.writer.COSAccessHelper; 22 | 23 | import com.qcloud.cos.model.CompleteMultipartUploadResult; 24 | import com.qcloud.cos.model.PartETag; 25 | import org.apache.hadoop.fs.FileMetadata; 26 | import org.apache.hadoop.fs.NativeFileSystemStore; 27 | 28 | import java.io.File; 29 | import java.io.FileNotFoundException; 30 | import java.io.FileOutputStream; 31 | import java.io.IOException; 32 | import java.io.InputStream; 33 | import java.io.OutputStream; 34 | import java.util.List; 35 | 36 | /** The AccessHelper for the CosN NativeFileSystemStore interface. */ 37 | public class HadoopCOSAccessHelper implements COSAccessHelper { 38 | 39 | private final NativeFileSystemStore store; 40 | 41 | public HadoopCOSAccessHelper(NativeFileSystemStore store) { 42 | this.store = store; 43 | } 44 | 45 | @Override 46 | public String startMultipartUpload(String key) throws IOException { 47 | return this.store.getUploadId(key); 48 | } 49 | 50 | @Override 51 | public PartETag uploadPart( 52 | String key, String uploadId, int partNumber, File inputFile, byte[] md5Hash) 53 | throws IOException { 54 | return this.store.uploadPart(inputFile, key, uploadId, partNumber, md5Hash); 55 | } 56 | 57 | @Override 58 | public void putObject(String key, File inputFile, byte[] md5Hash) throws IOException { 59 | this.store.storeFile(key, inputFile, md5Hash); 60 | } 61 | 62 | @Override 63 | public CompleteMultipartUploadResult commitMultipartUpload( 64 | String key, String uploadId, List partETags) throws IOException { 65 | return this.store.completeMultipartUpload(key, uploadId, partETags); 66 | } 67 | 68 | @Override 69 | public boolean deleteObject(String key) throws IOException { 70 | this.store.delete(key); 71 | return true; 72 | } 73 | 74 | @Override 75 | public long getObject(String key, File targetLocation) throws IOException { 76 | long numBytes = 0L; 77 | try (final OutputStream outStream = new FileOutputStream(targetLocation); 78 | final InputStream inStream = this.store.retrieve(key); ) { 79 | final byte[] buffer = new byte[32 * 1024]; 80 | 81 | int numRead; 82 | while ((numRead = inStream.read(buffer)) != -1) { 83 | outStream.write(buffer, 0, numRead); 84 | numBytes += numRead; 85 | } 86 | } 87 | 88 | // some sanity checks 89 | if (numBytes != targetLocation.length()) { 90 | throw new IOException( 91 | String.format( 92 | "Error recovering writer: " 93 | + "Downloading the last data chunk file gives incorrect length. " 94 | + "File=%d bytes, Stream=%d bytes", 95 | targetLocation.length(), numBytes)); 96 | } 97 | 98 | return numBytes; 99 | } 100 | 101 | @Override 102 | public FileMetadata getObjectMetadata(String key) throws IOException { 103 | FileMetadata fileMetadata = this.store.retrieveMetadata(key); 104 | if (null != fileMetadata) { 105 | return fileMetadata; 106 | } else { 107 | throw new FileNotFoundException("No such file for the key '" + key + "'"); 108 | } 109 | } 110 | 111 | @Override 112 | public boolean isPosixBucket() { 113 | return this.store.isPosixBucket(); 114 | } 115 | } 116 | -------------------------------------------------------------------------------- /flink-cos-fs-base/src/main/java/org/apache/flink/fs/cos/common/FlinkCOSFileSystem.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.fs.cos.common; 20 | 21 | import org.apache.flink.core.fs.FileSystemKind; 22 | import org.apache.flink.core.fs.Path; 23 | import org.apache.flink.core.fs.RecoverableWriter; 24 | import org.apache.flink.fs.cos.common.fswriter.COSPosixRecoverableWriter; 25 | import org.apache.flink.fs.cos.common.utils.RefCountedFile; 26 | import org.apache.flink.fs.cos.common.utils.RefCountedTmpFileCreator; 27 | import org.apache.flink.fs.cos.common.writer.COSAccessHelper; 28 | import org.apache.flink.fs.cos.common.writer.COSRecoverableWriter; 29 | import org.apache.flink.runtime.fs.hdfs.HadoopFileSystem; 30 | import org.apache.flink.util.Preconditions; 31 | import org.apache.flink.util.function.FunctionWithException; 32 | 33 | import org.apache.hadoop.fs.FileSystem; 34 | 35 | import java.io.File; 36 | import java.io.IOException; 37 | import java.util.concurrent.Executor; 38 | import java.util.concurrent.Executors; 39 | 40 | /** Flink COS FileSystem implementation. */ 41 | public class FlinkCOSFileSystem extends HadoopFileSystem { 42 | 43 | public static final long COS_MULTIPART_UPLOAD_PART_MIN_SIZE = 1 * 1024 * 1024; 44 | 45 | public static final long COS_RECOVER_WAIT_TIME_SECOND = 30; 46 | 47 | private final String localTmpDir; 48 | 49 | private final FunctionWithException tmpFileCreator; 50 | 51 | private final COSAccessHelper cosAccessHelper; 52 | 53 | private final Executor uploadThreadPool; 54 | 55 | private final long cosUploadPartSize; 56 | 57 | private final int maxConcurrentUploadsPerStream; 58 | 59 | private final long timeoutSec; 60 | 61 | private final long initTimestamp; // second 62 | 63 | private final boolean isPosixProcess; 64 | 65 | /** 66 | * Wraps the given Hadoop File System object as a Flink File System object. The given Hadoop 67 | * file system object is expected to be initialized already. 68 | * 69 | * @param hadoopFileSystem The Hadoop FileSystem that will be used under the hood. 70 | */ 71 | public FlinkCOSFileSystem( 72 | FileSystem hadoopFileSystem, 73 | String localTmpDir, 74 | COSAccessHelper cosAccessHelper, 75 | long cosUploadPartSize, 76 | int maxConcurrentUploadsPerStream, 77 | long timeoutSec, 78 | boolean isPosixProcess) { 79 | super(hadoopFileSystem); 80 | this.localTmpDir = Preconditions.checkNotNull(localTmpDir); 81 | this.tmpFileCreator = RefCountedTmpFileCreator.inDirectories(new File(localTmpDir)); 82 | this.cosAccessHelper = cosAccessHelper; 83 | this.uploadThreadPool = Executors.newCachedThreadPool(); 84 | this.cosUploadPartSize = cosUploadPartSize; 85 | this.maxConcurrentUploadsPerStream = maxConcurrentUploadsPerStream; 86 | this.timeoutSec = timeoutSec; 87 | this.initTimestamp = System.currentTimeMillis() / 1000; // second 88 | this.isPosixProcess = isPosixProcess; 89 | } 90 | 91 | public String getLocalTmpDir() { 92 | return localTmpDir; 93 | } 94 | 95 | @Override 96 | public FileSystemKind getKind() { 97 | return FileSystemKind.OBJECT_STORE; 98 | } 99 | 100 | @Override 101 | public RecoverableWriter createRecoverableWriter() throws IOException { 102 | if (null == this.cosAccessHelper) { 103 | throw new UnsupportedOperationException( 104 | "This cos file system implementation does not support recoverable writers."); 105 | } 106 | 107 | if (this.isPosixProcess) { 108 | return new COSPosixRecoverableWriter(getHadoopFileSystem()); 109 | } else { 110 | return COSRecoverableWriter.writer( 111 | getHadoopFileSystem(), 112 | this.tmpFileCreator, 113 | cosAccessHelper, 114 | this.uploadThreadPool, 115 | cosUploadPartSize, 116 | maxConcurrentUploadsPerStream, 117 | this.initTimestamp, 118 | this.timeoutSec); 119 | } 120 | } 121 | 122 | // utilities 123 | public static org.apache.hadoop.fs.Path toHadoopPath(Path path) { 124 | return new org.apache.hadoop.fs.Path(path.toUri()); 125 | } 126 | } 127 | -------------------------------------------------------------------------------- /flink-cos-fs-base/src/main/java/org/apache/flink/fs/cos/common/writer/COSRecoverableMultipartUploadFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.fs.cos.common.writer; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.core.fs.Path; 23 | import org.apache.flink.fs.cos.common.utils.BackPressuringExecutor; 24 | import org.apache.flink.fs.cos.common.utils.RefCountedFile; 25 | import org.apache.flink.runtime.fs.hdfs.HadoopFileSystem; 26 | import org.apache.flink.util.Preconditions; 27 | import org.apache.flink.util.function.FunctionWithException; 28 | 29 | import org.apache.hadoop.fs.FileSystem; 30 | 31 | import java.io.File; 32 | import java.io.IOException; 33 | import java.util.Optional; 34 | import java.util.concurrent.Executor; 35 | 36 | /** COSRecoverableMultipartUploadFactory. */ 37 | @Internal 38 | public final class COSRecoverableMultipartUploadFactory { 39 | private final org.apache.hadoop.fs.FileSystem fs; 40 | 41 | private final COSAccessHelper cosAccessHelper; 42 | 43 | private final FunctionWithException tmpFileSupplier; 44 | 45 | private final int maxConcurrentUploadsPerStream; 46 | 47 | private final Executor executor; 48 | 49 | public COSRecoverableMultipartUploadFactory( 50 | final FileSystem fs, 51 | final COSAccessHelper cosAccessHelper, 52 | final int maxConcurrentUploadsPerStream, 53 | final Executor executor, 54 | final FunctionWithException tmpFileSupplier) { 55 | this.fs = Preconditions.checkNotNull(fs); 56 | this.maxConcurrentUploadsPerStream = maxConcurrentUploadsPerStream; 57 | this.executor = executor; 58 | this.cosAccessHelper = cosAccessHelper; 59 | this.tmpFileSupplier = tmpFileSupplier; 60 | } 61 | 62 | RecoverableMultipartUpload getNewRecoverableUpload(Path path) throws IOException { 63 | 64 | return RecoverableMultipartUploadImpl.newUpload( 65 | this.cosAccessHelper, limitedExecutor(), pathToObjectName(path)); 66 | } 67 | 68 | RecoverableMultipartUpload recoverRecoverableUpload(COSRecoverable recoverable) 69 | throws IOException { 70 | final Optional incompletePart = recoverInProgressPart(recoverable); 71 | 72 | return RecoverableMultipartUploadImpl.recoverUpload( 73 | this.cosAccessHelper, 74 | limitedExecutor(), 75 | recoverable.getUploadId(), 76 | recoverable.getObjectName(), 77 | recoverable.getPartETags(), 78 | recoverable.getNumBytesInParts(), 79 | incompletePart); 80 | } 81 | 82 | private Optional recoverInProgressPart(COSRecoverable recoverable) throws IOException { 83 | 84 | final String objectKey = recoverable.getInCompleteObjectName(); 85 | if (objectKey == null) { 86 | return Optional.empty(); 87 | } 88 | 89 | // download the file (simple way) 90 | final RefCountedFile refCountedFile = tmpFileSupplier.apply(null); 91 | final File file = refCountedFile.getFile(); 92 | final long numBytes = this.cosAccessHelper.getObject(objectKey, file); 93 | 94 | if (numBytes != recoverable.getInCompleteObjectLength()) { 95 | throw new IOException( 96 | String.format( 97 | "Error recovering writer: " 98 | + "Downloading the last data chunk file gives incorrect length." 99 | + "File length is %d bytes, RecoveryData indicates %d bytes", 100 | numBytes, recoverable.getInCompleteObjectLength())); 101 | } 102 | 103 | return Optional.of(file); 104 | } 105 | 106 | private String pathToObjectName(final Path path) { 107 | org.apache.hadoop.fs.Path hadoopPath = HadoopFileSystem.toHadoopPath(path); 108 | if (!hadoopPath.isAbsolute()) { 109 | hadoopPath = new org.apache.hadoop.fs.Path(fs.getWorkingDirectory(), hadoopPath); 110 | } 111 | 112 | return hadoopPath.toUri().getScheme() != null && hadoopPath.toUri().getPath().isEmpty() 113 | ? "" 114 | : hadoopPath.toUri().getPath().substring(1); 115 | } 116 | 117 | private Executor limitedExecutor() { 118 | return maxConcurrentUploadsPerStream <= 0 119 | ? executor 120 | : new BackPressuringExecutor(executor, maxConcurrentUploadsPerStream); 121 | } 122 | } 123 | -------------------------------------------------------------------------------- /flink-cos-fs-base/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | flink-cos-fs 7 | com.qcloud.cos 8 | 1.10.0-0.2.5 9 | 10 | 4.0.0 11 | 12 | flink-cos-fs-base 13 | Flink : FileSystems : COS FS Base 14 | https://cloud.tencent.com/document/product/436 15 | 16 | jar 17 | 18 | 19 | 8 20 | 8 21 | true 22 | 5.6.244.4 23 | 3.5 24 | 8.3.28 25 | 2.8.9 26 | 27 | 28 | 29 | 30 | org.apache.flink 31 | flink-core 32 | ${flink.version} 33 | provided 34 | 35 | 36 | 37 | org.apache.flink 38 | flink-hadoop-fs 39 | ${flink.version} 40 | 41 | 42 | org.apache.flink 43 | flink-core 44 | 45 | 46 | 47 | 48 | 49 | commons-io 50 | commons-io 51 | 52 | 53 | 54 | 55 | com.qcloud.cos 56 | flink-fs-hadoop-shaded 57 | ${project.version} 58 | 59 | 60 | 61 | 62 | com.qcloud 63 | cos_api-bundle 64 | ${fs.cos.sdk.version} 65 | 66 | 67 | 68 | 69 | com.google.code.gson 70 | gson 71 | ${gson.version} 72 | 73 | 74 | 75 | com.qcloud 76 | ofs-sdk-definition 77 | 1.0.4 78 | 79 | 80 | 81 | com.qcloud 82 | chdfs_hadoop_plugin_network 83 | ${fs.ofs.sdk.version} 84 | 85 | 86 | org.apache.hadoop 87 | hadoop-common 88 | 89 | 90 | 91 | 92 | 93 | com.qcloud 94 | cosn-ranger-interface 95 | 1.0.4 96 | 97 | 98 | 99 | com.qcloud 100 | hadoop-ranger-client-for-hadoop 101 | 3.1.0-4.1 102 | 103 | 104 | 105 | 106 | com.qcloud.cos 107 | hadoop-cos 108 | ${fs.hadoopshaded.version}-${fs.cosn.version} 109 | 110 | 111 | org.apache.hadoop 112 | hadoop-common 113 | 114 | 115 | com.qcloud 116 | cos_api-bundle 117 | 118 | 119 | com.qcloud 120 | chdfs_hadoop_plugin_network 121 | 122 | 123 | 124 | 125 | 126 | 127 | 128 | 129 | 130 | org.apache.maven.plugins 131 | maven-deploy-plugin 132 | 133 | true 134 | 135 | 136 | 137 | 138 | 139 | -------------------------------------------------------------------------------- /flink-cos-fs-hadoop/src/main/java/org/apache/flink/fs/coshadoop/COSNFileSystemFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.fs.coshadoop; 20 | 21 | import org.apache.flink.configuration.Configuration; 22 | import org.apache.flink.fs.cos.common.AbstractCOSFileSystemFactory; 23 | import org.apache.flink.fs.cos.common.writer.COSAccessHelper; 24 | 25 | import org.apache.hadoop.fs.CosFileSystem; 26 | import org.apache.hadoop.fs.FileSystem; 27 | import org.apache.hadoop.fs.NativeFileSystemStore; 28 | import org.slf4j.Logger; 29 | import org.slf4j.LoggerFactory; 30 | 31 | import java.net.URI; 32 | import java.util.Collections; 33 | import java.util.Set; 34 | 35 | /** The CosN FileSystem Factory. */ 36 | public class COSNFileSystemFactory extends AbstractCOSFileSystemFactory { 37 | private static final Logger LOG = LoggerFactory.getLogger(COSNFileSystemFactory.class); 38 | 39 | private Configuration flinkConfig; 40 | 41 | private static final Set CONFIG_KEYS_TO_SHADE = 42 | Collections.singleton("fs.cosn.credentials.provider"); 43 | 44 | private static final String FLINK_SHADING_PREFIX = "org.apache.flink.fs.shaded.hadoop3."; 45 | 46 | private static final Set FLINK_TARGET_HADOOP_SHADE_PREFIX = 47 | Collections.singleton("org.apache.hadoop.fs"); 48 | 49 | /** 50 | * In order to simplify, we make flink cos configuration keys same with hadoop cos module. So, 51 | * we add all configuration key with prefix `fs.cosn` in flink conf to hadoop conf 52 | */ 53 | private static final String[] FLINK_CONFIG_PREFIXES = { 54 | "fs.cosn.", "fs.AbstractFileSystem.cosn.", "qcloud.object.storage." 55 | }; 56 | 57 | public COSNFileSystemFactory() { 58 | super("COSN FileSystem"); 59 | } 60 | 61 | @Override 62 | protected FileSystem createHadoopFileSystem() { 63 | return new CosFileSystem(); 64 | } 65 | 66 | @Override 67 | protected URI getInitURI(URI fsUri, org.apache.hadoop.conf.Configuration hadoopConfig) { 68 | final String scheme = fsUri.getScheme(); 69 | final String authority = fsUri.getAuthority(); 70 | 71 | if (scheme == null && authority == null) { 72 | fsUri = org.apache.hadoop.fs.FileSystem.getDefaultUri(hadoopConfig); 73 | } else if (scheme != null && authority == null) { 74 | URI defaultUri = org.apache.hadoop.fs.FileSystem.getDefaultUri(hadoopConfig); 75 | if (scheme.equals(defaultUri.getScheme()) && defaultUri.getAuthority() != null) { 76 | fsUri = defaultUri; 77 | } 78 | } 79 | 80 | return fsUri; 81 | } 82 | 83 | @Override 84 | protected COSAccessHelper getCosAccessHelper(NativeFileSystemStore nativeFileSystemStore) { 85 | return new HadoopCOSAccessHelper(nativeFileSystemStore); 86 | } 87 | 88 | @Override 89 | public void configure(org.apache.flink.configuration.Configuration config) { 90 | super.configure(config); 91 | this.flinkConfig = config; 92 | } 93 | 94 | private static String convertStringToHex(String str) { 95 | StringBuilder stringBuilder = new StringBuilder(); 96 | 97 | char[] charArray = str.toCharArray(); 98 | 99 | for (char c : charArray) { 100 | String charToHex = Integer.toHexString(c); 101 | stringBuilder.append(charToHex); 102 | } 103 | return stringBuilder.toString(); 104 | } 105 | 106 | @Override 107 | protected org.apache.hadoop.conf.Configuration getHadoopConfiguration() { 108 | org.apache.hadoop.conf.Configuration conf = new org.apache.hadoop.conf.Configuration(); 109 | if (flinkConfig == null) { 110 | return conf; 111 | } 112 | 113 | // read all configuration with prefix 'FLINK_CONFIG_PREFIXES' 114 | for (String key : flinkConfig.keySet()) { 115 | for (String prefix : FLINK_CONFIG_PREFIXES) { 116 | if (key.startsWith(prefix)) { 117 | String value = flinkConfig.getString(key, null); 118 | conf.set(key, value); 119 | if (value.startsWith(Constants.SHADING_VALUE_PREFIX)) { 120 | String shadeValue = FLINK_SHADING_PREFIX + value; 121 | conf.set(key, shadeValue); 122 | } 123 | if (!key.startsWith("fs.cosn.userinfo")) { 124 | LOG.info("Adding Flink config entry for {} as {} to Hadoop config", key, conf.get(key)); 125 | } 126 | } 127 | } 128 | } 129 | return conf; 130 | } 131 | 132 | @Override 133 | public String getScheme() { 134 | return "cosn"; 135 | } 136 | } 137 | -------------------------------------------------------------------------------- /flink-cos-fs-base/src/main/java/org/apache/flink/fs/cos/common/writer/COSCommitter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.fs.cos.common.writer; 20 | 21 | import org.apache.flink.core.fs.RecoverableFsDataOutputStream; 22 | import org.apache.flink.core.fs.RecoverableWriter; 23 | 24 | import com.qcloud.cos.model.PartETag; 25 | import org.apache.hadoop.fs.FileMetadata; 26 | import org.slf4j.Logger; 27 | import org.slf4j.LoggerFactory; 28 | 29 | import java.io.FileNotFoundException; 30 | import java.io.IOException; 31 | import java.util.List; 32 | import java.util.concurrent.atomic.AtomicInteger; 33 | 34 | /** The COSCommitter for implementing the exactly-once. */ 35 | public class COSCommitter implements RecoverableFsDataOutputStream.Committer { 36 | private static final Logger LOG = LoggerFactory.getLogger(COSCommitter.class); 37 | 38 | private final COSAccessHelper cosAccessHelper; 39 | private final String uploadId; 40 | private final String objectName; 41 | private final List partETags; 42 | 43 | private final long totalLength; 44 | 45 | public COSCommitter( 46 | COSAccessHelper cosAccessHelper, 47 | String uploadId, 48 | String objectName, 49 | List partETags, 50 | long totalLength) { 51 | this.cosAccessHelper = cosAccessHelper; 52 | this.uploadId = uploadId; 53 | this.objectName = objectName; 54 | this.partETags = partETags; 55 | this.totalLength = totalLength; 56 | } 57 | 58 | @Override 59 | public void commit() throws IOException { 60 | if (totalLength > 0L) { 61 | LOG.info("Committing {} with MultipartUpload ID: {}.", this.objectName, this.uploadId); 62 | 63 | final AtomicInteger errorCount = new AtomicInteger(); 64 | this.cosAccessHelper.commitMultipartUpload( 65 | this.objectName, this.uploadId, this.partETags); 66 | } else { 67 | LOG.debug("No data to commit for the file: {}.", this.objectName); 68 | } 69 | } 70 | 71 | @Override 72 | public void commitAfterRecovery() throws IOException { 73 | if (this.totalLength > 0L) { 74 | LOG.info( 75 | "Trying to commit after recovery {} with the MultipartUpload ID: {}.", 76 | this.objectName, 77 | this.uploadId); 78 | 79 | try { 80 | this.cosAccessHelper.commitMultipartUpload( 81 | this.objectName, this.uploadId, this.partETags); 82 | } catch (IOException e) { 83 | LOG.info( 84 | "Failed to commit after recovery {} with " 85 | + "the MultipartUpload ID: {}. Checking if the file was committed before...", 86 | this.objectName, 87 | this.uploadId); 88 | LOG.trace("Exception when committing.", e); 89 | 90 | try { 91 | FileMetadata fileMetadata = 92 | this.cosAccessHelper.getObjectMetadata(this.objectName); 93 | 94 | if (this.totalLength != fileMetadata.getLength()) { 95 | String message = 96 | String.format( 97 | "Inconsistent result for object %s: conflicting lengths. " 98 | + "Recovered committer for upload %s indicates %s bytes, present object is %s bytes", 99 | objectName, 100 | uploadId, 101 | totalLength, 102 | fileMetadata.getLength()); 103 | LOG.warn(message); 104 | throw new IOException(message, e); 105 | } 106 | } catch (FileNotFoundException fileNotFoundException) { 107 | LOG.warn( 108 | "Object {} not existing after failed recovery commit with MPU ID {}", 109 | this.objectName, 110 | this.uploadId); 111 | throw new IOException( 112 | String.format( 113 | "Recovering commit failed for object %s. " 114 | + "Object does not exist and MultiPart Upload %s is not valid.", 115 | this.objectName, this.uploadId), 116 | e); 117 | } 118 | } 119 | } else { 120 | LOG.debug("No data to commit for file: {}.", this.objectName); 121 | } 122 | } 123 | 124 | @Override 125 | public RecoverableWriter.CommitRecoverable getRecoverable() { 126 | return new COSRecoverable(this.uploadId, this.objectName, this.partETags, this.totalLength); 127 | } 128 | } 129 | -------------------------------------------------------------------------------- /pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 4.0.0 6 | pom 7 | 8 | 9 | flink-filesystems 10 | org.apache.flink 11 | 1.10.0 12 | 13 | 14 | com.qcloud.cos 15 | flink-cos-fs 16 | 1.10.0-0.2.5 17 | Flink : FileSystems : COS FS 18 | 19 | This project aims to provide COS's implementation of the Flink file system interface, 20 | so that Flink can not only use the ordinary read and write interfaces to read and write COS, but also use StreamingFileSink, 21 | an interface that requires exactly-once semantics to write to COS. 22 | 23 | 24 | 25 | 26 | Tencent Cloud with MIT License 27 | https://github.com/tencentyun/flink-cos-fs/blob/master/LICENSE 28 | 29 | 30 | 31 | 32 | 33 | iainyu 34 | Yang Yu 35 | iainyu@tencent.com 36 | Tencent Cloud 37 | https://cloud.tencent.com/ 38 | https://github.com/yuyang733 39 | 40 | 41 | alantong 42 | alantong@tencent.com 43 | 44 | 45 | 46 | 47 | scm:git:https://github.com/tencentyun/flink-cos-fs.git 48 | scm:git:https://github.com/tencentyun/flink-cos-fs.git 49 | https://github.com/tencentyun/flink-cos-fs 50 | 51 | 52 | 53 | 54 | oss 55 | flink-cos-fs 56 | https://oss.sonatype.org/service/local/staging/deploy/maven2 57 | 58 | 59 | 60 | oss 61 | flink-cos-fs 62 | https://oss.sonatype.org/content/repositories/snapshots 63 | 64 | 65 | 66 | 67 | 8 68 | 8 69 | 1.10.0 70 | 3.2.2 71 | 72 | 73 | 74 | flink-cos-fs-base 75 | flink-cos-fs-hadoop 76 | flink-fs-hadoop-shaded 77 | flink-cos-fs-examples 78 | 79 | 80 | 81 | 82 | 83 | org.apache.flink 84 | flink-test-utils-junit 85 | ${flink.version} 86 | 87 | 88 | 89 | 90 | 91 | 92 | org.apache.maven.plugins 93 | maven-source-plugin 94 | 95 | true 96 | 97 | 98 | 99 | compile 100 | 101 | jar 102 | 103 | 104 | 105 | 106 | 107 | 108 | org.apache.maven.plugins 109 | maven-javadoc-plugin 110 | 111 | -Xdoclint:none 112 | 113 | 114 | 115 | javadoc-jar 116 | package 117 | 118 | jar 119 | 120 | 121 | 122 | 123 | 124 | 125 | org.apache.maven.plugins 126 | maven-deploy-plugin 127 | 128 | 129 | deploy 130 | deploy 131 | 132 | deploy 133 | 134 | 135 | 136 | 137 | 138 | 139 | org.apache.maven.plugins 140 | maven-gpg-plugin 141 | 142 | 143 | sign-artifacts 144 | verify 145 | 146 | sign 147 | 148 | 149 | 150 | 151 | 152 | 153 | -------------------------------------------------------------------------------- /flink-cos-fs-base/src/main/java/org/apache/flink/fs/cos/common/writer/COSRecoverableSerializer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.fs.cos.common.writer; 20 | 21 | import org.apache.flink.core.io.SimpleVersionedSerializer; 22 | 23 | import com.qcloud.cos.model.PartETag; 24 | 25 | import java.io.IOException; 26 | import java.nio.ByteBuffer; 27 | import java.nio.ByteOrder; 28 | import java.nio.charset.Charset; 29 | import java.nio.charset.StandardCharsets; 30 | import java.util.ArrayList; 31 | import java.util.List; 32 | 33 | /** COSRecoverableSerializer. */ 34 | public class COSRecoverableSerializer implements SimpleVersionedSerializer { 35 | static final COSRecoverableSerializer INSTANCE = new COSRecoverableSerializer(); 36 | 37 | private static final Charset CHARSET = StandardCharsets.UTF_8; 38 | 39 | private static final int MAGIC_NUMBER = 0x98761432; 40 | 41 | public COSRecoverableSerializer() {} 42 | 43 | @Override 44 | public int getVersion() { 45 | return 1; 46 | } 47 | 48 | @Override 49 | public byte[] serialize(COSRecoverable obj) throws IOException { 50 | final List partList = obj.getPartETags(); 51 | final PartETag[] parts = partList.toArray(new PartETag[partList.size()]); 52 | 53 | final byte[] keyBytes = obj.getObjectName().getBytes(CHARSET); 54 | final byte[] uploadIdBytes = obj.getUploadId().getBytes(CHARSET); 55 | 56 | final byte[][] etags = new byte[parts.length][]; 57 | int partEtagBytes = 0; 58 | for (int i = 0; i < parts.length; i++) { 59 | etags[i] = parts[i].getETag().getBytes(CHARSET); 60 | partEtagBytes += etags[i].length + 2 * Integer.BYTES; 61 | } 62 | 63 | final String lastObjectKey = obj.getInCompleteObjectName(); 64 | final byte[] lastPartBytes = lastObjectKey == null ? null : lastObjectKey.getBytes(CHARSET); 65 | 66 | final byte[] targetBytes = 67 | new byte 68 | [Integer.BYTES 69 | + // magic number 70 | Integer.BYTES 71 | + keyBytes.length 72 | + Integer.BYTES 73 | + uploadIdBytes.length 74 | + Integer.BYTES 75 | + partEtagBytes 76 | + Long.BYTES 77 | + Integer.BYTES 78 | + (lastPartBytes == null ? 0 : lastPartBytes.length) 79 | + Long.BYTES]; 80 | 81 | ByteBuffer bb = ByteBuffer.wrap(targetBytes).order(ByteOrder.LITTLE_ENDIAN); 82 | bb.putInt(MAGIC_NUMBER); 83 | 84 | bb.putInt(keyBytes.length); 85 | bb.put(keyBytes); 86 | 87 | bb.putInt(uploadIdBytes.length); 88 | bb.put(uploadIdBytes); 89 | 90 | bb.putInt(etags.length); 91 | for (int i = 0; i < parts.length; i++) { 92 | PartETag pe = parts[i]; 93 | bb.putInt(pe.getPartNumber()); 94 | bb.putInt(etags[i].length); 95 | bb.put(etags[i]); 96 | } 97 | 98 | bb.putLong(obj.getNumBytesInParts()); 99 | 100 | if (lastPartBytes == null) { 101 | bb.putInt(0); 102 | } else { 103 | bb.putInt(lastPartBytes.length); 104 | bb.put(lastPartBytes); 105 | } 106 | 107 | bb.putLong(obj.getInCompleteObjectLength()); 108 | 109 | return targetBytes; 110 | } 111 | 112 | @Override 113 | public COSRecoverable deserialize(int version, byte[] serialized) throws IOException { 114 | switch (version) { 115 | case 1: 116 | return deserializeV1(serialized); 117 | default: 118 | throw new IOException("Unrecognized version or corrupt state: " + version); 119 | } 120 | } 121 | 122 | private static COSRecoverable deserializeV1(byte[] serialized) throws IOException { 123 | final ByteBuffer bb = ByteBuffer.wrap(serialized).order(ByteOrder.LITTLE_ENDIAN); 124 | 125 | if (bb.getInt() != MAGIC_NUMBER) { 126 | throw new IOException("Corrupt data: Unexpected magic number."); 127 | } 128 | 129 | final byte[] keyBytes = new byte[bb.getInt()]; 130 | bb.get(keyBytes); 131 | 132 | final byte[] uploadIdBytes = new byte[bb.getInt()]; 133 | bb.get(uploadIdBytes); 134 | 135 | final int numParts = bb.getInt(); 136 | final ArrayList parts = new ArrayList<>(numParts); 137 | for (int i = 0; i < numParts; i++) { 138 | final int partNum = bb.getInt(); 139 | final byte[] buffer = new byte[bb.getInt()]; 140 | bb.get(buffer); 141 | parts.add(new PartETag(partNum, new String(buffer, CHARSET))); 142 | } 143 | 144 | final long numBytes = bb.getLong(); 145 | 146 | final String lastPart; 147 | final int lastObjectArraySize = bb.getInt(); 148 | if (lastObjectArraySize == 0) { 149 | lastPart = null; 150 | } else { 151 | byte[] lastPartBytes = new byte[lastObjectArraySize]; 152 | bb.get(lastPartBytes); 153 | lastPart = new String(lastPartBytes, CHARSET); 154 | } 155 | 156 | final long lastPartLength = bb.getLong(); 157 | 158 | return new COSRecoverable( 159 | new String(uploadIdBytes, CHARSET), 160 | new String(keyBytes, CHARSET), 161 | parts, 162 | numBytes, 163 | lastPart, 164 | lastPartLength); 165 | } 166 | } 167 | -------------------------------------------------------------------------------- /flink-cos-fs-base/src/main/java/org/apache/flink/fs/cos/common/utils/RefCountedBufferingFileStream.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.fs.cos.common.utils; 20 | 21 | import org.apache.flink.annotation.VisibleForTesting; 22 | import org.apache.flink.util.function.FunctionWithException; 23 | 24 | import org.slf4j.Logger; 25 | import org.slf4j.LoggerFactory; 26 | 27 | import java.io.File; 28 | import java.io.IOException; 29 | 30 | import static org.apache.flink.util.Preconditions.checkArgument; 31 | import static org.apache.flink.util.Preconditions.checkNotNull; 32 | 33 | /** RefCountedBufferingFileStream. */ 34 | public class RefCountedBufferingFileStream extends RefCountedFSOutputStream { 35 | private static final Logger LOG = LoggerFactory.getLogger(RefCountedBufferingFileStream.class); 36 | 37 | public static final int BUFFER_SIZE = 4096; 38 | 39 | private final RefCountedFile currentTmpFile; 40 | 41 | /** The write buffer. */ 42 | private final byte[] buffer; 43 | 44 | /** Current position in the buffer, must be in [0, buffer.length]. */ 45 | private int positionInBuffer; 46 | 47 | private boolean closed; 48 | 49 | @VisibleForTesting 50 | public RefCountedBufferingFileStream(final RefCountedFile file, final int bufferSize) { 51 | 52 | checkArgument(bufferSize > 0L); 53 | 54 | this.currentTmpFile = checkNotNull(file); 55 | this.buffer = new byte[bufferSize]; 56 | this.positionInBuffer = 0; 57 | this.closed = false; 58 | } 59 | 60 | @Override 61 | public File getInputFile() { 62 | return currentTmpFile.getFile(); 63 | } 64 | 65 | @Override 66 | public long getPos() { 67 | return currentTmpFile.getLength() + positionInBuffer; 68 | } 69 | 70 | @Override 71 | public void write(int b) throws IOException { 72 | if (positionInBuffer >= buffer.length) { 73 | flush(); 74 | } 75 | 76 | requireOpen(); 77 | 78 | buffer[positionInBuffer++] = (byte) b; 79 | } 80 | 81 | @Override 82 | public void write(byte[] b, int off, int len) throws IOException { 83 | if (len >= buffer.length) { 84 | // circumvent the internal buffer for large writes 85 | flush(); 86 | currentTmpFile.write(b, off, len); 87 | return; 88 | } 89 | 90 | requireOpen(); 91 | 92 | if (len > buffer.length - positionInBuffer) { 93 | flush(); 94 | } 95 | 96 | System.arraycopy(b, off, buffer, positionInBuffer, len); 97 | positionInBuffer += len; 98 | } 99 | 100 | @Override 101 | public void flush() throws IOException { 102 | LOG.info("begin to flush the buffer to the file: {}.", this.currentTmpFile); 103 | currentTmpFile.write(buffer, 0, positionInBuffer); 104 | currentTmpFile.flush(); 105 | positionInBuffer = 0; 106 | LOG.info("finish to flush the buffer to the file: {}.", this.currentTmpFile); 107 | } 108 | 109 | @Override 110 | public void sync() throws IOException { 111 | throw new UnsupportedOperationException( 112 | "COSRecoverableFsDataOutputStream cannot sync state to COS. " 113 | + "Use persist() to create a persistent recoverable intermediate point."); 114 | } 115 | 116 | @Override 117 | public boolean isClosed() throws IOException { 118 | return closed; 119 | } 120 | 121 | @Override 122 | public void close() { 123 | if (!closed) { 124 | LOG.info("begin to close the file: {}. ", this.currentTmpFile); 125 | currentTmpFile.closeStream(); 126 | closed = true; 127 | LOG.info("end to close the file: {}.", this.currentTmpFile); 128 | } 129 | } 130 | 131 | @Override 132 | public void retain() { 133 | currentTmpFile.retain(); 134 | } 135 | 136 | @Override 137 | public boolean release() { 138 | return currentTmpFile.release(); 139 | } 140 | 141 | private void requireOpen() throws IOException { 142 | if (closed) { 143 | throw new IOException("Stream closed."); 144 | } 145 | } 146 | 147 | @Override 148 | public String toString() { 149 | return "Reference Counted File with {" 150 | + "path=\'" 151 | + currentTmpFile.getFile().toPath().toAbsolutePath() 152 | + "\'" 153 | + ", size=" 154 | + getPos() 155 | + ", reference counter=" 156 | + currentTmpFile.getReferenceCounter() 157 | + ", closed=" 158 | + closed 159 | + '}'; 160 | } 161 | 162 | @VisibleForTesting 163 | int getPositionInBuffer() { 164 | return positionInBuffer; 165 | } 166 | 167 | @VisibleForTesting 168 | public int getReferenceCounter() { 169 | return currentTmpFile.getReferenceCounter(); 170 | } 171 | 172 | // ------------------------- Factory Methods ------------------------- 173 | 174 | public static RefCountedBufferingFileStream openNew( 175 | final FunctionWithException tmpFileProvider) 176 | throws IOException { 177 | 178 | return new RefCountedBufferingFileStream(tmpFileProvider.apply(null), BUFFER_SIZE); 179 | } 180 | 181 | public static RefCountedBufferingFileStream restore( 182 | final FunctionWithException tmpFileProvider, 183 | final File initialTmpFile) 184 | throws IOException { 185 | 186 | return new RefCountedBufferingFileStream( 187 | tmpFileProvider.apply(initialTmpFile), BUFFER_SIZE); 188 | } 189 | } 190 | -------------------------------------------------------------------------------- /flink-cos-fs-hadoop/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | flink-cos-fs 7 | com.qcloud.cos 8 | 1.10.0-0.2.5 9 | 10 | 4.0.0 11 | 12 | flink-cos-fs-hadoop 13 | Flink : FileSystems : COS FS Hadoop 14 | 15 | 16 | 8 17 | 8 18 | 19 | org.apache.flink.fs.shaded.hadoop3 20 | org.apache.flink.fs.coshadoop 21 | false 22 | 23 | 24 | 25 | 26 | org.apache.flink 27 | flink-core 28 | ${flink.version} 29 | provided 30 | 31 | 32 | 33 | com.qcloud.cos 34 | flink-cos-fs-base 35 | ${project.version} 36 | 37 | 38 | 39 | 40 | 41 | 42 | org.apache.maven.plugins 43 | maven-shade-plugin 44 | 45 | 3.2.4 46 | 47 | 48 | shade-flink 49 | package 50 | 51 | shade 52 | 53 | 54 | false 55 | 56 | 57 | *:* 58 | 59 | 60 | 61 | 72 | 73 | org.apache.hadoop 74 | ${hadoop.shading.prefix}.org.apache.hadoop 75 | 76 | org.apache.hadoop.fs.cosn.ranger.protocol.* 77 | org/apache/flink/fs/coshadoop/Constants.class 78 | 79 | 80 | 81 | 82 | 88 | 89 | 90 | org.apache.commons 91 | ${hadoop.shading.prefix}.org.apache.commons 92 | 93 | 94 | com.google.common 95 | ${hadoop.shading.prefix}.com.google.common 96 | 97 | 98 | 99 | com.qcloud 100 | ${cos.shading.prefix}.shaded.com.qcloud 101 | 102 | 103 | com.qcloud.chdfs.** 104 | 105 | 106 | 107 | org.apache.flink.runtime.fs.hdfs 108 | ${cos.shading.prefix}.common 109 | 110 | 111 | 112 | org.apache.flink.runtime.util 113 | ${cos.shading.prefix}.common 114 | 115 | 116 | 117 | 118 | 119 | 120 | 121 | 122 | -------------------------------------------------------------------------------- /flink-cos-fs-base/src/main/java/org/apache/flink/fs/cos/common/fswriter/COSPosixRecoverableWriter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.fs.cos.common.fswriter; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.annotation.VisibleForTesting; 23 | import org.apache.flink.core.fs.Path; 24 | import org.apache.flink.core.fs.RecoverableFsDataOutputStream; 25 | import org.apache.flink.core.fs.RecoverableWriter; 26 | import org.apache.flink.core.io.SimpleVersionedSerializer; 27 | import org.apache.flink.fs.cos.common.FlinkCOSFileSystem; 28 | 29 | import org.slf4j.Logger; 30 | import org.slf4j.LoggerFactory; 31 | 32 | import java.io.IOException; 33 | import java.util.UUID; 34 | 35 | import static org.apache.flink.util.Preconditions.checkArgument; 36 | import static org.apache.flink.util.Preconditions.checkNotNull; 37 | 38 | /** 39 | * An implementation of the {@link RecoverableWriter} for 40 | * Hadoop's file system abstraction. 41 | */ 42 | @Internal 43 | public class COSPosixRecoverableWriter implements RecoverableWriter { 44 | 45 | private static final Logger LOG = LoggerFactory.getLogger(COSPosixRecoverableWriter.class); 46 | 47 | /** The Hadoop file system on which the writer operates. */ 48 | private final org.apache.hadoop.fs.FileSystem fs; 49 | 50 | /** 51 | * Creates a new Recoverable writer. 52 | * @param fs The Hadoop file system on which the writer operates. 53 | */ 54 | public COSPosixRecoverableWriter(org.apache.hadoop.fs.FileSystem fs) { 55 | LOG.debug("create the cos merge recoverable writer"); 56 | // todo other version check 57 | this.fs = checkNotNull(fs); 58 | } 59 | 60 | @Override 61 | public RecoverableFsDataOutputStream open(Path filePath) throws IOException { 62 | LOG.debug("cos merge recoverable writer to open"); 63 | final org.apache.hadoop.fs.Path targetFile = FlinkCOSFileSystem.toHadoopPath(filePath); 64 | final org.apache.hadoop.fs.Path tempFile = generateStagingTempFilePath(fs, targetFile); 65 | return new COSPosixRecoverableFsDataOutputStream(fs, targetFile, tempFile); 66 | } 67 | 68 | @Override 69 | public RecoverableFsDataOutputStream recover(ResumeRecoverable recoverable) throws IOException { 70 | // for (StackTraceElement ste : Thread.currentThread().getStackTrace()) { 71 | // LOG.info("cos merge recover detail call:{}", ste.toString()); 72 | // } 73 | if (recoverable instanceof COSPosixRecoverable) { 74 | LOG.debug("cos merge recoverable writer to recover"); 75 | return new COSPosixRecoverableFsDataOutputStream(fs, (COSPosixRecoverable) recoverable); 76 | } 77 | else { 78 | throw new IllegalArgumentException( 79 | "cos merge File System cannot recover a recoverable for another file system: " + recoverable); 80 | } 81 | } 82 | 83 | @Override 84 | public boolean requiresCleanupOfRecoverableState() { 85 | return false; 86 | } 87 | 88 | @Override 89 | public boolean cleanupRecoverableState(ResumeRecoverable resumable) throws IOException { 90 | return false; 91 | } 92 | 93 | @Override 94 | public RecoverableFsDataOutputStream.Committer recoverForCommit(CommitRecoverable recoverable) throws IOException { 95 | /* 96 | if (recoverable instanceof COSMergeRecoverable) { 97 | // final COSMergeRecoverable cosMergeRecoverable = cas 98 | final COSMergeRecoverableFsDataOutputStream recovered = (COSMergeRecoverableFsDataOutputStream) recover((COSMergeRecoverable)recoverable); 99 | return recovered.closeForCommit(); 100 | } else { 101 | throw new IllegalArgumentException( 102 | "cos merge File System cannot recover a recoverable for another file system: " + recoverable); 103 | } 104 | */ 105 | if (recoverable instanceof COSPosixRecoverable) { 106 | LOG.debug("cos merge recover for commit"); 107 | return new COSPosixRecoverableFsDataOutputStream.COSMergeCommitter(fs, (COSPosixRecoverable) recoverable); 108 | } 109 | else { 110 | throw new IllegalArgumentException( 111 | "cos merge File System cannot recover a recoverable for another file system: " + recoverable); 112 | } 113 | } 114 | 115 | @Override 116 | public SimpleVersionedSerializer getCommitRecoverableSerializer() { 117 | @SuppressWarnings("unchecked") 118 | SimpleVersionedSerializer typedSerializer = (SimpleVersionedSerializer) 119 | (SimpleVersionedSerializer) COSPosixRecoverableSerializer.INSTANCE; 120 | LOG.debug("cos merge get commit recoverable serializer"); 121 | 122 | return typedSerializer; 123 | } 124 | 125 | @Override 126 | public SimpleVersionedSerializer getResumeRecoverableSerializer() { 127 | @SuppressWarnings("unchecked") 128 | SimpleVersionedSerializer typedSerializer = (SimpleVersionedSerializer) 129 | (SimpleVersionedSerializer) COSPosixRecoverableSerializer.INSTANCE; 130 | LOG.debug("cos merge get resume recoverable serializer"); 131 | 132 | return typedSerializer; 133 | } 134 | 135 | @Override 136 | public boolean supportsResume() { 137 | LOG.debug("cos merge supports resume"); 138 | return true; 139 | } 140 | 141 | @VisibleForTesting 142 | static org.apache.hadoop.fs.Path generateStagingTempFilePath( 143 | org.apache.hadoop.fs.FileSystem fs, 144 | org.apache.hadoop.fs.Path targetFile) throws IOException { 145 | 146 | checkArgument(targetFile.isAbsolute(), "targetFile must be absolute"); 147 | 148 | final org.apache.hadoop.fs.Path parent = targetFile.getParent(); 149 | final String name = targetFile.getName(); 150 | 151 | checkArgument(parent != null, "targetFile must not be the root directory"); 152 | 153 | while (true) { 154 | org.apache.hadoop.fs.Path candidate = new org.apache.hadoop.fs.Path( 155 | parent, "." + name + ".inprogress." + UUID.randomUUID().toString()); 156 | if (!fs.exists(candidate)) { 157 | return candidate; 158 | } 159 | } 160 | } 161 | } 162 | -------------------------------------------------------------------------------- /flink-cos-fs-base/src/main/java/org/apache/flink/fs/cos/common/writer/COSRecoverableWriter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.fs.cos.common.writer; 20 | 21 | import org.apache.flink.core.fs.Path; 22 | import org.apache.flink.core.fs.RecoverableFsDataOutputStream; 23 | import org.apache.flink.core.fs.RecoverableWriter; 24 | import org.apache.flink.core.io.SimpleVersionedSerializer; 25 | import org.apache.flink.fs.cos.common.FlinkCOSFileSystem; 26 | import org.apache.flink.fs.cos.common.utils.RefCountedFile; 27 | import org.apache.flink.util.function.FunctionWithException; 28 | 29 | import org.apache.hadoop.fs.FileSystem; 30 | import org.slf4j.Logger; 31 | import org.slf4j.LoggerFactory; 32 | 33 | import java.io.File; 34 | import java.io.IOException; 35 | import java.util.concurrent.Executor; 36 | import java.util.concurrent.TimeUnit; 37 | 38 | import static org.apache.flink.util.Preconditions.checkArgument; 39 | import static org.apache.flink.util.Preconditions.checkNotNull; 40 | 41 | /** COSRecoverableWriter. */ 42 | public class COSRecoverableWriter implements RecoverableWriter { 43 | private static final Logger LOG = LoggerFactory.getLogger(COSRecoverableWriter.class); 44 | 45 | private final FunctionWithException tempFileCreator; 46 | 47 | private final long userDefinedMinPartSize; 48 | 49 | private final COSAccessHelper cosAccessHelper; 50 | 51 | private final COSRecoverableMultipartUploadFactory uploadFactory; 52 | 53 | public COSRecoverableWriter( 54 | final COSAccessHelper cosAccessHelper, 55 | final COSRecoverableMultipartUploadFactory uploadFactory, 56 | final FunctionWithException tempFileCreator, 57 | long userDefinedMinPartSize) { 58 | this.tempFileCreator = checkNotNull(tempFileCreator); 59 | this.userDefinedMinPartSize = userDefinedMinPartSize; 60 | this.cosAccessHelper = checkNotNull(cosAccessHelper); 61 | this.uploadFactory = checkNotNull(uploadFactory); 62 | } 63 | 64 | @Override 65 | public RecoverableFsDataOutputStream open(Path path) throws IOException { 66 | final RecoverableMultipartUpload upload = uploadFactory.getNewRecoverableUpload(path); 67 | 68 | return COSRecoverableFsDataOutputStream.newStream( 69 | upload, tempFileCreator, userDefinedMinPartSize); 70 | } 71 | 72 | @Override 73 | public RecoverableFsDataOutputStream recover(ResumeRecoverable resumable) throws IOException { 74 | final COSRecoverable cosRecoverable = castToCOSRecoverable(resumable); 75 | final RecoverableMultipartUpload upload = 76 | uploadFactory.recoverRecoverableUpload(cosRecoverable); 77 | return COSRecoverableFsDataOutputStream.recoverStream( 78 | upload, 79 | tempFileCreator, 80 | userDefinedMinPartSize, 81 | cosRecoverable.getNumBytesInParts()); 82 | } 83 | 84 | @Override 85 | public boolean requiresCleanupOfRecoverableState() { 86 | return true; 87 | } 88 | 89 | @Override 90 | public boolean cleanupRecoverableState(ResumeRecoverable resumable) throws IOException { 91 | final COSRecoverable cosRecoverable = castToCOSRecoverable(resumable); 92 | final String smallPartObjectToDelete = cosRecoverable.getInCompleteObjectName(); 93 | return smallPartObjectToDelete != null 94 | && cosAccessHelper.deleteObject(smallPartObjectToDelete); 95 | } 96 | 97 | @Override 98 | public RecoverableFsDataOutputStream.Committer recoverForCommit(CommitRecoverable resumable) 99 | throws IOException { 100 | final COSRecoverable cosRecoverable = castToCOSRecoverable(resumable); 101 | final COSRecoverableFsDataOutputStream recovered = 102 | (COSRecoverableFsDataOutputStream) recover(cosRecoverable); 103 | return recovered.closeForCommit(); 104 | } 105 | 106 | @Override 107 | @SuppressWarnings({"rawtypes", "unchecked"}) 108 | public SimpleVersionedSerializer getCommitRecoverableSerializer() { 109 | return (SimpleVersionedSerializer) COSRecoverableSerializer.INSTANCE; 110 | } 111 | 112 | @Override 113 | @SuppressWarnings({"rawtypes", "unchecked"}) 114 | public SimpleVersionedSerializer getResumeRecoverableSerializer() { 115 | return (SimpleVersionedSerializer) COSRecoverableSerializer.INSTANCE; 116 | } 117 | 118 | @Override 119 | public boolean supportsResume() { 120 | return true; 121 | } 122 | 123 | private static COSRecoverable castToCOSRecoverable(CommitRecoverable recoverable) { 124 | if (recoverable instanceof COSRecoverable) { 125 | return (COSRecoverable) recoverable; 126 | } 127 | throw new IllegalArgumentException( 128 | "COS File System cannot recover recoverable for other file system: " + recoverable); 129 | } 130 | 131 | private static void waitForFinish(long initTimestamp, long timeoutSec) { 132 | long wantedTimeSecond = initTimestamp + timeoutSec; 133 | long currentTimestamp = System.currentTimeMillis() / 1000; 134 | if (currentTimestamp < wantedTimeSecond) { 135 | try { 136 | TimeUnit.SECONDS.sleep(wantedTimeSecond - currentTimestamp); 137 | } catch (InterruptedException e) { 138 | LOG.warn("wait for finish occur the interrupt"); 139 | } 140 | } 141 | } 142 | 143 | public static COSRecoverableWriter writer( 144 | final FileSystem fs, 145 | final FunctionWithException tempFileCreator, 146 | final COSAccessHelper cosAccessHelper, 147 | final Executor uploadThreadPool, 148 | final long userDefinedMinPartSize, 149 | final int maxConcurrentUploadsPerStream, 150 | final long initTimestamp, 151 | final long timeoutSec) { 152 | 153 | checkArgument( 154 | userDefinedMinPartSize >= FlinkCOSFileSystem.COS_MULTIPART_UPLOAD_PART_MIN_SIZE); 155 | LOG.info( 156 | "create the cos recoverable writer, init: {}, timeout: {}", 157 | initTimestamp, 158 | timeoutSec); 159 | // only mpu way need wait for past part finish 160 | waitForFinish(initTimestamp, timeoutSec); 161 | 162 | final COSRecoverableMultipartUploadFactory uploadFactory = 163 | new COSRecoverableMultipartUploadFactory( 164 | fs, 165 | cosAccessHelper, 166 | maxConcurrentUploadsPerStream, 167 | uploadThreadPool, 168 | tempFileCreator); 169 | 170 | return new COSRecoverableWriter( 171 | cosAccessHelper, uploadFactory, tempFileCreator, userDefinedMinPartSize); 172 | } 173 | } 174 | -------------------------------------------------------------------------------- /flink-cos-fs-base/src/main/java/org/apache/flink/fs/cos/common/AbstractCOSFileSystemFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.fs.cos.common; 20 | 21 | import org.apache.flink.configuration.ConfigOption; 22 | import org.apache.flink.configuration.ConfigOptions; 23 | import org.apache.flink.configuration.Configuration; 24 | import org.apache.flink.configuration.ConfigurationUtils; 25 | import org.apache.flink.core.fs.FileSystem; 26 | import org.apache.flink.core.fs.FileSystemFactory; 27 | import org.apache.flink.fs.cos.common.writer.COSAccessHelper; 28 | import org.apache.flink.util.Preconditions; 29 | 30 | import org.apache.hadoop.fs.CosFileSystem; 31 | import org.apache.hadoop.fs.CosNConfigKeys; 32 | import org.apache.hadoop.fs.NativeFileSystemStore; 33 | import org.slf4j.Logger; 34 | import org.slf4j.LoggerFactory; 35 | 36 | import java.io.IOException; 37 | import java.net.URI; 38 | 39 | /** The base class for file system factories that create COS file systems. */ 40 | public abstract class AbstractCOSFileSystemFactory implements FileSystemFactory { 41 | private static final Logger LOG = LoggerFactory.getLogger(AbstractCOSFileSystemFactory.class); 42 | 43 | private static final ConfigOption UPLOAD_PART_MIN_SIZE = 44 | ConfigOptions.key("cos.upload.part.min.size") 45 | .defaultValue(FlinkCOSFileSystem.COS_MULTIPART_UPLOAD_PART_MIN_SIZE) 46 | .withDescription( 47 | "" 48 | + "This option is relevant to the Recoverable Writer and sets the min size of data that " 49 | + "buffered locally before being sent to the COS. This value is limited to the range: 1MB to 5GB."); 50 | 51 | public static final ConfigOption MAX_CONCURRENT_UPLOADS = 52 | ConfigOptions.key("cos.upload.max.concurrent.uploads") 53 | .defaultValue(Runtime.getRuntime().availableProcessors()) 54 | .withDescription( 55 | "This option is relevant to the Recoverable Writer and limits the number of " 56 | + "parts that can be concurrently in-flight. By default, this is set to " 57 | + Runtime.getRuntime().availableProcessors() 58 | + "."); 59 | 60 | private static final ConfigOption RECOVER_WAIT_TIMESEC = 61 | ConfigOptions.key("cos.recover.wait.time.seconds") 62 | .defaultValue(FlinkCOSFileSystem.COS_RECOVER_WAIT_TIME_SECOND) 63 | .withDescription( 64 | "" 65 | + "This option is the second wait after recover to make sure the request before recover finish" 66 | + "cos cgi default 60s break the link, it is better to set it bigger than 60"); 67 | 68 | // The name of the actual file system. 69 | private final String name; 70 | 71 | private Configuration flinkConfiguration; 72 | 73 | public AbstractCOSFileSystemFactory(String name) { 74 | this.name = name; 75 | } 76 | 77 | @Override 78 | public void configure(Configuration config) { 79 | this.flinkConfiguration = config; 80 | } 81 | 82 | @Override 83 | public FileSystem create(URI fsUri) throws IOException { 84 | Configuration flinkConfig = this.flinkConfiguration; 85 | 86 | if (flinkConfig == null) { 87 | LOG.warn( 88 | "Creating S3 FileSystem without configuring the factory. All behavior will be default."); 89 | flinkConfig = new Configuration(); 90 | } 91 | 92 | LOG.info("Creating the COS FileSystem backed by {}.", this.name); 93 | try { 94 | org.apache.hadoop.conf.Configuration hadoopConfiguration = 95 | this.getHadoopConfiguration(); 96 | org.apache.hadoop.fs.FileSystem fs = createHadoopFileSystem(); 97 | URI uri = getInitURI(fsUri, hadoopConfiguration); 98 | String bucket = uri.getHost(); 99 | fs.initialize(uri, hadoopConfiguration); 100 | 101 | final String[] localTempDirectories = 102 | ConfigurationUtils.parseTempDirectories(flinkConfiguration); 103 | Preconditions.checkArgument(localTempDirectories.length > 0); 104 | final String localTempDirectory = localTempDirectories[0]; 105 | final long cosMinPartSize = flinkConfig.getLong(UPLOAD_PART_MIN_SIZE); 106 | final int maxConcurrentUploads = flinkConfig.getInteger(MAX_CONCURRENT_UPLOADS); 107 | final long timeoutSec = flinkConfig.getLong(RECOVER_WAIT_TIMESEC); 108 | final COSAccessHelper cosAccessHelper = 109 | getCosAccessHelper(((CosFileSystem) fs).getStore()); 110 | 111 | // after hadoop cos fix the setting change to get flag from the cos access helper, avoid head bucket twice. 112 | // boolean isPosixBucket = cosAccessHelper.isPosixBucket(); 113 | boolean isPosixBucket = ((CosFileSystem) fs).getStore().headBucket(bucket).isMergeBucket(); 114 | boolean isPosixProcess = false; 115 | 116 | // according to the head bucket result and implement config to judge which writer to use. 117 | String bucketImpl = ""; 118 | if (isPosixBucket) { 119 | bucketImpl = hadoopConfiguration.get(CosNConfigKeys.COSN_POSIX_BUCKET_FS_IMPL); 120 | if (null != bucketImpl) { 121 | if (bucketImpl.equals(CosNConfigKeys.DEFAULT_COSN_POSIX_BUCKET_FS_IMPL)) { 122 | isPosixProcess = true; 123 | } 124 | } else { 125 | // default use the posix way to query posix bucket; 126 | isPosixProcess = true; 127 | } 128 | } 129 | 130 | LOG.info("Creating the Flink cos file system, " + 131 | "create posix process recover writer: {}, " + 132 | "bucket {} is posix bucket: {}, bucket impl {}.", 133 | isPosixProcess, bucket, isPosixBucket, bucketImpl); 134 | 135 | return new FlinkCOSFileSystem( 136 | fs, 137 | localTempDirectory, 138 | cosAccessHelper, 139 | cosMinPartSize, 140 | maxConcurrentUploads, 141 | timeoutSec, 142 | isPosixProcess); 143 | } catch (IOException e) { 144 | throw e; 145 | } catch (Exception e) { 146 | throw new IOException(e.getMessage(), e); 147 | } 148 | } 149 | 150 | protected abstract org.apache.hadoop.fs.FileSystem createHadoopFileSystem(); 151 | 152 | protected abstract URI getInitURI(URI fsUri, org.apache.hadoop.conf.Configuration hadoopConfig); 153 | 154 | protected abstract COSAccessHelper getCosAccessHelper( 155 | NativeFileSystemStore nativeFileSystemStore); 156 | 157 | protected abstract org.apache.hadoop.conf.Configuration getHadoopConfiguration(); 158 | } 159 | -------------------------------------------------------------------------------- /flink-cos-fs-base/src/main/java/org/apache/flink/fs/cos/common/writer/COSRecoverableFsDataOutputStream.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.fs.cos.common.writer; 20 | 21 | import org.apache.flink.annotation.PublicEvolving; 22 | import org.apache.flink.core.fs.RecoverableFsDataOutputStream; 23 | import org.apache.flink.core.fs.RecoverableWriter; 24 | import org.apache.flink.fs.cos.common.FlinkCOSFileSystem; 25 | import org.apache.flink.fs.cos.common.utils.RefCountedBufferingFileStream; 26 | import org.apache.flink.fs.cos.common.utils.RefCountedFSOutputStream; 27 | import org.apache.flink.fs.cos.common.utils.RefCountedFile; 28 | import org.apache.flink.util.function.FunctionWithException; 29 | 30 | import org.apache.commons.io.IOUtils; 31 | 32 | import javax.annotation.concurrent.NotThreadSafe; 33 | 34 | import java.io.File; 35 | import java.io.IOException; 36 | import java.util.Optional; 37 | import java.util.concurrent.locks.ReentrantLock; 38 | 39 | import static org.apache.flink.util.Preconditions.checkArgument; 40 | 41 | /** COSRecoverableFsDataOutputStream. */ 42 | @PublicEvolving 43 | @NotThreadSafe 44 | public class COSRecoverableFsDataOutputStream extends RecoverableFsDataOutputStream { 45 | private final ReentrantLock lock = new ReentrantLock(); 46 | 47 | private final RecoverableMultipartUpload upload; 48 | 49 | private final FunctionWithException tmpFileProvider; 50 | 51 | private final long userDefinedMinPartSize; 52 | 53 | private RefCountedFSOutputStream fileStream; 54 | 55 | private long bytesBeforeCurrentPart; 56 | 57 | public COSRecoverableFsDataOutputStream( 58 | RecoverableMultipartUpload upload, 59 | FunctionWithException tmpFileProvider, 60 | RefCountedFSOutputStream initialTmpFile, 61 | long userDefinedMinPartSize, 62 | long bytesBeforeCurrentPart) { 63 | 64 | checkArgument(bytesBeforeCurrentPart >= 0L); 65 | 66 | this.upload = upload; 67 | this.tmpFileProvider = tmpFileProvider; 68 | this.userDefinedMinPartSize = userDefinedMinPartSize; 69 | this.fileStream = initialTmpFile; 70 | this.bytesBeforeCurrentPart = bytesBeforeCurrentPart; 71 | } 72 | 73 | @Override 74 | public RecoverableWriter.ResumeRecoverable persist() throws IOException { 75 | this.lock(); 76 | try { 77 | this.fileStream.flush(); 78 | openNewPartIfNecessary(userDefinedMinPartSize); 79 | 80 | // We do not stop writing to the current file, we merely limit the upload to the 81 | // first n bytes of the current file 82 | 83 | return upload.snapshotAndGetRecoverable(fileStream); 84 | } finally { 85 | unlock(); 86 | } 87 | } 88 | 89 | @Override 90 | public Committer closeForCommit() throws IOException { 91 | lock(); 92 | try { 93 | closeAndUploadPart(); 94 | return upload.snapshotAndGetCommitter(); 95 | } finally { 96 | unlock(); 97 | } 98 | } 99 | 100 | @Override 101 | public long getPos() throws IOException { 102 | return bytesBeforeCurrentPart + fileStream.getPos(); 103 | } 104 | 105 | @Override 106 | public void write(int b) throws IOException { 107 | fileStream.write(b); 108 | } 109 | 110 | @Override 111 | public void write(byte[] b, int off, int len) throws IOException { 112 | fileStream.write(b, off, len); 113 | openNewPartIfNecessary(userDefinedMinPartSize); 114 | } 115 | 116 | @Override 117 | public void flush() throws IOException { 118 | fileStream.flush(); 119 | openNewPartIfNecessary(userDefinedMinPartSize); 120 | } 121 | 122 | @Override 123 | public void sync() throws IOException { 124 | fileStream.sync(); 125 | } 126 | 127 | @Override 128 | public void close() throws IOException { 129 | lock(); 130 | try { 131 | fileStream.flush(); 132 | } finally { 133 | IOUtils.closeQuietly(fileStream); 134 | fileStream.release(); 135 | unlock(); 136 | } 137 | } 138 | 139 | private void openNewPartIfNecessary(long sizeThreshold) throws IOException { 140 | final long fileLength = fileStream.getPos(); 141 | if (fileLength >= sizeThreshold) { 142 | lock(); 143 | try { 144 | uploadCurrentAndOpenNewPart(fileLength); 145 | } finally { 146 | unlock(); 147 | } 148 | } 149 | } 150 | 151 | private void uploadCurrentAndOpenNewPart(long fileLength) throws IOException { 152 | bytesBeforeCurrentPart += fileLength; 153 | closeAndUploadPart(); 154 | 155 | // initialize a new temp file 156 | fileStream = RefCountedBufferingFileStream.openNew(tmpFileProvider); 157 | } 158 | 159 | private void closeAndUploadPart() throws IOException { 160 | fileStream.flush(); 161 | fileStream.close(); 162 | if (fileStream.getPos() > 0L) { 163 | upload.uploadPart(fileStream); 164 | } 165 | fileStream.release(); 166 | } 167 | 168 | private void lock() throws IOException { 169 | try { 170 | lock.lockInterruptibly(); 171 | } catch (InterruptedException e) { 172 | Thread.currentThread().interrupt(); 173 | throw new IOException("interrupted"); 174 | } 175 | } 176 | 177 | private void unlock() { 178 | lock.unlock(); 179 | } 180 | 181 | public static COSRecoverableFsDataOutputStream newStream( 182 | final RecoverableMultipartUpload upload, 183 | final FunctionWithException tmpFileCreator, 184 | final long userDefinedMinPartSize) 185 | throws IOException { 186 | 187 | checkArgument( 188 | userDefinedMinPartSize >= FlinkCOSFileSystem.COS_MULTIPART_UPLOAD_PART_MIN_SIZE); 189 | 190 | final RefCountedBufferingFileStream fileStream = 191 | boundedBufferingFileStream(tmpFileCreator, Optional.empty()); 192 | 193 | return new COSRecoverableFsDataOutputStream( 194 | upload, tmpFileCreator, fileStream, userDefinedMinPartSize, 0L); 195 | } 196 | 197 | public static COSRecoverableFsDataOutputStream recoverStream( 198 | final RecoverableMultipartUpload upload, 199 | final FunctionWithException tmpFileCreator, 200 | final long userDefinedMinPartSize, 201 | final long bytesBeforeCurrentPart) 202 | throws IOException { 203 | 204 | checkArgument( 205 | userDefinedMinPartSize >= FlinkCOSFileSystem.COS_MULTIPART_UPLOAD_PART_MIN_SIZE); 206 | 207 | final RefCountedBufferingFileStream fileStream = 208 | boundedBufferingFileStream(tmpFileCreator, upload.getIncompletePart()); 209 | 210 | return new COSRecoverableFsDataOutputStream( 211 | upload, tmpFileCreator, fileStream, userDefinedMinPartSize, bytesBeforeCurrentPart); 212 | } 213 | 214 | private static RefCountedBufferingFileStream boundedBufferingFileStream( 215 | final FunctionWithException tmpFileCreator, 216 | final Optional incompletePart) 217 | throws IOException { 218 | 219 | if (!incompletePart.isPresent()) { 220 | return RefCountedBufferingFileStream.openNew(tmpFileCreator); 221 | } 222 | 223 | final File file = incompletePart.get(); 224 | return RefCountedBufferingFileStream.restore(tmpFileCreator, file); 225 | } 226 | } 227 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # Flink-cos-fs 2 | 3 | Flink-cos-fs 是腾讯云对象存储系统COS针对 Flink 的文件系统实现,并且支持了 recoverwriter 接口。 Flink 可以基于该文件系统实现读写 COS 上的数据以及作为流应用的状态后端。 4 | 5 | ## 使用环境 6 | 7 | ### 系统环境 8 | 9 | 支持 Linux 和 Mac OS 系统 10 | 11 | ### 软件依赖 12 | 13 | Flink 1.10 14 | 15 | 16 | ## 使用方法 17 | 18 | ### 编译或获取 Flink-cos-fs 发行包 19 | 20 | #### 编译 flink-cos-fs 21 | 22 | flink-cos-fs 默认依赖的是 hadoop-3.1.0 的版本(即是于 flink-1.10 保持一致),使用如下命令即可编译打包: 23 | 24 | ```bash 25 | mvn clean package -DskipTests 26 | ``` 27 | 28 | 如果需要编译依赖 hadoop 其他版本的发行包,则需要手动修改项目根路径 pom.xml 中的 `${fs.hadoopshaded.version}` 或者在编译命令中指定 `-Dfs.hadoopshaded.version=3.x.x`,同时修改 flink-fs-hadoop-shaded 模块下的 org.apache.hadoop.util.VersionInfo.java 文件中的 hadoop 版本信息: 29 | 30 | ```java 31 | // ... 32 | if ("common".equals(component)) { 33 | info.setProperty("version", "3.1.0"); 34 | info.setProperty("revision", "16b70619a24cdcf5d3b0fcf4b58ca77238ccbe6d"); 35 | info.setProperty("branch", "branch-3.1.0"); 36 | info.setProperty("user", "wtan"); 37 | info.setProperty("date", "2018-04-03T04:00Z"); 38 | info.setProperty("url", "git@github.com:hortonworks/hadoop-common-trunk.git"); 39 | info.setProperty("srcChecksum", "14182d20c972b3e2105580a1ad6990"); 40 | info.setProperty("protocVersion", "2.5.0"); 41 | } 42 | // ... 43 | ``` 44 | 45 | 特别地,如果需要编译 hadoop 2.x 的版本,除了上述操作以外,还需要在编译命令中,指定 maven 编译使用 `hadoop-2` 的配置: 46 | ```bash 47 | mvn clean package -DskipTests -Phadoop-2 -Dfs.hadoopshaded.version=2.x.x 48 | ``` 49 | 50 | 最后,编译完成以后,在 `${FLINK_COS_FS}/flink-cos-fs-hadoop/target` 就可以得到 `flink-cos-fs-hadoop-${flink.version}-{version}.jar` 的依赖包。 51 | 52 | 下载地址:[Flink-cos-fs release](https://github.com/yuyang733/flink-cos-fs/releases) 53 | 54 | 从 Maven 中央仓库中依赖: 55 | 56 | ```xml 57 | 58 | 59 | com.qcloud.cos 60 | flink-cos-fs-hadoop 61 | ${flink.version>-${version> 62 | 63 | 64 | 65 | ``` 66 | 67 | 68 | ### 安装Flink-cos-fs依赖 69 | 70 | 1.执行`mkdir ${FLINK_HOME}/plugins/cos-fs-hadoop`,在`${FLINK_HOME}/plugins`目录下创建flink-cos-fs-hadoop插件目录; 71 | 72 | 2.将对应版本的预编译包(flink-cos-fs-{flink.version}-{version}.jar)拷贝到`${FLINK_HOME}/plugins/cos-fs-hadoop`目录下; 73 | 74 | 3.在 `${FLINK_HOME}/conf/flink-conf.yaml` 中添加一些 CosN 相关配置以确保 Flink 能够访问到 COS 存储桶,这里的配置键与 CosN 完全兼容,可参考[hadoop-cos](https://cloud.tencent.com/document/product/436/6884),必须配置信息如下: 75 | 76 | ```yaml 77 | fs.cosn.impl: org.apache.hadoop.fs.CosFileSystem 78 | fs.AbstractFileSystem.cosn.impl: org.apache.hadoop.fs.CosN 79 | fs.cosn.userinfo.secretId: AKIDXXXXXXXXXXXXXXXXXXXXXXXXXXXXX 80 | fs.cosn.userinfo.secretKey: XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX 81 | fs.cosn.bucket.region: ap-guangzhou 82 | fs.cosn.bucket.endpoint_suffix: cos.ap-guangzhou.myqcloud.com 83 | 84 | ``` 85 | 86 | 4.在作业的 write 或 sink 路径中填写格式为:```cosn://bucket-appid/path```的路径信息即可,例如: 87 | 88 | ```java 89 | StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); 90 | // 采用 Streaming File Sink 写入的话,必须启用 checkpoint,这里使用 COS 作为 StateBackend 举例子,也可以使用其他 checkpoint storage。 91 | env.setStateBackend(new FsStateBackend("cosn://bucket-name-125xxxxxx/checkpoint")); 92 | env.enableCheckpointing(1000); 93 | // 构造 Streaming File Sink 写入 94 | StreamingFileSink streamingFileSink = 95 | StreamingFileSink.forRowFormat( 96 | new Path(outputPath), new SimpleStringEncoder("UTF-8")) 97 | .withRollingPolicy( 98 | DefaultRollingPolicy.builder() 99 | .withRolloverInterval(TimeUnit.SECONDS.toMillis(5)) 100 | .withInactivityInterval(TimeUnit.SECONDS.toMillis(5)) 101 | .withMaxPartSize(1024) 102 | .build()) 103 | .build(); 104 | ... 105 | ``` 106 | 107 | ⚠️**注意**:如果使用 Streaming File Sink 方式写入,需要同时启用 Flink 的 checkpoint,否则写入的数据始终处于 inprogress 不可见状态,无法被读取。 108 | 109 | ### 使用示例 110 | 111 | 以下给出 Flink Job 读写 COS 的示例代码: 112 | 113 | ```Java 114 | // Read from COS bucket 115 | env.readTextFile("cosn:///"); 116 | 117 | // Write to COS bucket 118 | stream.writeAsText("cosn:///"); 119 | 120 | // Use COS as FsStatebackend 121 | env.setStateBackend(new FsStateBackend("cosn:///")); 122 | 123 | // Use the streamingFileSink which supports the recoverable writer 124 | StreamingFileSink fileSink = StreamingFileSink.forRowFormat( 125 | new Path("cosn:///"),new SimpleStringEncoder("UTF-8")) 126 | .withRollingPolicy(build).build(); 127 | 128 | ``` 129 | 130 | 131 | ## 所有配置说明 132 | 133 | | 属性键 | 说明 | 默认值 | 必填项 | 134 | |:-----------------------------------:|:--------------------|:-----:|:---:| 135 | |fs.cosn.bucket.endpoint_suffix|指定要连接的COS endpoint,该项为非必填项目。对于公有云COS用户而言,只需要正确填写上述的region配置即可。兼容原配置项:fs.cosn.userinfo.endpoint_suffix。|无|否| 136 | |fs.cosn.userinfo.secretId/secretKey| 填写您账户的API 密钥信息。可通过 [云 API 密钥 控制台](https://console.cloud.tencent.com/capi) 查看。| 无 | 是| 137 | |fs.cosn.impl | cosn对FileSystem的实现类,固定为 org.apache.hadoop.fs.CosFileSystem。| 无|是| 138 | |fs.AbstractFileSystem.cosn.impl | cosn对AbstractFileSy stem的实现类,固定为org.apache.hadoop.fs.CosN。| 无 |是| 139 | |fs.cosn.bucket.region | 请填写您的地域信息,枚举值为 [可用地域](https://cloud.tencent.com/document/product/436/6224) 中的地域简称,如ap-beijing、ap-guangzhou等。 兼容原配置项:fs.cosn.userinfo.region。| 无 | 是| 140 | |fs.cosn.tmp.dir | 请设置一个实际存在的本地目录,运行过程中产生的临时文件会暂时放于此处。|/tmp/hadoop_cos | 否| 141 | |fs.cosn.block.size | CosN文件系统每个block的大小,默认为128MB | ‭134217728‬(128MB) | 否 | 142 | |fs.cosn.upload.buffer | CosN文件系统上传时依赖的缓冲区类型。当前支持三种类型的缓冲区:非直接内存缓冲区(non_direct_memory),直接内存缓冲区(direct_memory),磁盘映射缓冲区(mapped_disk)。非直接内存缓冲区使用的是JVM堆内存,直接内存缓冲区使用的是堆外内存,而磁盘映射缓冲区则是基于内存文件映射得到的缓冲区。| mapped_disk | 否 | 143 | |fs.cosn.upload.buffer.size | CosN文件系统上传时依赖的缓冲区大小,如果指定为-1,则表示不限制。若不限制缓冲区大小,则缓冲区类型必须为mapped_disk。如果指定大小大于0,则要求该值至少大于等于一个block的大小。兼容原配置项:fs.cosn.buffer.size。| 134217728(128MB)|否| 144 | |fs.cosn.upload.part.size | 分块上传时每个part的大小。由于 COS 的分块上传最多只能支持10000块,因此需要预估最大可能使用到的单文件大小。例如,part size 为8MB时,最大能够支持78GB的单文件上传。 part size 最大可以支持到2GB,即单文件最大可支持19TB。| 8388608(8MB)| 否 | 145 | |fs.cosn.upload_thread_pool | 文件流式上传到COS时,并发上传的线程数目 | 8 | 否| 146 | |fs.cosn.copy_thread_pool | 目录拷贝操作时,可用于并发拷贝和删除文件的线程数目 | 3 | 否 | 147 | |fs.cosn.read.ahead.block.size | 预读块的大小 | ‭1048576‬(1MB) | 否 | 148 | |fs.cosn.read.ahead.queue.size | 预读队列的长度 | 8 | 否 | 149 | |fs.cosn.maxRetries | 访问COS出现错误时,最多重试的次数 | 200 | 否 | 150 | |fs.cosn.retry.interval.seconds | 每次重试的时间间隔 | 3 | 否 | 151 | |fs.cosn.max.connection.num | 配置COS连接池中维持的最大连接数目,这个数目与单机读写COS的并发有关,建议至少大于或等于单机读写COS的并发数| 1024 | 否| 152 | |fs.cosn.customer.domain | 配置COS的自定义域名,默认为空| 无 | 否| 153 | |fs.cosn.server-side-encryption.algorithm | 配置COS服务端加密算法,支持SSE-C和SSE-COS,默认为空,不加密| 无 | 否| 154 | |fs.cosn.server-side-encryption.key | 当开启COS的SSE-C服务端加密算法时,必须配置SSE-C的密钥,密钥格式为base64编码的AES-256密钥,默认为空,不加密| 无 | 否| 155 | |fs.cosn.crc64.checksum.enabled | 是否开启CRC64校验。默认不开启,此时无法使用`hadoop fs -checksum`命令获取文件的CRC64校验值。| false | 否 | 156 | |fs.cosn.traffic.limit | 上传下载带宽的控制选项,819200 ~ 838860800,单位为bits/s。默认值为-1,表示不限制。 | -1 | 否 | 157 | 158 | ## 使用POSIX Bucket配置说明 159 | | 属性键 | 说明 | 默认值 | 必填项 | 160 | |:-----------------------------------:|:--------------------|:-----:|:---:| 161 | |fs.cosn.trsf.fs.AbstractFileSystem.ofs.impl|元数据加速实现类,com.qcloud.chdfs.fs.CHDFSDelegateFSAdapter|无|是| 162 | |fs.cosn.trsf.fs.ofs.impl| 元数据加速实现类,com.qcloud.chdfs.fs.CHDFSHadoopFileSystemAdapter | 无 | 是| 163 | |fs.cosn.trsf.fs.ofs.tmp.cache.dir | 元数据加速临时目录,给足空间及权限,例如/data/emr/hdfs/tmp/chdfs/ | 无|是| 164 | |fs.cosn.trsf.fs.ofs.user.appid | 客户bucket对应的appid | 无 |是| 165 | |fs.cosn.trsf.fs.ofs.bucket.region | 客户bucket所在的园区 | 无 | 是| 166 | |fs.cosn.trsf.fs.ofs.upload.flush.flag | 客户端flush开关,flink on cos场景必须设置为 true |false | 否| 167 | 168 | 169 | ## FAQ 170 | 171 | - Flink 既可以通过[hadoop-cos](https://github.com/tencentyun/hadoop-cos)读写 COS 中的对象文件,也可以通过 flink-cos-fs 来读写,这两种有什么区别? 172 | 173 | hadoop-cos 实现了 Hadoop 的兼容文件系统语义,Flink 可以通过写 Hadoop 兼容文件系统的形式写入数据到 COS 中,但是这种方式不支持的 Flink 的 recoverable writer 写入,当你使用 streamingFileSink 写入数据时,要求底层文件系统支持recoverable writer。 因此,flink-cos-fs 基于 Hadoop-COS (CosN) 扩展实现了 Flink 的recoverable writer,完整地支持了 Flink 文件系统的语义,因此推荐使用它来访问 COS 对象。 174 | 175 | 176 | ## 编译 177 | 目前由于元数据加速桶静态包使用loadjar方式,如何shade了org.apache.hadoop 会导致network和loadjar的pkg路径不同 178 | 1)解决方式通过COSNFileSystemFactory的useFlinkShade开启,pom.xml of flink-cos-fs-hadoop开启shade 179 | 2)或者关闭useFlinkShade,去掉shade 180 | 3)todo: 完善ofs sdk flink或其他框架可以指定shade加载不同动态包实现all shade模式 -------------------------------------------------------------------------------- /tools/maven/suppressions-core.xml: -------------------------------------------------------------------------------- 1 | 2 | 20 | 21 | 24 | 25 | 26 | 29 | 30 | 33 | 34 | 37 | 38 | 41 | 42 | 45 | 46 | 49 | 50 | 53 | 54 | 57 | 58 | 61 | 62 | 65 | 66 | 69 | 70 | 73 | 74 | 77 | 78 | 81 | 82 | 85 | 86 | 89 | 90 | 93 | 94 | 97 | 98 | 101 | 102 | -------------------------------------------------------------------------------- /tools/maven/suppressions-optimizer.xml: -------------------------------------------------------------------------------- 1 | 2 | 20 | 21 | 24 | 25 | 26 | 29 | 30 | 33 | 34 | 37 | 38 | 41 | 42 | 45 | 46 | 49 | 50 | 53 | 54 | 57 | 58 | 61 | 62 | 65 | 66 | 69 | 70 | 73 | 74 | 77 | 78 | 81 | 82 | 85 | 86 | 89 | 90 | 93 | 94 | 97 | 98 | 101 | 102 | 105 | 106 | -------------------------------------------------------------------------------- /flink-cos-fs-base/src/main/java/org/apache/flink/fs/cos/common/writer/RecoverableMultipartUploadImpl.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.fs.cos.common.writer; 20 | 21 | import org.apache.flink.core.fs.RecoverableFsDataOutputStream; 22 | import org.apache.flink.fs.cos.common.utils.RefCountedFSOutputStream; 23 | 24 | import com.qcloud.cos.model.PartETag; 25 | import com.qcloud.cos.thirdparty.org.apache.commons.codec.digest.DigestUtils; 26 | 27 | import javax.annotation.Nullable; 28 | 29 | import java.io.File; 30 | import java.io.FileInputStream; 31 | import java.io.IOException; 32 | import java.util.ArrayDeque; 33 | import java.util.ArrayList; 34 | import java.util.Deque; 35 | import java.util.List; 36 | import java.util.Optional; 37 | import java.util.UUID; 38 | import java.util.concurrent.CompletableFuture; 39 | import java.util.concurrent.ExecutionException; 40 | import java.util.concurrent.Executor; 41 | 42 | import static org.apache.flink.util.Preconditions.checkArgument; 43 | import static org.apache.flink.util.Preconditions.checkNotNull; 44 | import static org.apache.flink.util.Preconditions.checkState; 45 | 46 | /** The Recoverable MPU implementation. */ 47 | public class RecoverableMultipartUploadImpl implements RecoverableMultipartUpload { 48 | private final COSAccessHelper cosAccessHelper; 49 | 50 | private final Executor uploadThreadPool; 51 | 52 | private final Deque> uploadsInProgress; 53 | 54 | private final String namePrefixForTempObjects; 55 | 56 | private final MultipartUploadInfo currentUploadInfo; 57 | 58 | private RecoverableMultipartUploadImpl( 59 | COSAccessHelper cosAccessHelper, 60 | Executor uploadThreadPool, 61 | String uploadId, 62 | String objectName, 63 | List partsSoFar, 64 | long numBytes, 65 | Optional incompletePart) { 66 | checkArgument(numBytes >= 0L); 67 | this.cosAccessHelper = cosAccessHelper; 68 | this.uploadThreadPool = checkNotNull(uploadThreadPool); 69 | this.currentUploadInfo = 70 | new MultipartUploadInfo(objectName, uploadId, partsSoFar, numBytes, incompletePart); 71 | this.namePrefixForTempObjects = createIncompletePartObjectNamePrefix(objectName); 72 | this.uploadsInProgress = new ArrayDeque<>(); 73 | } 74 | 75 | @Override 76 | public void uploadPart(RefCountedFSOutputStream file) throws IOException { 77 | // this is to guarantee that nobody is 78 | // writing to the file we are uploading. 79 | checkState(file.isClosed()); 80 | 81 | final CompletableFuture future = new CompletableFuture<>(); 82 | uploadsInProgress.add(future); 83 | 84 | final long partLength = file.getPos(); 85 | currentUploadInfo.registerNewPart(partLength); 86 | 87 | file.retain(); // keep the file while the async upload still runs 88 | uploadThreadPool.execute( 89 | new UploadTask(this.cosAccessHelper, currentUploadInfo, file, future)); 90 | } 91 | 92 | @Override 93 | public RecoverableFsDataOutputStream.Committer snapshotAndGetCommitter() throws IOException { 94 | final COSRecoverable cosRecoverable = snapshotAndGetRecoverable(null); 95 | 96 | return new COSCommitter( 97 | this.cosAccessHelper, 98 | cosRecoverable.getUploadId(), 99 | cosRecoverable.getObjectName(), 100 | cosRecoverable.getPartETags(), 101 | cosRecoverable.getNumBytesInParts()); 102 | } 103 | 104 | @Override 105 | public COSRecoverable snapshotAndGetRecoverable( 106 | @Nullable RefCountedFSOutputStream incompletePartFile) throws IOException { 107 | final String incompletePartObjectName = safelyUploadSmallPart(incompletePartFile); 108 | 109 | awaitPendingPartsUpload(); 110 | 111 | final String objectName = this.currentUploadInfo.getObjectName(); 112 | final String uploadId = this.currentUploadInfo.getUploadId(); 113 | final List completedParts = 114 | this.currentUploadInfo.getCopyOfEtagsOfCompleteParts(); 115 | final long sizeInBytes = this.currentUploadInfo.getExpectedSizeInBytes(); 116 | 117 | if (null == incompletePartObjectName) { 118 | return new COSRecoverable(uploadId, objectName, completedParts, sizeInBytes); 119 | } else { 120 | return new COSRecoverable( 121 | uploadId, 122 | objectName, 123 | completedParts, 124 | sizeInBytes, 125 | incompletePartObjectName, 126 | incompletePartFile.getPos()); 127 | } 128 | } 129 | 130 | @Nullable 131 | private String safelyUploadSmallPart(@Nullable RefCountedFSOutputStream fsOutputStream) 132 | throws IOException { 133 | if (null == fsOutputStream || fsOutputStream.getPos() == 0L) { 134 | return null; 135 | } 136 | 137 | final String incompletePartObjectName = createIncompletePartObjectName(); 138 | fsOutputStream.retain(); 139 | 140 | try { 141 | this.cosAccessHelper.putObject( 142 | incompletePartObjectName, fsOutputStream.getInputFile(), null); 143 | } finally { 144 | fsOutputStream.release(); 145 | } 146 | 147 | return incompletePartObjectName; 148 | } 149 | 150 | static String createIncompletePartObjectNamePrefix(String objectName) { 151 | checkNotNull(objectName); 152 | 153 | final int lastSlash = objectName.lastIndexOf('/'); 154 | final String parent; 155 | final String child; 156 | 157 | if (lastSlash == -1) { 158 | parent = ""; 159 | child = objectName; 160 | } else { 161 | parent = objectName.substring(0, lastSlash + 1); 162 | child = objectName.substring(lastSlash + 1); 163 | } 164 | return parent + (child.isEmpty() ? "" : '_') + child + "_tmp_"; 165 | } 166 | 167 | private String createIncompletePartObjectName() { 168 | return namePrefixForTempObjects + UUID.randomUUID().toString(); 169 | } 170 | 171 | private void awaitPendingPartsUpload() throws IOException { 172 | checkState(this.currentUploadInfo.getRemainingParts() == this.uploadsInProgress.size()); 173 | 174 | while (this.currentUploadInfo.getRemainingParts() > 0) { 175 | CompletableFuture next = this.uploadsInProgress.peekFirst(); 176 | PartETag nextPart = awaitPendingPartUploadToComplete(next); 177 | this.currentUploadInfo.registerCompletePart(nextPart); 178 | this.uploadsInProgress.removeFirst(); 179 | } 180 | } 181 | 182 | private PartETag awaitPendingPartUploadToComplete(CompletableFuture upload) 183 | throws IOException { 184 | final PartETag completedUploadETag; 185 | 186 | try { 187 | completedUploadETag = upload.get(); 188 | } catch (InterruptedException interruptedException) { 189 | Thread.currentThread().interrupt(); 190 | throw new IOException("Interrupted while waiting for part uploads to complete"); 191 | } catch (ExecutionException e) { 192 | throw new IOException("Uploading parts failed", e.getCause()); 193 | } 194 | return completedUploadETag; 195 | } 196 | 197 | public static RecoverableMultipartUploadImpl newUpload( 198 | final COSAccessHelper cosAccessHelper, 199 | final Executor uploadThreadPool, 200 | final String objectName) 201 | throws IOException { 202 | 203 | final String multiPartUploadId = cosAccessHelper.startMultipartUpload(objectName); 204 | 205 | return new RecoverableMultipartUploadImpl( 206 | cosAccessHelper, 207 | uploadThreadPool, 208 | multiPartUploadId, 209 | objectName, 210 | new ArrayList<>(), 211 | 0L, 212 | Optional.empty()); 213 | } 214 | 215 | public static RecoverableMultipartUploadImpl recoverUpload( 216 | final COSAccessHelper cosAccessHelper, 217 | final Executor uploadThreadPool, 218 | final String multipartUploadId, 219 | final String objectName, 220 | final List partsSoFar, 221 | final long numBytesSoFar, 222 | final Optional incompletePart) { 223 | 224 | return new RecoverableMultipartUploadImpl( 225 | cosAccessHelper, 226 | uploadThreadPool, 227 | multipartUploadId, 228 | objectName, 229 | new ArrayList<>(partsSoFar), 230 | numBytesSoFar, 231 | incompletePart); 232 | } 233 | 234 | @Override 235 | public Optional getIncompletePart() { 236 | return this.currentUploadInfo.getIncompletePart(); 237 | } 238 | 239 | private static class UploadTask implements Runnable { 240 | 241 | private final COSAccessHelper cosAccessHelper; 242 | 243 | private final String objectName; 244 | 245 | private final String uploadId; 246 | 247 | private final int partNumber; 248 | 249 | private final RefCountedFSOutputStream file; 250 | 251 | private final CompletableFuture future; 252 | 253 | UploadTask( 254 | final COSAccessHelper cosAccessHelper, 255 | final MultipartUploadInfo currentUpload, 256 | final RefCountedFSOutputStream file, 257 | final CompletableFuture future) { 258 | 259 | checkNotNull(currentUpload); 260 | 261 | this.objectName = currentUpload.getObjectName(); 262 | this.uploadId = currentUpload.getUploadId(); 263 | this.partNumber = currentUpload.getNumberOfRegisteredParts(); 264 | 265 | checkArgument(partNumber >= 1 && partNumber <= 10000); 266 | 267 | this.cosAccessHelper = checkNotNull(cosAccessHelper); 268 | this.file = checkNotNull(file); 269 | this.future = checkNotNull(future); 270 | } 271 | 272 | @Override 273 | public void run() { 274 | try { 275 | File file1; 276 | FileInputStream fileInputStream = new FileInputStream(file.getInputFile()); 277 | byte[] md5Hash = DigestUtils.md5(fileInputStream); 278 | fileInputStream.close(); 279 | final PartETag result = 280 | this.cosAccessHelper.uploadPart( 281 | objectName, uploadId, partNumber, file.getInputFile(), md5Hash); 282 | future.complete(result); 283 | file.release(); 284 | } catch (Throwable t) { 285 | future.completeExceptionally(t); 286 | } 287 | } 288 | } 289 | } 290 | -------------------------------------------------------------------------------- /tools/maven/spotbugs-exclude.xml: -------------------------------------------------------------------------------- 1 | 2 | 20 | 21 | 22 | 23 | 24 | 25 | 26 | 27 | 28 | 29 | 30 | 31 | 32 | 33 | 34 | 35 | 36 | 37 | 38 | 39 | 40 | 41 | 42 | 43 | 44 | 45 | 46 | 47 | 48 | 49 | 50 | 51 | 52 | 53 | 54 | 55 | 56 | 57 | 58 | 59 | 60 | 61 | 62 | 63 | 64 | 65 | 66 | 67 | 68 | 69 | 70 | 71 | 72 | 73 | 74 | 75 | 76 | 77 | 78 | 79 | 80 | 81 | 82 | 83 | 84 | 85 | 86 | 87 | 88 | 89 | 90 | 91 | 92 | 93 | 94 | 95 | 96 | 97 | 98 | 99 | 100 | 101 | 102 | 103 | 104 | 105 | 106 | 107 | 108 | 109 | 110 | 111 | 112 | 113 | 114 | 115 | 116 | 117 | 118 | 119 | 120 | 121 | 122 | 123 | 124 | 125 | 126 | 127 | 128 | 129 | 130 | 131 | 132 | 133 | 134 | 135 | 136 | 137 | 138 | 139 | 140 | 141 | 142 | 143 | 144 | 145 | D 146 | 147 | 148 | 149 | 150 | 151 | 152 | 153 | 154 | 155 | 156 | 157 | 158 | 159 | 160 | 161 | 162 | 163 | 164 | 165 | 166 | 167 | 168 | 169 | 170 | 171 | 172 | 173 | 174 | 175 | 176 | 177 | 178 | 179 | 180 | 181 | 182 | 183 | 184 | 185 | 186 | 187 | 188 | 189 | 190 | 191 | 192 | 193 | 194 | 195 | 196 | 197 | 198 | 199 | 200 | 201 | 202 | 203 | 204 | 205 | 206 | 207 | 208 | 209 | 210 | 211 | 212 | 213 | 214 | 215 | 216 | 217 | 218 | 219 | 220 | 221 | 222 | 223 | 224 | 225 | 226 | 227 | 228 | 229 | 230 | 231 | 232 | 233 | 234 | 235 | 236 | 237 | 238 | 239 | 240 | 241 | 242 | 243 | 244 | 245 | 246 | 247 | 248 | 249 | 250 | 251 | 252 | 253 | 254 | 255 | 256 | 257 | 258 | 259 | 260 | 261 | 262 | 263 | 264 | 265 | 266 | 267 | 268 | 269 | 270 | 271 | 272 | 273 | 274 | 275 | 276 | 277 | 278 | 279 | 280 | 281 | 282 | 283 | 284 | 285 | 286 | 287 | 288 | 289 | 290 | 291 | 292 | 293 | 294 | 295 | 296 | 297 | 298 | 299 | 300 | 301 | 302 | 303 | 304 | 305 | 306 | 307 | 308 | 309 | 310 | 311 | 312 | 313 | 314 | 315 | 316 | 317 | 318 | 319 | 320 | 321 | 322 | 323 | 324 | 325 | 326 | 327 | 328 | 329 | 330 | 331 | 332 | 333 | 334 | 335 | 336 | 337 | 338 | 339 | 340 | 341 | 342 | 343 | 344 | 345 | 346 | 347 | 348 | 349 | 350 | 351 | 352 | 353 | 354 | 355 | 356 | 357 | 358 | 359 | 360 | 361 | 362 | 363 | 364 | 365 | 366 | 367 | 368 | 369 | 370 | 371 | 372 | 373 | 374 | 375 | 376 | 377 | 378 | 379 | 380 | 381 | 382 | 383 | 384 | 385 | 386 | 387 | 388 | 389 | 390 | 391 | 392 | 393 | 394 | 395 | 396 | 397 | 398 | 399 | 400 | 401 | 402 | 403 | 404 | 405 | 406 | 407 | 408 | 409 | 410 | 411 | 412 | 413 | 414 | 415 | 416 | 417 | 418 | 419 | 420 | 421 | 422 | 423 | 424 | 425 | 426 | 427 | 428 | 429 | 430 | 431 | 432 | 433 | 434 | 435 | 436 | 437 | 438 | 439 | 440 | 441 | 442 | 443 | 444 | 445 | 446 | 447 | 448 | 449 | 450 | 451 | 452 | 453 | 454 | 455 | 456 | 457 | 458 | 459 | 460 | -------------------------------------------------------------------------------- /flink-cos-fs-base/src/main/java/org/apache/flink/fs/cos/common/fswriter/COSPosixRecoverableFsDataOutputStream.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.fs.cos.common.fswriter; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.core.fs.RecoverableFsDataOutputStream; 23 | import org.apache.flink.core.fs.RecoverableWriter.CommitRecoverable; 24 | import org.apache.flink.core.fs.RecoverableWriter.ResumeRecoverable; 25 | import org.apache.flink.util.ExceptionUtils; 26 | import org.apache.flink.util.FlinkRuntimeException; 27 | import org.apache.flink.util.IOUtils; 28 | 29 | import org.apache.hadoop.fs.CosFileSystem; 30 | import org.apache.hadoop.fs.FSDataOutputStream; 31 | import org.apache.hadoop.fs.FileStatus; 32 | import org.apache.hadoop.fs.FileSystem; 33 | import org.apache.hadoop.fs.Path; 34 | import org.slf4j.Logger; 35 | import org.slf4j.LoggerFactory; 36 | 37 | import java.io.FileNotFoundException; 38 | import java.io.IOException; 39 | import java.lang.reflect.InvocationTargetException; 40 | import java.lang.reflect.Method; 41 | import java.lang.reflect.Modifier; 42 | 43 | import static org.apache.flink.util.Preconditions.checkNotNull; 44 | 45 | /** 46 | * An implementation of the {@link RecoverableFsDataOutputStream} for Hadoop's 47 | * file system abstraction. 48 | */ 49 | @Internal 50 | class COSPosixRecoverableFsDataOutputStream extends RecoverableFsDataOutputStream { 51 | 52 | private static final Logger LOG = LoggerFactory.getLogger(COSPosixRecoverableFsDataOutputStream.class); 53 | 54 | private static Method truncateHandle; 55 | 56 | private final FileSystem fs; 57 | 58 | private final Path targetFile; 59 | 60 | private final Path tempFile; 61 | 62 | private final FSDataOutputStream out; 63 | 64 | COSPosixRecoverableFsDataOutputStream( 65 | FileSystem fs, 66 | Path targetFile, 67 | Path tempFile) throws IOException { 68 | LOG.debug("cos merge recoverable fs data output stream " + 69 | "temp file recover way trigger, file: {}", tempFile.toString()); 70 | 71 | ensureTruncateInitialized(); 72 | 73 | this.fs = checkNotNull(fs); 74 | this.targetFile = checkNotNull(targetFile); 75 | this.tempFile = checkNotNull(tempFile); 76 | this.out = fs.create(tempFile); 77 | } 78 | 79 | COSPosixRecoverableFsDataOutputStream( 80 | FileSystem fs, 81 | COSPosixRecoverable recoverable) throws IOException { 82 | 83 | ensureTruncateInitialized(); 84 | this.fs = checkNotNull(fs); 85 | this.targetFile = checkNotNull(recoverable.targetFile()); 86 | this.tempFile = checkNotNull(recoverable.tempFile()); 87 | 88 | LOG.debug("cos merge recover process, temp:{}, recover:{}", tempFile.toString(), recoverable.toString()); 89 | 90 | safelyTruncateFile(fs, tempFile, recoverable); 91 | 92 | out = fs.append(tempFile); 93 | 94 | LOG.debug("after truncate and append file:{}, pos:{}", tempFile.toString(), out.getPos()); 95 | // sanity check 96 | long pos = out.getPos(); 97 | if (pos != recoverable.offset()) { 98 | IOUtils.closeQuietly(out); 99 | throw new IOException("Truncate failed: " + tempFile + 100 | " (requested=" + recoverable.offset() + " ,size=" + pos + ')'); 101 | } 102 | } 103 | 104 | @Override 105 | public void write(int b) throws IOException { 106 | out.write(b); 107 | } 108 | 109 | @Override 110 | public void write(byte[] b, int off, int len) throws IOException { 111 | out.write(b, off, len); 112 | } 113 | 114 | @Override 115 | public void flush() throws IOException { 116 | // whether real call the flush. 117 | out.flush(); 118 | } 119 | 120 | @Override 121 | public void sync() throws IOException { 122 | out.flush(); 123 | out.hsync(); 124 | } 125 | 126 | @Override 127 | public long getPos() throws IOException { 128 | return out.getPos(); 129 | } 130 | 131 | @Override 132 | public ResumeRecoverable persist() throws IOException { 133 | sync(); 134 | LOG.debug("cos merge persist target:{}, temp:{}, pos:{}", targetFile.toString(), 135 | tempFile.toString(), getPos()); 136 | return new COSPosixRecoverable(targetFile, tempFile, getPos()); 137 | } 138 | 139 | @Override 140 | public Committer closeForCommit() throws IOException { 141 | final long pos = getPos(); 142 | close(); 143 | LOG.debug("cos merge close for commit target:{}, temp:{}, pos:{}", targetFile.toString(), 144 | tempFile.toString(), pos); 145 | return new COSPosixRecoverableFsDataOutputStream.COSMergeCommitter(fs, 146 | new COSPosixRecoverable(targetFile, tempFile, pos)); 147 | } 148 | 149 | @Override 150 | public void close() throws IOException { 151 | LOG.debug("cos merge calling close file:{}, pos:{}", tempFile.toString(), out.getPos()); 152 | out.close(); 153 | } 154 | 155 | // ------------------------------------------------------------------------ 156 | // Reflection utils for truncation 157 | // These are needed the version after the static plugin of chdfs of 0.6.3 version 158 | // Use the 0.6.6 test version to poc 159 | // ------------------------------------------------------------------------ 160 | 161 | private static void safelyTruncateFile( 162 | final FileSystem fileSystem, 163 | final Path path, 164 | final COSPosixRecoverable recoverable) throws IOException { 165 | 166 | ensureTruncateInitialized(); 167 | 168 | // if fd or session not close when occur something interrupt, 169 | // the truncate will try to open fd again which may occur the 'can not open fd again' 170 | // so every time begin call the truncate, manual to unlock the fd. 171 | ((CosFileSystem) fileSystem).releaseFileLock(path); 172 | 173 | // truncate back and append 174 | boolean truncated; 175 | try { 176 | LOG.debug("cos merge safely truncate file path:{}, offset:{}", path.toString(), recoverable.offset()); 177 | truncated = truncate(fileSystem, path, recoverable.offset()); 178 | } catch (Exception e) { 179 | throw new IOException("Problem while truncating file: " + path, e); 180 | } 181 | 182 | // todo whether need the same recover lease to the hdfs file system 183 | } 184 | 185 | private static void ensureTruncateInitialized() throws FlinkRuntimeException { 186 | Method truncateMethod; 187 | try { 188 | truncateMethod = FileSystem.class.getMethod("truncate", Path.class, long.class); 189 | } 190 | catch (NoSuchMethodException e) { 191 | throw new FlinkRuntimeException("Could not find a public truncate method on the Hadoop File System."); 192 | } 193 | 194 | if (!Modifier.isPublic(truncateMethod.getModifiers())) { 195 | throw new FlinkRuntimeException("Could not find a public truncate method on the Hadoop File System."); 196 | } 197 | LOG.debug("ensure truncate initialized is ok"); 198 | truncateHandle = truncateMethod; 199 | } 200 | 201 | private static boolean truncate(final FileSystem hadoopFs, final Path file, final long length) throws IOException { 202 | if (truncateHandle != null) { 203 | try { 204 | return (Boolean) truncateHandle.invoke(hadoopFs, file, length); 205 | } 206 | catch (InvocationTargetException e) { 207 | ExceptionUtils.rethrowIOException(e.getTargetException()); 208 | } 209 | catch (Throwable t) { 210 | throw new IOException( 211 | "Truncation of file failed because of access/linking problems with Hadoop's truncate call. " + 212 | "This is most likely a dependency conflict or class loading problem."); 213 | } 214 | } 215 | else { 216 | throw new IllegalStateException("Truncation handle has not been initialized"); 217 | } 218 | return false; 219 | } 220 | 221 | // ------------------------------------------------------------------------ 222 | // Committer 223 | // ------------------------------------------------------------------------ 224 | 225 | /** 226 | * Implementation of a committer for the Hadoop File System abstraction. 227 | * This implementation commits by renaming the temp file to the final file path. 228 | * The temp file is truncated before renaming in case there is trailing garbage data. 229 | */ 230 | static class COSMergeCommitter implements Committer { 231 | private static final Logger LOG = LoggerFactory.getLogger(COSMergeCommitter.class); 232 | 233 | private final FileSystem fs; 234 | private final COSPosixRecoverable recoverable; 235 | 236 | COSMergeCommitter(FileSystem fs, COSPosixRecoverable recoverable) { 237 | this.fs = checkNotNull(fs); 238 | this.recoverable = checkNotNull(recoverable); 239 | } 240 | 241 | @Override 242 | public void commit() throws IOException { 243 | LOG.debug("cos merge committer begin commit"); 244 | final Path src = recoverable.tempFile(); 245 | final Path dest = recoverable.targetFile(); 246 | final long expectedLength = recoverable.offset(); 247 | 248 | final FileStatus srcStatus; 249 | try { 250 | srcStatus = fs.getFileStatus(src); 251 | } 252 | catch (IOException e) { 253 | LOG.info("cos merge output stream commit failed, src not exist {}, {}", src.toString(), dest.toString()); 254 | throw new IOException("Cannot clean commit: Staging file does not exist."); 255 | } 256 | 257 | if (srcStatus.getLen() != expectedLength) { 258 | // something was done to this file since the committer was created. 259 | // this is not the "clean" case 260 | throw new IOException("Cannot clean commit: File has trailing junk data."); 261 | } 262 | 263 | try { 264 | LOG.info("cos merge output stream commit {}, {}", src.toString(), dest.toString()); 265 | fs.rename(src, dest); 266 | } 267 | catch (IOException e) { 268 | throw new IOException("Committing file by rename failed: " + src + " to " + dest, e); 269 | } 270 | } 271 | 272 | @Override 273 | public void commitAfterRecovery() throws IOException { 274 | LOG.debug("cos merge committer begin commit after recovery"); 275 | final Path src = recoverable.tempFile(); 276 | final Path dest = recoverable.targetFile(); 277 | final long expectedLength = recoverable.offset(); 278 | 279 | FileStatus srcStatus = null; 280 | try { 281 | srcStatus = fs.getFileStatus(src); 282 | } 283 | catch (FileNotFoundException e) { 284 | // status remains null 285 | } 286 | catch (IOException e) { 287 | throw new IOException("Committing during recovery failed: Could not access status of source file."); 288 | } 289 | 290 | if (srcStatus != null) { 291 | if (srcStatus.getLen() > expectedLength) { 292 | // can happen if we go from persist to recovering for commit directly 293 | // truncate the trailing junk away 294 | LOG.info("cos merge commit after recovery process, src:{}, recover:{}", 295 | src.toString(), recoverable.toString()); 296 | safelyTruncateFile(fs, src, recoverable); 297 | } 298 | 299 | // rename to final location (if it exists, overwrite it) 300 | try { 301 | LOG.info("cos merge output stream commit after recovery {}, {}", src.toString(), dest.toString()); 302 | fs.rename(src, dest); 303 | } 304 | catch (IOException e) { 305 | throw new IOException("Committing file by rename failed: " + src + " to " + dest, e); 306 | } 307 | } 308 | else if (!fs.exists(dest)) { 309 | // neither exists - that can be a sign of 310 | // - (1) a serious problem (file system loss of data) 311 | // - (2) a recovery of a savepoint that is some time old and the users 312 | // removed the files in the meantime. 313 | 314 | // TODO how to handle this? 315 | // We probably need an option for users whether this should log, 316 | // or result in an exception or unrecoverable exception 317 | } 318 | } 319 | 320 | @Override 321 | public CommitRecoverable getRecoverable() { 322 | return recoverable; 323 | } 324 | } 325 | } 326 | --------------------------------------------------------------------------------