HDDS-965. Ozone: checkstyle improvements and code quality scripts. Contributed by Elek, Marton.
This commit is contained in:
parent
1a08302897
commit
32cf0412ce
|
@ -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);
|
||||
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -81,6 +81,7 @@ public class ContainerInfo implements Comparator<ContainerInfo>,
|
|||
@JsonIgnore
|
||||
private byte[] data;
|
||||
|
||||
@SuppressWarnings("parameternumber")
|
||||
ContainerInfo(
|
||||
long containerID,
|
||||
HddsProtos.LifeCycleState state,
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
|
|
|
@ -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.
|
||||
*/
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
||||
/**
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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> {
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
|
||||
/**
|
||||
|
|
|
@ -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> {
|
||||
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
}
|
|
@ -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,
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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.
|
||||
*/
|
|
@ -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
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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.
|
||||
*/
|
||||
|
|
|
@ -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.
|
||||
*/
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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.
|
||||
*/
|
|
@ -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) {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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 $?
|
|
@ -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
|
|
@ -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 $?
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
||||
|
|
@ -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
|
|
@ -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 {} \;
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -125,6 +125,7 @@ public class TestOzoneShell {
|
|||
}
|
||||
|
||||
@Parameterized.Parameter
|
||||
@SuppressWarnings("visibilitymodifier")
|
||||
public Class clientProtocol;
|
||||
/**
|
||||
* Create a MiniDFSCluster for testing with using distributed Ozone
|
||||
|
|
|
@ -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());
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -57,6 +57,7 @@ public class TestBucketsRatis {
|
|||
}
|
||||
|
||||
@Parameterized.Parameter
|
||||
@SuppressWarnings("visibilitymodifier")
|
||||
public static Class clientProtocol;
|
||||
|
||||
@BeforeClass
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -60,6 +60,7 @@ public class TestVolumeRatis {
|
|||
}
|
||||
|
||||
@Parameterized.Parameter
|
||||
@SuppressWarnings("visibilitymodifier")
|
||||
public Class clientProtocol;
|
||||
|
||||
@BeforeClass
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -74,6 +74,7 @@ public class BucketEndpoint extends EndpointBase {
|
|||
*/
|
||||
@GET
|
||||
@SuppressFBWarnings
|
||||
@SuppressWarnings("parameternumber")
|
||||
public Response list(
|
||||
@PathParam("bucket") String bucketName,
|
||||
@QueryParam("delimiter") String delimiter,
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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 {
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
Loading…
Reference in New Issue