HDDS-965. Ozone: checkstyle improvements and code quality scripts. Contributed by Elek, Marton.

This commit is contained in:
Márton Elek 2019-01-08 21:53:37 +01:00
parent 1a08302897
commit 32cf0412ce
94 changed files with 908 additions and 530 deletions

View File

@ -190,12 +190,6 @@ public final class XceiverClientRatis extends XceiverClientSpi {
return minIndex.isPresent() ? minIndex.getAsLong() : 0;
}
private void getFailedServer(
Collection<RaftProtos.CommitInfoProto> commitInfos) {
for (RaftProtos.CommitInfoProto proto : commitInfos) {
}
}
@Override
public long watchForCommit(long index, long timeout)
@ -217,7 +211,7 @@ public final class XceiverClientRatis extends XceiverClientSpi {
.sendWatchAsync(index, RaftProtos.ReplicationLevel.ALL_COMMITTED);
RaftClientReply reply;
try {
reply = replyFuture.get(timeout, TimeUnit.MILLISECONDS);
replyFuture.get(timeout, TimeUnit.MILLISECONDS);
} catch (TimeoutException toe) {
LOG.warn("3 way commit failed ", toe);

View File

@ -122,6 +122,7 @@ public class BlockOutputStream extends OutputStream {
* @param watchTimeout watch timeout
* @param checksum checksum
*/
@SuppressWarnings("parameternumber")
public BlockOutputStream(BlockID blockID, String key,
XceiverClientManager xceiverClientManager, XceiverClientSpi xceiverClient,
String traceID, int chunkSize, long streamBufferFlushSize,

View File

@ -18,22 +18,6 @@
package org.apache.hadoop.hdds;
import com.google.common.base.Strings;
import com.google.common.net.HostAndPort;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.metrics2.util.MBeans;
import org.apache.hadoop.net.DNS;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.management.ObjectName;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
@ -47,13 +31,26 @@ import java.util.Map;
import java.util.Optional;
import java.util.TimeZone;
import static org.apache.hadoop.hdfs.DFSConfigKeys
.DFS_DATANODE_DNS_INTERFACE_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys
.DFS_DATANODE_DNS_NAMESERVER_KEY;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.metrics2.util.MBeans;
import org.apache.hadoop.net.DNS;
import org.apache.hadoop.net.NetUtils;
import com.google.common.base.Strings;
import com.google.common.net.HostAndPort;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_INTERFACE_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_NAMESERVER_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ENABLED;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ENABLED_DEFAULT;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* HDDS specific stateless utility functions.

View File

@ -28,7 +28,7 @@ public class HddsVersionProvider implements IVersionProvider {
@Override
public String[] getVersion() throws Exception {
String[] result = new String[] {
HddsVersionInfo.getBuildVersion()
HddsVersionInfo.HDDS_VERSION_INFO.getBuildVersion()
};
return result;
}

View File

@ -139,6 +139,9 @@ public class HddsConfServlet extends HttpServlet {
}
}
/**
* Exception for signal bad content type.
*/
public static class BadFormatException extends Exception {
private static final long serialVersionUID = 1L;

View File

@ -81,6 +81,7 @@ public class ContainerInfo implements Comparator<ContainerInfo>,
@JsonIgnore
private byte[] data;
@SuppressWarnings("parameternumber")
ContainerInfo(
long containerID,
HddsProtos.LifeCycleState state,

View File

@ -18,7 +18,7 @@
package org.apache.hadoop.hdds.scm.pipeline;
import java.io.IOException;
import java.io.IOException;
/**
* Signals that a pipeline state is not recognized.

View File

@ -67,7 +67,6 @@ import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.stream.Collectors;
/**
* This class is the client-side translator to translate the requests made on

View File

@ -36,9 +36,13 @@ import java.util.Properties;
@InterfaceAudience.Public
@InterfaceStability.Stable
public class HddsVersionInfo {
private static final Logger LOG = LoggerFactory.getLogger(
HddsVersionInfo.class);
public static final HddsVersionInfo HDDS_VERSION_INFO =
new HddsVersionInfo("hdds");
private Properties info;
protected HddsVersionInfo(String component) {
@ -46,7 +50,8 @@ public class HddsVersionInfo {
String versionInfoFile = component + "-version-info.properties";
InputStream is = null;
try {
is = ThreadUtil.getResourceAsStream(HddsVersionInfo.class.getClassLoader(),
is = ThreadUtil.getResourceAsStream(
HddsVersionInfo.class.getClassLoader(),
versionInfoFile);
info.load(is);
} catch (IOException ex) {
@ -57,127 +62,56 @@ public class HddsVersionInfo {
}
}
protected String _getVersion() {
protected String getVersion() {
return info.getProperty("version", "Unknown");
}
protected String _getRevision() {
protected String getRevision() {
return info.getProperty("revision", "Unknown");
}
protected String _getBranch() {
protected String getBranch() {
return info.getProperty("branch", "Unknown");
}
protected String _getDate() {
protected String getDate() {
return info.getProperty("date", "Unknown");
}
protected String _getUser() {
protected String getUser() {
return info.getProperty("user", "Unknown");
}
protected String _getUrl() {
protected String getUrl() {
return info.getProperty("url", "Unknown");
}
protected String _getSrcChecksum() {
protected String getSrcChecksum() {
return info.getProperty("srcChecksum", "Unknown");
}
protected String _getBuildVersion(){
return _getVersion() +
" from " + _getRevision() +
" by " + _getUser() +
" source checksum " + _getSrcChecksum();
public String getBuildVersion() {
return HDDS_VERSION_INFO.getVersion() +
" from " + HDDS_VERSION_INFO.getRevision() +
" by " + getUser() +
" source checksum " + getSrcChecksum();
}
protected String _getProtocVersion() {
protected String getProtocVersion() {
return info.getProperty("protocVersion", "Unknown");
}
private static final HddsVersionInfo HDDS_VERSION_INFO =
new HddsVersionInfo("hdds");
/**
* Get the HDDS version.
* @return the Hdds version string, eg. "0.6.3-dev"
*/
public static String getVersion() {
return HDDS_VERSION_INFO._getVersion();
}
/**
* Get the Git commit hash of the repository when compiled.
* @return the commit hash, eg. "18f64065d5db6208daf50b02c1b5ed4ee3ce547a"
*/
public static String getRevision() {
return HDDS_VERSION_INFO._getRevision();
}
/**
* Get the branch on which this originated.
* @return The branch name, e.g. "trunk" or "branches/branch-0.20"
*/
public static String getBranch() {
return HDDS_VERSION_INFO._getBranch();
}
/**
* The date that HDDS was compiled.
* @return the compilation date in unix date format
*/
public static String getDate() {
return HDDS_VERSION_INFO._getDate();
}
/**
* The user that compiled HDDS.
* @return the username of the user
*/
public static String getUser() {
return HDDS_VERSION_INFO._getUser();
}
/**
* Get the URL for the HDDS repository.
* @return the URL of the Hdds repository
*/
public static String getUrl() {
return HDDS_VERSION_INFO._getUrl();
}
/**
* Get the checksum of the source files from which HDDS was built.
* @return the checksum of the source files
*/
public static String getSrcChecksum() {
return HDDS_VERSION_INFO._getSrcChecksum();
}
/**
* Returns the buildVersion which includes version,
* revision, user and date.
* @return the buildVersion
*/
public static String getBuildVersion(){
return HDDS_VERSION_INFO._getBuildVersion();
}
/**
* Returns the protoc version used for the build.
* @return the protoc version
*/
public static String getProtocVersion(){
return HDDS_VERSION_INFO._getProtocVersion();
}
public static void main(String[] args) {
System.out.println("Using HDDS " + getVersion());
System.out.println("Source code repository " + getUrl() + " -r " +
getRevision());
System.out.println("Compiled by " + getUser() + " on " + getDate());
System.out.println("Compiled with protoc " + getProtocVersion());
System.out.println("From source with checksum " + getSrcChecksum());
System.out.println("Using HDDS " + HDDS_VERSION_INFO.getVersion());
System.out.println(
"Source code repository " + HDDS_VERSION_INFO.getUrl() + " -r " +
HDDS_VERSION_INFO.getRevision());
System.out.println("Compiled by " + HDDS_VERSION_INFO.getUser() + " on "
+ HDDS_VERSION_INFO.getDate());
System.out.println(
"Compiled with protoc " + HDDS_VERSION_INFO.getProtocVersion());
System.out.println(
"From source with checksum " + HDDS_VERSION_INFO.getSrcChecksum());
LOG.debug("This command was run using " +
ClassUtil.findContainingJar(HddsVersionInfo.class));
}

View File

@ -18,10 +18,20 @@
package org.apache.hadoop.utils;
import com.google.common.annotations.VisibleForTesting;
import java.io.File;
import java.io.IOException;
import java.util.Optional;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import com.google.common.annotations.VisibleForTesting;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_LEVELDB;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_ROCKSDB;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_STORE_ROCKSDB_STATISTICS;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_STORE_ROCKSDB_STATISTICS_DEFAULT;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_STORE_ROCKSDB_STATISTICS_OFF;
import org.iq80.leveldb.Options;
import org.rocksdb.BlockBasedTableConfig;
import org.rocksdb.Statistics;
@ -29,22 +39,6 @@ import org.rocksdb.StatsLevel;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.IOException;
import java.util.Optional;
import java.util.function.Supplier;
import static org.apache.hadoop.ozone.OzoneConfigKeys
.OZONE_METADATA_STORE_IMPL_LEVELDB;
import static org.apache.hadoop.ozone.OzoneConfigKeys
.OZONE_METADATA_STORE_IMPL_ROCKSDB;
import static org.apache.hadoop.ozone.OzoneConfigKeys
.OZONE_METADATA_STORE_ROCKSDB_STATISTICS;
import static org.apache.hadoop.ozone.OzoneConfigKeys
.OZONE_METADATA_STORE_ROCKSDB_STATISTICS_DEFAULT;
import static org.apache.hadoop.ozone.OzoneConfigKeys
.OZONE_METADATA_STORE_ROCKSDB_STATISTICS_OFF;
/**
* Builder for metadata store.
*/

View File

@ -45,6 +45,7 @@ import org.apache.hadoop.ozone.container.keyvalue.TarContainerPacker;
* Dispatcher sends ContainerCommandRequests to Handler. Each Container Type
* should have an implementation for Handler.
*/
@SuppressWarnings("visibilitymodifier")
public abstract class Handler {
protected final Configuration conf;

View File

@ -86,7 +86,8 @@ public class VersionEndpointTask implements
Preconditions.checkNotNull(clusterId, "Reply from SCM: clusterId " +
"cannot be null");
// If version file does not exist create version file and also set scmId
// If version file does not exist
// create version file and also set scmId
for (Map.Entry<String, HddsVolume> entry : volumeMap.entrySet()) {
HddsVolume hddsVolume = entry.getValue();

View File

@ -26,7 +26,7 @@ import org.apache.hadoop.classification.InterfaceStability;
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class DispatcherContext {
public final class DispatcherContext {
/**
* Determines which stage of writeChunk a write chunk request is for.
*/
@ -82,8 +82,8 @@ public class DispatcherContext {
* @param stage WriteChunk Stage
* @return DispatcherContext.Builder
*/
public Builder setStage(WriteChunkStage stage) {
this.stage = stage;
public Builder setStage(WriteChunkStage writeChunkStage) {
this.stage = writeChunkStage;
return this;
}
@ -93,8 +93,8 @@ public class DispatcherContext {
* @param readFromTmpFile whether to read from tmp chunk file or not
* @return DispatcherContext.Builder
*/
public Builder setReadFromTmpFile(boolean readFromTmpFile) {
this.readFromTmpFile = readFromTmpFile;
public Builder setReadFromTmpFile(boolean setReadFromTmpFile) {
this.readFromTmpFile = setReadFromTmpFile;
return this;
}
@ -104,8 +104,8 @@ public class DispatcherContext {
* @param term current term
* @return DispatcherContext.Builder
*/
public Builder setTerm(long term) {
this.term = term;
public Builder setTerm(long currentTerm) {
this.term = currentTerm;
return this;
}
@ -115,8 +115,8 @@ public class DispatcherContext {
* @param logIndex log index
* @return DispatcherContext.Builder
*/
public Builder setLogIndex(long logIndex) {
this.logIndex = logIndex;
public Builder setLogIndex(long index) {
this.logIndex = index;
return this;
}

View File

@ -156,58 +156,19 @@ public final class XceiverServerRatis implements XceiverServerSpi {
final RaftProperties properties = new RaftProperties();
// Set rpc type
final String rpcType = conf.get(
OzoneConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_KEY,
OzoneConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_DEFAULT);
final RpcType rpc = SupportedRpcType.valueOfIgnoreCase(rpcType);
RaftConfigKeys.Rpc.setType(properties, rpc);
final RpcType rpc = setRpcType(conf, properties);
// set raft segment size
final int raftSegmentSize = (int)conf.getStorageSize(
OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_SIZE_KEY,
OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_SIZE_DEFAULT,
StorageUnit.BYTES);
RaftServerConfigKeys.Log.setSegmentSizeMax(properties,
SizeInBytes.valueOf(raftSegmentSize));
setRaftSegmentSize(conf, properties);
// set raft segment pre-allocated size
final int raftSegmentPreallocatedSize = (int) conf.getStorageSize(
OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_KEY,
OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT,
StorageUnit.BYTES);
int logAppenderQueueNumElements = conf.getInt(
OzoneConfigKeys.DFS_CONTAINER_RATIS_LOG_APPENDER_QUEUE_NUM_ELEMENTS,
OzoneConfigKeys
.DFS_CONTAINER_RATIS_LOG_APPENDER_QUEUE_NUM_ELEMENTS_DEFAULT);
final int logAppenderQueueByteLimit = (int) conf.getStorageSize(
OzoneConfigKeys.DFS_CONTAINER_RATIS_LOG_APPENDER_QUEUE_BYTE_LIMIT,
OzoneConfigKeys
.DFS_CONTAINER_RATIS_LOG_APPENDER_QUEUE_BYTE_LIMIT_DEFAULT,
StorageUnit.BYTES);
RaftServerConfigKeys.Log.Appender
.setBufferElementLimit(properties, logAppenderQueueNumElements);
RaftServerConfigKeys.Log.Appender.setBufferByteLimit(properties,
SizeInBytes.valueOf(logAppenderQueueByteLimit));
RaftServerConfigKeys.Log.setPreallocatedSize(properties,
SizeInBytes.valueOf(raftSegmentPreallocatedSize));
final int raftSegmentPreallocatedSize =
setRaftSegmentPreallocatedSize(conf, properties);
// Set max write buffer size, which is the scm chunk size
final int maxChunkSize = OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE;
RaftServerConfigKeys.Log.setWriteBufferSize(properties,
SizeInBytes.valueOf(maxChunkSize));
// Set the client requestTimeout
TimeUnit timeUnit =
OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_DEFAULT
.getUnit();
long duration = conf.getTimeDuration(
OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_KEY,
OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_DEFAULT
.getDuration(), timeUnit);
final TimeDuration clientRequestTimeout =
TimeDuration.valueOf(duration, timeUnit);
RaftClientConfigKeys.Rpc
.setRequestTimeout(properties, clientRequestTimeout);
final int maxChunkSize = setMaxWriteBuffer(conf, properties);
TimeUnit timeUnit;
long duration;
// set the configs enable and set the stateMachineData sync timeout
RaftServerConfigKeys.Log.StateMachineData.setSync(properties, true);
@ -224,66 +185,19 @@ public final class XceiverServerRatis implements XceiverServerSpi {
.setSyncTimeout(properties, dataSyncTimeout);
// Set the server Request timeout
timeUnit = OzoneConfigKeys.DFS_RATIS_SERVER_REQUEST_TIMEOUT_DURATION_DEFAULT
.getUnit();
duration = conf.getTimeDuration(
OzoneConfigKeys.DFS_RATIS_SERVER_REQUEST_TIMEOUT_DURATION_KEY,
OzoneConfigKeys.DFS_RATIS_SERVER_REQUEST_TIMEOUT_DURATION_DEFAULT
.getDuration(), timeUnit);
final TimeDuration serverRequestTimeout =
TimeDuration.valueOf(duration, timeUnit);
RaftServerConfigKeys.Rpc
.setRequestTimeout(properties, serverRequestTimeout);
setServerRequestTimeout(conf, properties);
// set timeout for a retry cache entry
timeUnit =
OzoneConfigKeys.DFS_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DURATION_DEFAULT
.getUnit();
duration = conf.getTimeDuration(
OzoneConfigKeys.DFS_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DURATION_KEY,
OzoneConfigKeys.DFS_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DURATION_DEFAULT
.getDuration(), timeUnit);
final TimeDuration retryCacheTimeout =
TimeDuration.valueOf(duration, timeUnit);
RaftServerConfigKeys.RetryCache
.setExpiryTime(properties, retryCacheTimeout);
setTimeoutForRetryCache(conf, properties);
// Set the ratis leader election timeout
TimeUnit leaderElectionMinTimeoutUnit =
OzoneConfigKeys.
DFS_RATIS_LEADER_ELECTION_MINIMUM_TIMEOUT_DURATION_DEFAULT
.getUnit();
duration = conf.getTimeDuration(
OzoneConfigKeys.DFS_RATIS_LEADER_ELECTION_MINIMUM_TIMEOUT_DURATION_KEY,
OzoneConfigKeys.
DFS_RATIS_LEADER_ELECTION_MINIMUM_TIMEOUT_DURATION_DEFAULT
.getDuration(), leaderElectionMinTimeoutUnit);
final TimeDuration leaderElectionMinTimeout =
TimeDuration.valueOf(duration, leaderElectionMinTimeoutUnit);
RaftServerConfigKeys.Rpc
.setTimeoutMin(properties, leaderElectionMinTimeout);
long leaderElectionMaxTimeout =
leaderElectionMinTimeout.toLong(TimeUnit.MILLISECONDS) + 200;
RaftServerConfigKeys.Rpc.setTimeoutMax(properties,
TimeDuration.valueOf(leaderElectionMaxTimeout, TimeUnit.MILLISECONDS));
setRatisLeaderElectionTimeout(conf, properties);
// Set the maximum cache segments
RaftServerConfigKeys.Log.setMaxCachedSegmentNum(properties, 2);
// set the node failure timeout
timeUnit = OzoneConfigKeys.DFS_RATIS_SERVER_FAILURE_DURATION_DEFAULT
.getUnit();
duration = conf.getTimeDuration(
OzoneConfigKeys.DFS_RATIS_SERVER_FAILURE_DURATION_KEY,
OzoneConfigKeys.DFS_RATIS_SERVER_FAILURE_DURATION_DEFAULT
.getDuration(), timeUnit);
final TimeDuration nodeFailureTimeout =
TimeDuration.valueOf(duration, timeUnit);
RaftServerConfigKeys.setLeaderElectionTimeout(properties,
nodeFailureTimeout);
RaftServerConfigKeys.Rpc.setSlownessTimeout(properties,
nodeFailureTimeout);
nodeFailureTimeoutMs = nodeFailureTimeout.toLong(TimeUnit.MILLISECONDS);
setNodeFailureTimeout(conf, properties);
// Set the ratis storage directory
String storageDir = HddsServerUtil.getOzoneDatanodeRatisDirectory(conf);
@ -331,6 +245,143 @@ public final class XceiverServerRatis implements XceiverServerSpi {
return properties;
}
private void setNodeFailureTimeout(Configuration conf,
RaftProperties properties) {
TimeUnit timeUnit;
long duration;
timeUnit = OzoneConfigKeys.DFS_RATIS_SERVER_FAILURE_DURATION_DEFAULT
.getUnit();
duration = conf.getTimeDuration(
OzoneConfigKeys.DFS_RATIS_SERVER_FAILURE_DURATION_KEY,
OzoneConfigKeys.DFS_RATIS_SERVER_FAILURE_DURATION_DEFAULT
.getDuration(), timeUnit);
final TimeDuration nodeFailureTimeout =
TimeDuration.valueOf(duration, timeUnit);
RaftServerConfigKeys.setLeaderElectionTimeout(properties,
nodeFailureTimeout);
RaftServerConfigKeys.Rpc.setSlownessTimeout(properties,
nodeFailureTimeout);
nodeFailureTimeoutMs = nodeFailureTimeout.toLong(TimeUnit.MILLISECONDS);
}
private void setRatisLeaderElectionTimeout(Configuration conf,
RaftProperties properties) {
long duration;
TimeUnit leaderElectionMinTimeoutUnit =
OzoneConfigKeys.
DFS_RATIS_LEADER_ELECTION_MINIMUM_TIMEOUT_DURATION_DEFAULT
.getUnit();
duration = conf.getTimeDuration(
OzoneConfigKeys.DFS_RATIS_LEADER_ELECTION_MINIMUM_TIMEOUT_DURATION_KEY,
OzoneConfigKeys.
DFS_RATIS_LEADER_ELECTION_MINIMUM_TIMEOUT_DURATION_DEFAULT
.getDuration(), leaderElectionMinTimeoutUnit);
final TimeDuration leaderElectionMinTimeout =
TimeDuration.valueOf(duration, leaderElectionMinTimeoutUnit);
RaftServerConfigKeys.Rpc
.setTimeoutMin(properties, leaderElectionMinTimeout);
long leaderElectionMaxTimeout =
leaderElectionMinTimeout.toLong(TimeUnit.MILLISECONDS) + 200;
RaftServerConfigKeys.Rpc.setTimeoutMax(properties,
TimeDuration.valueOf(leaderElectionMaxTimeout, TimeUnit.MILLISECONDS));
}
private void setTimeoutForRetryCache(Configuration conf,
RaftProperties properties) {
TimeUnit timeUnit;
long duration;
timeUnit =
OzoneConfigKeys.DFS_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DURATION_DEFAULT
.getUnit();
duration = conf.getTimeDuration(
OzoneConfigKeys.DFS_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DURATION_KEY,
OzoneConfigKeys.DFS_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DURATION_DEFAULT
.getDuration(), timeUnit);
final TimeDuration retryCacheTimeout =
TimeDuration.valueOf(duration, timeUnit);
RaftServerConfigKeys.RetryCache
.setExpiryTime(properties, retryCacheTimeout);
}
private void setServerRequestTimeout(Configuration conf,
RaftProperties properties) {
TimeUnit timeUnit;
long duration;
timeUnit = OzoneConfigKeys.DFS_RATIS_SERVER_REQUEST_TIMEOUT_DURATION_DEFAULT
.getUnit();
duration = conf.getTimeDuration(
OzoneConfigKeys.DFS_RATIS_SERVER_REQUEST_TIMEOUT_DURATION_KEY,
OzoneConfigKeys.DFS_RATIS_SERVER_REQUEST_TIMEOUT_DURATION_DEFAULT
.getDuration(), timeUnit);
final TimeDuration serverRequestTimeout =
TimeDuration.valueOf(duration, timeUnit);
RaftServerConfigKeys.Rpc
.setRequestTimeout(properties, serverRequestTimeout);
}
private int setMaxWriteBuffer(Configuration conf, RaftProperties properties) {
final int maxChunkSize = OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE;
RaftServerConfigKeys.Log.setWriteBufferSize(properties,
SizeInBytes.valueOf(maxChunkSize));
// Set the client requestTimeout
TimeUnit timeUnit =
OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_DEFAULT
.getUnit();
long duration = conf.getTimeDuration(
OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_KEY,
OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_DEFAULT
.getDuration(), timeUnit);
final TimeDuration clientRequestTimeout =
TimeDuration.valueOf(duration, timeUnit);
RaftClientConfigKeys.Rpc
.setRequestTimeout(properties, clientRequestTimeout);
return maxChunkSize;
}
private int setRaftSegmentPreallocatedSize(Configuration conf,
RaftProperties properties) {
final int raftSegmentPreallocatedSize = (int) conf.getStorageSize(
OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_KEY,
OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT,
StorageUnit.BYTES);
int logAppenderQueueNumElements = conf.getInt(
OzoneConfigKeys.DFS_CONTAINER_RATIS_LOG_APPENDER_QUEUE_NUM_ELEMENTS,
OzoneConfigKeys
.DFS_CONTAINER_RATIS_LOG_APPENDER_QUEUE_NUM_ELEMENTS_DEFAULT);
final int logAppenderQueueByteLimit = (int) conf.getStorageSize(
OzoneConfigKeys.DFS_CONTAINER_RATIS_LOG_APPENDER_QUEUE_BYTE_LIMIT,
OzoneConfigKeys
.DFS_CONTAINER_RATIS_LOG_APPENDER_QUEUE_BYTE_LIMIT_DEFAULT,
StorageUnit.BYTES);
RaftServerConfigKeys.Log.Appender
.setBufferElementLimit(properties, logAppenderQueueNumElements);
RaftServerConfigKeys.Log.Appender.setBufferByteLimit(properties,
SizeInBytes.valueOf(logAppenderQueueByteLimit));
RaftServerConfigKeys.Log.setPreallocatedSize(properties,
SizeInBytes.valueOf(raftSegmentPreallocatedSize));
return raftSegmentPreallocatedSize;
}
private void setRaftSegmentSize(Configuration conf,
RaftProperties properties) {
final int raftSegmentSize = (int)conf.getStorageSize(
OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_SIZE_KEY,
OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_SIZE_DEFAULT,
StorageUnit.BYTES);
RaftServerConfigKeys.Log.setSegmentSizeMax(properties,
SizeInBytes.valueOf(raftSegmentSize));
}
private RpcType setRpcType(Configuration conf, RaftProperties properties) {
final String rpcType = conf.get(
OzoneConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_KEY,
OzoneConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_DEFAULT);
final RpcType rpc = SupportedRpcType.valueOfIgnoreCase(rpcType);
RaftConfigKeys.Rpc.setType(properties, rpc);
return rpc;
}
public static XceiverServerRatis newXceiverServerRatis(
DatanodeDetails datanodeDetails, Configuration ozoneConf,
ContainerDispatcher dispatcher, StateContext context) throws IOException {

View File

@ -31,7 +31,7 @@ import java.io.IOException;
/**
* Stores information about a disk/volume.
*/
public class VolumeInfo {
public final class VolumeInfo {
private static final Logger LOG = LoggerFactory.getLogger(VolumeInfo.class);

View File

@ -41,10 +41,12 @@ public final class KeyValueContainerLocationUtil {
* @return containerMetadata Path to container metadata location where
* .container file will be stored.
*/
public static File getContainerMetaDataPath(String hddsVolumeDir, String scmId,
public static File getContainerMetaDataPath(String hddsVolumeDir,
String scmId,
long containerId) {
String containerMetaDataPath = getBaseContainerLocation(hddsVolumeDir, scmId,
containerId);
String containerMetaDataPath =
getBaseContainerLocation(hddsVolumeDir, scmId,
containerId);
containerMetaDataPath = containerMetaDataPath + File.separator +
OzoneConsts.CONTAINER_META_PATH;
return new File(containerMetaDataPath);
@ -72,8 +74,9 @@ public final class KeyValueContainerLocationUtil {
* @param containerId
* @return base directory for container.
*/
private static String getBaseContainerLocation(String hddsVolumeDir, String scmId,
long containerId) {
private static String getBaseContainerLocation(String hddsVolumeDir,
String scmId,
long containerId) {
Preconditions.checkNotNull(hddsVolumeDir, "Base Directory cannot be null");
Preconditions.checkNotNull(scmId, "scmUuid cannot be null");
Preconditions.checkState(containerId >= 0,

View File

@ -57,10 +57,10 @@ public class GrpcReplicationService extends
LOG.info("Streaming container data ({}) to other datanode",
request.getContainerID());
try {
GrpcOutputStream outputStream =
new GrpcOutputStream(responseObserver, request.getContainerID());
containerReplicationSource
.copyData(request.getContainerID(), outputStream);
GrpcOutputStream outputStream =
new GrpcOutputStream(responseObserver, request.getContainerID());
containerReplicationSource
.copyData(request.getContainerID(), outputStream);
} catch (IOException e) {
LOG.error("Can't stream the container data", e);

View File

@ -30,7 +30,6 @@ import java.util.concurrent.TimeUnit;
import java.util.function.Function;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.DatanodeDetails.Port.Name;
import org.apache.hadoop.ozone.OzoneConfigKeys;

View File

@ -24,13 +24,17 @@ import org.apache.hadoop.hdds.protocol.proto
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto;
/**
* Command status to report about block deletion.
*/
public class DeleteBlockCommandStatus extends CommandStatus {
private ContainerBlocksDeletionACKProto blocksDeletionAck = null;
public DeleteBlockCommandStatus(Type type, Long cmdId,
StorageContainerDatanodeProtocolProtos.CommandStatus.Status status,
String msg, ContainerBlocksDeletionACKProto blocksDeletionAck) {
String msg,
ContainerBlocksDeletionACKProto blocksDeletionAck) {
super(type, cmdId, status, msg);
this.blocksDeletionAck = blocksDeletionAck;
}
@ -53,7 +57,8 @@ public class DeleteBlockCommandStatus extends CommandStatus {
}
@Override
public StorageContainerDatanodeProtocolProtos.CommandStatus getProtoBufMessage() {
public StorageContainerDatanodeProtocolProtos.CommandStatus
getProtoBufMessage() {
StorageContainerDatanodeProtocolProtos.CommandStatus.Builder builder =
StorageContainerDatanodeProtocolProtos.CommandStatus.newBuilder()
.setCmdId(this.getCmdId())
@ -68,6 +73,9 @@ public class DeleteBlockCommandStatus extends CommandStatus {
return builder.build();
}
/**
* Builder for DeleteBlockCommandStatus.
*/
public static final class DeleteBlockCommandStatusBuilder
extends CommandStatusBuilder {
private ContainerBlocksDeletionACKProto blocksDeletionAck = null;

View File

@ -16,28 +16,24 @@
*/
package org.apache.hadoop.ozone.container.common;
import com.google.protobuf.BlockingService;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.HddsConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos
.StorageContainerDatanodeProtocolService;
import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.protocol.StorageContainerDatanodeProtocol;
import org.apache.hadoop.ozone.protocolPB.StorageContainerDatanodeProtocolPB;
import org.apache.hadoop.ozone.protocolPB
.StorageContainerDatanodeProtocolServerSideTranslatorPB;
import org.apache.hadoop.test.GenericTestUtils;
import java.io.IOException;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.ServerSocket;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.HddsConfigKeys;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.StorageContainerDatanodeProtocolService;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ozone.protocol.StorageContainerDatanodeProtocol;
import org.apache.hadoop.ozone.protocolPB.StorageContainerDatanodeProtocolPB;
import org.apache.hadoop.ozone.protocolPB.StorageContainerDatanodeProtocolServerSideTranslatorPB;
import org.apache.hadoop.test.GenericTestUtils;
import com.google.protobuf.BlockingService;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.HDDS_DATANODE_DIR_KEY;
/**

View File

@ -94,8 +94,8 @@ public class TestHandler {
Assert.assertEquals("New ContainerType detected. Not an invalid " +
"containerType", invalidContainerType, null);
Handler handler = dispatcher.getHandler(invalidContainerType);
Handler dispatcherHandler = dispatcher.getHandler(invalidContainerType);
Assert.assertEquals("Get Handler for Invalid ContainerType should " +
"return null.", handler, null);
"return null.", dispatcherHandler, null);
}
}

View File

@ -66,10 +66,10 @@ public abstract class EventWatcher<TIMEOUT_PAYLOAD extends
private final String name;
protected final Map<Long, TIMEOUT_PAYLOAD> trackedEventsByID =
private final Map<Long, TIMEOUT_PAYLOAD> trackedEventsByID =
new ConcurrentHashMap<>();
protected final Set<TIMEOUT_PAYLOAD> trackedEvents = new HashSet<>();
private final Set<TIMEOUT_PAYLOAD> trackedEvents = new HashSet<>();
private final Map<Long, Long> startTrackingTimes = new HashedMap();
@ -206,4 +206,12 @@ public abstract class EventWatcher<TIMEOUT_PAYLOAD extends
public TIMEOUT_PAYLOAD getTrackedEventbyId(long id) {
return trackedEventsByID.get(id);
}
public Map<Long, TIMEOUT_PAYLOAD> getTrackedEventsByID() {
return trackedEventsByID;
}
public Set<TIMEOUT_PAYLOAD> getTrackedEvents() {
return trackedEvents;
}
}

View File

@ -20,6 +20,9 @@ package org.apache.hadoop.hdds.scm.block;
import org.apache.hadoop.hdds.server.events.EventHandler;
import org.apache.hadoop.hdds.server.events.EventPublisher;
/**
* Event handler for PedingDeleteStatuList events.
*/
public class PendingDeleteHandler implements
EventHandler<PendingDeleteStatusList> {

View File

@ -211,9 +211,7 @@ public class ReplicationManager implements Runnable {
.add(replica.getDatanodeDetails());
});
for(UUID originId : originIdToDnMap.keySet()) {
final List<DatanodeDetails> listOfReplica =
originIdToDnMap.get(originId);
for (List<DatanodeDetails> listOfReplica : originIdToDnMap.values()) {
if (listOfReplica.size() > 1) {
final int toDelete = Math.min(listOfReplica.size() - 1,
numberOfReplicasToDelete);

View File

@ -38,6 +38,8 @@ import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes.DUP
* This information is built from the DN container reports.
*/
public class Node2ObjectsMap<T> {
@SuppressWarnings("visibilitymodifier")
protected final Map<UUID, Set<T>> dn2ObjectMap;
/**

View File

@ -27,10 +27,12 @@ import java.util.Set;
import java.util.UUID;
/**
* This data structure maintains the list of pipelines which the given datanode is a part of. This
* information will be added whenever a new pipeline allocation happens.
* This data structure maintains the list of pipelines which the given
* datanode is a part of. This information will be added whenever a new
* pipeline allocation happens.
*
* <p>TODO: this information needs to be regenerated from pipeline reports on SCM restart
* <p>TODO: this information needs to be regenerated from pipeline reports
* on SCM restart
*/
public class Node2PipelineMap extends Node2ObjectsMap<PipelineID> {

View File

@ -400,6 +400,7 @@ public final class TestUtils {
*
* @return ContainerInfo
*/
@SuppressWarnings("parameternumber")
public static ContainerReplicaProto createContainerInfo(
long containerId, long size, long keyCount, long bytesUsed,
long readCount, long readBytes, long writeCount, long writeBytes) {

View File

@ -45,6 +45,9 @@ import java.util.List;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.assertFalse;
/**
* Unit test for command status report handler.
*/
public class TestCommandStatusReportHandler implements EventPublisher {
private static final Logger LOG = LoggerFactory
@ -67,7 +70,6 @@ public class TestCommandStatusReportHandler implements EventPublisher {
assertFalse(logCapturer.getOutput().contains("Delete_Block_Status"));
assertFalse(logCapturer.getOutput().contains("Replicate_Command_Status"));
report = this.getStatusReport(this.getCommandStatusList());
cmdStatusReportHandler.onMessage(report, this);
assertTrue(logCapturer.getOutput().contains("firing event of type " +
@ -92,8 +94,8 @@ public class TestCommandStatusReportHandler implements EventPublisher {
}
@Override
public <PAYLOAD, EVENT_TYPE extends Event<PAYLOAD>> void fireEvent
(EVENT_TYPE event, PAYLOAD payload) {
public <PAYLOAD, EVENT_TYPE extends Event<PAYLOAD>> void
fireEvent(EVENT_TYPE event, PAYLOAD payload) {
LOG.info("firing event of type {}, payload {}", event.getName(), payload
.toString());
}

View File

@ -212,8 +212,8 @@ public class TestSCMContainerManager {
HddsProtos.LifeCycleEvent.FINALIZE);
containerManager.updateContainerState(id,
HddsProtos.LifeCycleEvent.CLOSE);
ContainerInfo closedContainer = containerManager.getContainer(id);
Assert.assertEquals(LifeCycleState.CLOSED, closedContainer.getState());
ContainerInfo closedContainer = containerManager.getContainer(id);
Assert.assertEquals(LifeCycleState.CLOSED, closedContainer.getState());
}
/**

View File

@ -36,6 +36,9 @@ import static org.mockito.Matchers.anyObject;
import org.mockito.Mockito;
import static org.mockito.Mockito.when;
/**
* Test for the scm container placement.
*/
public class TestSCMContainerPlacementCapacity {
@Test
public void chooseDatanodes() throws SCMException {

View File

@ -34,6 +34,9 @@ import static org.mockito.Matchers.anyObject;
import org.mockito.Mockito;
import static org.mockito.Mockito.when;
/**
* Test for the random container placement.
*/
public class TestSCMContainerPlacementRandom {
@Test

View File

@ -259,11 +259,11 @@ public class TestDeadNodeHandler {
"DeadNode event for a unregistered node"));
}
private void registerReplicas(ContainerManager containerManager,
private void registerReplicas(ContainerManager contManager,
ContainerInfo container, DatanodeDetails... datanodes)
throws ContainerNotFoundException {
for (DatanodeDetails datanode : datanodes) {
containerManager.updateContainerReplica(
contManager.updateContainerReplica(
new ContainerID(container.getContainerID()),
ContainerReplica.newBuilder()
.setContainerState(ContainerReplicaProto.State.OPEN)

View File

@ -35,6 +35,9 @@ import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Test for the Node Report Handler.
*/
public class TestNodeReportHandler implements EventPublisher {
private static final Logger LOG = LoggerFactory

105
hadoop-ozone/Jenkinsfile vendored Normal file
View File

@ -0,0 +1,105 @@
/**
* 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.
*/
node("ubuntu") {
docker.image('elek/ozone-build').pull()
docker.image('elek/ozone-build').inside {
stage('Checkout') {
checkout scm
}
stage('Clean') {
status = sh returnStatus: true, script: 'mvn clean'
}
stageRunner('Author', "author", {})
stageRunner('Isolation', "isolation", {})
stageRunner('Build', "build", {})
stageRunner('Licence', "rat", {
archiveArtifacts 'target/rat-aggregated.txt'
}, 'artifact/target/rat-aggregated.txt/*view*/')
stageRunner('Unit test', "unit", {
junit '**/target/surefire-reports/*.xml'
}, 'testReport/')
stageRunner('Findbugs', "findbugs", {
archiveArtifacts 'target/findbugs-all.txt'
}, 'artifact/target/findbugs-all.txt/*view*/')
stageRunner('Checkstyle', "checkstyle", {
checkstyle canComputeNew: false, canRunOnFailed: true, defaultEncoding: '', healthy: '', pattern: '**/checkstyle-result.xml', unHealthy: ''
}, 'checkstyleResult')
}
}
def stageRunner(name, type, processResult, url = '') {
try {
stage(name) {
prStatusStart(type)
status = sh returnStatus: true, script: 'hadoop-ozone/dev-support/checks/' + type + '.sh'
processResult()
prStatusResult(status, type, url)
}
return true
} catch (RuntimeException ex) {
currentBuild.result = "FAILED"
return false
}
}
def prStatusStart(name) {
if (env.CHANGE_ID) {
pullRequest.createStatus(status: "pending",
context: 'continuous-integration/jenkins/pr-merge/' + name,
description: name + " is started")
}
}
def prStatusResult(responseCode, name, url = '') {
status = "error"
desc = "failed"
if (responseCode == 0) {
status = "success"
desc = "passed"
}
message = name + " is " + desc
//System.out.println(responseCode)
if (env.CHANGE_ID) {
if (url) {
pullRequest.createStatus(status: status,
context: 'continuous-integration/jenkins/pr-merge/' + name,
description: message,
targetUrl: env.BUILD_URL + url)
} else {
pullRequest.createStatus(status: status,
context: 'continuous-integration/jenkins/pr-merge/' + name,
description: message)
}
}
if (responseCode != 0) {
throw new RuntimeException(message)
}
}

View File

@ -103,6 +103,7 @@ public class OzoneBucket {
* @param versioning versioning status of the bucket.
* @param creationTime creation time of the bucket.
*/
@SuppressWarnings("parameternumber")
public OzoneBucket(Configuration conf, ClientProtocol proxy,
String volumeName, String bucketName,
List<OzoneAcl> acls, StorageType storageType,
@ -125,6 +126,7 @@ public class OzoneBucket {
}
@VisibleForTesting
@SuppressWarnings("parameternumber")
OzoneBucket(String volumeName, String name,
ReplicationFactor defaultReplication,
ReplicationType defaultReplicationType,

View File

@ -35,6 +35,7 @@ public class OzoneKeyDetails extends OzoneKey {
/**
* Constructs OzoneKeyDetails from OmKeyInfo.
*/
@SuppressWarnings("parameternumber")
public OzoneKeyDetails(String volumeName, String bucketName, String keyName,
long size, long creationTime, long modificationTime,
List<OzoneKeyLocation> ozoneKeyLocations,

View File

@ -82,6 +82,7 @@ public class OzoneVolume {
* @param creationTime creation time of the volume
* @param acls ACLs associated with the volume.
*/
@SuppressWarnings("parameternumber")
public OzoneVolume(Configuration conf, ClientProtocol proxy, String name,
String admin, String owner, long quotaInBytes,
long creationTime, List<OzoneAcl> acls) {
@ -265,12 +266,13 @@ public class OzoneVolume {
/**
* Creates an Iterator to iterate over all buckets after prevBucket in the volume.
* Creates an Iterator to iterate over all buckets after prevBucket in
* the volume.
* If prevBucket is null it iterates from the first bucket in the volume.
* The returned buckets match bucket prefix.
* @param bucketPrefix
*/
public BucketIterator(String bucketPrefix, String prevBucket) {
BucketIterator(String bucketPrefix, String prevBucket) {
this.bucketPrefix = bucketPrefix;
this.currentValue = null;
this.currentIterator = getNextListOfBuckets(prevBucket).iterator();

View File

@ -136,7 +136,7 @@ public class ChunkGroupInputStream extends InputStream implements Seekable {
off += numBytesRead;
len -= numBytesRead;
if (current.getRemaining() <= 0 &&
((currentStreamIndex + 1) < streamEntries.size())) {
((currentStreamIndex + 1) < streamEntries.size())) {
currentStreamIndex += 1;
}
}

View File

@ -86,6 +86,7 @@ public class KeyOutputStream extends OutputStream {
* A constructor for testing purpose only.
*/
@VisibleForTesting
@SuppressWarnings("parameternumber")
public KeyOutputStream() {
streamEntries = new ArrayList<>();
omClient = null;
@ -143,6 +144,8 @@ public class KeyOutputStream extends OutputStream {
return locationInfoList;
}
@SuppressWarnings("parameternumber")
public KeyOutputStream(OpenKeySession handler,
XceiverClientManager xceiverClientManager,
StorageContainerLocationProtocolClientSideTranslatorPB scmClient,
@ -654,6 +657,7 @@ public class KeyOutputStream extends OutputStream {
private final long watchTimeout;
private List<ByteBuffer> bufferList;
@SuppressWarnings("parameternumber")
BlockOutputStreamEntry(BlockID blockID, String key,
XceiverClientManager xceiverClientManager,
XceiverClientSpi xceiverClient, String requestId, int chunkSize,

View File

@ -0,0 +1,20 @@
/**
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.hdds.protocol;
/**
* Helper classes for the hdds protocol.
*/

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.ozone;
import java.io.File;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.Collection;
import java.util.Optional;
@ -27,11 +26,8 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.HddsConfigKeys;
import org.apache.hadoop.hdds.server.ServerUtils;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.ozone.om.OMConfigKeys;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.hadoop.hdds.HddsUtils.getHostNameFromConfigKeys;
import static org.apache.hadoop.hdds.HddsUtils.getPortNumberFromConfigKeys;
@ -40,6 +36,8 @@ import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_BIND_HOST_DEFAULT
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_HTTP_ADDRESS_KEY;
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_HTTP_BIND_PORT_DEFAULT;
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_PORT_DEFAULT;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Stateless helper functions for the server and client side of OM

View File

@ -115,6 +115,8 @@ public class OzoneGetConf extends Configured implements Tool {
* {@link OzoneGetConf.Command}.
*/
static class CommandHandler {
@SuppressWarnings("visibilitymodifier")
protected String key; // Configuration key to lookup
CommandHandler() {

View File

@ -39,7 +39,8 @@ public class OmBucketInfoCodec implements Codec<OmBucketInfo> {
@Override
public OmBucketInfo fromPersistedFormat(byte[] rawData) {
Preconditions
.checkNotNull("Null byte array can't converted to real object.");
.checkNotNull(rawData,
"Null byte array can't converted to real object.");
try {
return OmBucketInfo.getFromProtobuf(BucketInfo.parseFrom(rawData));
} catch (InvalidProtocolBufferException e) {

View File

@ -39,7 +39,8 @@ public class OmKeyInfoCodec implements Codec<OmKeyInfo> {
@Override
public OmKeyInfo fromPersistedFormat(byte[] rawData) {
Preconditions
.checkNotNull("Null byte array can't converted to real object.");
.checkNotNull(rawData,
"Null byte array can't converted to real object.");
try {
return OmKeyInfo.getFromProtobuf(KeyInfo.parseFrom(rawData));
} catch (InvalidProtocolBufferException e) {

View File

@ -43,7 +43,7 @@ public class OmMultipartKeyInfoCodec implements Codec<OmMultipartKeyInfo> {
* return null.
*/
public OmMultipartKeyInfo fromPersistedFormat(byte[] rawData) {
Preconditions.checkNotNull(
Preconditions.checkNotNull(rawData,
"Null byte array can't converted to real object.");
try {
return OmMultipartKeyInfo.getFromProto(OzoneManagerProtocolProtos

View File

@ -39,7 +39,8 @@ public class OmVolumeArgsCodec implements Codec<OmVolumeArgs> {
@Override
public OmVolumeArgs fromPersistedFormat(byte[] rawData) {
Preconditions
.checkNotNull("Null byte array can't converted to real object.");
.checkNotNull(rawData,
"Null byte array can't converted to real object.");
try {
return OmVolumeArgs.getFromProtobuf(VolumeInfo.parseFrom(rawData));
} catch (InvalidProtocolBufferException e) {

View File

@ -38,7 +38,8 @@ public class VolumeListCodec implements Codec<VolumeList> {
@Override
public VolumeList fromPersistedFormat(byte[] rawData) {
Preconditions
.checkNotNull("Null byte array can't converted to real object.");
.checkNotNull(rawData,
"Null byte array can't converted to real object.");
try {
return VolumeList.parseFrom(rawData);
} catch (InvalidProtocolBufferException e) {

View File

@ -22,16 +22,15 @@ import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import com.google.common.base.Preconditions;
import org.apache.hadoop.hdds.protocol.StorageType;
import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
import org.apache.hadoop.ozone.OzoneAcl;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.audit.Auditable;
import org.apache.hadoop.ozone.protocol.proto
.OzoneManagerProtocolProtos.BucketArgs;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.BucketArgs;
import org.apache.hadoop.ozone.protocolPB.OMPBHelper;
import com.google.common.base.Preconditions;
/**
* A class that encapsulates Bucket Arguments.
*/

View File

@ -17,22 +17,21 @@
*/
package org.apache.hadoop.ozone.om.helpers;
import com.google.common.base.Preconditions;
import org.apache.hadoop.hdds.protocol.StorageType;
import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
import org.apache.hadoop.ozone.OzoneAcl;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.audit.Auditable;
import org.apache.hadoop.ozone.protocol.proto
.OzoneManagerProtocolProtos.BucketInfo;
import org.apache.hadoop.ozone.protocolPB.OMPBHelper;
import java.util.LinkedHashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import org.apache.hadoop.hdds.protocol.StorageType;
import org.apache.hadoop.ozone.OzoneAcl;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.audit.Auditable;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.BucketInfo;
import org.apache.hadoop.ozone.protocolPB.OMPBHelper;
import com.google.common.base.Preconditions;
/**
* A class that encapsulates Bucket Info.
*/

View File

@ -43,6 +43,7 @@ public final class OmKeyArgs implements Auditable {
private final String multipartUploadID;
private final int multipartUploadPartNumber;
@SuppressWarnings("parameternumber")
private OmKeyArgs(String volumeName, String bucketName, String keyName,
long dataSize, ReplicationType type, ReplicationFactor factor,
List<OmKeyLocationInfo> locationInfoList, boolean isMultipart,

View File

@ -45,6 +45,7 @@ public final class OmKeyInfo {
private HddsProtos.ReplicationType type;
private HddsProtos.ReplicationFactor factor;
@SuppressWarnings("parameternumber")
private OmKeyInfo(String volumeName, String bucketName, String keyName,
List<OmKeyLocationInfoGroup> versions, long dataSize,
long creationTime, long modificationTime,

View File

@ -37,7 +37,8 @@ import java.util.Properties;
@InterfaceAudience.Public
@InterfaceStability.Stable
public class OzoneVersionInfo {
private static final Logger LOG = LoggerFactory.getLogger(OzoneVersionInfo.class);
private static final Logger LOG =
LoggerFactory.getLogger(OzoneVersionInfo.class);
private Properties info;
@ -46,8 +47,9 @@ public class OzoneVersionInfo {
String versionInfoFile = component + "-version-info.properties";
InputStream is = null;
try {
is = ThreadUtil.getResourceAsStream(OzoneVersionInfo.class.getClassLoader(),
versionInfoFile);
is = ThreadUtil
.getResourceAsStream(OzoneVersionInfo.class.getClassLoader(),
versionInfoFile);
info.load(is);
} catch (IOException ex) {
LoggerFactory.getLogger(getClass()).warn("Could not read '" +
@ -57,130 +59,51 @@ public class OzoneVersionInfo {
}
}
protected String _getVersion() {
protected String getVersion() {
return info.getProperty("version", "Unknown");
}
protected String _getRelease() {
protected String getRelease() {
return info.getProperty("release", "Unknown");
}
protected String _getRevision() {
protected String getRevision() {
return info.getProperty("revision", "Unknown");
}
protected String _getBranch() {
protected String getBranch() {
return info.getProperty("branch", "Unknown");
}
protected String _getDate() {
protected String getDate() {
return info.getProperty("date", "Unknown");
}
protected String _getUser() {
protected String getUser() {
return info.getProperty("user", "Unknown");
}
protected String _getUrl() {
protected String getUrl() {
return info.getProperty("url", "Unknown");
}
protected String _getSrcChecksum() {
protected String getSrcChecksum() {
return info.getProperty("srcChecksum", "Unknown");
}
protected String _getBuildVersion(){
return _getVersion() +
" from " + _getRevision() +
" by " + _getUser() +
" source checksum " + _getSrcChecksum();
protected String getBuildVersion() {
return getVersion() +
" from " + getRevision() +
" by " + getUser() +
" source checksum " + getSrcChecksum();
}
protected String _getProtocVersion() {
protected String getProtocVersion() {
return info.getProperty("protocVersion", "Unknown");
}
private static OzoneVersionInfo OZONE_VERSION_INFO = new OzoneVersionInfo("ozone");
/**
* Get the Ozone version.
* @return the Ozone version string, eg. "0.6.3-dev"
*/
public static String getVersion() {
return OZONE_VERSION_INFO._getVersion();
}
/**
* Get the Ozone release name.
* @return the Ozone release string, eg. "Acadia"
*/
public static String getRelease() {
return OZONE_VERSION_INFO._getRelease();
}
/**
* Get the Git commit hash of the repository when compiled.
* @return the commit hash, eg. "18f64065d5db6208daf50b02c1b5ed4ee3ce547a"
*/
public static String getRevision() {
return OZONE_VERSION_INFO._getRevision();
}
/**
* Get the branch on which this originated.
* @return The branch name, e.g. "trunk" or "branches/branch-0.20"
*/
public static String getBranch() {
return OZONE_VERSION_INFO._getBranch();
}
/**
* The date that Ozone was compiled.
* @return the compilation date in unix date format
*/
public static String getDate() {
return OZONE_VERSION_INFO._getDate();
}
/**
* The user that compiled Ozone.
* @return the username of the user
*/
public static String getUser() {
return OZONE_VERSION_INFO._getUser();
}
/**
* Get the URL for the Ozone repository.
* @return the URL of the Ozone repository
*/
public static String getUrl() {
return OZONE_VERSION_INFO._getUrl();
}
/**
* Get the checksum of the source files from which Ozone was built.
* @return the checksum of the source files
*/
public static String getSrcChecksum() {
return OZONE_VERSION_INFO._getSrcChecksum();
}
/**
* Returns the buildVersion which includes version,
* revision, user and date.
* @return the buildVersion
*/
public static String getBuildVersion(){
return OZONE_VERSION_INFO._getBuildVersion();
}
/**
* Returns the protoc version used for the build.
* @return the protoc version
*/
public static String getProtocVersion(){
return OZONE_VERSION_INFO._getProtocVersion();
}
private static final OzoneVersionInfo OZONE_VERSION_INFO =
new OzoneVersionInfo("ozone");
public static void main(String[] args) {
System.out.println(
@ -200,12 +123,18 @@ public class OzoneVersionInfo {
" /////////// //////// \n" +
" ////// //////////// \n" +
" /// ////////// \n" +
" / "+ getVersion() + "("+ getRelease() +")\n");
System.out.println("Source code repository " + getUrl() + " -r " +
getRevision());
System.out.println("Compiled by " + getUser() + " on " + getDate());
System.out.println("Compiled with protoc " + getProtocVersion());
System.out.println("From source with checksum " + getSrcChecksum() + "\n");
" / " + OZONE_VERSION_INFO.getVersion() + "("
+ OZONE_VERSION_INFO.getRelease() + ")\n");
System.out.println(
"Source code repository " + OZONE_VERSION_INFO.getUrl() + " -r " +
OZONE_VERSION_INFO.getRevision());
System.out.println("Compiled by " + OZONE_VERSION_INFO.getUser() + " on "
+ OZONE_VERSION_INFO.getDate());
System.out.println(
"Compiled with protoc " + OZONE_VERSION_INFO.getProtocVersion());
System.out.println(
"From source with checksum " + OZONE_VERSION_INFO.getSrcChecksum()
+ "\n");
LOG.debug("This command was run using " +
ClassUtil.findContainingJar(OzoneVersionInfo.class));
HddsVersionInfo.main(args);

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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.util;
/**
* Ozone utilities.
*/

View File

@ -66,6 +66,7 @@ public class VolumeArgs extends UserArgs {
* @param headers - http headers
* @param groups - list of groups allowed to access the volume
*/
@SuppressWarnings("parameternumber")
public VolumeArgs(String userName, String volumeName, String requestID,
String hostName, Request request, UriInfo info,
HttpHeaders headers, String[] groups) {

View File

@ -48,14 +48,15 @@ public class TestOzoneObjInfo {
assertEquals(objInfo.getVolumeName(), volume);
}
private OzoneObjInfo.Builder getBuilder(String volume, String bucket,
String key) {
private OzoneObjInfo.Builder getBuilder(String withVolume,
String withBucket,
String withKey) {
return OzoneObjInfo.Builder.newBuilder()
.setResType(ResourceType.VOLUME)
.setStoreType(STORE)
.setVolumeName(volume)
.setBucketName(bucket)
.setKeyName(key);
.setVolumeName(withVolume)
.setBucketName(withBucket)
.setKeyName(withKey);
}
@Test

View File

@ -0,0 +1,18 @@
#!/usr/bin/env bash
# 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.
export HADOOP_VERSION=3
hadoop-ozone/dist/target/ozone-*-SNAPSHOT/smoketest/test.sh
exit $?

View File

@ -0,0 +1,22 @@
#!/usr/bin/env bash
# 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.
mkdir -p ./target
grep -r --include="*.java" "@author" .
if [ $? -gt 0 ]; then
exit 0
else
exit -1
fi

View File

@ -0,0 +1,18 @@
#!/usr/bin/env bash
# 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.
export MAVEN_OPTS="-Xmx4096m"
mvn -am -pl :hadoop-ozone-dist -P hdds -Dmaven.javadoc.skip=true -DskipTests clean install
exit $?

View File

@ -0,0 +1,23 @@
#!/usr/bin/env bash
# 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.
mvn -fn checkstyle:check -am -pl :hadoop-ozone-dist -Phdds
violations=$(grep -r error --include checkstyle-errors.xml | wc -l)
if [[ $violations -gt 0 ]]; then
echo "There are $violations checkstyle violations"
exit -1
fi
exit 0

View File

@ -0,0 +1,35 @@
#!/usr/bin/env bash
# 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.
FINDBUGS_ALL_FILE=./target/findbugs-all.txt
mkdir -p ./target
rm "$FINDBUGS_ALL_FILE" || true
touch "$FINDBUGS_ALL_FILE"
mvn -fn findbugs:check -Dfindbugs.failOnError=false -am -pl :hadoop-ozone-dist -Phdds
find hadoop-ozone -name findbugsXml.xml | xargs -n1 convertXmlToText >> "${FINDBUGS_ALL_FILE}"
find hadoop-hdds -name findbugsXml.xml | xargs -n1 convertXmlToText >> "${FINDBUGS_ALL_FILE}"
bugs=$(cat "$FINDBUGS_ALL_FILE" | wc -l)
if [[ ${bugs} -gt 0 ]]; then
exit -1
else
exit 0
fi

View File

@ -0,0 +1,24 @@
#!/usr/bin/env bash
# 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.
hadooplines=$(git diff --name-only HEAD~1..HEAD | grep -v hadoop-ozone | grep -v hadoop-hdds | wc -l )
if [ "$hadooplines" == "0" ]; then
echo "Only ozone/hdds subprojects are changed"
exit 0
else
echo "Main hadoop projects are changed in an ozone patch."
echo "Please do it in a HADOOP/HDFS patch and test it with hadoop precommit tests"
exit -1
fi

View File

@ -0,0 +1,24 @@
#!/usr/bin/env bash
# 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.
mkdir -p target
rm target/rat-aggregated.txt
mvn -fn org.apache.rat:apache-rat-plugin:0.13:check -am -pl :hadoop-ozone-dist -Phdds
grep -r --include=rat.txt "!????" | tee ./target/rat-aggregated.txt
if [ "$(cat target/rat-aggregated.txt)" ]; then
exit -1
fi

View File

@ -0,0 +1,24 @@
#!/usr/bin/env bash
# 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.
export MAVEN_OPTS="-Xmx4096m"
mvn -fn test -am -pl :hadoop-ozone-dist -P hdds
module_failed_tests=$(find "." -name 'TEST*.xml'\
| xargs "grep" -l -E "<failure|<error"\
| awk -F/ '{sub("'"TEST-JUNIT_TEST_OUTPUT_DIR"'",""); sub(".xml",""); print $NF}')
if [[ -n "${module_failed_tests}" ]] ; then
exit -1
fi
exit 0

View File

@ -0,0 +1,66 @@
# 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.
FROM alpine
RUN apk add --update --no-cache bash alpine-sdk maven grep openjdk8 py-pip rsync procps autoconf automake libtool findutils
#Install real glibc
RUN apk --no-cache add ca-certificates wget && \
wget -q -O /etc/apk/keys/sgerrand.rsa.pub https://alpine-pkgs.sgerrand.com/sgerrand.rsa.pub && \
wget https://github.com/sgerrand/alpine-pkg-glibc/releases/download/2.28-r0/glibc-2.28-r0.apk && \
apk add glibc-2.28-r0.apk
#Install protobuf
RUN mkdir -p /usr/local/src/ && \
cd /usr/local/src/ && \
wget https://github.com/google/protobuf/releases/download/v2.5.0/protobuf-2.5.0.tar.gz && \
tar xvf protobuf-2.5.0.tar.gz && \
cd protobuf-2.5.0 && \
./autogen.sh && \
./configure --prefix=/usr && \
make && \
make install && \
protoc --version
#Findbug install
RUN mkdir -p /opt && \
curl -sL https://sourceforge.net/projects/findbugs/files/findbugs/3.0.1/findbugs-3.0.1.tar.gz/download | tar -xz && \
mv findbugs-* /opt/findbugs
#Install apache-ant
RUN mkdir -p /opt && \
curl -sL 'https://www.apache.org/dyn/mirrors/mirrors.cgi?action=download&filename=/ant/binaries/apache-ant-1.10.5-bin.tar.gz' | tar -xz && \
mv apache-ant* /opt/ant
#Install docker-compose
RUN pip install docker-compose
ENV PATH=$PATH:/opt/findbugs/bin
RUN addgroup -g 1000 default && \
for i in $(seq 1 2000); do adduser jenkins$i -u $i -G default -h /tmp/ -H -D; done
#This is a very huge local maven cache. Usually the mvn repository is not safe to be
#shared between builds as concurrent installls are not handled very well
#A simple workaround is to provide all the required 3rd party lib in the docker image
#It will be cached by docker, and any additional dependency can be downloaded, artifacts
#can be installed
USER jenkins1000
RUN cd /tmp && \
git clone --depth=1 https://gitbox.apache.org/repos/asf/hadoop.git -b trunk && \
cd /tmp/hadoop && \
mvn package dependency:go-offline -DskipTests -P hdds -pl :hadoop-ozone-dist -am && \
rm -rf /tmp/.m2/repository/org/apache/hadoop/*hdds* && \
rm -rf /tmp/.m2/repository/org/apache/hadoop/*ozone* && \
find /tmp/.m2/repository -exec chmod o+wx {} \;

View File

@ -41,6 +41,9 @@ import static org.apache.hadoop.hdds.protocol.proto.HddsProtos
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos
.ReplicationFactor.THREE;
/**
* Test for the Node2Pipeline map.
*/
public class TestNode2PipelineMap {
private static MiniOzoneCluster cluster;
@ -80,7 +83,6 @@ public class TestNode2PipelineMap {
}
}
@Test
public void testPipelineMap() throws IOException {
@ -90,7 +92,7 @@ public class TestNode2PipelineMap {
ContainerID cId = ratisContainer.getContainerInfo().containerID();
Assert.assertEquals(1, set.size());
set.forEach(containerID ->
Assert.assertEquals(containerID, cId));
Assert.assertEquals(containerID, cId));
List<DatanodeDetails> dns = ratisContainer.getPipeline().getNodes();
Assert.assertEquals(3, dns.size());
@ -102,7 +104,6 @@ public class TestNode2PipelineMap {
pipelines.forEach(p -> Assert.assertEquals(p,
ratisContainer.getPipeline().getId()));
// Now close the container and it should not show up while fetching
// containers by pipeline
containerManager

View File

@ -211,7 +211,7 @@ public interface MiniOzoneCluster {
/**
* Builder class for MiniOzoneCluster.
*/
@SuppressWarnings("CheckStyle")
@SuppressWarnings("visibilitymodifier")
abstract class Builder {
protected static final int DEFAULT_HB_INTERVAL_MS = 1000;
@ -259,8 +259,8 @@ public interface MiniOzoneCluster {
return this;
}
public Builder setStartDataNodes(boolean startDataNodes) {
this.startDataNodes = startDataNodes;
public Builder setStartDataNodes(boolean nodes) {
this.startDataNodes = nodes;
return this;
}

View File

@ -254,15 +254,15 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
datanodeService.stop();
datanodeService.join();
// ensure same ports are used across restarts.
Configuration conf = datanodeService.getConf();
Configuration config = datanodeService.getConf();
int currentPort = datanodeService.getDatanodeDetails()
.getPort(DatanodeDetails.Port.Name.STANDALONE).getValue();
conf.setInt(DFS_CONTAINER_IPC_PORT, currentPort);
conf.setBoolean(DFS_CONTAINER_IPC_RANDOM_PORT, false);
config.setInt(DFS_CONTAINER_IPC_PORT, currentPort);
config.setBoolean(DFS_CONTAINER_IPC_RANDOM_PORT, false);
int ratisPort = datanodeService.getDatanodeDetails()
.getPort(DatanodeDetails.Port.Name.RATIS).getValue();
conf.setInt(DFS_CONTAINER_RATIS_IPC_PORT, ratisPort);
conf.setBoolean(DFS_CONTAINER_RATIS_IPC_RANDOM_PORT, false);
config.setInt(DFS_CONTAINER_RATIS_IPC_PORT, ratisPort);
config.setBoolean(DFS_CONTAINER_RATIS_IPC_RANDOM_PORT, false);
hddsDatanodes.remove(i);
if (waitForDatanode) {
// wait for node to be removed from SCM healthy node list.
@ -270,7 +270,7 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
}
String[] args = new String[]{};
HddsDatanodeService service =
HddsDatanodeService.createHddsDatanodeService(args, conf);
HddsDatanodeService.createHddsDatanodeService(args, config);
hddsDatanodes.add(i, service);
service.start(null);
if (waitForDatanode) {

View File

@ -84,7 +84,8 @@ public final class OzoneTestUtils {
CheckedConsumer<BlockID, Exception> consumer,
List<OmKeyLocationInfoGroup> omKeyLocationInfoGroups) throws Exception {
for (OmKeyLocationInfoGroup omKeyLocationInfoGroup : omKeyLocationInfoGroups) {
for (OmKeyLocationInfoGroup omKeyLocationInfoGroup :
omKeyLocationInfoGroups) {
List<OmKeyLocationInfo> omKeyLocationInfos =
omKeyLocationInfoGroup.getLocationList();
for (OmKeyLocationInfo omKeyLocationInfo : omKeyLocationInfos) {

View File

@ -47,6 +47,9 @@ import java.io.IOException;
import java.util.List;
import java.util.concurrent.TimeoutException;
/**
* Test container closing.
*/
public class TestCloseContainerByPipeline {
private static MiniOzoneCluster cluster;
@ -232,10 +235,12 @@ public class TestCloseContainerByPipeline {
}
}
private Boolean isContainerClosed(MiniOzoneCluster cluster, long containerID,
private Boolean isContainerClosed(MiniOzoneCluster ozoneCluster,
long containerID,
DatanodeDetails datanode) {
ContainerData containerData;
for (HddsDatanodeService datanodeService : cluster.getHddsDatanodes()) {
for (HddsDatanodeService datanodeService : ozoneCluster
.getHddsDatanodes()) {
if (datanode.equals(datanodeService.getDatanodeDetails())) {
containerData =
datanodeService.getDatanodeStateMachine().getContainer()

View File

@ -149,7 +149,8 @@ public class TestContainerServer {
XceiverClientSpi client = null;
String containerName = OzoneUtils.getRequestID();
try {
final Pipeline pipeline = ContainerTestHelper.createPipeline(numDatanodes);
final Pipeline pipeline =
ContainerTestHelper.createPipeline(numDatanodes);
final OzoneConfiguration conf = new OzoneConfiguration();
initConf.accept(pipeline, conf);

View File

@ -67,7 +67,6 @@ import static org.junit.Assert.fail;
* This test class specified for testing Ozone datanode shell command.
*/
@RunWith(value = Parameterized.class)
public class TestOzoneDatanodeShell {
private static final Logger LOG =
@ -100,6 +99,7 @@ public class TestOzoneDatanodeShell {
}
@Parameterized.Parameter
@SuppressWarnings("visibilitymodifier")
public Class clientProtocol;
/**
* Create a MiniDFSCluster for testing with using distributed Ozone

View File

@ -125,6 +125,7 @@ public class TestOzoneShell {
}
@Parameterized.Parameter
@SuppressWarnings("visibilitymodifier")
public Class clientProtocol;
/**
* Create a MiniDFSCluster for testing with using distributed Ozone

View File

@ -64,10 +64,11 @@ public class TestAllocateContainer {
@Test
public void testAllocate() throws Exception {
ContainerWithPipeline container = storageContainerLocationClient.allocateContainer(
xceiverClientManager.getType(),
xceiverClientManager.getFactor(),
containerOwner);
ContainerWithPipeline container =
storageContainerLocationClient.allocateContainer(
xceiverClientManager.getType(),
xceiverClientManager.getFactor(),
containerOwner);
Assert.assertNotNull(container);
Assert.assertNotNull(container.getPipeline().getFirstNode());

View File

@ -103,7 +103,8 @@ public class TestOzoneRestWithMiniCluster {
putKey(bucket, keyName, keyData);
}
private void putKey(OzoneBucket bucket, String keyName, String keyData) throws IOException {
private void putKey(OzoneBucket bucket, String keyName, String keyData)
throws IOException {
try (
OzoneOutputStream ozoneOutputStream = bucket
.createKey(keyName, 0, replicationType, replicationFactor);

View File

@ -78,6 +78,7 @@ public class TestBuckets {
return Arrays.asList(params);
}
@SuppressWarnings("visibilitymodifier")
@Parameterized.Parameter
public static Class clientProtocol;
@ -124,7 +125,7 @@ public class TestBuckets {
runTestCreateBucket(client);
}
static void runTestCreateBucket(ClientProtocol client)
static void runTestCreateBucket(ClientProtocol protocol)
throws IOException {
String volumeName = OzoneUtils.getRequestID().toLowerCase();
VolumeArgs volumeArgs = VolumeArgs.newBuilder()
@ -132,8 +133,8 @@ public class TestBuckets {
.setQuota("100TB")
.setAdmin("hdfs")
.build();
client.createVolume(volumeName, volumeArgs);
OzoneVolume vol = client.getVolumeDetails(volumeName);
protocol.createVolume(volumeName, volumeArgs);
OzoneVolume vol = protocol.getVolumeDetails(volumeName);
String[] acls = {"user:frodo:rw", "user:samwise:rw"};
// create 10 buckets under same volume
@ -154,7 +155,7 @@ public class TestBuckets {
// verify the bucket creation time
assertTrue((bucket.getCreationTime() / 1000) >= (currentTime / 1000));
}
client.close();
protocol.close();
assertEquals(vol.getName(), volumeName);
assertEquals(vol.getAdmin(), "hdfs");
@ -179,7 +180,7 @@ public class TestBuckets {
runTestAddBucketAcls(client);
}
static void runTestAddBucketAcls(ClientProtocol client)
static void runTestAddBucketAcls(ClientProtocol protocol)
throws OzoneException, IOException, ParseException {
String volumeName = OzoneUtils.getRequestID().toLowerCase();
VolumeArgs volumeArgs = VolumeArgs.newBuilder()
@ -187,8 +188,8 @@ public class TestBuckets {
.setQuota("100TB")
.setAdmin("hdfs")
.build();
client.createVolume(volumeName, volumeArgs);
OzoneVolume vol = client.getVolumeDetails(volumeName);
protocol.createVolume(volumeName, volumeArgs);
OzoneVolume vol = protocol.getVolumeDetails(volumeName);
String[] acls = {"user:frodo:rw", "user:samwise:rw"};
String bucketName = OzoneUtils.getRequestID().toLowerCase();
vol.createBucket(bucketName);
@ -203,7 +204,7 @@ public class TestBuckets {
// verify if the creation time is missing after update operation
assertTrue(
(updatedBucket.getCreationTime()) / 1000 >= 0);
client.close();
protocol.close();
}
@Test
@ -211,7 +212,7 @@ public class TestBuckets {
runTestRemoveBucketAcls(client);
}
static void runTestRemoveBucketAcls(ClientProtocol client)
static void runTestRemoveBucketAcls(ClientProtocol protocol)
throws OzoneException, IOException, ParseException {
String volumeName = OzoneUtils.getRequestID().toLowerCase();
VolumeArgs volumeArgs = VolumeArgs.newBuilder()
@ -219,8 +220,8 @@ public class TestBuckets {
.setQuota("100TB")
.setAdmin("hdfs")
.build();
client.createVolume(volumeName, volumeArgs);
OzoneVolume vol = client.getVolumeDetails(volumeName);
protocol.createVolume(volumeName, volumeArgs);
OzoneVolume vol = protocol.getVolumeDetails(volumeName);
String[] acls = {"user:frodo:rw", "user:samwise:rw"};
String bucketName = OzoneUtils.getRequestID().toLowerCase();
List<OzoneAcl> aclList =
@ -239,7 +240,7 @@ public class TestBuckets {
// verify if the creation time is missing after update operation
assertTrue(
(updatedBucket.getCreationTime() / 1000) >= 0);
client.close();
protocol.close();
}
@Test
@ -247,7 +248,7 @@ public class TestBuckets {
runTestDeleteBucket(client);
}
static void runTestDeleteBucket(ClientProtocol client)
static void runTestDeleteBucket(ClientProtocol protocol)
throws OzoneException, IOException {
String volumeName = OzoneUtils.getRequestID().toLowerCase();
VolumeArgs volumeArgs = VolumeArgs.newBuilder()
@ -255,8 +256,8 @@ public class TestBuckets {
.setQuota("100TB")
.setAdmin("hdfs")
.build();
client.createVolume(volumeName, volumeArgs);
OzoneVolume vol = client.getVolumeDetails(volumeName);
protocol.createVolume(volumeName, volumeArgs);
OzoneVolume vol = protocol.getVolumeDetails(volumeName);
String[] acls = {"user:frodo:rw", "user:samwise:rw"};
String bucketName = OzoneUtils.getRequestID().toLowerCase();
List<OzoneAcl> aclList =
@ -274,7 +275,7 @@ public class TestBuckets {
// must throw
assertNotNull(ex);
}
client.close();
protocol.close();
}
@Test
@ -282,7 +283,7 @@ public class TestBuckets {
runTestListBucket(client);
}
static void runTestListBucket(ClientProtocol client)
static void runTestListBucket(ClientProtocol protocol)
throws OzoneException, IOException, ParseException {
String volumeName = OzoneUtils.getRequestID().toLowerCase();
VolumeArgs volumeArgs = VolumeArgs.newBuilder()
@ -290,11 +291,11 @@ public class TestBuckets {
.setQuota("100TB")
.setAdmin("hdfs")
.build();
client.createVolume(volumeName, volumeArgs);
OzoneVolume vol = client.getVolumeDetails(volumeName);
protocol.createVolume(volumeName, volumeArgs);
OzoneVolume vol = protocol.getVolumeDetails(volumeName);
String[] acls = {"user:frodo:rw", "user:samwise:rw"};
List<OzoneAcl> aclList =
Arrays.stream(acls).map(acl -> OzoneAcl.parseAcl(acl))
Arrays.stream(acls).map(OzoneAcl::parseAcl)
.collect(Collectors.toList());
long currentTime = Time.now();
@ -321,7 +322,7 @@ public class TestBuckets {
bucketIterator = vol.listBuckets(null, "listbucket-test-3");
assertEquals(getSize(bucketIterator), 6);
client.close();
protocol.close();
}
private static int getSize(Iterator<? extends OzoneBucket> bucketIterator) {

View File

@ -57,6 +57,7 @@ public class TestBucketsRatis {
}
@Parameterized.Parameter
@SuppressWarnings("visibilitymodifier")
public static Class clientProtocol;
@BeforeClass

View File

@ -72,6 +72,7 @@ public class TestVolume {
return Arrays.asList(params);
}
@SuppressWarnings("visibilitymodifier")
@Parameterized.Parameter
public Class clientProtocol;
@ -120,7 +121,7 @@ public class TestVolume {
runTestCreateVolume(client);
}
static void runTestCreateVolume(ClientProtocol client)
static void runTestCreateVolume(ClientProtocol clientProtocol)
throws OzoneException, IOException, ParseException {
String volumeName = OzoneUtils.getRequestID().toLowerCase();
@ -131,8 +132,8 @@ public class TestVolume {
.setQuota("100TB")
.setAdmin("hdfs")
.build();
client.createVolume(volumeName, volumeArgs);
OzoneVolume vol = client.getVolumeDetails(volumeName);
clientProtocol.createVolume(volumeName, volumeArgs);
OzoneVolume vol = clientProtocol.getVolumeDetails(volumeName);
assertEquals(vol.getName(), volumeName);
assertEquals(vol.getAdmin(), "hdfs");
@ -147,7 +148,7 @@ public class TestVolume {
// not use Rule here because the test method is static.
try {
String invalidVolumeName = "#" + OzoneUtils.getRequestID().toLowerCase();
client.createVolume(invalidVolumeName);
clientProtocol.createVolume(invalidVolumeName);
/*
//TODO: RestClient and RpcClient should use HddsClientUtils to verify name
fail("Except the volume creation be failed because the"
@ -163,11 +164,11 @@ public class TestVolume {
runTestCreateDuplicateVolume(client);
}
static void runTestCreateDuplicateVolume(ClientProtocol client)
static void runTestCreateDuplicateVolume(ClientProtocol clientProtocol)
throws OzoneException, IOException {
try {
client.createVolume("testvol");
client.createVolume("testvol");
clientProtocol.createVolume("testvol");
clientProtocol.createVolume("testvol");
assertFalse(true);
} catch (IOException ioe) {
Assert.assertTrue(ioe.getMessage()
@ -180,11 +181,11 @@ public class TestVolume {
runTestDeleteVolume(client);
}
static void runTestDeleteVolume(ClientProtocol client)
static void runTestDeleteVolume(ClientProtocol clientProtocol)
throws OzoneException, IOException {
String volumeName = OzoneUtils.getRequestID().toLowerCase();
client.createVolume(volumeName);
client.deleteVolume(volumeName);
clientProtocol.createVolume(volumeName);
clientProtocol.deleteVolume(volumeName);
}
@Test
@ -192,13 +193,13 @@ public class TestVolume {
runTestChangeOwnerOnVolume(client);
}
static void runTestChangeOwnerOnVolume(ClientProtocol client)
static void runTestChangeOwnerOnVolume(ClientProtocol clientProtocol)
throws OzoneException, ParseException, IOException {
String volumeName = OzoneUtils.getRequestID().toLowerCase();
client.createVolume(volumeName);
client.getVolumeDetails(volumeName);
client.setVolumeOwner(volumeName, "frodo");
OzoneVolume newVol = client.getVolumeDetails(volumeName);
clientProtocol.createVolume(volumeName);
clientProtocol.getVolumeDetails(volumeName);
clientProtocol.setVolumeOwner(volumeName, "frodo");
OzoneVolume newVol = clientProtocol.getVolumeDetails(volumeName);
assertEquals(newVol.getOwner(), "frodo");
// verify if the creation time is missing after setting owner operation
assertTrue(newVol.getCreationTime() > 0);
@ -209,12 +210,12 @@ public class TestVolume {
runTestChangeQuotaOnVolume(client);
}
static void runTestChangeQuotaOnVolume(ClientProtocol client)
static void runTestChangeQuotaOnVolume(ClientProtocol clientProtocol)
throws OzoneException, IOException, ParseException {
String volumeName = OzoneUtils.getRequestID().toLowerCase();
client.createVolume(volumeName);
client.setVolumeQuota(volumeName, OzoneQuota.parseQuota("1000MB"));
OzoneVolume newVol = client.getVolumeDetails(volumeName);
clientProtocol.createVolume(volumeName);
clientProtocol.setVolumeQuota(volumeName, OzoneQuota.parseQuota("1000MB"));
OzoneVolume newVol = clientProtocol.getVolumeDetails(volumeName);
assertEquals(newVol.getQuota(),
OzoneQuota.parseQuota("1000MB").sizeInBytes());
// verify if the creation time is missing after setting quota operation
@ -229,14 +230,14 @@ public class TestVolume {
runTestListVolume(client);
}
static void runTestListVolume(ClientProtocol client)
static void runTestListVolume(ClientProtocol clientProtocol)
throws OzoneException, IOException {
for (int x = 0; x < 10; x++) {
String volumeName = OzoneUtils.getRequestID().toLowerCase();
client.createVolume(volumeName);
clientProtocol.createVolume(volumeName);
}
List<OzoneVolume> ovols = client.listVolumes(null, null, 100);
List<OzoneVolume> ovols = clientProtocol.listVolumes(null, null, 100);
assertTrue(ovols.size() >= 10);
}
@ -247,19 +248,19 @@ public class TestVolume {
runTestListVolumePagination(client);
}
static void runTestListVolumePagination(ClientProtocol client)
static void runTestListVolumePagination(ClientProtocol clientProtocol)
throws OzoneException, IOException {
final int volCount = 2000;
final int step = 100;
for (int x = 0; x < volCount; x++) {
String volumeName = OzoneUtils.getRequestID().toLowerCase();
client.createVolume(volumeName);
clientProtocol.createVolume(volumeName);
}
String prevKey = null;
int count = 0;
int pagecount = 0;
while (count < volCount) {
List<OzoneVolume> ovols = client.listVolumes(null, prevKey, step);
List<OzoneVolume> ovols = clientProtocol.listVolumes(null, prevKey, step);
count += ovols.size();
prevKey = ovols.get(ovols.size() - 1).getName();
pagecount++;
@ -274,7 +275,7 @@ public class TestVolume {
runTestListAllVolumes(client);
}
static void runTestListAllVolumes(ClientProtocol client)
static void runTestListAllVolumes(ClientProtocol clientProtocol)
throws OzoneException, IOException {
final int volCount = 200;
final int step = 10;
@ -288,15 +289,15 @@ public class TestVolume {
.setQuota("100TB")
.setAdmin("hdfs")
.build();
client.createVolume(volumeName, volumeArgs);
OzoneVolume vol = client.getVolumeDetails(volumeName);
clientProtocol.createVolume(volumeName, volumeArgs);
OzoneVolume vol = clientProtocol.getVolumeDetails(volumeName);
assertNotNull(vol);
}
String prevKey = null;
int count = 0;
int pagecount = 0;
while (count < volCount) {
List<OzoneVolume> ovols = client.listVolumes(null, prevKey, step);
List<OzoneVolume> ovols = clientProtocol.listVolumes(null, prevKey, step);
count += ovols.size();
if (ovols.size() > 0) {
prevKey = ovols.get(ovols.size() - 1).getName();
@ -316,7 +317,7 @@ public class TestVolume {
runTestListVolumes(client);
}
static void runTestListVolumes(ClientProtocol client)
static void runTestListVolumes(ClientProtocol clientProtocol)
throws OzoneException, IOException, ParseException {
final int volCount = 20;
final String user1 = "test-user-a";
@ -342,13 +343,14 @@ public class TestVolume {
.setQuota("100TB")
.setAdmin("hdfs")
.build();
client.createVolume(volumeName, volumeArgs);
OzoneVolume vol = client.getVolumeDetails(volumeName);
clientProtocol.createVolume(volumeName, volumeArgs);
OzoneVolume vol = clientProtocol.getVolumeDetails(volumeName);
assertNotNull(vol);
}
// list all the volumes belong to user1
List<OzoneVolume> volumeList = client.listVolumes(user1, null, null, 100);
List<OzoneVolume> volumeList =
clientProtocol.listVolumes(user1, null, null, 100);
assertEquals(10, volumeList.size());
// verify the owner name and creation time of volume
for (OzoneVolume vol : volumeList) {
@ -358,25 +360,25 @@ public class TestVolume {
}
// test max key parameter of listing volumes
volumeList = client.listVolumes(user1, null, null, 2);
volumeList = clientProtocol.listVolumes(user1, null, null, 2);
assertEquals(2, volumeList.size());
// test prefix parameter of listing volumes
volumeList = client.listVolumes(user1, "test-vol10", null, 10);
volumeList = clientProtocol.listVolumes(user1, "test-vol10", null, 10);
assertTrue(volumeList.size() == 1
&& volumeList.get(0).getName().equals("test-vol10"));
volumeList = client.listVolumes(user1, "test-vol1", null, 10);
volumeList = clientProtocol.listVolumes(user1, "test-vol1", null, 10);
assertEquals(5, volumeList.size());
// test start key parameter of listing volumes
volumeList = client.listVolumes(user2, null, "test-vol15", 10);
volumeList = clientProtocol.listVolumes(user2, null, "test-vol15", 10);
assertEquals(2, volumeList.size());
String volumeName;
for (int x = 0; x < volCount; x++) {
volumeName = "test-vol" + x;
client.deleteVolume(volumeName);
clientProtocol.deleteVolume(volumeName);
}
}
}

View File

@ -60,6 +60,7 @@ public class TestVolumeRatis {
}
@Parameterized.Parameter
@SuppressWarnings("visibilitymodifier")
public Class clientProtocol;
@BeforeClass

View File

@ -164,6 +164,7 @@ public interface Bucket {
+ "04:23:30 GMT", required = true, paramType = "header"),
@ApiImplicitParam(name = "Authorization", example = "OZONE", required =
true, paramType = "header")})
@SuppressWarnings("parameternumber")
Response listBucket(@PathParam("volume") String volume,
@PathParam("bucket") String bucket,
@DefaultValue(Header.OZONE_INFO_QUERY_KEY)

View File

@ -170,6 +170,7 @@ public interface Volume {
+ "04:23:30 GMT", required = true, paramType = "header"),
@ApiImplicitParam(name = "Authorization", example = "OZONE", required =
true, paramType = "header")})
@SuppressWarnings("parameternumber")
Response getVolumeInfo(@PathParam("volume") String volume,
@DefaultValue(Header.OZONE_INFO_QUERY_BUCKET)
@QueryParam(Header.OZONE_INFO_QUERY_TAG) String info,

View File

@ -74,6 +74,6 @@ public interface BucketManager {
* @throws IOException
*/
List<OmBucketInfo> listBuckets(String volumeName,
String startBucket, String bucketPrefix, int maxNumOfBuckets)
String startBucket, String bucketPrefix, int maxNumOfBuckets)
throws IOException;
}

View File

@ -22,10 +22,9 @@ import java.util.Properties;
import java.util.UUID;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.server.ServerUtils;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType;
import org.apache.hadoop.ozone.OmUtils;
import org.apache.hadoop.ozone.common.Storage;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType;
import static org.apache.hadoop.ozone.OzoneConsts.SCM_ID;

View File

@ -28,20 +28,19 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OmUtils;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.common.BlockGroup;
import org.apache.hadoop.ozone.om.codec.OmBucketInfoCodec;
import org.apache.hadoop.ozone.om.codec.OmKeyInfoCodec;
import org.apache.hadoop.ozone.om.codec.OmMultipartKeyInfoCodec;
import org.apache.hadoop.ozone.om.codec.OmVolumeArgsCodec;
import org.apache.hadoop.ozone.om.codec.VolumeListCodec;
import org.apache.hadoop.ozone.om.exceptions.OMException;
import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
import org.apache.hadoop.ozone.om.codec.OmBucketInfoCodec;
import org.apache.hadoop.ozone.om.helpers.OmMultipartKeyInfo;
import org.apache.hadoop.ozone.om.codec.OmMultipartKeyInfoCodec;
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
import org.apache.hadoop.ozone.om.codec.OmKeyInfoCodec;
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
import org.apache.hadoop.ozone.om.helpers.OmMultipartKeyInfo;
import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
import org.apache.hadoop.ozone.om.codec.OmVolumeArgsCodec;
import org.apache.hadoop.ozone.om.codec.VolumeListCodec;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeList;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.utils.db.DBStore;
import org.apache.hadoop.utils.db.DBStoreBuilder;
import org.apache.hadoop.utils.db.Table;

View File

@ -96,5 +96,5 @@ public interface VolumeManager {
* @throws IOException
*/
List<OmVolumeArgs> listVolumes(String userName, String prefix,
String startKey, int maxKeys) throws IOException;
String startKey, int maxKeys) throws IOException;
}

View File

@ -43,7 +43,8 @@ import org.slf4j.LoggerFactory;
/**
* Ratis helper methods for OM Ratis server and client.
*/
public class OMRatisHelper {
public final class OMRatisHelper {
private static final Logger LOG = LoggerFactory.getLogger(
OMRatisHelper.class);

View File

@ -214,6 +214,7 @@ public class OzoneManagerProtocolServerSideTranslatorPB implements
/**
* Submits request directly to OM.
*/
@SuppressWarnings("methodlength")
private OMResponse submitRequestToOM(OMRequest request)
throws ServiceException {
Type cmdType = request.getCmdType();

View File

@ -66,7 +66,7 @@ public class SignedChunksInputStream extends InputStream {
}
@Override
public int read(byte b[], int off, int len) throws IOException {
public int read(byte[] b, int off, int len) throws IOException {
if (b == null) {
throw new NullPointerException();
} else if (off < 0 || len < 0 || len > b.length - off) {

View File

@ -74,6 +74,7 @@ public class BucketEndpoint extends EndpointBase {
*/
@GET
@SuppressFBWarnings
@SuppressWarnings("parameternumber")
public Response list(
@PathParam("bucket") String bucketName,
@QueryParam("delimiter") String delimiter,

View File

@ -19,7 +19,6 @@
*/
package org.apache.hadoop.ozone.s3.endpoint;
import javax.ws.rs.core.Response;
import javax.xml.bind.JAXBException;
import java.io.IOException;
import java.util.HashSet;

View File

@ -145,7 +145,7 @@ public final class DatabaseHelper {
throws Exception {
ArrayList<AuditEntry> listResult = new ArrayList<AuditEntry>();
try(FileInputStream fis = new FileInputStream(filePath);
InputStreamReader isr = new InputStreamReader(fis);
InputStreamReader isr = new InputStreamReader(fis, "UTF-8");
BufferedReader bReader = new BufferedReader(isr)) {
String currentLine = null;
String[] entry = null;

View File

@ -62,7 +62,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.ObjectWriter;
import com.google.common.annotations.VisibleForTesting;
import static java.lang.Math.min;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.lang3.ArrayUtils;
import org.apache.commons.lang3.RandomStringUtils;
import org.apache.commons.lang3.time.DurationFormatUtils;

View File

@ -34,6 +34,9 @@ import static org.apache.hadoop.ozone.genesis.GenesisUtil.CACHE_1GB_TYPE;
import static org.apache.hadoop.ozone.genesis.GenesisUtil.CLOSED_TYPE;
import static org.apache.hadoop.ozone.genesis.GenesisUtil.DEFAULT_TYPE;
/**
* Measure metadatastore read performance.
*/
@State(Scope.Thread)
public class BenchMarkMetadataStoreReads {

View File

@ -32,10 +32,12 @@ import static org.apache.hadoop.ozone.genesis.GenesisUtil.CACHE_10MB_TYPE;
import static org.apache.hadoop.ozone.genesis.GenesisUtil.CACHE_1GB_TYPE;
import static org.apache.hadoop.ozone.genesis.GenesisUtil.DEFAULT_TYPE;
/**
* Measure default metadatastore put performance.
*/
@State(Scope.Thread)
public class BenchMarkMetadataStoreWrites {
private static final int DATA_LEN = 1024;
private static final long MAX_KEYS = 1024 * 10;

View File

@ -31,6 +31,9 @@ import java.io.IOException;
import java.nio.charset.Charset;
import java.nio.file.Paths;
/**
* Benchmark rocksdb store.
*/
@State(Scope.Thread)
public class BenchMarkRocksDbStore {
private static final int DATA_LEN = 1024;
@ -86,7 +89,8 @@ public class BenchMarkRocksDbStore {
opts.setLevel0SlowdownWritesTrigger(20);
opts.setLevel0StopWritesTrigger(40);
opts.setTargetFileSizeBase(
(long) StorageUnit.MB.toBytes(Long.parseLong(maxBytesForLevelBase)) / 10);
(long) StorageUnit.MB.toBytes(Long.parseLong(maxBytesForLevelBase))
/ 10);
opts.setMaxBackgroundCompactions(8);
opts.setUseFsync(false);
opts.setBytesPerSync(8388608);