HDFS-11196. Ozone: Improve logging and error handling in the container layer. Contributed by Anu Engineer.
This commit is contained in:
parent
32cc2b8f1a
commit
386dbc11da
|
@ -0,0 +1,104 @@
|
||||||
|
/*
|
||||||
|
* 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.scm.container.common.helpers;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Exceptions thrown from the Storage Container.
|
||||||
|
*/
|
||||||
|
public class StorageContainerException extends IOException {
|
||||||
|
private ContainerProtos.Result result;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructs an {@code IOException} with {@code null}
|
||||||
|
* as its error detail message.
|
||||||
|
*/
|
||||||
|
public StorageContainerException(ContainerProtos.Result result) {
|
||||||
|
this.result = result;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructs an {@code IOException} with the specified detail message.
|
||||||
|
*
|
||||||
|
* @param message The detail message (which is saved for later retrieval by
|
||||||
|
* the {@link #getMessage()} method)
|
||||||
|
* @param result - The result code
|
||||||
|
*/
|
||||||
|
public StorageContainerException(String message,
|
||||||
|
ContainerProtos.Result result) {
|
||||||
|
super(message);
|
||||||
|
this.result = result;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructs an {@code IOException} with the specified detail message
|
||||||
|
* and cause.
|
||||||
|
* <p>
|
||||||
|
* <p> Note that the detail message associated with {@code cause} is
|
||||||
|
* <i>not</i> automatically incorporated into this exception's detail
|
||||||
|
* message.
|
||||||
|
*
|
||||||
|
* @param message The detail message (which is saved for later retrieval by
|
||||||
|
* the {@link #getMessage()} method)
|
||||||
|
*
|
||||||
|
* @param cause The cause (which is saved for later retrieval by the {@link
|
||||||
|
* #getCause()} method). (A null value is permitted, and indicates that the
|
||||||
|
* cause is nonexistent or unknown.)
|
||||||
|
*
|
||||||
|
* @param result - The result code
|
||||||
|
* @since 1.6
|
||||||
|
*/
|
||||||
|
public StorageContainerException(String message, Throwable cause,
|
||||||
|
ContainerProtos.Result result) {
|
||||||
|
super(message, cause);
|
||||||
|
this.result = result;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructs an {@code IOException} with the specified cause and a
|
||||||
|
* detail message of {@code (cause==null ? null : cause.toString())}
|
||||||
|
* (which typically contains the class and detail message of {@code cause}).
|
||||||
|
* This constructor is useful for IO exceptions that are little more
|
||||||
|
* than wrappers for other throwables.
|
||||||
|
*
|
||||||
|
* @param cause The cause (which is saved for later retrieval by the {@link
|
||||||
|
* #getCause()} method). (A null value is permitted, and indicates that the
|
||||||
|
* cause is nonexistent or unknown.)
|
||||||
|
* @param result - The result code
|
||||||
|
* @since 1.6
|
||||||
|
*/
|
||||||
|
public StorageContainerException(Throwable cause, ContainerProtos.Result
|
||||||
|
result) {
|
||||||
|
super(cause);
|
||||||
|
this.result = result;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns Result.
|
||||||
|
*
|
||||||
|
* @return Result.
|
||||||
|
*/
|
||||||
|
public ContainerProtos.Result getResult() {
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
}
|
|
@ -18,13 +18,7 @@
|
||||||
|
|
||||||
package org.apache.hadoop.scm.storage;
|
package org.apache.hadoop.scm.storage;
|
||||||
|
|
||||||
import static java.net.HttpURLConnection.HTTP_BAD_REQUEST;
|
|
||||||
import static java.net.HttpURLConnection.HTTP_INTERNAL_ERROR;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
import com.google.protobuf.ByteString;
|
import com.google.protobuf.ByteString;
|
||||||
|
|
||||||
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos;
|
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos;
|
||||||
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.ChunkInfo;
|
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.ChunkInfo;
|
||||||
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
||||||
|
@ -35,9 +29,15 @@ import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
||||||
.GetKeyRequestProto;
|
.GetKeyRequestProto;
|
||||||
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
||||||
.GetKeyResponseProto;
|
.GetKeyResponseProto;
|
||||||
|
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
||||||
|
.GetSmallFileRequestProto;
|
||||||
|
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
||||||
|
.GetSmallFileResponseProto;
|
||||||
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.KeyData;
|
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.KeyData;
|
||||||
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
||||||
.PutKeyRequestProto;
|
.PutKeyRequestProto;
|
||||||
|
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
||||||
|
.PutSmallFileRequestProto;
|
||||||
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
||||||
.ReadChunkRequestProto;
|
.ReadChunkRequestProto;
|
||||||
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
||||||
|
@ -45,13 +45,10 @@ import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
||||||
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.Type;
|
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.Type;
|
||||||
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
||||||
.WriteChunkRequestProto;
|
.WriteChunkRequestProto;
|
||||||
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
|
||||||
.PutSmallFileRequestProto;
|
|
||||||
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
|
||||||
.GetSmallFileResponseProto;
|
|
||||||
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
|
||||||
.GetSmallFileRequestProto;
|
|
||||||
import org.apache.hadoop.scm.XceiverClient;
|
import org.apache.hadoop.scm.XceiverClient;
|
||||||
|
import org.apache.hadoop.scm.container.common.helpers.StorageContainerException;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Implementation of all container protocol calls performed by Container
|
* Implementation of all container protocol calls performed by Container
|
||||||
|
@ -87,7 +84,7 @@ public final class ContainerProtocolCalls {
|
||||||
.setGetKey(readKeyRequest)
|
.setGetKey(readKeyRequest)
|
||||||
.build();
|
.build();
|
||||||
ContainerCommandResponseProto response = xceiverClient.sendCommand(request);
|
ContainerCommandResponseProto response = xceiverClient.sendCommand(request);
|
||||||
validateContainerResponse(response, traceID);
|
validateContainerResponse(response);
|
||||||
return response.getGetKey();
|
return response.getGetKey();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -112,7 +109,7 @@ public final class ContainerProtocolCalls {
|
||||||
.setPutKey(createKeyRequest)
|
.setPutKey(createKeyRequest)
|
||||||
.build();
|
.build();
|
||||||
ContainerCommandResponseProto response = xceiverClient.sendCommand(request);
|
ContainerCommandResponseProto response = xceiverClient.sendCommand(request);
|
||||||
validateContainerResponse(response, traceID);
|
validateContainerResponse(response);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -140,7 +137,7 @@ public final class ContainerProtocolCalls {
|
||||||
.setReadChunk(readChunkRequest)
|
.setReadChunk(readChunkRequest)
|
||||||
.build();
|
.build();
|
||||||
ContainerCommandResponseProto response = xceiverClient.sendCommand(request);
|
ContainerCommandResponseProto response = xceiverClient.sendCommand(request);
|
||||||
validateContainerResponse(response, traceID);
|
validateContainerResponse(response);
|
||||||
return response.getReadChunk();
|
return response.getReadChunk();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -170,7 +167,7 @@ public final class ContainerProtocolCalls {
|
||||||
.setWriteChunk(writeChunkRequest)
|
.setWriteChunk(writeChunkRequest)
|
||||||
.build();
|
.build();
|
||||||
ContainerCommandResponseProto response = xceiverClient.sendCommand(request);
|
ContainerCommandResponseProto response = xceiverClient.sendCommand(request);
|
||||||
validateContainerResponse(response, traceID);
|
validateContainerResponse(response);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -218,7 +215,7 @@ public final class ContainerProtocolCalls {
|
||||||
.setPutSmallFile(putSmallFileRequest)
|
.setPutSmallFile(putSmallFileRequest)
|
||||||
.build();
|
.build();
|
||||||
ContainerCommandResponseProto response = client.sendCommand(request);
|
ContainerCommandResponseProto response = client.sendCommand(request);
|
||||||
validateContainerResponse(response, traceID);
|
validateContainerResponse(response);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -245,7 +242,7 @@ public final class ContainerProtocolCalls {
|
||||||
request.setTraceID(traceID);
|
request.setTraceID(traceID);
|
||||||
ContainerCommandResponseProto response = client.sendCommand(
|
ContainerCommandResponseProto response = client.sendCommand(
|
||||||
request.build());
|
request.build());
|
||||||
validateContainerResponse(response, traceID);
|
validateContainerResponse(response);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -280,7 +277,7 @@ public final class ContainerProtocolCalls {
|
||||||
.setGetSmallFile(getSmallFileRequest)
|
.setGetSmallFile(getSmallFileRequest)
|
||||||
.build();
|
.build();
|
||||||
ContainerCommandResponseProto response = client.sendCommand(request);
|
ContainerCommandResponseProto response = client.sendCommand(request);
|
||||||
validateContainerResponse(response, traceID);
|
validateContainerResponse(response);
|
||||||
return response.getGetSmallFile();
|
return response.getGetSmallFile();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -289,28 +286,15 @@ public final class ContainerProtocolCalls {
|
||||||
* return code is mapped to a corresponding exception and thrown.
|
* return code is mapped to a corresponding exception and thrown.
|
||||||
*
|
*
|
||||||
* @param response container protocol call response
|
* @param response container protocol call response
|
||||||
* @param traceID container protocol call args
|
|
||||||
* @throws IOException if the container protocol call failed
|
* @throws IOException if the container protocol call failed
|
||||||
*/
|
*/
|
||||||
private static void validateContainerResponse(
|
private static void validateContainerResponse(
|
||||||
ContainerCommandResponseProto response, String traceID
|
ContainerCommandResponseProto response
|
||||||
) throws IOException {
|
) throws StorageContainerException {
|
||||||
// TODO : throw the right type of exception
|
if (response.getResult() == ContainerProtos.Result.SUCCESS) {
|
||||||
switch (response.getResult()) {
|
return;
|
||||||
case SUCCESS:
|
|
||||||
break;
|
|
||||||
case MALFORMED_REQUEST:
|
|
||||||
throw new IOException(HTTP_BAD_REQUEST +
|
|
||||||
":Bad container request: " + traceID);
|
|
||||||
case UNSUPPORTED_REQUEST:
|
|
||||||
throw new IOException(HTTP_INTERNAL_ERROR +
|
|
||||||
"Unsupported container request: " + traceID);
|
|
||||||
case CONTAINER_INTERNAL_ERROR:
|
|
||||||
throw new IOException(HTTP_INTERNAL_ERROR +
|
|
||||||
"Container internal error:" + traceID);
|
|
||||||
default:
|
|
||||||
throw new IOException(HTTP_INTERNAL_ERROR +
|
|
||||||
"Unrecognized container response:" + traceID);
|
|
||||||
}
|
}
|
||||||
|
throw new StorageContainerException(
|
||||||
|
response.getMessage(), response.getResult());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -99,6 +99,23 @@ enum Result {
|
||||||
UNSUPPORTED_REQUEST = 2;
|
UNSUPPORTED_REQUEST = 2;
|
||||||
MALFORMED_REQUEST = 3;
|
MALFORMED_REQUEST = 3;
|
||||||
CONTAINER_INTERNAL_ERROR = 4;
|
CONTAINER_INTERNAL_ERROR = 4;
|
||||||
|
INVALID_CONFIG = 5;
|
||||||
|
INVALID_FILE_HASH_FOUND = 6;
|
||||||
|
CONTAINER_EXISTS = 7;
|
||||||
|
NO_SUCH_ALGORITHM = 8;
|
||||||
|
CONTAINER_NOT_FOUND = 9;
|
||||||
|
IO_EXCEPTION = 10;
|
||||||
|
UNABLE_TO_READ_METADATA_DB = 11;
|
||||||
|
NO_SUCH_KEY = 12;
|
||||||
|
OVERWRITE_FLAG_REQUIRED = 13;
|
||||||
|
UNABLE_TO_FIND_DATA_DIR = 14;
|
||||||
|
INVALID_WRITE_SIZE = 15;
|
||||||
|
CHECKSUM_MISMATCH = 16;
|
||||||
|
UNABLE_TO_FIND_CHUNK = 17;
|
||||||
|
PROTOC_DECODING_ERROR = 18;
|
||||||
|
INVALID_ARGUMENT = 19;
|
||||||
|
PUT_SMALL_FILE_ERROR = 20;
|
||||||
|
GET_SMALL_FILE_ERROR = 21;
|
||||||
}
|
}
|
||||||
|
|
||||||
message ContainerCommandRequestProto {
|
message ContainerCommandRequestProto {
|
||||||
|
|
|
@ -24,6 +24,7 @@ import org.apache.commons.codec.digest.DigestUtils;
|
||||||
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos;
|
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos;
|
||||||
import org.apache.hadoop.io.IOUtils;
|
import org.apache.hadoop.io.IOUtils;
|
||||||
import org.apache.hadoop.ozone.OzoneConsts;
|
import org.apache.hadoop.ozone.OzoneConsts;
|
||||||
|
import org.apache.hadoop.scm.container.common.helpers.StorageContainerException;
|
||||||
import org.apache.hadoop.ozone.container.common.impl.ChunkManagerImpl;
|
import org.apache.hadoop.ozone.container.common.impl.ChunkManagerImpl;
|
||||||
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
|
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
|
@ -39,6 +40,23 @@ import java.security.MessageDigest;
|
||||||
import java.security.NoSuchAlgorithmException;
|
import java.security.NoSuchAlgorithmException;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
|
|
||||||
|
import static org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
||||||
|
.Result.CHECKSUM_MISMATCH;
|
||||||
|
import static org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
||||||
|
.Result.CONTAINER_INTERNAL_ERROR;
|
||||||
|
import static org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
||||||
|
.Result.CONTAINER_NOT_FOUND;
|
||||||
|
import static org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
||||||
|
.Result.INVALID_WRITE_SIZE;
|
||||||
|
import static org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
||||||
|
.Result.IO_EXCEPTION;
|
||||||
|
import static org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
||||||
|
.Result.OVERWRITE_FLAG_REQUIRED;
|
||||||
|
import static org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
||||||
|
.Result.UNABLE_TO_FIND_CHUNK;
|
||||||
|
import static org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
||||||
|
.Result.UNABLE_TO_FIND_DATA_DIR;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Set of utility functions used by the chunk Manager.
|
* Set of utility functions used by the chunk Manager.
|
||||||
*/
|
*/
|
||||||
|
@ -90,10 +108,10 @@ public final class ChunkUtils {
|
||||||
* @param data - container data.
|
* @param data - container data.
|
||||||
* @param info - chunk info.
|
* @param info - chunk info.
|
||||||
* @return File
|
* @return File
|
||||||
* @throws IOException
|
* @throws StorageContainerException
|
||||||
*/
|
*/
|
||||||
public static File validateChunk(Pipeline pipeline, ContainerData data,
|
public static File validateChunk(Pipeline pipeline, ContainerData data,
|
||||||
ChunkInfo info) throws IOException {
|
ChunkInfo info) throws StorageContainerException {
|
||||||
|
|
||||||
Logger log = LoggerFactory.getLogger(ChunkManagerImpl.class);
|
Logger log = LoggerFactory.getLogger(ChunkManagerImpl.class);
|
||||||
|
|
||||||
|
@ -102,8 +120,9 @@ public final class ChunkUtils {
|
||||||
if (!ChunkUtils.isOverWritePermitted(info)) {
|
if (!ChunkUtils.isOverWritePermitted(info)) {
|
||||||
log.error("Rejecting write chunk request. Chunk overwrite " +
|
log.error("Rejecting write chunk request. Chunk overwrite " +
|
||||||
"without explicit request. {}", info.toString());
|
"without explicit request. {}", info.toString());
|
||||||
throw new IOException("Rejecting write chunk request. OverWrite " +
|
throw new StorageContainerException("Rejecting write chunk request. " +
|
||||||
"flag required." + info.toString());
|
"OverWrite flag required." + info.toString(),
|
||||||
|
OVERWRITE_FLAG_REQUIRED);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return chunkFile;
|
return chunkFile;
|
||||||
|
@ -116,22 +135,24 @@ public final class ChunkUtils {
|
||||||
* @param data - Container Data
|
* @param data - Container Data
|
||||||
* @param info - Chunk info
|
* @param info - Chunk info
|
||||||
* @return - File.
|
* @return - File.
|
||||||
* @throws IOException
|
* @throws StorageContainerException
|
||||||
*/
|
*/
|
||||||
public static File getChunkFile(Pipeline pipeline, ContainerData data,
|
public static File getChunkFile(Pipeline pipeline, ContainerData data,
|
||||||
ChunkInfo info) throws IOException {
|
ChunkInfo info) throws StorageContainerException {
|
||||||
|
|
||||||
Logger log = LoggerFactory.getLogger(ChunkManagerImpl.class);
|
Logger log = LoggerFactory.getLogger(ChunkManagerImpl.class);
|
||||||
if (data == null) {
|
if (data == null) {
|
||||||
log.error("Invalid container Name: {}", pipeline.getContainerName());
|
log.error("Invalid container Name: {}", pipeline.getContainerName());
|
||||||
throw new IOException("Unable to find the container Name: " +
|
throw new StorageContainerException("Unable to find the container Name:" +
|
||||||
pipeline.getContainerName());
|
" " +
|
||||||
|
pipeline.getContainerName(), CONTAINER_NOT_FOUND);
|
||||||
}
|
}
|
||||||
|
|
||||||
File dataDir = ContainerUtils.getDataDirectory(data).toFile();
|
File dataDir = ContainerUtils.getDataDirectory(data).toFile();
|
||||||
if (!dataDir.exists()) {
|
if (!dataDir.exists()) {
|
||||||
log.error("Unable to find the data directory: {}", dataDir);
|
log.error("Unable to find the data directory: {}", dataDir);
|
||||||
throw new IOException("Unable to find the data directory: " + dataDir);
|
throw new StorageContainerException("Unable to find the data directory:" +
|
||||||
|
" " + dataDir, UNABLE_TO_FIND_DATA_DIR);
|
||||||
}
|
}
|
||||||
|
|
||||||
return dataDir.toPath().resolve(info.getChunkName()).toFile();
|
return dataDir.toPath().resolve(info.getChunkName()).toFile();
|
||||||
|
@ -143,11 +164,12 @@ public final class ChunkUtils {
|
||||||
*
|
*
|
||||||
* @param chunkFile - File to write data to.
|
* @param chunkFile - File to write data to.
|
||||||
* @param chunkInfo - Data stream to write.
|
* @param chunkInfo - Data stream to write.
|
||||||
* @throws IOException
|
* @param data - The data buffer.
|
||||||
|
* @throws StorageContainerException
|
||||||
*/
|
*/
|
||||||
public static void writeData(File chunkFile, ChunkInfo chunkInfo,
|
public static void writeData(File chunkFile, ChunkInfo chunkInfo,
|
||||||
byte[] data)
|
byte[] data) throws
|
||||||
throws IOException, ExecutionException, InterruptedException,
|
StorageContainerException, ExecutionException, InterruptedException,
|
||||||
NoSuchAlgorithmException {
|
NoSuchAlgorithmException {
|
||||||
|
|
||||||
Logger log = LoggerFactory.getLogger(ChunkManagerImpl.class);
|
Logger log = LoggerFactory.getLogger(ChunkManagerImpl.class);
|
||||||
|
@ -156,7 +178,7 @@ public final class ChunkUtils {
|
||||||
"specified. DataLen: %d Byte Array: %d",
|
"specified. DataLen: %d Byte Array: %d",
|
||||||
chunkInfo.getLen(), data.length);
|
chunkInfo.getLen(), data.length);
|
||||||
log.error(err);
|
log.error(err);
|
||||||
throw new IOException(err);
|
throw new StorageContainerException(err, INVALID_WRITE_SIZE);
|
||||||
}
|
}
|
||||||
|
|
||||||
AsynchronousFileChannel file = null;
|
AsynchronousFileChannel file = null;
|
||||||
|
@ -175,15 +197,24 @@ public final class ChunkUtils {
|
||||||
verifyChecksum(chunkInfo, data, log);
|
verifyChecksum(chunkInfo, data, log);
|
||||||
}
|
}
|
||||||
int size = file.write(ByteBuffer.wrap(data), chunkInfo.getOffset()).get();
|
int size = file.write(ByteBuffer.wrap(data), chunkInfo.getOffset()).get();
|
||||||
if(size != data.length) {
|
if (size != data.length) {
|
||||||
log.error("Invalid write size found. Size:{} Expected: {} " , size,
|
log.error("Invalid write size found. Size:{} Expected: {} ", size,
|
||||||
data.length);
|
data.length);
|
||||||
throw new IOException("Invalid write size found. Size: " + size
|
throw new StorageContainerException("Invalid write size found. " +
|
||||||
+ " Expected: " + data.length);
|
"Size: " + size + " Expected: " + data.length, INVALID_WRITE_SIZE);
|
||||||
}
|
}
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new StorageContainerException(e, IO_EXCEPTION);
|
||||||
|
|
||||||
} finally {
|
} finally {
|
||||||
if (lock != null) {
|
if (lock != null) {
|
||||||
|
try {
|
||||||
lock.release();
|
lock.release();
|
||||||
|
} catch (IOException e) {
|
||||||
|
log.error("Unable to release lock ??, Fatal Error.");
|
||||||
|
throw new StorageContainerException(e, CONTAINER_INTERNAL_ERROR);
|
||||||
|
|
||||||
|
}
|
||||||
}
|
}
|
||||||
if (file != null) {
|
if (file != null) {
|
||||||
IOUtils.closeStream(file);
|
IOUtils.closeStream(file);
|
||||||
|
@ -198,19 +229,19 @@ public final class ChunkUtils {
|
||||||
* @param data - data buffer
|
* @param data - data buffer
|
||||||
* @param log - log
|
* @param log - log
|
||||||
* @throws NoSuchAlgorithmException
|
* @throws NoSuchAlgorithmException
|
||||||
* @throws IOException
|
* @throws StorageContainerException
|
||||||
*/
|
*/
|
||||||
private static void verifyChecksum(ChunkInfo chunkInfo, byte[] data, Logger
|
private static void verifyChecksum(ChunkInfo chunkInfo, byte[] data, Logger
|
||||||
log) throws NoSuchAlgorithmException, IOException {
|
log) throws NoSuchAlgorithmException, StorageContainerException {
|
||||||
MessageDigest sha = MessageDigest.getInstance(OzoneConsts.FILE_HASH);
|
MessageDigest sha = MessageDigest.getInstance(OzoneConsts.FILE_HASH);
|
||||||
sha.update(data);
|
sha.update(data);
|
||||||
if (!Hex.encodeHexString(sha.digest()).equals(
|
if (!Hex.encodeHexString(sha.digest()).equals(
|
||||||
chunkInfo.getChecksum())) {
|
chunkInfo.getChecksum())) {
|
||||||
log.error("Checksum mismatch. Provided: {} , computed: {}",
|
log.error("Checksum mismatch. Provided: {} , computed: {}",
|
||||||
chunkInfo.getChecksum(), DigestUtils.sha256Hex(sha.digest()));
|
chunkInfo.getChecksum(), DigestUtils.sha256Hex(sha.digest()));
|
||||||
throw new IOException("Checksum mismatch. Provided: " +
|
throw new StorageContainerException("Checksum mismatch. Provided: " +
|
||||||
chunkInfo.getChecksum() + " , computed: " +
|
chunkInfo.getChecksum() + " , computed: " +
|
||||||
DigestUtils.sha256Hex(sha.digest()));
|
DigestUtils.sha256Hex(sha.digest()), CHECKSUM_MISMATCH);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -219,21 +250,24 @@ public final class ChunkUtils {
|
||||||
*
|
*
|
||||||
* @param chunkFile - file where data lives.
|
* @param chunkFile - file where data lives.
|
||||||
* @param data - chunk definition.
|
* @param data - chunk definition.
|
||||||
|
*
|
||||||
* @return ByteBuffer
|
* @return ByteBuffer
|
||||||
* @throws IOException
|
*
|
||||||
|
* @throws StorageContainerException
|
||||||
* @throws ExecutionException
|
* @throws ExecutionException
|
||||||
* @throws InterruptedException
|
* @throws InterruptedException
|
||||||
*/
|
*/
|
||||||
public static ByteBuffer readData(File chunkFile, ChunkInfo data) throws
|
public static ByteBuffer readData(File chunkFile, ChunkInfo data) throws
|
||||||
IOException, ExecutionException, InterruptedException,
|
StorageContainerException, ExecutionException, InterruptedException,
|
||||||
NoSuchAlgorithmException {
|
NoSuchAlgorithmException {
|
||||||
Logger log = LoggerFactory.getLogger(ChunkManagerImpl.class);
|
Logger log = LoggerFactory.getLogger(ChunkManagerImpl.class);
|
||||||
|
|
||||||
if (!chunkFile.exists()) {
|
if (!chunkFile.exists()) {
|
||||||
log.error("Unable to find the chunk file. chunk info : {}",
|
log.error("Unable to find the chunk file. chunk info : {}",
|
||||||
data.toString());
|
data.toString());
|
||||||
throw new IOException("Unable to find the chunk file. chunk info " +
|
throw new StorageContainerException("Unable to find the chunk file. " +
|
||||||
data.toString());
|
"chunk info " +
|
||||||
|
data.toString(), UNABLE_TO_FIND_CHUNK);
|
||||||
}
|
}
|
||||||
|
|
||||||
AsynchronousFileChannel file = null;
|
AsynchronousFileChannel file = null;
|
||||||
|
@ -252,9 +286,15 @@ public final class ChunkUtils {
|
||||||
}
|
}
|
||||||
|
|
||||||
return buf;
|
return buf;
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new StorageContainerException(e, IO_EXCEPTION);
|
||||||
} finally {
|
} finally {
|
||||||
if (lock != null) {
|
if (lock != null) {
|
||||||
|
try {
|
||||||
lock.release();
|
lock.release();
|
||||||
|
} catch (IOException e) {
|
||||||
|
log.error("I/O error is lock release.");
|
||||||
|
}
|
||||||
}
|
}
|
||||||
if (file != null) {
|
if (file != null) {
|
||||||
IOUtils.closeStream(file);
|
IOUtils.closeStream(file);
|
||||||
|
@ -276,6 +316,7 @@ public final class ChunkUtils {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Gets a response to the read chunk calls.
|
* Gets a response to the read chunk calls.
|
||||||
|
*
|
||||||
* @param msg - Msg
|
* @param msg - Msg
|
||||||
* @param data - Data
|
* @param data - Data
|
||||||
* @param info - Info
|
* @param info - Info
|
||||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos;
|
||||||
import org.apache.hadoop.ozone.OzoneConsts;
|
import org.apache.hadoop.ozone.OzoneConsts;
|
||||||
import org.apache.hadoop.ozone.container.common.impl.ContainerManagerImpl;
|
import org.apache.hadoop.ozone.container.common.impl.ContainerManagerImpl;
|
||||||
import org.apache.hadoop.utils.LevelDBStore;
|
import org.apache.hadoop.utils.LevelDBStore;
|
||||||
|
import org.apache.hadoop.scm.container.common.helpers.StorageContainerException;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
@ -34,6 +35,10 @@ import java.nio.file.Path;
|
||||||
import java.nio.file.Paths;
|
import java.nio.file.Paths;
|
||||||
|
|
||||||
import static org.apache.commons.io.FilenameUtils.removeExtension;
|
import static org.apache.commons.io.FilenameUtils.removeExtension;
|
||||||
|
import static org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
||||||
|
.Result.INVALID_ARGUMENT;
|
||||||
|
import static org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
||||||
|
.Result.UNABLE_TO_FIND_DATA_DIR;
|
||||||
import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_EXTENSION;
|
import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_EXTENSION;
|
||||||
import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_META;
|
import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_META;
|
||||||
|
|
||||||
|
@ -42,6 +47,10 @@ import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_META;
|
||||||
*/
|
*/
|
||||||
public final class ContainerUtils {
|
public final class ContainerUtils {
|
||||||
|
|
||||||
|
private ContainerUtils() {
|
||||||
|
//never constructed.
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns a CreateContainer Response. This call is used by create and delete
|
* Returns a CreateContainer Response. This call is used by create and delete
|
||||||
* containers which have null success responses.
|
* containers which have null success responses.
|
||||||
|
@ -98,6 +107,37 @@ public final class ContainerUtils {
|
||||||
.setMessage(message);
|
.setMessage(message);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Logs the error and returns a response to the caller.
|
||||||
|
*
|
||||||
|
* @param log - Logger
|
||||||
|
* @param ex - Exception
|
||||||
|
* @param msg - Request Object
|
||||||
|
* @return Response
|
||||||
|
*/
|
||||||
|
public static ContainerProtos.ContainerCommandResponseProto logAndReturnError(
|
||||||
|
Logger log, StorageContainerException ex,
|
||||||
|
ContainerProtos.ContainerCommandRequestProto msg) {
|
||||||
|
log.info("Trace ID: {} : Message: {} : Result: {}", msg.getTraceID(),
|
||||||
|
ex.getMessage(), ex.getResult().getValueDescriptor().getName());
|
||||||
|
return getContainerResponse(msg, ex.getResult(), ex.getMessage()).build();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Logs the error and returns a response to the caller.
|
||||||
|
*
|
||||||
|
* @param log - Logger
|
||||||
|
* @param ex - Exception
|
||||||
|
* @param msg - Request Object
|
||||||
|
* @return Response
|
||||||
|
*/
|
||||||
|
public static ContainerProtos.ContainerCommandResponseProto logAndReturnError(
|
||||||
|
Logger log, RuntimeException ex,
|
||||||
|
ContainerProtos.ContainerCommandRequestProto msg) {
|
||||||
|
log.info("Trace ID: {} : Message: {} ", msg.getTraceID(), ex.getMessage());
|
||||||
|
return getContainerResponse(msg, INVALID_ARGUMENT, ex.getMessage()).build();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* We found a command type but no associated payload for the command. Hence
|
* We found a command type but no associated payload for the command. Hence
|
||||||
* return malformed Command as response.
|
* return malformed Command as response.
|
||||||
|
@ -133,7 +173,6 @@ public final class ContainerUtils {
|
||||||
Preconditions.checkNotNull(containerFile);
|
Preconditions.checkNotNull(containerFile);
|
||||||
return Paths.get(containerFile.getParent()).resolve(
|
return Paths.get(containerFile.getParent()).resolve(
|
||||||
removeExtension(containerFile.getName())).toString();
|
removeExtension(containerFile.getName())).toString();
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -239,6 +278,7 @@ public final class ContainerUtils {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns container file location.
|
* Returns container file location.
|
||||||
|
*
|
||||||
* @param containerData - Data
|
* @param containerData - Data
|
||||||
* @param location - Root path
|
* @param location - Root path
|
||||||
* @return Path
|
* @return Path
|
||||||
|
@ -252,6 +292,7 @@ public final class ContainerUtils {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Container metadata directory -- here is where the level DB lives.
|
* Container metadata directory -- here is where the level DB lives.
|
||||||
|
*
|
||||||
* @param cData - cData.
|
* @param cData - cData.
|
||||||
* @return Path to the parent directory where the DB lives.
|
* @return Path to the parent directory where the DB lives.
|
||||||
*/
|
*/
|
||||||
|
@ -264,20 +305,23 @@ public final class ContainerUtils {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns the path where data or chunks live for a given container.
|
* Returns the path where data or chunks live for a given container.
|
||||||
|
*
|
||||||
* @param cData - cData container
|
* @param cData - cData container
|
||||||
* @return - Path
|
* @return - Path
|
||||||
|
* @throws StorageContainerException
|
||||||
*/
|
*/
|
||||||
public static Path getDataDirectory(ContainerData cData) throws IOException {
|
public static Path getDataDirectory(ContainerData cData)
|
||||||
|
throws StorageContainerException {
|
||||||
Path path = getMetadataDirectory(cData);
|
Path path = getMetadataDirectory(cData);
|
||||||
Preconditions.checkNotNull(path);
|
Preconditions.checkNotNull(path);
|
||||||
path = path.getParent();
|
Path parentPath = path.getParent();
|
||||||
if(path == null) {
|
if (parentPath == null) {
|
||||||
throw new IOException("Unable to get Data directory. null path found");
|
throw new StorageContainerException("Unable to get Data directory."
|
||||||
|
+ path, UNABLE_TO_FIND_DATA_DIR);
|
||||||
}
|
}
|
||||||
return path.resolve(OzoneConsts.CONTAINER_DATA_PATH);
|
return parentPath.resolve(OzoneConsts.CONTAINER_DATA_PATH);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* remove Container if it is empty.
|
* remove Container if it is empty.
|
||||||
* <p/>
|
* <p/>
|
||||||
|
@ -323,8 +367,4 @@ public final class ContainerUtils {
|
||||||
FileUtils.forceDelete(containerPath.toFile());
|
FileUtils.forceDelete(containerPath.toFile());
|
||||||
FileUtils.forceDelete(metaPath.toFile());
|
FileUtils.forceDelete(metaPath.toFile());
|
||||||
}
|
}
|
||||||
|
|
||||||
private ContainerUtils() {
|
|
||||||
//never constructed.
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.hadoop.ozone.container.common.helpers;
|
||||||
|
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos;
|
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos;
|
||||||
|
import org.apache.hadoop.scm.container.common.helpers.StorageContainerException;
|
||||||
import org.apache.hadoop.ozone.container.common.impl.KeyManagerImpl;
|
import org.apache.hadoop.ozone.container.common.impl.KeyManagerImpl;
|
||||||
import org.apache.hadoop.ozone.container.common.utils.ContainerCache;
|
import org.apache.hadoop.ozone.container.common.utils.ContainerCache;
|
||||||
import org.apache.hadoop.utils.LevelDBStore;
|
import org.apache.hadoop.utils.LevelDBStore;
|
||||||
|
@ -29,6 +30,9 @@ import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.nio.charset.Charset;
|
import java.nio.charset.Charset;
|
||||||
|
|
||||||
|
import static org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
||||||
|
.Result.UNABLE_TO_READ_METADATA_DB;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Utils functions to help key functions.
|
* Utils functions to help key functions.
|
||||||
*/
|
*/
|
||||||
|
@ -60,18 +64,25 @@ public final class KeyUtils {
|
||||||
* @param container - container.
|
* @param container - container.
|
||||||
* @param cache - cache
|
* @param cache - cache
|
||||||
* @return LevelDB handle.
|
* @return LevelDB handle.
|
||||||
* @throws IOException
|
* @throws StorageContainerException
|
||||||
*/
|
*/
|
||||||
public static LevelDBStore getDB(ContainerData container,
|
public static LevelDBStore getDB(ContainerData container,
|
||||||
ContainerCache cache) throws IOException {
|
ContainerCache cache)
|
||||||
|
throws StorageContainerException {
|
||||||
Preconditions.checkNotNull(container);
|
Preconditions.checkNotNull(container);
|
||||||
Preconditions.checkNotNull(cache);
|
Preconditions.checkNotNull(cache);
|
||||||
|
try {
|
||||||
LevelDBStore db = cache.getDB(container.getContainerName());
|
LevelDBStore db = cache.getDB(container.getContainerName());
|
||||||
if (db == null) {
|
if (db == null) {
|
||||||
db = getDB(container.getDBPath());
|
db = getDB(container.getDBPath());
|
||||||
cache.putDB(container.getContainerName(), db);
|
cache.putDB(container.getContainerName(), db);
|
||||||
}
|
}
|
||||||
return db;
|
return db;
|
||||||
|
} catch (IOException ex) {
|
||||||
|
String message = "Unable to open DB. DB Name: %s, Path: %s. ex: %s"
|
||||||
|
.format(container.getContainerName(), container.getDBPath(), ex);
|
||||||
|
throw new StorageContainerException(message, UNABLE_TO_READ_METADATA_DB);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -107,8 +118,8 @@ public final class KeyUtils {
|
||||||
|
|
||||||
|
|
||||||
public static ContainerProtos.ContainerCommandResponseProto
|
public static ContainerProtos.ContainerCommandResponseProto
|
||||||
getKeyDataResponse(ContainerProtos.ContainerCommandRequestProto msg
|
getKeyDataResponse(ContainerProtos.ContainerCommandRequestProto msg,
|
||||||
, KeyData data) {
|
KeyData data) {
|
||||||
ContainerProtos.GetKeyResponseProto.Builder getKey = ContainerProtos
|
ContainerProtos.GetKeyResponseProto.Builder getKey = ContainerProtos
|
||||||
.GetKeyResponseProto.newBuilder();
|
.GetKeyResponseProto.newBuilder();
|
||||||
getKey.setKeyData(data.getProtoBufMessage());
|
getKey.setKeyData(data.getProtoBufMessage());
|
||||||
|
|
|
@ -19,19 +19,24 @@ package org.apache.hadoop.ozone.container.common.impl;
|
||||||
|
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import org.apache.hadoop.fs.FileUtil;
|
import org.apache.hadoop.fs.FileUtil;
|
||||||
|
import org.apache.hadoop.scm.container.common.helpers.StorageContainerException;
|
||||||
import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
|
import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
|
||||||
import org.apache.hadoop.ozone.container.common.helpers.ChunkUtils;
|
import org.apache.hadoop.ozone.container.common.helpers.ChunkUtils;
|
||||||
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
|
|
||||||
import org.apache.hadoop.ozone.container.common.interfaces.ChunkManager;
|
import org.apache.hadoop.ozone.container.common.interfaces.ChunkManager;
|
||||||
import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
|
import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
|
||||||
|
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
|
||||||
import java.security.NoSuchAlgorithmException;
|
import java.security.NoSuchAlgorithmException;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
|
|
||||||
|
import static org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
||||||
|
.Result.CONTAINER_INTERNAL_ERROR;
|
||||||
|
import static org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
||||||
|
.Result.UNSUPPORTED_REQUEST;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* An implementation of ChunkManager that is used by default in ozone.
|
* An implementation of ChunkManager that is used by default in ozone.
|
||||||
*/
|
*/
|
||||||
|
@ -56,32 +61,34 @@ public class ChunkManagerImpl implements ChunkManager {
|
||||||
* @param pipeline - Name and the set of machines that make this container.
|
* @param pipeline - Name and the set of machines that make this container.
|
||||||
* @param keyName - Name of the Key.
|
* @param keyName - Name of the Key.
|
||||||
* @param info - ChunkInfo.
|
* @param info - ChunkInfo.
|
||||||
* @throws IOException
|
* @throws StorageContainerException
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public void writeChunk(Pipeline pipeline, String keyName, ChunkInfo info,
|
public void writeChunk(Pipeline pipeline, String keyName, ChunkInfo info,
|
||||||
byte[] data)
|
byte[] data)
|
||||||
throws IOException {
|
throws StorageContainerException {
|
||||||
|
|
||||||
// we don't want container manager to go away while we are writing chunks.
|
// we don't want container manager to go away while we are writing chunks.
|
||||||
containerManager.readLock();
|
containerManager.readLock();
|
||||||
|
|
||||||
// TODO : Take keyManager Write lock here.
|
// TODO : Take keyManager Write lock here.
|
||||||
try {
|
try {
|
||||||
Preconditions.checkNotNull(pipeline);
|
Preconditions.checkNotNull(pipeline, "Pipeline cannot be null");
|
||||||
Preconditions.checkNotNull(pipeline.getContainerName());
|
Preconditions.checkNotNull(pipeline.getContainerName(),
|
||||||
|
"Container name cannot be null");
|
||||||
File chunkFile = ChunkUtils.validateChunk(pipeline,
|
File chunkFile = ChunkUtils.validateChunk(pipeline,
|
||||||
containerManager.readContainer(pipeline.getContainerName()), info);
|
containerManager.readContainer(pipeline.getContainerName()), info);
|
||||||
ChunkUtils.writeData(chunkFile, info, data);
|
ChunkUtils.writeData(chunkFile, info, data);
|
||||||
|
|
||||||
} catch (ExecutionException |
|
} catch (ExecutionException | NoSuchAlgorithmException e) {
|
||||||
NoSuchAlgorithmException e) {
|
|
||||||
LOG.error("write data failed. error: {}", e);
|
LOG.error("write data failed. error: {}", e);
|
||||||
throw new IOException("Internal error: ", e);
|
throw new StorageContainerException("Internal error: ", e,
|
||||||
|
CONTAINER_INTERNAL_ERROR);
|
||||||
} catch (InterruptedException e) {
|
} catch (InterruptedException e) {
|
||||||
Thread.currentThread().interrupt();
|
Thread.currentThread().interrupt();
|
||||||
LOG.error("write data failed. error: {}", e);
|
LOG.error("write data failed. error: {}", e);
|
||||||
throw new IOException("Internal error: ", e);
|
throw new StorageContainerException("Internal error: ", e,
|
||||||
|
CONTAINER_INTERNAL_ERROR);
|
||||||
} finally {
|
} finally {
|
||||||
containerManager.readUnlock();
|
containerManager.readUnlock();
|
||||||
}
|
}
|
||||||
|
@ -94,13 +101,13 @@ public class ChunkManagerImpl implements ChunkManager {
|
||||||
* @param keyName - Name of the Key
|
* @param keyName - Name of the Key
|
||||||
* @param info - ChunkInfo.
|
* @param info - ChunkInfo.
|
||||||
* @return byte array
|
* @return byte array
|
||||||
* @throws IOException TODO: Right now we do not support partial reads and
|
* @throws StorageContainerException
|
||||||
* writes of chunks. TODO: Explore if we need to do that
|
* TODO: Right now we do not support partial reads and writes of chunks.
|
||||||
* for ozone.
|
* TODO: Explore if we need to do that for ozone.
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public byte[] readChunk(Pipeline pipeline, String keyName, ChunkInfo info)
|
public byte[] readChunk(Pipeline pipeline, String keyName, ChunkInfo info)
|
||||||
throws IOException {
|
throws StorageContainerException {
|
||||||
containerManager.readLock();
|
containerManager.readLock();
|
||||||
try {
|
try {
|
||||||
File chunkFile = ChunkUtils.getChunkFile(pipeline, containerManager
|
File chunkFile = ChunkUtils.getChunkFile(pipeline, containerManager
|
||||||
|
@ -108,11 +115,13 @@ public class ChunkManagerImpl implements ChunkManager {
|
||||||
return ChunkUtils.readData(chunkFile, info).array();
|
return ChunkUtils.readData(chunkFile, info).array();
|
||||||
} catch (ExecutionException | NoSuchAlgorithmException e) {
|
} catch (ExecutionException | NoSuchAlgorithmException e) {
|
||||||
LOG.error("read data failed. error: {}", e);
|
LOG.error("read data failed. error: {}", e);
|
||||||
throw new IOException("Internal error: ", e);
|
throw new StorageContainerException("Internal error: ",
|
||||||
|
e, CONTAINER_INTERNAL_ERROR);
|
||||||
} catch (InterruptedException e) {
|
} catch (InterruptedException e) {
|
||||||
Thread.currentThread().interrupt();
|
Thread.currentThread().interrupt();
|
||||||
LOG.error("read data failed. error: {}", e);
|
LOG.error("read data failed. error: {}", e);
|
||||||
throw new IOException("Internal error: ", e);
|
throw new StorageContainerException("Internal error: ",
|
||||||
|
e, CONTAINER_INTERNAL_ERROR);
|
||||||
} finally {
|
} finally {
|
||||||
containerManager.readUnlock();
|
containerManager.readUnlock();
|
||||||
}
|
}
|
||||||
|
@ -124,11 +133,11 @@ public class ChunkManagerImpl implements ChunkManager {
|
||||||
* @param pipeline - Pipeline.
|
* @param pipeline - Pipeline.
|
||||||
* @param keyName - Key Name
|
* @param keyName - Key Name
|
||||||
* @param info - Chunk Info
|
* @param info - Chunk Info
|
||||||
* @throws IOException
|
* @throws StorageContainerException
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public void deleteChunk(Pipeline pipeline, String keyName, ChunkInfo info)
|
public void deleteChunk(Pipeline pipeline, String keyName, ChunkInfo info)
|
||||||
throws IOException {
|
throws StorageContainerException {
|
||||||
|
|
||||||
containerManager.readLock();
|
containerManager.readLock();
|
||||||
try {
|
try {
|
||||||
|
@ -139,8 +148,9 @@ public class ChunkManagerImpl implements ChunkManager {
|
||||||
} else {
|
} else {
|
||||||
LOG.error("Not Supported Operation. Trying to delete a " +
|
LOG.error("Not Supported Operation. Trying to delete a " +
|
||||||
"chunk that is in shared file. chunk info : " + info.toString());
|
"chunk that is in shared file. chunk info : " + info.toString());
|
||||||
throw new IOException("Not Supported Operation. Trying to delete a " +
|
throw new StorageContainerException("Not Supported Operation. " +
|
||||||
"chunk that is in shared file. chunk info : " + info.toString());
|
"Trying to delete a chunk that is in shared file. chunk info : "
|
||||||
|
+ info.toString(), UNSUPPORTED_REQUEST);
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
containerManager.readUnlock();
|
containerManager.readUnlock();
|
||||||
|
|
|
@ -30,14 +30,15 @@ import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.Type;
|
||||||
import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
|
import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
|
||||||
import org.apache.hadoop.ozone.container.common.helpers.ChunkUtils;
|
import org.apache.hadoop.ozone.container.common.helpers.ChunkUtils;
|
||||||
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
|
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
|
||||||
|
import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
|
||||||
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
|
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
|
||||||
import org.apache.hadoop.ozone.container.common.helpers.FileUtils;
|
import org.apache.hadoop.ozone.container.common.helpers.FileUtils;
|
||||||
import org.apache.hadoop.ozone.container.common.helpers.KeyData;
|
import org.apache.hadoop.ozone.container.common.helpers.KeyData;
|
||||||
import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
|
import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
|
||||||
import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
|
|
||||||
import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
|
import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
|
||||||
import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
|
import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
|
||||||
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
|
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
|
||||||
|
import org.apache.hadoop.scm.container.common.helpers.StorageContainerException;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
@ -45,6 +46,11 @@ import java.io.IOException;
|
||||||
import java.util.LinkedList;
|
import java.util.LinkedList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
|
import static org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
||||||
|
.Result.PUT_SMALL_FILE_ERROR;
|
||||||
|
import static org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
||||||
|
.Result.GET_SMALL_FILE_ERROR;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Ozone Container dispatcher takes a call from the netty server and routes it
|
* Ozone Container dispatcher takes a call from the netty server and routes it
|
||||||
* to the right handler function.
|
* to the right handler function.
|
||||||
|
@ -55,6 +61,7 @@ public class Dispatcher implements ContainerDispatcher {
|
||||||
private final ContainerManager containerManager;
|
private final ContainerManager containerManager;
|
||||||
private ContainerMetrics metrics;
|
private ContainerMetrics metrics;
|
||||||
private Configuration conf;
|
private Configuration conf;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Constructs an OzoneContainer that receives calls from
|
* Constructs an OzoneContainer that receives calls from
|
||||||
* XceiverServerHandler.
|
* XceiverServerHandler.
|
||||||
|
@ -79,9 +86,10 @@ public class Dispatcher implements ContainerDispatcher {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ContainerCommandResponseProto dispatch(
|
public ContainerCommandResponseProto dispatch(
|
||||||
ContainerCommandRequestProto msg) throws IOException {
|
ContainerCommandRequestProto msg) {
|
||||||
long startNanos = System.nanoTime();
|
long startNanos = System.nanoTime();
|
||||||
ContainerCommandResponseProto resp = null;
|
ContainerCommandResponseProto resp = null;
|
||||||
|
try {
|
||||||
Preconditions.checkNotNull(msg);
|
Preconditions.checkNotNull(msg);
|
||||||
Type cmdType = msg.getCmdType();
|
Type cmdType = msg.getCmdType();
|
||||||
metrics.incContainerOpcMetrics(cmdType);
|
metrics.incContainerOpcMetrics(cmdType);
|
||||||
|
@ -118,20 +126,27 @@ public class Dispatcher implements ContainerDispatcher {
|
||||||
}
|
}
|
||||||
|
|
||||||
return ContainerUtils.unsupportedRequest(msg);
|
return ContainerUtils.unsupportedRequest(msg);
|
||||||
|
} catch (StorageContainerException e) {
|
||||||
|
// This useful since the trace ID will allow us to correlate failures.
|
||||||
|
return ContainerUtils.logAndReturnError(LOG, e, msg);
|
||||||
|
} catch (IllegalStateException | NullPointerException e) {
|
||||||
|
return ContainerUtils.logAndReturnError(LOG, e, msg);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public ContainerMetrics getContainerMetrics() {
|
public ContainerMetrics getContainerMetrics() {
|
||||||
return metrics;
|
return metrics;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Handles the all Container related functionality.
|
* Handles the all Container related functionality.
|
||||||
*
|
*
|
||||||
* @param msg - command
|
* @param msg - command
|
||||||
* @return - response
|
* @return - response
|
||||||
* @throws IOException
|
* @throws StorageContainerException
|
||||||
*/
|
*/
|
||||||
private ContainerCommandResponseProto containerProcessHandler(
|
private ContainerCommandResponseProto containerProcessHandler(
|
||||||
ContainerCommandRequestProto msg) throws IOException {
|
ContainerCommandRequestProto msg) throws StorageContainerException {
|
||||||
try {
|
try {
|
||||||
|
|
||||||
switch (msg.getCmdType()) {
|
switch (msg.getCmdType()) {
|
||||||
|
@ -175,10 +190,10 @@ public class Dispatcher implements ContainerDispatcher {
|
||||||
*
|
*
|
||||||
* @param msg - command
|
* @param msg - command
|
||||||
* @return - response
|
* @return - response
|
||||||
* @throws IOException
|
* @throws StorageContainerException
|
||||||
*/
|
*/
|
||||||
private ContainerCommandResponseProto keyProcessHandler(
|
private ContainerCommandResponseProto keyProcessHandler(
|
||||||
ContainerCommandRequestProto msg) throws IOException {
|
ContainerCommandRequestProto msg) throws StorageContainerException {
|
||||||
try {
|
try {
|
||||||
switch (msg.getCmdType()) {
|
switch (msg.getCmdType()) {
|
||||||
case PutKey:
|
case PutKey:
|
||||||
|
@ -217,10 +232,10 @@ public class Dispatcher implements ContainerDispatcher {
|
||||||
*
|
*
|
||||||
* @param msg - command
|
* @param msg - command
|
||||||
* @return - response
|
* @return - response
|
||||||
* @throws IOException
|
* @throws StorageContainerException
|
||||||
*/
|
*/
|
||||||
private ContainerCommandResponseProto chunkProcessHandler(
|
private ContainerCommandResponseProto chunkProcessHandler(
|
||||||
ContainerCommandRequestProto msg) throws IOException {
|
ContainerCommandRequestProto msg) throws StorageContainerException {
|
||||||
try {
|
try {
|
||||||
switch (msg.getCmdType()) {
|
switch (msg.getCmdType()) {
|
||||||
case WriteChunk:
|
case WriteChunk:
|
||||||
|
@ -254,7 +269,7 @@ public class Dispatcher implements ContainerDispatcher {
|
||||||
}
|
}
|
||||||
|
|
||||||
private ContainerCommandResponseProto smallFileHandler(
|
private ContainerCommandResponseProto smallFileHandler(
|
||||||
ContainerCommandRequestProto msg) throws IOException {
|
ContainerCommandRequestProto msg) throws StorageContainerException {
|
||||||
switch (msg.getCmdType()) {
|
switch (msg.getCmdType()) {
|
||||||
case PutSmallFile:
|
case PutSmallFile:
|
||||||
return handlePutSmallFile(msg);
|
return handlePutSmallFile(msg);
|
||||||
|
@ -494,7 +509,6 @@ public class Dispatcher implements ContainerDispatcher {
|
||||||
String keyName = msg.getDeleteKey().getName();
|
String keyName = msg.getDeleteKey().getName();
|
||||||
Preconditions.checkNotNull(keyName);
|
Preconditions.checkNotNull(keyName);
|
||||||
Preconditions.checkState(!keyName.isEmpty());
|
Preconditions.checkState(!keyName.isEmpty());
|
||||||
|
|
||||||
this.containerManager.getKeyManager().deleteKey(pipeline, keyName);
|
this.containerManager.getKeyManager().deleteKey(pipeline, keyName);
|
||||||
return KeyUtils.getKeyResponse(msg);
|
return KeyUtils.getKeyResponse(msg);
|
||||||
}
|
}
|
||||||
|
@ -504,19 +518,22 @@ public class Dispatcher implements ContainerDispatcher {
|
||||||
*
|
*
|
||||||
* @param msg - Message.
|
* @param msg - Message.
|
||||||
* @return ContainerCommandResponseProto
|
* @return ContainerCommandResponseProto
|
||||||
* @throws IOException
|
* @throws StorageContainerException
|
||||||
*/
|
*/
|
||||||
private ContainerCommandResponseProto handlePutSmallFile(
|
private ContainerCommandResponseProto handlePutSmallFile(
|
||||||
ContainerCommandRequestProto msg) throws IOException {
|
ContainerCommandRequestProto msg) throws StorageContainerException {
|
||||||
|
|
||||||
if (!msg.hasPutSmallFile()) {
|
if (!msg.hasPutSmallFile()) {
|
||||||
LOG.debug("Malformed put small file request. trace ID: {}",
|
LOG.debug("Malformed put small file request. trace ID: {}",
|
||||||
msg.getTraceID());
|
msg.getTraceID());
|
||||||
return ContainerUtils.malformedRequest(msg);
|
return ContainerUtils.malformedRequest(msg);
|
||||||
}
|
}
|
||||||
|
try {
|
||||||
|
|
||||||
Pipeline pipeline =
|
Pipeline pipeline =
|
||||||
Pipeline.getFromProtoBuf(msg.getPutSmallFile().getKey().getPipeline());
|
Pipeline.getFromProtoBuf(msg.getPutSmallFile()
|
||||||
|
.getKey().getPipeline());
|
||||||
|
|
||||||
Preconditions.checkNotNull(pipeline);
|
Preconditions.checkNotNull(pipeline);
|
||||||
KeyData keyData = KeyData.getFromProtoBuf(msg.getPutSmallFile().getKey()
|
KeyData keyData = KeyData.getFromProtoBuf(msg.getPutSmallFile().getKey()
|
||||||
.getKeyData());
|
.getKeyData());
|
||||||
|
@ -532,6 +549,10 @@ public class Dispatcher implements ContainerDispatcher {
|
||||||
keyData.setChunks(chunks);
|
keyData.setChunks(chunks);
|
||||||
this.containerManager.getKeyManager().putKey(pipeline, keyData);
|
this.containerManager.getKeyManager().putKey(pipeline, keyData);
|
||||||
return FileUtils.getPutFileResponse(msg);
|
return FileUtils.getPutFileResponse(msg);
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new StorageContainerException("Put Small File Failed.", e,
|
||||||
|
PUT_SMALL_FILE_ERROR);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -540,17 +561,21 @@ public class Dispatcher implements ContainerDispatcher {
|
||||||
*
|
*
|
||||||
* @param msg - ContainerCommandRequestProto
|
* @param msg - ContainerCommandRequestProto
|
||||||
* @return ContainerCommandResponseProto
|
* @return ContainerCommandResponseProto
|
||||||
|
* @throws StorageContainerException
|
||||||
*/
|
*/
|
||||||
private ContainerCommandResponseProto handleGetSmallFile(
|
private ContainerCommandResponseProto handleGetSmallFile(
|
||||||
ContainerCommandRequestProto msg) throws IOException {
|
ContainerCommandRequestProto msg) throws StorageContainerException {
|
||||||
ByteString dataBuf = ByteString.EMPTY;
|
ByteString dataBuf = ByteString.EMPTY;
|
||||||
if (!msg.hasGetSmallFile()) {
|
if (!msg.hasGetSmallFile()) {
|
||||||
LOG.debug("Malformed get small file request. trace ID: {}",
|
LOG.debug("Malformed get small file request. trace ID: {}",
|
||||||
msg.getTraceID());
|
msg.getTraceID());
|
||||||
return ContainerUtils.malformedRequest(msg);
|
return ContainerUtils.malformedRequest(msg);
|
||||||
}
|
}
|
||||||
|
try {
|
||||||
Pipeline pipeline =
|
Pipeline pipeline =
|
||||||
Pipeline.getFromProtoBuf(msg.getGetSmallFile().getKey().getPipeline());
|
Pipeline.getFromProtoBuf(msg.getGetSmallFile()
|
||||||
|
.getKey().getPipeline());
|
||||||
|
|
||||||
long bytes = 0;
|
long bytes = 0;
|
||||||
Preconditions.checkNotNull(pipeline);
|
Preconditions.checkNotNull(pipeline);
|
||||||
KeyData keyData = KeyData.getFromProtoBuf(msg.getGetSmallFile()
|
KeyData keyData = KeyData.getFromProtoBuf(msg.getGetSmallFile()
|
||||||
|
@ -560,7 +585,8 @@ public class Dispatcher implements ContainerDispatcher {
|
||||||
for (ContainerProtos.ChunkInfo chunk : data.getChunks()) {
|
for (ContainerProtos.ChunkInfo chunk : data.getChunks()) {
|
||||||
bytes += chunk.getSerializedSize();
|
bytes += chunk.getSerializedSize();
|
||||||
ByteString current =
|
ByteString current =
|
||||||
ByteString.copyFrom(this.containerManager.getChunkManager().readChunk(
|
ByteString.copyFrom(this.containerManager.getChunkManager()
|
||||||
|
.readChunk(
|
||||||
pipeline, keyData.getKeyName(), ChunkInfo.getFromProtoBuf(
|
pipeline, keyData.getKeyName(), ChunkInfo.getFromProtoBuf(
|
||||||
chunk)));
|
chunk)));
|
||||||
dataBuf = dataBuf.concat(current);
|
dataBuf = dataBuf.concat(current);
|
||||||
|
@ -569,6 +595,10 @@ public class Dispatcher implements ContainerDispatcher {
|
||||||
metrics.incContainerBytesStats(Type.GetSmallFile, bytes);
|
metrics.incContainerBytesStats(Type.GetSmallFile, bytes);
|
||||||
return FileUtils.getGetSmallFileResponse(msg, dataBuf.toByteArray(),
|
return FileUtils.getGetSmallFileResponse(msg, dataBuf.toByteArray(),
|
||||||
ChunkInfo.getFromProtoBuf(c));
|
ChunkInfo.getFromProtoBuf(c));
|
||||||
}
|
} catch (IOException e) {
|
||||||
|
throw new StorageContainerException("Unable to decode protobuf", e,
|
||||||
|
GET_SMALL_FILE_ERROR);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -25,19 +25,31 @@ import org.apache.hadoop.ozone.OzoneConfigKeys;
|
||||||
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
|
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
|
||||||
import org.apache.hadoop.ozone.container.common.helpers.KeyData;
|
import org.apache.hadoop.ozone.container.common.helpers.KeyData;
|
||||||
import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
|
import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
|
||||||
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
|
|
||||||
import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
|
import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
|
||||||
import org.apache.hadoop.ozone.container.common.interfaces.KeyManager;
|
import org.apache.hadoop.ozone.container.common.interfaces.KeyManager;
|
||||||
import org.apache.hadoop.ozone.container.common.utils.ContainerCache;
|
import org.apache.hadoop.ozone.container.common.utils.ContainerCache;
|
||||||
|
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
|
||||||
|
import org.apache.hadoop.scm.container.common.helpers.StorageContainerException;
|
||||||
import org.apache.hadoop.utils.LevelDBStore;
|
import org.apache.hadoop.utils.LevelDBStore;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
|
import static org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
||||||
|
.Result.IO_EXCEPTION;
|
||||||
|
import static org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
||||||
|
.Result.NO_SUCH_KEY;
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Key Manager impl.
|
* Key Manager impl.
|
||||||
*/
|
*/
|
||||||
public class KeyManagerImpl implements KeyManager {
|
public class KeyManagerImpl implements KeyManager {
|
||||||
|
static final Logger LOG =
|
||||||
|
LoggerFactory.getLogger(KeyManagerImpl.class);
|
||||||
|
|
||||||
private static final float LOAD_FACTOR = 0.75f;
|
private static final float LOAD_FACTOR = 0.75f;
|
||||||
private final ContainerManager containerManager;
|
private final ContainerManager containerManager;
|
||||||
private final ContainerCache containerCache;
|
private final ContainerCache containerCache;
|
||||||
|
@ -48,8 +60,9 @@ public class KeyManagerImpl implements KeyManager {
|
||||||
* @param containerManager - Container Manager.
|
* @param containerManager - Container Manager.
|
||||||
*/
|
*/
|
||||||
public KeyManagerImpl(ContainerManager containerManager, Configuration conf) {
|
public KeyManagerImpl(ContainerManager containerManager, Configuration conf) {
|
||||||
Preconditions.checkNotNull(containerManager);
|
Preconditions.checkNotNull(containerManager, "Container manager cannot be" +
|
||||||
Preconditions.checkNotNull(conf);
|
" null");
|
||||||
|
Preconditions.checkNotNull(conf, "Config cannot be null");
|
||||||
int cacheSize = conf.getInt(OzoneConfigKeys.OZONE_KEY_CACHE,
|
int cacheSize = conf.getInt(OzoneConfigKeys.OZONE_KEY_CACHE,
|
||||||
OzoneConfigKeys.OZONE_KEY_CACHE_DEFAULT);
|
OzoneConfigKeys.OZONE_KEY_CACHE_DEFAULT);
|
||||||
this.containerManager = containerManager;
|
this.containerManager = containerManager;
|
||||||
|
@ -60,17 +73,22 @@ public class KeyManagerImpl implements KeyManager {
|
||||||
* {@inheritDoc}
|
* {@inheritDoc}
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public void putKey(Pipeline pipeline, KeyData data) throws IOException {
|
public void putKey(Pipeline pipeline, KeyData data)
|
||||||
|
throws StorageContainerException {
|
||||||
containerManager.readLock();
|
containerManager.readLock();
|
||||||
try {
|
try {
|
||||||
// We are not locking the key manager since LevelDb serializes all actions
|
// We are not locking the key manager since LevelDb serializes all actions
|
||||||
// against a single DB. We rely on DB level locking to avoid conflicts.
|
// against a single DB. We rely on DB level locking to avoid conflicts.
|
||||||
Preconditions.checkNotNull(pipeline);
|
Preconditions.checkNotNull(pipeline, "Pipeline cannot be null");
|
||||||
Preconditions.checkNotNull(pipeline.getContainerName());
|
Preconditions.checkNotNull(pipeline.getContainerName(),
|
||||||
|
"Container name cannot be null");
|
||||||
ContainerData cData = containerManager.readContainer(
|
ContainerData cData = containerManager.readContainer(
|
||||||
pipeline.getContainerName());
|
pipeline.getContainerName());
|
||||||
LevelDBStore db = KeyUtils.getDB(cData, containerCache);
|
LevelDBStore db = KeyUtils.getDB(cData, containerCache);
|
||||||
Preconditions.checkNotNull(db);
|
|
||||||
|
// This is a post condition that acts as a hint to the user.
|
||||||
|
// Should never fail.
|
||||||
|
Preconditions.checkNotNull(db, "DB cannot be null here");
|
||||||
db.put(data.getKeyName().getBytes(KeyUtils.ENCODING), data
|
db.put(data.getKeyName().getBytes(KeyUtils.ENCODING), data
|
||||||
.getProtoBufMessage().toByteArray());
|
.getProtoBufMessage().toByteArray());
|
||||||
} finally {
|
} finally {
|
||||||
|
@ -83,22 +101,30 @@ public class KeyManagerImpl implements KeyManager {
|
||||||
* {@inheritDoc}
|
* {@inheritDoc}
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public KeyData getKey(KeyData data) throws IOException {
|
public KeyData getKey(KeyData data) throws StorageContainerException {
|
||||||
containerManager.readLock();
|
containerManager.readLock();
|
||||||
try {
|
try {
|
||||||
Preconditions.checkNotNull(data);
|
Preconditions.checkNotNull(data, "Key data cannot be null");
|
||||||
Preconditions.checkNotNull(data.getContainerName());
|
Preconditions.checkNotNull(data.getContainerName(),
|
||||||
|
"Container name cannot be null");
|
||||||
ContainerData cData = containerManager.readContainer(data
|
ContainerData cData = containerManager.readContainer(data
|
||||||
.getContainerName());
|
.getContainerName());
|
||||||
LevelDBStore db = KeyUtils.getDB(cData, containerCache);
|
LevelDBStore db = KeyUtils.getDB(cData, containerCache);
|
||||||
Preconditions.checkNotNull(db);
|
|
||||||
|
// This is a post condition that acts as a hint to the user.
|
||||||
|
// Should never fail.
|
||||||
|
Preconditions.checkNotNull(db, "DB cannot be null here");
|
||||||
|
|
||||||
byte[] kData = db.get(data.getKeyName().getBytes(KeyUtils.ENCODING));
|
byte[] kData = db.get(data.getKeyName().getBytes(KeyUtils.ENCODING));
|
||||||
if(kData == null) {
|
if (kData == null) {
|
||||||
throw new IOException("Unable to find the key.");
|
throw new StorageContainerException("Unable to find the key.",
|
||||||
|
NO_SUCH_KEY);
|
||||||
}
|
}
|
||||||
ContainerProtos.KeyData keyData =
|
ContainerProtos.KeyData keyData =
|
||||||
ContainerProtos.KeyData.parseFrom(kData);
|
ContainerProtos.KeyData.parseFrom(kData);
|
||||||
return KeyData.getFromProtoBuf(keyData);
|
return KeyData.getFromProtoBuf(keyData);
|
||||||
|
} catch (IOException ex) {
|
||||||
|
throw new StorageContainerException(ex, IO_EXCEPTION);
|
||||||
} finally {
|
} finally {
|
||||||
containerManager.readUnlock();
|
containerManager.readUnlock();
|
||||||
}
|
}
|
||||||
|
@ -108,24 +134,29 @@ public class KeyManagerImpl implements KeyManager {
|
||||||
* {@inheritDoc}
|
* {@inheritDoc}
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public void deleteKey(Pipeline pipeline, String keyName) throws IOException {
|
public void deleteKey(Pipeline pipeline, String keyName)
|
||||||
|
throws StorageContainerException {
|
||||||
containerManager.readLock();
|
containerManager.readLock();
|
||||||
try {
|
try {
|
||||||
Preconditions.checkNotNull(pipeline);
|
Preconditions.checkNotNull(pipeline, "Pipeline cannot be null");
|
||||||
Preconditions.checkNotNull(pipeline.getContainerName());
|
Preconditions.checkNotNull(pipeline.getContainerName(),
|
||||||
|
"Container name cannot be null");
|
||||||
ContainerData cData = containerManager.readContainer(pipeline
|
ContainerData cData = containerManager.readContainer(pipeline
|
||||||
.getContainerName());
|
.getContainerName());
|
||||||
LevelDBStore db = KeyUtils.getDB(cData, containerCache);
|
LevelDBStore db = KeyUtils.getDB(cData, containerCache);
|
||||||
Preconditions.checkNotNull(db);
|
|
||||||
|
|
||||||
|
// This is a post condition that acts as a hint to the user.
|
||||||
|
// Should never fail.
|
||||||
|
Preconditions.checkNotNull(db, "DB cannot be null here");
|
||||||
// Note : There is a race condition here, since get and delete
|
// Note : There is a race condition here, since get and delete
|
||||||
// are not atomic. Leaving it here since the impact is refusing
|
// are not atomic. Leaving it here since the impact is refusing
|
||||||
// to delete a key which might have just gotten inserted after
|
// to delete a key which might have just gotten inserted after
|
||||||
// the get check.
|
// the get check.
|
||||||
|
|
||||||
byte[] kData = db.get(keyName.getBytes(KeyUtils.ENCODING));
|
byte[] kData = db.get(keyName.getBytes(KeyUtils.ENCODING));
|
||||||
if(kData == null) {
|
if (kData == null) {
|
||||||
throw new IOException("Unable to find the key.");
|
throw new StorageContainerException("Unable to find the key.",
|
||||||
|
NO_SUCH_KEY);
|
||||||
}
|
}
|
||||||
db.delete(keyName.getBytes(KeyUtils.ENCODING));
|
db.delete(keyName.getBytes(KeyUtils.ENCODING));
|
||||||
} finally {
|
} finally {
|
||||||
|
@ -139,7 +170,7 @@ public class KeyManagerImpl implements KeyManager {
|
||||||
@Override
|
@Override
|
||||||
public List<KeyData> listKey(Pipeline pipeline, String prefix, String
|
public List<KeyData> listKey(Pipeline pipeline, String prefix, String
|
||||||
prevKey, int count) {
|
prevKey, int count) {
|
||||||
// TODO :
|
// TODO : Implement listKey function.
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -148,7 +179,8 @@ public class KeyManagerImpl implements KeyManager {
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public void shutdown() {
|
public void shutdown() {
|
||||||
Preconditions.checkState(this.containerManager.hasWriteLock());
|
Preconditions.checkState(this.containerManager.hasWriteLock(), "asserts " +
|
||||||
|
"that we are holding the container manager lock when shutting down.");
|
||||||
KeyUtils.shutdownCache(containerCache);
|
KeyUtils.shutdownCache(containerCache);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -18,11 +18,10 @@
|
||||||
|
|
||||||
package org.apache.hadoop.ozone.container.common.interfaces;
|
package org.apache.hadoop.ozone.container.common.interfaces;
|
||||||
|
|
||||||
|
import org.apache.hadoop.scm.container.common.helpers.StorageContainerException;
|
||||||
import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
|
import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
|
||||||
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
|
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Chunk Manager allows read, write, delete and listing of chunks in
|
* Chunk Manager allows read, write, delete and listing of chunks in
|
||||||
* a container.
|
* a container.
|
||||||
|
@ -34,10 +33,10 @@ public interface ChunkManager {
|
||||||
* @param pipeline - Name and the set of machines that make this container.
|
* @param pipeline - Name and the set of machines that make this container.
|
||||||
* @param keyName - Name of the Key.
|
* @param keyName - Name of the Key.
|
||||||
* @param info - ChunkInfo.
|
* @param info - ChunkInfo.
|
||||||
* @throws IOException
|
* @throws StorageContainerException
|
||||||
*/
|
*/
|
||||||
void writeChunk(Pipeline pipeline, String keyName,
|
void writeChunk(Pipeline pipeline, String keyName,
|
||||||
ChunkInfo info, byte[] data) throws IOException;
|
ChunkInfo info, byte[] data) throws StorageContainerException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* reads the data defined by a chunk.
|
* reads the data defined by a chunk.
|
||||||
|
@ -45,23 +44,23 @@ public interface ChunkManager {
|
||||||
* @param keyName - Name of the Key
|
* @param keyName - Name of the Key
|
||||||
* @param info - ChunkInfo.
|
* @param info - ChunkInfo.
|
||||||
* @return byte array
|
* @return byte array
|
||||||
* @throws IOException
|
* @throws StorageContainerException
|
||||||
*
|
*
|
||||||
* TODO: Right now we do not support partial reads and writes of chunks.
|
* TODO: Right now we do not support partial reads and writes of chunks.
|
||||||
* TODO: Explore if we need to do that for ozone.
|
* TODO: Explore if we need to do that for ozone.
|
||||||
*/
|
*/
|
||||||
byte[] readChunk(Pipeline pipeline, String keyName, ChunkInfo info) throws
|
byte[] readChunk(Pipeline pipeline, String keyName, ChunkInfo info) throws
|
||||||
IOException;
|
StorageContainerException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Deletes a given chunk.
|
* Deletes a given chunk.
|
||||||
* @param pipeline - Pipeline.
|
* @param pipeline - Pipeline.
|
||||||
* @param keyName - Key Name
|
* @param keyName - Key Name
|
||||||
* @param info - Chunk Info
|
* @param info - Chunk Info
|
||||||
* @throws IOException
|
* @throws StorageContainerException
|
||||||
*/
|
*/
|
||||||
void deleteChunk(Pipeline pipeline, String keyName, ChunkInfo info) throws
|
void deleteChunk(Pipeline pipeline, String keyName, ChunkInfo info) throws
|
||||||
IOException;
|
StorageContainerException;
|
||||||
|
|
||||||
// TODO : Support list operations.
|
// TODO : Support list operations.
|
||||||
|
|
||||||
|
|
|
@ -21,8 +21,6 @@ package org.apache.hadoop.ozone.container.common.interfaces;
|
||||||
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.ContainerCommandResponseProto;
|
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.ContainerCommandResponseProto;
|
||||||
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.ContainerCommandRequestProto;
|
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.ContainerCommandRequestProto;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Dispatcher acts as the bridge between the transport layer and
|
* Dispatcher acts as the bridge between the transport layer and
|
||||||
* the actual container layer. This layer is capable of transforming
|
* the actual container layer. This layer is capable of transforming
|
||||||
|
@ -36,10 +34,8 @@ public interface ContainerDispatcher {
|
||||||
* Dispatches commands to container layer.
|
* Dispatches commands to container layer.
|
||||||
* @param msg - Command Request
|
* @param msg - Command Request
|
||||||
* @return Command Response
|
* @return Command Response
|
||||||
* @throws IOException
|
|
||||||
*/
|
*/
|
||||||
ContainerCommandResponseProto dispatch(ContainerCommandRequestProto msg)
|
ContainerCommandResponseProto dispatch(ContainerCommandRequestProto msg);
|
||||||
throws IOException;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Initialize the Dispatcher.
|
* Initialize the Dispatcher.
|
||||||
|
|
|
@ -23,6 +23,7 @@ import org.apache.hadoop.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
|
import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
|
||||||
import org.apache.hadoop.hdfs.util.RwLock;
|
import org.apache.hadoop.hdfs.util.RwLock;
|
||||||
|
import org.apache.hadoop.scm.container.common.helpers.StorageContainerException;
|
||||||
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
|
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.StorageContainerDatanodeProtocolProtos.SCMNodeReport;
|
.StorageContainerDatanodeProtocolProtos.SCMNodeReport;
|
||||||
|
@ -43,7 +44,7 @@ public interface ContainerManager extends RwLock {
|
||||||
*
|
*
|
||||||
* @param config - Configuration.
|
* @param config - Configuration.
|
||||||
* @param containerDirs - List of Metadata Container locations.
|
* @param containerDirs - List of Metadata Container locations.
|
||||||
* @throws IOException
|
* @throws StorageContainerException
|
||||||
*/
|
*/
|
||||||
void init(Configuration config, List<StorageLocation> containerDirs)
|
void init(Configuration config, List<StorageLocation> containerDirs)
|
||||||
throws IOException;
|
throws IOException;
|
||||||
|
@ -53,20 +54,20 @@ public interface ContainerManager extends RwLock {
|
||||||
*
|
*
|
||||||
* @param pipeline -- Nodes which make up this container.
|
* @param pipeline -- Nodes which make up this container.
|
||||||
* @param containerData - Container Name and metadata.
|
* @param containerData - Container Name and metadata.
|
||||||
* @throws IOException
|
* @throws StorageContainerException
|
||||||
*/
|
*/
|
||||||
void createContainer(Pipeline pipeline, ContainerData containerData)
|
void createContainer(Pipeline pipeline, ContainerData containerData)
|
||||||
throws IOException;
|
throws StorageContainerException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Deletes an existing container.
|
* Deletes an existing container.
|
||||||
*
|
*
|
||||||
* @param pipeline - nodes that make this container.
|
* @param pipeline - nodes that make this container.
|
||||||
* @param containerName - name of the container.
|
* @param containerName - name of the container.
|
||||||
* @throws IOException
|
* @throws StorageContainerException
|
||||||
*/
|
*/
|
||||||
void deleteContainer(Pipeline pipeline, String containerName)
|
void deleteContainer(Pipeline pipeline, String containerName)
|
||||||
throws IOException;
|
throws StorageContainerException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* As simple interface for container Iterations.
|
* As simple interface for container Iterations.
|
||||||
|
@ -75,25 +76,26 @@ public interface ContainerManager extends RwLock {
|
||||||
* @param count - how many to return
|
* @param count - how many to return
|
||||||
* @param prevKey - Previous key - Server returns results from this point.
|
* @param prevKey - Previous key - Server returns results from this point.
|
||||||
* @param data - Actual containerData
|
* @param data - Actual containerData
|
||||||
* @throws IOException
|
* @throws StorageContainerException
|
||||||
*/
|
*/
|
||||||
void listContainer(String prefix, long count, String prevKey,
|
void listContainer(String prefix, long count, String prevKey,
|
||||||
List<ContainerData> data)
|
List<ContainerData> data)
|
||||||
throws IOException;
|
throws StorageContainerException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get metadata about a specific container.
|
* Get metadata about a specific container.
|
||||||
*
|
*
|
||||||
* @param containerName - Name of the container
|
* @param containerName - Name of the container
|
||||||
* @return ContainerData - Container Data.
|
* @return ContainerData - Container Data.
|
||||||
* @throws IOException
|
* @throws StorageContainerException
|
||||||
*/
|
*/
|
||||||
ContainerData readContainer(String containerName) throws IOException;
|
ContainerData readContainer(String containerName)
|
||||||
|
throws StorageContainerException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Supports clean shutdown of container.
|
* Supports clean shutdown of container.
|
||||||
*
|
*
|
||||||
* @throws IOException
|
* @throws StorageContainerException
|
||||||
*/
|
*/
|
||||||
void shutdown() throws IOException;
|
void shutdown() throws IOException;
|
||||||
|
|
||||||
|
|
|
@ -17,10 +17,10 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.ozone.container.common.interfaces;
|
package org.apache.hadoop.ozone.container.common.interfaces;
|
||||||
|
|
||||||
|
import org.apache.hadoop.scm.container.common.helpers.StorageContainerException;
|
||||||
import org.apache.hadoop.ozone.container.common.helpers.KeyData;
|
import org.apache.hadoop.ozone.container.common.helpers.KeyData;
|
||||||
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
|
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -32,24 +32,28 @@ public interface KeyManager {
|
||||||
*
|
*
|
||||||
* @param pipeline - Pipeline.
|
* @param pipeline - Pipeline.
|
||||||
* @param data - Key Data.
|
* @param data - Key Data.
|
||||||
|
* @throws StorageContainerException
|
||||||
*/
|
*/
|
||||||
void putKey(Pipeline pipeline, KeyData data) throws IOException;
|
void putKey(Pipeline pipeline, KeyData data) throws StorageContainerException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Gets an existing key.
|
* Gets an existing key.
|
||||||
*
|
*
|
||||||
* @param data - Key Data.
|
* @param data - Key Data.
|
||||||
* @return Key Data.
|
* @return Key Data.
|
||||||
|
* @throws StorageContainerException
|
||||||
*/
|
*/
|
||||||
KeyData getKey(KeyData data) throws IOException;
|
KeyData getKey(KeyData data) throws StorageContainerException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Deletes an existing Key.
|
* Deletes an existing Key.
|
||||||
*
|
*
|
||||||
* @param pipeline - Pipeline.
|
* @param pipeline - Pipeline.
|
||||||
* @param keyName Key Data.
|
* @param keyName Key Data.
|
||||||
|
* @throws StorageContainerException
|
||||||
*/
|
*/
|
||||||
void deleteKey(Pipeline pipeline, String keyName) throws IOException;
|
void deleteKey(Pipeline pipeline, String keyName)
|
||||||
|
throws StorageContainerException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* List keys in a container.
|
* List keys in a container.
|
||||||
|
|
|
@ -44,9 +44,15 @@ import java.util.UUID;
|
||||||
/**
|
/**
|
||||||
* Helpers for container tests.
|
* Helpers for container tests.
|
||||||
*/
|
*/
|
||||||
public class ContainerTestHelper {
|
public final class ContainerTestHelper {
|
||||||
private static Random r = new Random();
|
private static Random r = new Random();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Never constructed.
|
||||||
|
*/
|
||||||
|
private ContainerTestHelper() {
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create a pipeline with single node replica.
|
* Create a pipeline with single node replica.
|
||||||
*
|
*
|
||||||
|
@ -291,8 +297,7 @@ public class ContainerTestHelper {
|
||||||
* @return ContainerCommandRequestProto.
|
* @return ContainerCommandRequestProto.
|
||||||
*/
|
*/
|
||||||
public static ContainerCommandResponseProto
|
public static ContainerCommandResponseProto
|
||||||
getCreateContainerResponse(ContainerCommandRequestProto request) throws
|
getCreateContainerResponse(ContainerCommandRequestProto request) {
|
||||||
IOException {
|
|
||||||
ContainerProtos.CreateContainerResponseProto.Builder createResponse =
|
ContainerProtos.CreateContainerResponseProto.Builder createResponse =
|
||||||
ContainerProtos.CreateContainerResponseProto.newBuilder();
|
ContainerProtos.CreateContainerResponseProto.newBuilder();
|
||||||
|
|
||||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.hadoop.ozone.MiniOzoneCluster;
|
||||||
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
||||||
import org.apache.hadoop.ozone.OzoneConfiguration;
|
import org.apache.hadoop.ozone.OzoneConfiguration;
|
||||||
import org.apache.hadoop.ozone.OzoneConsts;
|
import org.apache.hadoop.ozone.OzoneConsts;
|
||||||
|
import org.apache.hadoop.scm.container.common.helpers.StorageContainerException;
|
||||||
import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
|
import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
|
||||||
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
|
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
|
||||||
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
|
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
|
||||||
|
@ -121,8 +122,8 @@ public class TestContainerPersistence {
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void setupPaths() throws IOException {
|
public void setupPaths() throws IOException {
|
||||||
if (!new File(path).exists()) {
|
if (!new File(path).exists() && !new File(path).mkdirs()) {
|
||||||
new File(path).mkdirs();
|
throw new IOException("Unable to create paths. " + path);
|
||||||
}
|
}
|
||||||
pathLists.clear();
|
pathLists.clear();
|
||||||
containerManager.getContainerMap().clear();
|
containerManager.getContainerMap().clear();
|
||||||
|
@ -157,11 +158,8 @@ public class TestContainerPersistence {
|
||||||
Assert.assertTrue(new File(status.getContainer().getContainerPath())
|
Assert.assertTrue(new File(status.getContainer().getContainerPath())
|
||||||
.exists());
|
.exists());
|
||||||
|
|
||||||
String containerPathString = ContainerUtils.getContainerNameFromFile(new
|
|
||||||
File(status.getContainer().getContainerPath()));
|
|
||||||
|
|
||||||
Path meta = Paths.get(status.getContainer().getDBPath()).getParent();
|
Path meta = Paths.get(status.getContainer().getDBPath()).getParent();
|
||||||
Assert.assertTrue(Files.exists(meta));
|
Assert.assertTrue(meta != null && Files.exists(meta));
|
||||||
|
|
||||||
|
|
||||||
String dbPath = status.getContainer().getDBPath();
|
String dbPath = status.getContainer().getDBPath();
|
||||||
|
@ -363,8 +361,7 @@ public class TestContainerPersistence {
|
||||||
sha.update(FileUtils.readFileToByteArray(fname.toFile()));
|
sha.update(FileUtils.readFileToByteArray(fname.toFile()));
|
||||||
String val = Hex.encodeHexString(sha.digest());
|
String val = Hex.encodeHexString(sha.digest());
|
||||||
Assert.assertEquals(fileHashMap.get(fname.getFileName().toString())
|
Assert.assertEquals(fileHashMap.get(fname.getFileName().toString())
|
||||||
.getChecksum(),
|
.getChecksum(), val);
|
||||||
val);
|
|
||||||
count++;
|
count++;
|
||||||
sha.reset();
|
sha.reset();
|
||||||
}
|
}
|
||||||
|
@ -486,7 +483,7 @@ public class TestContainerPersistence {
|
||||||
setDataChecksum(info, data);
|
setDataChecksum(info, data);
|
||||||
chunkManager.writeChunk(pipeline, keyName, info, data);
|
chunkManager.writeChunk(pipeline, keyName, info, data);
|
||||||
chunkManager.deleteChunk(pipeline, keyName, info);
|
chunkManager.deleteChunk(pipeline, keyName, info);
|
||||||
exception.expect(IOException.class);
|
exception.expect(StorageContainerException.class);
|
||||||
exception.expectMessage("Unable to find the chunk file.");
|
exception.expectMessage("Unable to find the chunk file.");
|
||||||
chunkManager.readChunk(pipeline, keyName, info);
|
chunkManager.readChunk(pipeline, keyName, info);
|
||||||
}
|
}
|
||||||
|
@ -572,7 +569,7 @@ public class TestContainerPersistence {
|
||||||
keyData.setChunks(chunkList);
|
keyData.setChunks(chunkList);
|
||||||
keyManager.putKey(pipeline, keyData);
|
keyManager.putKey(pipeline, keyData);
|
||||||
keyManager.deleteKey(pipeline, keyName);
|
keyManager.deleteKey(pipeline, keyName);
|
||||||
exception.expect(IOException.class);
|
exception.expect(StorageContainerException.class);
|
||||||
exception.expectMessage("Unable to find the key.");
|
exception.expectMessage("Unable to find the key.");
|
||||||
keyManager.getKey(keyData);
|
keyManager.getKey(keyData);
|
||||||
}
|
}
|
||||||
|
@ -596,7 +593,7 @@ public class TestContainerPersistence {
|
||||||
keyData.setChunks(chunkList);
|
keyData.setChunks(chunkList);
|
||||||
keyManager.putKey(pipeline, keyData);
|
keyManager.putKey(pipeline, keyData);
|
||||||
keyManager.deleteKey(pipeline, keyName);
|
keyManager.deleteKey(pipeline, keyName);
|
||||||
exception.expect(IOException.class);
|
exception.expect(StorageContainerException.class);
|
||||||
exception.expectMessage("Unable to find the key.");
|
exception.expectMessage("Unable to find the key.");
|
||||||
keyManager.deleteKey(pipeline, keyName);
|
keyManager.deleteKey(pipeline, keyName);
|
||||||
}
|
}
|
||||||
|
|
|
@ -41,6 +41,9 @@ import java.io.IOException;
|
||||||
|
|
||||||
import static org.mockito.Mockito.mock;
|
import static org.mockito.Mockito.mock;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test Containers.
|
||||||
|
*/
|
||||||
public class TestContainerServer {
|
public class TestContainerServer {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -69,8 +72,8 @@ public class TestContainerServer {
|
||||||
XceiverClient client = null;
|
XceiverClient client = null;
|
||||||
String containerName = OzoneUtils.getRequestID();
|
String containerName = OzoneUtils.getRequestID();
|
||||||
try {
|
try {
|
||||||
Pipeline pipeline = ContainerTestHelper.createSingleNodePipeline
|
Pipeline pipeline = ContainerTestHelper.createSingleNodePipeline(
|
||||||
(containerName);
|
containerName);
|
||||||
OzoneConfiguration conf = new OzoneConfiguration();
|
OzoneConfiguration conf = new OzoneConfiguration();
|
||||||
conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
|
conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
|
||||||
pipeline.getLeader().getContainerPort());
|
pipeline.getLeader().getContainerPort());
|
||||||
|
@ -102,8 +105,8 @@ public class TestContainerServer {
|
||||||
String containerName = OzoneUtils.getRequestID();
|
String containerName = OzoneUtils.getRequestID();
|
||||||
|
|
||||||
try {
|
try {
|
||||||
Pipeline pipeline = ContainerTestHelper.createSingleNodePipeline
|
Pipeline pipeline = ContainerTestHelper.createSingleNodePipeline(
|
||||||
(containerName);
|
containerName);
|
||||||
OzoneConfiguration conf = new OzoneConfiguration();
|
OzoneConfiguration conf = new OzoneConfiguration();
|
||||||
conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
|
conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
|
||||||
pipeline.getLeader().getContainerPort());
|
pipeline.getLeader().getContainerPort());
|
||||||
|
@ -136,17 +139,16 @@ public class TestContainerServer {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private class TestContainerDispatcher implements ContainerDispatcher {
|
private static class TestContainerDispatcher implements ContainerDispatcher {
|
||||||
/**
|
/**
|
||||||
* Dispatches commands to container layer.
|
* Dispatches commands to container layer.
|
||||||
*
|
*
|
||||||
* @param msg - Command Request
|
* @param msg - Command Request
|
||||||
* @return Command Response
|
* @return Command Response
|
||||||
* @throws IOException
|
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public ContainerCommandResponseProto
|
public ContainerCommandResponseProto
|
||||||
dispatch(ContainerCommandRequestProto msg) throws IOException {
|
dispatch(ContainerCommandRequestProto msg) {
|
||||||
return ContainerTestHelper.getCreateContainerResponse(msg);
|
return ContainerTestHelper.getCreateContainerResponse(msg);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -49,6 +49,8 @@ public class TestAllocateContainer {
|
||||||
conf = new OzoneConfiguration();
|
conf = new OzoneConfiguration();
|
||||||
cluster = new MiniOzoneCluster.Builder(conf).numDataNodes(1)
|
cluster = new MiniOzoneCluster.Builder(conf).numDataNodes(1)
|
||||||
.setHandlerType("distributed").build();
|
.setHandlerType("distributed").build();
|
||||||
|
storageContainerLocationClient =
|
||||||
|
cluster.createStorageContainerLocationClient();
|
||||||
}
|
}
|
||||||
|
|
||||||
@AfterClass
|
@AfterClass
|
||||||
|
@ -61,8 +63,6 @@ public class TestAllocateContainer {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testAllocate() throws Exception {
|
public void testAllocate() throws Exception {
|
||||||
storageContainerLocationClient =
|
|
||||||
cluster.createStorageContainerLocationClient();
|
|
||||||
Pipeline pipeline = storageContainerLocationClient.allocateContainer(
|
Pipeline pipeline = storageContainerLocationClient.allocateContainer(
|
||||||
"container0");
|
"container0");
|
||||||
Assert.assertNotNull(pipeline);
|
Assert.assertNotNull(pipeline);
|
||||||
|
@ -72,8 +72,6 @@ public class TestAllocateContainer {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testAllocateNull() throws Exception {
|
public void testAllocateNull() throws Exception {
|
||||||
storageContainerLocationClient =
|
|
||||||
cluster.createStorageContainerLocationClient();
|
|
||||||
thrown.expect(NullPointerException.class);
|
thrown.expect(NullPointerException.class);
|
||||||
storageContainerLocationClient.allocateContainer(null);
|
storageContainerLocationClient.allocateContainer(null);
|
||||||
}
|
}
|
||||||
|
@ -81,12 +79,9 @@ public class TestAllocateContainer {
|
||||||
@Test
|
@Test
|
||||||
public void testAllocateDuplicate() throws Exception {
|
public void testAllocateDuplicate() throws Exception {
|
||||||
String containerName = RandomStringUtils.randomAlphanumeric(10);
|
String containerName = RandomStringUtils.randomAlphanumeric(10);
|
||||||
storageContainerLocationClient =
|
|
||||||
cluster.createStorageContainerLocationClient();
|
|
||||||
thrown.expect(IOException.class);
|
thrown.expect(IOException.class);
|
||||||
thrown.expectMessage("Specified container already exists");
|
thrown.expectMessage("Specified container already exists");
|
||||||
storageContainerLocationClient.allocateContainer(containerName);
|
storageContainerLocationClient.allocateContainer(containerName);
|
||||||
storageContainerLocationClient.allocateContainer(containerName);
|
storageContainerLocationClient.allocateContainer(containerName);
|
||||||
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,131 @@
|
||||||
|
/**
|
||||||
|
* 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.scm;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos;
|
||||||
|
import org.apache.hadoop.io.IOUtils;
|
||||||
|
import org.apache.hadoop.ozone.MiniOzoneCluster;
|
||||||
|
import org.apache.hadoop.ozone.OzoneConfiguration;
|
||||||
|
import org.apache.hadoop.scm.protocolPB
|
||||||
|
.StorageContainerLocationProtocolClientSideTranslatorPB;
|
||||||
|
import org.apache.hadoop.scm.XceiverClient;
|
||||||
|
import org.apache.hadoop.scm.XceiverClientManager;
|
||||||
|
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
|
||||||
|
import org.apache.hadoop.scm.container.common.helpers.StorageContainerException;
|
||||||
|
import org.apache.hadoop.scm.storage.ContainerProtocolCalls;
|
||||||
|
import org.junit.AfterClass;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.BeforeClass;
|
||||||
|
import org.junit.Rule;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.rules.ExpectedException;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.UUID;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test Container calls.
|
||||||
|
*/
|
||||||
|
public class TestContainerSmallFile {
|
||||||
|
@Rule
|
||||||
|
public ExpectedException thrown = ExpectedException.none();
|
||||||
|
|
||||||
|
private static MiniOzoneCluster cluster;
|
||||||
|
private static OzoneConfiguration ozoneConfig;
|
||||||
|
private static StorageContainerLocationProtocolClientSideTranslatorPB
|
||||||
|
storageContainerLocationClient;
|
||||||
|
private static XceiverClientManager xceiverClientManager;
|
||||||
|
|
||||||
|
@BeforeClass
|
||||||
|
public static void init() throws IOException {
|
||||||
|
ozoneConfig = new OzoneConfiguration();
|
||||||
|
cluster = new MiniOzoneCluster.Builder(ozoneConfig)
|
||||||
|
.numDataNodes(1).setHandlerType("distributed").build();
|
||||||
|
storageContainerLocationClient = cluster
|
||||||
|
.createStorageContainerLocationClient();
|
||||||
|
xceiverClientManager = new XceiverClientManager(ozoneConfig);
|
||||||
|
}
|
||||||
|
|
||||||
|
@AfterClass
|
||||||
|
public static void shutdown() throws InterruptedException {
|
||||||
|
if (cluster != null) {
|
||||||
|
cluster.shutdown();
|
||||||
|
}
|
||||||
|
IOUtils.cleanup(null, storageContainerLocationClient, cluster);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testAllocateWrite() throws Exception {
|
||||||
|
String traceID = UUID.randomUUID().toString();
|
||||||
|
String containerName = "container0";
|
||||||
|
Pipeline pipeline =
|
||||||
|
storageContainerLocationClient.allocateContainer(containerName);
|
||||||
|
XceiverClient client = xceiverClientManager.acquireClient(pipeline);
|
||||||
|
ContainerProtocolCalls.createContainer(client, traceID);
|
||||||
|
|
||||||
|
ContainerProtocolCalls.writeSmallFile(client, containerName,
|
||||||
|
"key", "data123".getBytes(), traceID);
|
||||||
|
ContainerProtos.GetSmallFileResponseProto response =
|
||||||
|
ContainerProtocolCalls.readSmallFile(client, containerName, "key",
|
||||||
|
traceID);
|
||||||
|
String readData = response.getData().getData().toStringUtf8();
|
||||||
|
Assert.assertEquals("data123", readData);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testInvalidKeyRead() throws Exception {
|
||||||
|
String traceID = UUID.randomUUID().toString();
|
||||||
|
String containerName = "container1";
|
||||||
|
Pipeline pipeline =
|
||||||
|
storageContainerLocationClient.allocateContainer(containerName);
|
||||||
|
XceiverClient client = xceiverClientManager.acquireClient(pipeline);
|
||||||
|
ContainerProtocolCalls.createContainer(client, traceID);
|
||||||
|
|
||||||
|
thrown.expect(StorageContainerException.class);
|
||||||
|
thrown.expectMessage("Unable to find the key");
|
||||||
|
|
||||||
|
// Try to read a Key Container Name
|
||||||
|
ContainerProtos.GetSmallFileResponseProto response =
|
||||||
|
ContainerProtocolCalls.readSmallFile(client, containerName, "key",
|
||||||
|
traceID);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testInvalidContainerRead() throws Exception {
|
||||||
|
String traceID = UUID.randomUUID().toString();
|
||||||
|
String invalidName = "invalidName";
|
||||||
|
String containerName = "container2";
|
||||||
|
Pipeline pipeline =
|
||||||
|
storageContainerLocationClient.allocateContainer(containerName);
|
||||||
|
XceiverClient client = xceiverClientManager.acquireClient(pipeline);
|
||||||
|
ContainerProtocolCalls.createContainer(client, traceID);
|
||||||
|
ContainerProtocolCalls.writeSmallFile(client, containerName,
|
||||||
|
"key", "data123".getBytes(), traceID);
|
||||||
|
|
||||||
|
|
||||||
|
thrown.expect(StorageContainerException.class);
|
||||||
|
thrown.expectMessage("Unable to find the container");
|
||||||
|
|
||||||
|
// Try to read a invalid key
|
||||||
|
ContainerProtos.GetSmallFileResponseProto response =
|
||||||
|
ContainerProtocolCalls.readSmallFile(client, invalidName, "key",
|
||||||
|
traceID);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue