瀏覽代碼

HDDS-1705. Recon: Add estimatedTotalCount to the response of containers and containers/{id} endpoints. Contributed by Vivek Ratnavel Subramanian.

Vivek Ratnavel Subramanian 5 年之前
父節點
當前提交
82d88a8d30
共有 22 個文件被更改,包括 1233 次插入335 次删除
  1. 3 3
      hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Storage.java
  2. 3 3
      hadoop-hdds/common/src/main/resources/ozone-default.xml
  3. 1 1
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerStarter.java
  4. 2 0
      hadoop-ozone/ozone-recon-codegen/src/main/java/org/hadoop/ozone/recon/codegen/ReconSchemaGenerationModule.java
  5. 61 0
      hadoop-ozone/ozone-recon-codegen/src/main/java/org/hadoop/ozone/recon/schema/StatsSchemaDefinition.java
  6. 7 1
      hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/ReconConstants.java
  7. 22 0
      hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/ReconServer.java
  8. 18 4
      hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerKeyService.java
  9. 94 0
      hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/ContainersResponse.java
  10. 93 0
      hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/KeysResponse.java
  11. 54 4
      hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/spi/ContainerDBServiceProvider.java
  12. 124 13
      hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/ContainerDBServiceProviderImpl.java
  13. 4 0
      hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/ReconContainerDBProvider.java
  14. 48 9
      hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTask.java
  15. 28 0
      hadoop-ozone/ozone-recon/src/test/java/org/apache/hadoop/ozone/recon/GuiceInjectorUtilsForTestsImpl.java
  16. 116 70
      hadoop-ozone/ozone-recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerKeyService.java
  17. 9 3
      hadoop-ozone/ozone-recon/src/test/java/org/apache/hadoop/ozone/recon/persistence/AbstractSqlDatabaseTest.java
  18. 147 0
      hadoop-ozone/ozone-recon/src/test/java/org/apache/hadoop/ozone/recon/persistence/TestStatsSchemaDefinition.java
  19. 196 130
      hadoop-ozone/ozone-recon/src/test/java/org/apache/hadoop/ozone/recon/spi/impl/TestContainerDBServiceProviderImpl.java
  20. 15 38
      hadoop-ozone/ozone-recon/src/test/java/org/apache/hadoop/ozone/recon/spi/impl/TestOzoneManagerServiceProviderImpl.java
  21. 71 56
      hadoop-ozone/ozone-recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestContainerKeyMapperTask.java
  22. 117 0
      hadoop-ozone/ozone-recon/src/test/java/org/apache/hadoop/ozone/recon/types/GuiceInjectorUtilsForTests.java

+ 3 - 3
hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Storage.java

