You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by yq...@apache.org on 2019/04/04 09:34:20 UTC

[hadoop] branch trunk updated: HDDS-1189. Recon Aggregate DB schema and ORM. Contributed by Siddharth Wagle.

This is an automated email from the ASF dual-hosted git repository.

yqlin pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/trunk by this push:
     new a92806d  HDDS-1189. Recon Aggregate DB schema and ORM. Contributed by Siddharth Wagle.
a92806d is described below

commit a92806d05a2eb1f586463fa07aa2f17ce9180401
Author: Yiqun Lin <yq...@apache.org>
AuthorDate: Thu Apr 4 17:33:37 2019 +0800

    HDDS-1189. Recon Aggregate DB schema and ORM. Contributed by Siddharth Wagle.
---
 .../common/src/main/resources/ozone-default.xml    |  95 ++++++++++++
 hadoop-hdds/tools/pom.xml                          |   3 +-
 hadoop-ozone/ozone-recon-codegen/pom.xml           |  58 +++++++
 .../ozone/recon/codegen/JooqCodeGenerator.java     | 170 +++++++++++++++++++++
 .../recon/codegen/ReconSchemaGenerationModule.java |  39 +++++
 .../ozone/recon/codegen/TableNamingStrategy.java   |  48 ++++++
 .../hadoop/ozone/recon/codegen/package-info.java   |  22 +++
 .../ozone/recon/schema/ReconSchemaDefinition.java  |  34 +++++
 .../recon/schema/UtilizationSchemaDefinition.java  |  69 +++++++++
 .../hadoop/ozone/recon/schema/package-info.java    |  22 +++
 .../dev-support/findbugsExcludeFile.xml            |  28 ++++
 hadoop-ozone/ozone-recon/pom.xml                   | 147 ++++++++++++++----
 .../hadoop/ozone/recon/ReconControllerModule.java  | 102 ++++++++++++-
 .../hadoop/ozone/recon/ReconServerConfigKeys.java  |  25 +++
 .../recon/persistence/DataSourceConfiguration.java |  86 +++++++++++
 .../persistence/DefaultDataSourceProvider.java     |  74 +++++++++
 .../recon/persistence/JooqPersistenceModule.java   | 111 ++++++++++++++
 .../TransactionalMethodInterceptor.java            |  76 +++++++++
 .../ozone/recon/persistence/package-info.java      |  22 +++
 .../recon/persistence/AbstractSqlDatabaseTest.java | 146 ++++++++++++++++++
 .../TestUtilizationSchemaDefinition.java           | 160 +++++++++++++++++++
 .../ozone/recon/persistence/package-info.java      |  22 +++
 hadoop-ozone/pom.xml                               |   1 +
 23 files changed, 1524 insertions(+), 36 deletions(-)

diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml
index 5580548..731bf28 100644
--- a/hadoop-hdds/common/src/main/resources/ozone-default.xml
+++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml
@@ -2394,4 +2394,99 @@
       If enabled, tracing information is sent to tracing server.
     </description>
   </property>
+  <property>
+    <name>ozone.recon.sql.db.driver</name>
+    <value>org.sqlite.JDBC</value>
+    <tag>OZONE, RECON</tag>
+    <description>
+      Database driver class name available on the
+      Ozone Recon classpath.
+    </description>
+  </property>
+  <property>
+    <name>ozone.recon.sql.db.jdbc.url</name>
+    <value>jdbc:sqlite:/${ozone.recon.db.dir}/ozone_recon_sqlite.db</value>
+    <tag>OZONE, RECON</tag>
+    <description>
+      Ozone Recon SQL database jdbc url.
+    </description>
+  </property>
+  <property>
+    <name>ozone.recon.sql.db.username</name>
+    <value/>
+    <tag>OZONE, RECON</tag>
+    <description>
+      Ozone Recon SQL database username.
+    </description>
+  </property>
+  <property>
+    <name>ozone.recon.sql.db.password</name>
+    <value/>
+    <tag>OZONE, RECON</tag>
+    <description>
+      Ozone Recon datbase password.
+    </description>
+  </property>
+  <property>
+    <name>ozone.recon.sql.db.auto.commit</name>
+    <value>false</value>
+    <tag>OZONE, RECON</tag>
+    <description>
+      Sets the Ozone Recon database connection property of auto-commit to
+      true/false.
+    </description>
+  </property>
+  <property>
+    <name>ozone.recon.sql.db.conn.timeout</name>
+    <value>30000</value>
+    <tag>OZONE, RECON</tag>
+    <description>
+      Sets time in milliseconds before call to getConnection is timed out.
+    </description>
+  </property>
+  <property>
+    <name>ozone.recon.sql.db.conn.max.active</name>
+    <value>1</value>
+    <tag>OZONE, RECON</tag>
+    <description>
+      The max active connections to the SQL database. The default SQLite
+      database only allows single active connection, set this to a
+      resonable value like 10, for external production database.
+    </description>
+  </property>
+  <property>
+    <name>ozone.recon.sql.db.conn.max.age</name>
+    <value>1800</value>
+    <tag>OZONE, RECON</tag>
+    <description>
+      Sets maximum time a connection can be active in seconds.
+    </description>
+  </property>
+  <property>
+    <name>ozone.recon.sql.db.conn.idle.max.age</name>
+    <value>3600</value>
+    <tag>OZONE, RECON</tag>
+    <description>
+      Sets maximum time to live for idle connection in seconds.
+    </description>
+  </property>
+  <property>
+    <name>ozone.recon.sql.db.conn.idle.test.period</name>
+    <value>60</value>
+    <tag>OZONE, RECON</tag>
+    <description>
+      This sets the time (in seconds), for a connection to remain idle before
+      sending a test query to the DB. This is useful to prevent a DB from
+      timing out connections on its end.
+    </description>
+  </property>
+  <property>
+    <name>ozone.recon.sql.db.conn.idle.test</name>
+    <value>SELECT 1</value>
+    <tag>OZONE, RECON</tag>
+    <description>
+      The query to send to the DB to maintain keep-alives and test for dead
+      connections.
+    </description>
+  </property>
 </configuration>
diff --git a/hadoop-hdds/tools/pom.xml b/hadoop-hdds/tools/pom.xml
index 0e39330..689bca7 100644
--- a/hadoop-hdds/tools/pom.xml
+++ b/hadoop-hdds/tools/pom.xml
@@ -49,9 +49,8 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
     <dependency>
       <groupId>org.xerial</groupId>
       <artifactId>sqlite-jdbc</artifactId>
-      <version>3.8.7</version>
+      <version>3.25.2</version>
     </dependency>
 
-
   </dependencies>
 </project>
