Skip to content

Commit

Permalink
HDDS-2233 - Remove ByteStringHelper and refactor the code to the plac…
Browse files Browse the repository at this point in the history
…e where it used (#1596)
  • Loading branch information
fapifta authored and bshashikant committed Oct 9, 2019
1 parent 87d9f36 commit 1d27930
Show file tree
Hide file tree
Showing 15 changed files with 217 additions and 224 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -36,15 +36,18 @@
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.OzoneSecurityUtil;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.Closeable;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.security.cert.CertificateException;
import java.security.cert.X509Certificate;
import java.util.concurrent.Callable;
import java.util.concurrent.TimeUnit;
import java.util.function.Function;

import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static org.apache.hadoop.hdds.conf.ConfigTag.OZONE;
Expand Down Expand Up @@ -307,6 +310,10 @@ public HddsProtos.ReplicationType getType() {
return HddsProtos.ReplicationType.STAND_ALONE;
}

public Function<ByteBuffer, ByteString> byteBufferToByteStringConversion(){
return ByteStringConversion.createByteBufferConversion(conf);
}

/**
* Get xceiver client metric.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@
import com.google.common.base.Preconditions;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.scm.ByteStringHelper;
import org.apache.hadoop.hdds.scm.XceiverClientReply;
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
Expand Down Expand Up @@ -590,7 +589,7 @@ public boolean isClosed() {
*/
private void writeChunkToContainer(ByteBuffer chunk) throws IOException {
int effectiveChunkSize = chunk.remaining();
ByteString data = ByteStringHelper.getByteString(chunk);
ByteString data = bufferPool.byteStringConversion().apply(chunk);
Checksum checksum = new Checksum(checksumType, bytesPerChecksum);
ChecksumData checksumData = checksum.computeChecksum(chunk);
ChunkInfo chunkInfo = ChunkInfo.newBuilder()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,13 @@
package org.apache.hadoop.hdds.scm.storage;

import com.google.common.base.Preconditions;
import org.apache.hadoop.hdds.scm.ByteStringConversion;
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;

import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
import java.util.function.Function;

/**
* This class creates and manages pool of n buffers.
Expand All @@ -33,12 +36,24 @@ public class BufferPool {
private int currentBufferIndex;
private final int bufferSize;
private final int capacity;
private final Function<ByteBuffer, ByteString> byteStringConversion;

public BufferPool(int bufferSize, int capacity) {
this(bufferSize, capacity,
ByteStringConversion.createByteBufferConversion(null));
}

public BufferPool(int bufferSize, int capacity,
Function<ByteBuffer, ByteString> byteStringConversion){
this.capacity = capacity;
this.bufferSize = bufferSize;
bufferList = new ArrayList<>(capacity);
currentBufferIndex = -1;
this.byteStringConversion = byteStringConversion;
}

public Function<ByteBuffer, ByteString> byteStringConversion(){
return byteStringConversion;
}

public ByteBuffer getCurrentBuffer() {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,62 @@
/*
* 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.hdds.scm;

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
import org.apache.ratis.thirdparty.com.google.protobuf.UnsafeByteOperations;

import java.nio.ByteBuffer;
import java.util.function.Function;

/**
* Helper class to create a conversion function from ByteBuffer to ByteString
* based on the property
* {@link OzoneConfigKeys#OZONE_UNSAFEBYTEOPERATIONS_ENABLED} in the
* Ozone configuration.
*/
public final class ByteStringConversion {
private ByteStringConversion(){} // no instantiation.

/**
* Creates the conversion function to be used to convert ByteBuffers to
* ByteString instances to be used in protobuf messages.
*
* @param config the Ozone configuration
* @return the conversion function defined by
* {@link OzoneConfigKeys#OZONE_UNSAFEBYTEOPERATIONS_ENABLED}
* @see <pre>ByteBuffer</pre>
*/
public static Function<ByteBuffer, ByteString> createByteBufferConversion(
Configuration config){
boolean unsafeEnabled =
config!=null && config.getBoolean(
OzoneConfigKeys.OZONE_UNSAFEBYTEOPERATIONS_ENABLED,
OzoneConfigKeys.OZONE_UNSAFEBYTEOPERATIONS_ENABLED_DEFAULT);
if (unsafeEnabled) {
return buffer -> UnsafeByteOperations.unsafeWrap(buffer);
} else {
return buffer -> {
ByteString retval = ByteString.copyFrom(buffer);
buffer.flip();
return retval;
};
}
}
}

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import java.util.List;
import java.util.Map;
import java.util.concurrent.locks.ReentrantLock;
import java.util.function.Function;

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.StorageUnit;
Expand All @@ -46,7 +47,7 @@
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.PutSmallFileRequestProto;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type;
import org.apache.hadoop.hdds.scm.ByteStringHelper;
import org.apache.hadoop.hdds.scm.ByteStringConversion;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.hdds.scm.container.common.helpers
.StorageContainerException;
Expand Down Expand Up @@ -102,6 +103,7 @@ public class KeyValueHandler extends Handler {
private final ChunkManager chunkManager;
private final VolumeChoosingPolicy volumeChoosingPolicy;
private final long maxContainerSize;
private final Function<ByteBuffer, ByteString> byteBufferToByteString;

// A lock that is held during container creation.
private final AutoCloseableLock containerCreationLock;
Expand All @@ -125,10 +127,8 @@ public KeyValueHandler(Configuration config, StateContext context,
// this handler lock is used for synchronizing createContainer Requests,
// so using a fair lock here.
containerCreationLock = new AutoCloseableLock(new ReentrantLock(true));
boolean isUnsafeByteOperationsEnabled = conf.getBoolean(
OzoneConfigKeys.OZONE_UNSAFEBYTEOPERATIONS_ENABLED,
OzoneConfigKeys.OZONE_UNSAFEBYTEOPERATIONS_ENABLED_DEFAULT);
ByteStringHelper.init(isUnsafeByteOperationsEnabled);
byteBufferToByteString =
ByteStringConversion.createByteBufferConversion(conf);
}

@VisibleForTesting
Expand Down Expand Up @@ -547,15 +547,15 @@ ContainerCommandResponseProto handleReadChunk(
}

// The container can become unhealthy after the lock is released.
// The operation will likely fail/timeout in that happens.
// The operation will likely fail/timeout if that happens.
try {
checkContainerIsHealthy(kvContainer);
} catch (StorageContainerException sce) {
return ContainerUtils.logAndReturnError(LOG, sce, request);
}

ChunkInfo chunkInfo;
byte[] data;
ByteBuffer data;
try {
BlockID blockID = BlockID.getFromProtobuf(
request.getReadChunk().getBlockID());
Expand All @@ -569,7 +569,7 @@ ContainerCommandResponseProto handleReadChunk(

data = chunkManager
.readChunk(kvContainer, blockID, chunkInfo, dispatcherContext);
metrics.incContainerBytesStats(Type.ReadChunk, data.length);
metrics.incContainerBytesStats(Type.ReadChunk, chunkInfo.getLen());
} catch (StorageContainerException ex) {
return ContainerUtils.logAndReturnError(LOG, ex, request);
} catch (IOException ex) {
Expand All @@ -578,7 +578,18 @@ ContainerCommandResponseProto handleReadChunk(
request);
}

return ChunkUtils.getReadChunkResponse(request, data, chunkInfo);
Preconditions.checkNotNull(data, "Chunk data is null");

ContainerProtos.ReadChunkResponseProto.Builder response =
ContainerProtos.ReadChunkResponseProto.newBuilder();
response.setChunkData(chunkInfo.getProtoBufMessage());
response.setData(byteBufferToByteString.apply(data));
response.setBlockID(request.getReadChunk().getBlockID());

ContainerCommandResponseProto.Builder builder =
ContainerUtils.getSuccessResponseBuilder(request);
builder.setReadChunk(response);
return builder.build();
}

/**
Expand Down Expand Up @@ -800,9 +811,9 @@ ContainerCommandResponseProto handleGetSmallFile(
for (ContainerProtos.ChunkInfo chunk : responseData.getChunks()) {
// if the block is committed, all chunks must have been committed.
// Tmp chunk files won't exist here.
byte[] data = chunkManager.readChunk(kvContainer, blockID,
ByteBuffer data = chunkManager.readChunk(kvContainer, blockID,
ChunkInfo.getFromProtoBuf(chunk), dispatcherContext);
ByteString current = ByteString.copyFrom(data);
ByteString current = byteBufferToByteString.apply(data);
dataBuf = dataBuf.concat(current);
chunkInfo = chunk;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,9 +24,6 @@
.ContainerCommandRequestProto;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerCommandResponseProto;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ReadChunkResponseProto;
import org.apache.hadoop.hdds.scm.ByteStringHelper;
import org.apache.hadoop.hdds.scm.container.common.helpers
.StorageContainerException;
import org.apache.hadoop.io.IOUtils;
Expand Down Expand Up @@ -142,8 +139,7 @@ public static void writeData(File chunkFile, ChunkInfo chunkInfo,
* @return ByteBuffer
*/
public static ByteBuffer readData(File chunkFile, ChunkInfo data,
VolumeIOStats volumeIOStats) throws StorageContainerException,
ExecutionException, InterruptedException {
VolumeIOStats volumeIOStats) throws StorageContainerException {
Logger log = LoggerFactory.getLogger(ChunkManagerImpl.class);

if (!chunkFile.exists()) {
Expand All @@ -168,6 +164,7 @@ public static ByteBuffer readData(File chunkFile, ChunkInfo data,

try (FileLock ignored = file.lock(offset, len, true)) {
file.read(buf, offset);
buf.flip();
}

// Increment volumeIO stats here.
Expand Down Expand Up @@ -287,33 +284,6 @@ public static ContainerCommandResponseProto getChunkResponseSuccess(
return ContainerUtils.getSuccessResponse(msg);
}

/**
* Gets a response to the read chunk calls.
*
* @param msg - Msg
* @param data - Data
* @param info - Info
* @return Response.
*/
public static ContainerCommandResponseProto getReadChunkResponse(
ContainerCommandRequestProto msg, byte[] data, ChunkInfo info) {
Preconditions.checkNotNull(msg);
Preconditions.checkNotNull(data, "Chunk data is null");
Preconditions.checkNotNull(info, "Chunk Info is null");

ReadChunkResponseProto.Builder response =
ReadChunkResponseProto.newBuilder();
response.setChunkData(info.getProtoBufMessage());
response.setData(
ByteStringHelper.getByteString(data));
response.setBlockID(msg.getReadChunk().getBlockID());

ContainerCommandResponseProto.Builder builder =
ContainerUtils.getSuccessResponseBuilder(msg);
builder.setReadChunk(response);
return builder.build();
}

@VisibleForTesting
static <T, E extends Exception> T processFileExclusively(
Path path, CheckedSupplier<T, E> op
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -120,8 +120,8 @@ public void writeChunk(Container container, BlockID blockID, ChunkInfo info,
* TODO: Explore if we need to do that for ozone.
*/
@Override
public byte[] readChunk(Container container, BlockID blockID, ChunkInfo info,
DispatcherContext dispatcherContext) {
public ByteBuffer readChunk(Container container, BlockID blockID,
ChunkInfo info, DispatcherContext dispatcherContext) {

long readStartTime = Time.monotonicNow();

Expand All @@ -138,7 +138,7 @@ public byte[] readChunk(Container container, BlockID blockID, ChunkInfo info,
volumeIOStats.incReadOpCount();
volumeIOStats.incReadBytes(info.getLen());

return data.array();
return data;
}

/**
Expand Down
Loading

0 comments on commit 1d27930

Please sign in to comment.