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;
}
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;
}

View File

@ -142,18 +142,18 @@ public static class Builder {
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 Builder setCreationTime(long createdOn) {
return this;
}
public Builder setQuotaInBytes(long quotaInBytes) {
this.quotaInBytes = quotaInBytes;
public Builder setQuotaInBytes(long quota) {
this.quotaInBytes = quota;
return this;
}

View File

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

View File

@ -20,7 +20,6 @@
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 ContainerCommandResponseProto sendCommand(
* @throws IOException
*/
@Override
public CompletableFuture<ContainerCommandResponseProto>
sendCommandAsync(ContainerCommandRequestProto request)
public CompletableFuture<ContainerCommandResponseProto> sendCommandAsync(
ContainerCommandRequestProto request)
throws IOException, ExecutionException, InterruptedException {
throw new IOException("Not implemented");
}

View File

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

View File

@ -82,13 +82,14 @@ public static ContainerInfo fromProtobuf(
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;
}

View File

@ -188,40 +188,33 @@ public static void writeChunk(XceiverClientSpi xceiverClient, ChunkInfo chunk,
* @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()
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)
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);
}

View File

@ -26,10 +26,13 @@
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<String, Boolean> testMatrix;
testMatrix = new HashMap<>();
@ -103,7 +106,7 @@ public void TestACLParse() {
}
@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);

View File

@ -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 {

View File

@ -23,9 +23,7 @@
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 static KeyData getKeyData(byte[] bytes) throws IOException {
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);
}
}
}

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,
String name) {
Map<String, BlockContainerInfo> containersByState = this.containers.get(state);
Map<String, BlockContainerInfo> containersByState =
this.containers.get(state);
return containersByState.get(name);
}
@ -325,8 +326,8 @@ private void refreshContainers() {
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) {
@ -356,8 +357,7 @@ private void refreshContainers() {
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 AllocatedBlock allocateBlock(final long size) throws IOException {
List<String> 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 AllocatedBlock allocateBlock(final long size) throws IOException {
}
// 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 AllocatedBlock allocateBlock(final long size) throws IOException {
// 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,11 +429,11 @@ public AllocatedBlock allocateBlock(final long size) throws IOException {
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.LifeCycleState newState = containerManager
.updateContainerState(containerName,
OzoneProtos.LifeCycleEvent.BEGIN_CREATE);
updateContainer(containerInfo.getState(), containerName, newState);
}
@ -457,15 +458,17 @@ public Pipeline getBlock(final String key) throws IOException {
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);
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);
}

View File

