diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java index 1184f42de036..1cfb771dc957 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java @@ -23,11 +23,13 @@ import java.io.FileOutputStream; import java.io.IOException; import java.io.OutputStream; +import java.nio.file.Paths; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.CompletionException; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -494,6 +496,29 @@ private CompletableFuture handleWriteChunk( return raftFuture; } + @Override + public CompletableFuture stream(RaftClientRequest request) { + return CompletableFuture.supplyAsync(() -> { + try { + ContainerCommandRequestProto requestProto = + getContainerCommandRequestProto(gid, + request.getMessage().getContent()); + DispatcherContext context = + new DispatcherContext.Builder() + .setStage(DispatcherContext.WriteChunkStage.WRITE_DATA) + .setContainer2BCSIDMap(container2BCSIDMap) + .build(); + + ContainerCommandResponseProto response = runCommand( + requestProto, context); + String path = response.getMessage(); + return new LocalStream(new StreamDataChannel(Paths.get(path))); + } catch (IOException e) { + throw new CompletionException("Failed to create data stream", e); + } + }, executor); + } + private ExecutorService getChunkExecutor(WriteChunkRequestProto req) { int hash = Objects.hashCode(req.getBlockID()); if (hash == Integer.MIN_VALUE) { diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/LocalStream.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/LocalStream.java new file mode 100644 index 000000000000..baae0139667d --- /dev/null +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/LocalStream.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.ozone.container.common.transport.server.ratis; + +import org.apache.ratis.statemachine.StateMachine; + +import java.io.IOException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; + +class LocalStream implements StateMachine.DataStream { + private final StateMachine.DataChannel dataChannel; + + LocalStream(StateMachine.DataChannel dataChannel) { + this.dataChannel = dataChannel; + } + + @Override + public StateMachine.DataChannel getDataChannel() { + return dataChannel; + } + + @Override + public CompletableFuture cleanUp() { + return CompletableFuture.supplyAsync(() -> { + try { + dataChannel.close(); + return true; + } catch (IOException e) { + throw new CompletionException("Failed to close data channel", e); + } + }); + } +} \ No newline at end of file diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/StreamDataChannel.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/StreamDataChannel.java new file mode 100644 index 000000000000..3df66e26dcc9 --- /dev/null +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/StreamDataChannel.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.ozone.container.common.transport.server.ratis; + +import org.apache.ratis.statemachine.StateMachine; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.RandomAccessFile; +import java.nio.ByteBuffer; +import java.nio.file.Path; + +class StreamDataChannel implements StateMachine.DataChannel { + private final Path path; + private final RandomAccessFile randomAccessFile; + + StreamDataChannel(Path path) throws FileNotFoundException { + this.path = path; + this.randomAccessFile = new RandomAccessFile(path.toFile(), "rw"); + } + + @Override + public void force(boolean metadata) throws IOException { + randomAccessFile.getChannel().force(metadata); + } + + @Override + public int write(ByteBuffer src) throws IOException { + return randomAccessFile.getChannel().write(src); + } + + @Override + public boolean isOpen() { + return randomAccessFile.getChannel().isOpen(); + } + + @Override + public void close() throws IOException { + randomAccessFile.close(); + } +}