Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
12 changes: 12 additions & 0 deletions hadoop-hdds/container-service/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -115,6 +115,18 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
<groupId>org.glassfish.jaxb</groupId>
<artifactId>jaxb-runtime</artifactId>
</dependency>
<dependency>
<groupId>io.netty</groupId>
<artifactId>netty-transport</artifactId>
</dependency>
<dependency>
<groupId>io.netty</groupId>
<artifactId>netty-codec</artifactId>
</dependency>
<dependency>
<groupId>io.netty</groupId>
<artifactId>netty-handler</artifactId>
</dependency>

</dependencies>

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,38 @@
/*
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.stream;

import java.nio.file.Path;
import java.nio.file.Paths;

/**
* Streaming binaries to single directory.
*/
public class DirectoryServerDestination implements StreamingDestination {

private Path root;

public DirectoryServerDestination(Path path) {
root = path;
}

@Override
public Path mapToDestination(String name) {
return root.resolve(Paths.get(name));
}
}
Original file line number Diff line number Diff line change
@@ -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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.stream;

import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.HashMap;
import java.util.Map;

/**
* Streaming files from single directory.
*/
public class DirectoryServerSource implements StreamingSource {

private Path root;

public DirectoryServerSource(Path root) {
this.root = root;
}

@Override
/**
* Return logicalNames and real file path to replicate.
*
* @param id name of the subdirectory to replitace relative to root.
*/
public Map<String, Path> getFilesToStream(String id)
throws InterruptedException {
Map<String, Path> files = new HashMap<>();
try {
Files.walk(root.resolve(id))
.filter(Files::isRegularFile)
.forEach(path -> {
files.put(root.relativize(path).toString(), path);
});
} catch (IOException e) {
throw new RuntimeException(e);
}
return files;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,143 @@
/*
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.stream;

import java.io.IOException;
import java.io.RandomAccessFile;
import java.nio.channels.FileChannel;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.nio.file.Path;

import io.netty.buffer.ByteBuf;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInboundHandlerAdapter;
import io.netty.util.ByteProcessor;
import io.netty.util.ReferenceCountUtil;

/**
* Protocol definition from streaming binary files.
*
* Format of the protocol (TCP/IP):
*
* LOGICAL_NAME SIZE
* ... (binary content)
* LOGICAL_NAME SIZE
* ... (binary content)
* END 0
*/
public class DirstreamClientHandler extends ChannelInboundHandlerAdapter {

private final StreamingDestination destination;
private boolean headerMode = true;
private StringBuilder currentFileName = new StringBuilder();
private RandomAccessFile destFile;

private FileChannel destFileChannel;

private long remaining;

public DirstreamClientHandler(StreamingDestination streamingDestination) {
this.destination = streamingDestination;
}

@Override
public void channelRead(ChannelHandlerContext ctx, Object msg)
throws IOException {
try {
ByteBuf buffer = (ByteBuf) msg;
doRead(ctx, buffer);
} finally {
ReferenceCountUtil.release(msg);
}
}

public void doRead(ChannelHandlerContext ctx, ByteBuf buffer)
throws IOException {
if (headerMode) {
int eolPosition = buffer.forEachByte(ByteProcessor.FIND_LF) - buffer
.readerIndex();
if (eolPosition > 0) {
headerMode = false;
final ByteBuf name = buffer.readBytes(eolPosition);
currentFileName.append(name
.toString(StandardCharsets.UTF_8));
name.release();
buffer.skipBytes(1);
String[] parts = currentFileName.toString().split(" ", 2);
remaining = Long.parseLong(parts[0]);
Path destFilePath = destination.mapToDestination(parts[1]);
final Path destfileParent = destFilePath.getParent();
if (destfileParent == null) {
throw new IllegalArgumentException("Streaming destination " +
"provider return with invalid path: " + destFilePath);
}
Files.createDirectories(destfileParent);
this.destFile =
new RandomAccessFile(destFilePath.toFile(), "rw");
destFileChannel = this.destFile.getChannel();

} else {
currentFileName
.append(buffer.toString(StandardCharsets.UTF_8));
}
}
if (!headerMode) {
final int readableBytes = buffer.readableBytes();
if (remaining >= readableBytes) {
remaining -=
buffer.readBytes(destFileChannel, readableBytes);
} else {
remaining -= buffer.readBytes(destFileChannel, (int) remaining);
currentFileName = new StringBuilder();
headerMode = true;
destFile.close();
if (readableBytes > 0) {
doRead(ctx, buffer);
}
}
}
}

@Override
public void channelUnregistered(ChannelHandlerContext ctx) {
try {
if (destFile != null) {
destFile.close();
}
} catch (IOException e) {
e.printStackTrace();
}
}

@Override
public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
cause.printStackTrace();
try {
destFileChannel.close();
destFile.close();
} catch (IOException e) {
e.printStackTrace();
}
ctx.close();
}

public String getCurrentFileName() {
return currentFileName.toString();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,135 @@
/*
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.stream;

import java.io.IOException;
import java.nio.charset.Charset;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.List;
import java.util.Map.Entry;

import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelFutureListener;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInboundHandlerAdapter;
import io.netty.channel.DefaultFileRegion;
import io.netty.util.ByteProcessor;

/**
* Protocol definition of the streaming.
*/
public class DirstreamServerHandler extends ChannelInboundHandlerAdapter {

public static final String END_MARKER = "0 END";

private final StringBuilder id = new StringBuilder();

private StreamingSource source;

private boolean headerProcessed = false;

public DirstreamServerHandler(StreamingSource source) {
this.source = source;
}

@Override
public void channelRead(ChannelHandlerContext ctx, Object msg)
throws Exception {
if (!headerProcessed) {
ByteBuf buffer = (ByteBuf) msg;
int eolPosition = buffer.forEachByte(ByteProcessor.FIND_LF) - buffer
.readerIndex();
if (eolPosition > 0) {
headerProcessed = true;
id.append(buffer.toString(Charset.defaultCharset()));
} else {
id.append(buffer.toString(0, eolPosition, Charset.defaultCharset()));
}
buffer.release();
}

if (headerProcessed) {
final List<Entry<String, Path>> entriesToWrite = new ArrayList<>(
source.getFilesToStream(id.toString().trim()).entrySet());

writeOneElement(ctx, entriesToWrite, 0);

}
}

public void writeOneElement(
ChannelHandlerContext ctx,
List<Entry<String, Path>> entriesToWrite,
int i
)
throws IOException {
final Entry<String, Path> entryToWrite = entriesToWrite.get(i);
Path file = entryToWrite.getValue();
String name = entryToWrite.getKey();
long fileSize = Files.size(file);
String identifier = fileSize + " " + name + "\n";
ByteBuf identifierBuf =
Unpooled.wrappedBuffer(identifier.getBytes(
StandardCharsets.UTF_8));

final int currentIndex = i;

ChannelFuture lastFuture = ctx.writeAndFlush(identifierBuf);
lastFuture.addListener(f -> {
ChannelFuture nextFuture = ctx.writeAndFlush(
new DefaultFileRegion(file.toFile(), 0, fileSize));
if (currentIndex == entriesToWrite.size() - 1) {
nextFuture.addListener(a ->
ctx.writeAndFlush(
Unpooled.wrappedBuffer(
END_MARKER.getBytes(StandardCharsets.UTF_8)))
.addListener(b -> {
ctx.channel().close();
}));
} else {
nextFuture.addListener(
a -> writeOneElement(ctx, entriesToWrite,
currentIndex + 1));
}
});

}

@Override
public void channelReadComplete(ChannelHandlerContext ctx) {
ctx.flush();
}

@Override
public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause)
throws Exception {
cause.printStackTrace();
if (ctx.channel().isActive()) {
ctx.writeAndFlush("ERR: " +
cause.getClass().getSimpleName() + ": " +
cause.getMessage() + '\n').addListener(
ChannelFutureListener.CLOSE);
}
ctx.close();
}
}
Loading