@ -158,15 +158,17 @@ private void initializeStateMachine() {
* {@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 OzoneProtos.LifeCycleState updateContainerState(String containerName,
lock.lock();
try {
byte[] dbKey = containerName.getBytes(encoding);
byte[] containerBytes =
containerStore.get(dbKey);
byte[] containerBytes = containerStore.get(dbKey);
if (containerBytes == null) {
throw new SCMException("Failed to update container state"
+ containerName + ", reason : container doesn't exist.",
throw new SCMException(
"Failed to update container state" + containerName
+ ", reason : container doesn't exist.",
SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
}
containerInfo = ContainerInfo.fromProtobuf(
@ -319,8 +321,9 @@ public OzoneProtos.LifeCycleState updateContainerState(String containerName,
try {
newState = stateMachine.getNextState(containerInfo.getState(), event);
} catch (InvalidStateTransitionException ex) {
throw new SCMException("Failed to update container state"
+ containerName + ", reason : invalid state transition from state: "
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);
}

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
@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 @@ Response putKey(@PathParam("volume") String volume,
*/
@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 @@ Response getKey(@PathParam("volume") String volume,
*/
@DELETE
Response deleteKey(@PathParam("volume") String volume,
@PathParam("bucket") String bucket,
@PathParam("keys") String keys,
@Context Request req,
@Context UriInfo info,
@Context HttpHeaders headers)
@PathParam("bucket") String bucket, @PathParam("keys") String keys,
@Context Request req, @Context UriInfo info, @Context HttpHeaders headers)
throws OzoneException;
}

View File

@ -67,11 +67,8 @@ public interface Volume {
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;
@Context Request req, @Context UriInfo uriInfo,
@Context HttpHeaders headers) throws OzoneException;
/**
* Updates a Volume owned by the user.
@ -95,11 +92,8 @@ Response createVolume(@PathParam("volume") String volume,
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;
@Context Request req, @Context UriInfo uriInfo,
@Context HttpHeaders headers) throws OzoneException;
/**
* Deletes a Volume if it is empty.
@ -112,10 +106,8 @@ Response updateVolume(@PathParam("volume") String 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
@ -132,20 +124,13 @@ Response deleteVolume(@PathParam("volume") String 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,
@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;
@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;
}

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
*/
public CloseableCleanupListener(Closeable... closeables) {
CloseableCleanupListener(Closeable... 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 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;

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.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;

View File

@ -20,9 +20,7 @@
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;

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) {
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);
}
}

View File

@ -306,18 +306,17 @@ private void heartbeatTaskHelper(InetSocketAddress scmAddress,
// 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)) {
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();
.setDatanodeID(getDatanodeID().getProtoBufMessage()).build();
rpcEndPoint.setState(EndpointStateMachine.EndPointStates.HEARTBEAT);
final StateContext stateContext = new StateContext(conf,
DatanodeStateMachine.DatanodeStates.RUNNING,
final StateContext stateContext =
new StateContext(conf, DatanodeStateMachine.DatanodeStates.RUNNING,
stateMachine);
HeartbeatEndpointTask endpointTask =
@ -360,8 +359,8 @@ public void testHeartbeatTaskRpcTimeOut() throws Exception {
* @return
*/
ContainerReport getRandomContainerReport() {
return new ContainerReport(UUID.randomUUID().toString()
,DigestUtils.sha256Hex("Random"));
return new ContainerReport(UUID.randomUUID().toString(),
DigestUtils.sha256Hex("Random"));
}
/**

View File

@ -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 void setupPaths() throws IOException {
@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

View File

@ -29,7 +29,6 @@
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;

View File

@ -523,14 +523,14 @@ public void testKeyOverwrite() throws IOException, OzoneException {
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());
}
/**

View File

@ -44,6 +44,10 @@
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);

View File

@ -31,7 +31,7 @@
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 void TestBucketInfoJson() throws IOException {
}
@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 void TestBucketInfoDBString() throws IOException {
}
@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 void TestBucketInfoAddAcls() throws IOException {
@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);

View File

@ -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 void TestErrorGen() {
}
@Test
public void TestErrorGenWithException() {
public void testErrorGenWithException() {
OzoneException e =
new OzoneException(ErrorTable.ACCESS_DENIED.getHttpCode(),
"short message", new Exception("Hello"));

View File

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

View File

@ -32,7 +32,7 @@
public class TestQuota {
@Test
public void TestParseQuota() {
public void testParseQuota() {
HashMap<String, Boolean> testMatrix;
testMatrix = new HashMap<String, Boolean>();
@ -60,7 +60,6 @@ public void TestParseQuota() {
testMatrix.put("BYTESbytes", Boolean.FALSE);
testMatrix.put("bytes", Boolean.FALSE);
Set<String> keys = testMatrix.keySet();
for (String key : keys) {
if (testMatrix.get(key)) {
@ -79,7 +78,7 @@ public void TestParseQuota() {
}
@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 void TestVerifyQuota() {
}
@Test
public void TestVerifyRemove() {
public void testVerifyRemove() {
assertTrue(OzoneQuota.isRemove("remove"));
assertFalse(OzoneQuota.isRemove("not remove"));
assertFalse(OzoneQuota.isRemove(null));

View File

@ -35,7 +35,7 @@ public class TestUtils {
* Tests if the bucket name handling is correct.
*/
@Test
public void TestValidBucketNames() {
public void testValidBucketNames() {
HashMap<String, Boolean> testMatrix;
// Init the Table with Strings and Expected Return values
testMatrix = new HashMap<String, Boolean>();
@ -47,9 +47,8 @@ public void TestValidBucketNames() {
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);
@ -90,11 +89,10 @@ public void TestValidBucketNames() {
* that this test is good enough.
*/
@Test
public void TestRequestIDisRandom(){
public void testRequestIDisRandom() {
HashSet<String> set = new HashSet<>();
for (int i = 0; i < 1000; i++) {
assertTrue(set.add(getRequestID()));
}
}
}

View File

@ -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 void testVolumeInfoParse() throws IOException {
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,7 +56,6 @@ public void testVolumeInfoValue() throws IOException {
@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),
@ -69,5 +67,4 @@ public void testVolumeListParse() throws IOException {
ListVolumes newList = ListVolumes.parse(listString);
assertEquals(list.toJsonString(), newList.toJsonString());
}
}

View File

@ -44,6 +44,7 @@
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 static void init() throws IOException,
}
/**
* shutdown MiniDFSCluster
* shutdown MiniDFSCluster.
*/
@AfterClass
public static void shutdown() {

View File

@ -110,7 +110,7 @@ public static void init() throws Exception {
}
/**
* shutdown MiniDFSCluster
* shutdown MiniDFSCluster.
*/
@AfterClass
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.
*/
@ -165,9 +165,9 @@ static class PutHelper {
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());

View File

@ -90,7 +90,7 @@ public static void init() throws Exception {
}
/**
* shutdown MiniDFSCluster
* shutdown MiniDFSCluster.
*/
@AfterClass
public static void shutdown() {
@ -255,10 +255,10 @@ static void runTestListAllVolumes(OzoneRestClient client)
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,8 +266,7 @@ static void runTestListAllVolumes(OzoneRestClient client)
int count = 0;
int pagecount = 0;
while (count < volCount) {
List<OzoneVolume> ovols = client.listAllVolumes(null, step,
prevKey);
List<OzoneVolume> ovols = client.listAllVolumes(null, step, prevKey);
count += ovols.size();
if (ovols.size() > 0) {
prevKey = ovols.get(ovols.size() - 1);