HDFS-11612. Ozone: Cleanup Checkstyle issues. Contributed by Shashikant Banerjee.

This commit is contained in:
Anu Engineer 2017-09-15 13:49:12 -07:00 committed by Owen O'Malley
parent b127ecd1ba
commit fe4139bdc3
47 changed files with 496 additions and 251 deletions

View File

@ -0,0 +1,22 @@
/**
* 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.cblock;
/**
This package contains ozone client side libraries.
*/

View File

@ -115,13 +115,13 @@ public Builder setDataSize(long size) {
return this; return this;
} }
public Builder setCreationTime(long creationTime) { public Builder setCreationTime(long crTime) {
this.creationTime = creationTime; this.creationTime = crTime;
return this; return this;
} }
public Builder setModificationTime(long modificationTime) { public Builder setModificationTime(long mTime) {
this.modificationTime = modificationTime; this.modificationTime = mTime;
return this; return this;
} }

View File

@ -142,18 +142,18 @@ public static class Builder {
aclMap = new KsmOzoneAclMap(); aclMap = new KsmOzoneAclMap();
} }
public Builder setAdminName(String adminName) { public Builder setAdminName(String admin) {
this.adminName = adminName; this.adminName = admin;
return this; return this;
} }
public Builder setOwnerName(String ownerName) { public Builder setOwnerName(String owner) {
this.ownerName = ownerName; this.ownerName = owner;
return this; return this;
} }
public Builder setVolume(String volume) { public Builder setVolume(String volumeName) {
this.volume = volume; this.volume = volumeName;
return this; return this;
} }
@ -162,8 +162,8 @@ public Builder setCreationTime(long createdOn) {
return this; return this;
} }
public Builder setQuotaInBytes(long quotaInBytes) { public Builder setQuotaInBytes(long quota) {
this.quotaInBytes = quotaInBytes; this.quotaInBytes = quota;
return this; return this;
} }

View File

