Pārlūkot izejas kodu

HDDS-2200 : Recon does not handle the NULL snapshot from OM DB cleanly.

Signed-off-by: Anu Engineer <aengineer@apache.org>
Aravindan Vijayan 5 gadi atpakaļ
vecāks
revīzija
b7cb8fe07c

+ 35 - 0
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconUtils.java

@@ -175,4 +175,39 @@ public class ReconUtils {
     }
   }
 
+  /**
+   * Load last known DB in Recon.
+   * @param reconDbDir
+   * @param fileNamePrefix
+   * @return
+   */
+  public File getLastKnownDB(File reconDbDir, String fileNamePrefix) {
+    String lastKnownSnapshotFileName = null;
+    long lastKnonwnSnapshotTs = Long.MIN_VALUE;
+    if (reconDbDir != null) {
+      File[] snapshotFiles = reconDbDir.listFiles((dir, name) ->
+          name.startsWith(fileNamePrefix));
+      if (snapshotFiles != null) {
+        for (File snapshotFile : snapshotFiles) {
+          String fileName = snapshotFile.getName();
+          try {
+            String[] fileNameSplits = fileName.split("_");
+            if (fileNameSplits.length <= 1) {
+              continue;
+            }
+            long snapshotTimestamp = Long.parseLong(fileNameSplits[1]);
+            if (lastKnonwnSnapshotTs < snapshotTimestamp) {
+              lastKnonwnSnapshotTs = snapshotTimestamp;
+              lastKnownSnapshotFileName = fileName;
+            }
+          } catch (NumberFormatException nfEx) {
+            LOG.warn("Unknown file found in Recon DB dir : {}", fileName);
+          }
+        }
+      }
+    }
+    return lastKnownSnapshotFileName == null ? null :
+        new File(reconDbDir.getPath(), lastKnownSnapshotFileName);
+  }
+
 }

+ 18 - 3
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/recovery/ReconOmMetadataManagerImpl.java

@@ -18,6 +18,9 @@
 
 package org.apache.hadoop.ozone.recon.recovery;
 
+import static org.apache.hadoop.ozone.recon.ReconConstants.RECON_OM_SNAPSHOT_DB;
+import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_OM_SNAPSHOT_DB_DIR;
+
 import java.io.File;
 import java.io.IOException;
 
@@ -28,6 +31,7 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
 import org.apache.hadoop.hdds.utils.db.DBStore;
 import org.apache.hadoop.hdds.utils.db.DBStoreBuilder;
+import org.apache.hadoop.ozone.recon.ReconUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -42,17 +46,28 @@ public class ReconOmMetadataManagerImpl extends OmMetadataManagerImpl
   private static final Logger LOG =
       LoggerFactory.getLogger(ReconOmMetadataManagerImpl.class);
 
-  @Inject
   private OzoneConfiguration ozoneConfiguration;
+  private ReconUtils reconUtils;
 
   @Inject
