diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java index c6be74a9cb34..b0560daa9a47 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java @@ -616,6 +616,11 @@ public final class OzoneConfigKeys { public static final boolean OZONE_OM_SNAPSHOT_FORCE_FULL_DIFF_DEFAULT = false; + public static final String OZONE_OM_DELTA_UPDATE_DATA_SIZE_MAX_LIMIT = + "ozone.om.delta.update.data.size.max.limit"; + public static final String + OZONE_OM_DELTA_UPDATE_DATA_SIZE_MAX_LIMIT_DEFAULT = "1024MB"; + /** * There is no need to instantiate this class. */ diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml index ec98bc3e16bc..a5b8a905dafa 100644 --- a/hadoop-hdds/common/src/main/resources/ozone-default.xml +++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml @@ -2851,6 +2851,18 @@ The actual fetched data might be larger than this limit. + + ozone.om.delta.update.data.size.max.limit + 1024MB + OM, MANAGEMENT + + Recon get a limited delta updates from OM periodically since sequence number. + Based on sequence number passed, OM DB delta update may have large number of + log files and each log batch data may be huge depending on frequent writes and + updates by ozone client, so to avoid increase in heap memory, this config is used + as limiting factor of default 1 GB while preparing DB updates object. + + recon.om.delta.update.loop.limit 10 diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/DBStoreBuilder.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/DBStoreBuilder.java index f30772466ede..5cd20ab237df 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/DBStoreBuilder.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/DBStoreBuilder.java @@ -45,6 +45,8 @@ import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_STORE_ROCKSDB_STATISTICS; import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_STORE_ROCKSDB_STATISTICS_DEFAULT; import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_STORE_ROCKSDB_STATISTICS_OFF; +import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OM_DELTA_UPDATE_DATA_SIZE_MAX_LIMIT; +import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OM_DELTA_UPDATE_DATA_SIZE_MAX_LIMIT_DEFAULT; import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OM_SNAPSHOT_COMPACTION_DAG_MAX_TIME_ALLOWED; import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OM_SNAPSHOT_COMPACTION_DAG_MAX_TIME_ALLOWED_DEFAULT; import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OM_SNAPSHOT_COMPACTION_DAG_PRUNE_DAEMON_RUN_INTERVAL; @@ -105,6 +107,9 @@ public final class DBStoreBuilder { private boolean enableCompactionLog; private long maxTimeAllowedForSnapshotInDag; private long pruneCompactionDagDaemonRunInterval; + // this is to track the total size of dbUpdates data since sequence + // number in request to avoid increase in heap memory. + private long maxDbUpdatesSizeThreshold; /** * Create DBStoreBuilder from a generic DBDefinition. @@ -162,6 +167,10 @@ private DBStoreBuilder(ConfigurationSource configuration, OZONE_OM_SNAPSHOT_COMPACTION_DAG_PRUNE_DAEMON_RUN_INTERVAL, OZONE_OM_SNAPSHOT_PRUNE_COMPACTION_DAG_DAEMON_RUN_INTERVAL_DEFAULT, TimeUnit.MILLISECONDS); + + this.maxDbUpdatesSizeThreshold = (long) configuration.getStorageSize( + OZONE_OM_DELTA_UPDATE_DATA_SIZE_MAX_LIMIT, + OZONE_OM_DELTA_UPDATE_DATA_SIZE_MAX_LIMIT_DEFAULT, StorageUnit.BYTES); } private void applyDBDefinition(DBDefinition definition) { @@ -219,7 +228,7 @@ public DBStore build() throws IOException { return new RDBStore(dbFile, rocksDBOption, writeOptions, tableConfigs, registry, openReadOnly, maxFSSnapshots, dbJmxBeanNameName, enableCompactionLog, maxTimeAllowedForSnapshotInDag, - pruneCompactionDagDaemonRunInterval); + pruneCompactionDagDaemonRunInterval, maxDbUpdatesSizeThreshold); } finally { tableConfigs.forEach(TableConfig::close); } diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/DBUpdatesWrapper.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/DBUpdatesWrapper.java index d2dcc0531b3b..1ba149f40308 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/DBUpdatesWrapper.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/DBUpdatesWrapper.java @@ -30,6 +30,7 @@ public class DBUpdatesWrapper { private List dataList = new ArrayList<>(); private long currentSequenceNumber = -1; private long latestSequenceNumber = -1; + private boolean isDBUpdateSuccess = true; public void addWriteBatch(byte[] data, long sequenceNumber) { dataList.add(data); @@ -57,5 +58,13 @@ public void setLatestSequenceNumber(long sequenceNumber) { public long getLatestSequenceNumber() { return latestSequenceNumber; } + + public boolean isDBUpdateSuccess() { + return isDBUpdateSuccess; + } + + public void setDBUpdateSuccess(boolean dbUpdateSuccess) { + this.isDBUpdateSuccess = dbUpdateSuccess; + } } diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStore.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStore.java index bb647bce359d..dd7a1e995b35 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStore.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStore.java @@ -78,12 +78,18 @@ public class RDBStore implements DBStore { */ private final String dbCompactionLogDirName = "compaction-log"; + // this is to track the total size of dbUpdates data since sequence + // number in request to avoid increase in heap memory. + private long maxDbUpdatesSizeThreshold; + @VisibleForTesting public RDBStore(File dbFile, ManagedDBOptions options, - Set families) throws IOException { + Set families, long maxDbUpdatesSizeThreshold) + throws IOException { this(dbFile, options, new ManagedWriteOptions(), families, new CodecRegistry(), false, 1000, null, false, - TimeUnit.DAYS.toMillis(1), TimeUnit.HOURS.toMillis(1)); + TimeUnit.DAYS.toMillis(1), TimeUnit.HOURS.toMillis(1), + maxDbUpdatesSizeThreshold); } @SuppressWarnings("parameternumber") @@ -92,11 +98,13 @@ public RDBStore(File dbFile, ManagedDBOptions dbOptions, CodecRegistry registry, boolean readOnly, int maxFSSnapshots, String dbJmxBeanNameName, boolean enableCompactionLog, long maxTimeAllowedForSnapshotInDag, - long compactionDagDaemonInterval) + long compactionDagDaemonInterval, + long maxDbUpdatesSizeThreshold) throws IOException { Preconditions.checkNotNull(dbFile, "DB file location cannot be null"); Preconditions.checkNotNull(families); Preconditions.checkArgument(!families.isEmpty()); + this.maxDbUpdatesSizeThreshold = maxDbUpdatesSizeThreshold; codecRegistry = registry; dbLocation = dbFile; dbJmxBeanName = dbJmxBeanNameName == null ? dbFile.getName() : @@ -363,6 +371,7 @@ public DBUpdatesWrapper getUpdatesSince(long sequenceNumber, long limitCount) if (limitCount <= 0) { throw new IllegalArgumentException("Illegal count for getUpdatesSince."); } + long cumulativeDBUpdateLogBatchSize = 0L; DBUpdatesWrapper dbUpdatesWrapper = new DBUpdatesWrapper(); try (ManagedTransactionLogIterator logIterator = db.getUpdatesSince(sequenceNumber)) { @@ -407,6 +416,10 @@ public DBUpdatesWrapper getUpdatesSince(long sequenceNumber, long limitCount) if (currSequenceNumber - sequenceNumber >= limitCount) { break; } + cumulativeDBUpdateLogBatchSize += result.writeBatch().getDataSize(); + if (cumulativeDBUpdateLogBatchSize >= maxDbUpdatesSizeThreshold) { + break; + } } finally { result.writeBatch().close(); } @@ -416,6 +429,7 @@ public DBUpdatesWrapper getUpdatesSince(long sequenceNumber, long limitCount) LOG.warn("Unable to get delta updates since sequenceNumber {}. " + "This exception will be thrown to the client", sequenceNumber, e); + dbUpdatesWrapper.setDBUpdateSuccess(false); // Throw the exception back to Recon. Expect Recon to fall back to // full snapshot. throw e; @@ -423,6 +437,7 @@ public DBUpdatesWrapper getUpdatesSince(long sequenceNumber, long limitCount) LOG.error("Unable to get delta updates since sequenceNumber {}. " + "This exception will not be thrown to the client ", sequenceNumber, e); + dbUpdatesWrapper.setDBUpdateSuccess(false); } dbUpdatesWrapper.setLatestSequenceNumber(db.getLatestSequenceNumber()); return dbUpdatesWrapper; diff --git a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStore.java b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStore.java index 21b1d533905c..7393bd789084 100644 --- a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStore.java +++ b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStore.java @@ -50,6 +50,7 @@ * RDBStore Tests. */ public class TestRDBStore { + public static final int MAX_DB_UPDATES_SIZE_THRESHOLD = 80; private final List families = Arrays.asList(StringUtils.bytes2String(RocksDB.DEFAULT_COLUMN_FAMILY), "First", "Second", "Third", @@ -74,7 +75,8 @@ public void setUp(@TempDir File tempDir) throws Exception { new ManagedColumnFamilyOptions()); configSet.add(newConfig); } - rdbStore = new RDBStore(tempDir, options, configSet); + rdbStore = new RDBStore(tempDir, options, configSet, + MAX_DB_UPDATES_SIZE_THRESHOLD); } @AfterEach @@ -262,7 +264,7 @@ public void testRocksDBCheckpoint() throws Exception { RDBStore restoredStoreFromCheckPoint = new RDBStore(checkpoint.getCheckpointLocation().toFile(), - options, configSet); + options, configSet, MAX_DB_UPDATES_SIZE_THRESHOLD); // Let us make sure that our estimate is not off by 10% Assertions.assertTrue( @@ -318,11 +320,24 @@ public void testGetDBUpdatesSinceWithLimitCount() throws Exception { org.apache.commons.codec.binary.StringUtils.getBytesUtf16("Key2"), org.apache.commons.codec.binary.StringUtils .getBytesUtf16("Value2")); + firstTable.put( + org.apache.commons.codec.binary.StringUtils.getBytesUtf16("Key3"), + org.apache.commons.codec.binary.StringUtils + .getBytesUtf16("Value3")); + firstTable.put( + org.apache.commons.codec.binary.StringUtils.getBytesUtf16("Key4"), + org.apache.commons.codec.binary.StringUtils + .getBytesUtf16("Value4")); + firstTable.put( + org.apache.commons.codec.binary.StringUtils.getBytesUtf16("Key5"), + org.apache.commons.codec.binary.StringUtils + .getBytesUtf16("Value5")); } - Assertions.assertEquals(2, rdbStore.getDb().getLatestSequenceNumber()); + Assertions.assertEquals(5, rdbStore.getDb().getLatestSequenceNumber()); - DBUpdatesWrapper dbUpdatesSince = rdbStore.getUpdatesSince(0, 1); - Assertions.assertEquals(1, dbUpdatesSince.getData().size()); + DBUpdatesWrapper dbUpdatesSince = rdbStore.getUpdatesSince(0, 5); + Assertions.assertEquals(2, dbUpdatesSince.getData().size()); + Assertions.assertEquals(2, dbUpdatesSince.getCurrentSequenceNumber()); } @Test @@ -352,7 +367,8 @@ public void testDowngrade() throws Exception { new ManagedColumnFamilyOptions()); configSet.add(newConfig); } - rdbStore = new RDBStore(rdbStore.getDbLocation(), options, configSet); + rdbStore = new RDBStore(rdbStore.getDbLocation(), options, configSet, + MAX_DB_UPDATES_SIZE_THRESHOLD); for (String family : familiesMinusOne) { try (Table table = rdbStore.getTable(family)) { Assertions.assertNotNull(table, family + "is null"); diff --git a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBTableStore.java b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBTableStore.java index c53c34fdb2c8..486d16ff52a2 100644 --- a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBTableStore.java +++ b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBTableStore.java @@ -51,6 +51,7 @@ * Tests for RocksDBTable Store. */ public class TestRDBTableStore { + public static final int MAX_DB_UPDATES_SIZE_THRESHOLD = 80; private static int count = 0; private final List families = Arrays.asList(StringUtils.bytes2String(RocksDB.DEFAULT_COLUMN_FAMILY), @@ -113,7 +114,8 @@ public void setUp() throws Exception { TableConfig newConfig = new TableConfig(name, cfOptions); configSet.add(newConfig); } - rdbStore = new RDBStore(tempDir, options, configSet); + rdbStore = new RDBStore(tempDir, options, configSet, + MAX_DB_UPDATES_SIZE_THRESHOLD); } @AfterEach diff --git a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestTypedRDBTableStore.java b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestTypedRDBTableStore.java index aebaebabf6cb..22360ef2b451 100644 --- a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestTypedRDBTableStore.java +++ b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestTypedRDBTableStore.java @@ -52,6 +52,7 @@ * Tests for RocksDBTable Store. */ public class TestTypedRDBTableStore { + public static final int MAX_DB_UPDATES_SIZE_THRESHOLD = 80; private static int count = 0; private final List families = Arrays.asList(StringUtils.bytes2String(RocksDB.DEFAULT_COLUMN_FAMILY), @@ -79,7 +80,8 @@ public void setUp(@TempDir File tempDir) throws Exception { new ManagedColumnFamilyOptions()); configSet.add(newConfig); } - rdbStore = new RDBStore(tempDir, options, configSet); + rdbStore = new RDBStore(tempDir, options, configSet, + MAX_DB_UPDATES_SIZE_THRESHOLD); codecRegistry = new CodecRegistry(); diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/DBUpdates.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/DBUpdates.java index 71916db6c7e6..4d9ae34501ad 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/DBUpdates.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/DBUpdates.java @@ -31,6 +31,8 @@ public class DBUpdates { private long latestSequenceNumber = -1L; + private boolean isDBUpdateSuccess = true; + public DBUpdates() { this.dataList = new ArrayList<>(); } @@ -65,4 +67,12 @@ public void setLatestSequenceNumber(long sequenceNumber) { public long getLatestSequenceNumber() { return latestSequenceNumber; } + + public boolean isDBUpdateSuccess() { + return isDBUpdateSuccess; + } + + public void setDBUpdateSuccess(boolean dbUpdateSuccess) { + this.isDBUpdateSuccess = dbUpdateSuccess; + } } diff --git a/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto b/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto index faa430ca1d7f..de5f0269601e 100644 --- a/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto +++ b/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto @@ -1339,6 +1339,7 @@ message DBUpdatesResponse { required uint64 sequenceNumber = 1; repeated bytes data = 2; optional uint64 latestSequenceNumber = 3; + optional bool dbUpdateSuccess = 4; } message RangerBGSyncRequest { diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java index 228a56ec6595..3e1209878879 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java @@ -3980,6 +3980,7 @@ public DBUpdates getDBUpdates( DBUpdates dbUpdates = new DBUpdates(updatesSince.getData()); dbUpdates.setCurrentSequenceNumber(updatesSince.getCurrentSequenceNumber()); dbUpdates.setLatestSequenceNumber(updatesSince.getLatestSequenceNumber()); + dbUpdates.setDBUpdateSuccess(updatesSince.isDBUpdateSuccess()); return dbUpdates; } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java index 03c31d024136..44ec2be5f910 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java @@ -358,6 +358,7 @@ private DBUpdatesResponse getOMDBUpdates( } builder.setSequenceNumber(dbUpdatesWrapper.getCurrentSequenceNumber()); builder.setLatestSequenceNumber(dbUpdatesWrapper.getLatestSequenceNumber()); + builder.setDbUpdateSuccess(dbUpdatesWrapper.isDBUpdateSuccess()); return builder.build(); } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java index a3897d9a5542..32370d0e845f 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java @@ -412,8 +412,15 @@ void getAndApplyDeltaUpdatesFromOM( long inLoopLatestSequenceNumber; while (loopCount < deltaUpdateLoopLimit && deltaUpdateCnt >= deltaUpdateLimit) { - innerGetAndApplyDeltaUpdatesFromOM( - inLoopStartSequenceNumber, omdbUpdatesHandler); + if (!innerGetAndApplyDeltaUpdatesFromOM( + inLoopStartSequenceNumber, omdbUpdatesHandler)) { + LOG.error( + "Retrieve OM DB delta update failed for sequence number : {}, " + + "so falling back to full snapshot.", inLoopStartSequenceNumber); + throw new RocksDBException( + "Unable to get delta updates since sequenceNumber - " + + inLoopStartSequenceNumber); + } inLoopLatestSequenceNumber = getCurrentOMDBSequenceNumber(); deltaUpdateCnt = inLoopLatestSequenceNumber - inLoopStartSequenceNumber; inLoopStartSequenceNumber = inLoopLatestSequenceNumber; @@ -433,7 +440,7 @@ void getAndApplyDeltaUpdatesFromOM( * @throws RocksDBException when writing to RocksDB fails. */ @VisibleForTesting - void innerGetAndApplyDeltaUpdatesFromOM(long fromSequenceNumber, + boolean innerGetAndApplyDeltaUpdatesFromOM(long fromSequenceNumber, OMDBUpdatesHandler omdbUpdatesHandler) throws IOException, RocksDBException { DBUpdatesRequest dbUpdatesRequest = DBUpdatesRequest.newBuilder() @@ -469,6 +476,7 @@ void innerGetAndApplyDeltaUpdatesFromOM(long fromSequenceNumber, LOG.info("Number of updates received from OM : {}, " + "SequenceNumber diff: {}, SequenceNumber Lag from OM {}.", numUpdates, getCurrentOMDBSequenceNumber() - fromSequenceNumber, lag); + return null != dbUpdates && dbUpdates.isDBUpdateSuccess(); } /** diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/spi/impl/TestOzoneManagerServiceProviderImpl.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/spi/impl/TestOzoneManagerServiceProviderImpl.java index 8f586252c2f5..25c7d9c407d3 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/spi/impl/TestOzoneManagerServiceProviderImpl.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/spi/impl/TestOzoneManagerServiceProviderImpl.java @@ -315,6 +315,11 @@ public void testGetAndApplyDeltaUpdatesFromOMWithLimit() throws Exception { getMockOzoneManagerClientWith4Updates(dbUpdatesWrapper[0], dbUpdatesWrapper[1], dbUpdatesWrapper[2], dbUpdatesWrapper[3])); + assertEquals(true, dbUpdatesWrapper[0].isDBUpdateSuccess()); + assertEquals(true, dbUpdatesWrapper[1].isDBUpdateSuccess()); + assertEquals(true, dbUpdatesWrapper[2].isDBUpdateSuccess()); + assertEquals(true, dbUpdatesWrapper[3].isDBUpdateSuccess()); + OMDBUpdatesHandler updatesHandler = new OMDBUpdatesHandler(omMetadataManager); ozoneManagerServiceProvider.getAndApplyDeltaUpdatesFromOM(