@ -123,11 +123,11 @@ public ContainerProtos.ContainerCommandResponseProto sendCommand(
ContainerProtos.ContainerCommandRequestProto request) ContainerProtos.ContainerCommandRequestProto request)
throws IOException { throws IOException {
try { try {
if ((channelFuture == null) || (!channelFuture.channel().isActive())) { if ((channelFuture == null) || (!channelFuture.channel().isActive())) {
throw new IOException("This channel is not connected."); throw new IOException("This channel is not connected.");
} }
XceiverClientHandler handler = XceiverClientHandler handler =
channelFuture.channel().pipeline().get(XceiverClientHandler.class); channelFuture.channel().pipeline().get(XceiverClientHandler.class);
return handler.sendCommand(request); return handler.sendCommand(request);
} catch (ExecutionException | InterruptedException e) { } catch (ExecutionException | InterruptedException e) {
@ -144,13 +144,13 @@ public ContainerProtos.ContainerCommandResponseProto sendCommand(
*/ */
@Override @Override
public CompletableFuture<ContainerProtos.ContainerCommandResponseProto> public CompletableFuture<ContainerProtos.ContainerCommandResponseProto>
sendCommandAsync(ContainerProtos.ContainerCommandRequestProto request) sendCommandAsync(ContainerProtos.ContainerCommandRequestProto request)
throws IOException, ExecutionException, InterruptedException { throws IOException, ExecutionException, InterruptedException {
if ((channelFuture == null) || (!channelFuture.channel().isActive())) { if ((channelFuture == null) || (!channelFuture.channel().isActive())) {
throw new IOException("This channel is not connected."); throw new IOException("This channel is not connected.");
} }
XceiverClientHandler handler = XceiverClientHandler handler =
channelFuture.channel().pipeline().get(XceiverClientHandler.class); channelFuture.channel().pipeline().get(XceiverClientHandler.class);
return handler.sendCommandAsync(request); return handler.sendCommandAsync(request);
} }
} }

View File

@ -109,8 +109,8 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
* @return -- response * @return -- response
*/ */
public ContainerCommandResponseProto public ContainerCommandResponseProto sendCommand(
sendCommand(ContainerProtos.ContainerCommandRequestProto request) ContainerProtos.ContainerCommandRequestProto request)
throws ExecutionException, InterruptedException { throws ExecutionException, InterruptedException {
Future<ContainerCommandResponseProto> future = sendCommandAsync(request); Future<ContainerCommandResponseProto> future = sendCommandAsync(request);
return future.get(); return future.get();
@ -123,11 +123,11 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
* @param request - Request to execute * @param request - Request to execute
* @return CompletableFuture * @return CompletableFuture
*/ */
public CompletableFuture<ContainerCommandResponseProto> public CompletableFuture<ContainerCommandResponseProto> sendCommandAsync(
sendCommandAsync(ContainerProtos.ContainerCommandRequestProto request) { ContainerProtos.ContainerCommandRequestProto request) {
// Throw an exception of request doesn't have traceId // Throw an exception of request doesn't have traceId
if(StringUtils.isEmpty(request.getTraceID())) { if (StringUtils.isEmpty(request.getTraceID())) {
throw new IllegalArgumentException("Invalid trace ID"); throw new IllegalArgumentException("Invalid trace ID");
} }

View File

@ -20,7 +20,6 @@
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos;
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.ContainerCommandRequestProto; import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.ContainerCommandRequestProto;
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.ContainerCommandResponseProto; import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.ContainerCommandResponseProto;
import org.apache.ratis.RatisHelper; import org.apache.ratis.RatisHelper;
@ -115,8 +114,8 @@ public ContainerCommandResponseProto sendCommand(
* @throws IOException * @throws IOException
*/ */
@Override @Override
public CompletableFuture<ContainerCommandResponseProto> public CompletableFuture<ContainerCommandResponseProto> sendCommandAsync(
sendCommandAsync(ContainerCommandRequestProto request) ContainerCommandRequestProto request)
throws IOException, ExecutionException, InterruptedException { throws IOException, ExecutionException, InterruptedException {
throw new IOException("Not implemented"); throw new IOException("Not implemented");
} }

View File

@ -103,8 +103,8 @@ public abstract ContainerCommandResponseProto sendCommand(
* @return Response to the command * @return Response to the command
* @throws IOException * @throws IOException
*/ */
public abstract CompletableFuture<ContainerCommandResponseProto> public abstract CompletableFuture<ContainerCommandResponseProto> sendCommandAsync(
sendCommandAsync(ContainerCommandRequestProto request) throws IOException, ContainerCommandRequestProto request)
ExecutionException, InterruptedException; throws IOException, ExecutionException, InterruptedException;
} }

View File

@ -82,13 +82,14 @@ public static ContainerInfo fromProtobuf(
return builder.build(); return builder.build();
} }
/** Builder class for ContainerInfo. */
public static class Builder { public static class Builder {
private OzoneProtos.LifeCycleState state; private OzoneProtos.LifeCycleState state;
private Pipeline pipeline; private Pipeline pipeline;
private long stateEnterTime; private long stateEnterTime;
public Builder setState(OzoneProtos.LifeCycleState state) { public Builder setState(OzoneProtos.LifeCycleState lifeCycleState) {
this.state = state; this.state = lifeCycleState;
return this; return this;
} }

View File

@ -188,40 +188,33 @@ public static void writeChunk(XceiverClientSpi xceiverClient, ChunkInfo chunk,
* @param traceID - Trace ID for logging purpose. * @param traceID - Trace ID for logging purpose.
* @throws IOException * @throws IOException
*/ */
public static void writeSmallFile(XceiverClientSpi client, String containerName, public static void writeSmallFile(XceiverClientSpi client,
String key, byte[] data, String traceID) throws IOException { String containerName, String key, byte[] data, String traceID)
throws IOException {
KeyData containerKeyData = KeyData KeyData containerKeyData =
.newBuilder() KeyData.newBuilder().setContainerName(containerName).setName(key)
.setContainerName(containerName) .build();
.setName(key).build(); PutKeyRequestProto.Builder createKeyRequest =
PutKeyRequestProto.Builder createKeyRequest = PutKeyRequestProto PutKeyRequestProto.newBuilder()
.newBuilder() .setPipeline(client.getPipeline().getProtobufMessage())
.setPipeline(client.getPipeline().getProtobufMessage()) .setKeyData(containerKeyData);
.setKeyData(containerKeyData);
KeyValue keyValue = KeyValue.newBuilder() KeyValue keyValue =
.setKey("OverWriteRequested").setValue("true").build(); KeyValue.newBuilder().setKey("OverWriteRequested").setValue("true")
ChunkInfo chunk = ChunkInfo .build();
.newBuilder() ChunkInfo chunk =
.setChunkName(key + "_chunk") ChunkInfo.newBuilder().setChunkName(key + "_chunk").setOffset(0)
.setOffset(0) .setLen(data.length).addMetadata(keyValue).build();
.setLen(data.length)
.addMetadata(keyValue)
.build();
PutSmallFileRequestProto putSmallFileRequest = PutSmallFileRequestProto PutSmallFileRequestProto putSmallFileRequest =
.newBuilder().setChunkInfo(chunk) PutSmallFileRequestProto.newBuilder().setChunkInfo(chunk)
.setKey(createKeyRequest) .setKey(createKeyRequest).setData(ByteString.copyFrom(data))
.setData(ByteString.copyFrom(data)) .build();
.build();
ContainerCommandRequestProto request = ContainerCommandRequestProto ContainerCommandRequestProto request =
.newBuilder() ContainerCommandRequestProto.newBuilder().setCmdType(Type.PutSmallFile)
.setCmdType(Type.PutSmallFile) .setTraceID(traceID).setPutSmallFile(putSmallFileRequest).build();
.setTraceID(traceID)
.setPutSmallFile(putSmallFileRequest)
.build();
ContainerCommandResponseProto response = client.sendCommand(request); ContainerCommandResponseProto response = client.sendCommand(request);
validateContainerResponse(response); validateContainerResponse(response);
} }

View File

@ -26,10 +26,13 @@
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail; import static org.junit.Assert.fail;
/**
* This class is to test acl stoarge and retreival in ozone store.
*/
public class TestOzoneAcls { public class TestOzoneAcls {
@Test @Test
public void TestACLParse() { public void testAclParse() {
HashMap<String, Boolean> testMatrix; HashMap<String, Boolean> testMatrix;
testMatrix = new HashMap<>(); testMatrix = new HashMap<>();
@ -103,7 +106,7 @@ public void TestACLParse() {
} }
@Test @Test
public void TestACLValues() { public void testAclValues() {
OzoneAcl acl = OzoneAcl.parseAcl("user:bilbo:rw"); OzoneAcl acl = OzoneAcl.parseAcl("user:bilbo:rw");
assertEquals(acl.getName(), "bilbo"); assertEquals(acl.getName(), "bilbo");
assertEquals(acl.getRights(), OzoneAcl.OzoneACLRights.READ_WRITE); assertEquals(acl.getRights(), OzoneAcl.OzoneACLRights.READ_WRITE);

View File

@ -46,6 +46,7 @@ public class TestArchive {
private long checksumWrite = 0L; private long checksumWrite = 0L;
private long checksumRead = 0L; private long checksumRead = 0L;
private long tmp = 0L; private long tmp = 0L;
private Checksum crc = new Adler32();
@Rule @Rule
public TemporaryFolder folder = new TemporaryFolder(); public TemporaryFolder folder = new TemporaryFolder();
@ -53,7 +54,6 @@ public class TestArchive {
@Rule @Rule
public TemporaryFolder outputFolder = new TemporaryFolder(); public TemporaryFolder outputFolder = new TemporaryFolder();
Checksum crc = new Adler32();
@Before @Before
public void setUp() throws Exception { public void setUp() throws Exception {

View File

@ -23,9 +23,7 @@
import org.apache.hadoop.scm.container.common.helpers.StorageContainerException; import org.apache.hadoop.scm.container.common.helpers.StorageContainerException;
import org.apache.hadoop.ozone.container.common.utils.ContainerCache; import org.apache.hadoop.ozone.container.common.utils.ContainerCache;
import org.apache.hadoop.utils.MetadataStore; import org.apache.hadoop.utils.MetadataStore;
import org.apache.hadoop.utils.MetadataStoreBuilder;
import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.nio.charset.Charset; import java.nio.charset.Charset;
@ -142,8 +140,8 @@ public static KeyData getKeyData(byte[] bytes) throws IOException {
KeyData data = KeyData.getFromProtoBuf(kd); KeyData data = KeyData.getFromProtoBuf(kd);
return data; return data;
} catch (IOException e) { } catch (IOException e) {
throw new StorageContainerException("Failed to parse key data from the bytes array.", throw new StorageContainerException("Failed to parse key data from the" +
NO_SUCH_KEY); " bytes array.", NO_SUCH_KEY);
} }
} }
} }

View File

@ -0,0 +1,18 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.ozone.container.common.utils;

View File

@ -290,7 +290,8 @@ private List <String> filterContainers(OzoneProtos.LifeCycleState state,
private BlockContainerInfo getContainer(OzoneProtos.LifeCycleState state, private BlockContainerInfo getContainer(OzoneProtos.LifeCycleState state,
String name) { String name) {
Map<String, BlockContainerInfo> containersByState = this.containers.get(state); Map<String, BlockContainerInfo> containersByState =
this.containers.get(state);
return containersByState.get(name); return containersByState.get(name);
} }
@ -318,15 +319,15 @@ private void updateContainer(OzoneProtos.LifeCycleState oldState, String name,
private void refreshContainers() { private void refreshContainers() {
Map<String, BlockContainerInfo> containersByState = Map<String, BlockContainerInfo> containersByState =
this.containers.get(OzoneProtos.LifeCycleState.CREATING); this.containers.get(OzoneProtos.LifeCycleState.CREATING);
for (String containerName: containersByState.keySet()) { for (String containerName : containersByState.keySet()) {
try { try {
ContainerInfo containerInfo = ContainerInfo containerInfo =
containerManager.getContainer(containerName); containerManager.getContainer(containerName);
if (containerInfo == null) { if (containerInfo == null) {
// TODO: clean up containers that has been deleted on SCM but // TODO: clean up containers that has been deleted on SCM but
// TODO: still in ALLOCATED state in block manager. // TODO: still in ALLOCATED state in block manager.
LOG.debug("Container {} allocated by block service" + LOG.debug("Container {} allocated by block service"
"can't be found in SCM", containerName); + "can't be found in SCM", containerName);
continue; continue;
} }
if (containerInfo.getState() == OzoneProtos.LifeCycleState.OPEN) { if (containerInfo.getState() == OzoneProtos.LifeCycleState.OPEN) {
@ -340,7 +341,7 @@ private void refreshContainers() {
LOG.debug("Failed to get container info for: {}", containerName); LOG.debug("Failed to get container info for: {}", containerName);
} }
} }
} }
/** /**
* Allocates a new block for a given size. * Allocates a new block for a given size.
@ -356,8 +357,7 @@ private void refreshContainers() {
public AllocatedBlock allocateBlock(final long size) throws IOException { public AllocatedBlock allocateBlock(final long size) throws IOException {
boolean createContainer = false; boolean createContainer = false;
if (size < 0 || size > containerSize) { if (size < 0 || size > containerSize) {
throw new SCMException("Unsupported block size", throw new SCMException("Unsupported block size", INVALID_BLOCK_SIZE);
INVALID_BLOCK_SIZE);
} }
if (!nodeManager.isOutOfNodeChillMode()) { if (!nodeManager.isOutOfNodeChillMode()) {
throw new SCMException("Unable to create block while in chill mode", throw new SCMException("Unable to create block while in chill mode",
@ -370,8 +370,8 @@ public AllocatedBlock allocateBlock(final long size) throws IOException {
List<String> candidates; List<String> candidates;
candidates = filterContainers(OzoneProtos.LifeCycleState.OPEN, size); candidates = filterContainers(OzoneProtos.LifeCycleState.OPEN, size);
if (candidates.size() == 0) { if (candidates.size() == 0) {
candidates = filterContainers(OzoneProtos.LifeCycleState.ALLOCATED, candidates =
size); filterContainers(OzoneProtos.LifeCycleState.ALLOCATED, size);
if (candidates.size() == 0) { if (candidates.size() == 0) {
try { try {
candidates = allocateContainers(containerProvisionBatchSize); candidates = allocateContainers(containerProvisionBatchSize);
@ -383,8 +383,9 @@ public AllocatedBlock allocateBlock(final long size) throws IOException {
} }
// now we should have some candidates in ALLOCATE state // now we should have some candidates in ALLOCATE state
if (candidates.size() == 0) { if (candidates.size() == 0) {
throw new SCMException("Fail to find any container to allocate block " throw new SCMException(
+ "of size " + size + ".", FAILED_TO_FIND_CONTAINER_WITH_SPACE); "Fail to find any container to allocate block " + "of size "
+ size + ".", FAILED_TO_FIND_CONTAINER_WITH_SPACE);
} }
} }
@ -416,8 +417,8 @@ public AllocatedBlock allocateBlock(final long size) throws IOException {
// TODO: make block key easier to debug (e.g., seq no) // TODO: make block key easier to debug (e.g., seq no)
// Allocate key for the block // Allocate key for the block
String blockKey = UUID.randomUUID().toString(); String blockKey = UUID.randomUUID().toString();
AllocatedBlock.Builder abb = new AllocatedBlock.Builder() AllocatedBlock.Builder abb = new AllocatedBlock.Builder().setKey(blockKey)
.setKey(blockKey).setPipeline(containerInfo.getPipeline()) .setPipeline(containerInfo.getPipeline())
.setShouldCreateContainer(createContainer); .setShouldCreateContainer(createContainer);
if (containerInfo.getPipeline().getMachines().size() > 0) { if (containerInfo.getPipeline().getMachines().size() > 0) {
blockStore.put(DFSUtil.string2Bytes(blockKey), blockStore.put(DFSUtil.string2Bytes(blockKey),
@ -428,12 +429,12 @@ public AllocatedBlock allocateBlock(final long size) throws IOException {
getContainer(containerInfo.getState(), containerName); getContainer(containerInfo.getState(), containerName);
Preconditions.checkNotNull(containerInfoUpdate); Preconditions.checkNotNull(containerInfoUpdate);
containerInfoUpdate.addAllocated(size); containerInfoUpdate.addAllocated(size);
containerStore.put(DFSUtil.string2Bytes(containerName), containerStore.put(DFSUtil.string2Bytes(containerName), DFSUtil
DFSUtil.string2Bytes(Long.toString(containerInfoUpdate.getAllocated()))); .string2Bytes(Long.toString(containerInfoUpdate.getAllocated())));
if (createContainer) { if (createContainer) {
OzoneProtos.LifeCycleState newState = OzoneProtos.LifeCycleState newState = containerManager
containerManager.updateContainerState(containerName, .updateContainerState(containerName,
OzoneProtos.LifeCycleEvent.BEGIN_CREATE); OzoneProtos.LifeCycleEvent.BEGIN_CREATE);
updateContainer(containerInfo.getState(), containerName, newState); updateContainer(containerInfo.getState(), containerName, newState);
} }
return abb.build(); return abb.build();
@ -457,17 +458,19 @@ public Pipeline getBlock(final String key) throws IOException {
try { try {
byte[] containerBytes = blockStore.get(DFSUtil.string2Bytes(key)); byte[] containerBytes = blockStore.get(DFSUtil.string2Bytes(key));
if (containerBytes == null) { if (containerBytes == null) {
throw new SCMException("Specified block key does not exist. key : " + throw new SCMException(
key, FAILED_TO_FIND_BLOCK); "Specified block key does not exist. key : " + key,
FAILED_TO_FIND_BLOCK);
} }
String containerName = DFSUtil.bytes2String(containerBytes); String containerName = DFSUtil.bytes2String(containerBytes);
ContainerInfo containerInfo = containerManager.getContainer( ContainerInfo containerInfo =
containerName); containerManager.getContainer(containerName);
if (containerInfo == null) { if (containerInfo == null) {
LOG.debug("Container {} allocated by block service" + LOG.debug(
"can't be found in SCM", containerName); "Container {} allocated by block service" + "can't be found in SCM",
throw new SCMException("Unable to find container for the block", containerName);
SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER); throw new SCMException("Unable to find container for the block",
SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
} }
return containerInfo.getPipeline(); return containerInfo.getPipeline();
} finally { } finally {

View File

@ -158,15 +158,17 @@ private void initializeStateMachine() {
* {@inheritDoc} * {@inheritDoc}
*/ */
@Override @Override
public ContainerInfo getContainer(final String containerName) throws IOException { public ContainerInfo getContainer(final String containerName)
throws IOException {
ContainerInfo containerInfo; ContainerInfo containerInfo;
lock.lock(); lock.lock();
try { try {
byte[] containerBytes = byte[] containerBytes =
containerStore.get(containerName.getBytes(encoding)); containerStore.get(containerName.getBytes(encoding));
if (containerBytes == null) { if (containerBytes == null) {
throw new SCMException("Specified key does not exist. key : " + throw new SCMException(
containerName, SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER); "Specified key does not exist. key : " + containerName,
SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
} }
containerInfo = ContainerInfo.fromProtobuf( containerInfo = ContainerInfo.fromProtobuf(
OzoneProtos.SCMContainerInfo.PARSER.parseFrom(containerBytes)); OzoneProtos.SCMContainerInfo.PARSER.parseFrom(containerBytes));
@ -305,11 +307,11 @@ public OzoneProtos.LifeCycleState updateContainerState(String containerName,
lock.lock(); lock.lock();
try { try {
byte[] dbKey = containerName.getBytes(encoding); byte[] dbKey = containerName.getBytes(encoding);
byte[] containerBytes = byte[] containerBytes = containerStore.get(dbKey);
containerStore.get(dbKey); if (containerBytes == null) {
if(containerBytes == null) { throw new SCMException(
throw new SCMException("Failed to update container state" "Failed to update container state" + containerName
+ containerName + ", reason : container doesn't exist.", + ", reason : container doesn't exist.",
SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER); SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
} }
containerInfo = ContainerInfo.fromProtobuf( containerInfo = ContainerInfo.fromProtobuf(
@ -317,11 +319,12 @@ public OzoneProtos.LifeCycleState updateContainerState(String containerName,
OzoneProtos.LifeCycleState newState; OzoneProtos.LifeCycleState newState;
try { try {
newState = stateMachine.getNextState(containerInfo.getState(), event); newState = stateMachine.getNextState(containerInfo.getState(), event);
} catch (InvalidStateTransitionException ex) { } catch (InvalidStateTransitionException ex) {
throw new SCMException("Failed to update container state" throw new SCMException(
+ containerName + ", reason : invalid state transition from state: " "Failed to update container state" + containerName
+ containerInfo.getState() + " upon event: " + event + ".", + ", reason : invalid state transition from state: "
+ containerInfo.getState() + " upon event: " + event + ".",
SCMException.ResultCodes.FAILED_TO_CHANGE_CONTAINER_STATE); SCMException.ResultCodes.FAILED_TO_CHANGE_CONTAINER_STATE);
} }
containerInfo.setState(newState); containerInfo.setState(newState);

View File

@ -0,0 +1,22 @@
/**
* 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.web.exceptions;
/**
This package contains ozone client side libraries.
*/

View File

@ -0,0 +1,22 @@
/**
* 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.web.handlers;
/**
This package contains ozone client side libraries.
*/

View File

@ -62,14 +62,9 @@ public interface Keys {
@PUT @PUT
@Consumes(MediaType.WILDCARD) @Consumes(MediaType.WILDCARD)
Response putKey(@PathParam("volume") String volume, Response putKey(@PathParam("volume") String volume,
@PathParam("bucket") String bucket, @PathParam("bucket") String bucket, @PathParam("keys") String keys,
@PathParam("keys") String keys, InputStream is, @Context Request req, @Context UriInfo info,
InputStream is, @Context HttpHeaders headers) throws OzoneException;
@Context Request req,
@Context UriInfo info,
@Context HttpHeaders headers)
throws OzoneException;
/** /**
* Gets the Key if it exists. * Gets the Key if it exists.
@ -88,14 +83,10 @@ Response putKey(@PathParam("volume") String volume,
*/ */
@GET @GET
Response getKey(@PathParam("volume") String volume, Response getKey(@PathParam("volume") String volume,
@PathParam("bucket") String bucket, @PathParam("bucket") String bucket, @PathParam("keys") String keys,
@PathParam("keys") String keys, @QueryParam(Header.OZONE_LIST_QUERY_TAG) String info,
@QueryParam(Header.OZONE_LIST_QUERY_TAG) @Context Request req, @Context UriInfo uriInfo,
String info, @Context HttpHeaders headers) throws OzoneException;
@Context Request req,
@Context UriInfo uriInfo,
@Context HttpHeaders headers)
throws OzoneException;
/** /**
* Deletes an existing key. * Deletes an existing key.
@ -112,11 +103,8 @@ Response getKey(@PathParam("volume") String volume,
*/ */
@DELETE @DELETE
Response deleteKey(@PathParam("volume") String volume, Response deleteKey(@PathParam("volume") String volume,
@PathParam("bucket") String bucket, @PathParam("bucket") String bucket, @PathParam("keys") String keys,
@PathParam("keys") String keys, @Context Request req, @Context UriInfo info, @Context HttpHeaders headers)
@Context Request req, throws OzoneException;
@Context UriInfo info,
@Context HttpHeaders headers)
throws OzoneException;
} }

View File

@ -65,13 +65,10 @@ public interface Volume {
@POST @POST
Response createVolume(@PathParam("volume") String volume, Response createVolume(@PathParam("volume") String volume,
@DefaultValue(Header.OZONE_QUOTA_UNDEFINED) @DefaultValue(Header.OZONE_QUOTA_UNDEFINED)
@QueryParam(Header.OZONE_QUOTA_QUERY_TAG) String quota, @QueryParam(Header.OZONE_QUOTA_QUERY_TAG) String quota,
@Context Request req, @Context Request req, @Context UriInfo uriInfo,
@Context UriInfo uriInfo, @Context HttpHeaders headers) throws OzoneException;
@Context HttpHeaders headers)
throws OzoneException;
/** /**
* Updates a Volume owned by the user. * Updates a Volume owned by the user.
@ -93,13 +90,10 @@ Response createVolume(@PathParam("volume") String volume,
*/ */
@PUT @PUT
Response updateVolume(@PathParam("volume") String volume, Response updateVolume(@PathParam("volume") String volume,
@DefaultValue(Header.OZONE_QUOTA_UNDEFINED) @DefaultValue(Header.OZONE_QUOTA_UNDEFINED)
@QueryParam(Header.OZONE_QUOTA_QUERY_TAG) String quota, @QueryParam(Header.OZONE_QUOTA_QUERY_TAG) String quota,
@Context Request req, @Context Request req, @Context UriInfo uriInfo,
@Context UriInfo uriInfo, @Context HttpHeaders headers) throws OzoneException;
@Context HttpHeaders headers)
throws OzoneException;
/** /**
* Deletes a Volume if it is empty. * Deletes a Volume if it is empty.
@ -112,10 +106,8 @@ Response updateVolume(@PathParam("volume") String volume,
*/ */
@DELETE @DELETE
Response deleteVolume(@PathParam("volume") String volume, Response deleteVolume(@PathParam("volume") String volume,
@Context Request req, @Context Request req, @Context UriInfo uriInfo,
@Context UriInfo uriInfo, @Context HttpHeaders headers) throws OzoneException;
@Context HttpHeaders headers)
throws OzoneException;
/** /**
* Returns Volume info. This API can be invoked either * Returns Volume info. This API can be invoked either
@ -131,21 +123,14 @@ Response deleteVolume(@PathParam("volume") String volume,
*/ */
@GET @GET
Response getVolumeInfo(@PathParam("volume") String volume, Response getVolumeInfo(@PathParam("volume") String volume,
@DefaultValue(Header.OZONE_LIST_QUERY_BUCKET) @DefaultValue(Header.OZONE_LIST_QUERY_BUCKET)
@QueryParam(Header.OZONE_LIST_QUERY_TAG) @QueryParam(Header.OZONE_LIST_QUERY_TAG) String info,
String info, @QueryParam(Header.OZONE_LIST_QUERY_PREFIX) String prefix,
@QueryParam(Header.OZONE_LIST_QUERY_PREFIX) @DefaultValue(Header.OZONE_DEFAULT_LIST_SIZE)
String prefix, @QueryParam(Header.OZONE_LIST_QUERY_MAXKEYS) int keys,
@DefaultValue(Header.OZONE_DEFAULT_LIST_SIZE) @QueryParam(Header.OZONE_LIST_QUERY_PREVKEY) String prevKey,
@QueryParam(Header.OZONE_LIST_QUERY_MAXKEYS) @QueryParam(Header.OZONE_LIST_QUERY_ROOTSCAN) boolean rootScan,
int keys, @Context Request req, @Context UriInfo uriInfo,
@QueryParam(Header.OZONE_LIST_QUERY_PREVKEY) @Context HttpHeaders headers) throws OzoneException;
String prevKey,
@QueryParam(Header.OZONE_LIST_QUERY_ROOTSCAN)
boolean rootScan,
@Context Request req,
@Context UriInfo uriInfo,
@Context HttpHeaders headers)
throws OzoneException;
} }

View File

@ -0,0 +1,22 @@
/**
* 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.web.interfaces;
/**
This package contains ozone client side libraries.
*/

View File

@ -0,0 +1,18 @@
/**
* 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.web.localstorage;

View File

@ -0,0 +1,18 @@
/**
* 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.web.messages;

View File

@ -35,7 +35,7 @@ final class CloseableCleanupListener implements ChannelFutureListener {
* *
* @param closeables any number of closeable resources * @param closeables any number of closeable resources
*/ */
public CloseableCleanupListener(Closeable... closeables) { CloseableCleanupListener(Closeable... closeables) {
this.closeables = closeables; this.closeables = closeables;
} }

View File

@ -213,7 +213,7 @@ private final class RequestRunner implements Runnable,
* @param respOut output stream for writing response body * @param respOut output stream for writing response body
* @param latch for coordinating asynchronous return of HTTP response * @param latch for coordinating asynchronous return of HTTP response
*/ */
public RequestRunner(HttpRequest nettyReq, InputStream reqIn, RequestRunner(HttpRequest nettyReq, InputStream reqIn,
OutputStream respOut, CountDownLatch latch) { OutputStream respOut, CountDownLatch latch) {
this.latch = latch; this.latch = latch;
this.nettyReq = nettyReq; this.nettyReq = nettyReq;

View File

@ -0,0 +1,23 @@
/*
* 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.
*/
/**
* Netty-based HTTP server implementation for Ozone.
*/
package org.apache.hadoop.ozone.web.ozShell.bucket;

View File

@ -20,9 +20,7 @@
import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.CommandLine;
import org.apache.commons.codec.digest.DigestUtils; import org.apache.commons.codec.digest.DigestUtils;
import org.apache.hadoop.ozone.web.client.OzoneBucket;
import org.apache.hadoop.ozone.web.client.OzoneRestClientException; import org.apache.hadoop.ozone.web.client.OzoneRestClientException;
import org.apache.hadoop.ozone.web.client.OzoneVolume;
import org.apache.hadoop.ozone.web.exceptions.OzoneException; import org.apache.hadoop.ozone.web.exceptions.OzoneException;
import org.apache.hadoop.ozone.web.ozShell.Handler; import org.apache.hadoop.ozone.web.ozShell.Handler;
import org.apache.hadoop.ozone.web.ozShell.Shell; import org.apache.hadoop.ozone.web.ozShell.Shell;

View File

@ -20,9 +20,7 @@
import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.CommandLine;
import org.apache.commons.codec.digest.DigestUtils; import org.apache.commons.codec.digest.DigestUtils;
import org.apache.hadoop.ozone.web.client.OzoneBucket;
import org.apache.hadoop.ozone.web.client.OzoneRestClientException; import org.apache.hadoop.ozone.web.client.OzoneRestClientException;
import org.apache.hadoop.ozone.web.client.OzoneVolume;
import org.apache.hadoop.ozone.web.exceptions.OzoneException; import org.apache.hadoop.ozone.web.exceptions.OzoneException;
import org.apache.hadoop.ozone.web.ozShell.Handler; import org.apache.hadoop.ozone.web.ozShell.Handler;
import org.apache.hadoop.ozone.web.ozShell.Shell; import org.apache.hadoop.ozone.web.ozShell.Shell;

View File

@ -0,0 +1,23 @@
/*
* 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.
*/
/**
* Netty-based HTTP server implementation for Ozone.
*/
package org.apache.hadoop.ozone.web.ozShell.keys;

View File

@ -0,0 +1,23 @@
/*
* 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.
*/
/**
* Netty-based HTTP server implementation for Ozone.
*/
package org.apache.hadoop.ozone.web.ozShell.volume;

View File

@ -0,0 +1,23 @@
/*
* 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.
*/
/**
* Netty-based HTTP server implementation for Ozone.
*/
package org.apache.hadoop.ozone.web.request;

View File

@ -0,0 +1,23 @@
/*
* 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.
*/
/**
* Netty-based HTTP server implementation for Ozone.
*/
package org.apache.hadoop.ozone.web.response;

View File

@ -0,0 +1,23 @@
/*
* 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.
*/
/**
* Netty-based HTTP server implementation for Ozone.
*/
package org.apache.hadoop.ozone.web.userauth;

View File

@ -442,13 +442,13 @@ public void testPipelineRecoveryStress() throws Exception {
}; };
for (String[] scmd: scmds) { for (String[] scmd: scmds) {
String scmd_str = StringUtils.join(" ", scmd); String scmdStr = StringUtils.join(" ", scmd);
try { try {
ShellCommandExecutor sce = new ShellCommandExecutor(scmd); ShellCommandExecutor sce = new ShellCommandExecutor(scmd);
sce.execute(); sce.execute();
LOG.info("'" + scmd_str + "' output:\n" + sce.getOutput()); LOG.info("'" + scmdStr + "' output:\n" + sce.getOutput());
} catch (IOException e) { } catch (IOException e) {
LOG.warn("Error when running '" + scmd_str + "'", e); LOG.warn("Error when running '" + scmdStr + "'", e);
} }
} }

View File

@ -306,28 +306,27 @@ private void heartbeatTaskHelper(InetSocketAddress scmAddress,
// Create a datanode state machine for stateConext used by endpoint task // Create a datanode state machine for stateConext used by endpoint task
try (DatanodeStateMachine stateMachine = try (DatanodeStateMachine stateMachine = new DatanodeStateMachine(
new DatanodeStateMachine(DFSTestUtil.getLocalDatanodeID(), conf); DFSTestUtil.getLocalDatanodeID(), conf);
EndpointStateMachine rpcEndPoint = SCMTestUtils.createEndpoint(conf, EndpointStateMachine rpcEndPoint = SCMTestUtils
scmAddress, rpcTimeout)) { .createEndpoint(conf, scmAddress, rpcTimeout)) {
ContainerNodeIDProto containerNodeID = ContainerNodeIDProto.newBuilder() ContainerNodeIDProto containerNodeID = ContainerNodeIDProto.newBuilder()
.setClusterID(UUID.randomUUID().toString()) .setClusterID(UUID.randomUUID().toString())
.setDatanodeID(getDatanodeID().getProtoBufMessage()) .setDatanodeID(getDatanodeID().getProtoBufMessage()).build();
.build(); rpcEndPoint.setState(EndpointStateMachine.EndPointStates.HEARTBEAT);
rpcEndPoint.setState(EndpointStateMachine.EndPointStates.HEARTBEAT);
final StateContext stateContext = new StateContext(conf, final StateContext stateContext =
DatanodeStateMachine.DatanodeStates.RUNNING, new StateContext(conf, DatanodeStateMachine.DatanodeStates.RUNNING,
stateMachine); stateMachine);
HeartbeatEndpointTask endpointTask = HeartbeatEndpointTask endpointTask =
new HeartbeatEndpointTask(rpcEndPoint, conf, stateContext); new HeartbeatEndpointTask(rpcEndPoint, conf, stateContext);
endpointTask.setContainerNodeIDProto(containerNodeID); endpointTask.setContainerNodeIDProto(containerNodeID);
endpointTask.call(); endpointTask.call();
Assert.assertNotNull(endpointTask.getContainerNodeIDProto()); Assert.assertNotNull(endpointTask.getContainerNodeIDProto());
Assert.assertEquals(EndpointStateMachine.EndPointStates.HEARTBEAT, Assert.assertEquals(EndpointStateMachine.EndPointStates.HEARTBEAT,
rpcEndPoint.getState()); rpcEndPoint.getState());
} }
} }
@ -360,8 +359,8 @@ public void testHeartbeatTaskRpcTimeOut() throws Exception {
* @return * @return
*/ */
ContainerReport getRandomContainerReport() { ContainerReport getRandomContainerReport() {
return new ContainerReport(UUID.randomUUID().toString() return new ContainerReport(UUID.randomUUID().toString(),
,DigestUtils.sha256Hex("Random")); DigestUtils.sha256Hex("Random"));
} }
/** /**

View File

@ -90,7 +90,7 @@ public class TestContainerPersistence {
@Rule @Rule
public Timeout testTimeout = new Timeout(300000); public Timeout testTimeout = new Timeout(300000);
private static Logger LOG = private static Logger log =
LoggerFactory.getLogger(TestContainerPersistence.class); LoggerFactory.getLogger(TestContainerPersistence.class);
private static String path; private static String path;
private static ContainerManagerImpl containerManager; private static ContainerManagerImpl containerManager;
@ -149,7 +149,7 @@ public void setupPaths() throws IOException {
@After @After
public void cleanupDir() throws IOException { public void cleanupDir() throws IOException {
// Clean up SCM metadata // Clean up SCM metadata
LOG.info("Deletting {}", path); log.info("Deletting {}", path);
FileUtils.deleteDirectory(new File(path)); FileUtils.deleteDirectory(new File(path));
// Clean up SCM datanode container metadata/data // Clean up SCM datanode container metadata/data

View File

@ -29,7 +29,6 @@
import org.apache.hadoop.ozone.web.interfaces.StorageHandler; import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
import org.apache.hadoop.ozone.web.utils.OzoneUtils; import org.apache.hadoop.ozone.web.utils.OzoneUtils;
import org.junit.Before; import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
import org.junit.runner.RunWith; import org.junit.runner.RunWith;
import org.junit.runners.Parameterized; import org.junit.runners.Parameterized;

View File

@ -523,14 +523,14 @@ public void testKeyOverwrite() throws IOException, OzoneException {
stream.write(dataString.getBytes()); stream.write(dataString.getBytes());
} }
// We allow the key overwrite to be successful. Please note : Till HDFS-11922 // We allow the key overwrite to be successful. Please note : Till
// is fixed this causes a data block leak on the data node side. That is // HDFS-11922 is fixed this causes a data block leak on the data node side.
// this overwrite only overwrites the keys on KSM. We need to garbage // That is this overwrite only overwrites the keys on KSM. We need to
// collect those blocks from datanode. // garbage collect those blocks from datanode.
KeyArgs keyArgs2 = new KeyArgs(volumeName, bucketName, keyName, userArgs); KeyArgs keyArgs2 = new KeyArgs(volumeName, bucketName, keyName, userArgs);
storageHandler.newKeyWriter(keyArgs2); storageHandler.newKeyWriter(keyArgs2);
Assert.assertEquals(numKeyAllocateFails, Assert
ksmMetrics.getNumKeyAllocateFails()); .assertEquals(numKeyAllocateFails, ksmMetrics.getNumKeyAllocateFails());
} }
/** /**

View File

@ -44,6 +44,10 @@
import javax.management.openmbean.CompositeData; import javax.management.openmbean.CompositeData;
import javax.management.openmbean.TabularData; import javax.management.openmbean.TabularData;
/**
*
* This class is to test JMX management interface for scm information.
*/
public class TestSCMMXBean { public class TestSCMMXBean {
public static final Log LOG = LogFactory.getLog(TestSCMMXBean.class); public static final Log LOG = LogFactory.getLog(TestSCMMXBean.class);

View File

@ -31,7 +31,7 @@
public class TestBucketInfo { public class TestBucketInfo {
@Test @Test
public void TestBucketInfoJson() throws IOException { public void testBucketInfoJson() throws IOException {
BucketInfo bucketInfo = new BucketInfo("volumeName", "bucketName"); BucketInfo bucketInfo = new BucketInfo("volumeName", "bucketName");
String bucketInfoString = bucketInfo.toJsonString(); String bucketInfoString = bucketInfo.toJsonString();
BucketInfo newBucketInfo = BucketInfo.parse(bucketInfoString); BucketInfo newBucketInfo = BucketInfo.parse(bucketInfoString);
@ -39,7 +39,7 @@ public void TestBucketInfoJson() throws IOException {
} }
@Test @Test
public void TestBucketInfoDBString() throws IOException { public void testBucketInfoDBString() throws IOException {
BucketInfo bucketInfo = new BucketInfo("volumeName", "bucketName"); BucketInfo bucketInfo = new BucketInfo("volumeName", "bucketName");
String bucketInfoString = bucketInfo.toDBString(); String bucketInfoString = bucketInfo.toDBString();
BucketInfo newBucketInfo = BucketInfo.parse(bucketInfoString); BucketInfo newBucketInfo = BucketInfo.parse(bucketInfoString);
@ -47,7 +47,7 @@ public void TestBucketInfoDBString() throws IOException {
} }
@Test @Test
public void TestBucketInfoAddAcls() throws IOException { public void testBucketInfoAddAcls() throws IOException {
BucketInfo bucketInfo = new BucketInfo("volumeName", "bucketName"); BucketInfo bucketInfo = new BucketInfo("volumeName", "bucketName");
String bucketInfoString = bucketInfo.toDBString(); String bucketInfoString = bucketInfo.toDBString();
BucketInfo newBucketInfo = BucketInfo.parse(bucketInfoString); BucketInfo newBucketInfo = BucketInfo.parse(bucketInfoString);
@ -64,7 +64,7 @@ public void TestBucketInfoAddAcls() throws IOException {
@Test @Test
public void TestBucketInfoVersionAndType() throws IOException { public void testBucketInfoVersionAndType() throws IOException {
BucketInfo bucketInfo = new BucketInfo("volumeName", "bucketName"); BucketInfo bucketInfo = new BucketInfo("volumeName", "bucketName");
bucketInfo.setVersioning(OzoneConsts.Versioning.ENABLED); bucketInfo.setVersioning(OzoneConsts.Versioning.ENABLED);
bucketInfo.setStorageType(StorageType.DISK); bucketInfo.setStorageType(StorageType.DISK);

View File

@ -30,7 +30,7 @@ public class TestErrorCode {
* Test Error Generator functions. * Test Error Generator functions.
*/ */
@Test @Test
public void TestErrorGen() { public void testErrorGen() {
OzoneException e = ErrorTable OzoneException e = ErrorTable
.newError(ErrorTable.ACCESS_DENIED, getRequestID(), "/test/path", .newError(ErrorTable.ACCESS_DENIED, getRequestID(), "/test/path",
"localhost"); "localhost");
@ -40,7 +40,7 @@ public void TestErrorGen() {
} }
@Test @Test
public void TestErrorGenWithException() { public void testErrorGenWithException() {
OzoneException e = OzoneException e =
new OzoneException(ErrorTable.ACCESS_DENIED.getHttpCode(), new OzoneException(ErrorTable.ACCESS_DENIED.getHttpCode(),
"short message", new Exception("Hello")); "short message", new Exception("Hello"));

View File

@ -48,7 +48,7 @@
public class TestOzoneWebAccess { public class TestOzoneWebAccess {
/** /**
* Set the timeout for every test * Set the timeout for every test.
*/ */
@Rule @Rule
public Timeout testTimeout = new Timeout(300000); public Timeout testTimeout = new Timeout(300000);

View File

@ -32,7 +32,7 @@
public class TestQuota { public class TestQuota {
@Test @Test
public void TestParseQuota() { public void testParseQuota() {
HashMap<String, Boolean> testMatrix; HashMap<String, Boolean> testMatrix;
testMatrix = new HashMap<String, Boolean>(); testMatrix = new HashMap<String, Boolean>();
@ -56,11 +56,10 @@ public void TestParseQuota() {
testMatrix.put("1024 bytes", Boolean.TRUE); testMatrix.put("1024 bytes", Boolean.TRUE);
testMatrix.put("1bytes", Boolean.TRUE); testMatrix.put("1bytes", Boolean.TRUE);
testMatrix.put("0bytes", Boolean.TRUE); testMatrix.put("0bytes", Boolean.TRUE);
testMatrix.put("10000 BYTES",Boolean.TRUE ); testMatrix.put("10000 BYTES", Boolean.TRUE);
testMatrix.put("BYTESbytes", Boolean.FALSE); testMatrix.put("BYTESbytes", Boolean.FALSE);
testMatrix.put("bytes", Boolean.FALSE); testMatrix.put("bytes", Boolean.FALSE);
Set<String> keys = testMatrix.keySet(); Set<String> keys = testMatrix.keySet();
for (String key : keys) { for (String key : keys) {
if (testMatrix.get(key)) { if (testMatrix.get(key)) {
@ -79,7 +78,7 @@ public void TestParseQuota() {
} }
@Test @Test
public void TestVerifyQuota() { public void testVerifyQuota() {
OzoneQuota qt = OzoneQuota.parseQuota("10TB"); OzoneQuota qt = OzoneQuota.parseQuota("10TB");
assertEquals(qt.getSize(), 10); assertEquals(qt.getSize(), 10);
assertEquals(qt.getUnit(), OzoneQuota.Units.TB); assertEquals(qt.getUnit(), OzoneQuota.Units.TB);
@ -107,7 +106,7 @@ public void TestVerifyQuota() {
} }
@Test @Test
public void TestVerifyRemove() { public void testVerifyRemove() {
assertTrue(OzoneQuota.isRemove("remove")); assertTrue(OzoneQuota.isRemove("remove"));
assertFalse(OzoneQuota.isRemove("not remove")); assertFalse(OzoneQuota.isRemove("not remove"));
assertFalse(OzoneQuota.isRemove(null)); assertFalse(OzoneQuota.isRemove(null));

View File

@ -35,7 +35,7 @@ public class TestUtils {
* Tests if the bucket name handling is correct. * Tests if the bucket name handling is correct.
*/ */
@Test @Test
public void TestValidBucketNames() { public void testValidBucketNames() {
HashMap<String, Boolean> testMatrix; HashMap<String, Boolean> testMatrix;
// Init the Table with Strings and Expected Return values // Init the Table with Strings and Expected Return values
testMatrix = new HashMap<String, Boolean>(); testMatrix = new HashMap<String, Boolean>();
@ -47,9 +47,8 @@ public void TestValidBucketNames() {
testMatrix.put("bucket..ozone.self", Boolean.FALSE); testMatrix.put("bucket..ozone.self", Boolean.FALSE);
testMatrix.put("192.1.1.1", Boolean.FALSE); testMatrix.put("192.1.1.1", Boolean.FALSE);
testMatrix.put("ab", Boolean.FALSE); testMatrix.put("ab", Boolean.FALSE);
testMatrix.put("bucket.ozone.self.this.is.a.really.long.name.that." + testMatrix.put("bucket.ozone.self.this.is.a.really.long.name.that."
"is.more.than.sixty.three.characters.long.for.sure", + "is.more.than.sixty.three.characters.long.for.sure", Boolean.FALSE);
Boolean.FALSE);
testMatrix.put(null, Boolean.FALSE); testMatrix.put(null, Boolean.FALSE);
testMatrix.put("bucket@$", Boolean.FALSE); testMatrix.put("bucket@$", Boolean.FALSE);
testMatrix.put("BUCKET", Boolean.FALSE); testMatrix.put("BUCKET", Boolean.FALSE);
@ -66,7 +65,7 @@ public void TestValidBucketNames() {
Set<String> keys = testMatrix.keySet(); Set<String> keys = testMatrix.keySet();
for (String key : keys) { for (String key : keys) {
if(testMatrix.get(key)) { if (testMatrix.get(key)) {
// For valid names there should be no exceptions at all // For valid names there should be no exceptions at all
verifyResourceName(key); verifyResourceName(key);
@ -74,8 +73,8 @@ public void TestValidBucketNames() {
try { try {
verifyResourceName(key); verifyResourceName(key);
// should never get here since the isValid call will throw // should never get here since the isValid call will throw
fail("An exception was expected but did not happen."); fail("An exception was expected but did not happen.");
} catch(IllegalArgumentException e){ } catch (IllegalArgumentException e) {
} }
} }
@ -90,11 +89,10 @@ public void TestValidBucketNames() {
* that this test is good enough. * that this test is good enough.
*/ */
@Test @Test
public void TestRequestIDisRandom(){ public void testRequestIDisRandom() {
HashSet<String> set = new HashSet<>(); HashSet<String> set = new HashSet<>();
for (int i = 0; i < 1000; i ++){ for (int i = 0; i < 1000; i++) {
assertTrue(set.add(getRequestID())); assertTrue(set.add(getRequestID()));
} }
} }
} }

View File

@ -32,8 +32,8 @@ public class TestVolumeStructs {
@Test @Test
public void testVolumeInfoParse() throws IOException { public void testVolumeInfoParse() throws IOException {
VolumeInfo volInfo = new VolumeInfo("testvol", VolumeInfo volInfo =
"Thu, Apr 9, 2015 10:23:45 GMT", "gandalf"); new VolumeInfo("testvol", "Thu, Apr 9, 2015 10:23:45 GMT", "gandalf");
VolumeOwner owner = new VolumeOwner("bilbo"); VolumeOwner owner = new VolumeOwner("bilbo");
volInfo.setOwner(owner); volInfo.setOwner(owner);
String jString = volInfo.toJsonString(); String jString = volInfo.toJsonString();
@ -48,8 +48,7 @@ public void testVolumeInfoParse() throws IOException {
public void testVolumeInfoValue() throws IOException { public void testVolumeInfoValue() throws IOException {
String createdOn = "Thu, Apr 9, 2015 10:23:45 GMT"; String createdOn = "Thu, Apr 9, 2015 10:23:45 GMT";
String createdBy = "gandalf"; String createdBy = "gandalf";
VolumeInfo volInfo = new VolumeInfo("testvol", VolumeInfo volInfo = new VolumeInfo("testvol", createdOn, createdBy);
createdOn, createdBy);
assertEquals(volInfo.getCreatedBy(), createdBy); assertEquals(volInfo.getCreatedBy(), createdBy);
assertEquals(volInfo.getCreatedOn(), createdOn); assertEquals(volInfo.getCreatedOn(), createdOn);
} }
@ -57,11 +56,10 @@ public void testVolumeInfoValue() throws IOException {
@Test @Test
public void testVolumeListParse() throws IOException { public void testVolumeListParse() throws IOException {
ListVolumes list = new ListVolumes(); ListVolumes list = new ListVolumes();
for (int x = 0; x < 100; x++) { for (int x = 0; x < 100; x++) {
VolumeInfo volInfo = new VolumeInfo("testvol" + Integer.toString(x), VolumeInfo volInfo = new VolumeInfo("testvol" + Integer.toString(x),
"Thu, Apr 9, 2015 10:23:45 GMT", "gandalf"); "Thu, Apr 9, 2015 10:23:45 GMT", "gandalf");
list.addVolume(volInfo); list.addVolume(volInfo);
} }
list.sort(); list.sort();
@ -69,5 +67,4 @@ public void testVolumeListParse() throws IOException {
ListVolumes newList = ListVolumes.parse(listString); ListVolumes newList = ListVolumes.parse(listString);
assertEquals(list.toJsonString(), newList.toJsonString()); assertEquals(list.toJsonString(), newList.toJsonString());
} }
} }

View File

@ -44,6 +44,7 @@
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail; import static org.junit.Assert.fail;
public class TestBuckets { public class TestBuckets {
/** /**
* Set the timeout for every test. * Set the timeout for every test.
@ -83,7 +84,7 @@ public static void init() throws IOException,
} }
/** /**
* shutdown MiniDFSCluster * shutdown MiniDFSCluster.
*/ */
@AfterClass @AfterClass
public static void shutdown() { public static void shutdown() {

View File

@ -110,7 +110,7 @@ public static void init() throws Exception {
} }
/** /**
* shutdown MiniDFSCluster * shutdown MiniDFSCluster.
*/ */
@AfterClass @AfterClass
public static void shutdown() { public static void shutdown() {
@ -120,7 +120,7 @@ public static void shutdown() {
} }
/** /**
* Creates a file with Random Data * Creates a file with Random Data.
* *
* @return File. * @return File.
*/ */
@ -165,9 +165,9 @@ static class PutHelper {
private final String dir; private final String dir;
private final String keyName; private final String keyName;
OzoneVolume vol; private OzoneVolume vol;
OzoneBucket bucket; private OzoneBucket bucket;
File file; private File file;
PutHelper(OzoneRestClient client, String dir) { PutHelper(OzoneRestClient client, String dir) {
this(client, dir, OzoneUtils.getRequestID().toLowerCase()); this(client, dir, OzoneUtils.getRequestID().toLowerCase());

View File

@ -90,7 +90,7 @@ public static void init() throws Exception {
} }
/** /**
* shutdown MiniDFSCluster * shutdown MiniDFSCluster.
*/ */
@AfterClass @AfterClass
public static void shutdown() { public static void shutdown() {
@ -255,10 +255,10 @@ static void runTestListAllVolumes(OzoneRestClient client)
final int step = 10; final int step = 10;
client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER); client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
for (int x = 0; x < volCount; x++) { for (int x = 0; x < volCount; x++) {
String userName = "frodo" + String userName =
RandomStringUtils.randomAlphabetic(5).toLowerCase(); "frodo" + RandomStringUtils.randomAlphabetic(5).toLowerCase();
String volumeName = "vol" + String volumeName =
RandomStringUtils.randomAlphabetic(5).toLowerCase(); "vol" + RandomStringUtils.randomAlphabetic(5).toLowerCase();
OzoneVolume vol = client.createVolume(volumeName, userName, "100TB"); OzoneVolume vol = client.createVolume(volumeName, userName, "100TB");
assertNotNull(vol); assertNotNull(vol);
} }
@ -266,17 +266,16 @@ static void runTestListAllVolumes(OzoneRestClient client)
int count = 0; int count = 0;
int pagecount = 0; int pagecount = 0;
while (count < volCount) { while (count < volCount) {
List<OzoneVolume> ovols = client.listAllVolumes(null, step, List<OzoneVolume> ovols = client.listAllVolumes(null, step, prevKey);
prevKey);
count += ovols.size(); count += ovols.size();
if(ovols.size() > 0) { if (ovols.size() > 0) {
prevKey = ovols.get(ovols.size() - 1); prevKey = ovols.get(ovols.size() - 1);
} }
pagecount++; pagecount++;
} }
// becasue we are querying an existing ozone store, there will // becasue we are querying an existing ozone store, there will
// be volumes created by other tests too. So we should get more page counts. // be volumes created by other tests too. So we should get more page counts.
Assert.assertEquals(volCount / step , pagecount); Assert.assertEquals(volCount / step, pagecount);
} }
@Test @Test