Browse Source

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

Vivek Ratnavel Subramanian 5 years ago
parent
commit
82d88a8d30
22 changed files with 1233 additions and 335 deletions
  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.
    * Gets the path of the Storage dir.
-   * @return Stoarge dir path
+   * @return Storage dir path
    */
    */
   public String getStorageDir() {
   public String getStorageDir() {
     return storageDir.getAbsoluteFile().toString();
     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.
    * version file properties.
    * @return the instance of the storageInfo class
    * @return the instance of the storageInfo class
    */
    */
@@ -128,7 +128,7 @@ public abstract class Storage {
   abstract protected Properties getNodeProperties();
   abstract protected Properties getNodeProperties();
 
 
   /**
   /**
-   * Sets the Node properties spaecific to OM/SCM.
+   * Sets the Node properties specific to OM/SCM.
    */
    */
   private void setNodeProperties() {
   private void setNodeProperties() {
     Properties nodeProperties = getNodeProperties();
     Properties nodeProperties = getNodeProperties();

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

@@ -659,7 +659,7 @@
     <value/>
     <value/>
     <tag>OZONE, OM, SCM, CONTAINER, STORAGE, REQUIRED</tag>
     <tag>OZONE, OM, SCM, CONTAINER, STORAGE, REQUIRED</tag>
     <description>
     <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
       to store their metadata. This setting may be used only in test/PoC
       clusters to simplify configuration.
       clusters to simplify configuration.
 
 
@@ -2457,7 +2457,7 @@
     <value/>
     <value/>
     <tag>OZONE, RECON</tag>
     <tag>OZONE, RECON</tag>
     <description>
     <description>
-      Ozone Recon datbase password.
+      Ozone Recon database password.
     </description>
     </description>
   </property>
   </property>
   <property>
   <property>
@@ -2484,7 +2484,7 @@
     <description>
     <description>
       The max active connections to the SQL database. The default SQLite
       The max active connections to the SQL database. The default SQLite
       database only allows single active connection, set this to a
       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>
     </description>
   </property>
   </property>
   <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 {
   public Void call() throws Exception {
     /**
     /**
      * This method is invoked only when a sub-command is not called. Therefore
      * 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.
      * which runs and starts the OM.
      */
      */
     commonInit();
     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.ReconInternalSchemaDefinition;
 import org.hadoop.ozone.recon.schema.ReconSchemaDefinition;
 import org.hadoop.ozone.recon.schema.ReconSchemaDefinition;
+import org.hadoop.ozone.recon.schema.StatsSchemaDefinition;
 import org.hadoop.ozone.recon.schema.UtilizationSchemaDefinition;
 import org.hadoop.ozone.recon.schema.UtilizationSchemaDefinition;
 
 
 import com.google.inject.AbstractModule;
 import com.google.inject.AbstractModule;
@@ -36,5 +37,6 @@ public class ReconSchemaGenerationModule extends AbstractModule {
         Multibinder.newSetBinder(binder(), ReconSchemaDefinition.class);
         Multibinder.newSetBinder(binder(), ReconSchemaDefinition.class);
     schemaBinder.addBinding().to(UtilizationSchemaDefinition.class);
     schemaBinder.addBinding().to(UtilizationSchemaDefinition.class);
     schemaBinder.addBinding().to(ReconInternalSchemaDefinition.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-" +
   public static final String RECON_CONTAINER_DB = "recon-" +
       CONTAINER_DB_SUFFIX;
       CONTAINER_DB_SUFFIX;
 
 
+  public static final String CONTAINER_COUNT_KEY = "totalCount";
+
   public static final String RECON_OM_SNAPSHOT_DB =
   public static final String RECON_OM_SNAPSHOT_DB =
       "om.snapshot.db";
       "om.snapshot.db";
 
 
   public static final String CONTAINER_KEY_TABLE =
   public static final String CONTAINER_KEY_TABLE =
       "containerKeyTable";
       "containerKeyTable";
 
 
+  public static final String CONTAINER_KEY_COUNT_TABLE =
+      "containerKeyCountTable";
+
   public static final String FETCH_ALL = "-1";
   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";
   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.ContainerDBServiceProvider;
 import org.apache.hadoop.ozone.recon.spi.OzoneManagerServiceProvider;
 import org.apache.hadoop.ozone.recon.spi.OzoneManagerServiceProvider;
 import org.apache.hadoop.ozone.recon.tasks.ContainerKeyMapperTask;
 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.Logger;
 import org.slf4j.LoggerFactory;
 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.
     //Pass on injector to listener that does the Guice - Jersey HK2 bridging.
     ReconGuiceServletContextListener.setInjector(injector);
     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);
     httpServer = injector.getInstance(ReconHttpServer.class);
     LOG.info("Starting Recon server");
     LOG.info("Starting Recon server");
     httpServer.start();
     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.om.helpers.OmKeyLocationInfoGroup;
 import org.apache.hadoop.ozone.recon.api.types.ContainerKeyPrefix;
 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.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;
 import org.apache.hadoop.ozone.recon.api.types.KeyMetadata.ContainerBlockMetadata;
 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.recovery.ReconOMMetadataManager;
 import org.apache.hadoop.ozone.recon.spi.ContainerDBServiceProvider;
 import org.apache.hadoop.ozone.recon.spi.ContainerDBServiceProvider;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
 import static org.apache.hadoop.ozone.recon.ReconConstants.FETCH_ALL;
 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_LIMIT;
 import static org.apache.hadoop.ozone.recon.ReconConstants.RECON_QUERY_PREVKEY;
 import static org.apache.hadoop.ozone.recon.ReconConstants.RECON_QUERY_PREVKEY;
 
 
@@ -84,15 +87,20 @@ public class ContainerKeyService {
   @GET
   @GET
   public Response getContainers(
   public Response getContainers(
       @DefaultValue(FETCH_ALL) @QueryParam(RECON_QUERY_LIMIT) int limit,
       @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;
     Map<Long, ContainerMetadata> containersMap;
+    long containersCount;
     try {
     try {
       containersMap = containerDBServiceProvider.getContainers(limit, prevKey);
       containersMap = containerDBServiceProvider.getContainers(limit, prevKey);
+      containersCount = containerDBServiceProvider.getCountForContainers();
     } catch (IOException ioEx) {
     } catch (IOException ioEx) {
       throw new WebApplicationException(ioEx,
       throw new WebApplicationException(ioEx,
           Response.Status.INTERNAL_SERVER_ERROR);
           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}
    * @return {@link Response}
    */
    */
   @GET
   @GET
-  @Path("/{id}")
+  @Path("/{id}/keys")
   public Response getKeysForContainer(
   public Response getKeysForContainer(
       @PathParam("id") Long containerID,
       @PathParam("id") Long containerID,
       @DefaultValue(FETCH_ALL) @QueryParam(RECON_QUERY_LIMIT) int limit,
       @DefaultValue(FETCH_ALL) @QueryParam(RECON_QUERY_LIMIT) int limit,
       @DefaultValue(StringUtils.EMPTY) @QueryParam(RECON_QUERY_PREVKEY)
       @DefaultValue(StringUtils.EMPTY) @QueryParam(RECON_QUERY_PREVKEY)
           String prevKeyPrefix) {
           String prevKeyPrefix) {
     Map<String, KeyMetadata> keyMetadataMap = new LinkedHashMap<>();
     Map<String, KeyMetadata> keyMetadataMap = new LinkedHashMap<>();
+    long totalCount;
     try {
     try {
       Map<ContainerKeyPrefix, Integer> containerKeyPrefixMap =
       Map<ContainerKeyPrefix, Integer> containerKeyPrefixMap =
           containerDBServiceProvider.getKeyPrefixesForContainer(containerID,
           containerDBServiceProvider.getKeyPrefixesForContainer(containerID,
@@ -187,10 +196,15 @@ public class ContainerKeyService {
             }});
             }});
         }
         }
       }
       }
+
+      totalCount =
+          containerDBServiceProvider.getKeyCountForContainer(containerID);
     } catch (IOException ioEx) {
     } catch (IOException ioEx) {
       throw new WebApplicationException(ioEx,
       throw new WebApplicationException(ioEx,
           Response.Status.INTERNAL_SERVER_ERROR);
           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;
                                 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.
    * @return count of keys with that prefix.
    */
    */
-  Integer getCountForForContainerKeyPrefix(
+  Integer getCountForContainerKeyPrefix(
       ContainerKeyPrefix containerKeyPrefix) throws IOException;
       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.
    * Get the stored key prefixes for the given containerId.
    *
    *
@@ -95,6 +129,7 @@ public interface ContainerDBServiceProvider {
 
 
   /**
   /**
    * Delete an entry in the container DB.
    * Delete an entry in the container DB.
+   *
    * @param containerKeyPrefix container key prefix to be deleted.
    * @param containerKeyPrefix container key prefix to be deleted.
    * @throws IOException exception.
    * @throws IOException exception.
    */
    */
@@ -105,5 +140,20 @@ public interface ContainerDBServiceProvider {
    * Get iterator to the entire container DB.
    * Get iterator to the entire container DB.
    * @return TableIterator
    * @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;
 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.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.File;
 import java.io.IOException;
 import java.io.IOException;
+import java.sql.Timestamp;
 import java.util.LinkedHashMap;
 import java.util.LinkedHashMap;
 import java.util.Map;
 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;
 import org.apache.hadoop.utils.db.Table.KeyValue;
 import org.apache.hadoop.utils.db.Table.KeyValue;
 import org.apache.hadoop.utils.db.TableIterator;
 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.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
@@ -52,6 +61,8 @@ public class ContainerDBServiceProviderImpl
       LoggerFactory.getLogger(ContainerDBServiceProviderImpl.class);
       LoggerFactory.getLogger(ContainerDBServiceProviderImpl.class);
 
 
   private Table<ContainerKeyPrefix, Integer> containerKeyTable;
   private Table<ContainerKeyPrefix, Integer> containerKeyTable;
+  private Table<Long, Long> containerKeyCountTable;
+  private GlobalStatsDao globalStatsDao;
 
 
   @Inject
   @Inject
   private OzoneConfiguration configuration;
   private OzoneConfiguration configuration;
@@ -60,20 +71,28 @@ public class ContainerDBServiceProviderImpl
   private DBStore containerDbStore;
   private DBStore containerDbStore;
 
 
   @Inject
   @Inject
-  public ContainerDBServiceProviderImpl(DBStore dbStore) {
+  private Configuration sqlConfiguration;
+
+  @Inject
+  public ContainerDBServiceProviderImpl(DBStore dbStore,
+                                        Configuration sqlConfiguration) {
+    globalStatsDao = new GlobalStatsDao(sqlConfiguration);
     try {
     try {
       this.containerKeyTable = dbStore.getTable(CONTAINER_KEY_TABLE,
       this.containerKeyTable = dbStore.getTable(CONTAINER_KEY_TABLE,
           ContainerKeyPrefix.class, Integer.class);
           ContainerKeyPrefix.class, Integer.class);
+      this.containerKeyCountTable = dbStore.getTable(CONTAINER_KEY_COUNT_TABLE,
+          Long.class, Long.class);
     } catch (IOException e) {
     } 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
    * Initialize a new container DB instance, getting rid of the old instance
    * and then storing the passed in container prefix counts into the created
    * 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
    * @throws IOException
    */
    */
   @Override
   @Override
@@ -91,17 +110,23 @@ public class ContainerDBServiceProviderImpl
           oldDBLocation.getAbsolutePath());
           oldDBLocation.getAbsolutePath());
       FileUtils.deleteDirectory(oldDBLocation);
       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.
    * 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.
    * @param count Count of the keys matching that prefix.
    * @throws IOException
    * @throws IOException
    */
    */
@@ -112,16 +137,54 @@ public class ContainerDBServiceProviderImpl
     containerKeyTable.put(containerKeyPrefix, count);
     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
    * Put together the key from the passed in object and get the count from
    * the container DB store.
    * 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
    * @throws IOException
    */
    */
   @Override
   @Override
-  public Integer getCountForForContainerKeyPrefix(
+  public Integer getCountForContainerKeyPrefix(
       ContainerKeyPrefix containerKeyPrefix) throws IOException {
       ContainerKeyPrefix containerKeyPrefix) throws IOException {
     Integer count =  containerKeyTable.get(containerKeyPrefix);
     Integer count =  containerKeyTable.get(containerKeyPrefix);
     return count == null ? Integer.valueOf(0) : count;
     return count == null ? Integer.valueOf(0) : count;
@@ -130,7 +193,7 @@ public class ContainerDBServiceProviderImpl
   /**
   /**
    * Get key prefixes for the given container ID.
    * 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).
    * @return Map of (Key-Prefix,Count of Keys).
    */
    */
   @Override
   @Override
@@ -271,8 +334,56 @@ public class ContainerDBServiceProviderImpl
     containerKeyTable.delete(containerKeyPrefix);
     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
   @Override
   public TableIterator getContainerTableIterator() {
   public TableIterator getContainerTableIterator() {
     return containerKeyTable.iterator();
     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;
 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.RECON_CONTAINER_DB;
 import static org.apache.hadoop.ozone.recon.ReconConstants.CONTAINER_KEY_TABLE;
 import static org.apache.hadoop.ozone.recon.ReconConstants.CONTAINER_KEY_TABLE;
 import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_DB_DIR;
 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.DBStore;
 import org.apache.hadoop.utils.db.DBStoreBuilder;
 import org.apache.hadoop.utils.db.DBStoreBuilder;
 import org.apache.hadoop.utils.db.IntegerCodec;
 import org.apache.hadoop.utils.db.IntegerCodec;
+import org.apache.hadoop.utils.db.LongCodec;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
@@ -69,7 +71,9 @@ public class ReconContainerDBProvider implements Provider<DBStore> {
           .setPath(metaDir)
           .setPath(metaDir)
           .setName(dbName)
           .setName(dbName)
           .addTable(CONTAINER_KEY_TABLE)
           .addTable(CONTAINER_KEY_TABLE)
+          .addTable(CONTAINER_KEY_COUNT_TABLE)
           .addCodec(ContainerKeyPrefix.class, new ContainerKeyPrefixCodec())
           .addCodec(ContainerKeyPrefix.class, new ContainerKeyPrefixCodec())
+          .addCodec(Long.class, new LongCodec())
           .addCodec(Integer.class, new IntegerCodec())
           .addCodec(Integer.class, new IntegerCodec())
           .build();
           .build();
     } catch (Exception ex) {
     } 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.time.Instant;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collection;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Iterator;
 import java.util.Set;
 import java.util.Set;
@@ -70,11 +71,14 @@ public class ContainerKeyMapperTask extends ReconDBUpdateTask {
    */
    */
   @Override
   @Override
   public Pair<String, Boolean> reprocess(OMMetadataManager omMetadataManager) {
   public Pair<String, Boolean> reprocess(OMMetadataManager omMetadataManager) {
-    int omKeyCount = 0;
+    long omKeyCount = 0;
     try {
     try {
       LOG.info("Starting a 'reprocess' run of ContainerKeyMapperTask.");
       LOG.info("Starting a 'reprocess' run of ContainerKeyMapperTask.");
       Instant start = Instant.now();
       Instant start = Instant.now();
 
 
+      // initialize new container DB
+      containerDBServiceProvider.initNewContainerDB(new HashMap<>());
+
       Table<String, OmKeyInfo> omKeyInfoTable = omMetadataManager.getKeyTable();
       Table<String, OmKeyInfo> omKeyInfoTable = omMetadataManager.getKeyTable();
       try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
       try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
                keyIter = omKeyInfoTable.iterator()) {
                keyIter = omKeyInfoTable.iterator()) {
@@ -98,7 +102,6 @@ public class ContainerKeyMapperTask extends ReconDBUpdateTask {
     return new ImmutablePair<>(getTaskName(), true);
     return new ImmutablePair<>(getTaskName(), true);
   }
   }
 
 
-
   @Override
   @Override
   protected Collection<String> getTaskTables() {
   protected Collection<String> getTaskTables() {
     return tables;
     return tables;
@@ -125,7 +128,8 @@ public class ContainerKeyMapperTask extends ReconDBUpdateTask {
             .getAction());
             .getAction());
         }
         }
       } catch (IOException e) {
       } 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);
         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.
    * @param key key String.
    * @throws IOException If Unable to write to container DB.
    * @throws IOException If Unable to write to container DB.
    */
    */
@@ -144,30 +150,42 @@ public class ContainerKeyMapperTask extends ReconDBUpdateTask {
         Table.KeyValue<ContainerKeyPrefix, Integer>> containerIterator =
         Table.KeyValue<ContainerKeyPrefix, Integer>> containerIterator =
         containerDBServiceProvider.getContainerTableIterator();
         containerDBServiceProvider.getContainerTableIterator();
 
 
-    Set<ContainerKeyPrefix> keysToDeDeleted = new HashSet<>();
+    Set<ContainerKeyPrefix> keysToBeDeleted = new HashSet<>();
 
 
     while (containerIterator.hasNext()) {
     while (containerIterator.hasNext()) {
       Table.KeyValue<ContainerKeyPrefix, Integer> keyValue =
       Table.KeyValue<ContainerKeyPrefix, Integer> keyValue =
           containerIterator.next();
           containerIterator.next();
       String keyPrefix = keyValue.getKey().getKeyPrefix();
       String keyPrefix = keyValue.getKey().getKeyPrefix();
       if (keyPrefix.equals(key)) {
       if (keyPrefix.equals(key)) {
-        keysToDeDeleted.add(keyValue.getKey());
+        keysToBeDeleted.add(keyValue.getKey());
       }
       }
     }
     }
 
 
-    for (ContainerKeyPrefix containerKeyPrefix : keysToDeDeleted) {
+    for (ContainerKeyPrefix containerKeyPrefix : keysToBeDeleted) {
       containerDBServiceProvider.deleteContainerMapping(containerKeyPrefix);
       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 key key String
    * @param omKeyInfo omKeyInfo value
    * @param omKeyInfo omKeyInfo value
    * @throws IOException if unable to write to recon DB.
    * @throws IOException if unable to write to recon DB.
    */
    */
   private void  writeOMKeyToContainerDB(String key, OmKeyInfo omKeyInfo)
   private void  writeOMKeyToContainerDB(String key, OmKeyInfo omKeyInfo)
       throws IOException {
       throws IOException {
+    long containerCountToIncrement = 0;
     for (OmKeyLocationInfoGroup omKeyLocationInfoGroup : omKeyInfo
     for (OmKeyLocationInfoGroup omKeyLocationInfoGroup : omKeyInfo
         .getKeyLocationVersions()) {
         .getKeyLocationVersions()) {
       long keyVersion = omKeyLocationInfoGroup.getVersion();
       long keyVersion = omKeyLocationInfoGroup.getVersion();
@@ -176,15 +194,36 @@ public class ContainerKeyMapperTask extends ReconDBUpdateTask {
         long containerId = omKeyLocationInfo.getContainerID();
         long containerId = omKeyLocationInfo.getContainerID();
         ContainerKeyPrefix containerKeyPrefix = new ContainerKeyPrefix(
         ContainerKeyPrefix containerKeyPrefix = new ContainerKeyPrefix(
             containerId, key, keyVersion);
             containerId, key, keyVersion);
-        if (containerDBServiceProvider.getCountForForContainerKeyPrefix(
+        if (containerDBServiceProvider.getCountForContainerKeyPrefix(
             containerKeyPrefix) == 0) {
             containerKeyPrefix) == 0) {
           // Save on writes. No need to save same container-key prefix
           // Save on writes. No need to save same container-key prefix
           // mapping again.
           // mapping again.
           containerDBServiceProvider.storeContainerKeyMapping(
           containerDBServiceProvider.storeContainerKeyMapping(
               containerKeyPrefix, 1);
               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;
 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.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertTrue;
 
 
 import java.io.File;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileInputStream;
-import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStream;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collection;
@@ -34,6 +31,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
 
 
+import javax.sql.DataSource;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.core.Response;
 
 
 import org.apache.commons.lang3.StringUtils;
 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.OmKeyLocationInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
 import org.apache.hadoop.ozone.recon.AbstractOMMetadataManagerTest;
 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.ReconUtils;
 import org.apache.hadoop.ozone.recon.api.types.ContainerMetadata;
 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;
+import org.apache.hadoop.ozone.recon.api.types.KeysResponse;
 import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
 import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
 import org.apache.hadoop.ozone.recon.spi.ContainerDBServiceProvider;
 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.OzoneManagerServiceProviderImpl;
-import org.apache.hadoop.ozone.recon.spi.impl.ReconContainerDBProvider;
 import org.apache.hadoop.ozone.recon.tasks.ContainerKeyMapperTask;
 import org.apache.hadoop.ozone.recon.tasks.ContainerKeyMapperTask;
 import org.apache.hadoop.utils.db.DBCheckpoint;
 import org.apache.hadoop.utils.db.DBCheckpoint;
-import org.apache.hadoop.utils.db.DBStore;
 import org.apache.http.impl.client.CloseableHttpClient;
 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.Before;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runner.RunWith;
 import org.powermock.api.mockito.PowerMockito;
 import org.powermock.api.mockito.PowerMockito;
@@ -68,9 +68,9 @@ import org.powermock.core.classloader.annotations.PrepareForTest;
 import org.powermock.modules.junit4.PowerMockRunner;
 import org.powermock.modules.junit4.PowerMockRunner;
 
 
 import com.google.inject.AbstractModule;
 import com.google.inject.AbstractModule;
-import com.google.inject.Guice;
 import com.google.inject.Injector;
 import com.google.inject.Injector;
-import com.google.inject.Singleton;
+
+import org.junit.rules.TemporaryFolder;
 
 
 /**
 /**
  * Test for container key service.
  * Test for container key service.
@@ -80,41 +80,57 @@ import com.google.inject.Singleton;
 @PrepareForTest(ReconUtils.class)
 @PrepareForTest(ReconUtils.class)
 public class TestContainerKeyService extends AbstractOMMetadataManagerTest {
 public class TestContainerKeyService extends AbstractOMMetadataManagerTest {
 
 
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
   private ContainerDBServiceProvider containerDbServiceProvider;
   private ContainerDBServiceProvider containerDbServiceProvider;
   private OMMetadataManager omMetadataManager;
   private OMMetadataManager omMetadataManager;
-  private ReconOMMetadataManager reconOMMetadataManager;
   private Injector injector;
   private Injector injector;
   private OzoneManagerServiceProviderImpl ozoneManagerServiceProvider;
   private OzoneManagerServiceProviderImpl ozoneManagerServiceProvider;
   private ContainerKeyService containerKeyService;
   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();
     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
       @Override
       protected void configure() {
       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
     //Write Data to OM
     Pipeline pipeline = getRandomPipeline();
     Pipeline pipeline = getRandomPipeline();
@@ -204,8 +220,11 @@ public class TestContainerKeyService extends AbstractOMMetadataManagerTest {
 
 
     Response response = containerKeyService.getKeysForContainer(1L, -1, "");
     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());
     assertEquals(2, keyMetadataList.size());
 
 
     Iterator<KeyMetadata> iterator = keyMetadataList.iterator();
     Iterator<KeyMetadata> iterator = keyMetadataList.iterator();
@@ -229,13 +248,19 @@ public class TestContainerKeyService extends AbstractOMMetadataManagerTest {
     assertEquals(104, blockIds.get(1L).iterator().next().getLocalID());
     assertEquals(104, blockIds.get(1L).iterator().next().getLocalID());
 
 
     response = containerKeyService.getKeysForContainer(3L, -1, "");
     response = containerKeyService.getKeysForContainer(3L, -1, "");
-    keyMetadataList = (Collection<KeyMetadata>) response.getEntity();
+    responseObject = (KeysResponse) response.getEntity();
+    data = responseObject.getKeysResponseData();
+    keyMetadataList = data.getKeys();
     assertTrue(keyMetadataList.isEmpty());
     assertTrue(keyMetadataList.isEmpty());
+    assertEquals(0, data.getTotalCount());
 
 
     // test if limit works as expected
     // test if limit works as expected
     response = containerKeyService.getKeysForContainer(1L, 1, "");
     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(1, keyMetadataList.size());
+    assertEquals(3, data.getTotalCount());
   }
   }
 
 
   @Test
   @Test
@@ -244,8 +269,14 @@ public class TestContainerKeyService extends AbstractOMMetadataManagerTest {
     Response response = containerKeyService.getKeysForContainer(
     Response response = containerKeyService.getKeysForContainer(
         1L, -1, "/sampleVol/bucketOne/key_one");
         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());
     assertEquals(1, keyMetadataList.size());
 
 
     Iterator<KeyMetadata> iterator = keyMetadataList.iterator();
     Iterator<KeyMetadata> iterator = keyMetadataList.iterator();
@@ -257,7 +288,11 @@ public class TestContainerKeyService extends AbstractOMMetadataManagerTest {
 
 
     response = containerKeyService.getKeysForContainer(
     response = containerKeyService.getKeysForContainer(
         1L, -1, StringUtils.EMPTY);
         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());
     assertEquals(2, keyMetadataList.size());
     iterator = keyMetadataList.iterator();
     iterator = keyMetadataList.iterator();
     keyMetadata = iterator.next();
     keyMetadata = iterator.next();
@@ -266,13 +301,19 @@ public class TestContainerKeyService extends AbstractOMMetadataManagerTest {
     // test for negative cases
     // test for negative cases
     response = containerKeyService.getKeysForContainer(
     response = containerKeyService.getKeysForContainer(
         1L, -1, "/sampleVol/bucketOne/invalid_key");
         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());
     assertEquals(0, keyMetadataList.size());
 
 
     response = containerKeyService.getKeysForContainer(
     response = containerKeyService.getKeysForContainer(
         5L, -1, "");
         5L, -1, "");
-    keyMetadataList = (Collection<KeyMetadata>) response.getEntity();
+    responseObject = (KeysResponse) response.getEntity();
+    data = responseObject.getKeysResponseData();
+    keyMetadataList = data.getKeys();
     assertEquals(0, keyMetadataList.size());
     assertEquals(0, keyMetadataList.size());
+    assertEquals(0, data.getTotalCount());
   }
   }
 
 
   @Test
   @Test
@@ -280,8 +321,14 @@ public class TestContainerKeyService extends AbstractOMMetadataManagerTest {
 
 
     Response response = containerKeyService.getContainers(-1, 0L);
     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();
     Iterator<ContainerMetadata> iterator = containers.iterator();
 
 
@@ -297,9 +344,11 @@ public class TestContainerKeyService extends AbstractOMMetadataManagerTest {
 
 
     // test if limit works as expected
     // test if limit works as expected
     response = containerKeyService.getContainers(1, 0L);
     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(1, containers.size());
+    assertEquals(2, data.getTotalCount());
   }
   }
 
 
   @Test
   @Test
@@ -307,8 +356,14 @@ public class TestContainerKeyService extends AbstractOMMetadataManagerTest {
 
 
     Response response = containerKeyService.getContainers(1, 1L);
     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();
     Iterator<ContainerMetadata> iterator = containers.iterator();
 
 
@@ -318,37 +373,28 @@ public class TestContainerKeyService extends AbstractOMMetadataManagerTest {
     assertEquals(2L, containerMetadata.getContainerID());
     assertEquals(2L, containerMetadata.getContainerID());
 
 
     response = containerKeyService.getContainers(-1, 0L);
     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, containers.size());
+    assertEquals(2, data.getTotalCount());
     iterator = containers.iterator();
     iterator = containers.iterator();
     containerMetadata = iterator.next();
     containerMetadata = iterator.next();
     assertEquals(1L, containerMetadata.getContainerID());
     assertEquals(1L, containerMetadata.getContainerID());
 
 
     // test for negative cases
     // test for negative cases
     response = containerKeyService.getContainers(-1, 5L);
     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(0, containers.size());
+    assertEquals(2, data.getTotalCount());
 
 
     response = containerKeyService.getContainers(-1, -1L);
     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());
     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
   @ClassRule
   public static TemporaryFolder temporaryFolder = new TemporaryFolder();
   public static TemporaryFolder temporaryFolder = new TemporaryFolder();
-  private static File tempDir;
 
 
   private static Injector injector;
   private static Injector injector;
   private static DSLContext dslContext;
   private static DSLContext dslContext;
 
 
   @BeforeClass
   @BeforeClass
   public static void setup() throws IOException {
   public static void setup() throws IOException {
-    tempDir = temporaryFolder.newFolder();
+    File tempDir = temporaryFolder.newFolder();
 
 
     DataSourceConfigurationProvider configurationProvider =
     DataSourceConfigurationProvider configurationProvider =
-        new DataSourceConfigurationProvider();
+        new DataSourceConfigurationProvider(tempDir);
+
     JooqPersistenceModule persistenceModule =
     JooqPersistenceModule persistenceModule =
         new JooqPersistenceModule(configurationProvider);
         new JooqPersistenceModule(configurationProvider);
 
 
@@ -86,6 +86,12 @@ public abstract class AbstractSqlDatabaseTest {
   public static class DataSourceConfigurationProvider implements
   public static class DataSourceConfigurationProvider implements
       Provider<DataSourceConfiguration> {
       Provider<DataSourceConfiguration> {
 
 
+    private final File tempDir;
+
+    public DataSourceConfigurationProvider(File tempDir) {
+      this.tempDir = tempDir;
+    }
+
     @Override
     @Override
     public DataSourceConfiguration get() {
     public DataSourceConfiguration get() {
       return new DataSourceConfiguration() {
       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;
 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.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertTrue;
 
 
-import java.io.File;
-import java.io.IOException;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.Map;
 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.ContainerKeyPrefix;
 import org.apache.hadoop.ozone.recon.api.types.ContainerMetadata;
 import org.apache.hadoop.ozone.recon.api.types.ContainerMetadata;
 import org.apache.hadoop.ozone.recon.spi.ContainerDBServiceProvider;
 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.Assert;
 import org.junit.Before;
 import org.junit.Before;
-import org.junit.Rule;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
 import org.junit.rules.TemporaryFolder;
 
 
-import com.google.inject.AbstractModule;
-import com.google.inject.Guice;
 import com.google.inject.Injector;
 import com.google.inject.Injector;
-import com.google.inject.Singleton;
+
+import javax.sql.DataSource;
 
 
 /**
 /**
  * Unit Tests for ContainerDBServiceProviderImpl.
  * Unit Tests for ContainerDBServiceProviderImpl.
  */
  */
 public class TestContainerDBServiceProviderImpl {
 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 = injector.getInstance(
         ContainerDBServiceProvider.class);
         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
   @Test
@@ -104,7 +130,7 @@ public class TestContainerDBServiceProviderImpl {
     }
     }
 
 
     assertEquals(1, containerDbServiceProvider
     assertEquals(1, containerDbServiceProvider
-        .getCountForForContainerKeyPrefix(ckp1).intValue());
+        .getCountForContainerKeyPrefix(ckp1).intValue());
 
 
     prefixCounts.clear();
     prefixCounts.clear();
     prefixCounts.put(ckp2, 12);
     prefixCounts.put(ckp2, 12);
@@ -127,7 +153,7 @@ public class TestContainerDBServiceProviderImpl {
     assertEquals(15, keyPrefixesForContainer.get(ckp5).intValue());
     assertEquals(15, keyPrefixesForContainer.get(ckp5).intValue());
 
 
     assertEquals(0, containerDbServiceProvider
     assertEquals(0, containerDbServiceProvider
-        .getCountForForContainerKeyPrefix(ckp1).intValue());
+        .getCountForContainerKeyPrefix(ckp1).intValue());
   }
   }
 
 
   @Test
   @Test
@@ -135,9 +161,9 @@ public class TestContainerDBServiceProviderImpl {
 
 
     long containerId = System.currentTimeMillis();
     long containerId = System.currentTimeMillis();
     Map<String, Integer> prefixCounts = new HashMap<>();
     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()) {
     for (String prefix : prefixCounts.keySet()) {
       ContainerKeyPrefix containerKeyPrefix = new ContainerKeyPrefix(
       ContainerKeyPrefix containerKeyPrefix = new ContainerKeyPrefix(
@@ -146,89 +172,119 @@ public class TestContainerDBServiceProviderImpl {
           containerKeyPrefix, prefixCounts.get(prefix));
           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
   @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();
     long containerId = System.currentTimeMillis();
 
 
     containerDbServiceProvider.storeContainerKeyMapping(new
     containerDbServiceProvider.storeContainerKeyMapping(new
-        ContainerKeyPrefix(containerId, "V2/B1/K1"), 2);
+        ContainerKeyPrefix(containerId, keyPrefix1), 2);
 
 
     Integer count = containerDbServiceProvider.
     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
   @Test
   public void testGetKeyPrefixesForContainer() throws Exception {
   public void testGetKeyPrefixesForContainer() throws Exception {
-    long containerId = System.currentTimeMillis();
+    long containerId = 1L;
+    long nextContainerId = 2L;
+    populateKeysInContainers(containerId, nextContainerId);
 
 
     ContainerKeyPrefix containerKeyPrefix1 = new
     ContainerKeyPrefix containerKeyPrefix1 = new
-        ContainerKeyPrefix(containerId, "V3/B1/K1", 0);
-    containerDbServiceProvider.storeContainerKeyMapping(containerKeyPrefix1,
-        1);
-
+        ContainerKeyPrefix(containerId, keyPrefix1, 0);
     ContainerKeyPrefix containerKeyPrefix2 = new ContainerKeyPrefix(
     ContainerKeyPrefix containerKeyPrefix2 = new ContainerKeyPrefix(
-        containerId, "V3/B1/K2", 0);
-    containerDbServiceProvider.storeContainerKeyMapping(containerKeyPrefix2,
-        2);
-
-    long nextContainerId = containerId + 1000L;
+        containerId, keyPrefix2, 0);
     ContainerKeyPrefix containerKeyPrefix3 = new ContainerKeyPrefix(
     ContainerKeyPrefix containerKeyPrefix3 = new ContainerKeyPrefix(
-        nextContainerId, "V3/B2/K1", 0);
-    containerDbServiceProvider.storeContainerKeyMapping(containerKeyPrefix3,
-        3);
+        nextContainerId, keyPrefix3, 0);
+
 
 
     Map<ContainerKeyPrefix, Integer> keyPrefixMap =
     Map<ContainerKeyPrefix, Integer> keyPrefixMap =
         containerDbServiceProvider.getKeyPrefixesForContainer(containerId);
         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(
     keyPrefixMap = containerDbServiceProvider.getKeyPrefixesForContainer(
         nextContainerId);
         nextContainerId);
-    assertTrue(keyPrefixMap.size() == 1);
-    assertTrue(keyPrefixMap.get(containerKeyPrefix3) == 3);
+    assertEquals(1, keyPrefixMap.size());
+    assertEquals(3, keyPrefixMap.get(containerKeyPrefix3).longValue());
   }
   }
 
 
   @Test
   @Test
   public void testGetKeyPrefixesForContainerWithKeyPrefix() throws Exception {
   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(
     ContainerKeyPrefix containerKeyPrefix2 = new ContainerKeyPrefix(
         containerId, keyPrefix2, 0);
         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 =
     Map<ContainerKeyPrefix, Integer> keyPrefixMap =
         containerDbServiceProvider.getKeyPrefixesForContainer(containerId,
         containerDbServiceProvider.getKeyPrefixesForContainer(containerId,
@@ -250,33 +306,15 @@ public class TestContainerDBServiceProviderImpl {
     assertEquals(0, keyPrefixMap.size());
     assertEquals(0, keyPrefixMap.size());
 
 
     keyPrefixMap = containerDbServiceProvider.getKeyPrefixesForContainer(
     keyPrefixMap = containerDbServiceProvider.getKeyPrefixesForContainer(
-        1L, "");
+        10L, "");
     assertEquals(0, keyPrefixMap.size());
     assertEquals(0, keyPrefixMap.size());
   }
   }
 
 
   @Test
   @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 =
     Map<Long, ContainerMetadata> containerMap =
         containerDbServiceProvider.getContainers(-1, 0L);
         containerDbServiceProvider.getContainers(-1, 0L);
@@ -304,7 +342,7 @@ public class TestContainerDBServiceProviderImpl {
 
 
     // test for negative cases
     // test for negative cases
     containerMap = containerDbServiceProvider.getContainers(
     containerMap = containerDbServiceProvider.getContainers(
-        -1, 1L);
+        -1, 10L);
     assertEquals(0, containerMap.size());
     assertEquals(0, containerMap.size());
 
 
     containerMap = containerDbServiceProvider.getContainers(
     containerMap = containerDbServiceProvider.getContainers(
@@ -313,27 +351,55 @@ public class TestContainerDBServiceProviderImpl {
   }
   }
 
 
   @Test
   @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 =
     Map<ContainerKeyPrefix, Integer> keyPrefixMap =
         containerDbServiceProvider.getKeyPrefixesForContainer(containerId);
         containerDbServiceProvider.getKeyPrefixesForContainer(containerId);
-    assertTrue(keyPrefixMap.size() == 2);
+    assertEquals(2, keyPrefixMap.size());
 
 
     containerDbServiceProvider.deleteContainerMapping(new ContainerKeyPrefix(
     containerDbServiceProvider.deleteContainerMapping(new ContainerKeyPrefix(
-        containerId, "V3/B1/K2", 0));
+        containerId, keyPrefix2, 0));
     keyPrefixMap =
     keyPrefixMap =
         containerDbServiceProvider.getKeyPrefixesForContainer(containerId);
         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;
 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.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertTrue;
 
 
@@ -26,17 +25,15 @@ import java.io.BufferedWriter;
 import java.io.File;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileInputStream;
 import java.io.FileWriter;
 import java.io.FileWriter;
-import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStream;
 import java.nio.file.Paths;
 import java.nio.file.Paths;
 
 
-import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.OmUtils;
 import org.apache.hadoop.ozone.OmUtils;
 import org.apache.hadoop.ozone.om.OMMetadataManager;
 import org.apache.hadoop.ozone.om.OMMetadataManager;
 import org.apache.hadoop.ozone.recon.AbstractOMMetadataManagerTest;
 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.ReconUtils;
 import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
 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.hadoop.utils.db.DBCheckpoint;
 import org.apache.http.impl.client.CloseableHttpClient;
 import org.apache.http.impl.client.CloseableHttpClient;
 import org.junit.Assert;
 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.core.classloader.annotations.PrepareForTest;
 import org.powermock.modules.junit4.PowerMockRunner;
 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.Injector;
 
 
 /**
 /**
@@ -66,7 +61,10 @@ public class TestOzoneManagerServiceProviderImpl extends
   private OMMetadataManager omMetadataManager;
   private OMMetadataManager omMetadataManager;
   private ReconOMMetadataManager reconOMMetadataManager;
   private ReconOMMetadataManager reconOMMetadataManager;
   private Injector injector;
   private Injector injector;
+  private GuiceInjectorUtilsForTestsImpl guiceInjectorTest =
+      new GuiceInjectorUtilsForTestsImpl();
   private OzoneManagerServiceProviderImpl ozoneManagerServiceProvider;
   private OzoneManagerServiceProviderImpl ozoneManagerServiceProvider;
+  private boolean isSetupDone = false;
 
 
   @Rule
   @Rule
   public TemporaryFolder temporaryFolder = new TemporaryFolder();
   public TemporaryFolder temporaryFolder = new TemporaryFolder();
@@ -74,25 +72,17 @@ public class TestOzoneManagerServiceProviderImpl extends
   @Before
   @Before
   public void setUp() throws Exception {
   public void setUp() throws Exception {
     omMetadataManager = initializeNewOmMetadataManager();
     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
   @Test
@@ -168,17 +158,4 @@ public class TestOzoneManagerServiceProviderImpl extends
     assertTrue(checkpoint.getCheckpointLocation().toFile()
     assertTrue(checkpoint.getCheckpointLocation().toFile()
         .listFiles().length == 2);
         .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;
 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.assertEquals;
 import static org.junit.Assert.assertTrue;
 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.OmKeyLocationInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
 import org.apache.hadoop.ozone.recon.AbstractOMMetadataManagerTest;
 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.ReconUtils;
 import org.apache.hadoop.ozone.recon.api.types.ContainerKeyPrefix;
 import org.apache.hadoop.ozone.recon.api.types.ContainerKeyPrefix;
 import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
 import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
 import org.apache.hadoop.ozone.recon.spi.ContainerDBServiceProvider;
 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.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.Before;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
 import org.junit.runner.RunWith;
 import org.junit.runner.RunWith;
 import org.powermock.core.classloader.annotations.PowerMockIgnore;
 import org.powermock.core.classloader.annotations.PowerMockIgnore;
 import org.powermock.core.classloader.annotations.PrepareForTest;
 import org.powermock.core.classloader.annotations.PrepareForTest;
 import org.powermock.modules.junit4.PowerMockRunner;
 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.Injector;
-import com.google.inject.Singleton;
+import javax.sql.DataSource;
 
 
 /**
 /**
  * Unit test for Container Key mapper task.
  * Unit test for Container Key mapper task.
@@ -73,33 +69,49 @@ public class TestContainerKeyMapperTask extends AbstractOMMetadataManagerTest {
   private ReconOMMetadataManager reconOMMetadataManager;
   private ReconOMMetadataManager reconOMMetadataManager;
   private Injector injector;
   private Injector injector;
   private OzoneManagerServiceProviderImpl ozoneManagerServiceProvider;
   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
   @Before
   public void setUp() throws Exception {
   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 = injector.getInstance(
         ContainerDBServiceProvider.class);
         ContainerDBServiceProvider.class);
   }
   }
@@ -146,7 +158,7 @@ public class TestContainerKeyMapperTask extends AbstractOMMetadataManagerTest {
 
 
     keyPrefixesForContainer =
     keyPrefixesForContainer =
         containerDbServiceProvider.getKeyPrefixesForContainer(1);
         containerDbServiceProvider.getKeyPrefixesForContainer(1);
-    assertTrue(keyPrefixesForContainer.size() == 1);
+    assertEquals(1, keyPrefixesForContainer.size());
     String omKey = omMetadataManager.getOzoneKey("sampleVol",
     String omKey = omMetadataManager.getOzoneKey("sampleVol",
         "bucketOne", "key_one");
         "bucketOne", "key_one");
     ContainerKeyPrefix containerKeyPrefix = new ContainerKeyPrefix(1,
     ContainerKeyPrefix containerKeyPrefix = new ContainerKeyPrefix(1,
@@ -156,11 +168,19 @@ public class TestContainerKeyMapperTask extends AbstractOMMetadataManagerTest {
 
 
     keyPrefixesForContainer =
     keyPrefixesForContainer =
         containerDbServiceProvider.getKeyPrefixesForContainer(2);
         containerDbServiceProvider.getKeyPrefixesForContainer(2);
-    assertTrue(keyPrefixesForContainer.size() == 1);
+    assertEquals(1, keyPrefixesForContainer.size());
     containerKeyPrefix = new ContainerKeyPrefix(2, omKey,
     containerKeyPrefix = new ContainerKeyPrefix(2, omKey,
         0);
         0);
     assertEquals(1,
     assertEquals(1,
         keyPrefixesForContainer.get(containerKeyPrefix).intValue());
         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
   @Test
@@ -245,7 +265,7 @@ public class TestContainerKeyMapperTask extends AbstractOMMetadataManagerTest {
 
 
     keyPrefixesForContainer = containerDbServiceProvider
     keyPrefixesForContainer = containerDbServiceProvider
         .getKeyPrefixesForContainer(1);
         .getKeyPrefixesForContainer(1);
-    assertTrue(keyPrefixesForContainer.size() == 1);
+    assertEquals(1, keyPrefixesForContainer.size());
 
 
     keyPrefixesForContainer = containerDbServiceProvider
     keyPrefixesForContainer = containerDbServiceProvider
         .getKeyPrefixesForContainer(2);
         .getKeyPrefixesForContainer(2);
@@ -253,23 +273,33 @@ public class TestContainerKeyMapperTask extends AbstractOMMetadataManagerTest {
 
 
     keyPrefixesForContainer = containerDbServiceProvider
     keyPrefixesForContainer = containerDbServiceProvider
         .getKeyPrefixesForContainer(3);
         .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.
     // Process PUT & DELETE event.
     containerKeyMapperTask.process(omUpdateEventBatch);
     containerKeyMapperTask.process(omUpdateEventBatch);
 
 
     keyPrefixesForContainer = containerDbServiceProvider
     keyPrefixesForContainer = containerDbServiceProvider
         .getKeyPrefixesForContainer(1);
         .getKeyPrefixesForContainer(1);
-    assertTrue(keyPrefixesForContainer.size() == 1);
+    assertEquals(1, keyPrefixesForContainer.size());
 
 
     keyPrefixesForContainer = containerDbServiceProvider
     keyPrefixesForContainer = containerDbServiceProvider
         .getKeyPrefixesForContainer(2);
         .getKeyPrefixesForContainer(2);
-    assertTrue(keyPrefixesForContainer.size() == 1);
+    assertEquals(1, keyPrefixesForContainer.size());
 
 
     keyPrefixesForContainer = containerDbServiceProvider
     keyPrefixesForContainer = containerDbServiceProvider
         .getKeyPrefixesForContainer(3);
         .getKeyPrefixesForContainer(3);
     assertTrue(keyPrefixesForContainer.isEmpty());
     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,
   private OmKeyInfo buildOmKeyInfo(String volume,
@@ -287,19 +317,4 @@ public class TestContainerKeyMapperTask extends AbstractOMMetadataManagerTest {
             omKeyLocationInfoGroup))
             omKeyLocationInfoGroup))
         .build();
         .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;
+  }
+}