Browse Source

Merge branch 'HDDS-4' into trunk

Xiaoyu Yao 6 years ago
parent
commit
751bc62df7
38 changed files with 889 additions and 224 deletions
  1. 0 3
      BUILDING.txt
  2. 2 0
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_xor_decoder.c
  3. 10 1
      hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java
  4. 5 5
      hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/ContainerOperationClient.java
  5. 1 1
      hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java
  6. 9 8
      hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockOutputStream.java
  7. 2 15
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
  8. 2 2
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
  9. 14 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOpsCountStatistics.java
  10. 28 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  11. 121 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
  12. 20 37
      hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntry.java
  13. 1 1
      hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyInputStream.java
  14. 31 29
      hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyOutputStream.java
  15. 1 1
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerStateMachineIdempotency.java
  16. 1 1
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java
  17. 4 4
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSmallFile.java
  18. 3 3
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestGetCommittedBlockLengthAndPutKey.java
  19. 45 7
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestXceiverClientManager.java
  20. 12 8
      hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java
  21. 2 2
      hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ObjectEndpoint.java
  22. 173 0
      hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/ContinueToken.java
  23. 5 69
      hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/RangeHeaderParserUtil.java
  24. 47 2
      hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketGet.java
  25. 50 0
      hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/util/TestContinueToken.java
  26. 10 10
      hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/util/TestRangeHeaderParserUtil.java
  27. 7 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
  28. 6 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstance.java
  29. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
  30. 11 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
  31. 79 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
  32. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/DataGeneratorForTest.java
  33. 5 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineSchemaCreator.java
  34. 28 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/SchemaCreator.java
  35. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java
  36. 29 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/DummyTimelineSchemaCreator.java
  37. 41 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestTimelineSchemaCreator.java
  38. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/bower.json

+ 0 - 3
BUILDING.txt

@@ -14,7 +14,6 @@ Requirements:
   Clang (community version), Clang (version for iOS 9 and later) (if compiling native code)
 * openssl devel (if compiling native hadoop-pipes and to get the best HDFS encryption performance)
 * Linux FUSE (Filesystem in Userspace) version 2.6 or above (if compiling fuse_dfs)
-* Jansson C XML parsing library ( if compiling libwebhdfs )
 * Doxygen ( if compiling libhdfspp and generating the documents )
 * Internet connection for first build (to fetch all Maven and Hadoop dependencies)
 * python (for releasedocs)
@@ -75,8 +74,6 @@ Optional packages:
   (OR https://github.com/01org/isa-l)
 * Bzip2
   $ sudo apt-get install bzip2 libbz2-dev
-* Jansson (C Library for JSON)
-  $ sudo apt-get install libjansson-dev
 * Linux FUSE
   $ sudo apt-get install fuse libfuse-dev
 * ZStandard compression

+ 2 - 0
hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_xor_decoder.c

@@ -66,6 +66,8 @@ Java_org_apache_hadoop_io_erasurecode_rawcoder_NativeXORRawDecoder_decodeImpl(
       numDataUnits + numParityUnits);
   getOutputs(env, outputs, outputOffsets, xorDecoder->outputs, numParityUnits);
 
+  memset(xorDecoder->outputs[0], 0, chunkSize);
+
   for (i = 0; i < numDataUnits + numParityUnits; i++) {
     if (xorDecoder->inputs[i] == NULL) {
       continue;

+ 10 - 1
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java

@@ -133,11 +133,20 @@ public class XceiverClientManager implements Closeable {
    * Releases a XceiverClientSpi after use.
    *
    * @param client client to release
+   * @param invalidateClient if true, invalidates the client in cache
    */
-  public void releaseClient(XceiverClientSpi client) {
+  public void releaseClient(XceiverClientSpi client, boolean invalidateClient) {
     Preconditions.checkNotNull(client);
     synchronized (clientCache) {
       client.decrementReference();
+      if (invalidateClient) {
+        Pipeline pipeline = client.getPipeline();
+        String key = pipeline.getId().getId().toString() + pipeline.getType();
+        XceiverClientSpi cachedClient = clientCache.getIfPresent(key);
+        if (cachedClient == client) {
+          clientCache.invalidate(key);
+        }
+      }
     }
   }
 

+ 5 - 5
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/ContainerOperationClient.java

@@ -100,7 +100,7 @@ public class ContainerOperationClient implements ScmClient {
       return containerWithPipeline;
     } finally {
       if (client != null) {
-        xceiverClientManager.releaseClient(client);
+        xceiverClientManager.releaseClient(client, false);
       }
     }
   }
@@ -191,7 +191,7 @@ public class ContainerOperationClient implements ScmClient {
       return containerWithPipeline;
     } finally {
       if (client != null) {
-        xceiverClientManager.releaseClient(client);
+        xceiverClientManager.releaseClient(client, false);
       }
     }
   }
@@ -269,7 +269,7 @@ public class ContainerOperationClient implements ScmClient {
       }
     } finally {
       if (client != null) {
-        xceiverClientManager.releaseClient(client);
+        xceiverClientManager.releaseClient(client, false);
       }
     }
   }
@@ -319,7 +319,7 @@ public class ContainerOperationClient implements ScmClient {
       return response.getContainerData();
     } finally {
       if (client != null) {
-        xceiverClientManager.releaseClient(client);
+        xceiverClientManager.releaseClient(client, false);
       }
     }
   }
@@ -412,7 +412,7 @@ public class ContainerOperationClient implements ScmClient {
           ObjectStageChangeRequestProto.Stage.complete);
     } finally {
       if (client != null) {
-        xceiverClientManager.releaseClient(client);
+        xceiverClientManager.releaseClient(client, false);
       }
     }
   }

+ 1 - 1
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java

