diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/cblock/package-info.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/cblock/package-info.java new file mode 100644 index 00000000000..5f5d3cd956c --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/cblock/package-info.java @@ -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. + */ diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyInfo.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyInfo.java index 6dea835c171..75b2bf8b5a1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyInfo.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyInfo.java @@ -115,13 +115,13 @@ public final class KsmKeyInfo { return this; } - public Builder setCreationTime(long creationTime) { - this.creationTime = creationTime; + public Builder setCreationTime(long crTime) { + this.creationTime = crTime; return this; } - public Builder setModificationTime(long modificationTime) { - this.modificationTime = modificationTime; + public Builder setModificationTime(long mTime) { + this.modificationTime = mTime; return this; } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmVolumeArgs.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmVolumeArgs.java index 4d60827e94f..6b1020c1242 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmVolumeArgs.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmVolumeArgs.java @@ -142,18 +142,18 @@ public final class KsmVolumeArgs { aclMap = new KsmOzoneAclMap(); } - public Builder setAdminName(String adminName) { - this.adminName = adminName; + public Builder setAdminName(String admin) { + this.adminName = admin; return this; } - public Builder setOwnerName(String ownerName) { - this.ownerName = ownerName; + public Builder setOwnerName(String owner) { + this.ownerName = owner; return this; } - public Builder setVolume(String volume) { - this.volume = volume; + public Builder setVolume(String volumeName) { + this.volume = volumeName; return this; } @@ -162,8 +162,8 @@ public final class KsmVolumeArgs { return this; } - public Builder setQuotaInBytes(long quotaInBytes) { - this.quotaInBytes = quotaInBytes; + public Builder setQuotaInBytes(long quota) { + this.quotaInBytes = quota; return this; } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/XceiverClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/XceiverClient.java index 4b987727551..9cdc379f214 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/XceiverClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/XceiverClient.java @@ -123,11 +123,11 @@ public class XceiverClient extends XceiverClientSpi { ContainerProtos.ContainerCommandRequestProto request) throws IOException { try { - if ((channelFuture == null) || (!channelFuture.channel().isActive())) { - throw new IOException("This channel is not connected."); - } - XceiverClientHandler handler = - channelFuture.channel().pipeline().get(XceiverClientHandler.class); + if ((channelFuture == null) || (!channelFuture.channel().isActive())) { + throw new IOException("This channel is not connected."); + } + XceiverClientHandler handler = + channelFuture.channel().pipeline().get(XceiverClientHandler.class); return handler.sendCommand(request); } catch (ExecutionException | InterruptedException e) { @@ -144,13 +144,13 @@ public class XceiverClient extends XceiverClientSpi { */ @Override public CompletableFuture - sendCommandAsync(ContainerProtos.ContainerCommandRequestProto request) + sendCommandAsync(ContainerProtos.ContainerCommandRequestProto request) throws IOException, ExecutionException, InterruptedException { - if ((channelFuture == null) || (!channelFuture.channel().isActive())) { - throw new IOException("This channel is not connected."); - } - XceiverClientHandler handler = - channelFuture.channel().pipeline().get(XceiverClientHandler.class); - return handler.sendCommandAsync(request); + if ((channelFuture == null) || (!channelFuture.channel().isActive())) { + throw new IOException("This channel is not connected."); + } + XceiverClientHandler handler = + channelFuture.channel().pipeline().get(XceiverClientHandler.class); + return handler.sendCommandAsync(request); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/XceiverClientHandler.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/XceiverClientHandler.java index 93d4438e99d..cffd91eed0b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/XceiverClientHandler.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/XceiverClientHandler.java @@ -109,8 +109,8 @@ public class XceiverClientHandler extends * @return -- response */ - public ContainerCommandResponseProto - sendCommand(ContainerProtos.ContainerCommandRequestProto request) + public ContainerCommandResponseProto sendCommand( + ContainerProtos.ContainerCommandRequestProto request) throws ExecutionException, InterruptedException { Future future = sendCommandAsync(request); return future.get(); @@ -123,11 +123,11 @@ public class XceiverClientHandler extends * @param request - Request to execute * @return CompletableFuture */ - public CompletableFuture - sendCommandAsync(ContainerProtos.ContainerCommandRequestProto request) { + public CompletableFuture sendCommandAsync( + ContainerProtos.ContainerCommandRequestProto request) { // 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"); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/XceiverClientRatis.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/XceiverClientRatis.java index 8ecf6f59d6f..d6c7937fabb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/XceiverClientRatis.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/XceiverClientRatis.java @@ -20,7 +20,6 @@ package org.apache.hadoop.scm; import com.google.common.base.Preconditions; 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.ContainerCommandResponseProto; import org.apache.ratis.RatisHelper; @@ -115,8 +114,8 @@ public final class XceiverClientRatis extends XceiverClientSpi { * @throws IOException */ @Override - public CompletableFuture - sendCommandAsync(ContainerCommandRequestProto request) + public CompletableFuture sendCommandAsync( + ContainerCommandRequestProto request) throws IOException, ExecutionException, InterruptedException { throw new IOException("Not implemented"); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/XceiverClientSpi.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/XceiverClientSpi.java index aed29dcb264..3f297ad2726 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/XceiverClientSpi.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/XceiverClientSpi.java @@ -103,8 +103,8 @@ public abstract class XceiverClientSpi implements Closeable { * @return Response to the command * @throws IOException */ - public abstract CompletableFuture - sendCommandAsync(ContainerCommandRequestProto request) throws IOException, - ExecutionException, InterruptedException; + public abstract CompletableFuture sendCommandAsync( + ContainerCommandRequestProto request) + throws IOException, ExecutionException, InterruptedException; } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/container/common/helpers/ContainerInfo.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/container/common/helpers/ContainerInfo.java index 16bc0db177e..34be5f1e241 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/container/common/helpers/ContainerInfo.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/container/common/helpers/ContainerInfo.java @@ -82,13 +82,14 @@ public class ContainerInfo { return builder.build(); } + /** Builder class for ContainerInfo. */ public static class Builder { private OzoneProtos.LifeCycleState state; private Pipeline pipeline; private long stateEnterTime; - public Builder setState(OzoneProtos.LifeCycleState state) { - this.state = state; + public Builder setState(OzoneProtos.LifeCycleState lifeCycleState) { + this.state = lifeCycleState; return this; } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/storage/ContainerProtocolCalls.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/storage/ContainerProtocolCalls.java index 48e335f07b1..0e992a5b9a2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/storage/ContainerProtocolCalls.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/storage/ContainerProtocolCalls.java @@ -188,40 +188,33 @@ public final class ContainerProtocolCalls { * @param traceID - Trace ID for logging purpose. * @throws IOException */ - public static void writeSmallFile(XceiverClientSpi client, String containerName, - String key, byte[] data, String traceID) throws IOException { + public static void writeSmallFile(XceiverClientSpi client, + String containerName, String key, byte[] data, String traceID) + throws IOException { - KeyData containerKeyData = KeyData - .newBuilder() - .setContainerName(containerName) - .setName(key).build(); - PutKeyRequestProto.Builder createKeyRequest = PutKeyRequestProto - .newBuilder() - .setPipeline(client.getPipeline().getProtobufMessage()) - .setKeyData(containerKeyData); + KeyData containerKeyData = + KeyData.newBuilder().setContainerName(containerName).setName(key) + .build(); + PutKeyRequestProto.Builder createKeyRequest = + PutKeyRequestProto.newBuilder() + .setPipeline(client.getPipeline().getProtobufMessage()) + .setKeyData(containerKeyData); - KeyValue keyValue = KeyValue.newBuilder() - .setKey("OverWriteRequested").setValue("true").build(); - ChunkInfo chunk = ChunkInfo - .newBuilder() - .setChunkName(key + "_chunk") - .setOffset(0) - .setLen(data.length) - .addMetadata(keyValue) - .build(); + KeyValue keyValue = + KeyValue.newBuilder().setKey("OverWriteRequested").setValue("true") + .build(); + ChunkInfo chunk = + ChunkInfo.newBuilder().setChunkName(key + "_chunk").setOffset(0) + .setLen(data.length).addMetadata(keyValue).build(); - PutSmallFileRequestProto putSmallFileRequest = PutSmallFileRequestProto - .newBuilder().setChunkInfo(chunk) - .setKey(createKeyRequest) - .setData(ByteString.copyFrom(data)) - .build(); + PutSmallFileRequestProto putSmallFileRequest = + PutSmallFileRequestProto.newBuilder().setChunkInfo(chunk) + .setKey(createKeyRequest).setData(ByteString.copyFrom(data)) + .build(); - ContainerCommandRequestProto request = ContainerCommandRequestProto - .newBuilder() - .setCmdType(Type.PutSmallFile) - .setTraceID(traceID) - .setPutSmallFile(putSmallFileRequest) - .build(); + ContainerCommandRequestProto request = + ContainerCommandRequestProto.newBuilder().setCmdType(Type.PutSmallFile) + .setTraceID(traceID).setPutSmallFile(putSmallFileRequest).build(); ContainerCommandResponseProto response = client.sendCommand(request); validateContainerResponse(response); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/ozone/TestOzoneAcls.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/ozone/TestOzoneAcls.java index e8fd4d3238f..c70b75ec441 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/ozone/TestOzoneAcls.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/ozone/TestOzoneAcls.java @@ -26,10 +26,13 @@ import java.util.Set; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; +/** + * This class is to test acl stoarge and retreival in ozone store. + */ public class TestOzoneAcls { @Test - public void TestACLParse() { + public void testAclParse() { HashMap testMatrix; testMatrix = new HashMap<>(); @@ -103,7 +106,7 @@ public class TestOzoneAcls { } @Test - public void TestACLValues() { + public void testAclValues() { OzoneAcl acl = OzoneAcl.parseAcl("user:bilbo:rw"); assertEquals(acl.getName(), "bilbo"); assertEquals(acl.getRights(), OzoneAcl.OzoneACLRights.READ_WRITE); diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/scm/TestArchive.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/scm/TestArchive.java index 30305758cfe..86adddb9cef 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/scm/TestArchive.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/scm/TestArchive.java @@ -46,6 +46,7 @@ public class TestArchive { private long checksumWrite = 0L; private long checksumRead = 0L; private long tmp = 0L; + private Checksum crc = new Adler32(); @Rule public TemporaryFolder folder = new TemporaryFolder(); @@ -53,7 +54,6 @@ public class TestArchive { @Rule public TemporaryFolder outputFolder = new TemporaryFolder(); - Checksum crc = new Adler32(); @Before public void setUp() throws Exception { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyUtils.java index d508e91bec0..b1c2b0017f3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyUtils.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyUtils.java @@ -23,9 +23,7 @@ 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.utils.ContainerCache; import org.apache.hadoop.utils.MetadataStore; -import org.apache.hadoop.utils.MetadataStoreBuilder; -import java.io.File; import java.io.IOException; import java.nio.charset.Charset; @@ -142,8 +140,8 @@ public final class KeyUtils { KeyData data = KeyData.getFromProtoBuf(kd); return data; } catch (IOException e) { - throw new StorageContainerException("Failed to parse key data from the bytes array.", - NO_SUCH_KEY); + throw new StorageContainerException("Failed to parse key data from the" + + " bytes array.", NO_SUCH_KEY); } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/utils/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/utils/package-info.java new file mode 100644 index 00000000000..08264f084a0 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/utils/package-info.java @@ -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; \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/block/BlockManagerImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/block/BlockManagerImpl.java index 23b1c8bdd2f..23faaf9430d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/block/BlockManagerImpl.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/block/BlockManagerImpl.java @@ -290,7 +290,8 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean { private BlockContainerInfo getContainer(OzoneProtos.LifeCycleState state, String name) { - Map containersByState = this.containers.get(state); + Map containersByState = + this.containers.get(state); return containersByState.get(name); } @@ -318,15 +319,15 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean { private void refreshContainers() { Map containersByState = this.containers.get(OzoneProtos.LifeCycleState.CREATING); - for (String containerName: containersByState.keySet()) { + for (String containerName : containersByState.keySet()) { try { ContainerInfo containerInfo = containerManager.getContainer(containerName); if (containerInfo == null) { // TODO: clean up containers that has been deleted on SCM but // TODO: still in ALLOCATED state in block manager. - LOG.debug("Container {} allocated by block service" + - "can't be found in SCM", containerName); + LOG.debug("Container {} allocated by block service" + + "can't be found in SCM", containerName); continue; } if (containerInfo.getState() == OzoneProtos.LifeCycleState.OPEN) { @@ -340,7 +341,7 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean { LOG.debug("Failed to get container info for: {}", containerName); } } - } + } /** * Allocates a new block for a given size. @@ -356,8 +357,7 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean { public AllocatedBlock allocateBlock(final long size) throws IOException { boolean createContainer = false; if (size < 0 || size > containerSize) { - throw new SCMException("Unsupported block size", - INVALID_BLOCK_SIZE); + throw new SCMException("Unsupported block size", INVALID_BLOCK_SIZE); } if (!nodeManager.isOutOfNodeChillMode()) { throw new SCMException("Unable to create block while in chill mode", @@ -370,8 +370,8 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean { List candidates; candidates = filterContainers(OzoneProtos.LifeCycleState.OPEN, size); if (candidates.size() == 0) { - candidates = filterContainers(OzoneProtos.LifeCycleState.ALLOCATED, - size); + candidates = + filterContainers(OzoneProtos.LifeCycleState.ALLOCATED, size); if (candidates.size() == 0) { try { candidates = allocateContainers(containerProvisionBatchSize); @@ -383,8 +383,9 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean { } // now we should have some candidates in ALLOCATE state if (candidates.size() == 0) { - throw new SCMException("Fail to find any container to allocate block " - + "of size " + size + ".", FAILED_TO_FIND_CONTAINER_WITH_SPACE); + throw new SCMException( + "Fail to find any container to allocate block " + "of size " + + size + ".", FAILED_TO_FIND_CONTAINER_WITH_SPACE); } } @@ -416,8 +417,8 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean { // TODO: make block key easier to debug (e.g., seq no) // Allocate key for the block String blockKey = UUID.randomUUID().toString(); - AllocatedBlock.Builder abb = new AllocatedBlock.Builder() - .setKey(blockKey).setPipeline(containerInfo.getPipeline()) + AllocatedBlock.Builder abb = new AllocatedBlock.Builder().setKey(blockKey) + .setPipeline(containerInfo.getPipeline()) .setShouldCreateContainer(createContainer); if (containerInfo.getPipeline().getMachines().size() > 0) { blockStore.put(DFSUtil.string2Bytes(blockKey), @@ -428,12 +429,12 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean { getContainer(containerInfo.getState(), containerName); Preconditions.checkNotNull(containerInfoUpdate); containerInfoUpdate.addAllocated(size); - containerStore.put(DFSUtil.string2Bytes(containerName), - DFSUtil.string2Bytes(Long.toString(containerInfoUpdate.getAllocated()))); + containerStore.put(DFSUtil.string2Bytes(containerName), DFSUtil + .string2Bytes(Long.toString(containerInfoUpdate.getAllocated()))); if (createContainer) { - OzoneProtos.LifeCycleState newState = - containerManager.updateContainerState(containerName, - OzoneProtos.LifeCycleEvent.BEGIN_CREATE); + OzoneProtos.LifeCycleState newState = containerManager + .updateContainerState(containerName, + OzoneProtos.LifeCycleEvent.BEGIN_CREATE); updateContainer(containerInfo.getState(), containerName, newState); } return abb.build(); @@ -457,17 +458,19 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean { try { byte[] containerBytes = blockStore.get(DFSUtil.string2Bytes(key)); if (containerBytes == null) { - throw new SCMException("Specified block key does not exist. key : " + - key, FAILED_TO_FIND_BLOCK); + throw new SCMException( + "Specified block key does not exist. key : " + key, + FAILED_TO_FIND_BLOCK); } String containerName = DFSUtil.bytes2String(containerBytes); - ContainerInfo containerInfo = containerManager.getContainer( - containerName); + ContainerInfo containerInfo = + containerManager.getContainer(containerName); if (containerInfo == null) { - LOG.debug("Container {} allocated by block service" + - "can't be found in SCM", containerName); - throw new SCMException("Unable to find container for the block", - SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER); + LOG.debug( + "Container {} allocated by block service" + "can't be found in SCM", + containerName); + throw new SCMException("Unable to find container for the block", + SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER); } return containerInfo.getPipeline(); } finally { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/container/ContainerMapping.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/container/ContainerMapping.java index c71f1279c5a..5a1af514c58 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/container/ContainerMapping.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/container/ContainerMapping.java @@ -158,15 +158,17 @@ public class ContainerMapping implements Mapping { * {@inheritDoc} */ @Override - public ContainerInfo getContainer(final String containerName) throws IOException { + public ContainerInfo getContainer(final String containerName) + throws IOException { ContainerInfo containerInfo; lock.lock(); try { byte[] containerBytes = containerStore.get(containerName.getBytes(encoding)); if (containerBytes == null) { - throw new SCMException("Specified key does not exist. key : " + - containerName, SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER); + throw new SCMException( + "Specified key does not exist. key : " + containerName, + SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER); } containerInfo = ContainerInfo.fromProtobuf( OzoneProtos.SCMContainerInfo.PARSER.parseFrom(containerBytes)); @@ -305,11 +307,11 @@ public class ContainerMapping implements Mapping { lock.lock(); try { byte[] dbKey = containerName.getBytes(encoding); - byte[] containerBytes = - containerStore.get(dbKey); - if(containerBytes == null) { - throw new SCMException("Failed to update container state" - + containerName + ", reason : container doesn't exist.", + byte[] containerBytes = containerStore.get(dbKey); + if (containerBytes == null) { + throw new SCMException( + "Failed to update container state" + containerName + + ", reason : container doesn't exist.", SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER); } containerInfo = ContainerInfo.fromProtobuf( @@ -317,11 +319,12 @@ public class ContainerMapping implements Mapping { OzoneProtos.LifeCycleState newState; try { - newState = stateMachine.getNextState(containerInfo.getState(), event); + newState = stateMachine.getNextState(containerInfo.getState(), event); } catch (InvalidStateTransitionException ex) { - throw new SCMException("Failed to update container state" - + containerName + ", reason : invalid state transition from state: " - + containerInfo.getState() + " upon event: " + event + ".", + throw new SCMException( + "Failed to update container state" + containerName + + ", reason : invalid state transition from state: " + + containerInfo.getState() + " upon event: " + event + ".", SCMException.ResultCodes.FAILED_TO_CHANGE_CONTAINER_STATE); } containerInfo.setState(newState); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/exceptions/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/exceptions/package-info.java new file mode 100644 index 00000000000..59cf724c50e --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/exceptions/package-info.java @@ -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. + */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/package-info.java new file mode 100644 index 00000000000..4d34c2d0131 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/package-info.java @@ -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. + */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/Keys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/Keys.java index 1e830b161c7..ef70a6bcea0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/Keys.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/Keys.java @@ -62,14 +62,9 @@ public interface Keys { @PUT @Consumes(MediaType.WILDCARD) Response putKey(@PathParam("volume") String volume, - @PathParam("bucket") String bucket, - @PathParam("keys") String keys, - InputStream is, - @Context Request req, - @Context UriInfo info, - @Context HttpHeaders headers) - throws OzoneException; - + @PathParam("bucket") String bucket, @PathParam("keys") String keys, + InputStream is, @Context Request req, @Context UriInfo info, + @Context HttpHeaders headers) throws OzoneException; /** * Gets the Key if it exists. @@ -88,14 +83,10 @@ public interface Keys { */ @GET Response getKey(@PathParam("volume") String volume, - @PathParam("bucket") String bucket, - @PathParam("keys") String keys, - @QueryParam(Header.OZONE_LIST_QUERY_TAG) - String info, - @Context Request req, - @Context UriInfo uriInfo, - @Context HttpHeaders headers) - throws OzoneException; + @PathParam("bucket") String bucket, @PathParam("keys") String keys, + @QueryParam(Header.OZONE_LIST_QUERY_TAG) String info, + @Context Request req, @Context UriInfo uriInfo, + @Context HttpHeaders headers) throws OzoneException; /** * Deletes an existing key. @@ -112,11 +103,8 @@ public interface Keys { */ @DELETE Response deleteKey(@PathParam("volume") String volume, - @PathParam("bucket") String bucket, - @PathParam("keys") String keys, - @Context Request req, - @Context UriInfo info, - @Context HttpHeaders headers) - throws OzoneException; + @PathParam("bucket") String bucket, @PathParam("keys") String keys, + @Context Request req, @Context UriInfo info, @Context HttpHeaders headers) + throws OzoneException; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/Volume.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/Volume.java index 91ede366226..1d0b1af9671 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/Volume.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/Volume.java @@ -65,13 +65,10 @@ public interface Volume { @POST Response createVolume(@PathParam("volume") String volume, - @DefaultValue(Header.OZONE_QUOTA_UNDEFINED) - @QueryParam(Header.OZONE_QUOTA_QUERY_TAG) String quota, - @Context Request req, - @Context UriInfo uriInfo, - @Context HttpHeaders headers) - throws OzoneException; - + @DefaultValue(Header.OZONE_QUOTA_UNDEFINED) + @QueryParam(Header.OZONE_QUOTA_QUERY_TAG) String quota, + @Context Request req, @Context UriInfo uriInfo, + @Context HttpHeaders headers) throws OzoneException; /** * Updates a Volume owned by the user. @@ -93,13 +90,10 @@ public interface Volume { */ @PUT Response updateVolume(@PathParam("volume") String volume, - @DefaultValue(Header.OZONE_QUOTA_UNDEFINED) - @QueryParam(Header.OZONE_QUOTA_QUERY_TAG) String quota, - @Context Request req, - @Context UriInfo uriInfo, - @Context HttpHeaders headers) - throws OzoneException; - + @DefaultValue(Header.OZONE_QUOTA_UNDEFINED) + @QueryParam(Header.OZONE_QUOTA_QUERY_TAG) String quota, + @Context Request req, @Context UriInfo uriInfo, + @Context HttpHeaders headers) throws OzoneException; /** * Deletes a Volume if it is empty. @@ -112,10 +106,8 @@ public interface Volume { */ @DELETE Response deleteVolume(@PathParam("volume") String volume, - @Context Request req, - @Context UriInfo uriInfo, - @Context HttpHeaders headers) - throws OzoneException; + @Context Request req, @Context UriInfo uriInfo, + @Context HttpHeaders headers) throws OzoneException; /** * Returns Volume info. This API can be invoked either @@ -131,21 +123,14 @@ public interface Volume { */ @GET Response getVolumeInfo(@PathParam("volume") String volume, - @DefaultValue(Header.OZONE_LIST_QUERY_BUCKET) - @QueryParam(Header.OZONE_LIST_QUERY_TAG) - String info, - @QueryParam(Header.OZONE_LIST_QUERY_PREFIX) - String prefix, - @DefaultValue(Header.OZONE_DEFAULT_LIST_SIZE) - @QueryParam(Header.OZONE_LIST_QUERY_MAXKEYS) - int keys, - @QueryParam(Header.OZONE_LIST_QUERY_PREVKEY) - String prevKey, - @QueryParam(Header.OZONE_LIST_QUERY_ROOTSCAN) - boolean rootScan, - @Context Request req, - @Context UriInfo uriInfo, - @Context HttpHeaders headers) - throws OzoneException; + @DefaultValue(Header.OZONE_LIST_QUERY_BUCKET) + @QueryParam(Header.OZONE_LIST_QUERY_TAG) String info, + @QueryParam(Header.OZONE_LIST_QUERY_PREFIX) String prefix, + @DefaultValue(Header.OZONE_DEFAULT_LIST_SIZE) + @QueryParam(Header.OZONE_LIST_QUERY_MAXKEYS) int keys, + @QueryParam(Header.OZONE_LIST_QUERY_PREVKEY) String prevKey, + @QueryParam(Header.OZONE_LIST_QUERY_ROOTSCAN) boolean rootScan, + @Context Request req, @Context UriInfo uriInfo, + @Context HttpHeaders headers) throws OzoneException; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/package-info.java new file mode 100644 index 00000000000..940f1799e3c --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/package-info.java @@ -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. + */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/package-info.java new file mode 100644 index 00000000000..6bf66433ea6 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/package-info.java @@ -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; \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/messages/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/messages/package-info.java new file mode 100644 index 00000000000..273b3f59227 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/messages/package-info.java @@ -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; \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/netty/CloseableCleanupListener.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/netty/CloseableCleanupListener.java index 5cfaa75dad7..eac8e6fbf96 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/netty/CloseableCleanupListener.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/netty/CloseableCleanupListener.java @@ -35,7 +35,7 @@ final class CloseableCleanupListener implements ChannelFutureListener { * * @param closeables any number of closeable resources */ - public CloseableCleanupListener(Closeable... closeables) { + CloseableCleanupListener(Closeable... closeables) { this.closeables = closeables; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/netty/ObjectStoreJerseyContainer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/netty/ObjectStoreJerseyContainer.java index 4a5ea27e7d8..efdc2882770 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/netty/ObjectStoreJerseyContainer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/netty/ObjectStoreJerseyContainer.java @@ -213,7 +213,7 @@ public final class ObjectStoreJerseyContainer { * @param respOut output stream for writing response body * @param latch for coordinating asynchronous return of HTTP response */ - public RequestRunner(HttpRequest nettyReq, InputStream reqIn, + RequestRunner(HttpRequest nettyReq, InputStream reqIn, OutputStream respOut, CountDownLatch latch) { this.latch = latch; this.nettyReq = nettyReq; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/package-info.java new file mode 100644 index 00000000000..c344c35ae11 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/package-info.java @@ -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; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/GetKeyHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/GetKeyHandler.java index a244a249f47..5a6dc9f5d69 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/GetKeyHandler.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/GetKeyHandler.java @@ -20,9 +20,7 @@ package org.apache.hadoop.ozone.web.ozShell.keys; import org.apache.commons.cli.CommandLine; 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.OzoneVolume; import org.apache.hadoop.ozone.web.exceptions.OzoneException; import org.apache.hadoop.ozone.web.ozShell.Handler; import org.apache.hadoop.ozone.web.ozShell.Shell; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/PutKeyHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/PutKeyHandler.java index 30cc2cbc024..dea919e2af0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/PutKeyHandler.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/PutKeyHandler.java @@ -20,9 +20,7 @@ package org.apache.hadoop.ozone.web.ozShell.keys; import org.apache.commons.cli.CommandLine; 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.OzoneVolume; import org.apache.hadoop.ozone.web.exceptions.OzoneException; import org.apache.hadoop.ozone.web.ozShell.Handler; import org.apache.hadoop.ozone.web.ozShell.Shell; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/package-info.java new file mode 100644 index 00000000000..1deb7ad419b --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/package-info.java @@ -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; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/package-info.java new file mode 100644 index 00000000000..fc192741e8d --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/package-info.java @@ -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; \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/request/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/request/package-info.java new file mode 100644 index 00000000000..4fbc18ff6e4 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/request/package-info.java @@ -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; \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/response/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/response/package-info.java new file mode 100644 index 00000000000..3bf66c861be --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/response/package-info.java @@ -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; \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/userauth/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/userauth/package-info.java new file mode 100644 index 00000000000..d498fc8ebeb --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/userauth/package-info.java @@ -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; \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java index f9e1fd0b476..3038769d1fd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java @@ -442,13 +442,13 @@ public class TestPipelinesFailover { }; for (String[] scmd: scmds) { - String scmd_str = StringUtils.join(" ", scmd); + String scmdStr = StringUtils.join(" ", scmd); try { ShellCommandExecutor sce = new ShellCommandExecutor(scmd); sce.execute(); - LOG.info("'" + scmd_str + "' output:\n" + sce.getOutput()); + LOG.info("'" + scmdStr + "' output:\n" + sce.getOutput()); } catch (IOException e) { - LOG.warn("Error when running '" + scmd_str + "'", e); + LOG.warn("Error when running '" + scmdStr + "'", e); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java index 35f2861b16c..65cf654b036 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java @@ -306,28 +306,27 @@ public class TestEndPoint { // Create a datanode state machine for stateConext used by endpoint task - try (DatanodeStateMachine stateMachine = - new DatanodeStateMachine(DFSTestUtil.getLocalDatanodeID(), conf); - EndpointStateMachine rpcEndPoint = SCMTestUtils.createEndpoint(conf, - scmAddress, rpcTimeout)) { - ContainerNodeIDProto containerNodeID = ContainerNodeIDProto.newBuilder() - .setClusterID(UUID.randomUUID().toString()) - .setDatanodeID(getDatanodeID().getProtoBufMessage()) - .build(); - rpcEndPoint.setState(EndpointStateMachine.EndPointStates.HEARTBEAT); + try (DatanodeStateMachine stateMachine = new DatanodeStateMachine( + DFSTestUtil.getLocalDatanodeID(), conf); + EndpointStateMachine rpcEndPoint = SCMTestUtils + .createEndpoint(conf, scmAddress, rpcTimeout)) { + ContainerNodeIDProto containerNodeID = ContainerNodeIDProto.newBuilder() + .setClusterID(UUID.randomUUID().toString()) + .setDatanodeID(getDatanodeID().getProtoBufMessage()).build(); + rpcEndPoint.setState(EndpointStateMachine.EndPointStates.HEARTBEAT); - final StateContext stateContext = new StateContext(conf, - DatanodeStateMachine.DatanodeStates.RUNNING, - stateMachine); + final StateContext stateContext = + new StateContext(conf, DatanodeStateMachine.DatanodeStates.RUNNING, + stateMachine); - HeartbeatEndpointTask endpointTask = - new HeartbeatEndpointTask(rpcEndPoint, conf, stateContext); - endpointTask.setContainerNodeIDProto(containerNodeID); - endpointTask.call(); - Assert.assertNotNull(endpointTask.getContainerNodeIDProto()); + HeartbeatEndpointTask endpointTask = + new HeartbeatEndpointTask(rpcEndPoint, conf, stateContext); + endpointTask.setContainerNodeIDProto(containerNodeID); + endpointTask.call(); + Assert.assertNotNull(endpointTask.getContainerNodeIDProto()); - Assert.assertEquals(EndpointStateMachine.EndPointStates.HEARTBEAT, - rpcEndPoint.getState()); + Assert.assertEquals(EndpointStateMachine.EndPointStates.HEARTBEAT, + rpcEndPoint.getState()); } } @@ -360,8 +359,8 @@ public class TestEndPoint { * @return */ ContainerReport getRandomContainerReport() { - return new ContainerReport(UUID.randomUUID().toString() - ,DigestUtils.sha256Hex("Random")); + return new ContainerReport(UUID.randomUUID().toString(), + DigestUtils.sha256Hex("Random")); } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java index db1774d0961..5b2fce1df99 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java @@ -90,7 +90,7 @@ public class TestContainerPersistence { @Rule public Timeout testTimeout = new Timeout(300000); - private static Logger LOG = + private static Logger log = LoggerFactory.getLogger(TestContainerPersistence.class); private static String path; private static ContainerManagerImpl containerManager; @@ -149,7 +149,7 @@ public class TestContainerPersistence { @After public void cleanupDir() throws IOException { // Clean up SCM metadata - LOG.info("Deletting {}", path); + log.info("Deletting {}", path); FileUtils.deleteDirectory(new File(path)); // Clean up SCM datanode container metadata/data diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ksm/TestKSMSQLCli.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ksm/TestKSMSQLCli.java index 9fa7ec5e5d7..4ca981f5016 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ksm/TestKSMSQLCli.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ksm/TestKSMSQLCli.java @@ -29,7 +29,6 @@ import org.apache.hadoop.ozone.web.handlers.VolumeArgs; import org.apache.hadoop.ozone.web.interfaces.StorageHandler; import org.apache.hadoop.ozone.web.utils.OzoneUtils; import org.junit.Before; -import org.junit.BeforeClass; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ksm/TestKeySpaceManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ksm/TestKeySpaceManager.java index 63654007b9f..00367dce0ff 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ksm/TestKeySpaceManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ksm/TestKeySpaceManager.java @@ -523,14 +523,14 @@ public class TestKeySpaceManager { stream.write(dataString.getBytes()); } - // We allow the key overwrite to be successful. Please note : Till HDFS-11922 - // is fixed this causes a data block leak on the data node side. That is - // this overwrite only overwrites the keys on KSM. We need to garbage - // collect those blocks from datanode. + // We allow the key overwrite to be successful. Please note : Till + // HDFS-11922 is fixed this causes a data block leak on the data node side. + // That is this overwrite only overwrites the keys on KSM. We need to + // garbage collect those blocks from datanode. KeyArgs keyArgs2 = new KeyArgs(volumeName, bucketName, keyName, userArgs); storageHandler.newKeyWriter(keyArgs2); - Assert.assertEquals(numKeyAllocateFails, - ksmMetrics.getNumKeyAllocateFails()); + Assert + .assertEquals(numKeyAllocateFails, ksmMetrics.getNumKeyAllocateFails()); } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/TestSCMMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/TestSCMMXBean.java index fadead19554..b97e420ceae 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/TestSCMMXBean.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/TestSCMMXBean.java @@ -44,6 +44,10 @@ import java.util.concurrent.TimeoutException; import javax.management.openmbean.CompositeData; import javax.management.openmbean.TabularData; +/** + * + * This class is to test JMX management interface for scm information. + */ public class TestSCMMXBean { public static final Log LOG = LogFactory.getLog(TestSCMMXBean.class); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestBucketInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestBucketInfo.java index c7b3b16c8d7..0678b630531 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestBucketInfo.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestBucketInfo.java @@ -31,7 +31,7 @@ import java.util.List; public class TestBucketInfo { @Test - public void TestBucketInfoJson() throws IOException { + public void testBucketInfoJson() throws IOException { BucketInfo bucketInfo = new BucketInfo("volumeName", "bucketName"); String bucketInfoString = bucketInfo.toJsonString(); BucketInfo newBucketInfo = BucketInfo.parse(bucketInfoString); @@ -39,7 +39,7 @@ public class TestBucketInfo { } @Test - public void TestBucketInfoDBString() throws IOException { + public void testBucketInfoDBString() throws IOException { BucketInfo bucketInfo = new BucketInfo("volumeName", "bucketName"); String bucketInfoString = bucketInfo.toDBString(); BucketInfo newBucketInfo = BucketInfo.parse(bucketInfoString); @@ -47,7 +47,7 @@ public class TestBucketInfo { } @Test - public void TestBucketInfoAddAcls() throws IOException { + public void testBucketInfoAddAcls() throws IOException { BucketInfo bucketInfo = new BucketInfo("volumeName", "bucketName"); String bucketInfoString = bucketInfo.toDBString(); BucketInfo newBucketInfo = BucketInfo.parse(bucketInfoString); @@ -64,7 +64,7 @@ public class TestBucketInfo { @Test - public void TestBucketInfoVersionAndType() throws IOException { + public void testBucketInfoVersionAndType() throws IOException { BucketInfo bucketInfo = new BucketInfo("volumeName", "bucketName"); bucketInfo.setVersioning(OzoneConsts.Versioning.ENABLED); bucketInfo.setStorageType(StorageType.DISK); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestErrorCode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestErrorCode.java index 6639e4816d4..b707bc8b935 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestErrorCode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestErrorCode.java @@ -30,7 +30,7 @@ public class TestErrorCode { * Test Error Generator functions. */ @Test - public void TestErrorGen() { + public void testErrorGen() { OzoneException e = ErrorTable .newError(ErrorTable.ACCESS_DENIED, getRequestID(), "/test/path", "localhost"); @@ -40,7 +40,7 @@ public class TestErrorCode { } @Test - public void TestErrorGenWithException() { + public void testErrorGenWithException() { OzoneException e = new OzoneException(ErrorTable.ACCESS_DENIED.getHttpCode(), "short message", new Exception("Hello")); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneWebAccess.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneWebAccess.java index 09dd79798b2..dd38bd59d52 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneWebAccess.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneWebAccess.java @@ -48,7 +48,7 @@ import static org.junit.Assert.assertEquals; public class TestOzoneWebAccess { /** - * Set the timeout for every test + * Set the timeout for every test. */ @Rule public Timeout testTimeout = new Timeout(300000); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestQuota.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestQuota.java index c846147fc8e..9b5a578dad9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestQuota.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestQuota.java @@ -32,7 +32,7 @@ import static org.junit.Assert.fail; public class TestQuota { @Test - public void TestParseQuota() { + public void testParseQuota() { HashMap testMatrix; testMatrix = new HashMap(); @@ -56,11 +56,10 @@ public class TestQuota { testMatrix.put("1024 bytes", Boolean.TRUE); testMatrix.put("1bytes", 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("bytes", Boolean.FALSE); - Set keys = testMatrix.keySet(); for (String key : keys) { if (testMatrix.get(key)) { @@ -79,7 +78,7 @@ public class TestQuota { } @Test - public void TestVerifyQuota() { + public void testVerifyQuota() { OzoneQuota qt = OzoneQuota.parseQuota("10TB"); assertEquals(qt.getSize(), 10); assertEquals(qt.getUnit(), OzoneQuota.Units.TB); @@ -107,7 +106,7 @@ public class TestQuota { } @Test - public void TestVerifyRemove() { + public void testVerifyRemove() { assertTrue(OzoneQuota.isRemove("remove")); assertFalse(OzoneQuota.isRemove("not remove")); assertFalse(OzoneQuota.isRemove(null)); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestUtils.java index 3917016d095..4891e4cb6f0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestUtils.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestUtils.java @@ -35,7 +35,7 @@ public class TestUtils { * Tests if the bucket name handling is correct. */ @Test - public void TestValidBucketNames() { + public void testValidBucketNames() { HashMap testMatrix; // Init the Table with Strings and Expected Return values testMatrix = new HashMap(); @@ -47,9 +47,8 @@ public class TestUtils { testMatrix.put("bucket..ozone.self", Boolean.FALSE); testMatrix.put("192.1.1.1", Boolean.FALSE); testMatrix.put("ab", Boolean.FALSE); - testMatrix.put("bucket.ozone.self.this.is.a.really.long.name.that." + - "is.more.than.sixty.three.characters.long.for.sure", - Boolean.FALSE); + testMatrix.put("bucket.ozone.self.this.is.a.really.long.name.that." + + "is.more.than.sixty.three.characters.long.for.sure", Boolean.FALSE); testMatrix.put(null, Boolean.FALSE); testMatrix.put("bucket@$", Boolean.FALSE); testMatrix.put("BUCKET", Boolean.FALSE); @@ -66,7 +65,7 @@ public class TestUtils { Set keys = testMatrix.keySet(); for (String key : keys) { - if(testMatrix.get(key)) { + if (testMatrix.get(key)) { // For valid names there should be no exceptions at all verifyResourceName(key); @@ -74,8 +73,8 @@ public class TestUtils { try { verifyResourceName(key); // should never get here since the isValid call will throw - fail("An exception was expected but did not happen."); - } catch(IllegalArgumentException e){ + fail("An exception was expected but did not happen."); + } catch (IllegalArgumentException e) { } } @@ -90,11 +89,10 @@ public class TestUtils { * that this test is good enough. */ @Test - public void TestRequestIDisRandom(){ + public void testRequestIDisRandom() { HashSet set = new HashSet<>(); - for (int i = 0; i < 1000; i ++){ + for (int i = 0; i < 1000; i++) { assertTrue(set.add(getRequestID())); } } - } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestVolumeStructs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestVolumeStructs.java index 551e7d48095..ef5a3468006 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestVolumeStructs.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestVolumeStructs.java @@ -32,8 +32,8 @@ public class TestVolumeStructs { @Test public void testVolumeInfoParse() throws IOException { - VolumeInfo volInfo = new VolumeInfo("testvol", - "Thu, Apr 9, 2015 10:23:45 GMT", "gandalf"); + VolumeInfo volInfo = + new VolumeInfo("testvol", "Thu, Apr 9, 2015 10:23:45 GMT", "gandalf"); VolumeOwner owner = new VolumeOwner("bilbo"); volInfo.setOwner(owner); String jString = volInfo.toJsonString(); @@ -48,8 +48,7 @@ public class TestVolumeStructs { public void testVolumeInfoValue() throws IOException { String createdOn = "Thu, Apr 9, 2015 10:23:45 GMT"; String createdBy = "gandalf"; - VolumeInfo volInfo = new VolumeInfo("testvol", - createdOn, createdBy); + VolumeInfo volInfo = new VolumeInfo("testvol", createdOn, createdBy); assertEquals(volInfo.getCreatedBy(), createdBy); assertEquals(volInfo.getCreatedOn(), createdOn); } @@ -57,11 +56,10 @@ public class TestVolumeStructs { @Test public void testVolumeListParse() throws IOException { - ListVolumes list = new ListVolumes(); for (int x = 0; x < 100; 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.sort(); @@ -69,5 +67,4 @@ public class TestVolumeStructs { ListVolumes newList = ListVolumes.parse(listString); assertEquals(list.toJsonString(), newList.toJsonString()); } - } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java index a288bdabab5..eb5316290bd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java @@ -44,6 +44,7 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; + public class TestBuckets { /** * Set the timeout for every test. @@ -83,7 +84,7 @@ public class TestBuckets { } /** - * shutdown MiniDFSCluster + * shutdown MiniDFSCluster. */ @AfterClass public static void shutdown() { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java index 81a89a7be65..97f11a919c9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java @@ -110,7 +110,7 @@ public class TestKeys { } /** - * shutdown MiniDFSCluster + * shutdown MiniDFSCluster. */ @AfterClass public static void shutdown() { @@ -120,7 +120,7 @@ public class TestKeys { } /** - * Creates a file with Random Data + * Creates a file with Random Data. * * @return File. */ @@ -165,9 +165,9 @@ public class TestKeys { private final String dir; private final String keyName; - OzoneVolume vol; - OzoneBucket bucket; - File file; + private OzoneVolume vol; + private OzoneBucket bucket; + private File file; PutHelper(OzoneRestClient client, String dir) { this(client, dir, OzoneUtils.getRequestID().toLowerCase()); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java index 9c28f089f68..698765135dd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java @@ -90,7 +90,7 @@ public class TestVolume { } /** - * shutdown MiniDFSCluster + * shutdown MiniDFSCluster. */ @AfterClass public static void shutdown() { @@ -255,10 +255,10 @@ public class TestVolume { final int step = 10; client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER); for (int x = 0; x < volCount; x++) { - String userName = "frodo" + - RandomStringUtils.randomAlphabetic(5).toLowerCase(); - String volumeName = "vol" + - RandomStringUtils.randomAlphabetic(5).toLowerCase(); + String userName = + "frodo" + RandomStringUtils.randomAlphabetic(5).toLowerCase(); + String volumeName = + "vol" + RandomStringUtils.randomAlphabetic(5).toLowerCase(); OzoneVolume vol = client.createVolume(volumeName, userName, "100TB"); assertNotNull(vol); } @@ -266,17 +266,16 @@ public class TestVolume { int count = 0; int pagecount = 0; while (count < volCount) { - List ovols = client.listAllVolumes(null, step, - prevKey); + List ovols = client.listAllVolumes(null, step, prevKey); count += ovols.size(); - if(ovols.size() > 0) { + if (ovols.size() > 0) { prevKey = ovols.get(ovols.size() - 1); } pagecount++; } // becasue we are querying an existing ozone store, there will // 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