diff --git a/hadoop-ozone/ozone-recon-codegen/pom.xml b/hadoop-ozone/ozone-recon-codegen/pom.xml
new file mode 100644
index 0000000..336fc1a
--- /dev/null
+++ b/hadoop-ozone/ozone-recon-codegen/pom.xml
@@ -0,0 +1,58 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <parent>
+    <artifactId>hadoop-ozone</artifactId>
+    <groupId>org.apache.hadoop</groupId>
+    <version>0.5.0-SNAPSHOT</version>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+  <artifactId>hadoop-ozone-recon-codegen</artifactId>
+  <name>Apache Hadoop Ozone Recon CodeGen</name>
+  <properties>
+    <jooq.version>3.11.10</jooq.version>
+    <guice.version>4.1.0</guice.version>
+  </properties>
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-ozone-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.xerial</groupId>
+      <artifactId>sqlite-jdbc</artifactId>
+      <version>3.25.2</version>
+    </dependency>
+    <dependency>
+      <groupId>com.google.inject.extensions</groupId>
+      <artifactId>guice-multibindings</artifactId>
+      <version>${guice.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.springframework</groupId>
+      <artifactId>spring-jdbc</artifactId>
+      <version>5.1.3.RELEASE</version>
+    </dependency>
+    <dependency>
+      <groupId>org.jooq</groupId>
+      <artifactId>jooq-codegen</artifactId>
+      <version>${jooq.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.jooq</groupId>
+      <artifactId>jooq-meta</artifactId>
+      <version>${jooq.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.jooq</groupId>
+      <artifactId>jooq</artifactId>
+      <version>${jooq.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.google.inject</groupId>
+      <artifactId>guice</artifactId>
+      <version>${guice.version}</version>
+    </dependency>
+  </dependencies>
+</project>
\ No newline at end of file
diff --git a/hadoop-ozone/ozone-recon-codegen/src/main/java/org/hadoop/ozone/recon/codegen/JooqCodeGenerator.java b/hadoop-ozone/ozone-recon-codegen/src/main/java/org/hadoop/ozone/recon/codegen/JooqCodeGenerator.java
new file mode 100644
index 0000000..fce4e0b
--- /dev/null
+++ b/hadoop-ozone/ozone-recon-codegen/src/main/java/org/hadoop/ozone/recon/codegen/JooqCodeGenerator.java
@@ -0,0 +1,170 @@
+/**
+ * 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.codegen;
+
+import java.io.File;
+import java.sql.SQLException;
+import java.util.Set;
+
+import javax.sql.DataSource;
+
+import org.apache.commons.io.FileUtils;
+import org.hadoop.ozone.recon.schema.ReconSchemaDefinition;
+import org.jooq.codegen.GenerationTool;
+import org.jooq.meta.jaxb.Configuration;
+import org.jooq.meta.jaxb.Database;
+import org.jooq.meta.jaxb.Generate;
+import org.jooq.meta.jaxb.Generator;
+import org.jooq.meta.jaxb.Jdbc;
+import org.jooq.meta.jaxb.Strategy;
+import org.jooq.meta.jaxb.Target;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.sqlite.SQLiteDataSource;
+
+import com.google.inject.AbstractModule;
+import com.google.inject.Guice;
+import com.google.inject.Inject;
+import com.google.inject.Injector;
+import com.google.inject.Provider;
+
+/**
+ * Utility class that generates the Dao and Pojos for Recon schema. The
+ * implementations of {@link ReconSchemaDefinition} are discovered through
+ * Guice bindings in order to avoid ugly reflection code, and invoked to
+ * generate the schema over an embedded database. The jooq code generator then
+ * runs over the embedded database to generate classes for recon.
+ */
+public class JooqCodeGenerator {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(JooqCodeGenerator.class);
+
+  private static final String SQLITE_DB =
+      System.getProperty("java.io.tmpdir") + "/recon-generated-schema";
+  private static final String JDBC_URL = "jdbc:sqlite:" + SQLITE_DB;
+
+  private final Set<ReconSchemaDefinition> allDefinitions;
+
+  @Inject
+  public JooqCodeGenerator(Set<ReconSchemaDefinition> allDefinitions) {
+    this.allDefinitions = allDefinitions;
+  }
+
+  /**
+   * Create schema.
+   */
+  private void initializeSchema() throws SQLException {
+    for (ReconSchemaDefinition definition : allDefinitions) {
+      definition.initializeSchema();
+    }
+  }
+
+  /**
+   * Generate entity and DAO classes.
+   */
+  private void generateSourceCode(String outputDir) throws Exception {
+    Configuration configuration =
+        new Configuration()
+            .withJdbc(new Jdbc()
+                .withDriver("org.sqlite.JDBC")
+                .withUrl(JDBC_URL)
+                .withUser("sa")
+                .withPassword("sa"))
+            .withGenerator(new Generator()
+                .withDatabase(new Database()
+                    .withName("org.jooq.meta.sqlite.SQLiteDatabase")
+                    .withOutputSchemaToDefault(true)
+                    .withIncludeTables(true)
+                    .withIncludePrimaryKeys(true))
+                .withGenerate(new Generate()
+                    .withDaos(true)
+                    .withEmptyCatalogs(true)
+                    .withEmptySchemas(true))
+                .withStrategy(new Strategy().withName(
+                    "org.hadoop.ozone.recon.codegen.TableNamingStrategy"))
+                .withTarget(new Target()
+                    .withPackageName("org.hadoop.ozone.recon.schema")
+                    .withClean(true)
+                    .withDirectory(outputDir)));
+    GenerationTool.generate(configuration);
+  }
+
+  /**
+   * Provider for embedded datasource.
+   */
+  static class LocalDataSourceProvider implements Provider<DataSource> {
+    private static SQLiteDataSource db;
+
+    static {
+      db = new SQLiteDataSource();
+      db.setUrl(JDBC_URL);
+    }
+
+    @Override
+    public DataSource get() {
+      return db;
+    }
+
+    static void cleanup() {
+      FileUtils.deleteQuietly(new File(SQLITE_DB));
+    }
+  }
+
+  public static void main(String[] args) {
+    if (args.length < 1) {
+      throw new IllegalArgumentException("Missing required arguments: " +
+          "Need a ouput directory for generated code.\nUsage: " +
+          "org.apache.hadoop.ozone.recon.persistence.JooqCodeGenerator " +
+          "<outputDirectory>.");
+    }
+
+    String outputDir = args[0];
+    Injector injector = Guice.createInjector(
+        new ReconSchemaGenerationModule(),
+        new AbstractModule() {
+          @Override
+          protected void configure() {
+            bind(DataSource.class).toProvider(new LocalDataSourceProvider());
+            bind(JooqCodeGenerator.class);
+          }
+        });
+
+    JooqCodeGenerator codeGenerator =
+        injector.getInstance(JooqCodeGenerator.class);
+
+    // Create tables
+    try {
+      codeGenerator.initializeSchema();
+    } catch (SQLException e) {
+      LOG.error("Unable to initialize schema.", e);
+      throw new ExceptionInInitializerError(e);
+    }
+
+    // Generate Pojos and Daos
+    try {
+      codeGenerator.generateSourceCode(outputDir);
+    } catch (Exception e) {
+      LOG.error("Code generation failed. Aborting build.", e);
+      throw new ExceptionInInitializerError(e);
+    }
+
+    // Cleanup after
+    LocalDataSourceProvider.cleanup();
+  }
+}
diff --git a/hadoop-ozone/ozone-recon-codegen/src/main/java/org/hadoop/ozone/recon/codegen/ReconSchemaGenerationModule.java b/hadoop-ozone/ozone-recon-codegen/src/main/java/org/hadoop/ozone/recon/codegen/ReconSchemaGenerationModule.java
new file mode 100644
index 0000000..fa44e46
--- /dev/null
+++ b/hadoop-ozone/ozone-recon-codegen/src/main/java/org/hadoop/ozone/recon/codegen/ReconSchemaGenerationModule.java
@@ -0,0 +1,39 @@
+/**
+ * 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.codegen;
+
+import org.hadoop.ozone.recon.schema.ReconSchemaDefinition;
+import org.hadoop.ozone.recon.schema.UtilizationSchemaDefinition;
+
+import com.google.inject.AbstractModule;
+import com.google.inject.multibindings.Multibinder;
+
+/**
+ * Bindings for DDL generation and used by
+ * {@link org.hadoop.ozone.recon.codegen.JooqCodeGenerator}.
+ */
+public class ReconSchemaGenerationModule extends AbstractModule {
+  @Override
+  protected void configure() {
+    // SQL schema creation and related bindings
+    Multibinder<ReconSchemaDefinition> schemaBinder =
+        Multibinder.newSetBinder(binder(), ReconSchemaDefinition.class);
+    schemaBinder.addBinding().to(UtilizationSchemaDefinition.class);
+
+  }
+}
diff --git a/hadoop-ozone/ozone-recon-codegen/src/main/java/org/hadoop/ozone/recon/codegen/TableNamingStrategy.java b/hadoop-ozone/ozone-recon-codegen/src/main/java/org/hadoop/ozone/recon/codegen/TableNamingStrategy.java
new file mode 100644
index 0000000..93c23c4
--- /dev/null
+++ b/hadoop-ozone/ozone-recon-codegen/src/main/java/org/hadoop/ozone/recon/codegen/TableNamingStrategy.java
@@ -0,0 +1,48 @@
+/**
+ * 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.codegen;
+
+import org.jooq.codegen.DefaultGeneratorStrategy;
+import org.jooq.meta.Definition;
+import org.jooq.meta.TableDefinition;
+import org.jooq.tools.StringUtils;
+
+/**
+ * Generate Table classes with a different name from POJOS to improve
+ * readability, loaded at runtime.
+ */
+public class TableNamingStrategy extends DefaultGeneratorStrategy {
+  @Override
+  public String getJavaClassName(Definition definition, Mode mode) {
+    if (definition instanceof TableDefinition && mode == Mode.DEFAULT) {
+      StringBuilder result = new StringBuilder();
+
+      result.append(StringUtils.toCamelCase(
+          definition.getOutputName()
+              .replace(' ', '_')
+              .replace('-', '_')
+              .replace('.', '_')
+      ));
+
+      result.append("Table");
+      return result.toString();
+    } else {
+      return super.getJavaClassName(definition, mode);
+    }
+  }
+}
diff --git a/hadoop-ozone/ozone-recon-codegen/src/main/java/org/hadoop/ozone/recon/codegen/package-info.java b/hadoop-ozone/ozone-recon-codegen/src/main/java/org/hadoop/ozone/recon/codegen/package-info.java
new file mode 100644
index 0000000..2e5cf0f
--- /dev/null
+++ b/hadoop-ozone/ozone-recon-codegen/src/main/java/org/hadoop/ozone/recon/codegen/package-info.java
@@ -0,0 +1,22 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Recon code generation support for entities and daos.
+ */
+package org.hadoop.ozone.recon.codegen;
\ No newline at end of file
diff --git a/hadoop-ozone/ozone-recon-codegen/src/main/java/org/hadoop/ozone/recon/schema/ReconSchemaDefinition.java b/hadoop-ozone/ozone-recon-codegen/src/main/java/org/hadoop/ozone/recon/schema/ReconSchemaDefinition.java
new file mode 100644
index 0000000..72a105e
--- /dev/null
+++ b/hadoop-ozone/ozone-recon-codegen/src/main/java/org/hadoop/ozone/recon/schema/ReconSchemaDefinition.java
@@ -0,0 +1,34 @@
+/**
+ * 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 java.sql.SQLException;
+
+/**
+ * Classes meant to initialize the SQL schema for Recon. The implementations of
+ * this class will be used to create the SQL schema programmatically.
+ * Note: Make sure add a binding for your implementation to the Guice module,
+ * otherwise code-generator will not pick up the schema changes.
+ */
+public interface ReconSchemaDefinition {
+
+  /**
+   * Execute DDL that will create Recon schema.
+   */
+  void initializeSchema() throws SQLException;
+}
diff --git a/hadoop-ozone/ozone-recon-codegen/src/main/java/org/hadoop/ozone/recon/schema/UtilizationSchemaDefinition.java b/hadoop-ozone/ozone-recon-codegen/src/main/java/org/hadoop/ozone/recon/schema/UtilizationSchemaDefinition.java
new file mode 100644
index 0000000..977a3b3
--- /dev/null
+++ b/hadoop-ozone/ozone-recon-codegen/src/main/java/org/hadoop/ozone/recon/schema/UtilizationSchemaDefinition.java
@@ -0,0 +1,69 @@
+/**
+ * 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 java.sql.Connection;
+import java.sql.SQLException;
+
+import javax.sql.DataSource;
+
+import org.jooq.impl.DSL;
+import org.jooq.impl.SQLDataType;
+import org.springframework.transaction.annotation.Transactional;
+
+import com.google.inject.Inject;
+
+/**
+ * Programmatic definition of Recon DDL.
+ */
+public class UtilizationSchemaDefinition implements ReconSchemaDefinition {
+
+  private final DataSource dataSource;
+
+  public static final String CLUSTER_GROWTH_DAILY_TABLE_NAME =
+      "cluster_growth_daily";
+
+  @Inject
+  UtilizationSchemaDefinition(DataSource dataSource) {
+    this.dataSource = dataSource;
+  }
+
+  @Override
+  @Transactional
+  public void initializeSchema() throws SQLException {
+    Connection conn = dataSource.getConnection();
+    createClusterGrowthTable(conn);
+  }
+
+  void createClusterGrowthTable(Connection conn) {
+    DSL.using(conn).createTableIfNotExists(CLUSTER_GROWTH_DAILY_TABLE_NAME)
+        .column("timestamp", SQLDataType.TIMESTAMP)
+        .column("datanode_id", SQLDataType.INTEGER)
+        .column("datanode_host", SQLDataType.VARCHAR(1024))
+        .column("rack_id", SQLDataType.VARCHAR(1024))
+        .column("available_size", SQLDataType.BIGINT)
+        .column("used_size", SQLDataType.BIGINT)
+        .column("container_count", SQLDataType.INTEGER)
+        .column("block_count", SQLDataType.INTEGER)
+        .constraint(DSL.constraint("pk_timestamp_datanode_id")
+            .primaryKey("timestamp", "datanode_id"))
+        .execute();
+  }
+
+
+}
diff --git a/hadoop-ozone/ozone-recon-codegen/src/main/java/org/hadoop/ozone/recon/schema/package-info.java b/hadoop-ozone/ozone-recon-codegen/src/main/java/org/hadoop/ozone/recon/schema/package-info.java
new file mode 100644
index 0000000..3c701f9
--- /dev/null
+++ b/hadoop-ozone/ozone-recon-codegen/src/main/java/org/hadoop/ozone/recon/schema/package-info.java
@@ -0,0 +1,22 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Classes in this package define the schema for Recon Sql database.
+ */
+package org.hadoop.ozone.recon.schema;
\ No newline at end of file
diff --git a/hadoop-ozone/ozone-recon/dev-support/findbugsExcludeFile.xml b/hadoop-ozone/ozone-recon/dev-support/findbugsExcludeFile.xml
new file mode 100644
index 0000000..7c0ba4d
--- /dev/null
+++ b/hadoop-ozone/ozone-recon/dev-support/findbugsExcludeFile.xml
@@ -0,0 +1,28 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+   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.
+-->
+<FindBugsFilter>
+  <Match>
+    <Package name="org.hadoop.ozone.recon.schema"/>
+  </Match>
+  <Match>
+    <Package name="org.hadoop.ozone.recon.schema.tables"/>
+  </Match>
+  <Match>
+    <Package name="org.hadoop.ozone.recon.schema.tables.pojos"/>
+  </Match>
+</FindBugsFilter>
diff --git a/hadoop-ozone/ozone-recon/pom.xml b/hadoop-ozone/ozone-recon/pom.xml
index ef23770..969db66 100644
--- a/hadoop-ozone/ozone-recon/pom.xml
+++ b/hadoop-ozone/ozone-recon/pom.xml
@@ -23,6 +23,86 @@
   <name>Apache Hadoop Ozone Recon</name>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>hadoop-ozone-recon</artifactId>
+  <properties>
+    <jooq.version>3.11.10</jooq.version>
+    <spring.version>5.1.3.RELEASE</spring.version>
+    <guice.version>4.1.0</guice.version>
+  </properties>
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>exec-maven-plugin</artifactId>
+        <version>${exec-maven-plugin.version}</version>
+        <executions>
+          <execution>
+            <phase>generate-resources</phase>
+            <goals>
+              <goal>java</goal>
+            </goals>
+          </execution>
+        </executions>
+        <configuration>
+          <executable>java</executable>
+          <classpathScope>compile</classpathScope>
+          <mainClass>org.hadoop.ozone.recon.codegen.JooqCodeGenerator</mainClass>
+          <arguments>
+            <argument>${project.build.directory}/generated-sources/java</argument>
+          </arguments>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>build-helper-maven-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>add-source</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>add-source</goal>
+            </goals>
+            <configuration>
+              <sources>
+                <source>${project.build.directory}/generated-sources/java</source>
+              </sources>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>findbugs-maven-plugin</artifactId>
+        <configuration>
+          <excludeFilterFile>${basedir}/dev-support/findbugsExcludeFile.xml</excludeFilterFile>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>copy-common-html</id>
+            <phase>prepare-package</phase>
+            <goals>
+              <goal>unpack</goal>
+            </goals>
+            <configuration>
+              <artifactItems>
+                <artifactItem>
+                  <groupId>org.apache.hadoop</groupId>
+                  <artifactId>hadoop-hdds-server-framework</artifactId>
+                  <outputDirectory>${project.build.outputDirectory}
+                  </outputDirectory>
+                  <includes>webapps/static/**/*.*</includes>
+                </artifactItem>
+              </artifactItems>
+              <overWriteSnapshots>true</overWriteSnapshots>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
   <dependencies>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
@@ -44,6 +124,11 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-ozone-recon-codegen</artifactId>
+      <version>${version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-ozone-ozone-manager</artifactId>
     </dependency>
     <dependency>
@@ -66,6 +151,7 @@
           <artifactId>hk2-api</artifactId>
         </exclusion>
       </exclusions>
+      <scope>compile</scope>
     </dependency>
     <dependency>
       <groupId>org.glassfish.jersey.containers</groupId>
@@ -90,7 +176,7 @@
     <dependency>
       <groupId>com.google.inject.extensions</groupId>
       <artifactId>guice-assistedinject</artifactId>
-      <version>4.1.0</version>
+      <version>${guice.version}</version>
     </dependency>
     <dependency>
       <groupId>org.glassfish.jersey.inject</groupId>
@@ -146,34 +232,35 @@
         </exclusion>
       </exclusions>
     </dependency>
+    <dependency>
+      <groupId>org.jooq</groupId>
+      <artifactId>jooq</artifactId>
+      <version>${jooq.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.jooq</groupId>
+      <artifactId>jooq-meta</artifactId>
+      <version>${jooq.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.jooq</groupId>
+      <artifactId>jooq-codegen</artifactId>
+      <version>${jooq.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.jolbox</groupId>
+      <artifactId>bonecp</artifactId>
+      <version>0.8.0.RELEASE</version>
+    </dependency>
+    <dependency>
+      <groupId>org.xerial</groupId>
+      <artifactId>sqlite-jdbc</artifactId>
+      <version>3.25.2</version>
+    </dependency>
+    <dependency>
+      <groupId>org.springframework</groupId>
+      <artifactId>spring-jdbc</artifactId>
+      <version>${spring.version}</version>
+    </dependency>
   </dependencies>
-  <build>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-dependency-plugin</artifactId>
-        <executions>
-          <execution>
-            <id>copy-common-html</id>
-            <phase>prepare-package</phase>
-            <goals>
-              <goal>unpack</goal>
-            </goals>
-            <configuration>
-              <artifactItems>
-                <artifactItem>
-                  <groupId>org.apache.hadoop</groupId>
-                  <artifactId>hadoop-hdds-server-framework</artifactId>
-                  <outputDirectory>${project.build.outputDirectory}
-                  </outputDirectory>
-                  <includes>webapps/static/**/*.*</includes>
-                </artifactItem>
-              </artifactItems>
-              <overWriteSnapshots>true</overWriteSnapshots>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-    </plugins>
-  </build>
 </project>
\ No newline at end of file
diff --git a/hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/ReconControllerModule.java b/hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/ReconControllerModule.java
index 2b2049a..0576c6b 100644
--- a/hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/ReconControllerModule.java
+++ b/hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/ReconControllerModule.java
@@ -17,17 +17,32 @@
  */
 package org.apache.hadoop.ozone.recon;
 
+import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_SQL_AUTO_COMMIT;
+import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_SQL_CONNECTION_TIMEOUT;
+import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_SQL_DB_DRIVER;
+import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_SQL_DB_JDBC_URL;
+import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_SQL_DB_PASSWORD;
+import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_SQL_DB_USER;
+import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_SQL_IDLE_CONNECTION_TEST_PERIOD;
+import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_SQL_MAX_ACTIVE_CONNECTIONS;
+import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_SQL_MAX_CONNECTION_AGE;
+import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_SQL_MAX_IDLE_CONNECTION_AGE;
+import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_SQL_MAX_IDLE_CONNECTION_TEST_STMT;
+
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+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.recovery.ReconOmMetadataManagerImpl;
+import org.apache.hadoop.ozone.recon.spi.ContainerDBServiceProvider;
 import org.apache.hadoop.ozone.recon.spi.OzoneManagerServiceProvider;
 import org.apache.hadoop.ozone.recon.spi.impl.ReconContainerDBProvider;
-import org.apache.hadoop.ozone.recon.spi.ContainerDBServiceProvider;
 import org.apache.hadoop.ozone.recon.spi.impl.ContainerDBServiceProviderImpl;
 import org.apache.hadoop.ozone.recon.spi.impl.OzoneManagerServiceProviderImpl;
 import org.apache.hadoop.utils.db.DBStore;
 
 import com.google.inject.AbstractModule;
+import com.google.inject.Provides;
 import com.google.inject.Singleton;
 
 /**
@@ -41,11 +56,90 @@ public class ReconControllerModule extends AbstractModule {
     bind(DBStore.class)
         .toProvider(ReconContainerDBProvider.class).in(Singleton.class);
     bind(ReconOMMetadataManager.class)
-      .to(ReconOmMetadataManagerImpl.class).in(Singleton.class);
+        .to(ReconOmMetadataManagerImpl.class).in(Singleton.class);
     bind(ContainerDBServiceProvider.class)
-      .to(ContainerDBServiceProviderImpl.class).in(Singleton.class);
+        .to(ContainerDBServiceProviderImpl.class).in(Singleton.class);
     bind(OzoneManagerServiceProvider.class)
-      .to(OzoneManagerServiceProviderImpl.class).in(Singleton.class);
+        .to(OzoneManagerServiceProviderImpl.class).in(Singleton.class);
+
+    // Persistence - inject configuration provider
+    install(new JooqPersistenceModule(
+        getProvider(DataSourceConfiguration.class)));
   }
 
+  @Provides
+  DataSourceConfiguration getDataSourceConfiguration(
+      final OzoneConfiguration ozoneConfiguration) {
+
+    return new DataSourceConfiguration() {
+      @Override
+      public String getDriverClass() {
+        return ozoneConfiguration.get(OZONE_RECON_SQL_DB_DRIVER,
+            "org.sqlite.JDBC");
+      }
+
+      @Override
+      public String getJdbcUrl() {
+        return ozoneConfiguration.get(OZONE_RECON_SQL_DB_JDBC_URL);
+      }
+
+      @Override
+      public String getUserName() {
+        return ozoneConfiguration.get(OZONE_RECON_SQL_DB_USER);
+      }
+
+      @Override
+      public String getPassword() {
+        return ozoneConfiguration.get(OZONE_RECON_SQL_DB_PASSWORD);
+      }
+
+      @Override
+      public boolean setAutoCommit() {
+        return ozoneConfiguration.getBoolean(
+            OZONE_RECON_SQL_AUTO_COMMIT, false);
+      }
+
+      @Override
+      public long getConnectionTimeout() {
+        return ozoneConfiguration.getLong(
+            OZONE_RECON_SQL_CONNECTION_TIMEOUT, 30000);
+      }
+
+      @Override
+      public String getSqlDialect() {
+        return JooqPersistenceModule.DEFAULT_DIALECT.toString();
+      }
+
+      @Override
+      public Integer getMaxActiveConnections() {
+        return ozoneConfiguration.getInt(
+            OZONE_RECON_SQL_MAX_ACTIVE_CONNECTIONS, 10);
+      }
+
+      @Override
+      public Integer getMaxConnectionAge() {
+        return ozoneConfiguration.getInt(
+            OZONE_RECON_SQL_MAX_CONNECTION_AGE, 1800);
+      }
+
+      @Override
+      public Integer getMaxIdleConnectionAge() {
+        return ozoneConfiguration.getInt(
+            OZONE_RECON_SQL_MAX_IDLE_CONNECTION_AGE, 3600);
+      }
+
+      @Override
+      public String getConnectionTestStatement() {
+        return ozoneConfiguration.get(
+            OZONE_RECON_SQL_MAX_IDLE_CONNECTION_TEST_STMT, "SELECT 1");
+      }
+
+      @Override
+      public Integer getIdleConnectionTestPeriod() {
+        return ozoneConfiguration.getInt(
+            OZONE_RECON_SQL_IDLE_CONNECTION_TEST_PERIOD, 60);
+      }
+    };
+
+  }
 }
diff --git a/hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/ReconServerConfigKeys.java b/hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/ReconServerConfigKeys.java
index 1abc513..c779e11 100644
--- a/hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/ReconServerConfigKeys.java
+++ b/hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/ReconServerConfigKeys.java
@@ -67,6 +67,7 @@ public final class ReconServerConfigKeys {
 
   public static final String RECON_OM_CONNECTION_REQUEST_TIMEOUT =
       "recon.om.connection.request.timeout";
+
   public static final String RECON_OM_CONNECTION_REQUEST_TIMEOUT_DEFAULT = "5s";
 
   public static final String RECON_OM_SNAPSHOT_TASK_INITIAL_DELAY =
@@ -87,6 +88,30 @@ public final class ReconServerConfigKeys {
   public static final String RECON_OM_SNAPSHOT_TASK_FLUSH_PARAM =
       "recon.om.snapshot.task.flush.param";
 
+  // Persistence properties
+  public static final String OZONE_RECON_SQL_DB_DRIVER =
+      "ozone.recon.sql.db.driver";
+  public static final String OZONE_RECON_SQL_DB_JDBC_URL =
+      "ozone.recon.sql.db.jdbc.url";
+  public static final String OZONE_RECON_SQL_DB_USER =
+      "ozone.recon.sql.db.username";
+  public static final String OZONE_RECON_SQL_DB_PASSWORD =
+      "ozone.recon.sql.db.password";
+  public static final String OZONE_RECON_SQL_AUTO_COMMIT =
+      "ozone.recon.sql.db.auto.commit";
+  public static final String OZONE_RECON_SQL_CONNECTION_TIMEOUT =
+      "ozone.recon.sql.db.conn.timeout";
+  public static final String OZONE_RECON_SQL_MAX_ACTIVE_CONNECTIONS =
+      "ozone.recon.sql.db.conn.max.active";
+  public static final String OZONE_RECON_SQL_MAX_CONNECTION_AGE =
+      "ozone.recon.sql.db.conn.max.age";
+  public static final String OZONE_RECON_SQL_MAX_IDLE_CONNECTION_AGE =
+      "ozone.recon.sql.db.conn.idle.max.age";
+  public static final String OZONE_RECON_SQL_IDLE_CONNECTION_TEST_PERIOD =
+      "ozone.recon.sql.db.conn.idle.test.period";
+  public static final String OZONE_RECON_SQL_MAX_IDLE_CONNECTION_TEST_STMT =
+      "ozone.recon.sql.db.conn.idle.test";
+
   /**
    * Private constructor for utility class.
    */
diff --git a/hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/persistence/DataSourceConfiguration.java b/hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/persistence/DataSourceConfiguration.java
new file mode 100644
index 0000000..ec6995a
--- /dev/null
+++ b/hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/persistence/DataSourceConfiguration.java
@@ -0,0 +1,86 @@
+package org.apache.hadoop.ozone.recon.persistence;
+
+/**
+ * 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.
+ */
+
+/**
+ * Common configuration needed to instantiate {@link javax.sql.DataSource}.
+ */
+public interface DataSourceConfiguration {
+  /**
+   * Get database driver class name available on the classpath.
+   */
+  String getDriverClass();
+
+  /**
+   * Get Jdbc Url for the database server.
+   */
+  String getJdbcUrl();
+
+  /**
+   * Get username for the db.
+   */
+  String getUserName();
+
+  /**
+   * Get password for the db.
+   */
+  String getPassword();
+
+  /**
+   * Should autocommit be turned on for the datasource.
+   */
+  boolean setAutoCommit();
+
+  /**
+   * Sets the maximum time (in milliseconds) to wait before a call to
+   * getConnection is timed out.
+   */
+  long getConnectionTimeout();
+
+  /**
+   * Get a string representation of {@link org.jooq.SQLDialect}.
+   */
+  String getSqlDialect();
+
+  /**
+   * In a production database this should be set to something like 10.
+   * SQLite does not allow multiple connections, hence this defaults to 1.
+   */
+  Integer getMaxActiveConnections();
+
+  /**
+   * Sets the maximum connection age (in seconds).
+   */
+  Integer getMaxConnectionAge();
+
+  /**
+   * Sets the maximum idle connection age (in seconds).
+   */
+  Integer getMaxIdleConnectionAge();
+
+  /**
+   * Statement specific to database, usually SELECT 1.
+   */
+  String getConnectionTestStatement();
+
+  /**
+   * How often to test idle connections for being active (in seconds).
+   */
+  Integer getIdleConnectionTestPeriod();
+}
diff --git a/hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/persistence/DefaultDataSourceProvider.java b/hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/persistence/DefaultDataSourceProvider.java
new file mode 100644
index 0000000..7b28d00
--- /dev/null
+++ b/hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/persistence/DefaultDataSourceProvider.java
@@ -0,0 +1,74 @@
+package org.apache.hadoop.ozone.recon.persistence;
+
+/**
+ * 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.
+ */
+
+import javax.sql.DataSource;
+
+import org.apache.commons.lang3.StringUtils;
+import org.sqlite.SQLiteDataSource;
+
+import com.google.inject.Inject;
+import com.google.inject.Provider;
+import com.jolbox.bonecp.BoneCPDataSource;
+
+/**
+ * Provide a {@link javax.sql.DataSource} for the application.
+ */
+public class DefaultDataSourceProvider implements Provider<DataSource> {
+
+  @Inject
+  private DataSourceConfiguration configuration;
+
+  /**
+   * Create a pooled datasource for the application.
+   *
+   * Default sqlite database does not work with a connection pool, actually
+   * most embedded databases do not, hence returning native implementation for
+   * default db.
+   */
+  @Override
+  public DataSource get() {
+    if (StringUtils.contains(configuration.getJdbcUrl(), "sqlite")) {
+      SQLiteDataSource ds = new SQLiteDataSource();
+      ds.setUrl(configuration.getJdbcUrl());
+      return ds;
+    }
+
+    BoneCPDataSource cpDataSource = new BoneCPDataSource();
+
+    cpDataSource.setDriverClass(configuration.getDriverClass());
+    cpDataSource.setJdbcUrl(configuration.getJdbcUrl());
+    cpDataSource.setUsername(configuration.getUserName());
+    cpDataSource.setPassword(configuration.getPassword());
+    cpDataSource.setDefaultAutoCommit(configuration.setAutoCommit());
+    cpDataSource.setConnectionTimeoutInMs(configuration.getConnectionTimeout());
+    cpDataSource.setMaxConnectionsPerPartition(
+        configuration.getMaxActiveConnections());
+    cpDataSource.setMaxConnectionAgeInSeconds(
+        configuration.getMaxConnectionAge());
+    cpDataSource.setIdleMaxAgeInSeconds(
+        configuration.getMaxIdleConnectionAge());
+    cpDataSource.setIdleConnectionTestPeriodInSeconds(
+        configuration.getIdleConnectionTestPeriod());
+    cpDataSource.setConnectionTestStatement(
+        configuration.getConnectionTestStatement());
+
+    return cpDataSource;
+  }
+}
diff --git a/hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/persistence/JooqPersistenceModule.java b/hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/persistence/JooqPersistenceModule.java
new file mode 100644
index 0000000..c5c8310
--- /dev/null
+++ b/hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/persistence/JooqPersistenceModule.java
@@ -0,0 +1,111 @@
+/**
+ * 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 static com.google.inject.matcher.Matchers.annotatedWith;
+import static com.google.inject.matcher.Matchers.any;
+
+import java.sql.Connection;
+import javax.sql.DataSource;
+
+import org.jooq.Configuration;
+import org.jooq.ConnectionProvider;
+import org.jooq.SQLDialect;
+import org.jooq.impl.DefaultConfiguration;
+import org.springframework.dao.DataAccessException;
+import org.springframework.jdbc.datasource.DataSourceTransactionManager;
+import org.springframework.jdbc.datasource.DataSourceUtils;
+import org.springframework.jdbc.datasource.TransactionAwareDataSourceProxy;
+import org.springframework.transaction.annotation.Transactional;
+
+import com.google.inject.AbstractModule;
+import com.google.inject.Provider;
+import com.google.inject.Provides;
+import com.google.inject.Singleton;
+
+/**
+ * Persistence module that provides binding for {@link DataSource} and
+ * a MethodInterceptor for nested transactions support.
+ */
+public class JooqPersistenceModule extends AbstractModule {
+
+  private Provider<DataSourceConfiguration> configurationProvider;
+  public static final SQLDialect DEFAULT_DIALECT = SQLDialect.SQLITE;
+
+  public JooqPersistenceModule(
+      Provider<DataSourceConfiguration> configurationProvider) {
+    this.configurationProvider = configurationProvider;
+  }
+
+  @Override
+  protected void configure() {
+    bind(DataSourceConfiguration.class).toProvider(configurationProvider);
+    bind(DataSource.class).toProvider(DefaultDataSourceProvider.class)
+        .in(Singleton.class);
+
+    TransactionalMethodInterceptor interceptor =
+        new TransactionalMethodInterceptor(
+            getProvider(DataSourceTransactionManager.class));
+
+    bindInterceptor(annotatedWith(Transactional.class), any(), interceptor);
+    bindInterceptor(any(), annotatedWith(Transactional.class), interceptor);
+  }
+
+  @Provides
+  @Singleton
+  Configuration getConfiguration(DefaultDataSourceProvider provider) {
+    DataSource dataSource = provider.get();
+
+    return new DefaultConfiguration()
+        .set(dataSource)
+        .set(new SpringConnectionProvider(dataSource))
+        .set(SQLDialect.valueOf(configurationProvider.get().getSqlDialect()));
+  }
+
+  @Provides
+  @Singleton
+  DataSourceTransactionManager provideDataSourceTransactionManager(
+      DataSource dataSource) {
+    return new DataSourceTransactionManager(
+        new TransactionAwareDataSourceProxy(dataSource));
+  }
+
+  /**
+   * This connection provider uses Spring to extract the
+   * {@link TransactionAwareDataSourceProxy} from our BoneCP pooled connection
+   * {@link DataSource}.
+   */
+  static class SpringConnectionProvider implements ConnectionProvider {
+
+    private final DataSource dataSource;
+
+    SpringConnectionProvider(DataSource dataSource) {
+      this.dataSource = dataSource;
+    }
+
+    @Override
+    public Connection acquire() throws DataAccessException {
+      return DataSourceUtils.getConnection(dataSource);
+    }
+
+    @Override
+    public void release(Connection connection) throws DataAccessException {
+      DataSourceUtils.releaseConnection(connection, dataSource);
+    }
+  }
+}
diff --git a/hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/persistence/TransactionalMethodInterceptor.java b/hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/persistence/TransactionalMethodInterceptor.java
new file mode 100644
index 0000000..4479ddd
--- /dev/null
+++ b/hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/persistence/TransactionalMethodInterceptor.java
@@ -0,0 +1,76 @@
+/**
+ * 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.aopalliance.intercept.MethodInterceptor;
+import org.aopalliance.intercept.MethodInvocation;
+import org.springframework.jdbc.datasource.DataSourceTransactionManager;
+import org.springframework.transaction.TransactionStatus;
+import org.springframework.transaction.UnexpectedRollbackException;
+import org.springframework.transaction.support.DefaultTransactionDefinition;
+
+import com.google.inject.Provider;
+
+/**
+ * A {@link MethodInterceptor} that implements nested transactions.
+ * <p>
+ * Only the outermost transactional method will <code>commit()</code> or
+ * <code>rollback()</code> the contextual transaction. This can be verified
+ * through {@link TransactionStatus#isNewTransaction()}, which returns
+ * <code>true</code> only for the outermost transactional method call.
+ * <p>
+ */
+public class TransactionalMethodInterceptor implements MethodInterceptor {
+
+  private Provider<DataSourceTransactionManager> transactionManagerProvider;
+
+  TransactionalMethodInterceptor(
+      Provider<DataSourceTransactionManager> transactionManagerProvider) {
+    this.transactionManagerProvider = transactionManagerProvider;
+  }
+
+  @Override
+  public Object invoke(MethodInvocation invocation) throws Throwable {
+    DataSourceTransactionManager transactionManager =
+        transactionManagerProvider.get();
+
+    DefaultTransactionDefinition transactionDefinition =
+        new DefaultTransactionDefinition();
+    TransactionStatus transaction = transactionManager.getTransaction(
+        transactionDefinition);
+
+    try {
+      Object result = invocation.proceed();
+
+      try {
+        if (transaction.isNewTransaction()) {
+          transactionManager.commit(transaction);
+        }
+      } catch (UnexpectedRollbackException ignore) {
+      }
+
+      return result;
+    } catch (Exception e) {
+      if (transaction.isNewTransaction()) {
+        transactionManager.rollback(transaction);
+      }
+
+      throw e;
+    }
+  }
+}
diff --git a/hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/persistence/package-info.java b/hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/persistence/package-info.java
new file mode 100644
index 0000000..0ba0fa4
--- /dev/null
+++ b/hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/persistence/package-info.java
@@ -0,0 +1,22 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * This package defines the persistence interfaces for Recon SQL DB.
+ */
+package org.apache.hadoop.ozone.recon.persistence;
\ No newline at end of file
diff --git a/hadoop-ozone/ozone-recon/src/test/java/org/apache/hadoop/ozone/recon/persistence/AbstractSqlDatabaseTest.java b/hadoop-ozone/ozone-recon/src/test/java/org/apache/hadoop/ozone/recon/persistence/AbstractSqlDatabaseTest.java
new file mode 100644
index 0000000..2fab932
--- /dev/null
+++ b/hadoop-ozone/ozone-recon/src/test/java/org/apache/hadoop/ozone/recon/persistence/AbstractSqlDatabaseTest.java
@@ -0,0 +1,146 @@
+/**
+ * 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 java.io.File;
+import java.io.IOException;
+
+import javax.sql.DataSource;
+
+import org.jooq.DSLContext;
+import org.jooq.SQLDialect;
+import org.jooq.impl.DSL;
+import org.jooq.impl.DefaultConfiguration;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.rules.TemporaryFolder;
+
+import com.google.inject.Guice;
+import com.google.inject.Injector;
+import com.google.inject.Provider;
+
+/**
+ * Create an injector for tests that need to access the SQl database.
+ */
+public abstract class AbstractSqlDatabaseTest {
+
+  @ClassRule
+  public static TemporaryFolder temporaryFolder = new TemporaryFolder();
+  private static File tempDir;
+
+  private static Injector injector;
+  private static DSLContext dslContext;
+
+  @BeforeClass
+  public static void setup() throws IOException {
+    tempDir = temporaryFolder.newFolder();
+
+    DataSourceConfigurationProvider configurationProvider =
+        new DataSourceConfigurationProvider();
+    JooqPersistenceModule persistenceModule =
+        new JooqPersistenceModule(configurationProvider);
+
+    injector = Guice.createInjector(persistenceModule);
+    dslContext = DSL.using(new DefaultConfiguration().set(
+        injector.getInstance(DataSource.class)));
+  }
+
+  @AfterClass
+  public static void tearDown() {
+    temporaryFolder.delete();
+  }
+
+  protected Injector getInjector() {
+    return injector;
+  }
+
+  protected DSLContext getDslContext() {
+    return dslContext;
+  }
+
+  static class DataSourceConfigurationProvider implements
+      Provider<DataSourceConfiguration> {
+
+    @Override
+    public DataSourceConfiguration get() {
+      return new DataSourceConfiguration() {
+        @Override
+        public String getDriverClass() {
+          return "org.sqlite.JDBC";
+        }
+
+        @Override
+        public String getJdbcUrl() {
+          return "jdbc:sqlite:" + tempDir.getAbsolutePath() +
+              File.separator + "sqlite_recon.db";
+        }
+
+        @Override
+        public String getUserName() {
+          return null;
+        }
+
+        @Override
+        public String getPassword() {
+          return null;
+        }
+
+        @Override
+        public boolean setAutoCommit() {
+          return true;
+        }
+
+        @Override
+        public long getConnectionTimeout() {
+          return 10000;
+        }
+
+        @Override
+        public String getSqlDialect() {
+          return SQLDialect.SQLITE.toString();
+        }
+
+        @Override
+        public Integer getMaxActiveConnections() {
+          return 2;
+        }
+
+        @Override
+        public Integer getMaxConnectionAge() {
+          return 120;
+        }
+
+        @Override
+        public Integer getMaxIdleConnectionAge() {
+          return 120;
+        }
+
+        @Override
+        public String getConnectionTestStatement() {
+          return "SELECT 1";
+        }
+
+        @Override
+        public Integer getIdleConnectionTestPeriod() {
+          return 30;
+        }
+      };
+    }
+  }
+}
diff --git a/hadoop-ozone/ozone-recon/src/test/java/org/apache/hadoop/ozone/recon/persistence/TestUtilizationSchemaDefinition.java b/hadoop-ozone/ozone-recon/src/test/java/org/apache/hadoop/ozone/recon/persistence/TestUtilizationSchemaDefinition.java
new file mode 100644
index 0000000..9110a31
--- /dev/null
+++ b/hadoop-ozone/ozone-recon/src/test/java/org/apache/hadoop/ozone/recon/persistence/TestUtilizationSchemaDefinition.java
@@ -0,0 +1,160 @@
+/**
+ * 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 static org.hadoop.ozone.recon.schema.UtilizationSchemaDefinition.CLUSTER_GROWTH_DAILY_TABLE_NAME;
+import static org.hadoop.ozone.recon.schema.tables.ClusterGrowthDailyTable.CLUSTER_GROWTH_DAILY;
+
+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 javax.sql.DataSource;
+
+import org.apache.commons.lang3.tuple.ImmutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+import org.hadoop.ozone.recon.schema.UtilizationSchemaDefinition;
+import org.hadoop.ozone.recon.schema.tables.daos.ClusterGrowthDailyDao;
+import org.hadoop.ozone.recon.schema.tables.pojos.ClusterGrowthDaily;
+import org.jooq.Configuration;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test persistence module provides connection and transaction awareness.
+ */
+public class TestUtilizationSchemaDefinition extends AbstractSqlDatabaseTest {
+
+  @Test
+  public void testReconSchemaCreated() throws Exception {
+    UtilizationSchemaDefinition schemaDefinition = getInjector().getInstance(
+        UtilizationSchemaDefinition.class);
+
+    schemaDefinition.initializeSchema();
+
+    Connection connection =
+        getInjector().getInstance(DataSource.class).getConnection();
+    // Verify table definition
+    DatabaseMetaData metaData = connection.getMetaData();
+    ResultSet resultSet = metaData.getColumns(null, null,
+        CLUSTER_GROWTH_DAILY_TABLE_NAME, null);
+
+    List<Pair<String, Integer>> expectedPairs = new ArrayList<>();
+
+    expectedPairs.add(new ImmutablePair<>("timestamp", Types.VARCHAR));
+    expectedPairs.add(new ImmutablePair<>("datanode_id", Types.INTEGER));
+    expectedPairs.add(new ImmutablePair<>("datanode_host", Types.VARCHAR));
+    expectedPairs.add(new ImmutablePair<>("rack_id", Types.VARCHAR));
+    expectedPairs.add(new ImmutablePair<>("available_size", Types.INTEGER));
+    expectedPairs.add(new ImmutablePair<>("used_size", Types.INTEGER));
+    expectedPairs.add(new ImmutablePair<>("container_count", Types.INTEGER));
+    expectedPairs.add(new ImmutablePair<>("block_count", Types.INTEGER));
+
+    List<Pair<String, Integer>> actualPairs = new ArrayList<>();
+
+    while (resultSet.next()) {
+      actualPairs.add(new ImmutablePair<>(resultSet.getString("COLUMN_NAME"),
+          resultSet.getInt("DATA_TYPE")));
+    }
+
+    Assert.assertEquals(8, actualPairs.size());
+    Assert.assertEquals(expectedPairs, actualPairs);
+  }
+
+  @Test
+  public void testClusterGrowthDailyCRUDOperations() throws Exception {
+    // Verify table exists
+    UtilizationSchemaDefinition schemaDefinition = getInjector().getInstance(
+        UtilizationSchemaDefinition.class);
+
+    schemaDefinition.initializeSchema();
+
+    DataSource ds = getInjector().getInstance(DataSource.class);
+    Connection connection = ds.getConnection();
+
+    DatabaseMetaData metaData = connection.getMetaData();
+    ResultSet resultSet = metaData.getTables(null, null,
+        CLUSTER_GROWTH_DAILY_TABLE_NAME, null);
+
+    while (resultSet.next()) {
+      Assert.assertEquals(CLUSTER_GROWTH_DAILY_TABLE_NAME,
+          resultSet.getString("TABLE_NAME"));
+    }
+
+    ClusterGrowthDailyDao dao = new ClusterGrowthDailyDao(
+        getInjector().getInstance(Configuration.class));
+
+    long now = System.currentTimeMillis();
+    ClusterGrowthDaily newRecord = new ClusterGrowthDaily();
+    newRecord.setTimestamp(new Timestamp(now));
+    newRecord.setDatanodeId(10);
+    newRecord.setDatanodeHost("host1");
+    newRecord.setRackId("rack1");
+    newRecord.setAvailableSize(1024L);
+    newRecord.setUsedSize(512L);
+    newRecord.setContainerCount(10);
+    newRecord.setBlockCount(25);
+
+    // Create
+    dao.insert(newRecord);
+
+    // Read
+    ClusterGrowthDaily dbRecord =
+        dao.findById(getDslContext().newRecord(CLUSTER_GROWTH_DAILY.TIMESTAMP,
+            CLUSTER_GROWTH_DAILY.DATANODE_ID)
+            .value1(new Timestamp(now)).value2(10));
+
+    Assert.assertEquals("host1", dbRecord.getDatanodeHost());
+    Assert.assertEquals("rack1", dbRecord.getRackId());
+    Assert.assertEquals(Long.valueOf(1024), dbRecord.getAvailableSize());
+    Assert.assertEquals(Long.valueOf(512), dbRecord.getUsedSize());
+    Assert.assertEquals(Integer.valueOf(10), dbRecord.getContainerCount());
+    Assert.assertEquals(Integer.valueOf(25), dbRecord.getBlockCount());
+
+    // Update
+    dbRecord.setUsedSize(700L);
+    dbRecord.setBlockCount(30);
+    dao.update(dbRecord);
+
+    // Read updated
+    dbRecord =
+        dao.findById(getDslContext().newRecord(CLUSTER_GROWTH_DAILY.TIMESTAMP,
+            CLUSTER_GROWTH_DAILY.DATANODE_ID)
+            .value1(new Timestamp(now)).value2(10));
+
+    Assert.assertEquals(Long.valueOf(700), dbRecord.getUsedSize());
+    Assert.assertEquals(Integer.valueOf(30), dbRecord.getBlockCount());
+
+    // Delete
+    dao.deleteById(getDslContext().newRecord(CLUSTER_GROWTH_DAILY.TIMESTAMP,
+        CLUSTER_GROWTH_DAILY.DATANODE_ID)
+        .value1(new Timestamp(now)).value2(10));
+
+    // Verify
+    dbRecord =
+        dao.findById(getDslContext().newRecord(CLUSTER_GROWTH_DAILY.TIMESTAMP,
+            CLUSTER_GROWTH_DAILY.DATANODE_ID)
+            .value1(new Timestamp(now)).value2(10));
+
+    Assert.assertNull(dbRecord);
+  }
+}
diff --git a/hadoop-ozone/ozone-recon/src/test/java/org/apache/hadoop/ozone/recon/persistence/package-info.java b/hadoop-ozone/ozone-recon/src/test/java/org/apache/hadoop/ozone/recon/persistence/package-info.java
new file mode 100644
index 0000000..63b8505
--- /dev/null
+++ b/hadoop-ozone/ozone-recon/src/test/java/org/apache/hadoop/ozone/recon/persistence/package-info.java
@@ -0,0 +1,22 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * End to end tests for persistence classes.
+ */
+package org.apache.hadoop.ozone.recon.persistence;
\ No newline at end of file
diff --git a/hadoop-ozone/pom.xml b/hadoop-ozone/pom.xml
index 1149894..7010878 100644
--- a/hadoop-ozone/pom.xml
+++ b/hadoop-ozone/pom.xml
@@ -49,6 +49,7 @@
     <module>s3gateway</module>
     <module>dist</module>
     <module>ozone-recon</module>
+    <module>ozone-recon-codegen</module>
   </modules>
 
   <repositories>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org