@@ -141,7 +141,7 @@ public class BlockInputStream extends InputStream implements Seekable {
   @Override
   public synchronized void close() {
     if (xceiverClientManager != null && xceiverClient != null) {
-      xceiverClientManager.releaseClient(xceiverClient);
+      xceiverClientManager.releaseClient(xceiverClient, false);
       xceiverClientManager = null;
       xceiverClient = null;
     }

+ 9 - 8
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockOutputStream.java

@@ -21,6 +21,7 @@ import com.google.common.base.Preconditions;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.scm.XceiverClientAsyncReply;
 import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
 import org.apache.hadoop.ozone.common.Checksum;
 import org.apache.hadoop.ozone.common.ChecksumData;
 import org.apache.hadoop.ozone.common.OzoneChecksumException;
@@ -113,7 +114,7 @@ public class BlockOutputStream extends OutputStream {
    * @param blockID              block ID
    * @param key                  chunk key
    * @param xceiverClientManager client manager that controls client
-   * @param xceiverClient        client to perform container calls
+   * @param pipeline             pipeline where block will be written
    * @param traceID              container protocol call args
    * @param chunkSize            chunk size
    * @param bufferList           list of byte buffers
@@ -124,10 +125,10 @@ public class BlockOutputStream extends OutputStream {
    */
   @SuppressWarnings("parameternumber")
   public BlockOutputStream(BlockID blockID, String key,
-      XceiverClientManager xceiverClientManager, XceiverClientSpi xceiverClient,
+      XceiverClientManager xceiverClientManager, Pipeline pipeline,
       String traceID, int chunkSize, long streamBufferFlushSize,
-      long streamBufferMaxSize, long watchTimeout,
-      List<ByteBuffer> bufferList, Checksum checksum) {
+      long streamBufferMaxSize, long watchTimeout, List<ByteBuffer> bufferList,
+      Checksum checksum) throws IOException {
     this.blockID = blockID;
     this.key = key;
     this.traceID = traceID;
@@ -138,7 +139,7 @@ public class BlockOutputStream extends OutputStream {
         BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
             .addMetadata(keyValue);
     this.xceiverClientManager = xceiverClientManager;
-    this.xceiverClient = xceiverClient;
+    this.xceiverClient = xceiverClientManager.acquireClient(pipeline);
     this.streamId = UUID.randomUUID().toString();
     this.chunkIndex = 0;
     this.streamBufferFlushSize = streamBufferFlushSize;
@@ -500,7 +501,7 @@ public class BlockOutputStream extends OutputStream {
           throw new IOException(
               "Unexpected Storage Container Exception: " + e.toString(), e);
         } finally {
-          cleanup();
+          cleanup(false);
         }
       }
       // clear the currentBuffer
@@ -541,9 +542,9 @@ public class BlockOutputStream extends OutputStream {
     }
   }
 
-  public void cleanup() {
+  public void cleanup(boolean invalidateClient) {
     if (xceiverClientManager != null) {
-      xceiverClientManager.releaseClient(xceiverClient);
+      xceiverClientManager.releaseClient(xceiverClient, invalidateClient);
     }
     xceiverClientManager = null;
     xceiverClient = null;

+ 2 - 15
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java

@@ -256,21 +256,8 @@ public final class ContainerDataYaml {
         kvData.setMetadata(meta);
         kvData.setChecksum((String) nodes.get(OzoneConsts.CHECKSUM));
         String state = (String) nodes.get(OzoneConsts.STATE);
-        switch (state) {
-        case "OPEN":
-          kvData.setState(ContainerProtos.ContainerDataProto.State.OPEN);
-          break;
-        case "CLOSING":
-          kvData.setState(ContainerProtos.ContainerDataProto.State.CLOSING);
-          break;
-        case "CLOSED":
-          kvData.setState(ContainerProtos.ContainerDataProto.State.CLOSED);
-          break;
-        default:
-          throw new IllegalStateException("Unexpected " +
-              "ContainerLifeCycleState " + state + " for the containerId " +
-              nodes.get(OzoneConsts.CONTAINER_ID));
-        }
+        kvData
+            .setState(ContainerProtos.ContainerDataProto.State.valueOf(state));
         return kvData;
       }
     }

+ 2 - 2
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java

@@ -96,8 +96,8 @@ public class ContainerReader implements Runnable {
     try {
       readVolume(hddsVolumeDir);
     } catch (RuntimeException ex) {
-      LOG.info("Caught an Run time exception during reading container files" +
-          " from Volume {}", hddsVolumeDir);
+      LOG.error("Caught a Run time exception during reading container files" +
+          " from Volume {} {}", hddsVolumeDir, ex);
     }
   }
 

+ 14 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOpsCountStatistics.java

@@ -41,12 +41,15 @@ public class DFSOpsCountStatistics extends StorageStatistics {
 
   /** This is for counting distributed file system operations. */
   public enum OpType {
+    ADD_CACHE_DIRECTIVE("op_add_cache_directive"),
+    ADD_CACHE_POOL("op_add_cache_pool"),
     ADD_EC_POLICY("op_add_ec_policy"),
     ALLOW_SNAPSHOT("op_allow_snapshot"),
     APPEND(CommonStatisticNames.OP_APPEND),
     CONCAT("op_concat"),
     COPY_FROM_LOCAL_FILE(CommonStatisticNames.OP_COPY_FROM_LOCAL_FILE),
     CREATE(CommonStatisticNames.OP_CREATE),
+    CREATE_ENCRYPTION_ZONE("op_create_encryption_zone"),
     CREATE_NON_RECURSIVE(CommonStatisticNames.OP_CREATE_NON_RECURSIVE),
     CREATE_SNAPSHOT("op_create_snapshot"),
     CREATE_SYM_LINK("op_create_symlink"),
@@ -61,6 +64,7 @@ public class DFSOpsCountStatistics extends StorageStatistics {
     GET_EC_CODECS("op_get_ec_codecs"),
     GET_EC_POLICY("op_get_ec_policy"),
     GET_EC_POLICIES("op_get_ec_policies"),
+    GET_ENCRYPTION_ZONE("op_get_encryption_zone"),
     GET_FILE_BLOCK_LOCATIONS("op_get_file_block_locations"),
     GET_FILE_CHECKSUM(CommonStatisticNames.OP_GET_FILE_CHECKSUM),
     GET_FILE_LINK_STATUS("op_get_file_link_status"),
@@ -72,8 +76,13 @@ public class DFSOpsCountStatistics extends StorageStatistics {
     GET_STORAGE_POLICY("op_get_storage_policy"),
     GET_TRASH_ROOT("op_get_trash_root"),
     GET_XATTR("op_get_xattr"),
+    LIST_CACHE_DIRECTIVE("op_list_cache_directive"),
+    LIST_CACHE_POOL("op_list_cache_pool"),
+    LIST_ENCRYPTION_ZONE("op_list_encryption_zone"),
     LIST_LOCATED_STATUS(CommonStatisticNames.OP_LIST_LOCATED_STATUS),
     LIST_STATUS(CommonStatisticNames.OP_LIST_STATUS),
+    MODIFY_CACHE_POOL("op_modify_cache_pool"),
+    MODIFY_CACHE_DIRECTIVE("op_modify_cache_directive"),
     MKDIRS(CommonStatisticNames.OP_MKDIRS),
     MODIFY_ACL_ENTRIES(CommonStatisticNames.OP_MODIFY_ACL_ENTRIES),
     OPEN(CommonStatisticNames.OP_OPEN),
@@ -81,16 +90,21 @@ public class DFSOpsCountStatistics extends StorageStatistics {
     PRIMITIVE_MKDIR("op_primitive_mkdir"),
     REMOVE_ACL(CommonStatisticNames.OP_REMOVE_ACL),
     REMOVE_ACL_ENTRIES(CommonStatisticNames.OP_REMOVE_ACL_ENTRIES),
+    REMOVE_CACHE_DIRECTIVE("op_remove_cache_directive"),
+    REMOVE_CACHE_POOL("op_remove_cache_pool"),
     REMOVE_DEFAULT_ACL(CommonStatisticNames.OP_REMOVE_DEFAULT_ACL),
     REMOVE_EC_POLICY("op_remove_ec_policy"),
     REMOVE_XATTR("op_remove_xattr"),
     RENAME(CommonStatisticNames.OP_RENAME),
     RENAME_SNAPSHOT("op_rename_snapshot"),
     RESOLVE_LINK("op_resolve_link"),
+    SATISFY_STORAGE_POLICY("op_satisfy_storagepolicy"),
     SET_ACL(CommonStatisticNames.OP_SET_ACL),
     SET_EC_POLICY("op_set_ec_policy"),
     SET_OWNER(CommonStatisticNames.OP_SET_OWNER),
     SET_PERMISSION(CommonStatisticNames.OP_SET_PERMISSION),
+    SET_QUOTA_BYTSTORAGEYPE("op_set_quota_bystoragetype"),
+    SET_QUOTA_USAGE("op_set_quota_usage"),
     SET_REPLICATION("op_set_replication"),
     SET_STORAGE_POLICY("op_set_storagePolicy"),
     SET_TIMES(CommonStatisticNames.OP_SET_TIMES),

+ 28 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -1002,6 +1002,8 @@ public class DistributedFileSystem extends FileSystem
    */
   public void setQuota(Path src, final long namespaceQuota,
       final long storagespaceQuota) throws IOException {
+    statistics.incrementWriteOps(1);
+    storageStatistics.incrementOpCounter(OpType.SET_QUOTA_USAGE);
     Path absF = fixRelativePart(src);
     new FileSystemLinkResolver<Void>() {
       @Override
@@ -1030,6 +1032,8 @@ public class DistributedFileSystem extends FileSystem
   public void setQuotaByStorageType(Path src, final StorageType type,
       final long quota)
       throws IOException {
+    statistics.incrementWriteOps(1);
+    storageStatistics.incrementOpCounter(OpType.SET_QUOTA_BYTSTORAGEYPE);
     Path absF = fixRelativePart(src);
     new FileSystemLinkResolver<Void>() {
       @Override
@@ -2222,6 +2226,8 @@ public class DistributedFileSystem extends FileSystem
    */
   public long addCacheDirective(
       CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException {
+    statistics.incrementWriteOps(1);
+    storageStatistics.incrementOpCounter(OpType.ADD_CACHE_DIRECTIVE);
     Preconditions.checkNotNull(info.getPath());
     Path path = new Path(getPathName(fixRelativePart(info.getPath()))).
         makeQualified(getUri(), getWorkingDirectory());
@@ -2249,6 +2255,8 @@ public class DistributedFileSystem extends FileSystem
    */
   public void modifyCacheDirective(
       CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException {
+    statistics.incrementWriteOps(1);
+    storageStatistics.incrementOpCounter(OpType.MODIFY_CACHE_DIRECTIVE);
     if (info.getPath() != null) {
       info = new CacheDirectiveInfo.Builder(info).
           setPath(new Path(getPathName(fixRelativePart(info.getPath()))).
@@ -2265,6 +2273,8 @@ public class DistributedFileSystem extends FileSystem
    */
   public void removeCacheDirective(long id)
       throws IOException {
+    statistics.incrementWriteOps(1);
+    storageStatistics.incrementOpCounter(OpType.REMOVE_CACHE_DIRECTIVE);
     dfs.removeCacheDirective(id);
   }
 
@@ -2277,6 +2287,8 @@ public class DistributedFileSystem extends FileSystem
    */
   public RemoteIterator<CacheDirectiveEntry> listCacheDirectives(
       CacheDirectiveInfo filter) throws IOException {
+    statistics.incrementReadOps(1);
+    storageStatistics.incrementOpCounter(OpType.LIST_CACHE_DIRECTIVE);
     if (filter == null) {
       filter = new CacheDirectiveInfo.Builder().build();
     }
@@ -2317,6 +2329,8 @@ public class DistributedFileSystem extends FileSystem
    *          If the request could not be completed.
    */
   public void addCachePool(CachePoolInfo info) throws IOException {
+    statistics.incrementWriteOps(1);
+    storageStatistics.incrementOpCounter(OpType.ADD_CACHE_POOL);
     CachePoolInfo.validate(info);
     dfs.addCachePool(info);
   }
@@ -2330,6 +2344,8 @@ public class DistributedFileSystem extends FileSystem
    *          If the request could not be completed.
    */
   public void modifyCachePool(CachePoolInfo info) throws IOException {
+    statistics.incrementWriteOps(1);
+    storageStatistics.incrementOpCounter(OpType.MODIFY_CACHE_POOL);
     CachePoolInfo.validate(info);
     dfs.modifyCachePool(info);
   }
@@ -2343,6 +2359,8 @@ public class DistributedFileSystem extends FileSystem
    *          if the cache pool did not exist, or could not be removed.
    */
   public void removeCachePool(String poolName) throws IOException {
+    statistics.incrementWriteOps(1);
+    storageStatistics.incrementOpCounter(OpType.REMOVE_CACHE_POOL);
     CachePoolInfo.validateName(poolName);
     dfs.removeCachePool(poolName);
   }
@@ -2356,6 +2374,8 @@ public class DistributedFileSystem extends FileSystem
    *          If there was an error listing cache pools.
    */
   public RemoteIterator<CachePoolEntry> listCachePools() throws IOException {
+    statistics.incrementReadOps(1);
+    storageStatistics.incrementOpCounter(OpType.LIST_CACHE_POOL);
     return dfs.listCachePools();
   }
 
@@ -2497,6 +2517,8 @@ public class DistributedFileSystem extends FileSystem
   /* HDFS only */
   public void createEncryptionZone(final Path path, final String keyName)
       throws IOException {
+    statistics.incrementWriteOps(1);
+    storageStatistics.incrementOpCounter(OpType.CREATE_ENCRYPTION_ZONE);
     Path absF = fixRelativePart(path);
     new FileSystemLinkResolver<Void>() {
       @Override
@@ -2524,6 +2546,8 @@ public class DistributedFileSystem extends FileSystem
   /* HDFS only */
   public EncryptionZone getEZForPath(final Path path)
       throws IOException {
+    statistics.incrementReadOps(1);
+    storageStatistics.incrementOpCounter(OpType.GET_ENCRYPTION_ZONE);
     Preconditions.checkNotNull(path);
     Path absF = fixRelativePart(path);
     return new FileSystemLinkResolver<EncryptionZone>() {
@@ -2551,6 +2575,8 @@ public class DistributedFileSystem extends FileSystem
   /* HDFS only */
   public RemoteIterator<EncryptionZone> listEncryptionZones()
       throws IOException {
+    statistics.incrementReadOps(1);
+    storageStatistics.incrementOpCounter(OpType.LIST_ENCRYPTION_ZONE);
     return dfs.listEncryptionZones();
   }
 
@@ -2875,6 +2901,8 @@ public class DistributedFileSystem extends FileSystem
    * @throws IOException
    */
   public void satisfyStoragePolicy(final Path path) throws IOException {
+    statistics.incrementWriteOps(1);
+    storageStatistics.incrementOpCounter(OpType.SATISFY_STORAGE_POLICY);
     Path absF = fixRelativePart(path);
     new FileSystemLinkResolver<Void>() {
 

+ 121 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java

@@ -37,6 +37,7 @@ import java.net.InetSocketAddress;
 import java.net.ServerSocket;
 import java.net.SocketTimeoutException;
 import java.net.URI;
+import java.security.NoSuchAlgorithmException;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -46,13 +47,17 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Random;
 import java.util.Set;
+import java.util.UUID;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.key.JavaKeyStoreProvider;
+import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -78,10 +83,13 @@ import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.client.impl.LeaseRenewer;
 import org.apache.hadoop.hdfs.DFSOpsCountStatistics.OpType;
 import org.apache.hadoop.hdfs.net.Peer;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
+import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfierMode;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
@@ -775,6 +783,119 @@ public class TestDistributedFileSystem {
     }
   }
 
+  @Test
+  public void testStatistics2() throws IOException, NoSuchAlgorithmException {
+    HdfsConfiguration conf = new HdfsConfiguration();
+    conf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.EXTERNAL.toString());
+    File tmpDir = GenericTestUtils.getTestDir(UUID.randomUUID().toString());
+    final Path jksPath = new Path(tmpDir.toString(), "test.jks");
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
+        JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri());
+
+    try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build()) {
+      cluster.waitActive();
+      final DistributedFileSystem dfs = cluster.getFileSystem();
+      Path dir = new Path("/testStat");
+      dfs.mkdirs(dir);
+      int readOps = 0;
+      int writeOps = 0;
+      FileSystem.clearStatistics();
+
+      // Quota Commands.
+      long opCount = getOpStatistics(OpType.SET_QUOTA_USAGE);
+      dfs.setQuota(dir, 100, 1000);
+      checkStatistics(dfs, readOps, ++writeOps, 0);
+      checkOpStatistics(OpType.SET_QUOTA_USAGE, opCount + 1);
+
+      opCount = getOpStatistics(OpType.SET_QUOTA_BYTSTORAGEYPE);
+      dfs.setQuotaByStorageType(dir, StorageType.DEFAULT, 2000);
+      checkStatistics(dfs, readOps, ++writeOps, 0);
+      checkOpStatistics(OpType.SET_QUOTA_BYTSTORAGEYPE, opCount + 1);
+
+      opCount = getOpStatistics(OpType.GET_QUOTA_USAGE);
+      dfs.getQuotaUsage(dir);
+      checkStatistics(dfs, ++readOps, writeOps, 0);
+      checkOpStatistics(OpType.GET_QUOTA_USAGE, opCount + 1);
+
+      // Satisfy Storage Policy.
+      opCount = getOpStatistics(OpType.SATISFY_STORAGE_POLICY);
+      dfs.satisfyStoragePolicy(dir);
+      checkStatistics(dfs, readOps, ++writeOps, 0);
+      checkOpStatistics(OpType.SATISFY_STORAGE_POLICY, opCount + 1);
+
+      // Cache Commands.
+      CachePoolInfo cacheInfo =
+          new CachePoolInfo("pool1").setMode(new FsPermission((short) 0));
+
+      opCount = getOpStatistics(OpType.ADD_CACHE_POOL);
+      dfs.addCachePool(cacheInfo);
+      checkStatistics(dfs, readOps, ++writeOps, 0);
+      checkOpStatistics(OpType.ADD_CACHE_POOL, opCount + 1);
+
+      CacheDirectiveInfo directive = new CacheDirectiveInfo.Builder()
+          .setPath(new Path(".")).setPool("pool1").build();
+
+      opCount = getOpStatistics(OpType.ADD_CACHE_DIRECTIVE);
+      long id = dfs.addCacheDirective(directive);
+      checkStatistics(dfs, readOps, ++writeOps, 0);
+      checkOpStatistics(OpType.ADD_CACHE_DIRECTIVE, opCount + 1);
+
+      opCount = getOpStatistics(OpType.LIST_CACHE_DIRECTIVE);
+      dfs.listCacheDirectives(null);
+      checkStatistics(dfs, ++readOps, writeOps, 0);
+      checkOpStatistics(OpType.LIST_CACHE_DIRECTIVE, opCount + 1);
+
+      opCount = getOpStatistics(OpType.MODIFY_CACHE_DIRECTIVE);
+      dfs.modifyCacheDirective(new CacheDirectiveInfo.Builder().setId(id)
+          .setReplication((short) 2).build());
+      checkStatistics(dfs, readOps, ++writeOps, 0);
+      checkOpStatistics(OpType.MODIFY_CACHE_DIRECTIVE, opCount + 1);
+
+      opCount = getOpStatistics(OpType.REMOVE_CACHE_DIRECTIVE);
+      dfs.removeCacheDirective(id);
+      checkStatistics(dfs, readOps, ++writeOps, 0);
+      checkOpStatistics(OpType.REMOVE_CACHE_DIRECTIVE, opCount + 1);
+
+      opCount = getOpStatistics(OpType.MODIFY_CACHE_POOL);
+      dfs.modifyCachePool(cacheInfo);
+      checkStatistics(dfs, readOps, ++writeOps, 0);
+      checkOpStatistics(OpType.MODIFY_CACHE_POOL, opCount + 1);
+
+      opCount = getOpStatistics(OpType.LIST_CACHE_POOL);
+      dfs.listCachePools();
+      checkStatistics(dfs, ++readOps, writeOps, 0);
+      checkOpStatistics(OpType.LIST_CACHE_POOL, opCount + 1);
+
+      opCount = getOpStatistics(OpType.REMOVE_CACHE_POOL);
+      dfs.removeCachePool(cacheInfo.getPoolName());
+      checkStatistics(dfs, readOps, ++writeOps, 0);
+      checkOpStatistics(OpType.REMOVE_CACHE_POOL, opCount + 1);
+
+      // Crypto Commands.
+      final KeyProvider provider =
+          cluster.getNameNode().getNamesystem().getProvider();
+      final KeyProvider.Options options = KeyProvider.options(conf);
+      provider.createKey("key", options);
+      provider.flush();
+
+      opCount = getOpStatistics(OpType.CREATE_ENCRYPTION_ZONE);
+      dfs.createEncryptionZone(dir, "key");
+      checkStatistics(dfs, readOps, ++writeOps, 0);
+      checkOpStatistics(OpType.CREATE_ENCRYPTION_ZONE, opCount + 1);
+
+      opCount = getOpStatistics(OpType.LIST_ENCRYPTION_ZONE);
+      dfs.listEncryptionZones();
+      checkStatistics(dfs, ++readOps, writeOps, 0);
+      checkOpStatistics(OpType.LIST_ENCRYPTION_ZONE, opCount + 1);
+
+      opCount = getOpStatistics(OpType.GET_ENCRYPTION_ZONE);
+      dfs.getEZForPath(dir);
+      checkStatistics(dfs, ++readOps, writeOps, 0);
+      checkOpStatistics(OpType.GET_ENCRYPTION_ZONE, opCount + 1);
+    }
+  }
+
   @Test
   public void testECStatistics() throws IOException {
     try (MiniDFSCluster cluster =

+ 20 - 37
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntry.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.ozone.client.io;
 import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.hdds.scm.XceiverClientManager;
 import org.apache.hadoop.hdds.scm.XceiverClientSpi;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
 import org.apache.hadoop.hdds.scm.storage.BlockOutputStream;
 import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
 import org.apache.hadoop.ozone.common.Checksum;
@@ -40,7 +41,7 @@ public class BlockOutputStreamEntry extends OutputStream {
   private BlockID blockID;
   private final String key;
   private final XceiverClientManager xceiverClientManager;
-  private final XceiverClientSpi xceiverClient;
+  private final Pipeline pipeline;
   private final Checksum checksum;
   private final String requestId;
   private final int chunkSize;
@@ -57,7 +58,7 @@ public class BlockOutputStreamEntry extends OutputStream {
 
   private BlockOutputStreamEntry(BlockID blockID, String key,
       XceiverClientManager xceiverClientManager,
-      XceiverClientSpi xceiverClient, String requestId, int chunkSize,
+      Pipeline pipeline, String requestId, int chunkSize,
       long length, long streamBufferFlushSize, long streamBufferMaxSize,
       long watchTimeout, List<ByteBuffer> bufferList, Checksum checksum,
       Token<OzoneBlockTokenIdentifier> token) {
@@ -65,7 +66,7 @@ public class BlockOutputStreamEntry extends OutputStream {
     this.blockID = blockID;
     this.key = key;
     this.xceiverClientManager = xceiverClientManager;
-    this.xceiverClient = xceiverClient;
+    this.pipeline = pipeline;
     this.requestId = requestId;
     this.chunkSize = chunkSize;
     this.token = token;
@@ -78,31 +79,6 @@ public class BlockOutputStreamEntry extends OutputStream {
     this.checksum = checksum;
   }
 
-  /**
-   * For testing purpose, taking a some random created stream instance.
-   *
-   * @param outputStream a existing writable output stream
-   * @param length the length of data to write to the stream
-   */
-  BlockOutputStreamEntry(OutputStream outputStream, long length,
-                         Checksum checksum) {
-    this.outputStream = outputStream;
-    this.blockID = null;
-    this.key = null;
-    this.xceiverClientManager = null;
-    this.xceiverClient = null;
-    this.requestId = null;
-    this.chunkSize = -1;
-    this.token = null;
-    this.length = length;
-    this.currentPosition = 0;
-    streamBufferFlushSize = 0;
-    streamBufferMaxSize = 0;
-    bufferList = null;
-    watchTimeout = 0;
-    this.checksum = checksum;
-  }
-
   long getLength() {
     return length;
   }
@@ -115,6 +91,12 @@ public class BlockOutputStreamEntry extends OutputStream {
     return length - currentPosition;
   }
 
+  /**
+   * BlockOutputStream is initialized in this function. This makes sure that
+   * xceiverClient initialization is not done during preallocation and only
+   * done when data is written.
+   * @throws IOException if xceiverClient initialization fails
+   */
   private void checkStream() throws IOException {
     if (this.outputStream == null) {
       if (getToken() != null) {
@@ -122,11 +104,12 @@ public class BlockOutputStreamEntry extends OutputStream {
       }
       this.outputStream =
           new BlockOutputStream(blockID, key, xceiverClientManager,
-              xceiverClient, requestId, chunkSize, streamBufferFlushSize,
+              pipeline, requestId, chunkSize, streamBufferFlushSize,
               streamBufferMaxSize, watchTimeout, bufferList, checksum);
     }
   }
 
+
   @Override
   public void write(int b) throws IOException {
     checkStream();
@@ -187,11 +170,11 @@ public class BlockOutputStreamEntry extends OutputStream {
     throw new IOException("Invalid Output Stream for Key: " + key);
   }
 
-  void cleanup() throws IOException{
+  void cleanup(boolean invalidateClient) throws IOException {
     checkStream();
     if (this.outputStream instanceof BlockOutputStream) {
       BlockOutputStream out = (BlockOutputStream) this.outputStream;
-      out.cleanup();
+      out.cleanup(invalidateClient);
     }
   }
 
@@ -214,7 +197,7 @@ public class BlockOutputStreamEntry extends OutputStream {
     private BlockID blockID;
     private String key;
     private XceiverClientManager xceiverClientManager;
-    private XceiverClientSpi xceiverClient;
+    private Pipeline pipeline;
     private String requestId;
     private int chunkSize;
     private long length;
@@ -246,8 +229,8 @@ public class BlockOutputStreamEntry extends OutputStream {
       return this;
     }
 
-    public Builder setXceiverClient(XceiverClientSpi client) {
-      this.xceiverClient = client;
+    public Builder setPipeline(Pipeline pipeline) {
+      this.pipeline = pipeline;
       return this;
     }
 
@@ -293,7 +276,7 @@ public class BlockOutputStreamEntry extends OutputStream {
 
     public BlockOutputStreamEntry build() {
       return new BlockOutputStreamEntry(blockID, key,
-          xceiverClientManager, xceiverClient, requestId, chunkSize,
+          xceiverClientManager, pipeline, requestId, chunkSize,
           length, streamBufferFlushSize, streamBufferMaxSize, watchTimeout,
           bufferList, checksum, token);
     }
@@ -315,8 +298,8 @@ public class BlockOutputStreamEntry extends OutputStream {
     return xceiverClientManager;
   }
 
-  public XceiverClientSpi getXceiverClient() {
-    return xceiverClient;
+  public Pipeline getPipeline() {
+    return pipeline;
   }
 
   public Checksum getChecksum() {

+ 1 - 1
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyInputStream.java

@@ -316,7 +316,7 @@ public class KeyInputStream extends InputStream implements Seekable {
             omKeyLocationInfo.getLength());
       } finally {
         if (!success) {
-          xceiverClientManager.releaseClient(xceiverClient);
+          xceiverClientManager.releaseClient(xceiverClient, false);
         }
       }
     }

+ 31 - 29
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyOutputStream.java

@@ -21,22 +21,21 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.fs.FSExceptionMessages;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result;
-import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.scm.XceiverClientSpi;
 import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerNotOpenException;
 import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
-import org.apache.hadoop.hdds.scm.storage.BlockOutputStream;
 import org.apache.hadoop.ozone.common.Checksum;
 import org.apache.hadoop.ozone.om.helpers.*;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
 import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdds.scm.XceiverClientManager;
-import org.apache.hadoop.hdds.scm.XceiverClientSpi;
 import org.apache.hadoop.hdds.scm.container.common.helpers
     .StorageContainerException;
 import org.apache.hadoop.hdds.scm.protocolPB
     .StorageContainerLocationProtocolClientSideTranslatorPB;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.ratis.protocol.AlreadyClosedException;
 import org.apache.ratis.protocol.RaftRetryFailureException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -108,19 +107,6 @@ public class KeyOutputStream extends OutputStream {
     this.checksum = new Checksum();
   }
 
-  /**
-   * For testing purpose only. Not building output stream from blocks, but
-   * taking from externally.
-   *
-   * @param outputStream
-   * @param length
-   */
-  @VisibleForTesting
-  public void addStream(OutputStream outputStream, long length) {
-    streamEntries.add(
-        new BlockOutputStreamEntry(outputStream, length, checksum));
-  }
-
   @VisibleForTesting
   public List<BlockOutputStreamEntry> getStreamEntries() {
     return streamEntries;
@@ -223,7 +209,7 @@ public class KeyOutputStream extends OutputStream {
             .setBlockID(subKeyInfo.getBlockID())
             .setKey(keyArgs.getKeyName())
             .setXceiverClientManager(xceiverClientManager)
-            .setXceiverClient(xceiverClient)
+            .setPipeline(containerWithPipeline.getPipeline())
             .setRequestId(requestID)
             .setChunkSize(chunkSize)
             .setLength(subKeyInfo.getLength())
@@ -311,12 +297,14 @@ public class KeyOutputStream extends OutputStream {
           current.write(b, off, writeLen);
         }
       } catch (IOException ioe) {
-        if (checkIfContainerIsClosed(ioe) || checkIfTimeoutException(ioe)) {
+        boolean retryFailure = checkForRetryFailure(ioe);
+        if (checkIfContainerIsClosed(ioe) || checkIfTimeoutException(ioe)
+            || retryFailure) {
           // for the current iteration, totalDataWritten - currentPos gives the
           // amount of data already written to the buffer
           writeLen = (int) (current.getWrittenDataLength() - currentPos);
           LOG.debug("writeLen {}, total len {}", writeLen, len);
-          handleException(current, currentStreamIndex);
+          handleException(current, currentStreamIndex, retryFailure);
         } else {
           throw ioe;
         }
@@ -376,17 +364,19 @@ public class KeyOutputStream extends OutputStream {
    *
    * @param streamEntry StreamEntry
    * @param streamIndex Index of the entry
+   * @param retryFailure if true the xceiverClient needs to be invalidated in
+   *                     the client cache.
    * @throws IOException Throws IOException if Write fails
    */
   private void handleException(BlockOutputStreamEntry streamEntry,
-      int streamIndex) throws IOException {
+      int streamIndex, boolean retryFailure) throws IOException {
     long totalSuccessfulFlushedData =
         streamEntry.getTotalSuccessfulFlushedData();
     //set the correct length for the current stream
     streamEntry.setCurrentPosition(totalSuccessfulFlushedData);
     long bufferedDataLen = computeBufferData();
     // just clean up the current stream.
-    streamEntry.cleanup();
+    streamEntry.cleanup(retryFailure);
     if (bufferedDataLen > 0) {
       // If the data is still cached in the underlying stream, we need to
       // allocate new block and write this data in the datanode.
@@ -404,7 +394,7 @@ public class KeyOutputStream extends OutputStream {
 
   private boolean checkIfContainerIsClosed(IOException ioe) {
     if (ioe.getCause() != null) {
-      return checkIfContainerNotOpenOrRaftRetryFailureException(ioe) || Optional
+      return checkForException(ioe, ContainerNotOpenException.class) || Optional
           .of(ioe.getCause())
           .filter(e -> e instanceof StorageContainerException)
           .map(e -> (StorageContainerException) e)
@@ -414,13 +404,23 @@ public class KeyOutputStream extends OutputStream {
     return false;
   }
 
-  private boolean checkIfContainerNotOpenOrRaftRetryFailureException(
-      IOException ioe) {
+  /**
+   * Checks if the provided exception signifies retry failure in ratis client.
+   * In case of retry failure, ratis client throws RaftRetryFailureException
+   * and all succeeding operations are failed with AlreadyClosedException.
+   */
+  private boolean checkForRetryFailure(IOException ioe) {
+    return checkForException(ioe, RaftRetryFailureException.class,
+        AlreadyClosedException.class);
+  }
+
+  private boolean checkForException(IOException ioe, Class... classes) {
     Throwable t = ioe.getCause();
     while (t != null) {
-      if (t instanceof ContainerNotOpenException
-          || t instanceof RaftRetryFailureException) {
-        return true;
+      for (Class cls : classes) {
+        if (cls.isInstance(t)) {
+          return true;
+        }
       }
       t = t.getCause();
     }
@@ -483,11 +483,13 @@ public class KeyOutputStream extends OutputStream {
           entry.flush();
         }
       } catch (IOException ioe) {
-        if (checkIfContainerIsClosed(ioe) || checkIfTimeoutException(ioe)) {
+        boolean retryFailure = checkForRetryFailure(ioe);
+        if (checkIfContainerIsClosed(ioe) || checkIfTimeoutException(ioe)
+            || retryFailure) {
           // This call will allocate a new streamEntry and write the Data.
           // Close needs to be retried on the newly allocated streamEntry as
           // as well.
-          handleException(entry, streamIndex);
+          handleException(entry, streamIndex, retryFailure);
           handleFlushOrClose(close);
         } else {
           throw ioe;

+ 1 - 1
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerStateMachineIdempotency.java

@@ -119,6 +119,6 @@ public class TestContainerStateMachineIdempotency {
     } catch (IOException ioe) {
       Assert.fail("Container operation failed" + ioe);
     }
-    xceiverClientManager.releaseClient(client);
+    xceiverClientManager.releaseClient(client, false);
   }
 }

+ 1 - 1
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java

@@ -729,7 +729,7 @@ public abstract class TestOzoneRpcClientAbstract {
       Assert.assertTrue(
           e.getMessage().contains("on the pipeline " + pipeline.getId()));
     }
-    manager.releaseClient(clientSpi);
+    manager.releaseClient(clientSpi, false);
   }
 
   private void readKey(OzoneBucket bucket, String keyName, String data)

+ 4 - 4
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSmallFile.java

@@ -98,7 +98,7 @@ public class TestContainerSmallFile {
         ContainerProtocolCalls.readSmallFile(client, blockID, traceID);
     String readData = response.getData().getData().toStringUtf8();
     Assert.assertEquals("data123", readData);
-    xceiverClientManager.releaseClient(client);
+    xceiverClientManager.releaseClient(client, false);
   }
 
   @Test
@@ -121,7 +121,7 @@ public class TestContainerSmallFile {
     // Try to read a Key Container Name
     ContainerProtos.GetSmallFileResponseProto response =
         ContainerProtocolCalls.readSmallFile(client, blockID, traceID);
-    xceiverClientManager.releaseClient(client);
+    xceiverClientManager.releaseClient(client, false);
   }
 
   @Test
@@ -149,7 +149,7 @@ public class TestContainerSmallFile {
         ContainerProtocolCalls.readSmallFile(client,
             ContainerTestHelper.getTestBlockID(
                 nonExistContainerID), traceID);
-    xceiverClientManager.releaseClient(client);
+    xceiverClientManager.releaseClient(client, false);
   }
 
   @Test
@@ -202,7 +202,7 @@ public class TestContainerSmallFile {
         ContainerProtocolCalls.readSmallFile(client, blockID1, traceID);
     String readData = response.getData().getData().toStringUtf8();
     Assert.assertEquals("data123", readData);
-    xceiverClientManager.releaseClient(client);
+    xceiverClientManager.releaseClient(client, false);
   }
 }
 

+ 3 - 3
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestGetCommittedBlockLengthAndPutKey.java

@@ -114,7 +114,7 @@ public class TestGetCommittedBlockLengthAndPutKey {
     Assert.assertTrue(
         BlockID.getFromProtobuf(response.getBlockID()).equals(blockID));
     Assert.assertTrue(response.getBlockLength() == data.length);
-    xceiverClientManager.releaseClient(client);
+    xceiverClientManager.releaseClient(client, false);
   }
 
   @Test
@@ -139,7 +139,7 @@ public class TestGetCommittedBlockLengthAndPutKey {
     } catch (StorageContainerException sce) {
       Assert.assertTrue(sce.getMessage().contains("Unable to find the block"));
     }
-    xceiverClientManager.releaseClient(client);
+    xceiverClientManager.releaseClient(client, false);
   }
 
   @Test
@@ -180,6 +180,6 @@ public class TestGetCommittedBlockLengthAndPutKey {
     // This will also ensure that closing the container committed the block
     // on the Datanodes.
     Assert.assertEquals(responseBlockID, blockID);
-    xceiverClientManager.releaseClient(client);
+    xceiverClientManager.releaseClient(client, false);
   }
 }

+ 45 - 7
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestXceiverClientManager.java

@@ -103,9 +103,9 @@ public class TestXceiverClientManager {
     Assert.assertEquals(2, client3.getRefcount());
     Assert.assertEquals(2, client1.getRefcount());
     Assert.assertEquals(client1, client3);
-    clientManager.releaseClient(client1);
-    clientManager.releaseClient(client2);
-    clientManager.releaseClient(client3);
+    clientManager.releaseClient(client1, false);
+    clientManager.releaseClient(client2, false);
+    clientManager.releaseClient(client3, false);
   }
 
   @Test
@@ -150,7 +150,7 @@ public class TestXceiverClientManager {
 
     // After releasing the client, this connection should be closed
     // and any container operations should fail
-    clientManager.releaseClient(client1);
+    clientManager.releaseClient(client1, false);
 
     String expectedMessage = "This channel is not connected.";
     try {
@@ -162,7 +162,7 @@ public class TestXceiverClientManager {
       Assert.assertEquals(e.getClass(), IOException.class);
       Assert.assertTrue(e.getMessage().contains(expectedMessage));
     }
-    clientManager.releaseClient(client2);
+    clientManager.releaseClient(client2, false);
   }
 
   @Test
@@ -184,7 +184,7 @@ public class TestXceiverClientManager {
         .acquireClient(container1.getPipeline());
     Assert.assertEquals(1, client1.getRefcount());
 
-    clientManager.releaseClient(client1);
+    clientManager.releaseClient(client1, false);
     Assert.assertEquals(0, client1.getRefcount());
 
     ContainerWithPipeline container2 = storageContainerLocationClient
@@ -213,6 +213,44 @@ public class TestXceiverClientManager {
       Assert.assertEquals(e.getClass(), IOException.class);
       Assert.assertTrue(e.getMessage().contains(expectedMessage));
     }
-    clientManager.releaseClient(client2);
+    clientManager.releaseClient(client2, false);
+  }
+
+  @Test
+  public void testFreeByRetryFailure() throws IOException {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    conf.setInt(SCM_CONTAINER_CLIENT_MAX_SIZE_KEY, 1);
+    XceiverClientManager clientManager = new XceiverClientManager(conf);
+    Cache<String, XceiverClientSpi> cache =
+        clientManager.getClientCache();
+
+    // client is added in cache
+    ContainerWithPipeline container1 = storageContainerLocationClient
+        .allocateContainer(clientManager.getType(), clientManager.getFactor(),
+            containerOwner);
+    XceiverClientSpi client1 =
+        clientManager.acquireClient(container1.getPipeline());
+    clientManager.acquireClient(container1.getPipeline());
+    Assert.assertEquals(2, client1.getRefcount());
+
+    // client should be invalidated in the cache
+    clientManager.releaseClient(client1, true);
+    Assert.assertEquals(1, client1.getRefcount());
+    Assert.assertNull(cache.getIfPresent(
+        container1.getContainerInfo().getPipelineID().getId().toString()
+            + container1.getContainerInfo().getReplicationType()));
+
+    // new client should be added in cache
+    XceiverClientSpi client2 =
+        clientManager.acquireClient(container1.getPipeline());
+    Assert.assertNotEquals(client1, client2);
+    Assert.assertEquals(1, client2.getRefcount());
+
+    // on releasing the old client the entry in cache should not be invalidated
+    clientManager.releaseClient(client1, true);
+    Assert.assertEquals(0, client1.getRefcount());
+    Assert.assertNotNull(cache.getIfPresent(
+        container1.getContainerInfo().getPipelineID().getId().toString()
+            + container1.getContainerInfo().getReplicationType()));
   }
 }

+ 12 - 8
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java

@@ -46,17 +46,16 @@ import org.apache.hadoop.ozone.s3.endpoint.MultiDeleteResponse.DeletedObject;
 import org.apache.hadoop.ozone.s3.endpoint.MultiDeleteResponse.Error;
 import org.apache.hadoop.ozone.s3.exception.OS3Exception;
 import org.apache.hadoop.ozone.s3.exception.S3ErrorTable;
+import org.apache.hadoop.ozone.s3.util.ContinueToken;
+import org.apache.hadoop.ozone.s3.util.S3StorageType;
 
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 import org.apache.commons.lang3.StringUtils;
-import org.apache.hadoop.ozone.s3.util.S3StorageType;
-import org.apache.hadoop.ozone.s3.util.S3utils;
+import static org.apache.hadoop.ozone.s3.util.S3Consts.ENCODING_TYPE;
 import org.apache.http.HttpStatus;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.hadoop.ozone.s3.util.S3Consts.ENCODING_TYPE;
-
 /**
  * Bucket level rest endpoints.
  */
@@ -104,16 +103,17 @@ public class BucketEndpoint extends EndpointBase {
 
     Iterator<? extends OzoneKey> ozoneKeyIterator;
 
-    String decodedToken = S3utils.decodeContinueToken(continueToken);
+    ContinueToken decodedToken =
+        ContinueToken.decodeFromString(continueToken);
 
     if (startAfter != null && continueToken != null) {
       // If continuation token and start after both are provided, then we
       // ignore start After
-      ozoneKeyIterator = bucket.listKeys(prefix, decodedToken);
+      ozoneKeyIterator = bucket.listKeys(prefix, decodedToken.getLastKey());
     } else if (startAfter != null && continueToken == null) {
       ozoneKeyIterator = bucket.listKeys(prefix, startAfter);
     } else if (startAfter == null && continueToken != null){
-      ozoneKeyIterator = bucket.listKeys(prefix, decodedToken);
+      ozoneKeyIterator = bucket.listKeys(prefix, decodedToken.getLastKey());
     } else {
       ozoneKeyIterator = bucket.listKeys(prefix);
     }
@@ -130,6 +130,9 @@ public class BucketEndpoint extends EndpointBase {
     response.setContinueToken(continueToken);
 
     String prevDir = null;
+    if (continueToken != null) {
+      prevDir = decodedToken.getLastDir();
+    }
     String lastKey = null;
     int count = 0;
     while (ozoneKeyIterator.hasNext()) {
@@ -176,7 +179,8 @@ public class BucketEndpoint extends EndpointBase {
       response.setTruncated(false);
     } else if(ozoneKeyIterator.hasNext()) {
       response.setTruncated(true);
-      response.setNextToken(S3utils.generateContinueToken(lastKey));
+      ContinueToken nextToken = new ContinueToken(lastKey, prevDir);
+      response.setNextToken(nextToken.encodeToString());
     } else {
       response.setTruncated(false);
     }

+ 2 - 2
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ObjectEndpoint.java

@@ -60,8 +60,8 @@ import org.apache.hadoop.ozone.s3.exception.S3ErrorTable;
 import org.apache.hadoop.ozone.s3.io.S3WrapperInputStream;
 import org.apache.hadoop.ozone.s3.util.RFC1123Util;
 import org.apache.hadoop.ozone.s3.util.RangeHeader;
+import org.apache.hadoop.ozone.s3.util.RangeHeaderParserUtil;
 import org.apache.hadoop.ozone.s3.util.S3StorageType;
-import org.apache.hadoop.ozone.s3.util.S3utils;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 import org.apache.hadoop.util.Time;
 
@@ -210,7 +210,7 @@ public class ObjectEndpoint extends EndpointBase {
       LOG.debug("range Header provided value is {}", rangeHeaderVal);
 
       if (rangeHeaderVal != null) {
-        rangeHeader = S3utils.parseRangeHeader(rangeHeaderVal,
+        rangeHeader = RangeHeaderParserUtil.parseRangeHeader(rangeHeaderVal,
             length);
         LOG.debug("range Header provided value is {}", rangeHeader);
         if (rangeHeader.isInValidRange()) {

+ 173 - 0
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/ContinueToken.java

@@ -0,0 +1,173 @@
+/**
+ * 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.s3.util;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Objects;
+
+import org.apache.hadoop.ozone.s3.exception.OS3Exception;
+import org.apache.hadoop.ozone.s3.exception.S3ErrorTable;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+
+/**
+ * Token which holds enough information to continue the key iteration.
+ */
+public class ContinueToken {
+
+  private String lastKey;
+
+  private String lastDir;
+
+  private static final String CONTINUE_TOKEN_SEPERATOR = "-";
+
+  public ContinueToken(String lastKey, String lastDir) {
+    Preconditions.checkNotNull(lastKey,
+        "The last key can't be null in the continue token.");
+    this.lastKey = lastKey;
+    if (lastDir != null && lastDir.length() > 0) {
+      this.lastDir = lastDir;
+    }
+  }
+
+  /**
+   * Generate a continuation token which is used in get Bucket.
+   *
+   * @return if key is not null return continuation token, else returns null.
+   */
+  public String encodeToString() {
+    if (this.lastKey != null) {
+
+      ByteBuffer buffer = ByteBuffer
+          .allocate(4 + lastKey.length()
+              + (lastDir == null ? 0 : lastDir.length()));
+      buffer.putInt(lastKey.length());
+      buffer.put(lastKey.getBytes(StandardCharsets.UTF_8));
+      if (lastDir != null) {
+        buffer.put(lastDir.getBytes(StandardCharsets.UTF_8));
+      }
+
+      String hex = Hex.encodeHexString(buffer.array());
+      String digest = DigestUtils.sha256Hex(hex);
+      return hex + CONTINUE_TOKEN_SEPERATOR + digest;
+    } else {
+      return null;
+    }
+  }
+
+  /**
+   * Decode a continuation token which is used in get Bucket.
+   *
+   * @param key
+   * @return if key is not null return decoded token, otherwise returns null.
+   * @throws OS3Exception
+   */
+  public static ContinueToken decodeFromString(String key) throws OS3Exception {
+    if (key != null) {
+      int indexSeparator = key.indexOf(CONTINUE_TOKEN_SEPERATOR);
+      if (indexSeparator == -1) {
+        throw S3ErrorTable.newError(S3ErrorTable.INVALID_ARGUMENT, key);
+      }
+      String hex = key.substring(0, indexSeparator);
+      String digest = key.substring(indexSeparator + 1);
+      try {
+        checkHash(key, hex, digest);
+
+        ByteBuffer buffer = ByteBuffer.wrap(Hex.decodeHex(hex));
+        int keySize = buffer.getInt();
+
+        byte[] actualKeyBytes = new byte[keySize];
+        buffer.get(actualKeyBytes);
+
+        byte[] actualDirBytes = new byte[buffer.remaining()];
+        buffer.get(actualDirBytes);
+
+        return new ContinueToken(
+            new String(actualKeyBytes, StandardCharsets.UTF_8),
+            new String(actualDirBytes, StandardCharsets.UTF_8)
+        );
+
+      } catch (DecoderException ex) {
+        OS3Exception os3Exception = S3ErrorTable.newError(S3ErrorTable
+            .INVALID_ARGUMENT, key);
+        os3Exception.setErrorMessage("The continuation token provided is " +
+            "incorrect");
+        throw os3Exception;
+      }
+    } else {
+      return null;
+    }
+  }
+
+  private static void checkHash(String key, String hex, String digest)
+      throws OS3Exception {
+    String digestActualKey = DigestUtils.sha256Hex(hex);
+    if (!digest.equals(digestActualKey)) {
+      OS3Exception ex = S3ErrorTable.newError(S3ErrorTable
+          .INVALID_ARGUMENT, key);
+      ex.setErrorMessage("The continuation token provided is incorrect");
+      throw ex;
+    }
+  }
+
+  public String getLastKey() {
+    return lastKey;
+  }
+
+  public void setLastKey(String lastKey) {
+    this.lastKey = lastKey;
+  }
+
+  public String getLastDir() {
+    return lastDir;
+  }
+
+  public void setLastDir(String lastDir) {
+    this.lastDir = lastDir;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    ContinueToken that = (ContinueToken) o;
+    return lastKey.equals(that.lastKey) &&
+        Objects.equals(lastDir, that.lastDir);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(lastKey);
+  }
+
+  @Override
+  public String toString() {
+    return "ContinueToken{" +
+        "lastKey='" + lastKey + '\'' +
+        ", lastDir='" + lastDir + '\'' +
+        '}';
+  }
+}

+ 5 - 69
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/S3utils.java → hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/RangeHeaderParserUtil.java

@@ -18,83 +18,19 @@
  */
 package org.apache.hadoop.ozone.s3.util;
 
-import org.apache.commons.codec.DecoderException;
-import org.apache.commons.codec.binary.Hex;
-import org.apache.commons.codec.digest.DigestUtils;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.ozone.s3.exception.OS3Exception;
-import org.apache.hadoop.ozone.s3.exception.S3ErrorTable;
-
-
-import java.nio.charset.StandardCharsets;
 import java.util.regex.Matcher;
 
-import static org.apache.hadoop.ozone.s3.util.S3Consts
-    .RANGE_HEADER_MATCH_PATTERN;
+import org.apache.hadoop.classification.InterfaceAudience;
+
+import static org.apache.hadoop.ozone.s3.util.S3Consts.RANGE_HEADER_MATCH_PATTERN;
 /**
  * Utility class for S3.
  */
 @InterfaceAudience.Private
-public final class S3utils {
-
-  private S3utils() {
+public final class RangeHeaderParserUtil {
 
+  private RangeHeaderParserUtil() {
   }
-  private static final String CONTINUE_TOKEN_SEPERATOR = "-";
-
-  /**
-   * Generate a continuation token which is used in get Bucket.
-   * @param key
-   * @return if key is not null return continuation token, else returns null.
-   */
-  public static String generateContinueToken(String key) {
-    if (key != null) {
-      byte[] byteData = key.getBytes(StandardCharsets.UTF_8);
-      String hex = Hex.encodeHexString(byteData);
-      String digest = DigestUtils.sha256Hex(key);
-      return hex + CONTINUE_TOKEN_SEPERATOR + digest;
-    } else {
-      return null;
-    }
-  }
-
-  /**
-   * Decode a continuation token which is used in get Bucket.
-   * @param key
-   * @return if key is not null return decoded token, otherwise returns null.
-   * @throws OS3Exception
-   */
-  public static String decodeContinueToken(String key) throws OS3Exception {
-    if (key != null) {
-      int indexSeparator = key.indexOf(CONTINUE_TOKEN_SEPERATOR);
-      if (indexSeparator == -1) {
-        throw S3ErrorTable.newError(S3ErrorTable.INVALID_ARGUMENT, key);
-      }
-      String hex = key.substring(0, indexSeparator);
-      String digest = key.substring(indexSeparator + 1);
-      try {
-        byte[] actualKeyBytes = Hex.decodeHex(hex);
-        String digestActualKey = DigestUtils.sha256Hex(actualKeyBytes);
-        if (digest.equals(digestActualKey)) {
-          return new String(actualKeyBytes, StandardCharsets.UTF_8);
-        } else {
-          OS3Exception ex = S3ErrorTable.newError(S3ErrorTable
-              .INVALID_ARGUMENT, key);
-          ex.setErrorMessage("The continuation token provided is incorrect");
-          throw ex;
-        }
-      } catch (DecoderException ex) {
-        OS3Exception os3Exception = S3ErrorTable.newError(S3ErrorTable
-            .INVALID_ARGUMENT, key);
-        os3Exception.setErrorMessage("The continuation token provided is " +
-            "incorrect");
-        throw os3Exception;
-      }
-    } else {
-      return null;
-    }
-  }
-
 
   /**
    * Parse the rangeHeader and set the start and end offset.

+ 47 - 2
hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketGet.java

@@ -211,6 +211,53 @@ public class TestBucketGet {
 
   }
 
+  @Test
+  public void listWithContinuationTokenDirBreak()
+      throws OS3Exception, IOException {
+
+    BucketEndpoint getBucket = new BucketEndpoint();
+
+    OzoneClient ozoneClient =
+        createClientWithKeys(
+            "test/dir1/file1",
+            "test/dir1/file2",
+            "test/dir1/file3",
+            "test/dir2/file4",
+            "test/dir2/file5",
+            "test/dir2/file6",
+            "test/dir3/file7",
+            "test/file8");
+
+    getBucket.setClient(ozoneClient);
+
+    int maxKeys = 2;
+
+    ListObjectResponse getBucketResponse;
+
+    getBucketResponse =
+        (ListObjectResponse) getBucket.list("b1", "/", null, null, maxKeys,
+            "test/", null, null, null, null).getEntity();
+
+    Assert.assertEquals(0, getBucketResponse.getContents().size());
+    Assert.assertEquals(2, getBucketResponse.getCommonPrefixes().size());
+    Assert.assertEquals("test/dir1/",
+        getBucketResponse.getCommonPrefixes().get(0).getPrefix());
+    Assert.assertEquals("test/dir2/",
+        getBucketResponse.getCommonPrefixes().get(1).getPrefix());
+
+    getBucketResponse =
+        (ListObjectResponse) getBucket.list("b1", "/", null, null, maxKeys,
+            "test/", null, getBucketResponse.getNextToken(), null, null)
+            .getEntity();
+    Assert.assertEquals(1, getBucketResponse.getContents().size());
+    Assert.assertEquals(1, getBucketResponse.getCommonPrefixes().size());
+    Assert.assertEquals("test/dir3/",
+        getBucketResponse.getCommonPrefixes().get(0).getPrefix());
+    Assert.assertEquals("test/file8",
+        getBucketResponse.getContents().get(0).getKey());
+
+  }
+
   @Test
   /**
    * This test is with prefix and delimiter and verify continuation-token
@@ -237,7 +284,6 @@ public class TestBucketGet {
     Assert.assertTrue(getBucketResponse.isTruncated());
     Assert.assertTrue(getBucketResponse.getCommonPrefixes().size() == 2);
 
-
     // 2nd time
     String continueToken = getBucketResponse.getNextToken();
     getBucketResponse =
@@ -246,7 +292,6 @@ public class TestBucketGet {
     Assert.assertTrue(getBucketResponse.isTruncated());
     Assert.assertTrue(getBucketResponse.getCommonPrefixes().size() == 2);
 
-
     //3rd time
     continueToken = getBucketResponse.getNextToken();
     getBucketResponse =

+ 50 - 0
hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/util/TestContinueToken.java

@@ -0,0 +1,50 @@
+/**
+ * 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.s3.util;
+
+import org.apache.hadoop.ozone.s3.exception.OS3Exception;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test encode/decode of the continue token.
+ */
+public class TestContinueToken {
+
+  @Test
+  public void encodeDecode() throws OS3Exception {
+    ContinueToken ct = new ContinueToken("key1", "dir1");
+
+    ContinueToken parsedToken =
+        ContinueToken.decodeFromString(ct.encodeToString());
+
+    Assert.assertEquals(ct, parsedToken);
+  }
+
+  @Test
+  public void encodeDecodeNullDir() throws OS3Exception {
+    ContinueToken ct = new ContinueToken("key1", null);
+
+    ContinueToken parsedToken =
+        ContinueToken.decodeFromString(ct.encodeToString());
+
+    Assert.assertEquals(ct, parsedToken);
+  }
+
+}

+ 10 - 10
hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/util/TestS3utils.java → hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/util/TestRangeHeaderParserUtil.java

@@ -23,9 +23,9 @@ import org.junit.Test;
 import static org.junit.Assert.assertEquals;
 
 /**
- * Test class to test S3utils.
+ * Test class to test RangeHeaderParserUtil.
  */
-public class TestS3utils {
+public class TestRangeHeaderParserUtil {
 
   @Test
   public void testRangeHeaderParser() {
@@ -34,14 +34,14 @@ public class TestS3utils {
 
 
     //range is with in file length
-    rangeHeader = S3utils.parseRangeHeader("bytes=0-8", 10);
+    rangeHeader = RangeHeaderParserUtil.parseRangeHeader("bytes=0-8", 10);
     assertEquals(0, rangeHeader.getStartOffset());
     assertEquals(8, rangeHeader.getEndOffset());
     assertEquals(false, rangeHeader.isReadFull());
     assertEquals(false, rangeHeader.isInValidRange());
 
     //range is with in file length, both start and end offset are same
-    rangeHeader = S3utils.parseRangeHeader("bytes=0-0", 10);
+    rangeHeader = RangeHeaderParserUtil.parseRangeHeader("bytes=0-0", 10);
     assertEquals(0, rangeHeader.getStartOffset());
     assertEquals(0, rangeHeader.getEndOffset());
     assertEquals(false, rangeHeader.isReadFull());
@@ -49,39 +49,39 @@ public class TestS3utils {
 
     //range is not with in file length, both start and end offset are greater
     // than length
-    rangeHeader = S3utils.parseRangeHeader("bytes=11-10", 10);
+    rangeHeader = RangeHeaderParserUtil.parseRangeHeader("bytes=11-10", 10);
     assertEquals(true, rangeHeader.isInValidRange());
 
     // range is satisfying, one of the range is with in the length. So, read
     // full file
-    rangeHeader = S3utils.parseRangeHeader("bytes=11-8", 10);
+    rangeHeader = RangeHeaderParserUtil.parseRangeHeader("bytes=11-8", 10);
     assertEquals(0, rangeHeader.getStartOffset());
     assertEquals(9, rangeHeader.getEndOffset());
     assertEquals(true, rangeHeader.isReadFull());
     assertEquals(false, rangeHeader.isInValidRange());
 
     // bytes spec is wrong
-    rangeHeader = S3utils.parseRangeHeader("mb=11-8", 10);
+    rangeHeader = RangeHeaderParserUtil.parseRangeHeader("mb=11-8", 10);
     assertEquals(0, rangeHeader.getStartOffset());
     assertEquals(9, rangeHeader.getEndOffset());
     assertEquals(true, rangeHeader.isReadFull());
     assertEquals(false, rangeHeader.isInValidRange());
 
     // range specified is invalid
-    rangeHeader = S3utils.parseRangeHeader("bytes=-11-8", 10);
+    rangeHeader = RangeHeaderParserUtil.parseRangeHeader("bytes=-11-8", 10);
     assertEquals(0, rangeHeader.getStartOffset());
     assertEquals(9, rangeHeader.getEndOffset());
     assertEquals(true, rangeHeader.isReadFull());
     assertEquals(false, rangeHeader.isInValidRange());
 
     //Last n bytes
-    rangeHeader = S3utils.parseRangeHeader("bytes=-6", 10);
+    rangeHeader = RangeHeaderParserUtil.parseRangeHeader("bytes=-6", 10);
     assertEquals(4, rangeHeader.getStartOffset());
     assertEquals(9, rangeHeader.getEndOffset());
     assertEquals(false, rangeHeader.isReadFull());
     assertEquals(false, rangeHeader.isInValidRange());
 
-    rangeHeader = S3utils.parseRangeHeader("bytes=-106", 10);
+    rangeHeader = RangeHeaderParserUtil.parseRangeHeader("bytes=-106", 10);
     assertEquals(0, rangeHeader.getStartOffset());
     assertEquals(9, rangeHeader.getEndOffset());
     assertEquals(false, rangeHeader.isInValidRange());

+ 7 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java

@@ -2697,6 +2697,13 @@ public class YarnConfiguration extends Configuration {
       "org.apache.hadoop.yarn.server.timelineservice.storage" +
           ".HBaseTimelineReaderImpl";
 
+  public static final String TIMELINE_SERVICE_SCHEMA_CREATOR_CLASS =
+      TIMELINE_SERVICE_PREFIX + "schema-creator.class";
+
+  public static final String DEFAULT_TIMELINE_SERVICE_SCHEMA_CREATOR_CLASS =
+      "org.apache.hadoop.yarn.server.timelineservice.storage" +
+          ".HBaseTimelineSchemaCreator";
+
   /**
    * default schema prefix for hbase tables.
    */

+ 6 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstance.java

@@ -350,8 +350,10 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
         // record in ATS
         LOG.info("Publishing component instance status {} {} ",
             event.getContainerId(), containerState);
+        int exitStatus = failureBeforeLaunch || event.getStatus() == null ?
+            ContainerExitStatus.INVALID : event.getStatus().getExitStatus();
         compInstance.serviceTimelinePublisher.componentInstanceFinished(
-            event.getContainerId(), event.getStatus().getExitStatus(),
+            event.getContainerId(), exitStatus,
             containerState, containerDiag);
       }
 
@@ -366,8 +368,10 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
 
       if (compInstance.timelineServiceEnabled) {
         // record in ATS
+        int exitStatus = failureBeforeLaunch || event.getStatus() == null ?
+            ContainerExitStatus.INVALID : event.getStatus().getExitStatus();
         compInstance.serviceTimelinePublisher.componentInstanceFinished(
-            event.getContainerId(), event.getStatus().getExitStatus(),
+            event.getContainerId(), exitStatus,
             containerState, containerDiag);
       }
 

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml

@@ -539,8 +539,8 @@
 
       This property allows users to set ACLs of their choice instead of using
       the default mechanism. For fencing to work, the ACLs should be
-      carefully set differently on each ResourceManger such that all the
-      ResourceManagers have shared admin access and the Active ResourceManger
+      carefully set differently on each ResourceManager such that all the
+      ResourceManagers have shared admin access and the Active ResourceManager
       takes over (exclusively) the create-delete access.
     </description>
     <name>yarn.resourcemanager.zk-state-store.root-node.acl</name>

+ 11 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java

@@ -437,9 +437,11 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
           RMAppAttemptState.FAILED,
           EnumSet.of(
               RMAppAttemptEventType.LAUNCHED,
+              RMAppAttemptEventType.LAUNCH_FAILED,
               RMAppAttemptEventType.EXPIRE,
               RMAppAttemptEventType.KILL,
               RMAppAttemptEventType.FAIL,
+              RMAppAttemptEventType.REGISTERED,
               RMAppAttemptEventType.UNREGISTERED,
               RMAppAttemptEventType.STATUS_UPDATE,
               RMAppAttemptEventType.CONTAINER_ALLOCATED))
@@ -1203,10 +1205,16 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
       }
 
       // Set the masterContainer
-      appAttempt.setMasterContainer(amContainerAllocation.getContainers()
-          .get(0));
+      Container amContainer = amContainerAllocation.getContainers().get(0);
       RMContainerImpl rmMasterContainer = (RMContainerImpl)appAttempt.scheduler
-          .getRMContainer(appAttempt.getMasterContainer().getId());
+          .getRMContainer(amContainer.getId());
+      //while one NM is removed, the scheduler will clean the container,the
+      //following CONTAINER_FINISHED event will handle the cleaned container.
+      //so just return RMAppAttemptState.SCHEDULED
+      if (rmMasterContainer == null) {
+        return RMAppAttemptState.SCHEDULED;
+      }
+      appAttempt.setMasterContainer(amContainer);
       rmMasterContainer.setAMContainer(true);
       // The node set in NMTokenSecrentManager is used for marking whether the
       // NMToken has been issued for this node to the AM.

+ 79 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java

@@ -986,7 +986,7 @@ public class TestRMAppAttemptTransitions {
   public void testAttemptAddedAtFinalSaving() {
     submitApplicationAttempt();
 
-    // SUBNITED->FINAL_SAVING
+    // SUBMITTED->FINAL_SAVING
     applicationAttempt.handle(new RMAppAttemptEvent(applicationAttempt
                    .getAppAttemptId(), RMAppAttemptEventType.KILL));
     assertEquals(RMAppAttemptState.FINAL_SAVING,
@@ -999,6 +999,56 @@ public class TestRMAppAttemptTransitions {
                    applicationAttempt.getAppAttemptState());
   }
 
+  @Test(timeout = 10000)
+  public void testAttemptRegisteredAtFailed() {
+    Container amContainer = allocateApplicationAttempt();
+    launchApplicationAttempt(amContainer);
+
+    //send CONTAINER_FINISHED event
+    NodeId anyNodeId = NodeId.newInstance("host", 1234);
+    applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
+        applicationAttempt.getAppAttemptId(), BuilderUtils.newContainerStatus(
+        amContainer.getId(), ContainerState.COMPLETE, "", 0,
+        amContainer.getResource()), anyNodeId));
+    assertEquals(RMAppAttemptState.FINAL_SAVING,
+        applicationAttempt.getAppAttemptState());
+
+    sendAttemptUpdateSavedEvent(applicationAttempt);
+    assertEquals(RMAppAttemptState.FAILED,
+        applicationAttempt.getAppAttemptState());
+
+    //send REGISTERED event
+    applicationAttempt.handle(new RMAppAttemptEvent(applicationAttempt
+        .getAppAttemptId(), RMAppAttemptEventType.REGISTERED));
+
+    assertEquals(RMAppAttemptState.FAILED,
+        applicationAttempt.getAppAttemptState());
+  }
+
+  @Test
+  public void testAttemptLaunchFailedAtFailed() {
+    Container amContainer = allocateApplicationAttempt();
+    launchApplicationAttempt(amContainer);
+    //send CONTAINER_FINISHED event
+    NodeId anyNodeId = NodeId.newInstance("host", 1234);
+    applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
+        applicationAttempt.getAppAttemptId(), BuilderUtils.newContainerStatus(
+        amContainer.getId(), ContainerState.COMPLETE, "", 0,
+        amContainer.getResource()), anyNodeId));
+    assertEquals(RMAppAttemptState.FINAL_SAVING,
+        applicationAttempt.getAppAttemptState());
+    sendAttemptUpdateSavedEvent(applicationAttempt);
+    assertEquals(RMAppAttemptState.FAILED,
+        applicationAttempt.getAppAttemptState());
+
+    //send LAUNCH_FAILED event
+    applicationAttempt.handle(new RMAppAttemptEvent(applicationAttempt
+        .getAppAttemptId(), RMAppAttemptEventType.LAUNCH_FAILED));
+
+    assertEquals(RMAppAttemptState.FAILED,
+        applicationAttempt.getAppAttemptState());
+  }
+
   @Test
   public void testAMCrashAtAllocated() {
     Container amContainer = allocateApplicationAttempt();
@@ -1598,6 +1648,34 @@ public class TestRMAppAttemptTransitions {
     assertTrue(found);
   }
 
+  @Test
+  public void testContainerRemovedBeforeAllocate() {
+    scheduleApplicationAttempt();
+
+    // Mock the allocation of AM container
+    Container container = mock(Container.class);
+    Resource resource = BuilderUtils.newResource(2048, 1);
+    when(container.getId()).thenReturn(
+        BuilderUtils.newContainerId(applicationAttempt.getAppAttemptId(), 1));
+    when(container.getResource()).thenReturn(resource);
+    Allocation allocation = mock(Allocation.class);
+    when(allocation.getContainers()).
+        thenReturn(Collections.singletonList(container));
+    when(scheduler.allocate(any(ApplicationAttemptId.class), any(List.class),
+        any(List.class), any(List.class), any(List.class), any(List.class),
+        any(ContainerUpdates.class))).
+        thenReturn(allocation);
+
+    //container removed, so return null
+    when(scheduler.getRMContainer(container.getId())).
+        thenReturn(null);
+
+    applicationAttempt.handle(
+        new RMAppAttemptEvent(applicationAttempt.getAppAttemptId(),
+            RMAppAttemptEventType.CONTAINER_ALLOCATED));
+    assertEquals(RMAppAttemptState.SCHEDULED,
+        applicationAttempt.getAppAttemptState());
+  }
 
   @SuppressWarnings("deprecation")
   @Test

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/DataGeneratorForTest.java

@@ -57,7 +57,7 @@ public final class DataGeneratorForTest {
     // the coprocessor class is loaded from classpath
     conf.set(YarnConfiguration.FLOW_RUN_COPROCESSOR_JAR_HDFS_LOCATION, " ");
     // now create all tables
-    TimelineSchemaCreator.createAllTables(conf, false);
+    HBaseTimelineSchemaCreator.createAllTables(conf, false);
   }
 
   public static void loadApps(HBaseTestingUtility util, long ts)

+ 5 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineSchemaCreator.java

@@ -56,13 +56,13 @@ import org.slf4j.LoggerFactory;
  */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
-public final class TimelineSchemaCreator {
-  private TimelineSchemaCreator() {
+public final class HBaseTimelineSchemaCreator implements SchemaCreator {
+  public HBaseTimelineSchemaCreator() {
   }
 
-  final static String NAME = TimelineSchemaCreator.class.getSimpleName();
+  final static String NAME = HBaseTimelineSchemaCreator.class.getSimpleName();
   private static final Logger LOG =
-      LoggerFactory.getLogger(TimelineSchemaCreator.class);
+      LoggerFactory.getLogger(HBaseTimelineSchemaCreator.class);
   private static final String SKIP_EXISTING_TABLE_OPTION_SHORT = "s";
   private static final String APP_METRICS_TTL_OPTION_SHORT = "ma";
   private static final String SUB_APP_METRICS_TTL_OPTION_SHORT = "msa";
@@ -74,7 +74,7 @@ public final class TimelineSchemaCreator {
   private static final String HELP_SHORT = "h";
   private static final String CREATE_TABLES_SHORT = "c";
 
-  public static void main(String[] args) throws Exception {
+  public void createTimelineSchema(String[] args) throws Exception {
 
     LOG.info("Starting the schema creation");
     Configuration hbaseConf =

+ 28 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/SchemaCreator.java

@@ -0,0 +1,28 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.timelineservice.storage;
+
+/**
+ * This interface is for creating Timeline Schema. The backend for Timeline
+ * Service have to implement this.
+ */
+public interface SchemaCreator {
+
+  void createTimelineSchema(String[] args) throws Exception;
+}

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java

@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.timelineservice.storage;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This creates the timeline schema for storing application timeline
+ * information. Each backend has to implement the {@link SchemaCreator} for
+ * creating the schema in its backend and should be configured in yarn-site.xml.
+ */
+public class TimelineSchemaCreator extends Configured implements Tool {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TimelineSchemaCreator.class);
+
+  public static void main(String[] args) {
+    try {
+      int status = ToolRunner.run(new YarnConfiguration(),
+          new TimelineSchemaCreator(), args);
+      System.exit(status);
+    } catch (Exception e) {
+      LOG.error("Error while creating Timeline Schema : ", e);
+    }
+  }
+
+  @Override
+  public int run(String[] args) throws Exception {
+    Configuration conf = getConf();
+    return createTimelineSchema(args, conf);
+  }
+
+  @VisibleForTesting
+  int createTimelineSchema(String[] args, Configuration conf) throws Exception {
+    String schemaCreatorClassName = conf.get(
+        YarnConfiguration.TIMELINE_SERVICE_SCHEMA_CREATOR_CLASS,
+        YarnConfiguration.DEFAULT_TIMELINE_SERVICE_SCHEMA_CREATOR_CLASS);
+    LOG.info("Using {} for creating Timeline Service Schema ",
+        schemaCreatorClassName);
+    try {
+      Class<?> schemaCreatorClass = Class.forName(schemaCreatorClassName);
+      if (SchemaCreator.class.isAssignableFrom(schemaCreatorClass)) {
+        SchemaCreator schemaCreator = (SchemaCreator) ReflectionUtils
+            .newInstance(schemaCreatorClass, conf);
+        schemaCreator.createTimelineSchema(args);
+        return 0;
+      } else {
+        throw new YarnRuntimeException("Class: " + schemaCreatorClassName
+            + " not instance of " + SchemaCreator.class.getCanonicalName());
+      }
+    } catch (ClassNotFoundException e) {
+      throw new YarnRuntimeException("Could not instantiate TimelineReader: "
+          + schemaCreatorClassName, e);
+    }
+  }
+}

+ 29 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/DummyTimelineSchemaCreator.java

@@ -0,0 +1,29 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.timelineservice.storage;
+
+/**
+ * Dummy Implementation of {@link SchemaCreator} for test.
+ */
+public class DummyTimelineSchemaCreator implements SchemaCreator {
+
+  @Override
+  public void createTimelineSchema(String[] args) {
+  }
+}

+ 41 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestTimelineSchemaCreator.java

@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.timelineservice.storage;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test cases for {@link TimelineSchemaCreator}.
+ */
+public class TestTimelineSchemaCreator {
+
+  @Test
+  public void testTimelineSchemaCreation() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(YarnConfiguration.TIMELINE_SERVICE_SCHEMA_CREATOR_CLASS,
+        "org.apache.hadoop.yarn.server.timelineservice.storage" +
+            ".DummyTimelineSchemaCreator");
+    TimelineSchemaCreator timelineSchemaCreator = new TimelineSchemaCreator();
+    Assert.assertEquals(0, timelineSchemaCreator
+        .createTimelineSchema(new String[]{}, conf));
+  }
+}

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/bower.json

@@ -13,7 +13,7 @@
     "qunit": "1.19.0",
     "jquery-ui": "1.11.4",
     "moment": "2.12.0",
-    "moment-timezone": "0.5.0",
+    "moment-timezone": "0.5.1",
     "more-js": "0.8.2",
     "bootstrap": "3.3.6",
     "d3": "~3.5.6",