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;
|
||||
|
||||
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 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
|
||||
|
@ -35,9 +29,15 @@ import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
|||
.GetKeyRequestProto;
|
||||
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
||||
.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
|
||||
.PutKeyRequestProto;
|
||||
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
||||
.PutSmallFileRequestProto;
|
||||
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
||||
.ReadChunkRequestProto;
|
||||
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
|
||||
.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.container.common.helpers.StorageContainerException;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Implementation of all container protocol calls performed by Container
|
||||
|
@ -87,7 +84,7 @@ public final class ContainerProtocolCalls {
|
|||
.setGetKey(readKeyRequest)
|
||||
.build();
|
||||
ContainerCommandResponseProto response = xceiverClient.sendCommand(request);
|
||||
validateContainerResponse(response, traceID);
|
||||
validateContainerResponse(response);
|
||||
return response.getGetKey();
|
||||
}
|
||||
|
||||
|
@ -112,7 +109,7 @@ public final class ContainerProtocolCalls {
|
|||
.setPutKey(createKeyRequest)
|
||||
.build();
|
||||
ContainerCommandResponseProto response = xceiverClient.sendCommand(request);
|
||||
validateContainerResponse(response, traceID);
|
||||
validateContainerResponse(response);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -140,7 +137,7 @@ public final class ContainerProtocolCalls {
|
|||
.setReadChunk(readChunkRequest)
|
||||
.build();
|
||||
ContainerCommandResponseProto response = xceiverClient.sendCommand(request);
|
||||
validateContainerResponse(response, traceID);
|
||||
validateContainerResponse(response);
|
||||
return response.getReadChunk();
|
||||
}
|
||||
|
||||
|
@ -170,7 +167,7 @@ public final class ContainerProtocolCalls {
|
|||
.setWriteChunk(writeChunkRequest)
|
||||
.build();
|
||||
ContainerCommandResponseProto response = xceiverClient.sendCommand(request);
|
||||
validateContainerResponse(response, traceID);
|
||||
validateContainerResponse(response);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -218,7 +215,7 @@ public final class ContainerProtocolCalls {
|
|||
.setPutSmallFile(putSmallFileRequest)
|
||||
.build();
|
||||
ContainerCommandResponseProto response = client.sendCommand(request);
|
||||
validateContainerResponse(response, traceID);
|
||||
validateContainerResponse(response);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -245,7 +242,7 @@ public final class ContainerProtocolCalls {
|
|||
request.setTraceID(traceID);
|
||||
ContainerCommandResponseProto response = client.sendCommand(
|
||||
request.build());
|
||||
validateContainerResponse(response, traceID);
|
||||
validateContainerResponse(response);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -280,7 +277,7 @@ public final class ContainerProtocolCalls {
|
|||
.setGetSmallFile(getSmallFileRequest)
|
||||
.build();
|
||||
ContainerCommandResponseProto response = client.sendCommand(request);
|
||||
validateContainerResponse(response, traceID);
|
||||
validateContainerResponse(response);
|
||||
return response.getGetSmallFile();
|
||||
}
|
||||
|
||||
|
@ -289,28 +286,15 @@ public final class ContainerProtocolCalls {
|
|||
* return code is mapped to a corresponding exception and thrown.
|
||||
*
|
||||
* @param response container protocol call response
|
||||
* @param traceID container protocol call args
|
||||
* @throws IOException if the container protocol call failed
|
||||
*/
|
||||
private static void validateContainerResponse(
|
||||
ContainerCommandResponseProto response, String traceID
|
||||
) throws IOException {
|
||||
// TODO : throw the right type of exception
|
||||
switch (response.getResult()) {
|
||||
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);
|
||||
ContainerCommandResponseProto response
|
||||
) throws StorageContainerException {
|
||||
if (response.getResult() == ContainerProtos.Result.SUCCESS) {
|
||||
return;
|
||||
}
|
||||
throw new StorageContainerException(
|
||||
response.getMessage(), response.getResult());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -99,6 +99,23 @@ enum Result {
|
|||
UNSUPPORTED_REQUEST = 2;
|
||||
MALFORMED_REQUEST = 3;
|
||||
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 {
|
||||
|
|
|
@ -24,6 +24,7 @@ import org.apache.commons.codec.digest.DigestUtils;
|
|||
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
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.scm.container.common.helpers.Pipeline;
|
||||
import org.slf4j.Logger;
|
||||
|
@ -39,6 +40,23 @@ import java.security.MessageDigest;
|
|||
import java.security.NoSuchAlgorithmException;
|
||||
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.
|
||||
*/
|
||||
|
@ -87,13 +105,13 @@ public final class ChunkUtils {
|
|||
* expected to write data to.
|
||||
*
|
||||
* @param pipeline - pipeline.
|
||||
* @param data - container data.
|
||||
* @param info - chunk info.
|
||||
* @param data - container data.
|
||||
* @param info - chunk info.
|
||||
* @return File
|
||||
* @throws IOException
|
||||
* @throws StorageContainerException
|
||||
*/
|
||||
public static File validateChunk(Pipeline pipeline, ContainerData data,
|
||||
ChunkInfo info) throws IOException {
|
||||
ChunkInfo info) throws StorageContainerException {
|
||||
|
||||
Logger log = LoggerFactory.getLogger(ChunkManagerImpl.class);
|
||||
|
||||
|
@ -102,8 +120,9 @@ public final class ChunkUtils {
|
|||
if (!ChunkUtils.isOverWritePermitted(info)) {
|
||||
log.error("Rejecting write chunk request. Chunk overwrite " +
|
||||
"without explicit request. {}", info.toString());
|
||||
throw new IOException("Rejecting write chunk request. OverWrite " +
|
||||
"flag required." + info.toString());
|
||||
throw new StorageContainerException("Rejecting write chunk request. " +
|
||||
"OverWrite flag required." + info.toString(),
|
||||
OVERWRITE_FLAG_REQUIRED);
|
||||
}
|
||||
}
|
||||
return chunkFile;
|
||||
|
@ -113,25 +132,27 @@ public final class ChunkUtils {
|
|||
* Validates that Path to chunk file exists.
|
||||
*
|
||||
* @param pipeline - Container Info.
|
||||
* @param data - Container Data
|
||||
* @param info - Chunk info
|
||||
* @param data - Container Data
|
||||
* @param info - Chunk info
|
||||
* @return - File.
|
||||
* @throws IOException
|
||||
* @throws StorageContainerException
|
||||
*/
|
||||
public static File getChunkFile(Pipeline pipeline, ContainerData data,
|
||||
ChunkInfo info) throws IOException {
|
||||
ChunkInfo info) throws StorageContainerException {
|
||||
|
||||
Logger log = LoggerFactory.getLogger(ChunkManagerImpl.class);
|
||||
if (data == null) {
|
||||
log.error("Invalid container Name: {}", pipeline.getContainerName());
|
||||
throw new IOException("Unable to find the container Name: " +
|
||||
pipeline.getContainerName());
|
||||
throw new StorageContainerException("Unable to find the container Name:" +
|
||||
" " +
|
||||
pipeline.getContainerName(), CONTAINER_NOT_FOUND);
|
||||
}
|
||||
|
||||
File dataDir = ContainerUtils.getDataDirectory(data).toFile();
|
||||
if (!dataDir.exists()) {
|
||||
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();
|
||||
|
@ -143,11 +164,12 @@ public final class ChunkUtils {
|
|||
*
|
||||
* @param chunkFile - File to write data to.
|
||||
* @param chunkInfo - Data stream to write.
|
||||
* @throws IOException
|
||||
* @param data - The data buffer.
|
||||
* @throws StorageContainerException
|
||||
*/
|
||||
public static void writeData(File chunkFile, ChunkInfo chunkInfo,
|
||||
byte[] data)
|
||||
throws IOException, ExecutionException, InterruptedException,
|
||||
byte[] data) throws
|
||||
StorageContainerException, ExecutionException, InterruptedException,
|
||||
NoSuchAlgorithmException {
|
||||
|
||||
Logger log = LoggerFactory.getLogger(ChunkManagerImpl.class);
|
||||
|
@ -156,7 +178,7 @@ public final class ChunkUtils {
|
|||
"specified. DataLen: %d Byte Array: %d",
|
||||
chunkInfo.getLen(), data.length);
|
||||
log.error(err);
|
||||
throw new IOException(err);
|
||||
throw new StorageContainerException(err, INVALID_WRITE_SIZE);
|
||||
}
|
||||
|
||||
AsynchronousFileChannel file = null;
|
||||
|
@ -175,15 +197,24 @@ public final class ChunkUtils {
|
|||
verifyChecksum(chunkInfo, data, log);
|
||||
}
|
||||
int size = file.write(ByteBuffer.wrap(data), chunkInfo.getOffset()).get();
|
||||
if(size != data.length) {
|
||||
log.error("Invalid write size found. Size:{} Expected: {} " , size,
|
||||
if (size != data.length) {
|
||||
log.error("Invalid write size found. Size:{} Expected: {} ", size,
|
||||
data.length);
|
||||
throw new IOException("Invalid write size found. Size: " + size
|
||||
+ " Expected: " + data.length);
|
||||
throw new StorageContainerException("Invalid write size found. " +
|
||||
"Size: " + size + " Expected: " + data.length, INVALID_WRITE_SIZE);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new StorageContainerException(e, IO_EXCEPTION);
|
||||
|
||||
} finally {
|
||||
if (lock != null) {
|
||||
lock.release();
|
||||
try {
|
||||
lock.release();
|
||||
} catch (IOException e) {
|
||||
log.error("Unable to release lock ??, Fatal Error.");
|
||||
throw new StorageContainerException(e, CONTAINER_INTERNAL_ERROR);
|
||||
|
||||
}
|
||||
}
|
||||
if (file != null) {
|
||||
IOUtils.closeStream(file);
|
||||
|
@ -195,22 +226,22 @@ public final class ChunkUtils {
|
|||
* Verifies the checksum of a chunk against the data buffer.
|
||||
*
|
||||
* @param chunkInfo - Chunk Info.
|
||||
* @param data - data buffer
|
||||
* @param log - log
|
||||
* @param data - data buffer
|
||||
* @param log - log
|
||||
* @throws NoSuchAlgorithmException
|
||||
* @throws IOException
|
||||
* @throws StorageContainerException
|
||||
*/
|
||||
private static void verifyChecksum(ChunkInfo chunkInfo, byte[] data, Logger
|
||||
log) throws NoSuchAlgorithmException, IOException {
|
||||
log) throws NoSuchAlgorithmException, StorageContainerException {
|
||||
MessageDigest sha = MessageDigest.getInstance(OzoneConsts.FILE_HASH);
|
||||
sha.update(data);
|
||||
if (!Hex.encodeHexString(sha.digest()).equals(
|
||||
chunkInfo.getChecksum())) {
|
||||
log.error("Checksum mismatch. Provided: {} , computed: {}",
|
||||
chunkInfo.getChecksum(), DigestUtils.sha256Hex(sha.digest()));
|
||||
throw new IOException("Checksum mismatch. Provided: " +
|
||||
throw new StorageContainerException("Checksum mismatch. Provided: " +
|
||||
chunkInfo.getChecksum() + " , computed: " +
|
||||
DigestUtils.sha256Hex(sha.digest()));
|
||||
DigestUtils.sha256Hex(sha.digest()), CHECKSUM_MISMATCH);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -218,22 +249,25 @@ public final class ChunkUtils {
|
|||
* Reads data from an existing chunk file.
|
||||
*
|
||||
* @param chunkFile - file where data lives.
|
||||
* @param data - chunk definition.
|
||||
* @param data - chunk definition.
|
||||
*
|
||||
* @return ByteBuffer
|
||||
* @throws IOException
|
||||
*
|
||||
* @throws StorageContainerException
|
||||
* @throws ExecutionException
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
public static ByteBuffer readData(File chunkFile, ChunkInfo data) throws
|
||||
IOException, ExecutionException, InterruptedException,
|
||||
StorageContainerException, ExecutionException, InterruptedException,
|
||||
NoSuchAlgorithmException {
|
||||
Logger log = LoggerFactory.getLogger(ChunkManagerImpl.class);
|
||||
|
||||
if (!chunkFile.exists()) {
|
||||
log.error("Unable to find the chunk file. chunk info : {}",
|
||||
data.toString());
|
||||
throw new IOException("Unable to find the chunk file. chunk info " +
|
||||
data.toString());
|
||||
throw new StorageContainerException("Unable to find the chunk file. " +
|
||||
"chunk info " +
|
||||
data.toString(), UNABLE_TO_FIND_CHUNK);
|
||||
}
|
||||
|
||||
AsynchronousFileChannel file = null;
|
||||
|
@ -252,9 +286,15 @@ public final class ChunkUtils {
|
|||
}
|
||||
|
||||
return buf;
|
||||
} catch (IOException e) {
|
||||
throw new StorageContainerException(e, IO_EXCEPTION);
|
||||
} finally {
|
||||
if (lock != null) {
|
||||
lock.release();
|
||||
try {
|
||||
lock.release();
|
||||
} catch (IOException e) {
|
||||
log.error("I/O error is lock release.");
|
||||
}
|
||||
}
|
||||
if (file != null) {
|
||||
IOUtils.closeStream(file);
|
||||
|
@ -276,14 +316,15 @@ public final class ChunkUtils {
|
|||
|
||||
/**
|
||||
* Gets a response to the read chunk calls.
|
||||
*
|
||||
* @param msg - Msg
|
||||
* @param data - Data
|
||||
* @param info - Info
|
||||
* @return Response.
|
||||
* @param data - Data
|
||||
* @param info - Info
|
||||
* @return Response.
|
||||
*/
|
||||
public static ContainerProtos.ContainerCommandResponseProto
|
||||
getReadChunkResponse(ContainerProtos.ContainerCommandRequestProto msg,
|
||||
byte[] data, ChunkInfo info) {
|
||||
byte[] data, ChunkInfo info) {
|
||||
Preconditions.checkNotNull(msg);
|
||||
|
||||
ContainerProtos.ReadChunkResponseProto.Builder response =
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos;
|
|||
import org.apache.hadoop.ozone.OzoneConsts;
|
||||
import org.apache.hadoop.ozone.container.common.impl.ContainerManagerImpl;
|
||||
import org.apache.hadoop.utils.LevelDBStore;
|
||||
import org.apache.hadoop.scm.container.common.helpers.StorageContainerException;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -34,6 +35,10 @@ import java.nio.file.Path;
|
|||
import java.nio.file.Paths;
|
||||
|
||||
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_META;
|
||||
|
||||
|
@ -42,6 +47,10 @@ import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_META;
|
|||
*/
|
||||
public final class ContainerUtils {
|
||||
|
||||
private ContainerUtils() {
|
||||
//never constructed.
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a CreateContainer Response. This call is used by create and delete
|
||||
* containers which have null success responses.
|
||||
|
@ -59,13 +68,13 @@ public final class ContainerUtils {
|
|||
/**
|
||||
* Returns a ReadContainer Response.
|
||||
*
|
||||
* @param msg Request
|
||||
* @param msg Request
|
||||
* @param containerData - data
|
||||
* @return Response.
|
||||
*/
|
||||
public static ContainerProtos.ContainerCommandResponseProto
|
||||
getReadContainerResponse(ContainerProtos.ContainerCommandRequestProto msg,
|
||||
ContainerData containerData) {
|
||||
ContainerData containerData) {
|
||||
Preconditions.checkNotNull(containerData);
|
||||
|
||||
ContainerProtos.ReadContainerResponseProto.Builder response =
|
||||
|
@ -82,14 +91,14 @@ public final class ContainerUtils {
|
|||
* We found a command type but no associated payload for the command. Hence
|
||||
* return malformed Command as response.
|
||||
*
|
||||
* @param msg - Protobuf message.
|
||||
* @param result - result
|
||||
* @param msg - Protobuf message.
|
||||
* @param result - result
|
||||
* @param message - Error message.
|
||||
* @return ContainerCommandResponseProto - MALFORMED_REQUEST.
|
||||
*/
|
||||
public static ContainerProtos.ContainerCommandResponseProto.Builder
|
||||
getContainerResponse(ContainerProtos.ContainerCommandRequestProto msg,
|
||||
ContainerProtos.Result result, String message) {
|
||||
ContainerProtos.Result result, String message) {
|
||||
return
|
||||
ContainerProtos.ContainerCommandResponseProto.newBuilder()
|
||||
.setCmdType(msg.getCmdType())
|
||||
|
@ -98,6 +107,37 @@ public final class ContainerUtils {
|
|||
.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
|
||||
* return malformed Command as response.
|
||||
|
@ -133,14 +173,13 @@ public final class ContainerUtils {
|
|||
Preconditions.checkNotNull(containerFile);
|
||||
return Paths.get(containerFile.getParent()).resolve(
|
||||
removeExtension(containerFile.getName())).toString();
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Verifies that this in indeed a new container.
|
||||
*
|
||||
* @param containerFile - Container File to verify
|
||||
* @param metadataFile - metadata File to verify
|
||||
* @param metadataFile - metadata File to verify
|
||||
* @throws IOException
|
||||
*/
|
||||
public static void verifyIsNewContainer(File containerFile, File metadataFile)
|
||||
|
@ -227,11 +266,11 @@ public final class ContainerUtils {
|
|||
* Returns Metadata location.
|
||||
*
|
||||
* @param containerData - Data
|
||||
* @param location - Path
|
||||
* @param location - Path
|
||||
* @return Path
|
||||
*/
|
||||
public static File getMetadataFile(ContainerData containerData,
|
||||
Path location) {
|
||||
Path location) {
|
||||
return location.resolve(containerData
|
||||
.getContainerName().concat(CONTAINER_META))
|
||||
.toFile();
|
||||
|
@ -239,12 +278,13 @@ public final class ContainerUtils {
|
|||
|
||||
/**
|
||||
* Returns container file location.
|
||||
* @param containerData - Data
|
||||
*
|
||||
* @param containerData - Data
|
||||
* @param location - Root path
|
||||
* @return Path
|
||||
*/
|
||||
public static File getContainerFile(ContainerData containerData,
|
||||
Path location) {
|
||||
Path location) {
|
||||
return location.resolve(containerData
|
||||
.getContainerName().concat(CONTAINER_EXTENSION))
|
||||
.toFile();
|
||||
|
@ -252,6 +292,7 @@ public final class ContainerUtils {
|
|||
|
||||
/**
|
||||
* Container metadata directory -- here is where the level DB lives.
|
||||
*
|
||||
* @param cData - cData.
|
||||
* @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.
|
||||
*
|
||||
* @param cData - cData container
|
||||
* @return - Path
|
||||
* @throws StorageContainerException
|
||||
*/
|
||||
public static Path getDataDirectory(ContainerData cData) throws IOException {
|
||||
public static Path getDataDirectory(ContainerData cData)
|
||||
throws StorageContainerException {
|
||||
Path path = getMetadataDirectory(cData);
|
||||
Preconditions.checkNotNull(path);
|
||||
path = path.getParent();
|
||||
if(path == null) {
|
||||
throw new IOException("Unable to get Data directory. null path found");
|
||||
Path parentPath = path.getParent();
|
||||
if (parentPath == null) {
|
||||
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.
|
||||
* <p/>
|
||||
|
@ -323,8 +367,4 @@ public final class ContainerUtils {
|
|||
FileUtils.forceDelete(containerPath.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 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.utils.ContainerCache;
|
||||
import org.apache.hadoop.utils.LevelDBStore;
|
||||
|
@ -29,6 +30,9 @@ import java.io.File;
|
|||
import java.io.IOException;
|
||||
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.
|
||||
*/
|
||||
|
@ -60,18 +64,25 @@ public final class KeyUtils {
|
|||
* @param container - container.
|
||||
* @param cache - cache
|
||||
* @return LevelDB handle.
|
||||
* @throws IOException
|
||||
* @throws StorageContainerException
|
||||
*/
|
||||
public static LevelDBStore getDB(ContainerData container,
|
||||
ContainerCache cache) throws IOException {
|
||||
ContainerCache cache)
|
||||
throws StorageContainerException {
|
||||
Preconditions.checkNotNull(container);
|
||||
Preconditions.checkNotNull(cache);
|
||||
LevelDBStore db = cache.getDB(container.getContainerName());
|
||||
if (db == null) {
|
||||
db = getDB(container.getDBPath());
|
||||
cache.putDB(container.getContainerName(), db);
|
||||
try {
|
||||
LevelDBStore db = cache.getDB(container.getContainerName());
|
||||
if (db == null) {
|
||||
db = getDB(container.getDBPath());
|
||||
cache.putDB(container.getContainerName(), 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);
|
||||
}
|
||||
return db;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -107,8 +118,8 @@ public final class KeyUtils {
|
|||
|
||||
|
||||
public static ContainerProtos.ContainerCommandResponseProto
|
||||
getKeyDataResponse(ContainerProtos.ContainerCommandRequestProto msg
|
||||
, KeyData data) {
|
||||
getKeyDataResponse(ContainerProtos.ContainerCommandRequestProto msg,
|
||||
KeyData data) {
|
||||
ContainerProtos.GetKeyResponseProto.Builder getKey = ContainerProtos
|
||||
.GetKeyResponseProto.newBuilder();
|
||||
getKey.setKeyData(data.getProtoBufMessage());
|
||||
|
|
|
@ -19,19 +19,24 @@ package org.apache.hadoop.ozone.container.common.impl;
|
|||
|
||||
import com.google.common.base.Preconditions;
|
||||
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.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.ContainerManager;
|
||||
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.security.NoSuchAlgorithmException;
|
||||
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.
|
||||
*/
|
||||
|
@ -54,34 +59,36 @@ public class ChunkManagerImpl implements ChunkManager {
|
|||
* writes a given chunk.
|
||||
*
|
||||
* @param pipeline - Name and the set of machines that make this container.
|
||||
* @param keyName - Name of the Key.
|
||||
* @param info - ChunkInfo.
|
||||
* @throws IOException
|
||||
* @param keyName - Name of the Key.
|
||||
* @param info - ChunkInfo.
|
||||
* @throws StorageContainerException
|
||||
*/
|
||||
@Override
|
||||
public void writeChunk(Pipeline pipeline, String keyName, ChunkInfo info,
|
||||
byte[] data)
|
||||
throws IOException {
|
||||
byte[] data)
|
||||
throws StorageContainerException {
|
||||
|
||||
// we don't want container manager to go away while we are writing chunks.
|
||||
containerManager.readLock();
|
||||
|
||||
// TODO : Take keyManager Write lock here.
|
||||
try {
|
||||
Preconditions.checkNotNull(pipeline);
|
||||
Preconditions.checkNotNull(pipeline.getContainerName());
|
||||
Preconditions.checkNotNull(pipeline, "Pipeline cannot be null");
|
||||
Preconditions.checkNotNull(pipeline.getContainerName(),
|
||||
"Container name cannot be null");
|
||||
File chunkFile = ChunkUtils.validateChunk(pipeline,
|
||||
containerManager.readContainer(pipeline.getContainerName()), info);
|
||||
ChunkUtils.writeData(chunkFile, info, data);
|
||||
|
||||
} catch (ExecutionException |
|
||||
NoSuchAlgorithmException e) {
|
||||
} catch (ExecutionException | NoSuchAlgorithmException 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) {
|
||||
Thread.currentThread().interrupt();
|
||||
LOG.error("write data failed. error: {}", e);
|
||||
throw new IOException("Internal error: ", e);
|
||||
throw new StorageContainerException("Internal error: ", e,
|
||||
CONTAINER_INTERNAL_ERROR);
|
||||
} finally {
|
||||
containerManager.readUnlock();
|
||||
}
|
||||
|
@ -91,16 +98,16 @@ public class ChunkManagerImpl implements ChunkManager {
|
|||
* reads the data defined by a chunk.
|
||||
*
|
||||
* @param pipeline - container pipeline.
|
||||
* @param keyName - Name of the Key
|
||||
* @param info - ChunkInfo.
|
||||
* @param keyName - Name of the Key
|
||||
* @param info - ChunkInfo.
|
||||
* @return byte array
|
||||
* @throws IOException TODO: Right now we do not support partial reads and
|
||||
* writes of chunks. TODO: Explore if we need to do that
|
||||
* for ozone.
|
||||
* @throws StorageContainerException
|
||||
* TODO: Right now we do not support partial reads and writes of chunks.
|
||||
* TODO: Explore if we need to do that for ozone.
|
||||
*/
|
||||
@Override
|
||||
public byte[] readChunk(Pipeline pipeline, String keyName, ChunkInfo info)
|
||||
throws IOException {
|
||||
throws StorageContainerException {
|
||||
containerManager.readLock();
|
||||
try {
|
||||
File chunkFile = ChunkUtils.getChunkFile(pipeline, containerManager
|
||||
|
@ -108,11 +115,13 @@ public class ChunkManagerImpl implements ChunkManager {
|
|||
return ChunkUtils.readData(chunkFile, info).array();
|
||||
} catch (ExecutionException | NoSuchAlgorithmException 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) {
|
||||
Thread.currentThread().interrupt();
|
||||
LOG.error("read data failed. error: {}", e);
|
||||
throw new IOException("Internal error: ", e);
|
||||
throw new StorageContainerException("Internal error: ",
|
||||
e, CONTAINER_INTERNAL_ERROR);
|
||||
} finally {
|
||||
containerManager.readUnlock();
|
||||
}
|
||||
|
@ -122,13 +131,13 @@ public class ChunkManagerImpl implements ChunkManager {
|
|||
* Deletes a given chunk.
|
||||
*
|
||||
* @param pipeline - Pipeline.
|
||||
* @param keyName - Key Name
|
||||
* @param info - Chunk Info
|
||||
* @throws IOException
|
||||
* @param keyName - Key Name
|
||||
* @param info - Chunk Info
|
||||
* @throws StorageContainerException
|
||||
*/
|
||||
@Override
|
||||
public void deleteChunk(Pipeline pipeline, String keyName, ChunkInfo info)
|
||||
throws IOException {
|
||||
throws StorageContainerException {
|
||||
|
||||
containerManager.readLock();
|
||||
try {
|
||||
|
@ -139,8 +148,9 @@ public class ChunkManagerImpl implements ChunkManager {
|
|||
} else {
|
||||
LOG.error("Not Supported Operation. Trying to delete a " +
|
||||
"chunk that is in shared file. chunk info : " + info.toString());
|
||||
throw new IOException("Not Supported Operation. Trying to delete a " +
|
||||
"chunk that is in shared file. chunk info : " + info.toString());
|
||||
throw new StorageContainerException("Not Supported Operation. " +
|
||||
"Trying to delete a chunk that is in shared file. chunk info : "
|
||||
+ info.toString(), UNSUPPORTED_REQUEST);
|
||||
}
|
||||
} finally {
|
||||
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.ChunkUtils;
|
||||
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.FileUtils;
|
||||
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.ContainerMetrics;
|
||||
import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
|
||||
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.StorageContainerException;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -45,6 +46,11 @@ import java.io.IOException;
|
|||
import java.util.LinkedList;
|
||||
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
|
||||
* to the right handler function.
|
||||
|
@ -55,6 +61,7 @@ public class Dispatcher implements ContainerDispatcher {
|
|||
private final ContainerManager containerManager;
|
||||
private ContainerMetrics metrics;
|
||||
private Configuration conf;
|
||||
|
||||
/**
|
||||
* Constructs an OzoneContainer that receives calls from
|
||||
* XceiverServerHandler.
|
||||
|
@ -79,59 +86,67 @@ public class Dispatcher implements ContainerDispatcher {
|
|||
|
||||
@Override
|
||||
public ContainerCommandResponseProto dispatch(
|
||||
ContainerCommandRequestProto msg) throws IOException {
|
||||
ContainerCommandRequestProto msg) {
|
||||
long startNanos = System.nanoTime();
|
||||
ContainerCommandResponseProto resp = null;
|
||||
Preconditions.checkNotNull(msg);
|
||||
Type cmdType = msg.getCmdType();
|
||||
metrics.incContainerOpcMetrics(cmdType);
|
||||
if ((cmdType == Type.CreateContainer) ||
|
||||
(cmdType == Type.DeleteContainer) ||
|
||||
(cmdType == Type.ReadContainer) ||
|
||||
(cmdType == Type.ListContainer) ||
|
||||
(cmdType == Type.UpdateContainer)) {
|
||||
resp = containerProcessHandler(msg);
|
||||
}
|
||||
try {
|
||||
Preconditions.checkNotNull(msg);
|
||||
Type cmdType = msg.getCmdType();
|
||||
metrics.incContainerOpcMetrics(cmdType);
|
||||
if ((cmdType == Type.CreateContainer) ||
|
||||
(cmdType == Type.DeleteContainer) ||
|
||||
(cmdType == Type.ReadContainer) ||
|
||||
(cmdType == Type.ListContainer) ||
|
||||
(cmdType == Type.UpdateContainer)) {
|
||||
resp = containerProcessHandler(msg);
|
||||
}
|
||||
|
||||
if ((cmdType == Type.PutKey) ||
|
||||
(cmdType == Type.GetKey) ||
|
||||
(cmdType == Type.DeleteKey) ||
|
||||
(cmdType == Type.ListKey)) {
|
||||
resp = keyProcessHandler(msg);
|
||||
}
|
||||
if ((cmdType == Type.PutKey) ||
|
||||
(cmdType == Type.GetKey) ||
|
||||
(cmdType == Type.DeleteKey) ||
|
||||
(cmdType == Type.ListKey)) {
|
||||
resp = keyProcessHandler(msg);
|
||||
}
|
||||
|
||||
if ((cmdType == Type.WriteChunk) ||
|
||||
(cmdType == Type.ReadChunk) ||
|
||||
(cmdType == Type.DeleteChunk)) {
|
||||
resp = chunkProcessHandler(msg);
|
||||
}
|
||||
if ((cmdType == Type.WriteChunk) ||
|
||||
(cmdType == Type.ReadChunk) ||
|
||||
(cmdType == Type.DeleteChunk)) {
|
||||
resp = chunkProcessHandler(msg);
|
||||
}
|
||||
|
||||
if ((cmdType == Type.PutSmallFile) ||
|
||||
(cmdType == Type.GetSmallFile)) {
|
||||
resp = smallFileHandler(msg);
|
||||
}
|
||||
if ((cmdType == Type.PutSmallFile) ||
|
||||
(cmdType == Type.GetSmallFile)) {
|
||||
resp = smallFileHandler(msg);
|
||||
}
|
||||
|
||||
if (resp != null) {
|
||||
metrics.incContainerOpsLatencies(cmdType,
|
||||
System.nanoTime() - startNanos);
|
||||
return resp;
|
||||
}
|
||||
if (resp != null) {
|
||||
metrics.incContainerOpsLatencies(cmdType,
|
||||
System.nanoTime() - startNanos);
|
||||
return resp;
|
||||
}
|
||||
|
||||
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() {
|
||||
return metrics;
|
||||
}
|
||||
|
||||
/**
|
||||
* Handles the all Container related functionality.
|
||||
*
|
||||
* @param msg - command
|
||||
* @return - response
|
||||
* @throws IOException
|
||||
* @throws StorageContainerException
|
||||
*/
|
||||
private ContainerCommandResponseProto containerProcessHandler(
|
||||
ContainerCommandRequestProto msg) throws IOException {
|
||||
ContainerCommandRequestProto msg) throws StorageContainerException {
|
||||
try {
|
||||
|
||||
switch (msg.getCmdType()) {
|
||||
|
@ -175,10 +190,10 @@ public class Dispatcher implements ContainerDispatcher {
|
|||
*
|
||||
* @param msg - command
|
||||
* @return - response
|
||||
* @throws IOException
|
||||
* @throws StorageContainerException
|
||||
*/
|
||||
private ContainerCommandResponseProto keyProcessHandler(
|
||||
ContainerCommandRequestProto msg) throws IOException {
|
||||
ContainerCommandRequestProto msg) throws StorageContainerException {
|
||||
try {
|
||||
switch (msg.getCmdType()) {
|
||||
case PutKey:
|
||||
|
@ -217,10 +232,10 @@ public class Dispatcher implements ContainerDispatcher {
|
|||
*
|
||||
* @param msg - command
|
||||
* @return - response
|
||||
* @throws IOException
|
||||
* @throws StorageContainerException
|
||||
*/
|
||||
private ContainerCommandResponseProto chunkProcessHandler(
|
||||
ContainerCommandRequestProto msg) throws IOException {
|
||||
ContainerCommandRequestProto msg) throws StorageContainerException {
|
||||
try {
|
||||
switch (msg.getCmdType()) {
|
||||
case WriteChunk:
|
||||
|
@ -254,7 +269,7 @@ public class Dispatcher implements ContainerDispatcher {
|
|||
}
|
||||
|
||||
private ContainerCommandResponseProto smallFileHandler(
|
||||
ContainerCommandRequestProto msg) throws IOException {
|
||||
ContainerCommandRequestProto msg) throws StorageContainerException {
|
||||
switch (msg.getCmdType()) {
|
||||
case PutSmallFile:
|
||||
return handlePutSmallFile(msg);
|
||||
|
@ -494,7 +509,6 @@ public class Dispatcher implements ContainerDispatcher {
|
|||
String keyName = msg.getDeleteKey().getName();
|
||||
Preconditions.checkNotNull(keyName);
|
||||
Preconditions.checkState(!keyName.isEmpty());
|
||||
|
||||
this.containerManager.getKeyManager().deleteKey(pipeline, keyName);
|
||||
return KeyUtils.getKeyResponse(msg);
|
||||
}
|
||||
|
@ -504,34 +518,41 @@ public class Dispatcher implements ContainerDispatcher {
|
|||
*
|
||||
* @param msg - Message.
|
||||
* @return ContainerCommandResponseProto
|
||||
* @throws IOException
|
||||
* @throws StorageContainerException
|
||||
*/
|
||||
private ContainerCommandResponseProto handlePutSmallFile(
|
||||
ContainerCommandRequestProto msg) throws IOException {
|
||||
ContainerCommandRequestProto msg) throws StorageContainerException {
|
||||
|
||||
if (!msg.hasPutSmallFile()) {
|
||||
LOG.debug("Malformed put small file request. trace ID: {}",
|
||||
msg.getTraceID());
|
||||
return ContainerUtils.malformedRequest(msg);
|
||||
}
|
||||
try {
|
||||
|
||||
Pipeline pipeline =
|
||||
Pipeline.getFromProtoBuf(msg.getPutSmallFile().getKey().getPipeline());
|
||||
Preconditions.checkNotNull(pipeline);
|
||||
KeyData keyData = KeyData.getFromProtoBuf(msg.getPutSmallFile().getKey()
|
||||
.getKeyData());
|
||||
ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf(msg.getPutSmallFile()
|
||||
.getChunkInfo());
|
||||
byte[] data = msg.getPutSmallFile().getData().toByteArray();
|
||||
Pipeline pipeline =
|
||||
Pipeline.getFromProtoBuf(msg.getPutSmallFile()
|
||||
.getKey().getPipeline());
|
||||
|
||||
metrics.incContainerBytesStats(Type.PutSmallFile, data.length);
|
||||
this.containerManager.getChunkManager().writeChunk(pipeline, keyData
|
||||
.getKeyName(), chunkInfo, data);
|
||||
List<ContainerProtos.ChunkInfo> chunks = new LinkedList<>();
|
||||
chunks.add(chunkInfo.getProtoBufMessage());
|
||||
keyData.setChunks(chunks);
|
||||
this.containerManager.getKeyManager().putKey(pipeline, keyData);
|
||||
return FileUtils.getPutFileResponse(msg);
|
||||
Preconditions.checkNotNull(pipeline);
|
||||
KeyData keyData = KeyData.getFromProtoBuf(msg.getPutSmallFile().getKey()
|
||||
.getKeyData());
|
||||
ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf(msg.getPutSmallFile()
|
||||
.getChunkInfo());
|
||||
byte[] data = msg.getPutSmallFile().getData().toByteArray();
|
||||
|
||||
metrics.incContainerBytesStats(Type.PutSmallFile, data.length);
|
||||
this.containerManager.getChunkManager().writeChunk(pipeline, keyData
|
||||
.getKeyName(), chunkInfo, data);
|
||||
List<ContainerProtos.ChunkInfo> chunks = new LinkedList<>();
|
||||
chunks.add(chunkInfo.getProtoBufMessage());
|
||||
keyData.setChunks(chunks);
|
||||
this.containerManager.getKeyManager().putKey(pipeline, keyData);
|
||||
return FileUtils.getPutFileResponse(msg);
|
||||
} catch (IOException e) {
|
||||
throw new StorageContainerException("Put Small File Failed.", e,
|
||||
PUT_SMALL_FILE_ERROR);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -540,35 +561,44 @@ public class Dispatcher implements ContainerDispatcher {
|
|||
*
|
||||
* @param msg - ContainerCommandRequestProto
|
||||
* @return ContainerCommandResponseProto
|
||||
* @throws StorageContainerException
|
||||
*/
|
||||
private ContainerCommandResponseProto handleGetSmallFile(
|
||||
ContainerCommandRequestProto msg) throws IOException {
|
||||
ContainerCommandRequestProto msg) throws StorageContainerException {
|
||||
ByteString dataBuf = ByteString.EMPTY;
|
||||
if (!msg.hasGetSmallFile()) {
|
||||
LOG.debug("Malformed get small file request. trace ID: {}",
|
||||
msg.getTraceID());
|
||||
return ContainerUtils.malformedRequest(msg);
|
||||
}
|
||||
Pipeline pipeline =
|
||||
Pipeline.getFromProtoBuf(msg.getGetSmallFile().getKey().getPipeline());
|
||||
long bytes = 0;
|
||||
Preconditions.checkNotNull(pipeline);
|
||||
KeyData keyData = KeyData.getFromProtoBuf(msg.getGetSmallFile()
|
||||
.getKey().getKeyData());
|
||||
KeyData data = this.containerManager.getKeyManager().getKey(keyData);
|
||||
ContainerProtos.ChunkInfo c = null;
|
||||
for (ContainerProtos.ChunkInfo chunk : data.getChunks()) {
|
||||
bytes += chunk.getSerializedSize();
|
||||
ByteString current =
|
||||
ByteString.copyFrom(this.containerManager.getChunkManager().readChunk(
|
||||
pipeline, keyData.getKeyName(), ChunkInfo.getFromProtoBuf(
|
||||
chunk)));
|
||||
dataBuf = dataBuf.concat(current);
|
||||
c = chunk;
|
||||
}
|
||||
metrics.incContainerBytesStats(Type.GetSmallFile, bytes);
|
||||
return FileUtils.getGetSmallFileResponse(msg, dataBuf.toByteArray(),
|
||||
ChunkInfo.getFromProtoBuf(c));
|
||||
}
|
||||
try {
|
||||
Pipeline pipeline =
|
||||
Pipeline.getFromProtoBuf(msg.getGetSmallFile()
|
||||
.getKey().getPipeline());
|
||||
|
||||
long bytes = 0;
|
||||
Preconditions.checkNotNull(pipeline);
|
||||
KeyData keyData = KeyData.getFromProtoBuf(msg.getGetSmallFile()
|
||||
.getKey().getKeyData());
|
||||
KeyData data = this.containerManager.getKeyManager().getKey(keyData);
|
||||
ContainerProtos.ChunkInfo c = null;
|
||||
for (ContainerProtos.ChunkInfo chunk : data.getChunks()) {
|
||||
bytes += chunk.getSerializedSize();
|
||||
ByteString current =
|
||||
ByteString.copyFrom(this.containerManager.getChunkManager()
|
||||
.readChunk(
|
||||
pipeline, keyData.getKeyName(), ChunkInfo.getFromProtoBuf(
|
||||
chunk)));
|
||||
dataBuf = dataBuf.concat(current);
|
||||
c = chunk;
|
||||
}
|
||||
metrics.incContainerBytesStats(Type.GetSmallFile, bytes);
|
||||
return FileUtils.getGetSmallFileResponse(msg, dataBuf.toByteArray(),
|
||||
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.KeyData;
|
||||
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.KeyManager;
|
||||
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.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
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.
|
||||
*/
|
||||
public class KeyManagerImpl implements KeyManager {
|
||||
static final Logger LOG =
|
||||
LoggerFactory.getLogger(KeyManagerImpl.class);
|
||||
|
||||
private static final float LOAD_FACTOR = 0.75f;
|
||||
private final ContainerManager containerManager;
|
||||
private final ContainerCache containerCache;
|
||||
|
@ -48,8 +60,9 @@ public class KeyManagerImpl implements KeyManager {
|
|||
* @param containerManager - Container Manager.
|
||||
*/
|
||||
public KeyManagerImpl(ContainerManager containerManager, Configuration conf) {
|
||||
Preconditions.checkNotNull(containerManager);
|
||||
Preconditions.checkNotNull(conf);
|
||||
Preconditions.checkNotNull(containerManager, "Container manager cannot be" +
|
||||
" null");
|
||||
Preconditions.checkNotNull(conf, "Config cannot be null");
|
||||
int cacheSize = conf.getInt(OzoneConfigKeys.OZONE_KEY_CACHE,
|
||||
OzoneConfigKeys.OZONE_KEY_CACHE_DEFAULT);
|
||||
this.containerManager = containerManager;
|
||||
|
@ -60,17 +73,22 @@ public class KeyManagerImpl implements KeyManager {
|
|||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void putKey(Pipeline pipeline, KeyData data) throws IOException {
|
||||
public void putKey(Pipeline pipeline, KeyData data)
|
||||
throws StorageContainerException {
|
||||
containerManager.readLock();
|
||||
try {
|
||||
// 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.
|
||||
Preconditions.checkNotNull(pipeline);
|
||||
Preconditions.checkNotNull(pipeline.getContainerName());
|
||||
Preconditions.checkNotNull(pipeline, "Pipeline cannot be null");
|
||||
Preconditions.checkNotNull(pipeline.getContainerName(),
|
||||
"Container name cannot be null");
|
||||
ContainerData cData = containerManager.readContainer(
|
||||
pipeline.getContainerName());
|
||||
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
|
||||
.getProtoBufMessage().toByteArray());
|
||||
} finally {
|
||||
|
@ -83,22 +101,30 @@ public class KeyManagerImpl implements KeyManager {
|
|||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public KeyData getKey(KeyData data) throws IOException {
|
||||
public KeyData getKey(KeyData data) throws StorageContainerException {
|
||||
containerManager.readLock();
|
||||
try {
|
||||
Preconditions.checkNotNull(data);
|
||||
Preconditions.checkNotNull(data.getContainerName());
|
||||
Preconditions.checkNotNull(data, "Key data cannot be null");
|
||||
Preconditions.checkNotNull(data.getContainerName(),
|
||||
"Container name cannot be null");
|
||||
ContainerData cData = containerManager.readContainer(data
|
||||
.getContainerName());
|
||||
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));
|
||||
if(kData == null) {
|
||||
throw new IOException("Unable to find the key.");
|
||||
if (kData == null) {
|
||||
throw new StorageContainerException("Unable to find the key.",
|
||||
NO_SUCH_KEY);
|
||||
}
|
||||
ContainerProtos.KeyData keyData =
|
||||
ContainerProtos.KeyData.parseFrom(kData);
|
||||
return KeyData.getFromProtoBuf(keyData);
|
||||
} catch (IOException ex) {
|
||||
throw new StorageContainerException(ex, IO_EXCEPTION);
|
||||
} finally {
|
||||
containerManager.readUnlock();
|
||||
}
|
||||
|
@ -108,24 +134,29 @@ public class KeyManagerImpl implements KeyManager {
|
|||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void deleteKey(Pipeline pipeline, String keyName) throws IOException {
|
||||
public void deleteKey(Pipeline pipeline, String keyName)
|
||||
throws StorageContainerException {
|
||||
containerManager.readLock();
|
||||
try {
|
||||
Preconditions.checkNotNull(pipeline);
|
||||
Preconditions.checkNotNull(pipeline.getContainerName());
|
||||
Preconditions.checkNotNull(pipeline, "Pipeline cannot be null");
|
||||
Preconditions.checkNotNull(pipeline.getContainerName(),
|
||||
"Container name cannot be null");
|
||||
ContainerData cData = containerManager.readContainer(pipeline
|
||||
.getContainerName());
|
||||
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
|
||||
// are not atomic. Leaving it here since the impact is refusing
|
||||
// to delete a key which might have just gotten inserted after
|
||||
// the get check.
|
||||
|
||||
byte[] kData = db.get(keyName.getBytes(KeyUtils.ENCODING));
|
||||
if(kData == null) {
|
||||
throw new IOException("Unable to find the key.");
|
||||
if (kData == null) {
|
||||
throw new StorageContainerException("Unable to find the key.",
|
||||
NO_SUCH_KEY);
|
||||
}
|
||||
db.delete(keyName.getBytes(KeyUtils.ENCODING));
|
||||
} finally {
|
||||
|
@ -139,7 +170,7 @@ public class KeyManagerImpl implements KeyManager {
|
|||
@Override
|
||||
public List<KeyData> listKey(Pipeline pipeline, String prefix, String
|
||||
prevKey, int count) {
|
||||
// TODO :
|
||||
// TODO : Implement listKey function.
|
||||
return null;
|
||||
}
|
||||
|
||||
|
@ -148,7 +179,8 @@ public class KeyManagerImpl implements KeyManager {
|
|||
*/
|
||||
@Override
|
||||
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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,11 +18,10 @@
|
|||
|
||||
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.scm.container.common.helpers.Pipeline;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Chunk Manager allows read, write, delete and listing of chunks in
|
||||
* a container.
|
||||
|
@ -34,10 +33,10 @@ public interface ChunkManager {
|
|||
* @param pipeline - Name and the set of machines that make this container.
|
||||
* @param keyName - Name of the Key.
|
||||
* @param info - ChunkInfo.
|
||||
* @throws IOException
|
||||
* @throws StorageContainerException
|
||||
*/
|
||||
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.
|
||||
|
@ -45,23 +44,23 @@ public interface ChunkManager {
|
|||
* @param keyName - Name of the Key
|
||||
* @param info - ChunkInfo.
|
||||
* @return byte array
|
||||
* @throws IOException
|
||||
* @throws StorageContainerException
|
||||
*
|
||||
* TODO: Right now we do not support partial reads and writes of chunks.
|
||||
* TODO: Explore if we need to do that for ozone.
|
||||
*/
|
||||
byte[] readChunk(Pipeline pipeline, String keyName, ChunkInfo info) throws
|
||||
IOException;
|
||||
StorageContainerException;
|
||||
|
||||
/**
|
||||
* Deletes a given chunk.
|
||||
* @param pipeline - Pipeline.
|
||||
* @param keyName - Key Name
|
||||
* @param info - Chunk Info
|
||||
* @throws IOException
|
||||
* @throws StorageContainerException
|
||||
*/
|
||||
void deleteChunk(Pipeline pipeline, String keyName, ChunkInfo info) throws
|
||||
IOException;
|
||||
StorageContainerException;
|
||||
|
||||
// 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.ContainerCommandRequestProto;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Dispatcher acts as the bridge between the transport layer and
|
||||
* the actual container layer. This layer is capable of transforming
|
||||
|
@ -36,10 +34,8 @@ public interface ContainerDispatcher {
|
|||
* Dispatches commands to container layer.
|
||||
* @param msg - Command Request
|
||||
* @return Command Response
|
||||
* @throws IOException
|
||||
*/
|
||||
ContainerCommandResponseProto dispatch(ContainerCommandRequestProto msg)
|
||||
throws IOException;
|
||||
ContainerCommandResponseProto dispatch(ContainerCommandRequestProto msg);
|
||||
|
||||
/**
|
||||
* Initialize the Dispatcher.
|
||||
|
|
|
@ -23,6 +23,7 @@ import org.apache.hadoop.classification.InterfaceStability;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
|
||||
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.protocol.proto
|
||||
.StorageContainerDatanodeProtocolProtos.SCMNodeReport;
|
||||
|
@ -43,7 +44,7 @@ public interface ContainerManager extends RwLock {
|
|||
*
|
||||
* @param config - Configuration.
|
||||
* @param containerDirs - List of Metadata Container locations.
|
||||
* @throws IOException
|
||||
* @throws StorageContainerException
|
||||
*/
|
||||
void init(Configuration config, List<StorageLocation> containerDirs)
|
||||
throws IOException;
|
||||
|
@ -53,20 +54,20 @@ public interface ContainerManager extends RwLock {
|
|||
*
|
||||
* @param pipeline -- Nodes which make up this container.
|
||||
* @param containerData - Container Name and metadata.
|
||||
* @throws IOException
|
||||
* @throws StorageContainerException
|
||||
*/
|
||||
void createContainer(Pipeline pipeline, ContainerData containerData)
|
||||
throws IOException;
|
||||
throws StorageContainerException;
|
||||
|
||||
/**
|
||||
* Deletes an existing container.
|
||||
*
|
||||
* @param pipeline - nodes that make this container.
|
||||
* @param containerName - name of the container.
|
||||
* @throws IOException
|
||||
* @throws StorageContainerException
|
||||
*/
|
||||
void deleteContainer(Pipeline pipeline, String containerName)
|
||||
throws IOException;
|
||||
throws StorageContainerException;
|
||||
|
||||
/**
|
||||
* As simple interface for container Iterations.
|
||||
|
@ -75,25 +76,26 @@ public interface ContainerManager extends RwLock {
|
|||
* @param count - how many to return
|
||||
* @param prevKey - Previous key - Server returns results from this point.
|
||||
* @param data - Actual containerData
|
||||
* @throws IOException
|
||||
* @throws StorageContainerException
|
||||
*/
|
||||
void listContainer(String prefix, long count, String prevKey,
|
||||
List<ContainerData> data)
|
||||
throws IOException;
|
||||
throws StorageContainerException;
|
||||
|
||||
/**
|
||||
* Get metadata about a specific container.
|
||||
*
|
||||
* @param containerName - Name of the container
|
||||
* @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.
|
||||
*
|
||||
* @throws IOException
|
||||
* @throws StorageContainerException
|
||||
*/
|
||||
void shutdown() throws IOException;
|
||||
|
||||
|
|
|
@ -17,10 +17,10 @@
|
|||
*/
|
||||
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.scm.container.common.helpers.Pipeline;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
|
@ -32,24 +32,28 @@ public interface KeyManager {
|
|||
*
|
||||
* @param pipeline - Pipeline.
|
||||
* @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.
|
||||
*
|
||||
* @param data - Key Data.
|
||||
* @return Key Data.
|
||||
* @throws StorageContainerException
|
||||
*/
|
||||
KeyData getKey(KeyData data) throws IOException;
|
||||
KeyData getKey(KeyData data) throws StorageContainerException;
|
||||
|
||||
/**
|
||||
* Deletes an existing Key.
|
||||
*
|
||||
* @param pipeline - Pipeline.
|
||||
* @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.
|
||||
|
|
|
@ -44,9 +44,15 @@ import java.util.UUID;
|
|||
/**
|
||||
* Helpers for container tests.
|
||||
*/
|
||||
public class ContainerTestHelper {
|
||||
public final class ContainerTestHelper {
|
||||
private static Random r = new Random();
|
||||
|
||||
/**
|
||||
* Never constructed.
|
||||
*/
|
||||
private ContainerTestHelper() {
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a pipeline with single node replica.
|
||||
*
|
||||
|
@ -291,8 +297,7 @@ public class ContainerTestHelper {
|
|||
* @return ContainerCommandRequestProto.
|
||||
*/
|
||||
public static ContainerCommandResponseProto
|
||||
getCreateContainerResponse(ContainerCommandRequestProto request) throws
|
||||
IOException {
|
||||
getCreateContainerResponse(ContainerCommandRequestProto request) {
|
||||
ContainerProtos.CreateContainerResponseProto.Builder createResponse =
|
||||
ContainerProtos.CreateContainerResponseProto.newBuilder();
|
||||
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.hadoop.ozone.MiniOzoneCluster;
|
|||
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
||||
import org.apache.hadoop.ozone.OzoneConfiguration;
|
||||
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.ContainerData;
|
||||
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
|
||||
|
@ -121,8 +122,8 @@ public class TestContainerPersistence {
|
|||
|
||||
@Before
|
||||
public void setupPaths() throws IOException {
|
||||
if (!new File(path).exists()) {
|
||||
new File(path).mkdirs();
|
||||
if (!new File(path).exists() && !new File(path).mkdirs()) {
|
||||
throw new IOException("Unable to create paths. " + path);
|
||||
}
|
||||
pathLists.clear();
|
||||
containerManager.getContainerMap().clear();
|
||||
|
@ -157,11 +158,8 @@ public class TestContainerPersistence {
|
|||
Assert.assertTrue(new File(status.getContainer().getContainerPath())
|
||||
.exists());
|
||||
|
||||
String containerPathString = ContainerUtils.getContainerNameFromFile(new
|
||||
File(status.getContainer().getContainerPath()));
|
||||
|
||||
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();
|
||||
|
@ -363,8 +361,7 @@ public class TestContainerPersistence {
|
|||
sha.update(FileUtils.readFileToByteArray(fname.toFile()));
|
||||
String val = Hex.encodeHexString(sha.digest());
|
||||
Assert.assertEquals(fileHashMap.get(fname.getFileName().toString())
|
||||
.getChecksum(),
|
||||
val);
|
||||
.getChecksum(), val);
|
||||
count++;
|
||||
sha.reset();
|
||||
}
|
||||
|
@ -486,7 +483,7 @@ public class TestContainerPersistence {
|
|||
setDataChecksum(info, data);
|
||||
chunkManager.writeChunk(pipeline, keyName, info, data);
|
||||
chunkManager.deleteChunk(pipeline, keyName, info);
|
||||
exception.expect(IOException.class);
|
||||
exception.expect(StorageContainerException.class);
|
||||
exception.expectMessage("Unable to find the chunk file.");
|
||||
chunkManager.readChunk(pipeline, keyName, info);
|
||||
}
|
||||
|
@ -572,7 +569,7 @@ public class TestContainerPersistence {
|
|||
keyData.setChunks(chunkList);
|
||||
keyManager.putKey(pipeline, keyData);
|
||||
keyManager.deleteKey(pipeline, keyName);
|
||||
exception.expect(IOException.class);
|
||||
exception.expect(StorageContainerException.class);
|
||||
exception.expectMessage("Unable to find the key.");
|
||||
keyManager.getKey(keyData);
|
||||
}
|
||||
|
@ -596,7 +593,7 @@ public class TestContainerPersistence {
|
|||
keyData.setChunks(chunkList);
|
||||
keyManager.putKey(pipeline, keyData);
|
||||
keyManager.deleteKey(pipeline, keyName);
|
||||
exception.expect(IOException.class);
|
||||
exception.expect(StorageContainerException.class);
|
||||
exception.expectMessage("Unable to find the key.");
|
||||
keyManager.deleteKey(pipeline, keyName);
|
||||
}
|
||||
|
|
|
@ -41,6 +41,9 @@ import java.io.IOException;
|
|||
|
||||
import static org.mockito.Mockito.mock;
|
||||
|
||||
/**
|
||||
* Test Containers.
|
||||
*/
|
||||
public class TestContainerServer {
|
||||
|
||||
@Test
|
||||
|
@ -69,8 +72,8 @@ public class TestContainerServer {
|
|||
XceiverClient client = null;
|
||||
String containerName = OzoneUtils.getRequestID();
|
||||
try {
|
||||
Pipeline pipeline = ContainerTestHelper.createSingleNodePipeline
|
||||
(containerName);
|
||||
Pipeline pipeline = ContainerTestHelper.createSingleNodePipeline(
|
||||
containerName);
|
||||
OzoneConfiguration conf = new OzoneConfiguration();
|
||||
conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
|
||||
pipeline.getLeader().getContainerPort());
|
||||
|
@ -102,8 +105,8 @@ public class TestContainerServer {
|
|||
String containerName = OzoneUtils.getRequestID();
|
||||
|
||||
try {
|
||||
Pipeline pipeline = ContainerTestHelper.createSingleNodePipeline
|
||||
(containerName);
|
||||
Pipeline pipeline = ContainerTestHelper.createSingleNodePipeline(
|
||||
containerName);
|
||||
OzoneConfiguration conf = new OzoneConfiguration();
|
||||
conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
|
||||
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.
|
||||
*
|
||||
* @param msg - Command Request
|
||||
* @return Command Response
|
||||
* @throws IOException
|
||||
*/
|
||||
@Override
|
||||
public ContainerCommandResponseProto
|
||||
dispatch(ContainerCommandRequestProto msg) throws IOException {
|
||||
dispatch(ContainerCommandRequestProto msg) {
|
||||
return ContainerTestHelper.getCreateContainerResponse(msg);
|
||||
}
|
||||
|
||||
|
|
|
@ -49,6 +49,8 @@ public class TestAllocateContainer {
|
|||
conf = new OzoneConfiguration();
|
||||
cluster = new MiniOzoneCluster.Builder(conf).numDataNodes(1)
|
||||
.setHandlerType("distributed").build();
|
||||
storageContainerLocationClient =
|
||||
cluster.createStorageContainerLocationClient();
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
|
@ -61,8 +63,6 @@ public class TestAllocateContainer {
|
|||
|
||||
@Test
|
||||
public void testAllocate() throws Exception {
|
||||
storageContainerLocationClient =
|
||||
cluster.createStorageContainerLocationClient();
|
||||
Pipeline pipeline = storageContainerLocationClient.allocateContainer(
|
||||
"container0");
|
||||
Assert.assertNotNull(pipeline);
|
||||
|
@ -72,8 +72,6 @@ public class TestAllocateContainer {
|
|||
|
||||
@Test
|
||||
public void testAllocateNull() throws Exception {
|
||||
storageContainerLocationClient =
|
||||
cluster.createStorageContainerLocationClient();
|
||||
thrown.expect(NullPointerException.class);
|
||||
storageContainerLocationClient.allocateContainer(null);
|
||||
}
|
||||
|
@ -81,12 +79,9 @@ public class TestAllocateContainer {
|
|||
@Test
|
||||
public void testAllocateDuplicate() throws Exception {
|
||||
String containerName = RandomStringUtils.randomAlphanumeric(10);
|
||||
storageContainerLocationClient =
|
||||
cluster.createStorageContainerLocationClient();
|
||||
thrown.expect(IOException.class);
|
||||
thrown.expectMessage("Specified container already exists");
|
||||
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