@@ -81,7 +81,7 @@ public abstract class Storage {
 
   /**
    * Gets the path of the Storage dir.
-   * @return Stoarge dir path
+   * @return Storage dir path
    */
   public String getStorageDir() {
     return storageDir.getAbsoluteFile().toString();
@@ -117,7 +117,7 @@ public abstract class Storage {
   }
 
   /**
-   * Retreives the storageInfo instance to read/write the common
+   * Retrieves the storageInfo instance to read/write the common
    * version file properties.
    * @return the instance of the storageInfo class
    */
@@ -128,7 +128,7 @@ public abstract class Storage {
   abstract protected Properties getNodeProperties();
 
   /**
-   * Sets the Node properties spaecific to OM/SCM.
+   * Sets the Node properties specific to OM/SCM.
    */
   private void setNodeProperties() {
     Properties nodeProperties = getNodeProperties();

+ 3 - 3
hadoop-hdds/common/src/main/resources/ozone-default.xml

@@ -659,7 +659,7 @@
     <value/>
     <tag>OZONE, OM, SCM, CONTAINER, STORAGE, REQUIRED</tag>
     <description>
-      This setting is the fallback location for SCM, OM and DataNodes
+      This setting is the fallback location for SCM, OM, Recon and DataNodes
       to store their metadata. This setting may be used only in test/PoC
       clusters to simplify configuration.
 
@@ -2457,7 +2457,7 @@
     <value/>
     <tag>OZONE, RECON</tag>
     <description>
-      Ozone Recon datbase password.
+      Ozone Recon database password.
     </description>
   </property>
   <property>
@@ -2484,7 +2484,7 @@
     <description>
       The max active connections to the SQL database. The default SQLite
       database only allows single active connection, set this to a
-      resonable value like 10, for external production database.
+      reasonable value like 10, for external production database.
     </description>
   </property>
   <property>

+ 1 - 1
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerStarter.java

@@ -60,7 +60,7 @@ public class OzoneManagerStarter extends GenericCli {
   public Void call() throws Exception {
     /**
      * This method is invoked only when a sub-command is not called. Therefore
-     * if someone runs "ozone om" with no parameters, this is the methood
+     * if someone runs "ozone om" with no parameters, this is the method
      * which runs and starts the OM.
      */
     commonInit();

+ 2 - 0
hadoop-ozone/ozone-recon-codegen/src/main/java/org/hadoop/ozone/recon/codegen/ReconSchemaGenerationModule.java

@@ -19,6 +19,7 @@ package org.hadoop.ozone.recon.codegen;
 
 import org.hadoop.ozone.recon.schema.ReconInternalSchemaDefinition;
 import org.hadoop.ozone.recon.schema.ReconSchemaDefinition;
+import org.hadoop.ozone.recon.schema.StatsSchemaDefinition;
 import org.hadoop.ozone.recon.schema.UtilizationSchemaDefinition;
 
 import com.google.inject.AbstractModule;
@@ -36,5 +37,6 @@ public class ReconSchemaGenerationModule extends AbstractModule {
         Multibinder.newSetBinder(binder(), ReconSchemaDefinition.class);
     schemaBinder.addBinding().to(UtilizationSchemaDefinition.class);
     schemaBinder.addBinding().to(ReconInternalSchemaDefinition.class);
+    schemaBinder.addBinding().to(StatsSchemaDefinition.class);
   }
 }

+ 61 - 0
hadoop-ozone/ozone-recon-codegen/src/main/java/org/hadoop/ozone/recon/schema/StatsSchemaDefinition.java

@@ -0,0 +1,61 @@
+/**
+ * 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.hadoop.ozone.recon.schema;
+
+import com.google.inject.Inject;
+import org.jooq.impl.DSL;
+import org.jooq.impl.SQLDataType;
+
+import javax.sql.DataSource;
+import java.sql.Connection;
+import java.sql.SQLException;
+
+/**
+ * Class used to create tables that are required for storing Ozone statistics.
+ */
+public class StatsSchemaDefinition implements ReconSchemaDefinition {
+
+  public static final String GLOBAL_STATS_TABLE_NAME = "global_stats";
+  private final DataSource dataSource;
+
+  @Inject
+  StatsSchemaDefinition(DataSource dataSource) {
+    this.dataSource = dataSource;
+  }
+
+  @Override
+  public void initializeSchema() throws SQLException {
+    Connection conn = dataSource.getConnection();
+    createGlobalStatsTable(conn);
+  }
+
+  /**
+   * Create the Ozone Global Stats table.
+   * @param conn connection
+   */
+  private void createGlobalStatsTable(Connection conn) {
+    DSL.using(conn).createTableIfNotExists(GLOBAL_STATS_TABLE_NAME)
+        .column("key", SQLDataType.VARCHAR(255))
+        .column("value", SQLDataType.BIGINT)
+        .column("last_updated_timestamp", SQLDataType.TIMESTAMP)
+        .constraint(DSL.constraint("pk_key")
+            .primaryKey("key"))
+        .execute();
+  }
+}

+ 7 - 1
hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/ReconConstants.java

@@ -32,14 +32,20 @@ public final class ReconConstants {
   public static final String RECON_CONTAINER_DB = "recon-" +
       CONTAINER_DB_SUFFIX;
 
+  public static final String CONTAINER_COUNT_KEY = "totalCount";
+
   public static final String RECON_OM_SNAPSHOT_DB =
       "om.snapshot.db";
 
   public static final String CONTAINER_KEY_TABLE =
       "containerKeyTable";
 
+  public static final String CONTAINER_KEY_COUNT_TABLE =
+      "containerKeyCountTable";
+
   public static final String FETCH_ALL = "-1";
-  public static final String RECON_QUERY_PREVKEY = "prev-key";
+  public static final String RECON_QUERY_PREVKEY = "prevKey";
+  public static final String PREV_CONTAINER_ID_DEFAULT_VALUE = "0";
   public static final String RECON_QUERY_LIMIT = "limit";
 
 }

+ 22 - 0
hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/ReconServer.java

@@ -33,6 +33,9 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.recon.spi.ContainerDBServiceProvider;
 import org.apache.hadoop.ozone.recon.spi.OzoneManagerServiceProvider;
 import org.apache.hadoop.ozone.recon.tasks.ContainerKeyMapperTask;
+import org.hadoop.ozone.recon.schema.ReconInternalSchemaDefinition;
+import org.hadoop.ozone.recon.schema.StatsSchemaDefinition;
+import org.hadoop.ozone.recon.schema.UtilizationSchemaDefinition;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -75,6 +78,25 @@ public class ReconServer extends GenericCli {
     //Pass on injector to listener that does the Guice - Jersey HK2 bridging.
     ReconGuiceServletContextListener.setInjector(injector);
 
+    LOG.info("Initializing Recon server...");
+    try {
+      StatsSchemaDefinition statsSchemaDefinition = injector.getInstance(
+          StatsSchemaDefinition.class);
+      statsSchemaDefinition.initializeSchema();
+
+      UtilizationSchemaDefinition utilizationSchemaDefinition =
+          injector.getInstance(UtilizationSchemaDefinition.class);
+      utilizationSchemaDefinition.initializeSchema();
+
+      ReconInternalSchemaDefinition reconInternalSchemaDefinition =
+          injector.getInstance(ReconInternalSchemaDefinition.class);
+      reconInternalSchemaDefinition.initializeSchema();
+
+      LOG.info("Recon server initialized successfully!");
+    } catch (Exception e) {
+      LOG.error("Error during initializing Recon server.", e);
+    }
+
     httpServer = injector.getInstance(ReconHttpServer.class);
     LOG.info("Starting Recon server");
     httpServer.start();

+ 18 - 4
hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerKeyService.java

@@ -44,14 +44,17 @@ import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
 import org.apache.hadoop.ozone.recon.api.types.ContainerKeyPrefix;
 import org.apache.hadoop.ozone.recon.api.types.ContainerMetadata;
+import org.apache.hadoop.ozone.recon.api.types.ContainersResponse;
 import org.apache.hadoop.ozone.recon.api.types.KeyMetadata;
 import org.apache.hadoop.ozone.recon.api.types.KeyMetadata.ContainerBlockMetadata;
+import org.apache.hadoop.ozone.recon.api.types.KeysResponse;
 import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
 import org.apache.hadoop.ozone.recon.spi.ContainerDBServiceProvider;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static org.apache.hadoop.ozone.recon.ReconConstants.FETCH_ALL;
+import static org.apache.hadoop.ozone.recon.ReconConstants.PREV_CONTAINER_ID_DEFAULT_VALUE;
 import static org.apache.hadoop.ozone.recon.ReconConstants.RECON_QUERY_LIMIT;
 import static org.apache.hadoop.ozone.recon.ReconConstants.RECON_QUERY_PREVKEY;
 
@@ -84,15 +87,20 @@ public class ContainerKeyService {
   @GET
   public Response getContainers(
       @DefaultValue(FETCH_ALL) @QueryParam(RECON_QUERY_LIMIT) int limit,
-      @DefaultValue("0") @QueryParam(RECON_QUERY_PREVKEY) long prevKey) {
+      @DefaultValue(PREV_CONTAINER_ID_DEFAULT_VALUE)
+      @QueryParam(RECON_QUERY_PREVKEY) long prevKey) {
     Map<Long, ContainerMetadata> containersMap;
+    long containersCount;
     try {
       containersMap = containerDBServiceProvider.getContainers(limit, prevKey);
+      containersCount = containerDBServiceProvider.getCountForContainers();
     } catch (IOException ioEx) {
       throw new WebApplicationException(ioEx,
           Response.Status.INTERNAL_SERVER_ERROR);
     }
-    return Response.ok(containersMap.values()).build();
+    ContainersResponse containersResponse =
+        new ContainersResponse(containersCount, containersMap.values());
+    return Response.ok(containersResponse).build();
   }
 
   /**
@@ -107,13 +115,14 @@ public class ContainerKeyService {
    * @return {@link Response}
    */
   @GET
-  @Path("/{id}")
+  @Path("/{id}/keys")
   public Response getKeysForContainer(
       @PathParam("id") Long containerID,
       @DefaultValue(FETCH_ALL) @QueryParam(RECON_QUERY_LIMIT) int limit,
       @DefaultValue(StringUtils.EMPTY) @QueryParam(RECON_QUERY_PREVKEY)
           String prevKeyPrefix) {
     Map<String, KeyMetadata> keyMetadataMap = new LinkedHashMap<>();
+    long totalCount;
     try {
       Map<ContainerKeyPrefix, Integer> containerKeyPrefixMap =
           containerDBServiceProvider.getKeyPrefixesForContainer(containerID,
@@ -187,10 +196,15 @@ public class ContainerKeyService {
             }});
         }
       }
+
+      totalCount =
+          containerDBServiceProvider.getKeyCountForContainer(containerID);
     } catch (IOException ioEx) {
       throw new WebApplicationException(ioEx,
           Response.Status.INTERNAL_SERVER_ERROR);
     }
-    return Response.ok(keyMetadataMap.values()).build();
+    KeysResponse keysResponse =
+        new KeysResponse(totalCount, keyMetadataMap.values());
+    return Response.ok(keysResponse).build();
   }
 }

+ 94 - 0
hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/ContainersResponse.java

@@ -0,0 +1,94 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.recon.api.types;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.hadoop.ozone.web.utils.JsonUtils;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+
+/**
+ * Class that represents the API Response structure of Containers.
+ */
+public class ContainersResponse {
+  /**
+   * Contains a map with total count of containers and list of containers.
+   */
+  @JsonProperty("data")
+  private ContainersResponseData containersResponseData;
+
+  public ContainersResponse() {
+    this(0, new ArrayList<>());
+  }
+
+  public ContainersResponse(long totalCount,
+                            Collection<ContainerMetadata> containers) {
+    this.containersResponseData =
+        new ContainersResponseData(totalCount, containers);
+  }
+
+  public String toJsonString() {
+    try {
+      return JsonUtils.toJsonString(this);
+    } catch (IOException ignored) {
+      return null;
+    }
+  }
+
+  public ContainersResponseData getContainersResponseData() {
+    return containersResponseData;
+  }
+
+  public void setContainersResponseData(ContainersResponseData
+                                            containersResponseData) {
+    this.containersResponseData = containersResponseData;
+  }
+
+  /**
+   * Class that encapsulates the data presented in Containers API Response.
+   */
+  public static class ContainersResponseData {
+    /**
+     * Total count of the containers.
+     */
+    @JsonProperty("totalCount")
+    private long totalCount;
+
+    /**
+     * An array of containers.
+     */
+    @JsonProperty("containers")
+    private Collection<ContainerMetadata> containers;
+
+    ContainersResponseData(long totalCount,
+                           Collection<ContainerMetadata> containers) {
+      this.totalCount = totalCount;
+      this.containers = containers;
+    }
+
+    public long getTotalCount() {
+      return totalCount;
+    }
+
+    public Collection<ContainerMetadata> getContainers() {
+      return containers;
+    }
+  }
+}

+ 93 - 0
hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/KeysResponse.java

@@ -0,0 +1,93 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.recon.api.types;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.hadoop.ozone.web.utils.JsonUtils;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+
+/**
+ * Class that represents the API Response structure of Keys within a container.
+ */
+public class KeysResponse {
+  /**
+   * Contains a map with total count of keys inside the given container and a
+   * list of keys with metadata.
+   */
+  @JsonProperty("data")
+  private KeysResponseData keysResponseData;
+
+  public KeysResponse() {
+    this(0, new ArrayList<>());
+  }
+
+  public KeysResponse(long totalCount,
+                      Collection<KeyMetadata> keys) {
+    this.keysResponseData =
+        new KeysResponseData(totalCount, keys);
+  }
+
+  public String toJsonString() {
+    try {
+      return JsonUtils.toJsonString(this);
+    } catch (IOException ignored) {
+      return null;
+    }
+  }
+
+  public KeysResponseData getKeysResponseData() {
+    return keysResponseData;
+  }
+
+  public void setKeysResponseData(KeysResponseData keysResponseData) {
+    this.keysResponseData = keysResponseData;
+  }
+
+  /**
+   * Class that encapsulates the data presented in Keys API Response.
+   */
+  public static class KeysResponseData {
+    /**
+     * Total count of the keys.
+     */
+    @JsonProperty("totalCount")
+    private long totalCount;
+
+    /**
+     * An array of keys.
+     */
+    @JsonProperty("keys")
+    private Collection<KeyMetadata> keys;
+
+    KeysResponseData(long totalCount, Collection<KeyMetadata> keys) {
+      this.totalCount = totalCount;
+      this.keys = keys;
+    }
+
+    public long getTotalCount() {
+      return totalCount;
+    }
+
+    public Collection<KeyMetadata> getKeys() {
+      return keys;
+    }
+  }
+}

+ 54 - 4
hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/spi/ContainerDBServiceProvider.java

@@ -52,14 +52,48 @@ public interface ContainerDBServiceProvider {
                                 Integer count) throws IOException;
 
   /**
-   * Get the stored key prefix count for the given containerId, key prefix.
+   * Store the containerID -> no. of keys count into the container DB store.
    *
-   * @param containerKeyPrefix the containerId, key-prefix tuple.
+   * @param containerID the containerID.
+   * @param count count of the keys within the given containerID.
+   * @throws IOException
+   */
+  void storeContainerKeyCount(Long containerID, Long count) throws IOException;
+
+  /**
+   * Store the total count of containers into the container DB store.
+   *
+   * @param count count of the containers present in the system.
+   */
+  void storeContainerCount(Long count);
+
+  /**
+   * Get the stored key prefix count for the given containerID, key prefix.
+   *
+   * @param containerKeyPrefix the containerID, key-prefix tuple.
    * @return count of keys with that prefix.
    */
-  Integer getCountForForContainerKeyPrefix(
+  Integer getCountForContainerKeyPrefix(
       ContainerKeyPrefix containerKeyPrefix) throws IOException;
 
+  /**
+   * Get the total count of keys within the given containerID.
+   *
+   * @param containerID the given containerId.
+   * @return count of keys within the given containerID.
+   * @throws IOException
+   */
+  long getKeyCountForContainer(Long containerID) throws IOException;
+
+  /**
+   * Get if a containerID exists or not.
+   *
+   * @param containerID the given containerID.
+   * @return if the given ContainerID exists or not.
+   * @throws IOException
+   */
+  boolean doesContainerExists(Long containerID) throws IOException;
+
   /**
    * Get the stored key prefixes for the given containerId.
    *
@@ -95,6 +129,7 @@ public interface ContainerDBServiceProvider {
 
   /**
    * Delete an entry in the container DB.
+   *
    * @param containerKeyPrefix container key prefix to be deleted.
    * @throws IOException exception.
    */
@@ -105,5 +140,20 @@ public interface ContainerDBServiceProvider {
    * Get iterator to the entire container DB.
    * @return TableIterator
    */
-  TableIterator getContainerTableIterator();
+  TableIterator getContainerTableIterator() throws IOException;
+
+  /**
+   * Get the total count of containers present in the system.
+   *
+   * @return total count of containers.
+   * @throws IOException
+   */
+  long getCountForContainers() throws IOException;
+
+  /**
+   * Increment the total count for containers in the system by the given count.
+   *
+   * @param count no. of new containers to add to containers total count.
+   */
+  void incrementContainerCountBy(long count);
 }

+ 124 - 13
hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/ContainerDBServiceProviderImpl.java

@@ -18,10 +18,16 @@
 
 package org.apache.hadoop.ozone.recon.spi.impl;
 
+import static org.apache.hadoop.ozone.recon.ReconConstants.CONTAINER_COUNT_KEY;
+import static org.apache.hadoop.ozone.recon.ReconConstants.CONTAINER_KEY_COUNT_TABLE;
 import static org.apache.hadoop.ozone.recon.ReconConstants.CONTAINER_KEY_TABLE;
+import static org.jooq.impl.DSL.currentTimestamp;
+import static org.jooq.impl.DSL.select;
+import static org.jooq.impl.DSL.using;
 
 import java.io.File;
 import java.io.IOException;
+import java.sql.Timestamp;
 import java.util.LinkedHashMap;
 import java.util.Map;
 
@@ -38,6 +44,9 @@ import org.apache.hadoop.utils.db.DBStore;
 import org.apache.hadoop.utils.db.Table;
 import org.apache.hadoop.utils.db.Table.KeyValue;
 import org.apache.hadoop.utils.db.TableIterator;
+import org.hadoop.ozone.recon.schema.tables.daos.GlobalStatsDao;
+import org.hadoop.ozone.recon.schema.tables.pojos.GlobalStats;
+import org.jooq.Configuration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -52,6 +61,8 @@ public class ContainerDBServiceProviderImpl
       LoggerFactory.getLogger(ContainerDBServiceProviderImpl.class);
 
   private Table<ContainerKeyPrefix, Integer> containerKeyTable;
+  private Table<Long, Long> containerKeyCountTable;
+  private GlobalStatsDao globalStatsDao;
 
   @Inject
   private OzoneConfiguration configuration;
@@ -60,20 +71,28 @@ public class ContainerDBServiceProviderImpl
   private DBStore containerDbStore;
 
   @Inject
-  public ContainerDBServiceProviderImpl(DBStore dbStore) {
+  private Configuration sqlConfiguration;
+
+  @Inject
+  public ContainerDBServiceProviderImpl(DBStore dbStore,
+                                        Configuration sqlConfiguration) {
+    globalStatsDao = new GlobalStatsDao(sqlConfiguration);
     try {
       this.containerKeyTable = dbStore.getTable(CONTAINER_KEY_TABLE,
           ContainerKeyPrefix.class, Integer.class);
+      this.containerKeyCountTable = dbStore.getTable(CONTAINER_KEY_COUNT_TABLE,
+          Long.class, Long.class);
     } catch (IOException e) {
-      LOG.error("Unable to create Container Key Table. " + e);
+      LOG.error("Unable to create Container Key tables." + e);
     }
   }
 
   /**
    * Initialize a new container DB instance, getting rid of the old instance
    * and then storing the passed in container prefix counts into the created
-   * DB instance.
-   * @param containerKeyPrefixCounts Map of containerId, key-prefix tuple to
+   * DB instance. Also, truncate or reset the SQL tables as required.
+   * @param containerKeyPrefixCounts Map of container key-prefix to
+   *                                 number of keys with the prefix.
    * @throws IOException
    */
   @Override
@@ -91,17 +110,23 @@ public class ContainerDBServiceProviderImpl
           oldDBLocation.getAbsolutePath());
       FileUtils.deleteDirectory(oldDBLocation);
     }
-    for (Map.Entry<ContainerKeyPrefix, Integer> entry :
-        containerKeyPrefixCounts.entrySet()) {
-      containerKeyTable.put(entry.getKey(), entry.getValue());
+
+    if (containerKeyPrefixCounts != null) {
+      for (Map.Entry<ContainerKeyPrefix, Integer> entry :
+          containerKeyPrefixCounts.entrySet()) {
+        containerKeyTable.put(entry.getKey(), entry.getValue());
+      }
     }
+
+    // reset total count of containers to zero
+    storeContainerCount(0L);
   }
 
   /**
-   * Concatenate the containerId and Key Prefix using a delimiter and store the
+   * Concatenate the containerID and Key Prefix using a delimiter and store the
    * count into the container DB store.
    *
-   * @param containerKeyPrefix the containerId, key-prefix tuple.
+   * @param containerKeyPrefix the containerID, key-prefix tuple.
    * @param count Count of the keys matching that prefix.
    * @throws IOException
    */
@@ -112,16 +137,54 @@ public class ContainerDBServiceProviderImpl
     containerKeyTable.put(containerKeyPrefix, count);
   }
 
+  /**
+   * Store the containerID -> no. of keys count into the container DB store.
+   *
+   * @param containerID the containerID.
+   * @param count count of the keys within the given containerID.
+   * @throws IOException
+   */
+  @Override
+  public void storeContainerKeyCount(Long containerID, Long count)
+      throws IOException {
+    containerKeyCountTable.put(containerID, count);
+  }
+
+  /**
+   * Get the total count of keys within the given containerID.
+   *
+   * @param containerID the given containerID.
+   * @return count of keys within the given containerID.
+   * @throws IOException
+   */
+  @Override
+  public long getKeyCountForContainer(Long containerID) throws IOException {
+    Long keyCount = containerKeyCountTable.get(containerID);
+    return keyCount == null ? 0L : keyCount;
+  }
+
+  /**
+   * Get if a containerID exists or not.
+   *
+   * @param containerID the given containerID.
+   * @return if the given ContainerID exists or not.
+   * @throws IOException
+   */
+  @Override
+  public boolean doesContainerExists(Long containerID) throws IOException {
+    return containerKeyCountTable.get(containerID) != null;
+  }
+
   /**
    * Put together the key from the passed in object and get the count from
    * the container DB store.
    *
-   * @param containerKeyPrefix the containerId, key-prefix tuple.
-   * @return count of keys matching the containerId, key-prefix.
+   * @param containerKeyPrefix the containerID, key-prefix tuple.
+   * @return count of keys matching the containerID, key-prefix.
    * @throws IOException
    */
   @Override
-  public Integer getCountForForContainerKeyPrefix(
+  public Integer getCountForContainerKeyPrefix(
       ContainerKeyPrefix containerKeyPrefix) throws IOException {
     Integer count =  containerKeyTable.get(containerKeyPrefix);
     return count == null ? Integer.valueOf(0) : count;
@@ -130,7 +193,7 @@ public class ContainerDBServiceProviderImpl
   /**
    * Get key prefixes for the given container ID.
    *
-   * @param containerId the given containerId.
+   * @param containerId the given containerID.
    * @return Map of (Key-Prefix,Count of Keys).
    */
   @Override
@@ -271,8 +334,56 @@ public class ContainerDBServiceProviderImpl
     containerKeyTable.delete(containerKeyPrefix);
   }
 
+  /**
+   * Get total count of containers.
+   *
+   * @return total count of containers.
+   */
+  @Override
+  public long getCountForContainers() {
+    GlobalStats containerCountRecord =
+        globalStatsDao.fetchOneByKey(CONTAINER_COUNT_KEY);
+
+    return (containerCountRecord == null) ? 0L :
+        containerCountRecord.getValue();
+  }
+
   @Override
   public TableIterator getContainerTableIterator() {
     return containerKeyTable.iterator();
   }
+
+  /**
+   * Store the total count of containers into the container DB store.
+   *
+   * @param count count of the containers present in the system.
+   */
+  @Override
+  public void storeContainerCount(Long count) {
+    // Get the current timestamp
+    Timestamp now =
+        using(sqlConfiguration).fetchValue(select(currentTimestamp()));
+    GlobalStats containerCountRecord =
+        globalStatsDao.fetchOneByKey(CONTAINER_COUNT_KEY);
+    GlobalStats globalStatsRecord =
+        new GlobalStats(CONTAINER_COUNT_KEY, count, now);
+
+    // Insert a new record for CONTAINER_COUNT_KEY if it does not exist
+    if (containerCountRecord == null) {
+      globalStatsDao.insert(globalStatsRecord);
+    } else {
+      globalStatsDao.update(globalStatsRecord);
+    }
+  }
+
+  /**
+   * Increment the total count for containers in the system by the given count.
+   *
+   * @param count no. of new containers to add to containers total count.
+   */
+  @Override
+  public void incrementContainerCountBy(long count) {
+    long containersCount = getCountForContainers();
+    storeContainerCount(containersCount + count);
+  }
 }

+ 4 - 0
hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/ReconContainerDBProvider.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.ozone.recon.spi.impl;
 
+import static org.apache.hadoop.ozone.recon.ReconConstants.CONTAINER_KEY_COUNT_TABLE;
 import static org.apache.hadoop.ozone.recon.ReconConstants.RECON_CONTAINER_DB;
 import static org.apache.hadoop.ozone.recon.ReconConstants.CONTAINER_KEY_TABLE;
 import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_DB_DIR;
@@ -30,6 +31,7 @@ import org.apache.hadoop.ozone.recon.api.types.ContainerKeyPrefix;
 import org.apache.hadoop.utils.db.DBStore;
 import org.apache.hadoop.utils.db.DBStoreBuilder;
 import org.apache.hadoop.utils.db.IntegerCodec;
+import org.apache.hadoop.utils.db.LongCodec;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -69,7 +71,9 @@ public class ReconContainerDBProvider implements Provider<DBStore> {
           .setPath(metaDir)
           .setName(dbName)
           .addTable(CONTAINER_KEY_TABLE)
+          .addTable(CONTAINER_KEY_COUNT_TABLE)
           .addCodec(ContainerKeyPrefix.class, new ContainerKeyPrefixCodec())
+          .addCodec(Long.class, new LongCodec())
           .addCodec(Integer.class, new IntegerCodec())
           .build();
     } catch (Exception ex) {

+ 48 - 9
hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTask.java

@@ -23,6 +23,7 @@ import java.time.Duration;
 import java.time.Instant;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Set;
@@ -70,11 +71,14 @@ public class ContainerKeyMapperTask extends ReconDBUpdateTask {
    */
   @Override
   public Pair<String, Boolean> reprocess(OMMetadataManager omMetadataManager) {
-    int omKeyCount = 0;
+    long omKeyCount = 0;
     try {
       LOG.info("Starting a 'reprocess' run of ContainerKeyMapperTask.");
       Instant start = Instant.now();
 
+      // initialize new container DB
+      containerDBServiceProvider.initNewContainerDB(new HashMap<>());
+
       Table<String, OmKeyInfo> omKeyInfoTable = omMetadataManager.getKeyTable();
       try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
                keyIter = omKeyInfoTable.iterator()) {
@@ -98,7 +102,6 @@ public class ContainerKeyMapperTask extends ReconDBUpdateTask {
     return new ImmutablePair<>(getTaskName(), true);
   }
 
-
   @Override
   protected Collection<String> getTaskTables() {
     return tables;
@@ -125,7 +128,8 @@ public class ContainerKeyMapperTask extends ReconDBUpdateTask {
             .getAction());
         }
       } catch (IOException e) {
-        LOG.error("Unexpected exception while updating key data : {} ", e);
+        LOG.error("Unexpected exception while updating key data : {} ",
+            updatedKey, e);
         return new ImmutablePair<>(getTaskName(), false);
       }
     }
@@ -133,7 +137,9 @@ public class ContainerKeyMapperTask extends ReconDBUpdateTask {
   }
 
   /**
-   * Delete an OM Key from Container DB.
+   * Delete an OM Key from Container DB and update containerID -> no. of keys
+   * count.
+   *
    * @param key key String.
    * @throws IOException If Unable to write to container DB.
    */
@@ -144,30 +150,42 @@ public class ContainerKeyMapperTask extends ReconDBUpdateTask {
         Table.KeyValue<ContainerKeyPrefix, Integer>> containerIterator =
         containerDBServiceProvider.getContainerTableIterator();
 
-    Set<ContainerKeyPrefix> keysToDeDeleted = new HashSet<>();
+    Set<ContainerKeyPrefix> keysToBeDeleted = new HashSet<>();
 
     while (containerIterator.hasNext()) {
       Table.KeyValue<ContainerKeyPrefix, Integer> keyValue =
           containerIterator.next();
       String keyPrefix = keyValue.getKey().getKeyPrefix();
       if (keyPrefix.equals(key)) {
-        keysToDeDeleted.add(keyValue.getKey());
+        keysToBeDeleted.add(keyValue.getKey());
       }
     }
 
-    for (ContainerKeyPrefix containerKeyPrefix : keysToDeDeleted) {
+    for (ContainerKeyPrefix containerKeyPrefix : keysToBeDeleted) {
       containerDBServiceProvider.deleteContainerMapping(containerKeyPrefix);
+
+      // decrement count and update containerKeyCount.
+      Long containerID = containerKeyPrefix.getContainerId();
+      long keyCount =
+          containerDBServiceProvider.getKeyCountForContainer(containerID);
+      if (keyCount > 0) {
+        containerDBServiceProvider.storeContainerKeyCount(containerID,
+            --keyCount);
+      }
     }
   }
 
   /**
-   * Write an OM key to container DB.
+   * Write an OM key to container DB and update containerID -> no. of keys
+   * count.
+   *
    * @param key key String
    * @param omKeyInfo omKeyInfo value
    * @throws IOException if unable to write to recon DB.
    */
   private void  writeOMKeyToContainerDB(String key, OmKeyInfo omKeyInfo)
       throws IOException {
+    long containerCountToIncrement = 0;
     for (OmKeyLocationInfoGroup omKeyLocationInfoGroup : omKeyInfo
         .getKeyLocationVersions()) {
       long keyVersion = omKeyLocationInfoGroup.getVersion();
@@ -176,15 +194,36 @@ public class ContainerKeyMapperTask extends ReconDBUpdateTask {
         long containerId = omKeyLocationInfo.getContainerID();
         ContainerKeyPrefix containerKeyPrefix = new ContainerKeyPrefix(
             containerId, key, keyVersion);
-        if (containerDBServiceProvider.getCountForForContainerKeyPrefix(
+        if (containerDBServiceProvider.getCountForContainerKeyPrefix(
             containerKeyPrefix) == 0) {
           // Save on writes. No need to save same container-key prefix
           // mapping again.
           containerDBServiceProvider.storeContainerKeyMapping(
               containerKeyPrefix, 1);
+
+          // check if container already exists and
+          // increment the count of containers if it does not exist
+          if (!containerDBServiceProvider.doesContainerExists(containerId)) {
+            containerCountToIncrement++;
+          }
+
+          // update the count of keys for the given containerID
+          long keyCount =
+              containerDBServiceProvider.getKeyCountForContainer(containerId);
+
+          // increment the count and update containerKeyCount.
+          // keyCount will be 0 if containerID is not found. So, there is no
+          // need to initialize keyCount for the first time.
+          containerDBServiceProvider.storeContainerKeyCount(containerId,
+              ++keyCount);
         }
       }
     }
+
+    if (containerCountToIncrement > 0) {
+      containerDBServiceProvider
+          .incrementContainerCountBy(containerCountToIncrement);
+    }
   }
 
 }

+ 28 - 0
hadoop-ozone/ozone-recon/src/test/java/org/apache/hadoop/ozone/recon/GuiceInjectorUtilsForTestsImpl.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
+ * <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.recon;
+
+import org.apache.hadoop.ozone.recon.types.GuiceInjectorUtilsForTests;
+
+/**
+ * Implementation for GuiceInjectorUtilsForTests.
+ */
+public class GuiceInjectorUtilsForTestsImpl implements
+    GuiceInjectorUtilsForTests {
+}

+ 116 - 70
hadoop-ozone/ozone-recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerKeyService.java

@@ -18,14 +18,11 @@
 
 package org.apache.hadoop.ozone.recon.api;
 
-import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_DB_DIR;
-import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_OM_SNAPSHOT_DB_DIR;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.io.File;
 import java.io.FileInputStream;
-import java.io.IOException;
 import java.io.InputStream;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -34,6 +31,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
+import javax.sql.DataSource;
 import javax.ws.rs.core.Response;
 
 import org.apache.commons.lang3.StringUtils;
@@ -45,21 +43,23 @@ import org.apache.hadoop.ozone.om.OMMetadataManager;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
 import org.apache.hadoop.ozone.recon.AbstractOMMetadataManagerTest;
+import org.apache.hadoop.ozone.recon.GuiceInjectorUtilsForTestsImpl;
 import org.apache.hadoop.ozone.recon.ReconUtils;
 import org.apache.hadoop.ozone.recon.api.types.ContainerMetadata;
+import org.apache.hadoop.ozone.recon.api.types.ContainersResponse;
 import org.apache.hadoop.ozone.recon.api.types.KeyMetadata;
+import org.apache.hadoop.ozone.recon.api.types.KeysResponse;
 import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
 import org.apache.hadoop.ozone.recon.spi.ContainerDBServiceProvider;
-import org.apache.hadoop.ozone.recon.spi.OzoneManagerServiceProvider;
-import org.apache.hadoop.ozone.recon.spi.impl.ContainerDBServiceProviderImpl;
 import org.apache.hadoop.ozone.recon.spi.impl.OzoneManagerServiceProviderImpl;
-import org.apache.hadoop.ozone.recon.spi.impl.ReconContainerDBProvider;
 import org.apache.hadoop.ozone.recon.tasks.ContainerKeyMapperTask;
 import org.apache.hadoop.utils.db.DBCheckpoint;
-import org.apache.hadoop.utils.db.DBStore;
 import org.apache.http.impl.client.CloseableHttpClient;
-import org.junit.Assert;
+import org.hadoop.ozone.recon.schema.StatsSchemaDefinition;
+import org.jooq.impl.DSL;
+import org.jooq.impl.DefaultConfiguration;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.powermock.api.mockito.PowerMockito;
@@ -68,9 +68,9 @@ import org.powermock.core.classloader.annotations.PrepareForTest;
 import org.powermock.modules.junit4.PowerMockRunner;
 
 import com.google.inject.AbstractModule;
-import com.google.inject.Guice;
 import com.google.inject.Injector;
-import com.google.inject.Singleton;
+
+import org.junit.rules.TemporaryFolder;
 
 /**
  * Test for container key service.
@@ -80,41 +80,57 @@ import com.google.inject.Singleton;
 @PrepareForTest(ReconUtils.class)
 public class TestContainerKeyService extends AbstractOMMetadataManagerTest {
 
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
   private ContainerDBServiceProvider containerDbServiceProvider;
   private OMMetadataManager omMetadataManager;
-  private ReconOMMetadataManager reconOMMetadataManager;
   private Injector injector;
   private OzoneManagerServiceProviderImpl ozoneManagerServiceProvider;
   private ContainerKeyService containerKeyService;
+  private GuiceInjectorUtilsForTestsImpl guiceInjectorTest =
+      new GuiceInjectorUtilsForTestsImpl();
+  private boolean isSetupDone = false;
 
-  @Before
-  public void setUp() throws Exception {
+  private void initializeInjector() throws Exception {
     omMetadataManager = initializeNewOmMetadataManager();
-    injector = Guice.createInjector(new AbstractModule() {
+    OzoneConfiguration configuration =
+        guiceInjectorTest.getTestOzoneConfiguration(temporaryFolder);
+
+    ozoneManagerServiceProvider = new OzoneManagerServiceProviderImpl(
+        configuration);
+    ReconOMMetadataManager reconOMMetadataManager =
+        getTestMetadataManager(omMetadataManager);
+
+    Injector parentInjector = guiceInjectorTest.getInjector(
+        ozoneManagerServiceProvider, reconOMMetadataManager, temporaryFolder);
+
+    injector = parentInjector.createChildInjector(new AbstractModule() {
       @Override
       protected void configure() {
-        try {
-          bind(OzoneConfiguration.class).toInstance(
-              getTestOzoneConfiguration());
-          reconOMMetadataManager = getTestMetadataManager(omMetadataManager);
-          bind(ReconOMMetadataManager.class).toInstance(reconOMMetadataManager);
-          bind(DBStore.class).toProvider(ReconContainerDBProvider.class).
-              in(Singleton.class);
-          bind(ContainerDBServiceProvider.class).to(
-              ContainerDBServiceProviderImpl.class).in(Singleton.class);
-          ozoneManagerServiceProvider = new OzoneManagerServiceProviderImpl(
-              getTestOzoneConfiguration());
-          bind(OzoneManagerServiceProvider.class)
-              .toInstance(ozoneManagerServiceProvider);
-          containerKeyService = new ContainerKeyService();
-          bind(ContainerKeyService.class).toInstance(containerKeyService);
-        } catch (IOException e) {
-          Assert.fail();
-        }
+        containerKeyService = new ContainerKeyService();
+        bind(ContainerKeyService.class).toInstance(containerKeyService);
       }
     });
-    containerDbServiceProvider = injector.getInstance(
-        ContainerDBServiceProvider.class);
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    // The following setup runs only once
+    if (!isSetupDone) {
+      initializeInjector();
+
+      DSL.using(new DefaultConfiguration().set(
+          injector.getInstance(DataSource.class)));
+
+      containerDbServiceProvider = injector.getInstance(
+          ContainerDBServiceProvider.class);
+
+      StatsSchemaDefinition schemaDefinition = injector.getInstance(
+          StatsSchemaDefinition.class);
+      schemaDefinition.initializeSchema();
+
+      isSetupDone = true;
+    }
 
     //Write Data to OM
     Pipeline pipeline = getRandomPipeline();
@@ -204,8 +220,11 @@ public class TestContainerKeyService extends AbstractOMMetadataManagerTest {
 
     Response response = containerKeyService.getKeysForContainer(1L, -1, "");
 
-    Collection<KeyMetadata> keyMetadataList =
-        (Collection<KeyMetadata>) response.getEntity();
+    KeysResponse responseObject = (KeysResponse) response.getEntity();
+    KeysResponse.KeysResponseData data = responseObject.getKeysResponseData();
+    Collection<KeyMetadata> keyMetadataList = data.getKeys();
+
+    assertEquals(3, data.getTotalCount());
     assertEquals(2, keyMetadataList.size());
 
     Iterator<KeyMetadata> iterator = keyMetadataList.iterator();
@@ -229,13 +248,19 @@ public class TestContainerKeyService extends AbstractOMMetadataManagerTest {
     assertEquals(104, blockIds.get(1L).iterator().next().getLocalID());
 
     response = containerKeyService.getKeysForContainer(3L, -1, "");
-    keyMetadataList = (Collection<KeyMetadata>) response.getEntity();
+    responseObject = (KeysResponse) response.getEntity();
+    data = responseObject.getKeysResponseData();
+    keyMetadataList = data.getKeys();
     assertTrue(keyMetadataList.isEmpty());
+    assertEquals(0, data.getTotalCount());
 
     // test if limit works as expected
     response = containerKeyService.getKeysForContainer(1L, 1, "");
-    keyMetadataList = (Collection<KeyMetadata>) response.getEntity();
+    responseObject = (KeysResponse) response.getEntity();
+    data = responseObject.getKeysResponseData();
+    keyMetadataList = data.getKeys();
     assertEquals(1, keyMetadataList.size());
+    assertEquals(3, data.getTotalCount());
   }
 
   @Test
@@ -244,8 +269,14 @@ public class TestContainerKeyService extends AbstractOMMetadataManagerTest {
     Response response = containerKeyService.getKeysForContainer(
         1L, -1, "/sampleVol/bucketOne/key_one");
 
-    Collection<KeyMetadata> keyMetadataList =
-        (Collection<KeyMetadata>) response.getEntity();
+    KeysResponse responseObject =
+        (KeysResponse) response.getEntity();
+
+    KeysResponse.KeysResponseData data =
+        responseObject.getKeysResponseData();
+    assertEquals(3, data.getTotalCount());
+
+    Collection<KeyMetadata> keyMetadataList = data.getKeys();
     assertEquals(1, keyMetadataList.size());
 
     Iterator<KeyMetadata> iterator = keyMetadataList.iterator();
@@ -257,7 +288,11 @@ public class TestContainerKeyService extends AbstractOMMetadataManagerTest {
 
     response = containerKeyService.getKeysForContainer(
         1L, -1, StringUtils.EMPTY);
-    keyMetadataList = (Collection<KeyMetadata>) response.getEntity();
+    responseObject = (KeysResponse) response.getEntity();
+    data = responseObject.getKeysResponseData();
+    keyMetadataList = data.getKeys();
+
+    assertEquals(3, data.getTotalCount());
     assertEquals(2, keyMetadataList.size());
     iterator = keyMetadataList.iterator();
     keyMetadata = iterator.next();
@@ -266,13 +301,19 @@ public class TestContainerKeyService extends AbstractOMMetadataManagerTest {
     // test for negative cases
     response = containerKeyService.getKeysForContainer(
         1L, -1, "/sampleVol/bucketOne/invalid_key");
-    keyMetadataList = (Collection<KeyMetadata>) response.getEntity();
+    responseObject = (KeysResponse) response.getEntity();
+    data = responseObject.getKeysResponseData();
+    keyMetadataList = data.getKeys();
+    assertEquals(3, data.getTotalCount());
     assertEquals(0, keyMetadataList.size());
 
     response = containerKeyService.getKeysForContainer(
         5L, -1, "");
-    keyMetadataList = (Collection<KeyMetadata>) response.getEntity();
+    responseObject = (KeysResponse) response.getEntity();
+    data = responseObject.getKeysResponseData();
+    keyMetadataList = data.getKeys();
     assertEquals(0, keyMetadataList.size());
+    assertEquals(0, data.getTotalCount());
   }
 
   @Test
@@ -280,8 +321,14 @@ public class TestContainerKeyService extends AbstractOMMetadataManagerTest {
 
     Response response = containerKeyService.getContainers(-1, 0L);
 
-    List<ContainerMetadata> containers = new ArrayList<>(
-        (Collection<ContainerMetadata>) response.getEntity());
+    ContainersResponse responseObject =
+        (ContainersResponse) response.getEntity();
+
+    ContainersResponse.ContainersResponseData data =
+        responseObject.getContainersResponseData();
+    assertEquals(2, data.getTotalCount());
+
+    List<ContainerMetadata> containers = new ArrayList<>(data.getContainers());
 
     Iterator<ContainerMetadata> iterator = containers.iterator();
 
@@ -297,9 +344,11 @@ public class TestContainerKeyService extends AbstractOMMetadataManagerTest {
 
     // test if limit works as expected
     response = containerKeyService.getContainers(1, 0L);
-    containers = new ArrayList<>(
-        (Collection<ContainerMetadata>) response.getEntity());
+    responseObject = (ContainersResponse) response.getEntity();
+    data = responseObject.getContainersResponseData();
+    containers = new ArrayList<>(data.getContainers());
     assertEquals(1, containers.size());
+    assertEquals(2, data.getTotalCount());
   }
 
   @Test
@@ -307,8 +356,14 @@ public class TestContainerKeyService extends AbstractOMMetadataManagerTest {
 
     Response response = containerKeyService.getContainers(1, 1L);
 
-    List<ContainerMetadata> containers = new ArrayList<>(
-        (Collection<ContainerMetadata>) response.getEntity());
+    ContainersResponse responseObject =
+        (ContainersResponse) response.getEntity();
+
+    ContainersResponse.ContainersResponseData data =
+        responseObject.getContainersResponseData();
+    assertEquals(2, data.getTotalCount());
+
+    List<ContainerMetadata> containers = new ArrayList<>(data.getContainers());
 
     Iterator<ContainerMetadata> iterator = containers.iterator();
 
@@ -318,37 +373,28 @@ public class TestContainerKeyService extends AbstractOMMetadataManagerTest {
     assertEquals(2L, containerMetadata.getContainerID());
 
     response = containerKeyService.getContainers(-1, 0L);
-    containers = new ArrayList<>(
-        (Collection<ContainerMetadata>) response.getEntity());
+    responseObject = (ContainersResponse) response.getEntity();
+    data = responseObject.getContainersResponseData();
+    containers = new ArrayList<>(data.getContainers());
     assertEquals(2, containers.size());
+    assertEquals(2, data.getTotalCount());
     iterator = containers.iterator();
     containerMetadata = iterator.next();
     assertEquals(1L, containerMetadata.getContainerID());
 
     // test for negative cases
     response = containerKeyService.getContainers(-1, 5L);
-    containers = new ArrayList<>(
-        (Collection<ContainerMetadata>) response.getEntity());
+    responseObject = (ContainersResponse) response.getEntity();
+    data = responseObject.getContainersResponseData();
+    containers = new ArrayList<>(data.getContainers());
     assertEquals(0, containers.size());
+    assertEquals(2, data.getTotalCount());
 
     response = containerKeyService.getContainers(-1, -1L);
-    containers = new ArrayList<>(
-        (Collection<ContainerMetadata>) response.getEntity());
+    responseObject = (ContainersResponse) response.getEntity();
+    data = responseObject.getContainersResponseData();
+    containers = new ArrayList<>(data.getContainers());
     assertEquals(2, containers.size());
-  }
-
-  /**
-   * Get Test OzoneConfiguration instance.
-   * @return OzoneConfiguration
-   * @throws IOException ioEx.
-   */
-  private OzoneConfiguration getTestOzoneConfiguration()
-      throws IOException {
-    OzoneConfiguration configuration = new OzoneConfiguration();
-    configuration.set(OZONE_RECON_OM_SNAPSHOT_DB_DIR,
-        temporaryFolder.newFolder().getAbsolutePath());
-    configuration.set(OZONE_RECON_DB_DIR, temporaryFolder.newFolder()
-        .getAbsolutePath());
-    return configuration;
+    assertEquals(2, data.getTotalCount());
   }
 }

+ 9 - 3
hadoop-ozone/ozone-recon/src/test/java/org/apache/hadoop/ozone/recon/persistence/AbstractSqlDatabaseTest.java

@@ -43,17 +43,17 @@ public abstract class AbstractSqlDatabaseTest {
 
   @ClassRule
   public static TemporaryFolder temporaryFolder = new TemporaryFolder();
-  private static File tempDir;
 
   private static Injector injector;
   private static DSLContext dslContext;
 
   @BeforeClass
   public static void setup() throws IOException {
-    tempDir = temporaryFolder.newFolder();
+    File tempDir = temporaryFolder.newFolder();
 
     DataSourceConfigurationProvider configurationProvider =
-        new DataSourceConfigurationProvider();
+        new DataSourceConfigurationProvider(tempDir);
+
     JooqPersistenceModule persistenceModule =
         new JooqPersistenceModule(configurationProvider);
 
@@ -86,6 +86,12 @@ public abstract class AbstractSqlDatabaseTest {
   public static class DataSourceConfigurationProvider implements
       Provider<DataSourceConfiguration> {
 
+    private final File tempDir;
+
+    public DataSourceConfigurationProvider(File tempDir) {
+      this.tempDir = tempDir;
+    }
+
     @Override
     public DataSourceConfiguration get() {
       return new DataSourceConfiguration() {

+ 147 - 0
hadoop-ozone/ozone-recon/src/test/java/org/apache/hadoop/ozone/recon/persistence/TestStatsSchemaDefinition.java

@@ -0,0 +1,147 @@
+/**
+ * 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.recon.persistence;
+
+import org.apache.commons.lang3.tuple.ImmutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+import org.hadoop.ozone.recon.schema.StatsSchemaDefinition;
+import org.hadoop.ozone.recon.schema.tables.daos.GlobalStatsDao;
+import org.hadoop.ozone.recon.schema.tables.pojos.GlobalStats;
+import org.jooq.Configuration;
+import org.junit.Assert;
+import org.junit.Test;
+
+import javax.sql.DataSource;
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.ResultSet;
+import java.sql.Timestamp;
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.hadoop.ozone.recon.schema.StatsSchemaDefinition.GLOBAL_STATS_TABLE_NAME;
+
+/**
+ * Class used to test StatsSchemaDefinition.
+ */
+public class TestStatsSchemaDefinition extends AbstractSqlDatabaseTest {
+
+  @Test
+  public void testIfStatsSchemaCreated() throws Exception {
+    StatsSchemaDefinition schemaDefinition = getInjector().getInstance(
+        StatsSchemaDefinition.class);
+
+    schemaDefinition.initializeSchema();
+
+    Connection connection =
+        getInjector().getInstance(DataSource.class).getConnection();
+    // Verify table definition
+    DatabaseMetaData metaData = connection.getMetaData();
+    ResultSet resultSet = metaData.getColumns(null, null,
+        GLOBAL_STATS_TABLE_NAME, null);
+
+    List<Pair<String, Integer>> expectedPairs = new ArrayList<>();
+
+    expectedPairs.add(new ImmutablePair<>("key", Types.VARCHAR));
+    expectedPairs.add(new ImmutablePair<>("value", Types.INTEGER));
+    expectedPairs.add(new ImmutablePair<>("last_updated_timestamp",
+        Types.VARCHAR));
+
+    List<Pair<String, Integer>> actualPairs = new ArrayList<>();
+
+    while (resultSet.next()) {
+      actualPairs.add(new ImmutablePair<>(resultSet.getString("COLUMN_NAME"),
+          resultSet.getInt("DATA_TYPE")));
+    }
+
+    Assert.assertEquals(3, actualPairs.size());
+    Assert.assertEquals(expectedPairs, actualPairs);
+  }
+
+  @Test
+  public void testGlobalStatsCRUDOperations() throws Exception {
+    // Verify table exists
+    StatsSchemaDefinition schemaDefinition = getInjector().getInstance(
+        StatsSchemaDefinition.class);
+
+    schemaDefinition.initializeSchema();
+
+    DataSource ds = getInjector().getInstance(DataSource.class);
+    Connection connection = ds.getConnection();
+
+    DatabaseMetaData metaData = connection.getMetaData();
+    ResultSet resultSet = metaData.getTables(null, null,
+        GLOBAL_STATS_TABLE_NAME, null);
+
+    while (resultSet.next()) {
+      Assert.assertEquals(GLOBAL_STATS_TABLE_NAME,
+          resultSet.getString("TABLE_NAME"));
+    }
+
+    GlobalStatsDao dao = new GlobalStatsDao(
+        getInjector().getInstance(Configuration.class));
+
+    long now = System.currentTimeMillis();
+    GlobalStats newRecord = new GlobalStats();
+    newRecord.setLastUpdatedTimestamp(new Timestamp(now));
+    newRecord.setKey("key1");
+    newRecord.setValue(500L);
+
+    // Create
+    dao.insert(newRecord);
+    GlobalStats newRecord2 = new GlobalStats();
+    newRecord2.setLastUpdatedTimestamp(new Timestamp(now + 1000L));
+    newRecord2.setKey("key2");
+    newRecord2.setValue(10L);
+    dao.insert(newRecord2);
+
+    // Read
+    GlobalStats dbRecord = dao.findById("key1");
+
+    Assert.assertEquals("key1", dbRecord.getKey());
+    Assert.assertEquals(Long.valueOf(500), dbRecord.getValue());
+    Assert.assertEquals(new Timestamp(now), dbRecord.getLastUpdatedTimestamp());
+
+    dbRecord = dao.findById("key2");
+    Assert.assertEquals("key2", dbRecord.getKey());
+    Assert.assertEquals(Long.valueOf(10), dbRecord.getValue());
+    Assert.assertEquals(new Timestamp(now + 1000L),
+        dbRecord.getLastUpdatedTimestamp());
+
+    // Update
+    dbRecord.setValue(100L);
+    dbRecord.setLastUpdatedTimestamp(new Timestamp(now + 2000L));
+    dao.update(dbRecord);
+
+    // Read updated
+    dbRecord = dao.findById("key2");
+
+    Assert.assertEquals(new Timestamp(now + 2000L),
+        dbRecord.getLastUpdatedTimestamp());
+    Assert.assertEquals(Long.valueOf(100L), dbRecord.getValue());
+
+    // Delete
+    dao.deleteById("key1");
+
+    // Verify
+    dbRecord = dao.findById("key1");
+
+    Assert.assertNull(dbRecord);
+  }
+}

+ 196 - 130
hadoop-ozone/ozone-recon/src/test/java/org/apache/hadoop/ozone/recon/spi/impl/TestContainerDBServiceProviderImpl.java

@@ -18,67 +18,93 @@
 
 package org.apache.hadoop.ozone.recon.spi.impl;
 
-import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_DB_DIR;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
-import java.io.File;
-import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.recon.GuiceInjectorUtilsForTestsImpl;
 import org.apache.hadoop.ozone.recon.api.types.ContainerKeyPrefix;
 import org.apache.hadoop.ozone.recon.api.types.ContainerMetadata;
 import org.apache.hadoop.ozone.recon.spi.ContainerDBServiceProvider;
-import org.apache.hadoop.utils.db.DBStore;
-import org.junit.After;
+import org.hadoop.ozone.recon.schema.StatsSchemaDefinition;
+import org.jooq.impl.DSL;
+import org.jooq.impl.DefaultConfiguration;
 import org.junit.Assert;
 import org.junit.Before;
-import org.junit.Rule;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
 
-import com.google.inject.AbstractModule;
-import com.google.inject.Guice;
 import com.google.inject.Injector;
-import com.google.inject.Singleton;
+
+import javax.sql.DataSource;
 
 /**
  * Unit Tests for ContainerDBServiceProviderImpl.
  */
 public class TestContainerDBServiceProviderImpl {
 
-  @Rule
-  public TemporaryFolder tempFolder = new TemporaryFolder();
+  @ClassRule
+  public static TemporaryFolder tempFolder = new TemporaryFolder();
+  private static ContainerDBServiceProvider containerDbServiceProvider;
+  private static Injector injector;
+  private static GuiceInjectorUtilsForTestsImpl guiceInjectorTest =
+      new GuiceInjectorUtilsForTestsImpl();
 
-  private ContainerDBServiceProvider containerDbServiceProvider;
-  private Injector injector;
+  private String keyPrefix1 = "V3/B1/K1";
+  private String keyPrefix2 = "V3/B1/K2";
+  private String keyPrefix3 = "V3/B2/K1";
+
+  private void populateKeysInContainers(long containerId1, long containerId2)
+      throws Exception {
+
+    ContainerKeyPrefix containerKeyPrefix1 = new
+        ContainerKeyPrefix(containerId1, keyPrefix1, 0);
+    containerDbServiceProvider.storeContainerKeyMapping(containerKeyPrefix1,
+        1);
+
+    ContainerKeyPrefix containerKeyPrefix2 = new ContainerKeyPrefix(
+        containerId1, keyPrefix2, 0);
+    containerDbServiceProvider.storeContainerKeyMapping(containerKeyPrefix2,
+        2);
+
+    ContainerKeyPrefix containerKeyPrefix3 = new ContainerKeyPrefix(
+        containerId2, keyPrefix3, 0);
+
+    containerDbServiceProvider.storeContainerKeyMapping(containerKeyPrefix3,
+        3);
+  }
+
+  private static void initializeInjector() throws Exception {
+    injector = guiceInjectorTest.getInjector(
+        null, null, tempFolder);
+  }
+
+  @BeforeClass
+  public static void setupOnce() throws Exception {
+
+    initializeInjector();
+
+    DSL.using(new DefaultConfiguration().set(
+        injector.getInstance(DataSource.class)));
 
-  @Before
-  public void setUp() throws IOException {
-    tempFolder.create();
-    injector = Guice.createInjector(new AbstractModule() {
-      @Override
-      protected void configure() {
-        File dbDir = tempFolder.getRoot();
-        OzoneConfiguration configuration = new OzoneConfiguration();
-        configuration.set(OZONE_RECON_DB_DIR, dbDir.getAbsolutePath());
-        bind(OzoneConfiguration.class).toInstance(configuration);
-        bind(DBStore.class).toProvider(ReconContainerDBProvider.class).in(
-            Singleton.class);
-        bind(ContainerDBServiceProvider.class).to(
-            ContainerDBServiceProviderImpl.class).in(Singleton.class);
-      }
-    });
     containerDbServiceProvider = injector.getInstance(
         ContainerDBServiceProvider.class);
+
+    StatsSchemaDefinition schemaDefinition = injector.getInstance(
+        StatsSchemaDefinition.class);
+    schemaDefinition.initializeSchema();
   }
 
-  @After
-  public void tearDown() throws Exception {
-    tempFolder.delete();
+  @Before
+  public void setUp() throws Exception {
+    // Reset containerDB before running each test
+    containerDbServiceProvider.initNewContainerDB(null);
   }
 
   @Test
@@ -104,7 +130,7 @@ public class TestContainerDBServiceProviderImpl {
     }
 
     assertEquals(1, containerDbServiceProvider
-        .getCountForForContainerKeyPrefix(ckp1).intValue());
+        .getCountForContainerKeyPrefix(ckp1).intValue());
 
     prefixCounts.clear();
     prefixCounts.put(ckp2, 12);
@@ -127,7 +153,7 @@ public class TestContainerDBServiceProviderImpl {
     assertEquals(15, keyPrefixesForContainer.get(ckp5).intValue());
 
     assertEquals(0, containerDbServiceProvider
-        .getCountForForContainerKeyPrefix(ckp1).intValue());
+        .getCountForContainerKeyPrefix(ckp1).intValue());
   }
 
   @Test
@@ -135,9 +161,9 @@ public class TestContainerDBServiceProviderImpl {
 
     long containerId = System.currentTimeMillis();
     Map<String, Integer> prefixCounts = new HashMap<>();
-    prefixCounts.put("V1/B1/K1", 1);
-    prefixCounts.put("V1/B1/K2", 2);
-    prefixCounts.put("V1/B2/K3", 3);
+    prefixCounts.put(keyPrefix1, 1);
+    prefixCounts.put(keyPrefix2, 2);
+    prefixCounts.put(keyPrefix3, 3);
 
     for (String prefix : prefixCounts.keySet()) {
       ContainerKeyPrefix containerKeyPrefix = new ContainerKeyPrefix(
@@ -146,89 +172,119 @@ public class TestContainerDBServiceProviderImpl {
           containerKeyPrefix, prefixCounts.get(prefix));
     }
 
-    Assert.assertTrue(
-        containerDbServiceProvider.getCountForForContainerKeyPrefix(
-            new ContainerKeyPrefix(containerId, "V1/B1/K1",
-                0)) == 1);
-    Assert.assertTrue(
-        containerDbServiceProvider.getCountForForContainerKeyPrefix(
-            new ContainerKeyPrefix(containerId, "V1/B1/K2",
-                0)) == 2);
-    Assert.assertTrue(
-        containerDbServiceProvider.getCountForForContainerKeyPrefix(
-            new ContainerKeyPrefix(containerId, "V1/B2/K3",
-                0)) == 3);
+    Assert.assertEquals(1,
+        containerDbServiceProvider.getCountForContainerKeyPrefix(
+            new ContainerKeyPrefix(containerId, keyPrefix1,
+                0)).longValue());
+    Assert.assertEquals(2,
+        containerDbServiceProvider.getCountForContainerKeyPrefix(
+            new ContainerKeyPrefix(containerId, keyPrefix2,
+                0)).longValue());
+    Assert.assertEquals(3,
+        containerDbServiceProvider.getCountForContainerKeyPrefix(
+            new ContainerKeyPrefix(containerId, keyPrefix3,
+                0)).longValue());
+  }
+
+  @Test
+  public void testStoreContainerKeyCount() throws Exception {
+    long containerId = 1L;
+    long nextContainerId = 2L;
+    containerDbServiceProvider.storeContainerKeyCount(containerId, 2L);
+    containerDbServiceProvider.storeContainerKeyCount(nextContainerId, 3L);
+
+    assertEquals(2,
+        containerDbServiceProvider.getKeyCountForContainer(containerId));
+    assertEquals(3,
+        containerDbServiceProvider.getKeyCountForContainer(nextContainerId));
+
+    containerDbServiceProvider.storeContainerKeyCount(containerId, 20L);
+    assertEquals(20,
+        containerDbServiceProvider.getKeyCountForContainer(containerId));
   }
 
   @Test
-  public void testGetCountForForContainerKeyPrefix() throws Exception {
+  public void testGetKeyCountForContainer() throws Exception {
+    long containerId = 1L;
+    long nextContainerId = 2L;
+    containerDbServiceProvider.storeContainerKeyCount(containerId, 2L);
+    containerDbServiceProvider.storeContainerKeyCount(nextContainerId, 3L);
+
+    assertEquals(2,
+        containerDbServiceProvider.getKeyCountForContainer(containerId));
+    assertEquals(3,
+        containerDbServiceProvider.getKeyCountForContainer(nextContainerId));
+
+    assertEquals(0,
+        containerDbServiceProvider.getKeyCountForContainer(5L));
+  }
+
+  @Test
+  public void testDoesContainerExists() throws Exception {
+    long containerId = 1L;
+    long nextContainerId = 2L;
+    containerDbServiceProvider.storeContainerKeyCount(containerId, 2L);
+    containerDbServiceProvider.storeContainerKeyCount(nextContainerId, 3L);
+
+    assertTrue(containerDbServiceProvider.doesContainerExists(containerId));
+    assertTrue(containerDbServiceProvider.doesContainerExists(nextContainerId));
+    assertFalse(containerDbServiceProvider.doesContainerExists(0L));
+    assertFalse(containerDbServiceProvider.doesContainerExists(3L));
+  }
+
+  @Test
+  public void testGetCountForContainerKeyPrefix() throws Exception {
     long containerId = System.currentTimeMillis();
 
     containerDbServiceProvider.storeContainerKeyMapping(new
-        ContainerKeyPrefix(containerId, "V2/B1/K1"), 2);
+        ContainerKeyPrefix(containerId, keyPrefix1), 2);
 
     Integer count = containerDbServiceProvider.
-        getCountForForContainerKeyPrefix(new ContainerKeyPrefix(containerId,
-            "V2/B1/K1"));
-    assertTrue(count == 2);
+        getCountForContainerKeyPrefix(new ContainerKeyPrefix(containerId,
+            keyPrefix1));
+    assertEquals(2L, count.longValue());
+
+    count = containerDbServiceProvider.
+        getCountForContainerKeyPrefix(new ContainerKeyPrefix(containerId,
+            "invalid"));
+    assertEquals(0L, count.longValue());
   }
 
   @Test
   public void testGetKeyPrefixesForContainer() throws Exception {
-    long containerId = System.currentTimeMillis();
+    long containerId = 1L;
+    long nextContainerId = 2L;
+    populateKeysInContainers(containerId, nextContainerId);
 
     ContainerKeyPrefix containerKeyPrefix1 = new
-        ContainerKeyPrefix(containerId, "V3/B1/K1", 0);
-    containerDbServiceProvider.storeContainerKeyMapping(containerKeyPrefix1,
-        1);
-
+        ContainerKeyPrefix(containerId, keyPrefix1, 0);
     ContainerKeyPrefix containerKeyPrefix2 = new ContainerKeyPrefix(
-        containerId, "V3/B1/K2", 0);
-    containerDbServiceProvider.storeContainerKeyMapping(containerKeyPrefix2,
-        2);
-
-    long nextContainerId = containerId + 1000L;
+        containerId, keyPrefix2, 0);
     ContainerKeyPrefix containerKeyPrefix3 = new ContainerKeyPrefix(
-        nextContainerId, "V3/B2/K1", 0);
-    containerDbServiceProvider.storeContainerKeyMapping(containerKeyPrefix3,
-        3);
+        nextContainerId, keyPrefix3, 0);
+
 
     Map<ContainerKeyPrefix, Integer> keyPrefixMap =
         containerDbServiceProvider.getKeyPrefixesForContainer(containerId);
-    assertTrue(keyPrefixMap.size() == 2);
+    assertEquals(2, keyPrefixMap.size());
 
-    assertTrue(keyPrefixMap.get(containerKeyPrefix1) == 1);
-    assertTrue(keyPrefixMap.get(containerKeyPrefix2) == 2);
+    assertEquals(1, keyPrefixMap.get(containerKeyPrefix1).longValue());
+    assertEquals(2, keyPrefixMap.get(containerKeyPrefix2).longValue());
 
     keyPrefixMap = containerDbServiceProvider.getKeyPrefixesForContainer(
         nextContainerId);
-    assertTrue(keyPrefixMap.size() == 1);
-    assertTrue(keyPrefixMap.get(containerKeyPrefix3) == 3);
+    assertEquals(1, keyPrefixMap.size());
+    assertEquals(3, keyPrefixMap.get(containerKeyPrefix3).longValue());
   }
 
   @Test
   public void testGetKeyPrefixesForContainerWithKeyPrefix() throws Exception {
-    long containerId = System.currentTimeMillis();
-
-    String keyPrefix1 = "V3/B1/K1";
-    String keyPrefix2 = "V3/B1/K2";
-    String keyPrefix3 = "V3/B2/K1";
-
-    ContainerKeyPrefix containerKeyPrefix1 = new
-        ContainerKeyPrefix(containerId, keyPrefix1, 0);
-    containerDbServiceProvider.storeContainerKeyMapping(containerKeyPrefix1,
-        1);
+    long containerId = 1L;
+    long nextContainerId = 2L;
+    populateKeysInContainers(containerId, nextContainerId);
 
     ContainerKeyPrefix containerKeyPrefix2 = new ContainerKeyPrefix(
         containerId, keyPrefix2, 0);
-    containerDbServiceProvider.storeContainerKeyMapping(containerKeyPrefix2,
-        2);
-
-    long nextContainerId = containerId + 1000L;
-    ContainerKeyPrefix containerKeyPrefix3 = new ContainerKeyPrefix(
-        nextContainerId, keyPrefix3, 0);
-    containerDbServiceProvider.storeContainerKeyMapping(containerKeyPrefix3,
-        3);
 
     Map<ContainerKeyPrefix, Integer> keyPrefixMap =
         containerDbServiceProvider.getKeyPrefixesForContainer(containerId,
@@ -250,33 +306,15 @@ public class TestContainerDBServiceProviderImpl {
     assertEquals(0, keyPrefixMap.size());
 
     keyPrefixMap = containerDbServiceProvider.getKeyPrefixesForContainer(
-        1L, "");
+        10L, "");
     assertEquals(0, keyPrefixMap.size());
   }
 
   @Test
-  public void testGetContainersWithPrevKey() throws Exception {
-    long containerId = System.currentTimeMillis();
-
-    String keyPrefix1 = "V3/B1/K1";
-    String keyPrefix2 = "V3/B1/K2";
-    String keyPrefix3 = "V3/B2/K1";
-
-    ContainerKeyPrefix containerKeyPrefix1 = new
-        ContainerKeyPrefix(containerId, keyPrefix1, 0);
-    containerDbServiceProvider.storeContainerKeyMapping(containerKeyPrefix1,
-        1);
-
-    ContainerKeyPrefix containerKeyPrefix2 = new ContainerKeyPrefix(
-        containerId, keyPrefix2, 0);
-    containerDbServiceProvider.storeContainerKeyMapping(containerKeyPrefix2,
-        2);
-
-    long nextContainerId = containerId + 1000L;
-    ContainerKeyPrefix containerKeyPrefix3 = new ContainerKeyPrefix(
-        nextContainerId, keyPrefix3, 0);
-    containerDbServiceProvider.storeContainerKeyMapping(containerKeyPrefix3,
-        3);
+  public void testGetContainersWithPrevContainer() throws Exception {
+    long containerId = 1L;
+    long nextContainerId = 2L;
+    populateKeysInContainers(containerId, nextContainerId);
 
     Map<Long, ContainerMetadata> containerMap =
         containerDbServiceProvider.getContainers(-1, 0L);
@@ -304,7 +342,7 @@ public class TestContainerDBServiceProviderImpl {
 
     // test for negative cases
     containerMap = containerDbServiceProvider.getContainers(
-        -1, 1L);
+        -1, 10L);
     assertEquals(0, containerMap.size());
 
     containerMap = containerDbServiceProvider.getContainers(
@@ -313,27 +351,55 @@ public class TestContainerDBServiceProviderImpl {
   }
 
   @Test
-  public void testDeleteContainerMapping() throws IOException {
-    long containerId = System.currentTimeMillis();
-
-    ContainerKeyPrefix containerKeyPrefix1 = new
-        ContainerKeyPrefix(containerId, "V3/B1/K1", 0);
-    containerDbServiceProvider.storeContainerKeyMapping(containerKeyPrefix1,
-        1);
-
-    ContainerKeyPrefix containerKeyPrefix2 = new ContainerKeyPrefix(
-        containerId, "V3/B1/K2", 0);
-    containerDbServiceProvider.storeContainerKeyMapping(containerKeyPrefix2,
-        2);
+  public void testDeleteContainerMapping() throws Exception {
+    long containerId = 1L;
+    long nextContainerId = 2L;
+    populateKeysInContainers(containerId, nextContainerId);
 
     Map<ContainerKeyPrefix, Integer> keyPrefixMap =
         containerDbServiceProvider.getKeyPrefixesForContainer(containerId);
-    assertTrue(keyPrefixMap.size() == 2);
+    assertEquals(2, keyPrefixMap.size());
 
     containerDbServiceProvider.deleteContainerMapping(new ContainerKeyPrefix(
-        containerId, "V3/B1/K2", 0));
+        containerId, keyPrefix2, 0));
     keyPrefixMap =
         containerDbServiceProvider.getKeyPrefixesForContainer(containerId);
-    assertTrue(keyPrefixMap.size() == 1);
+    assertEquals(1, keyPrefixMap.size());
+  }
+
+  @Test
+  public void testGetCountForContainers() throws Exception {
+
+    assertEquals(0, containerDbServiceProvider.getCountForContainers());
+
+    containerDbServiceProvider.storeContainerCount(5L);
+
+    assertEquals(5L, containerDbServiceProvider.getCountForContainers());
+    containerDbServiceProvider.incrementContainerCountBy(1L);
+
+    assertEquals(6L, containerDbServiceProvider.getCountForContainers());
+
+    containerDbServiceProvider.storeContainerCount(10L);
+    assertEquals(10L, containerDbServiceProvider.getCountForContainers());
+  }
+
+  @Test
+  public void testStoreContainerCount() throws Exception {
+    containerDbServiceProvider.storeContainerCount(3L);
+    assertEquals(3L, containerDbServiceProvider.getCountForContainers());
+
+    containerDbServiceProvider.storeContainerCount(5L);
+    assertEquals(5L, containerDbServiceProvider.getCountForContainers());
+  }
+
+  @Test
+  public void testIncrementContainerCountBy() throws Exception {
+    assertEquals(0, containerDbServiceProvider.getCountForContainers());
+
+    containerDbServiceProvider.incrementContainerCountBy(1L);
+    assertEquals(1L, containerDbServiceProvider.getCountForContainers());
+
+    containerDbServiceProvider.incrementContainerCountBy(3L);
+    assertEquals(4L, containerDbServiceProvider.getCountForContainers());
   }
 }

+ 15 - 38
hadoop-ozone/ozone-recon/src/test/java/org/apache/hadoop/ozone/recon/spi/impl/TestOzoneManagerServiceProviderImpl.java

@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.ozone.recon.spi.impl;
 
-import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_OM_SNAPSHOT_DB_DIR;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
@@ -26,17 +25,15 @@ import java.io.BufferedWriter;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileWriter;
-import java.io.IOException;
 import java.io.InputStream;
 import java.nio.file.Paths;
 
-import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.OmUtils;
 import org.apache.hadoop.ozone.om.OMMetadataManager;
 import org.apache.hadoop.ozone.recon.AbstractOMMetadataManagerTest;
+import org.apache.hadoop.ozone.recon.GuiceInjectorUtilsForTestsImpl;
 import org.apache.hadoop.ozone.recon.ReconUtils;
 import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
-import org.apache.hadoop.ozone.recon.spi.OzoneManagerServiceProvider;
 import org.apache.hadoop.utils.db.DBCheckpoint;
 import org.apache.http.impl.client.CloseableHttpClient;
 import org.junit.Assert;
@@ -50,8 +47,6 @@ import org.powermock.core.classloader.annotations.PowerMockIgnore;
 import org.powermock.core.classloader.annotations.PrepareForTest;
 import org.powermock.modules.junit4.PowerMockRunner;
 
-import com.google.inject.AbstractModule;
-import com.google.inject.Guice;
 import com.google.inject.Injector;
 
 /**
@@ -66,7 +61,10 @@ public class TestOzoneManagerServiceProviderImpl extends
   private OMMetadataManager omMetadataManager;
   private ReconOMMetadataManager reconOMMetadataManager;
   private Injector injector;
+  private GuiceInjectorUtilsForTestsImpl guiceInjectorTest =
+      new GuiceInjectorUtilsForTestsImpl();
   private OzoneManagerServiceProviderImpl ozoneManagerServiceProvider;
+  private boolean isSetupDone = false;
 
   @Rule
   public TemporaryFolder temporaryFolder = new TemporaryFolder();
@@ -74,25 +72,17 @@ public class TestOzoneManagerServiceProviderImpl extends
   @Before
   public void setUp() throws Exception {
     omMetadataManager = initializeNewOmMetadataManager();
-    injector = Guice.createInjector(new AbstractModule() {
-      @Override
-      protected void configure() {
-        try {
-          initializeNewOmMetadataManager();
-          writeDataToOm(omMetadataManager, "key_one");
-          bind(OzoneConfiguration.class).toInstance(
-              getTestOzoneConfiguration());
-          reconOMMetadataManager = getTestMetadataManager(omMetadataManager);
-          bind(ReconOMMetadataManager.class).toInstance(reconOMMetadataManager);
-          ozoneManagerServiceProvider = new OzoneManagerServiceProviderImpl(
-              getTestOzoneConfiguration());
-          bind(OzoneManagerServiceProvider.class)
-              .toInstance(ozoneManagerServiceProvider);
-        } catch (IOException e) {
-          Assert.fail();
-        }
-      }
-    });
+    writeDataToOm(omMetadataManager, "key_one");
+    reconOMMetadataManager = getTestMetadataManager(omMetadataManager);
+    ozoneManagerServiceProvider =
+        new OzoneManagerServiceProviderImpl(
+            guiceInjectorTest.getTestOzoneConfiguration(temporaryFolder));
+    if (!isSetupDone) {
+      injector = guiceInjectorTest.getInjector(ozoneManagerServiceProvider,
+          reconOMMetadataManager, temporaryFolder);
+
+      isSetupDone = true;
+    }
   }
 
   @Test
@@ -168,17 +158,4 @@ public class TestOzoneManagerServiceProviderImpl extends
     assertTrue(checkpoint.getCheckpointLocation().toFile()
         .listFiles().length == 2);
   }
-
-  /**
-   * Get Test OzoneConfiguration instance.
-   * @return OzoneConfiguration
-   * @throws IOException ioEx.
-   */
-  private OzoneConfiguration getTestOzoneConfiguration() throws IOException {
-    OzoneConfiguration configuration = new OzoneConfiguration();
-    configuration.set(OZONE_RECON_OM_SNAPSHOT_DB_DIR,
-        temporaryFolder.newFolder().getAbsolutePath());
-    return configuration;
-  }
-
 }

+ 71 - 56
hadoop-ozone/ozone-recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestContainerKeyMapperTask.java

@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.ozone.recon.tasks;
 
-import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_DB_DIR;
-import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_OM_SNAPSHOT_DB_DIR;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -38,27 +36,25 @@ import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
 import org.apache.hadoop.ozone.recon.AbstractOMMetadataManagerTest;
+import org.apache.hadoop.ozone.recon.GuiceInjectorUtilsForTestsImpl;
 import org.apache.hadoop.ozone.recon.ReconUtils;
 import org.apache.hadoop.ozone.recon.api.types.ContainerKeyPrefix;
 import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
 import org.apache.hadoop.ozone.recon.spi.ContainerDBServiceProvider;
-import org.apache.hadoop.ozone.recon.spi.OzoneManagerServiceProvider;
-import org.apache.hadoop.ozone.recon.spi.impl.ContainerDBServiceProviderImpl;
 import org.apache.hadoop.ozone.recon.spi.impl.OzoneManagerServiceProviderImpl;
-import org.apache.hadoop.ozone.recon.spi.impl.ReconContainerDBProvider;
-import org.apache.hadoop.utils.db.DBStore;
-import org.junit.Assert;
+import org.hadoop.ozone.recon.schema.StatsSchemaDefinition;
+import org.jooq.impl.DSL;
+import org.jooq.impl.DefaultConfiguration;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
 import org.junit.runner.RunWith;
 import org.powermock.core.classloader.annotations.PowerMockIgnore;
 import org.powermock.core.classloader.annotations.PrepareForTest;
 import org.powermock.modules.junit4.PowerMockRunner;
-
-import com.google.inject.AbstractModule;
-import com.google.inject.Guice;
 import com.google.inject.Injector;
-import com.google.inject.Singleton;
+import javax.sql.DataSource;
 
 /**
  * Unit test for Container Key mapper task.
@@ -73,33 +69,49 @@ public class TestContainerKeyMapperTask extends AbstractOMMetadataManagerTest {
   private ReconOMMetadataManager reconOMMetadataManager;
   private Injector injector;
   private OzoneManagerServiceProviderImpl ozoneManagerServiceProvider;
+  private boolean setUpIsDone = false;
+  private GuiceInjectorUtilsForTestsImpl guiceInjectorTest =
+      new GuiceInjectorUtilsForTestsImpl();
+
+  private Injector getInjector() {
+    return injector;
+  }
+
+  @Rule
+  TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  private void initializeInjector() throws Exception {
+    omMetadataManager = initializeNewOmMetadataManager();
+    OzoneConfiguration configuration =
+        guiceInjectorTest.getTestOzoneConfiguration(temporaryFolder);
+
+    ozoneManagerServiceProvider = new OzoneManagerServiceProviderImpl(
+        configuration);
+    reconOMMetadataManager = getTestMetadataManager(omMetadataManager);
+
+    injector = guiceInjectorTest.getInjector(
+        ozoneManagerServiceProvider, reconOMMetadataManager, temporaryFolder);
+  }
 
   @Before
   public void setUp() throws Exception {
-    omMetadataManager = initializeNewOmMetadataManager();
-    injector = Guice.createInjector(new AbstractModule() {
-      @Override
-      protected void configure() {
-        try {
-          bind(OzoneConfiguration.class).toInstance(
-              getTestOzoneConfiguration());
-
-          reconOMMetadataManager = getTestMetadataManager(omMetadataManager);
-          bind(ReconOMMetadataManager.class).toInstance(reconOMMetadataManager);
-          ozoneManagerServiceProvider = new OzoneManagerServiceProviderImpl(
-              getTestOzoneConfiguration());
-          bind(OzoneManagerServiceProvider.class)
-              .toInstance(ozoneManagerServiceProvider);
-
-          bind(DBStore.class).toProvider(ReconContainerDBProvider.class).
-              in(Singleton.class);
-          bind(ContainerDBServiceProvider.class).to(
-              ContainerDBServiceProviderImpl.class).in(Singleton.class);
-        } catch (IOException e) {
-          Assert.fail();
-        }
-      }
-    });
+    // The following setup is run only once
+    if (!setUpIsDone) {
+      initializeInjector();
+
+      DSL.using(new DefaultConfiguration().set(
+          injector.getInstance(DataSource.class)));
+
+      containerDbServiceProvider = injector.getInstance(
+          ContainerDBServiceProvider.class);
+
+      StatsSchemaDefinition schemaDefinition = getInjector().getInstance(
+          StatsSchemaDefinition.class);
+      schemaDefinition.initializeSchema();
+
+      setUpIsDone = true;
+    }
+
     containerDbServiceProvider = injector.getInstance(
         ContainerDBServiceProvider.class);
   }
@@ -146,7 +158,7 @@ public class TestContainerKeyMapperTask extends AbstractOMMetadataManagerTest {
 
     keyPrefixesForContainer =
         containerDbServiceProvider.getKeyPrefixesForContainer(1);
-    assertTrue(keyPrefixesForContainer.size() == 1);
+    assertEquals(1, keyPrefixesForContainer.size());
     String omKey = omMetadataManager.getOzoneKey("sampleVol",
         "bucketOne", "key_one");
     ContainerKeyPrefix containerKeyPrefix = new ContainerKeyPrefix(1,
@@ -156,11 +168,19 @@ public class TestContainerKeyMapperTask extends AbstractOMMetadataManagerTest {
 
     keyPrefixesForContainer =
         containerDbServiceProvider.getKeyPrefixesForContainer(2);
-    assertTrue(keyPrefixesForContainer.size() == 1);
+    assertEquals(1, keyPrefixesForContainer.size());
     containerKeyPrefix = new ContainerKeyPrefix(2, omKey,
         0);
     assertEquals(1,
         keyPrefixesForContainer.get(containerKeyPrefix).intValue());
+
+    // Test if container key counts are updated
+    assertEquals(1, containerDbServiceProvider.getKeyCountForContainer(1L));
+    assertEquals(1, containerDbServiceProvider.getKeyCountForContainer(2L));
+    assertEquals(0, containerDbServiceProvider.getKeyCountForContainer(3L));
+
+    // Test if container count is updated
+    assertEquals(2, containerDbServiceProvider.getCountForContainers());
   }
 
   @Test
@@ -245,7 +265,7 @@ public class TestContainerKeyMapperTask extends AbstractOMMetadataManagerTest {
 
     keyPrefixesForContainer = containerDbServiceProvider
         .getKeyPrefixesForContainer(1);
-    assertTrue(keyPrefixesForContainer.size() == 1);
+    assertEquals(1, keyPrefixesForContainer.size());
 
     keyPrefixesForContainer = containerDbServiceProvider
         .getKeyPrefixesForContainer(2);
@@ -253,23 +273,33 @@ public class TestContainerKeyMapperTask extends AbstractOMMetadataManagerTest {
 
     keyPrefixesForContainer = containerDbServiceProvider
         .getKeyPrefixesForContainer(3);
-    assertTrue(keyPrefixesForContainer.size() == 1);
+    assertEquals(1, keyPrefixesForContainer.size());
+
+    assertEquals(1, containerDbServiceProvider.getKeyCountForContainer(1L));
+    assertEquals(0, containerDbServiceProvider.getKeyCountForContainer(2L));
+    assertEquals(1, containerDbServiceProvider.getKeyCountForContainer(3L));
 
     // Process PUT & DELETE event.
     containerKeyMapperTask.process(omUpdateEventBatch);
 
     keyPrefixesForContainer = containerDbServiceProvider
         .getKeyPrefixesForContainer(1);
-    assertTrue(keyPrefixesForContainer.size() == 1);
+    assertEquals(1, keyPrefixesForContainer.size());
 
     keyPrefixesForContainer = containerDbServiceProvider
         .getKeyPrefixesForContainer(2);
-    assertTrue(keyPrefixesForContainer.size() == 1);
+    assertEquals(1, keyPrefixesForContainer.size());
 
     keyPrefixesForContainer = containerDbServiceProvider
         .getKeyPrefixesForContainer(3);
     assertTrue(keyPrefixesForContainer.isEmpty());
 
+    assertEquals(1, containerDbServiceProvider.getKeyCountForContainer(1L));
+    assertEquals(1, containerDbServiceProvider.getKeyCountForContainer(2L));
+    assertEquals(0, containerDbServiceProvider.getKeyCountForContainer(3L));
+
+    // Test if container count is updated
+    assertEquals(3, containerDbServiceProvider.getCountForContainers());
   }
 
   private OmKeyInfo buildOmKeyInfo(String volume,
@@ -287,19 +317,4 @@ public class TestContainerKeyMapperTask extends AbstractOMMetadataManagerTest {
             omKeyLocationInfoGroup))
         .build();
   }
-  /**
-   * Get Test OzoneConfiguration instance.
-   * @return OzoneConfiguration
-   * @throws IOException ioEx.
-   */
-  private OzoneConfiguration getTestOzoneConfiguration()
-      throws IOException {
-    OzoneConfiguration configuration = new OzoneConfiguration();
-    configuration.set(OZONE_RECON_OM_SNAPSHOT_DB_DIR,
-        temporaryFolder.newFolder().getAbsolutePath());
-    configuration.set(OZONE_RECON_DB_DIR, temporaryFolder.newFolder()
-        .getAbsolutePath());
-    return configuration;
-  }
-
 }

+ 117 - 0
hadoop-ozone/ozone-recon/src/test/java/org/apache/hadoop/ozone/recon/types/GuiceInjectorUtilsForTests.java

@@ -0,0 +1,117 @@
+/**
+ * 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.recon.types;
+
+import com.google.inject.AbstractModule;
+import com.google.inject.Guice;
+import com.google.inject.Injector;
+import com.google.inject.Singleton;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.recon.persistence.AbstractSqlDatabaseTest;
+import org.apache.hadoop.ozone.recon.persistence.DataSourceConfiguration;
+import org.apache.hadoop.ozone.recon.persistence.JooqPersistenceModule;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ContainerDBServiceProvider;
+import org.apache.hadoop.ozone.recon.spi.OzoneManagerServiceProvider;
+import org.apache.hadoop.ozone.recon.spi.impl.ContainerDBServiceProviderImpl;
+import org.apache.hadoop.ozone.recon.spi.impl.OzoneManagerServiceProviderImpl;
+import org.apache.hadoop.ozone.recon.spi.impl.ReconContainerDBProvider;
+import org.apache.hadoop.utils.db.DBStore;
+import org.junit.Assert;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+
+import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_DB_DIR;
+import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_OM_SNAPSHOT_DB_DIR;
+
+/**
+ * Utility methods to get guice injector and ozone configuration.
+ */
+public interface GuiceInjectorUtilsForTests {
+
+  /**
+   * Get Guice Injector with bindings.
+   * @param ozoneManagerServiceProvider
+   * @param reconOMMetadataManager
+   * @param temporaryFolder
+   * @return Injector
+   * @throws IOException ioEx.
+   */
+  default Injector getInjector(
+      OzoneManagerServiceProviderImpl ozoneManagerServiceProvider,
+      ReconOMMetadataManager reconOMMetadataManager,
+      TemporaryFolder temporaryFolder
+  ) throws IOException {
+
+    File tempDir = temporaryFolder.newFolder();
+    AbstractSqlDatabaseTest.DataSourceConfigurationProvider
+        configurationProvider =
+        new AbstractSqlDatabaseTest.DataSourceConfigurationProvider(tempDir);
+
+    JooqPersistenceModule jooqPersistenceModule =
+        new JooqPersistenceModule(configurationProvider);
+
+    return Guice.createInjector(jooqPersistenceModule,
+        new AbstractModule() {
+          @Override
+          protected void configure() {
+            try {
+              bind(DataSourceConfiguration.class)
+                  .toProvider(configurationProvider);
+              bind(OzoneConfiguration.class).toInstance(
+                  getTestOzoneConfiguration(temporaryFolder));
+
+              if (reconOMMetadataManager != null) {
+                bind(ReconOMMetadataManager.class)
+                    .toInstance(reconOMMetadataManager);
+              }
+
+              if (ozoneManagerServiceProvider != null) {
+                bind(OzoneManagerServiceProvider.class)
+                    .toInstance(ozoneManagerServiceProvider);
+              }
+
+              bind(DBStore.class).toProvider(ReconContainerDBProvider.class).
+                  in(Singleton.class);
+              bind(ContainerDBServiceProvider.class).to(
+                  ContainerDBServiceProviderImpl.class).in(Singleton.class);
+            } catch (IOException e) {
+              Assert.fail();
+            }
+          }
+        });
+  }
+
+  /**
+   * Get Test OzoneConfiguration instance.
+   * @return OzoneConfiguration
+   * @throws IOException ioEx.
+   */
+  default OzoneConfiguration getTestOzoneConfiguration(
+      TemporaryFolder temporaryFolder) throws IOException {
+    OzoneConfiguration configuration = new OzoneConfiguration();
+    configuration.set(OZONE_RECON_OM_SNAPSHOT_DB_DIR,
+        temporaryFolder.newFolder().getAbsolutePath());
+    configuration.set(OZONE_RECON_DB_DIR, temporaryFolder.newFolder()
+        .getAbsolutePath());
+    return configuration;
+  }
+}