-  public ReconOmMetadataManagerImpl(OzoneConfiguration configuration) {
+  public ReconOmMetadataManagerImpl(OzoneConfiguration configuration,
+                                    ReconUtils reconUtils) {
+    this.reconUtils = reconUtils;
     this.ozoneConfiguration = configuration;
   }
 
   @Override
   public void start(OzoneConfiguration configuration) throws IOException {
     LOG.info("Starting ReconOMMetadataManagerImpl");
+    File reconDbDir =
+        reconUtils.getReconDbDir(configuration, OZONE_RECON_OM_SNAPSHOT_DB_DIR);
+    File lastKnownOMSnapshot =
+        reconUtils.getLastKnownDB(reconDbDir, RECON_OM_SNAPSHOT_DB);
+    if (lastKnownOMSnapshot != null) {
+      LOG.info("Last known snapshot for OM : {}",
+          lastKnownOMSnapshot.getAbsolutePath());
+      initializeNewRdbStore(lastKnownOMSnapshot);
+    }
   }
 
   /**
@@ -69,7 +84,7 @@ public class ReconOmMetadataManagerImpl extends OmMetadataManagerImpl
       addOMTablesAndCodecs(dbStoreBuilder);
       DBStore newStore = dbStoreBuilder.build();
       setStore(newStore);
-      LOG.info("Created new OM DB snapshot at {}.",
+      LOG.info("Created OM DB snapshot at {}.",
           dbFile.getAbsolutePath());
     } catch (IOException ioEx) {
       LOG.error("Unable to initialize Recon OM DB snapshot store.",

+ 18 - 10
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/ContainerDBServiceProviderImpl.java

@@ -81,14 +81,7 @@ public class ContainerDBServiceProviderImpl
   public ContainerDBServiceProviderImpl(DBStore dbStore,
                                         Configuration sqlConfiguration) {
     globalStatsDao = new GlobalStatsDao(sqlConfiguration);
-    try {
-      this.containerKeyTable = dbStore.getTable(CONTAINER_KEY_TABLE,
-          ContainerKeyPrefix.class, Integer.class);
-      this.containerKeyCountTable = dbStore.getTable(CONTAINER_KEY_COUNT_TABLE,
-          Long.class, Long.class);
-    } catch (IOException e) {
-      LOG.error("Unable to create Container Key tables." + e);
-    }
+    initializeTables(dbStore);
   }
 
   /**
@@ -107,8 +100,9 @@ public class ContainerDBServiceProviderImpl
     File oldDBLocation = containerDbStore.getDbLocation();
     containerDbStore = ReconContainerDBProvider
         .getNewDBStore(configuration, reconUtils);
-    containerKeyTable = containerDbStore.getTable(CONTAINER_KEY_TABLE,
-        ContainerKeyPrefix.class, Integer.class);
+    LOG.info("Creating new Recon Container DB at {}",
+        containerDbStore.getDbLocation().getAbsolutePath());
+    initializeTables(containerDbStore);
 
     if (oldDBLocation.exists()) {
       LOG.info("Cleaning up old Recon Container DB at {}.",
@@ -127,6 +121,20 @@ public class ContainerDBServiceProviderImpl
     storeContainerCount(0L);
   }
 
+  /**
+   * Initialize the container DB tables.
+   * @param dbStore
+   */
+  private void initializeTables(DBStore dbStore) {
+    try {
+      this.containerKeyTable = dbStore.getTable(CONTAINER_KEY_TABLE,
+          ContainerKeyPrefix.class, Integer.class);
+      this.containerKeyCountTable = dbStore.getTable(CONTAINER_KEY_COUNT_TABLE,
+          Long.class, Long.class);
+    } catch (IOException e) {
+      LOG.error("Unable to create Container Key tables." + e);
+    }
+  }
   /**
    * Concatenate the containerID and Key Prefix using a delimiter and store the
    * count into the container DB store.

+ 24 - 9
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java

@@ -178,6 +178,11 @@ public class OzoneManagerServiceProviderImpl
 
   @Override
   public void start() {
+    try {
+      omMetadataManager.start(configuration);
+    } catch (IOException ioEx) {
+      LOG.error("Error staring Recon OM Metadata Manager.", ioEx);
+    }
     long initialDelay = configuration.getTimeDuration(
         RECON_OM_SNAPSHOT_TASK_INITIAL_DELAY,
         RECON_OM_SNAPSHOT_TASK_INITIAL_DELAY_DEFAULT,
@@ -235,20 +240,24 @@ public class OzoneManagerServiceProviderImpl
    * @throws IOException
    */
   @VisibleForTesting
-  void updateReconOmDBWithNewSnapshot() throws IOException {
+  boolean updateReconOmDBWithNewSnapshot() throws IOException {
     // Obtain the current DB snapshot from OM and
     // update the in house OM metadata managed DB instance.
     DBCheckpoint dbSnapshot = getOzoneManagerDBSnapshot();
     if (dbSnapshot != null && dbSnapshot.getCheckpointLocation() != null) {
+      LOG.info("Got new checkpoint from OM : " +
+          dbSnapshot.getCheckpointLocation());
       try {
         omMetadataManager.updateOmDB(dbSnapshot.getCheckpointLocation()
             .toFile());
+        return true;
       } catch (IOException e) {
         LOG.error("Unable to refresh Recon OM DB Snapshot. ", e);
       }
     } else {
       LOG.error("Null snapshot location got from OM.");
     }
+    return false;
   }
 
   /**
@@ -287,6 +296,7 @@ public class OzoneManagerServiceProviderImpl
    */
   @VisibleForTesting
   void syncDataFromOM() {
+    LOG.info("Syncing data from Ozone Manager.");
     long currentSequenceNumber = getCurrentOMDBSequenceNumber();
     boolean fullSnapshot = false;
 
@@ -296,6 +306,7 @@ public class OzoneManagerServiceProviderImpl
       OMDBUpdatesHandler omdbUpdatesHandler =
           new OMDBUpdatesHandler(omMetadataManager);
       try {
+        LOG.info("Obtaining delta updates from Ozone Manager");
         // Get updates from OM and apply to local Recon OM DB.
         getAndApplyDeltaUpdatesFromOM(currentSequenceNumber,
             omdbUpdatesHandler);
@@ -315,16 +326,20 @@ public class OzoneManagerServiceProviderImpl
 
     if (fullSnapshot) {
       try {
+        LOG.info("Obtaining full snapshot from Ozone Manager");
         // Update local Recon OM DB to new snapshot.
-        updateReconOmDBWithNewSnapshot();
+        boolean success = updateReconOmDBWithNewSnapshot();
         // Update timestamp of successful delta updates query.
-        ReconTaskStatus reconTaskStatusRecord =
-            new ReconTaskStatus(
-                OmSnapshotTaskName.OM_DB_FULL_SNAPSHOT.name(),
-                System.currentTimeMillis(), getCurrentOMDBSequenceNumber());
-        reconTaskStatusDao.update(reconTaskStatusRecord);
-        // Reinitialize tasks that are listening.
-        reconTaskController.reInitializeTasks(omMetadataManager);
+        if (success) {
+          ReconTaskStatus reconTaskStatusRecord =
+              new ReconTaskStatus(
+                  OmSnapshotTaskName.OM_DB_FULL_SNAPSHOT.name(),
+                  System.currentTimeMillis(), getCurrentOMDBSequenceNumber());
+          reconTaskStatusDao.update(reconTaskStatusRecord);
+          // Reinitialize tasks that are listening.
+          LOG.info("Calling reprocess on Recon tasks.");
+          reconTaskController.reInitializeTasks(omMetadataManager);
+        }
       } catch (IOException | InterruptedException e) {
         LOG.error("Unable to update Recon's OM DB with new snapshot ", e);
       }

+ 26 - 6
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/ReconContainerDBProvider.java

@@ -23,6 +23,7 @@ import static org.apache.hadoop.ozone.recon.ReconConstants.RECON_CONTAINER_DB;
 import static org.apache.hadoop.ozone.recon.ReconConstants.CONTAINER_KEY_TABLE;
 import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_DB_DIR;
 
+import java.io.File;
 import java.nio.file.Path;
 
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
@@ -49,15 +50,29 @@ public class ReconContainerDBProvider implements Provider<DBStore> {
   private static final Logger LOG =
       LoggerFactory.getLogger(ReconContainerDBProvider.class);
 
-  @Inject
   private OzoneConfiguration configuration;
+  private ReconUtils reconUtils;
 
   @Inject
-  private ReconUtils reconUtils;
+  public ReconContainerDBProvider(OzoneConfiguration configuration,
+                                  ReconUtils reconUtils) {
+    this.configuration = configuration;
+    this.reconUtils = reconUtils;
+  }
 
   @Override
   public DBStore get() {
-    DBStore dbStore = getNewDBStore(configuration, reconUtils);
+    DBStore dbStore;
+    File reconDbDir =
+        reconUtils.getReconDbDir(configuration, OZONE_RECON_DB_DIR);
+    File lastKnownOMSnapshot =
+        reconUtils.getLastKnownDB(reconDbDir, RECON_CONTAINER_DB);
+    if (lastKnownOMSnapshot != null) {
+      dbStore = getDBStore(configuration, reconUtils,
+          lastKnownOMSnapshot.getName());
+    } else {
+      dbStore = getNewDBStore(configuration, reconUtils);
+    }
     if (dbStore == null) {
       throw new ProvisionException("Unable to provide instance of DBStore " +
           "store.");
@@ -65,10 +80,9 @@ public class ReconContainerDBProvider implements Provider<DBStore> {
     return dbStore;
   }
 
-  public static DBStore getNewDBStore(OzoneConfiguration configuration,
-                                      ReconUtils reconUtils) {
+  private static DBStore getDBStore(OzoneConfiguration configuration,
+                            ReconUtils reconUtils, String dbName) {
     DBStore dbStore = null;
-    String dbName = RECON_CONTAINER_DB + "_" + System.currentTimeMillis();
     try {
       Path metaDir = reconUtils.getReconDbDir(
           configuration, OZONE_RECON_DB_DIR).toPath();
@@ -86,4 +100,10 @@ public class ReconContainerDBProvider implements Provider<DBStore> {
     }
     return dbStore;
   }
+
+  static DBStore getNewDBStore(OzoneConfiguration configuration,
+                               ReconUtils reconUtils) {
+    String dbName = RECON_CONTAINER_DB + "_" + System.currentTimeMillis();
+    return getDBStore(configuration, reconUtils, dbName);
+  }
 }

+ 1 - 1
hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/AbstractOMMetadataManagerTest.java

@@ -120,7 +120,7 @@ public abstract class AbstractOMMetadataManagerTest {
         .getAbsolutePath());
 
     ReconOMMetadataManager reconOMMetaMgr =
-        new ReconOmMetadataManagerImpl(configuration);
+        new ReconOmMetadataManagerImpl(configuration, new ReconUtils());
     reconOMMetaMgr.start(configuration);
 
     reconOMMetaMgr.updateOmDB(

+ 31 - 0
hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/TestReconUtils.java

@@ -133,4 +133,35 @@ public class TestReconUtils {
     assertEquals("File 1 Contents", contents);
   }
 
+  @Test
+  public void testGetLastKnownDB() throws IOException {
+    File newDir = folder.newFolder();
+
+    File file1 = Paths.get(newDir.getAbsolutePath(), "valid_1")
+        .toFile();
+    String str = "File1 Contents";
+    BufferedWriter writer = new BufferedWriter(new FileWriter(
+        file1.getAbsolutePath()));
+    writer.write(str);
+    writer.close();
+
+    File file2 = Paths.get(newDir.getAbsolutePath(), "valid_2")
+        .toFile();
+    str = "File2 Contents";
+    writer = new BufferedWriter(new FileWriter(file2.getAbsolutePath()));
+    writer.write(str);
+    writer.close();
+
+
+    File file3 = Paths.get(newDir.getAbsolutePath(), "invalid_3")
+        .toFile();
+    str = "File3 Contents";
+    writer = new BufferedWriter(new FileWriter(file3.getAbsolutePath()));
+    writer.write(str);
+    writer.close();
+
+    ReconUtils reconUtils = new ReconUtils();
+    File latestValidFile = reconUtils.getLastKnownDB(newDir, "valid");
+    assertTrue(latestValidFile.getName().equals("valid_2"));
+  }
 }

+ 90 - 43
hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/recovery/TestReconOmMetadataManagerImpl.java

@@ -22,7 +22,9 @@ import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_DB_DIRS;
 import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_OM_SNAPSHOT_DB_DIR;
 
 import java.io.File;
+import java.io.IOException;
 
+import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.ozone.om.OMMetadataManager;
@@ -31,6 +33,7 @@ import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
 import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.hdds.utils.db.DBCheckpoint;
+import org.apache.hadoop.ozone.recon.ReconUtils;
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
@@ -44,9 +47,95 @@ public class TestReconOmMetadataManagerImpl {
   @Rule
   public TemporaryFolder temporaryFolder = new TemporaryFolder();
 
+  @Test
+  public void testStart() throws Exception {
+
+    OMMetadataManager omMetadataManager = getOMMetadataManager();
+
+    //Take checkpoint of the above OM DB.
+    DBCheckpoint checkpoint = omMetadataManager.getStore()
+        .getCheckpoint(true);
+    File snapshotFile = new File(
+        checkpoint.getCheckpointLocation().getParent() + "/" +
+            "om.snapshot.db_" + System.currentTimeMillis());
+    checkpoint.getCheckpointLocation().toFile().renameTo(snapshotFile);
+
+    //Create new Recon OM Metadata manager instance.
+    File reconOmDbDir = temporaryFolder.newFolder();
+    OzoneConfiguration configuration = new OzoneConfiguration();
+    configuration.set(OZONE_RECON_OM_SNAPSHOT_DB_DIR, reconOmDbDir
+        .getAbsolutePath());
+    FileUtils.copyDirectory(snapshotFile.getParentFile(), reconOmDbDir);
+
+    ReconOMMetadataManager reconOMMetadataManager =
+        new ReconOmMetadataManagerImpl(configuration, new ReconUtils());
+    reconOMMetadataManager.start(configuration);
+
+    Assert.assertNotNull(reconOMMetadataManager.getBucketTable());
+    Assert.assertNotNull(reconOMMetadataManager.getVolumeTable()
+        .get("/sampleVol"));
+    Assert.assertNotNull(reconOMMetadataManager.getBucketTable()
+        .get("/sampleVol/bucketOne"));
+    Assert.assertNotNull(reconOMMetadataManager.getKeyTable()
+        .get("/sampleVol/bucketOne/key_one"));
+    Assert.assertNotNull(reconOMMetadataManager.getKeyTable()
+        .get("/sampleVol/bucketOne/key_two"));
+  }
+
   @Test
   public void testUpdateOmDB() throws Exception {
 
+    OMMetadataManager omMetadataManager = getOMMetadataManager();
+    //Make sure OM Metadata reflects the keys that were inserted.
+    Assert.assertNotNull(omMetadataManager.getKeyTable()
+        .get("/sampleVol/bucketOne/key_one"));
+    Assert.assertNotNull(omMetadataManager.getKeyTable()
+        .get("/sampleVol/bucketOne/key_two"));
+
+    //Take checkpoint of OM DB.
+    DBCheckpoint checkpoint = omMetadataManager.getStore()
+        .getCheckpoint(true);
+    Assert.assertNotNull(checkpoint.getCheckpointLocation());
+
+    //Create new Recon OM Metadata manager instance.
+    File reconOmDbDir = temporaryFolder.newFolder();
+    OzoneConfiguration configuration = new OzoneConfiguration();
+    configuration.set(OZONE_RECON_OM_SNAPSHOT_DB_DIR, reconOmDbDir
+        .getAbsolutePath());
+    ReconOMMetadataManager reconOMMetadataManager =
+        new ReconOmMetadataManagerImpl(configuration, new ReconUtils());
+    reconOMMetadataManager.start(configuration);
+
+    //Before accepting a snapshot, the metadata should have null tables.
+    Assert.assertNull(reconOMMetadataManager.getBucketTable());
+
+    //Update Recon OM DB with the OM DB checkpoint location.
+    reconOMMetadataManager.updateOmDB(
+        checkpoint.getCheckpointLocation().toFile());
+
+    //Now, the tables should have been initialized.
+    Assert.assertNotNull(reconOMMetadataManager.getBucketTable());
+
+    // Check volume and bucket entries.
+    Assert.assertNotNull(reconOMMetadataManager.getVolumeTable()
+        .get("/sampleVol"));
+    Assert.assertNotNull(reconOMMetadataManager.getBucketTable()
+        .get("/sampleVol/bucketOne"));
+
+    //Verify Keys inserted in OM DB are available in Recon OM DB.
+    Assert.assertNotNull(reconOMMetadataManager.getKeyTable()
+        .get("/sampleVol/bucketOne/key_one"));
+    Assert.assertNotNull(reconOMMetadataManager.getKeyTable()
+        .get("/sampleVol/bucketOne/key_two"));
+
+  }
+
+  /**
+   * Get test OM metadata manager.
+   * @return OMMetadataManager instance
+   * @throws IOException
+   */
+  private OMMetadataManager getOMMetadataManager() throws IOException {
     //Create a new OM Metadata Manager instance + DB.
     File omDbDir = temporaryFolder.newFolder();
     OzoneConfiguration omConfiguration = new OzoneConfiguration();
@@ -93,48 +182,6 @@ public class TestReconOmMetadataManagerImpl {
             .setReplicationType(HddsProtos.ReplicationType.STAND_ALONE)
             .build());
 
-    //Make sure OM Metadata reflects the keys that were inserted.
-    Assert.assertNotNull(omMetadataManager.getKeyTable()
-        .get("/sampleVol/bucketOne/key_one"));
-    Assert.assertNotNull(omMetadataManager.getKeyTable()
-        .get("/sampleVol/bucketOne/key_two"));
-
-    //Take checkpoint of OM DB.
-    DBCheckpoint checkpoint = omMetadataManager.getStore()
-        .getCheckpoint(true);
-    Assert.assertNotNull(checkpoint.getCheckpointLocation());
-
-    //Create new Recon OM Metadata manager instance.
-    File reconOmDbDir = temporaryFolder.newFolder();
-    OzoneConfiguration configuration = new OzoneConfiguration();
-    configuration.set(OZONE_RECON_OM_SNAPSHOT_DB_DIR, reconOmDbDir
-        .getAbsolutePath());
-    ReconOMMetadataManager reconOMMetadataManager =
-        new ReconOmMetadataManagerImpl(configuration);
-    reconOMMetadataManager.start(configuration);
-
-    //Before accepting a snapshot, the metadata should have null tables.
-    Assert.assertNull(reconOMMetadataManager.getBucketTable());
-
-    //Update Recon OM DB with the OM DB checkpoint location.
-    reconOMMetadataManager.updateOmDB(
-        checkpoint.getCheckpointLocation().toFile());
-
-    //Now, the tables should have been initialized.
-    Assert.assertNotNull(reconOMMetadataManager.getBucketTable());
-
-    // Check volume and bucket entries.
-    Assert.assertNotNull(reconOMMetadataManager.getVolumeTable()
-        .get(volumeKey));
-    Assert.assertNotNull(reconOMMetadataManager.getBucketTable()
-        .get(bucketKey));
-
-    //Verify Keys inserted in OM DB are available in Recon OM DB.
-    Assert.assertNotNull(reconOMMetadataManager.getKeyTable()
-        .get("/sampleVol/bucketOne/key_one"));
-    Assert.assertNotNull(reconOMMetadataManager.getKeyTable()
-        .get("/sampleVol/bucketOne/key_two"));
-
+    return omMetadataManager;
   }
-
 }

+ 25 - 4
hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/spi/impl/TestOzoneManagerServiceProviderImpl.java

@@ -114,7 +114,7 @@ public class TestOzoneManagerServiceProviderImpl extends
     Assert.assertNull(reconOMMetadataManager.getKeyTable()
         .get("/sampleVol/bucketOne/key_two"));
 
-    ozoneManagerServiceProvider.updateReconOmDBWithNewSnapshot();
+    assertTrue(ozoneManagerServiceProvider.updateReconOmDBWithNewSnapshot());
 
     assertNotNull(reconOMMetadataManager.getKeyTable()
         .get("/sampleVol/bucketOne/key_one"));
@@ -241,10 +241,10 @@ public class TestOzoneManagerServiceProviderImpl extends
         .reInitializeTasks(omMetadataManager);
 
     OzoneManagerServiceProviderImpl ozoneManagerServiceProvider =
-        new OzoneManagerServiceProviderImpl(configuration, omMetadataManager,
+        new MockOzoneServiceProvider(configuration, omMetadataManager,
             reconTaskControllerMock, new ReconUtils(), ozoneManagerProtocol);
 
-    //Should trigger full snapshot request.
+    // Should trigger full snapshot request.
     ozoneManagerServiceProvider.syncDataFromOM();
 
     ArgumentCaptor<ReconTaskStatus> captor =
@@ -313,5 +313,26 @@ public class TestOzoneManagerServiceProviderImpl extends
         .DBUpdatesRequest.class))).thenReturn(dbUpdatesWrapper);
     return ozoneManagerProtocolMock;
   }
+}
+
+/**
+ * Mock OzoneManagerServiceProviderImpl which overrides
+ * updateReconOmDBWithNewSnapshot.
+ */
+class MockOzoneServiceProvider extends OzoneManagerServiceProviderImpl {
+
+  MockOzoneServiceProvider(OzoneConfiguration configuration,
+                           ReconOMMetadataManager omMetadataManager,
+                           ReconTaskController reconTaskController,
+                           ReconUtils reconUtils,
+                           OzoneManagerProtocol ozoneManagerClient)
+      throws IOException {
+    super(configuration, omMetadataManager, reconTaskController, reconUtils,
+        ozoneManagerClient);
+  }
 
-}
+  @Override
+  public boolean updateReconOmDBWithNewSnapshot() {
+    return true;
+  }
+}

+ 0 - 13
hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/spi/impl/TestReconContainerDBProvider.java

@@ -20,8 +20,6 @@ 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.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 
 import java.io.File;
 import java.io.IOException;
@@ -36,7 +34,6 @@ import org.junit.rules.TemporaryFolder;
 import com.google.inject.AbstractModule;
 import com.google.inject.Guice;
 import com.google.inject.Injector;
-import com.google.inject.ProvisionException;
 import com.google.inject.Singleton;
 
 /**
@@ -68,20 +65,10 @@ public class TestReconContainerDBProvider {
 
   @Test
   public void testGet() throws Exception {
-
     ReconContainerDBProvider reconContainerDBProvider = injector.getInstance(
         ReconContainerDBProvider.class);
     DBStore dbStore = reconContainerDBProvider.get();
     assertNotNull(dbStore);
-
-    ReconContainerDBProvider reconContainerDBProviderNew = new
-        ReconContainerDBProvider();
-    try {
-      reconContainerDBProviderNew.get();
-      fail();
-    } catch (Exception e) {
-      assertTrue(e instanceof ProvisionException);
-    }
   }
 
 }