You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by ja...@apache.org on 2020/03/23 19:27:30 UTC

[incubator-pinot] branch upsert-refactor created (now 39f5ba8)

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

jamesshao pushed a change to branch upsert-refactor
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git.


      at 39f5ba8  update to fix unit tests

This branch includes the following new commits:

     new 7832c02  add coordinator related codes
     new 8d2c875  apply pinot core changes
     new 92bba12  temp update for adding bunch of stuff
     new 63e52fe  basic split of logics
     new 592fb5f  test build
     new 18a2da4  more refactor
     new 32899a4  fix bug
     new c793323  fix another test
     new 39f5ba8  update to fix unit tests

The 9 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



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


[incubator-pinot] 02/09: apply pinot core changes

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jamesshao pushed a commit to branch upsert-refactor
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git

commit 8d2c875e1344f0b61960c0879dd9d1e3c34341c4
Author: james Shao <sj...@uber.com>
AuthorDate: Mon Feb 24 15:47:11 2020 -0800

    apply pinot core changes
---
 pinot-core/pom.xml                                 |   9 +
 .../core/data/manager/BaseTableDataManager.java    |   6 +-
 .../core/data/manager/InstanceDataManager.java     |   6 +
 .../manager/UpsertSegmentDataManager.java}         |  35 +--
 .../manager/config/TableDataManagerConfig.java     |  10 +-
 .../offline/ImmutableSegmentDataManager.java       |   2 +-
 .../manager/offline/TableDataManagerProvider.java  |  18 +-
 .../offline/UpsertImmutableSegmentDataManager.java |  67 ++++
 .../AppendLLRealtimeSegmentDataManager.java        |  58 ++++
 .../realtime/AppendRealtimeTableDataManager.java   |  55 ++++
 .../realtime/HLRealtimeSegmentDataManager.java     |  10 +-
 .../realtime/LLRealtimeSegmentDataManager.java     |  50 +--
 .../manager/realtime/RealtimeTableDataManager.java |  36 ++-
 .../UpsertLLRealtimeSegmentDataManager.java        | 153 +++++++++
 .../realtime/UpsertRealtimeTableDataManager.java   |  82 +++++
 .../UpsertSegment.java}                            |  35 +--
 .../immutable/ImmutableSegmentImpl.java            |   8 +-
 .../immutable/ImmutableSegmentLoader.java          |  27 +-
 .../immutable/ImmutableUpsertSegmentImpl.java      | 251 +++++++++++++++
 .../mutable/MutableAppendSegmentImpl.java}         |  31 +-
 .../indexsegment/mutable/MutableSegmentImpl.java   |  55 +++-
 .../mutable/MutableUpsertSegmentImpl.java          | 159 ++++++++++
 .../core/realtime/impl/RealtimeSegmentConfig.java  |  16 +-
 .../core/segment/index/SegmentMetadataImpl.java    |  29 +-
 .../SegmentDeletionListener.java}                  |  27 +-
 .../pinot/core/segment/updater/SegmentUpdater.java | 349 +++++++++++++++++++++
 .../SegmentUpdaterConfig.java}                     |  29 +-
 .../segment/updater/UpsertWaterMarkManager.java    |  90 ++++++
 .../virtualcolumn/BaseVirtualColumnProvider.java   |  18 ++
 .../virtualcolumn/VirtualColumnContext.java        |   8 +-
 .../virtualcolumn/VirtualColumnProvider.java       |   4 +
 .../mutable/BaseLongVirtualColumnProvider.java     |  57 ++++
 .../BaseVirtualColumnSingleValueReaderWriter.java} |  27 +-
 .../ValidFromInMemoryVirtualColumnProvider.java}   |  39 +--
 .../ValidUntilInMemoryVirtualColumnProvider.java}  |  39 +--
 .../VirtualColumnLongValueReaderWriter.java        | 108 +++++++
 .../realtime/LLRealtimeSegmentDataManagerTest.java |   2 +-
 .../immutable/ImmutableUpsertSegmentImplTest.java  | 127 ++++++++
 .../mutable/MutableSegmentImplTestUtils.java       |   2 +-
 ...erSegmentAggregationSingleValueQueriesTest.java |   5 +-
 .../starter/helix/HelixInstanceDataManager.java    |  11 +-
 .../realtime/provisioning/MemoryEstimator.java     |   5 +-
 42 files changed, 1909 insertions(+), 246 deletions(-)

diff --git a/pinot-core/pom.xml b/pinot-core/pom.xml
index c7179cf..74528c9 100644
--- a/pinot-core/pom.xml
+++ b/pinot-core/pom.xml
@@ -69,6 +69,15 @@
       <artifactId>pinot-common</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.apache.pinot</groupId>
+      <artifactId>pinot-grigio-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.testng</groupId>
+      <artifactId>testng</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>joda-time</groupId>
       <artifactId>joda-time</artifactId>
     </dependency>
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java
index e92d85f..a39bdfc 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java
@@ -115,7 +115,7 @@ public abstract class BaseTableDataManager implements TableDataManager {
         immutableSegment.getSegmentMetadata().getTotalRawDocs());
     _serverMetrics.addValueToTableGauge(_tableNameWithType, ServerGauge.SEGMENT_COUNT, 1L);
 
-    ImmutableSegmentDataManager newSegmentManager = new ImmutableSegmentDataManager(immutableSegment);
+    ImmutableSegmentDataManager newSegmentManager = getImmutableSegmentDataManager(immutableSegment);
     SegmentDataManager oldSegmentManager = _segmentDataManagerMap.put(segmentName, newSegmentManager);
     if (oldSegmentManager == null) {
       _logger.info("Added new immutable segment: {} to table: {}", segmentName, _tableNameWithType);
@@ -211,4 +211,8 @@ public abstract class BaseTableDataManager implements TableDataManager {
   public String getTableName() {
     return _tableNameWithType;
   }
+
+  protected ImmutableSegmentDataManager getImmutableSegmentDataManager(ImmutableSegment immutableSegment) {
+    return new ImmutableSegmentDataManager(immutableSegment);
+  }
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/InstanceDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/InstanceDataManager.java
index 4da0c07..175f5a7 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/InstanceDataManager.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/InstanceDataManager.java
@@ -20,6 +20,7 @@ package org.apache.pinot.core.data.manager;
 
 import java.io.File;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import javax.annotation.Nullable;
 import javax.annotation.concurrent.ThreadSafe;
@@ -130,4 +131,9 @@ public interface InstanceDataManager {
    * Returns the Helix property store.
    */
   ZkHelixPropertyStore<ZNRecord> getPropertyStore();
+
+  /**
+   * Return the mappings from partition -> low water marks of all the tables hosted in this server.
+   */
+  Map<String, Map<Integer, Long>> getLowWaterMarks();
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/VirtualColumnContext.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/UpsertSegmentDataManager.java
similarity index 53%
copy from pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/VirtualColumnContext.java
copy to pinot-core/src/main/java/org/apache/pinot/core/data/manager/UpsertSegmentDataManager.java
index bb32320..4f589f6 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/VirtualColumnContext.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/UpsertSegmentDataManager.java
@@ -16,29 +16,24 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.core.segment.virtualcolumn;
+package org.apache.pinot.core.data.manager;
 
-import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.grigio.common.storageProvider.UpdateLogEntry;
 
+import java.util.List;
 
-/**
- * Miscellaneous context information about the virtual column.
- * It will be used to build various components (dictionary, reader, etc) in the virtual column provider.
- */
-public class VirtualColumnContext {
-  private FieldSpec _fieldSpec;
-  private int _totalDocCount;
-
-  public VirtualColumnContext(FieldSpec fieldSpec, int totalDocCount) {
-    _fieldSpec = fieldSpec;
-    _totalDocCount = totalDocCount;
-  }
+public interface UpsertSegmentDataManager {
 
-  public FieldSpec getFieldSpec() {
-    return _fieldSpec;
-  }
+  /**
+   * update the upsert-related virtual columns with the new values in this list of update logs
+   * @param messages list of updates logs to update the virtual columns
+   */
+  void updateVirtualColumns(List<UpdateLogEntry> messages);
 
-  public int getTotalDocCount() {
-    return _totalDocCount;
-  }
+  /**
+   * get the upsert related virtual column debug info given an offset
+   * @param offset the offset we want to look up the virtual column info from
+   * @return debug info describing the upsert-related virtual column info
+   */
+  String getVirtualColumnInfo(long offset);
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/config/TableDataManagerConfig.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/config/TableDataManagerConfig.java
index 1c01ef3..3bf51df 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/config/TableDataManagerConfig.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/config/TableDataManagerConfig.java
@@ -19,13 +19,15 @@
 package org.apache.pinot.core.data.manager.config;
 
 import com.google.common.base.Preconditions;
-import javax.annotation.Nonnull;
 import org.apache.commons.configuration.Configuration;
 import org.apache.commons.configuration.PropertiesConfiguration;
 import org.apache.pinot.common.config.TableConfig;
 import org.apache.pinot.common.config.TableNameBuilder;
+import org.apache.pinot.common.utils.CommonConstants;
 import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
 
+import javax.annotation.Nonnull;
+
 
 /**
  * The config used for TableDataManager.
@@ -35,6 +37,7 @@ public class TableDataManagerConfig {
   private static final String TABLE_DATA_MANAGER_DATA_DIRECTORY = "directory";
   private static final String TABLE_DATA_MANAGER_CONSUMER_DIRECTORY = "consumerDirectory";
   private static final String TABLE_DATA_MANAGER_NAME = "name";
+  private static final String TABLE_UPDATE_SEMANTIC = "updateSemantic";
 
   private final Configuration _tableDataManagerConfig;
 
@@ -62,6 +65,10 @@ public class TableDataManagerConfig {
     return _tableDataManagerConfig.getString(TABLE_DATA_MANAGER_NAME);
   }
 
+  public CommonConstants.UpdateSemantic getUpdateSemantic() {
+    return CommonConstants.UpdateSemantic.getUpdateSemantic(_tableDataManagerConfig.getString(TABLE_UPDATE_SEMANTIC));
+  }
+
   public static TableDataManagerConfig getDefaultHelixTableDataManagerConfig(
       @Nonnull InstanceDataManagerConfig instanceDataManagerConfig, @Nonnull String tableNameWithType) {
     Configuration defaultConfig = new PropertiesConfiguration();
@@ -83,5 +90,6 @@ public class TableDataManagerConfig {
     // If we wish to override some table level configs using table config, override them here
     // Note: the configs in TableDataManagerConfig is immutable once the table is created, which mean it will not pick
     // up the latest table config
+    _tableDataManagerConfig.setProperty(TABLE_UPDATE_SEMANTIC, tableConfig.getUpdateSemantic().toString());
   }
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/ImmutableSegmentDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/ImmutableSegmentDataManager.java
index 5ffe55f..0401f61 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/ImmutableSegmentDataManager.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/ImmutableSegmentDataManager.java
@@ -27,7 +27,7 @@ import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
  */
 public class ImmutableSegmentDataManager extends SegmentDataManager {
 
-  private final ImmutableSegment _immutableSegment;
+  protected final ImmutableSegment _immutableSegment;
 
   public ImmutableSegmentDataManager(ImmutableSegment immutableSegment) {
     _immutableSegment = immutableSegment;
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/TableDataManagerProvider.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/TableDataManagerProvider.java
index aa36055..3ec1bbd 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/TableDataManagerProvider.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/TableDataManagerProvider.java
@@ -18,8 +18,6 @@
  */
 package org.apache.pinot.core.data.manager.offline;
 
-import java.util.concurrent.Semaphore;
-import javax.annotation.Nonnull;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.store.zk.ZkHelixPropertyStore;
 import org.apache.pinot.common.metrics.ServerMetrics;
@@ -27,13 +25,21 @@ import org.apache.pinot.common.utils.CommonConstants;
 import org.apache.pinot.core.data.manager.TableDataManager;
 import org.apache.pinot.core.data.manager.config.InstanceDataManagerConfig;
 import org.apache.pinot.core.data.manager.config.TableDataManagerConfig;
-import org.apache.pinot.core.data.manager.realtime.RealtimeTableDataManager;
+import org.apache.pinot.core.data.manager.realtime.AppendRealtimeTableDataManager;
+import org.apache.pinot.core.data.manager.realtime.UpsertRealtimeTableDataManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import java.util.concurrent.Semaphore;
 
 
 /**
  * Factory for {@link TableDataManager}.
  */
 public class TableDataManagerProvider {
+  private static final Logger LOGGER = LoggerFactory.getLogger(TableDataManagerProvider.class);
+
   private static Semaphore _segmentBuildSemaphore;
 
   private TableDataManagerProvider() {
@@ -55,7 +61,11 @@ public class TableDataManagerProvider {
         tableDataManager = new OfflineTableDataManager();
         break;
       case REALTIME:
-        tableDataManager = new RealtimeTableDataManager(_segmentBuildSemaphore);
+        if (tableDataManagerConfig.getUpdateSemantic() == CommonConstants.UpdateSemantic.UPSERT) {
+          tableDataManager = new UpsertRealtimeTableDataManager(_segmentBuildSemaphore);
+        } else {
+          tableDataManager = new AppendRealtimeTableDataManager(_segmentBuildSemaphore);
+        }
         break;
       default:
         throw new IllegalStateException();
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/UpsertImmutableSegmentDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/UpsertImmutableSegmentDataManager.java
new file mode 100644
index 0000000..790e912
--- /dev/null
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/UpsertImmutableSegmentDataManager.java
@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.data.manager.offline;
+
+import org.apache.pinot.common.config.TableNameBuilder;
+import org.apache.pinot.common.utils.CommonConstants;
+import org.apache.pinot.common.utils.LLCSegmentName;
+import org.apache.pinot.core.data.manager.UpsertSegmentDataManager;
+import org.apache.pinot.core.indexsegment.UpsertSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.segment.updater.SegmentUpdater;
+import org.apache.pinot.grigio.common.storageProvider.UpdateLogEntry;
+
+import java.io.IOException;
+import java.util.List;
+
+public class UpsertImmutableSegmentDataManager extends ImmutableSegmentDataManager implements UpsertSegmentDataManager {
+
+  private String _tableNameWithType;
+
+  public UpsertImmutableSegmentDataManager(ImmutableSegment immutableSegment) throws IOException {
+    super(immutableSegment);
+    _tableNameWithType = TableNameBuilder.ensureTableNameWithType(immutableSegment.getSegmentMetadata().getTableName(),
+        CommonConstants.Helix.TableType.REALTIME);
+    initVirtualColumns();
+  }
+
+  @Override
+  public void updateVirtualColumns(List<UpdateLogEntry> messages) {
+    ((UpsertSegment) _immutableSegment).updateVirtualColumn(messages);
+  }
+
+  @Override
+  public String getVirtualColumnInfo(long offset) {
+    return ((UpsertSegment) _immutableSegment).getVirtualColumnInfo(offset);
+  }
+
+  @Override
+  public void destroy() {
+    SegmentUpdater.getInstance().removeSegmentDataManager(_tableNameWithType, getSegmentName(), this);
+    super.destroy();
+  }
+
+  private void initVirtualColumns() throws IOException {
+    // 1. add listener for update events
+    // 2. load all existing messages
+    // ensure the above orders so we can ensure all events are received by this data manager
+    SegmentUpdater.getInstance().addSegmentDataManager(_tableNameWithType, new LLCSegmentName(getSegmentName()), this);
+    ((UpsertSegment) _immutableSegment).initVirtualColumn();
+  }
+}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/AppendLLRealtimeSegmentDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/AppendLLRealtimeSegmentDataManager.java
new file mode 100644
index 0000000..f725cc8
--- /dev/null
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/AppendLLRealtimeSegmentDataManager.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.data.manager.realtime;
+
+import org.apache.pinot.common.config.TableConfig;
+import org.apache.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.common.utils.LLCSegmentName;
+import org.apache.pinot.core.indexsegment.mutable.MutableAppendSegmentImpl;
+import org.apache.pinot.core.indexsegment.mutable.MutableSegmentImpl;
+import org.apache.pinot.core.realtime.impl.RealtimeSegmentConfig;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+
+import java.util.concurrent.Semaphore;
+
+public class AppendLLRealtimeSegmentDataManager extends LLRealtimeSegmentDataManager {
+
+  public AppendLLRealtimeSegmentDataManager(RealtimeSegmentZKMetadata segmentZKMetadata, TableConfig tableConfig,
+      RealtimeTableDataManager realtimeTableDataManager, String resourceDataDir, IndexLoadingConfig indexLoadingConfig,
+      Schema schema, LLCSegmentName llcSegmentName, Semaphore partitionConsumerSemaphore, ServerMetrics serverMetrics) {
+    super(segmentZKMetadata, tableConfig, realtimeTableDataManager, resourceDataDir, indexLoadingConfig, schema,
+            llcSegmentName, partitionConsumerSemaphore, serverMetrics);
+  }
+
+  @Override
+  protected MutableSegmentImpl createMutableSegment(RealtimeSegmentConfig config) {
+    return new MutableAppendSegmentImpl(config);
+  }
+
+  @Override
+  protected void processTransformedRow(GenericRow row, long offset) {
+    // do nothing
+  }
+
+  @Override
+  protected void postIndexProcessing(GenericRow row, long offset) {
+    // do nothing
+  }
+
+}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/AppendRealtimeTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/AppendRealtimeTableDataManager.java
new file mode 100644
index 0000000..741fb08
--- /dev/null
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/AppendRealtimeTableDataManager.java
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.data.manager.realtime;
+
+import org.apache.pinot.common.config.TableConfig;
+import org.apache.pinot.common.metadata.instance.InstanceZKMetadata;
+import org.apache.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.common.utils.LLCSegmentName;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.spi.data.Schema;
+
+import java.io.File;
+import java.util.concurrent.Semaphore;
+
+public class AppendRealtimeTableDataManager extends RealtimeTableDataManager {
+
+  public AppendRealtimeTableDataManager(Semaphore segmentBuildSemaphore) {
+    super(segmentBuildSemaphore);
+  }
+
+  @Override
+  protected LLRealtimeSegmentDataManager getLLRealtimeSegmentDataManager(
+          RealtimeSegmentZKMetadata realtimeSegmentZKMetadata, TableConfig tableConfig,
+          RealtimeTableDataManager realtimeTableDataManager, String indexDirPath, IndexLoadingConfig indexLoadingConfig,
+          Schema schema, LLCSegmentName llcSegmentName, Semaphore partitionConsumerSemaphore, ServerMetrics serverMetrics)
+      throws Exception {
+    return new AppendLLRealtimeSegmentDataManager(realtimeSegmentZKMetadata, tableConfig, realtimeTableDataManager,
+            indexDirPath, indexLoadingConfig, schema, llcSegmentName, partitionConsumerSemaphore, serverMetrics);
+  }
+
+  @Override
+  protected ImmutableSegment loadImmutableSegment(File indexDir, IndexLoadingConfig indexLoadingConfig, Schema schema)
+      throws Exception {
+    return ImmutableSegmentLoader.load(indexDir, indexLoadingConfig, schema);
+  }
+}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/HLRealtimeSegmentDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/HLRealtimeSegmentDataManager.java
index 421fa3f..086812e 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/HLRealtimeSegmentDataManager.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/HLRealtimeSegmentDataManager.java
@@ -30,6 +30,7 @@ import java.util.concurrent.atomic.AtomicLong;
 import org.apache.commons.io.FileUtils;
 import org.apache.pinot.common.config.IndexingConfig;
 import org.apache.pinot.common.config.TableConfig;
+import org.apache.pinot.core.indexsegment.mutable.MutableAppendSegmentImpl;
 import org.apache.pinot.spi.data.Schema;
 import org.apache.pinot.common.metadata.instance.InstanceZKMetadata;
 import org.apache.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
@@ -101,9 +102,9 @@ public class HLRealtimeSegmentDataManager extends RealtimeSegmentDataManager {
   // An instance of this class exists only for the duration of the realtime segment that is currently being consumed.
   // Once the segment is committed, the segment is handled by OfflineSegmentDataManager
   public HLRealtimeSegmentDataManager(final RealtimeSegmentZKMetadata realtimeSegmentZKMetadata,
-      final TableConfig tableConfig, InstanceZKMetadata instanceMetadata,
-      final RealtimeTableDataManager realtimeTableDataManager, final String resourceDataDir,
-      final IndexLoadingConfig indexLoadingConfig, final Schema schema, final ServerMetrics serverMetrics)
+                                      final TableConfig tableConfig, InstanceZKMetadata instanceMetadata,
+                                      final RealtimeTableDataManager realtimeTableDataManager, final String resourceDataDir,
+                                      final IndexLoadingConfig indexLoadingConfig, final Schema schema, final ServerMetrics serverMetrics)
       throws Exception {
     super();
     _segmentVersion = indexLoadingConfig.getSegmentVersion();
@@ -194,7 +195,8 @@ public class HLRealtimeSegmentDataManager extends RealtimeSegmentDataManager {
                 indexLoadingConfig.isDirectRealtimeOffheapAllocation(), serverMetrics))
             .setStatsHistory(realtimeTableDataManager.getStatsHistory())
             .setNullHandlingEnabled(indexingConfig.isNullHandlingEnabled()).build();
-    realtimeSegment = new MutableSegmentImpl(realtimeSegmentConfig);
+    realtimeSegment = new MutableAppendSegmentImpl(realtimeSegmentConfig);
+
 
     notifier = realtimeTableDataManager;
 
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java
index ab04b74..7765e9e 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java
@@ -83,7 +83,7 @@ import org.slf4j.LoggerFactory;
 /**
  * Segment data manager for low level consumer realtime segments, which manages consumption and segment completion.
  */
-public class LLRealtimeSegmentDataManager extends RealtimeSegmentDataManager {
+public abstract class LLRealtimeSegmentDataManager extends RealtimeSegmentDataManager {
   protected enum State {
     // The state machine starts off with this state. While in this state we consume stream events
     // and index them in memory. We continue to be in this state until the end criteria is satisfied
@@ -202,7 +202,7 @@ public class LLRealtimeSegmentDataManager extends RealtimeSegmentDataManager {
   private final int _segmentMaxRowCount;
   private final String _resourceDataDir;
   private final IndexLoadingConfig _indexLoadingConfig;
-  private final Schema _schema;
+  protected final Schema _schema;
   // Semaphore for each partitionId only, which is to prevent two different Kafka consumers
   // from consuming with the same partitionId in parallel in the same host.
   // See the comments in {@link RealtimeTableDataManager}.
@@ -212,21 +212,22 @@ public class LLRealtimeSegmentDataManager extends RealtimeSegmentDataManager {
   // modify the permit. This boolean make sure the semaphore gets released only once when the partition stops consuming.
   private final AtomicBoolean _acquiredConsumerSemaphore;
   private final String _metricKeyName;
-  private final ServerMetrics _serverMetrics;
-  private final MutableSegmentImpl _realtimeSegment;
+  protected final ServerMetrics _serverMetrics;
+  protected final MutableSegmentImpl _realtimeSegment;
   private volatile long _currentOffset;
-  private volatile State _state;
+  protected volatile State _state;
   private volatile int _numRowsConsumed = 0;
   private volatile int _numRowsIndexed = 0; // Can be different from _numRowsConsumed when metrics update is enabled.
   private volatile int _numRowsErrored = 0;
   private volatile int consecutiveErrorCount = 0;
   private long _startTimeMs = 0;
-  private final String _segmentNameStr;
   private final SegmentVersion _segmentVersion;
   private final SegmentBuildTimeLeaseExtender _leaseExtender;
   private SegmentBuildDescriptor _segmentBuildDescriptor;
   private StreamConsumerFactory _streamConsumerFactory;
 
+  protected final String _segmentNameStr;
+
   // Segment end criteria
   private volatile long _consumeEndTime = 0;
   private volatile long _finalOffset = -1;
@@ -238,14 +239,14 @@ public class LLRealtimeSegmentDataManager extends RealtimeSegmentDataManager {
 
   private Thread _consumerThread;
   private final String _streamTopic;
-  private final int _streamPartitionId;
+  protected final int _streamPartitionId;
   final String _clientId;
-  private final LLCSegmentName _llcSegmentName;
+  protected final LLCSegmentName _llcSegmentName;
   private final RecordTransformer _recordTransformer;
   private PartitionLevelConsumer _partitionLevelConsumer = null;
   private StreamMetadataProvider _streamMetadataProvider = null;
   private final File _resourceTmpDir;
-  private final String _tableNameWithType;
+  protected final String _tableNameWithType;
   private final String _timeColumnName;
   private final List<String> _invertedIndexColumns;
   private final List<String> _textIndexColumns;
@@ -253,7 +254,7 @@ public class LLRealtimeSegmentDataManager extends RealtimeSegmentDataManager {
   private final List<String> _varLengthDictionaryColumns;
   private final StarTreeIndexSpec _starTreeIndexSpec;
   private final String _sortedColumn;
-  private Logger segmentLogger;
+  protected Logger segmentLogger;
   private final String _tableStreamName;
   private final PinotDataBufferMemoryManager _memoryManager;
   private AtomicLong _lastUpdatedRowsIndexed = new AtomicLong(0);
@@ -466,16 +467,19 @@ public class LLRealtimeSegmentDataManager extends RealtimeSegmentDataManager {
           GenericRow transformedRow = _recordTransformer.transform(decodedRow);
 
           if (transformedRow != null) {
-            realtimeRowsConsumedMeter = _serverMetrics
-                .addMeteredTableValue(_metricKeyName, ServerMeter.REALTIME_ROWS_CONSUMED, 1, realtimeRowsConsumedMeter);
+            processTransformedRow(transformedRow, _currentOffset);
+            realtimeRowsConsumedMeter =
+                _serverMetrics.addMeteredTableValue(_metricKeyName, ServerMeter.REALTIME_ROWS_CONSUMED, 1,
+                    realtimeRowsConsumedMeter);
             indexedMessageCount++;
+
+            canTakeMore = _realtimeSegment.index(transformedRow, msgMetadata);
+            postIndexProcessing(transformedRow, _currentOffset);
           } else {
             realtimeRowsDroppedMeter = _serverMetrics
                 .addMeteredTableValue(_metricKeyName, ServerMeter.INVALID_REALTIME_ROWS_DROPPED, 1,
                     realtimeRowsDroppedMeter);
           }
-
-          canTakeMore = _realtimeSegment.index(transformedRow, msgMetadata);
         } catch (Exception e) {
           segmentLogger.error("Caught exception while transforming the record: {}", decodedRow, e);
           _numRowsErrored++;
@@ -501,6 +505,11 @@ public class LLRealtimeSegmentDataManager extends RealtimeSegmentDataManager {
     }
   }
 
+  protected abstract void processTransformedRow(GenericRow row, long offset);
+
+
+  protected abstract void postIndexProcessing(GenericRow row, long offset);
+
   public class PartitionConsumer implements Runnable {
     public void run() {
       long initialConsumptionEnd = 0L;
@@ -795,7 +804,7 @@ public class LLRealtimeSegmentDataManager extends RealtimeSegmentDataManager {
       return false;
     }
 
-    _realtimeTableDataManager.replaceLLSegment(_segmentNameStr, _indexLoadingConfig);
+    _realtimeTableDataManager.replaceLLSegment(_segmentNameStr, _indexLoadingConfig, _schema);
     removeSegmentFile();
     return true;
   }
@@ -828,7 +837,7 @@ public class LLRealtimeSegmentDataManager extends RealtimeSegmentDataManager {
       return false;
     }
 
-    _realtimeTableDataManager.replaceLLSegment(_segmentNameStr, _indexLoadingConfig);
+    _realtimeTableDataManager.replaceLLSegment(_segmentNameStr, _indexLoadingConfig, _schema);
     return true;
   }
 
@@ -977,7 +986,7 @@ public class LLRealtimeSegmentDataManager extends RealtimeSegmentDataManager {
 
   protected void downloadSegmentAndReplace(LLCRealtimeSegmentZKMetadata metadata) {
     closeKafkaConsumers();
-    _realtimeTableDataManager.downloadAndReplaceSegment(_segmentNameStr, metadata, _indexLoadingConfig);
+    _realtimeTableDataManager.downloadAndReplaceSegment(_segmentNameStr, metadata, _indexLoadingConfig, _schema);
   }
 
   protected long now() {
@@ -1132,7 +1141,8 @@ public class LLRealtimeSegmentDataManager extends RealtimeSegmentDataManager {
     // Start new realtime segment
     String consumerDir = realtimeTableDataManager.getConsumerDir();
     RealtimeSegmentConfig.Builder realtimeSegmentConfigBuilder =
-        new RealtimeSegmentConfig.Builder().setSegmentName(_segmentNameStr).setStreamName(_streamTopic)
+        new RealtimeSegmentConfig.Builder().setTableName(_tableNameWithType)
+            .setSegmentName(_segmentNameStr).setStreamName(_streamTopic)
             .setSchema(_schema).setCapacity(_segmentMaxRowCount)
             .setAvgNumMultiValues(indexLoadingConfig.getRealtimeAvgMultiValueCount())
             .setNoDictionaryColumns(indexLoadingConfig.getNoDictionaryColumns())
@@ -1175,7 +1185,7 @@ public class LLRealtimeSegmentDataManager extends RealtimeSegmentDataManager {
       }
     }
 
-    _realtimeSegment = new MutableSegmentImpl(realtimeSegmentConfigBuilder.build());
+    _realtimeSegment = createMutableSegment(realtimeSegmentConfigBuilder.build());
     _startOffset = _segmentZKMetadata.getStartOffset();
     _currentOffset = _startOffset;
     _resourceTmpDir = new File(resourceDataDir, "_tmp");
@@ -1211,6 +1221,8 @@ public class LLRealtimeSegmentDataManager extends RealtimeSegmentDataManager {
     start();
   }
 
+  protected abstract MutableSegmentImpl createMutableSegment(RealtimeSegmentConfig config);
+
   /**
    * Creates a new stream consumer
    * @param reason
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java
index 94c9614..2be3ca1 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java
@@ -39,6 +39,7 @@ import org.apache.pinot.common.metadata.ZKMetadataProvider;
 import org.apache.pinot.common.metadata.instance.InstanceZKMetadata;
 import org.apache.pinot.common.metadata.segment.LLCRealtimeSegmentZKMetadata;
 import org.apache.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
+import org.apache.pinot.common.metrics.ServerMetrics;
 import org.apache.pinot.common.utils.CommonConstants.Segment.Realtime.Status;
 import org.apache.pinot.common.utils.LLCSegmentName;
 import org.apache.pinot.common.utils.NamedThreadFactory;
@@ -54,10 +55,11 @@ import org.apache.pinot.core.segment.index.loader.LoaderUtils;
 import org.apache.pinot.core.segment.virtualcolumn.VirtualColumnProviderFactory;
 import org.apache.pinot.spi.data.FieldSpec;
 import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
 
 
 @ThreadSafe
-public class RealtimeTableDataManager extends BaseTableDataManager {
+public abstract class RealtimeTableDataManager extends BaseTableDataManager {
   private final ExecutorService _segmentAsyncExecutorService =
       Executors.newSingleThreadExecutor(new NamedThreadFactory("SegmentAsyncExecutorService"));
   private SegmentBuildTimeLeaseExtender _leaseExtender;
@@ -225,7 +227,7 @@ public class RealtimeTableDataManager extends BaseTableDataManager {
     if (indexDir.exists() && (realtimeSegmentZKMetadata.getStatus() == Status.DONE)) {
       // Segment already exists on disk, and metadata has been committed. Treat it like an offline segment
 
-      addSegment(ImmutableSegmentLoader.load(indexDir, indexLoadingConfig, schema));
+      addSegment(loadImmutableSegment(indexDir, indexLoadingConfig, schema));
     } else {
       // Either we don't have the segment on disk or we have not committed in ZK. We should be starting the consumer
       // for realtime segment here. If we wrote it on disk but could not get to commit to zk yet, we should replace the
@@ -247,7 +249,7 @@ public class RealtimeTableDataManager extends BaseTableDataManager {
         LLCRealtimeSegmentZKMetadata llcSegmentMetadata = (LLCRealtimeSegmentZKMetadata) realtimeSegmentZKMetadata;
         if (realtimeSegmentZKMetadata.getStatus().equals(Status.DONE)) {
           // TODO Remove code duplication here and in LLRealtimeSegmentDataManager
-          downloadAndReplaceSegment(segmentName, llcSegmentMetadata, indexLoadingConfig);
+          downloadAndReplaceSegment(segmentName, llcSegmentMetadata, indexLoadingConfig, schema);
           return;
         }
 
@@ -255,10 +257,9 @@ public class RealtimeTableDataManager extends BaseTableDataManager {
         LLCSegmentName llcSegmentName = new LLCSegmentName(segmentName);
         int streamPartitionId = llcSegmentName.getPartitionId();
         _partitionIdToSemaphoreMap.putIfAbsent(streamPartitionId, new Semaphore(1));
-        manager =
-            new LLRealtimeSegmentDataManager(realtimeSegmentZKMetadata, tableConfig, this, _indexDir.getAbsolutePath(),
-                indexLoadingConfig, schema, llcSegmentName, _partitionIdToSemaphoreMap.get(streamPartitionId),
-                _serverMetrics);
+        manager = getLLRealtimeSegmentDataManager(realtimeSegmentZKMetadata, tableConfig, this,
+                _indexDir.getAbsolutePath(), indexLoadingConfig, schema, llcSegmentName,
+                _partitionIdToSemaphoreMap.get(streamPartitionId), _serverMetrics);
       }
       _logger.info("Initialize RealtimeSegmentDataManager - " + segmentName);
       _segmentDataManagerMap.put(segmentName, manager);
@@ -266,7 +267,7 @@ public class RealtimeTableDataManager extends BaseTableDataManager {
   }
 
   public void downloadAndReplaceSegment(String segmentName, LLCRealtimeSegmentZKMetadata llcSegmentMetadata,
-      IndexLoadingConfig indexLoadingConfig) {
+      IndexLoadingConfig indexLoadingConfig, Schema schema) {
     final String uri = llcSegmentMetadata.getDownloadUrl();
     File tempSegmentFolder = new File(_indexDir, "tmp-" + segmentName + "." + System.currentTimeMillis());
     File tempFile = new File(_indexDir, segmentName + ".tar.gz");
@@ -279,7 +280,7 @@ public class RealtimeTableDataManager extends BaseTableDataManager {
       _logger.info("Uncompressed file {} into tmp dir {}", tempFile, tempSegmentFolder);
       FileUtils.moveDirectory(tempSegmentFolder.listFiles()[0], segmentFolder);
       _logger.info("Replacing LLC Segment {}", segmentName);
-      replaceLLSegment(segmentName, indexLoadingConfig);
+      replaceLLSegment(segmentName, indexLoadingConfig, schema);
     } catch (Exception e) {
       throw new RuntimeException(e);
     } finally {
@@ -302,11 +303,10 @@ public class RealtimeTableDataManager extends BaseTableDataManager {
   /**
    * Replaces a committed LLC REALTIME segment.
    */
-  public void replaceLLSegment(String segmentName, IndexLoadingConfig indexLoadingConfig) {
+  public void replaceLLSegment(String segmentName, IndexLoadingConfig indexLoadingConfig, Schema schema) {
     try {
       File indexDir = new File(_indexDir, segmentName);
-      Schema schema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
-      addSegment(ImmutableSegmentLoader.load(indexDir, indexLoadingConfig, schema));
+      addSegment(loadImmutableSegment(indexDir, indexLoadingConfig, schema));
     } catch (Exception e) {
       throw new RuntimeException(e);
     }
@@ -317,6 +317,18 @@ public class RealtimeTableDataManager extends BaseTableDataManager {
   }
 
   /**
+   *  allow {@link UpsertRealtimeTableDataManager} to override this method
+   */
+  protected abstract LLRealtimeSegmentDataManager getLLRealtimeSegmentDataManager(
+          RealtimeSegmentZKMetadata realtimeSegmentZKMetadata, TableConfig tableConfig,
+          RealtimeTableDataManager realtimeTableDataManager, String indexDirPath, IndexLoadingConfig indexLoadingConfig,
+          Schema schema, LLCSegmentName llcSegmentName, Semaphore partitionConsumerSemaphore, ServerMetrics serverMetrics)
+      throws Exception;
+
+  protected abstract ImmutableSegment loadImmutableSegment(File indexDir, IndexLoadingConfig indexLoadingConfig, Schema schema)
+      throws Exception;
+
+  /**
    * Validate a schema against the table config for real-time record consumption.
    * Ideally, we should validate these things when schema is added or table is created, but either of these
    * may be changed while the table is already provisioned. For the change to take effect, we need to restart the
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/UpsertLLRealtimeSegmentDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/UpsertLLRealtimeSegmentDataManager.java
new file mode 100644
index 0000000..f08037a
--- /dev/null
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/UpsertLLRealtimeSegmentDataManager.java
@@ -0,0 +1,153 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.data.manager.realtime;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.common.config.TableConfig;
+import org.apache.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
+import org.apache.pinot.common.metrics.ServerMeter;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.common.utils.LLCSegmentName;
+import org.apache.pinot.core.data.manager.UpsertSegmentDataManager;
+import org.apache.pinot.core.indexsegment.UpsertSegment;
+import org.apache.pinot.core.indexsegment.mutable.MutableSegmentImpl;
+import org.apache.pinot.core.indexsegment.mutable.MutableUpsertSegmentImpl;
+import org.apache.pinot.core.realtime.impl.RealtimeSegmentConfig;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.core.segment.updater.SegmentUpdater;
+import org.apache.pinot.grigio.common.messages.KeyCoordinatorQueueMsg;
+import org.apache.pinot.grigio.common.rpcQueue.ProduceTask;
+import org.apache.pinot.grigio.common.rpcQueue.QueueProducer;
+import org.apache.pinot.grigio.common.storageProvider.UpdateLogEntry;
+import org.apache.pinot.grigio.servers.KeyCoordinatorProvider;
+import org.apache.pinot.spi.data.DimensionFieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.TimeFieldSpec;
+import org.apache.pinot.spi.data.readers.GenericRow;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.Semaphore;
+
+/**
+ * class design is pretty bad right now, need to rework inheritance to abstract the base class or use composition instead
+ */
+public class UpsertLLRealtimeSegmentDataManager extends LLRealtimeSegmentDataManager implements UpsertSegmentDataManager {
+
+  private final QueueProducer _keyCoordinatorQueueProducer;
+
+  public UpsertLLRealtimeSegmentDataManager(RealtimeSegmentZKMetadata segmentZKMetadata, TableConfig tableConfig,
+      RealtimeTableDataManager realtimeTableDataManager, String resourceDataDir, IndexLoadingConfig indexLoadingConfig,
+      Schema schema, LLCSegmentName llcSegmentName, Semaphore partitionConsumerSemaphore, ServerMetrics serverMetrics) throws Exception {
+    super(segmentZKMetadata, tableConfig, realtimeTableDataManager, resourceDataDir, indexLoadingConfig, schema,
+            llcSegmentName, partitionConsumerSemaphore, serverMetrics);
+    Preconditions.checkState(_schema.getPrimaryKeyFieldSpec() != null, "primary key not found");
+    Preconditions.checkState(_schema.getOffsetKeyFieldSpec() != null, "offset key not found");
+    _keyCoordinatorQueueProducer = KeyCoordinatorProvider.getInstance().getCachedProducer(_tableNameWithType);
+    initVirtualColumns();
+  }
+
+  public String getSegmentName() {
+    return _segmentNameStr;
+  }
+
+  @Override
+  public void updateVirtualColumns(List<UpdateLogEntry> messages) {
+    ((UpsertSegment) _realtimeSegment).updateVirtualColumn(messages);
+  }
+
+  @Override
+  public String getVirtualColumnInfo(long offset) {
+    return ((UpsertSegment) _realtimeSegment).getVirtualColumnInfo(offset);
+  }
+
+  @Override
+  public void destroy() {
+    SegmentUpdater.getInstance().removeSegmentDataManager(_tableNameWithType, _llcSegmentName.getSegmentName(), this);
+    super.destroy();
+  }
+
+  @Override
+  protected MutableSegmentImpl createMutableSegment(RealtimeSegmentConfig config) {
+    return new MutableUpsertSegmentImpl(config);
+  }
+
+  @Override
+  protected void processTransformedRow(GenericRow row, long offset) {
+    row.putField(_schema.getOffsetKey(), offset);
+  }
+
+  @Override
+  protected void postIndexProcessing(GenericRow row, long offset) {
+    emitEventToKeyCoordinator(row, offset);
+  }
+
+  @Override
+  protected boolean consumeLoop() throws Exception {
+    boolean result = super.consumeLoop();
+    segmentLogger.info("flushing kafka producer");
+    _keyCoordinatorQueueProducer.flush();
+    segmentLogger.info("done flushing kafka producer");
+    return result;
+  }
+
+  /**
+   * not handling error right now
+   * @param row
+   * @param offset
+   */
+  private void emitEventToKeyCoordinator(GenericRow row, long offset) {
+    final byte[] primaryKeyBytes = getPrimaryKeyBytesFromRow(row);
+    final long timestampMillis = getTimestampFromRow(row);
+    ProduceTask<byte[], KeyCoordinatorQueueMsg> task = new ProduceTask<>(primaryKeyBytes,
+        new KeyCoordinatorQueueMsg(primaryKeyBytes, _segmentNameStr, timestampMillis, offset));
+    task.setCallback(new ProduceTask.Callback() {
+      @Override
+      public void onSuccess() {
+        // do nothing on success
+      }
+
+      @Override
+      public void onFailure(Exception ex) {
+        // right now we just log the error and not really doing anything
+        // TODO: retries/record logics
+        _serverMetrics.addMeteredGlobalValue(ServerMeter.MESSAGE_PRODUCE_FAILED_COUNT, 1);
+      }
+    });
+    _keyCoordinatorQueueProducer.produce(task);
+  }
+
+  private byte[] getPrimaryKeyBytesFromRow(GenericRow row) {
+    DimensionFieldSpec primaryKeyDimension = _schema.getPrimaryKeyFieldSpec();
+    return _schema.getByteArrayFromField(row.getValue(primaryKeyDimension.getName()), primaryKeyDimension);
+  }
+
+  private long getTimestampFromRow(GenericRow row) {
+    TimeFieldSpec spec = _schema.getTimeFieldSpec();
+    return spec.getIncomingGranularitySpec().toMillis(row.getValue(spec.getIncomingTimeColumnName()));
+  }
+
+
+  private void initVirtualColumns() throws IOException {
+    // 1. ensure the data manager can capture all update events
+    // 2. load all existing messages
+    SegmentUpdater.getInstance().addSegmentDataManager(_tableNameWithType, _llcSegmentName, this);
+    ((UpsertSegment) _realtimeSegment).initVirtualColumn();
+  }
+}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/UpsertRealtimeTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/UpsertRealtimeTableDataManager.java
new file mode 100644
index 0000000..12569db
--- /dev/null
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/UpsertRealtimeTableDataManager.java
@@ -0,0 +1,82 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.data.manager.realtime;
+
+import org.apache.pinot.common.utils.LLCSegmentName;
+import org.apache.pinot.core.data.manager.offline.UpsertImmutableSegmentDataManager;
+import org.apache.pinot.common.config.TableConfig;
+import org.apache.pinot.common.metadata.instance.InstanceZKMetadata;
+import org.apache.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.core.data.manager.offline.ImmutableSegmentDataManager;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.grigio.common.storageProvider.UpdateLogStorageProvider;
+import org.apache.pinot.spi.data.Schema;
+
+import javax.annotation.Nonnull;
+import java.io.File;
+import java.io.IOException;
+import java.util.concurrent.Semaphore;
+
+public class UpsertRealtimeTableDataManager extends RealtimeTableDataManager {
+  private UpdateLogStorageProvider _updateLogStorageProvider;
+
+  public UpsertRealtimeTableDataManager(Semaphore segmentBuildSemaphore) {
+    super(segmentBuildSemaphore);
+    _updateLogStorageProvider = UpdateLogStorageProvider.getInstance();
+  }
+
+  @Override
+  public void addSegment(@Nonnull String segmentName, @Nonnull TableConfig tableConfig,
+                         @Nonnull IndexLoadingConfig indexLoadingConfig) throws Exception {
+    _updateLogStorageProvider.addSegment(_tableNameWithType, segmentName);
+    super.addSegment(segmentName, tableConfig, indexLoadingConfig);
+  }
+
+  @Override
+  protected LLRealtimeSegmentDataManager getLLRealtimeSegmentDataManager(
+          RealtimeSegmentZKMetadata realtimeSegmentZKMetadata, TableConfig tableConfig,
+          RealtimeTableDataManager realtimeTableDataManager, String indexDirPath, IndexLoadingConfig indexLoadingConfig,
+          Schema schema, LLCSegmentName llcSegmentName, Semaphore partitionConsumerSemaphore, ServerMetrics serverMetrics)
+      throws Exception {
+    return new UpsertLLRealtimeSegmentDataManager(realtimeSegmentZKMetadata, tableConfig, realtimeTableDataManager,
+            indexDirPath, indexLoadingConfig, schema, llcSegmentName, partitionConsumerSemaphore, serverMetrics);
+  }
+
+  @Override
+  protected ImmutableSegmentDataManager getImmutableSegmentDataManager(ImmutableSegment immutableSegment) {
+    try {
+      return new UpsertImmutableSegmentDataManager(immutableSegment);
+    } catch (IOException e) {
+      throw new RuntimeException("failed to init the upsert immutable segment", e);
+    }
+  }
+
+  @Override
+  protected ImmutableSegment loadImmutableSegment(File indexDir, IndexLoadingConfig indexLoadingConfig, Schema schema) {
+    try {
+      return ImmutableSegmentLoader.loadUpsertSegment(indexDir, indexLoadingConfig, schema);
+    } catch (Exception e) {
+      throw new RuntimeException("failed to load immutable segment", e);
+    }
+  }
+
+}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/VirtualColumnContext.java b/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/UpsertSegment.java
similarity index 52%
copy from pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/VirtualColumnContext.java
copy to pinot-core/src/main/java/org/apache/pinot/core/indexsegment/UpsertSegment.java
index bb32320..125cf06 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/VirtualColumnContext.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/UpsertSegment.java
@@ -16,29 +16,26 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.core.segment.virtualcolumn;
+package org.apache.pinot.core.indexsegment;
 
-import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.grigio.common.storageProvider.UpdateLogEntry;
 
+import java.io.IOException;
 
-/**
- * Miscellaneous context information about the virtual column.
- * It will be used to build various components (dictionary, reader, etc) in the virtual column provider.
- */
-public class VirtualColumnContext {
-  private FieldSpec _fieldSpec;
-  private int _totalDocCount;
+public interface UpsertSegment {
 
-  public VirtualColumnContext(FieldSpec fieldSpec, int totalDocCount) {
-    _fieldSpec = fieldSpec;
-    _totalDocCount = totalDocCount;
-  }
+  /**
+   * update the upsert-related virtual columns with the new values in this list of update logs
+   * @param messages list of updates logs to update the virtual columns
+   */
+  void updateVirtualColumn(Iterable<UpdateLogEntry> messages);
 
-  public FieldSpec getFieldSpec() {
-    return _fieldSpec;
-  }
+  /**
+   * get the upsert related virtual column debug info given an offset
+   * @param offset the offset we want to look up the virtual column info from
+   * @return debug info describing the upsert-related virtual column info
+   */
+  String getVirtualColumnInfo(long offset);
 
-  public int getTotalDocCount() {
-    return _totalDocCount;
-  }
+  void initVirtualColumn() throws IOException;
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/immutable/ImmutableSegmentImpl.java b/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/immutable/ImmutableSegmentImpl.java
index 600660a..21f056c 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/immutable/ImmutableSegmentImpl.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/immutable/ImmutableSegmentImpl.java
@@ -43,10 +43,10 @@ import org.slf4j.LoggerFactory;
 public class ImmutableSegmentImpl implements ImmutableSegment {
   private static final Logger LOGGER = LoggerFactory.getLogger(ImmutableSegmentImpl.class);
 
-  private final SegmentDirectory _segmentDirectory;
-  private final SegmentMetadataImpl _segmentMetadata;
-  private final Map<String, ColumnIndexContainer> _indexContainerMap;
-  private final StarTreeIndexContainer _starTreeIndexContainer;
+  protected final SegmentDirectory _segmentDirectory;
+  protected final SegmentMetadataImpl _segmentMetadata;
+  protected final Map<String, ColumnIndexContainer> _indexContainerMap;
+  protected final StarTreeIndexContainer _starTreeIndexContainer;
 
   public ImmutableSegmentImpl(SegmentDirectory segmentDirectory, SegmentMetadataImpl segmentMetadata,
       Map<String, ColumnIndexContainer> columnIndexContainerMap,
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/immutable/ImmutableSegmentLoader.java b/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/immutable/ImmutableSegmentLoader.java
index 90eb839..49d7729 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/immutable/ImmutableSegmentLoader.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/immutable/ImmutableSegmentLoader.java
@@ -44,6 +44,8 @@ import org.apache.pinot.spi.data.Schema;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.annotation.Nonnull;
+
 
 public class ImmutableSegmentLoader {
   private ImmutableSegmentLoader() {
@@ -70,9 +72,23 @@ public class ImmutableSegmentLoader {
   }
 
   public static ImmutableSegment load(File indexDir, IndexLoadingConfig indexLoadingConfig, @Nullable Schema schema)
-      throws Exception {
-    Preconditions
-        .checkArgument(indexDir.isDirectory(), "Index directory: {} does not exist or is not a directory", indexDir);
+          throws Exception {
+    return loadHelper(indexDir, indexLoadingConfig, schema);
+  }
+
+  /**
+   * to load upsert related segment
+   */
+  public static ImmutableUpsertSegmentImpl loadUpsertSegment(File indexDir, IndexLoadingConfig indexLoadingConfig,
+                                                             Schema schema) throws Exception {
+    ImmutableSegmentImpl segment = loadHelper(indexDir, indexLoadingConfig, schema);
+    return ImmutableUpsertSegmentImpl.copyOf(segment);
+  }
+
+  private static ImmutableSegmentImpl loadHelper(File indexDir, IndexLoadingConfig indexLoadingConfig,
+                                                 @Nullable Schema schema) throws Exception {
+    Preconditions.checkArgument(indexDir.isDirectory(), "Index directory: {} does not exist or is not a directory",
+        indexDir);
 
     // Convert segment version if necessary
     // NOTE: this step may modify the segment metadata
@@ -100,7 +116,7 @@ public class ImmutableSegmentLoader {
     }
 
     // Load the metadata again since converter and pre-processor may have changed it
-    SegmentMetadataImpl segmentMetadata = new SegmentMetadataImpl(indexDir);
+    SegmentMetadataImpl segmentMetadata = new SegmentMetadataImpl(indexDir, schema);
 
     // Load the segment
     ReadMode readMode = indexLoadingConfig.getReadMode();
@@ -125,7 +141,7 @@ public class ImmutableSegmentLoader {
         String columnName = fieldSpec.getName();
         VirtualColumnProvider provider =
             VirtualColumnProviderFactory.buildProvider(fieldSpec.getVirtualColumnProvider());
-        VirtualColumnContext context = new VirtualColumnContext(fieldSpec, segmentMetadata.getTotalDocs());
+        VirtualColumnContext context = new VirtualColumnContext(fieldSpec, segmentMetadata.getTotalDocs(), false);
         indexContainerMap.put(columnName, provider.buildColumnIndexContainer(context));
         segmentMetadata.getColumnMetadataMap().put(columnName, provider.buildMetadata(context));
       }
@@ -141,4 +157,5 @@ public class ImmutableSegmentLoader {
 
     return new ImmutableSegmentImpl(segmentDirectory, segmentMetadata, indexContainerMap, starTreeIndexContainer);
   }
+
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/immutable/ImmutableUpsertSegmentImpl.java b/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/immutable/ImmutableUpsertSegmentImpl.java
new file mode 100644
index 0000000..5ab42f5
--- /dev/null
+++ b/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/immutable/ImmutableUpsertSegmentImpl.java
@@ -0,0 +1,251 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.indexsegment.immutable;
+
+import com.clearspring.analytics.util.Preconditions;
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.pinot.common.Utils;
+import org.apache.pinot.common.config.TableNameBuilder;
+import org.apache.pinot.common.utils.CommonConstants;
+import org.apache.pinot.core.indexsegment.UpsertSegment;
+import org.apache.pinot.core.io.reader.BaseSingleColumnSingleValueReader;
+import org.apache.pinot.core.io.reader.DataFileReader;
+import org.apache.pinot.core.segment.index.SegmentMetadataImpl;
+import org.apache.pinot.core.segment.index.column.ColumnIndexContainer;
+import org.apache.pinot.core.segment.index.readers.Dictionary;
+import org.apache.pinot.core.segment.store.SegmentDirectory;
+import org.apache.pinot.core.segment.updater.UpsertWaterMarkManager;
+import org.apache.pinot.core.segment.virtualcolumn.mutable.VirtualColumnLongValueReaderWriter;
+import org.apache.pinot.core.startree.v2.store.StarTreeIndexContainer;
+import org.apache.pinot.grigio.common.storageProvider.UpdateLogEntry;
+import org.apache.pinot.grigio.common.storageProvider.UpdateLogEntrySet;
+import org.apache.pinot.grigio.common.storageProvider.UpdateLogStorageProvider;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class ImmutableUpsertSegmentImpl extends ImmutableSegmentImpl implements UpsertSegment {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(ImmutableUpsertSegmentImpl.class);
+
+  private final List<VirtualColumnLongValueReaderWriter> _virtualColumnsReaderWriter;
+  private final String _tableNameWithType;
+  private final String _segmentName;
+  private final int _totalDoc;
+  private long _minSourceOffset;
+  private final UpsertWaterMarkManager _upsertWaterMarkManager;
+  private final UpdateLogStorageProvider _updateLogStorageProvider;
+  // use array for mapping bewteen offset to docId, where actual offset = min_offset + array_index
+  // use 4 bytes per record
+  private int[] _sourceOffsetToDocIdArray;
+
+  private static final int DEFAULT_DOC_ID_FOR_MISSING_ENTRY = -1;
+
+  public ImmutableUpsertSegmentImpl(SegmentDirectory segmentDirectory,
+                                    SegmentMetadataImpl segmentMetadata,
+                                    Map<String, ColumnIndexContainer> columnIndexContainerMap,
+                                    @Nullable StarTreeIndexContainer starTreeIndexContainer) {
+    super(segmentDirectory, segmentMetadata, columnIndexContainerMap, starTreeIndexContainer);
+    Preconditions.checkState(segmentMetadata.getSchema().isTableForUpsert(), "table should be upsert but it is not");
+    _tableNameWithType = TableNameBuilder.ensureTableNameWithType(segmentMetadata.getTableName(),
+        CommonConstants.Helix.TableType.REALTIME);
+    _segmentName = segmentMetadata.getName();
+    _totalDoc = segmentMetadata.getTotalDocs();
+    _upsertWaterMarkManager = UpsertWaterMarkManager.getInstance();
+    _updateLogStorageProvider = UpdateLogStorageProvider.getInstance();
+    _virtualColumnsReaderWriter = new ArrayList<>();
+    for (Map.Entry<String, ColumnIndexContainer> entry: columnIndexContainerMap.entrySet()) {
+      String columnName = entry.getKey();
+      ColumnIndexContainer container = entry.getValue();
+      if (segmentMetadata.getSchema().isVirtualColumn(columnName) && (container.getForwardIndex() instanceof VirtualColumnLongValueReaderWriter)) {
+        _virtualColumnsReaderWriter.add((VirtualColumnLongValueReaderWriter) container.getForwardIndex());
+      }
+    }
+     buildOffsetToDocIdMap(columnIndexContainerMap.get(segmentMetadata.getSchema().getOffsetKey()));
+  }
+
+  /** constructor used for creating instance in test cases
+   * should not be used for creating regular segment
+   */
+  @VisibleForTesting
+  protected ImmutableUpsertSegmentImpl(List<VirtualColumnLongValueReaderWriter> readerWriters,
+                                       int totalDoc, UpsertWaterMarkManager manager,
+                                       UpdateLogStorageProvider updateLogStorageProvider,
+                                       long minSourceOffset, int[] offsetToDocId) {
+    super(null, null, null, null);
+    _tableNameWithType = "testTable";
+    _segmentName = "testSegment";
+    _virtualColumnsReaderWriter = readerWriters;
+    _totalDoc = totalDoc;
+    _upsertWaterMarkManager = manager;
+    _updateLogStorageProvider = updateLogStorageProvider;
+    _minSourceOffset = minSourceOffset;
+    _sourceOffsetToDocIdArray = offsetToDocId;
+
+  }
+
+  private void buildOffsetToDocIdMap(ColumnIndexContainer offsetColumnIndexContainer) {
+    long start = System.currentTimeMillis();
+    final DataFileReader reader = offsetColumnIndexContainer.getForwardIndex();
+    final Dictionary dictionary = offsetColumnIndexContainer.getDictionary();
+    Map<Long, Integer> kafkaOffsetToDocIdMap = new HashMap<>();
+    long minOffset = Long.MAX_VALUE;
+    long maxOffset = 0;
+    if (reader instanceof BaseSingleColumnSingleValueReader) {
+      BaseSingleColumnSingleValueReader scsvReader = (BaseSingleColumnSingleValueReader) reader;
+      for (int docId = 0; docId < _totalDoc; docId++) {
+        final Long offset;
+        if (dictionary == null) {
+          offset = scsvReader.getLong(docId);
+        } else {
+          offset = (Long) dictionary.get(scsvReader.getInt(docId));
+        }
+        if (offset == null) {
+          LOGGER.error("kafka offset is null at docID {}", docId);
+        } else {
+          minOffset = Math.min(offset, minOffset);
+          maxOffset = Math.max(offset, maxOffset);
+          kafkaOffsetToDocIdMap.put(offset, docId);
+        }
+      }
+      _minSourceOffset = minOffset;
+      int size = Math.toIntExact(maxOffset - minOffset + 1);
+      _sourceOffsetToDocIdArray = new int[size];
+      for (int i = 0; i < size; i++) {
+        _sourceOffsetToDocIdArray[i] = kafkaOffsetToDocIdMap.getOrDefault(i + minOffset,
+            DEFAULT_DOC_ID_FOR_MISSING_ENTRY);
+      }
+    } else {
+      throw new RuntimeException("unexpected forward reader type for kafka offset column " + reader.getClass());
+    }
+    LOGGER.info("built offset to DocId map for segment {} with {} documents in {} ms", _segmentName, _totalDoc, System.currentTimeMillis() - start);
+  }
+
+  public static ImmutableUpsertSegmentImpl copyOf(ImmutableSegmentImpl immutableSegment) {
+
+    return new ImmutableUpsertSegmentImpl(immutableSegment._segmentDirectory, immutableSegment._segmentMetadata,
+        immutableSegment._indexContainerMap, immutableSegment._starTreeIndexContainer);
+  }
+
+  @Override
+  public void updateVirtualColumn(Iterable<UpdateLogEntry> logEntries) {
+    for (UpdateLogEntry logEntry: logEntries) {
+      boolean updated = false;
+      int docId = getDocIdFromSourceOffset(logEntry.getOffset());
+      for (VirtualColumnLongValueReaderWriter readerWriter : _virtualColumnsReaderWriter) {
+        updated = readerWriter.update(docId, logEntry.getValue(), logEntry.getType()) || updated;
+      }
+      if (updated) {
+        _upsertWaterMarkManager.processMessage(_tableNameWithType, _segmentName, logEntry);
+      }
+    }
+  }
+
+  @Override
+  public String getVirtualColumnInfo(long offset) {
+    int docId = getDocIdFromSourceOffset(offset);
+    StringBuilder result = new StringBuilder("matched: ");
+    for (VirtualColumnLongValueReaderWriter readerWriter : _virtualColumnsReaderWriter) {
+      result.append(readerWriter.getInt(docId)).append("; ");
+    }
+    return result.toString();
+  }
+
+  /**
+   * this method will fetch all updates from update logs in local disk and apply those updates to the current virtual columns
+   * it traverses through all records in the current segment and match existing updates log to its kafka offsets
+   * @throws IOException
+   */
+  @Override
+  public void initVirtualColumn() throws IOException {
+    long start = System.currentTimeMillis();
+    final UpdateLogEntrySet updateLogEntries = _updateLogStorageProvider.getAllMessages(_tableNameWithType, _segmentName);
+    LOGGER.info("load {} update log entry from update log storage provider for segment {} in {} ms",
+        updateLogEntries.size(), _segmentName, System.currentTimeMillis() - start);
+
+    start = System.currentTimeMillis();
+    final long maxOffset = _totalDoc + _minSourceOffset;
+    int unmatchedLogEntryCount = 0;
+    try {
+      Map<Integer, Long> partitionToHighestWatermark = new HashMap<>();
+      int readerWriteCount = _virtualColumnsReaderWriter.size();
+      for (UpdateLogEntry logEntry: updateLogEntries) {
+        final int partition = logEntry.getPartition();
+        final long offset = logEntry.getOffset();
+        if (offset >= _minSourceOffset && offset < maxOffset) {
+          final int docId = _sourceOffsetToDocIdArray[Math.toIntExact(offset - _minSourceOffset)];
+          if (docId != DEFAULT_DOC_ID_FOR_MISSING_ENTRY) {
+            // use traditional for loop over int instead of foreach to give hints for JIT to do loop unroll in byte code
+            for (int i = 0; i < readerWriteCount; i++) {
+              _virtualColumnsReaderWriter.get(i).update(docId, logEntry.getValue(), logEntry.getType());
+            }
+            if (logEntry.getValue() > partitionToHighestWatermark.getOrDefault(partition, -1L)) {
+              partitionToHighestWatermark.put(partition, logEntry.getValue());
+            }
+          } else {
+            LOGGER.error("segment {} got in-range update log at offset {} but no matching docId", _segmentName, offset);
+          }
+        } else {
+          unmatchedLogEntryCount++;
+        }
+      }
+      if (unmatchedLogEntryCount > 0) {
+        LOGGER.info("segment {} encountered {} update logs that are outside of its range", _segmentName,
+            unmatchedLogEntryCount);
+      }
+      partitionToHighestWatermark.forEach((partition, value) ->
+          _upsertWaterMarkManager.processVersionUpdate(_tableNameWithType, partition, value));
+
+    } catch (Exception e) {
+      LOGGER.error("failed to load the offset with thread pool");
+      Utils.rethrowException(e);
+    }
+    LOGGER.info("populated all log entries to virtual columns for current immutable segment {} in {} ms",
+        _segmentName, System.currentTimeMillis() - start);
+  }
+
+  /**
+   * given a offset from source kafka topic, return the docId associated with it
+   * throw exception if there is no docId for the associated kafka offset (because kafka offset might not be continuous)
+   * @param offset offset in the source topic
+   * @return the corresponding docId
+   */
+  private int getDocIdFromSourceOffset(long offset) throws RuntimeException {
+    if (offset < _minSourceOffset || offset - _minSourceOffset >= _sourceOffsetToDocIdArray.length) {
+      LOGGER.error("offset {} is outside range for current segment {} start offset {} size {}",
+          offset, _segmentName, _minSourceOffset, _sourceOffsetToDocIdArray.length);
+      throw new RuntimeException("offset outside range");
+    } else {
+      int position = Math.toIntExact(offset - _minSourceOffset);
+      if (_sourceOffsetToDocIdArray[position] == DEFAULT_DOC_ID_FOR_MISSING_ENTRY) {
+        LOGGER.error("no docId associated with offset {} for segment {}", offset, _segmentName);
+        throw new RuntimeException("docId not found");
+      } else {
+        return _sourceOffsetToDocIdArray[position];
+      }
+    }
+
+  }
+}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/VirtualColumnContext.java b/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableAppendSegmentImpl.java
similarity index 56%
copy from pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/VirtualColumnContext.java
copy to pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableAppendSegmentImpl.java
index bb32320..1c9effc 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/VirtualColumnContext.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableAppendSegmentImpl.java
@@ -16,29 +16,22 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.core.segment.virtualcolumn;
+package org.apache.pinot.core.indexsegment.mutable;
 
-import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.core.realtime.impl.RealtimeSegmentConfig;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
+public class MutableAppendSegmentImpl extends MutableSegmentImpl {
+  private static final Logger LOGGER = LoggerFactory.getLogger(MutableUpsertSegmentImpl.class);
 
-/**
- * Miscellaneous context information about the virtual column.
- * It will be used to build various components (dictionary, reader, etc) in the virtual column provider.
- */
-public class VirtualColumnContext {
-  private FieldSpec _fieldSpec;
-  private int _totalDocCount;
-
-  public VirtualColumnContext(FieldSpec fieldSpec, int totalDocCount) {
-    _fieldSpec = fieldSpec;
-    _totalDocCount = totalDocCount;
-  }
-
-  public FieldSpec getFieldSpec() {
-    return _fieldSpec;
+  public MutableAppendSegmentImpl(RealtimeSegmentConfig config) {
+    super(config);
   }
 
-  public int getTotalDocCount() {
-    return _totalDocCount;
+  @Override
+  protected void postProcessRecords(GenericRow row, int docId) {
+    // nothing
   }
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImpl.java b/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImpl.java
index f800d95..b80ac64 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImpl.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImpl.java
@@ -31,6 +31,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import javax.annotation.Nullable;
+
 import org.apache.pinot.common.config.SegmentPartitionConfig;
 import org.apache.pinot.common.segment.SegmentMetadata;
 import org.apache.pinot.core.indexsegment.IndexSegmentUtils;
@@ -56,6 +57,7 @@ import org.apache.pinot.core.segment.index.data.source.ColumnDataSource;
 import org.apache.pinot.core.segment.index.readers.BloomFilterReader;
 import org.apache.pinot.core.segment.index.readers.InvertedIndexReader;
 import org.apache.pinot.core.segment.index.readers.NullValueVectorReader;
+import org.apache.pinot.core.segment.index.readers.Dictionary;
 import org.apache.pinot.core.segment.virtualcolumn.VirtualColumnContext;
 import org.apache.pinot.core.segment.virtualcolumn.VirtualColumnProvider;
 import org.apache.pinot.core.segment.virtualcolumn.VirtualColumnProviderFactory;
@@ -74,7 +76,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 
-public class MutableSegmentImpl implements MutableSegment {
+public abstract class MutableSegmentImpl implements MutableSegment {
   // For multi-valued column, forward-index.
   // Maximum number of multi-values per row. We assert on this.
   private static final int MAX_MULTI_VALUES_PER_ROW = 1000;
@@ -86,11 +88,12 @@ public class MutableSegmentImpl implements MutableSegment {
   private static final int NODICT_VARIABLE_WIDTH_ESTIMATED_AVERAGE_VALUE_LENGTH_DEFAULT = 100;
   private static final int NODICT_VARIABLE_WIDTH_ESTIMATED_NUMBER_OF_VALUES_DEFAULT = 100_000;
 
-  private final Logger _logger;
+  protected final Logger _logger;
   private final long _startTimeMillis = System.currentTimeMillis();
 
-  private final String _segmentName;
-  private final Schema _schema;
+  protected final String _tableName;
+  protected final String _segmentName;
+  protected final Schema _schema;
   private final int _capacity;
   private final SegmentMetadata _segmentMetadata;
   private final boolean _offHeap;
@@ -108,18 +111,23 @@ public class MutableSegmentImpl implements MutableSegment {
   private final IdMap<FixedIntArray> _recordIdMap;
   private boolean _aggregateMetrics;
 
-  private volatile int _numDocsIndexed = 0;
+  protected volatile int _numDocsIndexed = 0;
 
   // to compute the rolling interval
-  private volatile long _minTime = Long.MAX_VALUE;
-  private volatile long _maxTime = Long.MIN_VALUE;
-  private final int _numKeyColumns;
+  protected volatile long _minTime = Long.MAX_VALUE;
+  protected volatile long _maxTime = Long.MIN_VALUE;
+  protected final int _numKeyColumns;
 
   // Cache the physical (non-virtual) field specs
   private final Collection<FieldSpec> _physicalFieldSpecs;
   private final Collection<DimensionFieldSpec> _physicalDimensionFieldSpecs;
   private final Collection<MetricFieldSpec> _physicalMetricFieldSpecs;
 
+  // Cache some virtual columns
+  protected final Map<String, VirtualColumnProvider> _virtualColumnProviderMap = new HashMap<>();
+  protected final Map<String, Dictionary> _virtualColumnDictionary = new HashMap<>();
+  protected final Map<String, DataFileReader> _virtualColumnIndexReader = new HashMap<>();
+
   // default message metadata
   private volatile long _lastIndexedTimeMs = Long.MIN_VALUE;
   private volatile long _latestIngestionTimeMs = Long.MIN_VALUE;
@@ -127,6 +135,7 @@ public class MutableSegmentImpl implements MutableSegment {
   private RealtimeLuceneReaders _realtimeLuceneReaders;
 
   public MutableSegmentImpl(RealtimeSegmentConfig config) {
+    _tableName = config.getTableName();
     _segmentName = config.getSegmentName();
     _schema = config.getSchema();
     _capacity = config.getCapacity();
@@ -161,6 +170,7 @@ public class MutableSegmentImpl implements MutableSegment {
 
     Collection<FieldSpec> allFieldSpecs = _schema.getAllFieldSpecs();
     List<FieldSpec> physicalFieldSpecs = new ArrayList<>(allFieldSpecs.size());
+    List<FieldSpec> virtualFieldSpecs = new ArrayList<>(allFieldSpecs.size());
     List<DimensionFieldSpec> physicalDimensionFieldSpecs = new ArrayList<>(_schema.getDimensionNames().size());
     List<MetricFieldSpec> physicalMetricFieldSpecs = new ArrayList<>(_schema.getMetricNames().size());
 
@@ -174,6 +184,9 @@ public class MutableSegmentImpl implements MutableSegment {
         } else if (fieldType == FieldSpec.FieldType.METRIC) {
           physicalMetricFieldSpecs.add((MetricFieldSpec) fieldSpec);
         }
+      } else {
+        virtualFieldSpecs.add(fieldSpec);
+
       }
     }
     _physicalFieldSpecs = Collections.unmodifiableCollection(physicalFieldSpecs);
@@ -191,6 +204,7 @@ public class MutableSegmentImpl implements MutableSegment {
 
     int avgNumMultiValues = config.getAvgNumMultiValues();
 
+
     // Initialize for each column
     for (FieldSpec fieldSpec : _physicalFieldSpecs) {
       String column = fieldSpec.getName();
@@ -269,6 +283,7 @@ public class MutableSegmentImpl implements MutableSegment {
 
       _indexReaderWriterMap.put(column, indexReaderWriter);
 
+
       if (invertedIndexColumns.contains(column)) {
         _invertedIndexMap.put(column, new RealtimeInvertedIndexReader());
       }
@@ -287,6 +302,21 @@ public class MutableSegmentImpl implements MutableSegment {
       }
     }
 
+    for (FieldSpec fieldSpec : _physicalFieldSpecs) {
+      String column = fieldSpec.getName();
+      VirtualColumnContext virtualColumnContext = new VirtualColumnContext(fieldSpec, _capacity, true);
+      final VirtualColumnProvider provider =
+              VirtualColumnProviderFactory.buildProvider(fieldSpec.getVirtualColumnProvider());
+      if (provider == null) {
+        throw new RuntimeException(String.format("failed to create virtual segment provider for field %s", fieldSpec.getName()));
+      }
+      DataFileReader reader = provider.buildReader(virtualColumnContext);
+      Dictionary dictionary = provider.buildDictionary(virtualColumnContext);
+      _virtualColumnProviderMap.put(column, provider);
+      _virtualColumnIndexReader.put(column, reader);
+      _virtualColumnDictionary.put(column, dictionary);
+    }
+
     if (_realtimeLuceneReaders != null) {
       // add the realtime lucene index readers to the global queue for refresh task to pick up
       RealtimeLuceneIndexRefreshState realtimeLuceneIndexRefreshState = RealtimeLuceneIndexRefreshState.getInstance();
@@ -347,6 +377,7 @@ public class MutableSegmentImpl implements MutableSegment {
 
       // Update number of document indexed at last to make the latest record queryable
       canTakeMore = _numDocsIndexed++ < _capacity;
+      postProcessRecords(row, docId);
     } else {
       Preconditions
           .checkState(_aggregateMetrics, "Invalid document-id during indexing: " + docId + " expected: " + numDocs);
@@ -491,6 +522,8 @@ public class MutableSegmentImpl implements MutableSegment {
     }
   }
 
+  protected abstract void postProcessRecords(GenericRow row, int docId);
+
   private boolean aggregateMetrics(GenericRow row, int docId) {
     for (MetricFieldSpec metricFieldSpec : _physicalMetricFieldSpecs) {
       String column = metricFieldSpec.getName();
@@ -556,9 +589,9 @@ public class MutableSegmentImpl implements MutableSegment {
   public ColumnDataSource getDataSource(String columnName) {
     FieldSpec fieldSpec = _schema.getFieldSpecFor(columnName);
     if (fieldSpec.isVirtualColumn()) {
-      VirtualColumnContext virtualColumnContext = new VirtualColumnContext(fieldSpec, _numDocsIndexed);
-      VirtualColumnProvider virtualColumnProvider =
-          VirtualColumnProviderFactory.buildProvider(_schema.getFieldSpecFor(columnName).getVirtualColumnProvider());
+      // FIXME
+      VirtualColumnContext virtualColumnContext = new VirtualColumnContext(fieldSpec, _numDocsIndexed, true);
+      VirtualColumnProvider virtualColumnProvider = _virtualColumnProviderMap.get(columnName);
       return new ColumnDataSource(virtualColumnProvider.buildColumnIndexContainer(virtualColumnContext),
           virtualColumnProvider.buildMetadata(virtualColumnContext));
     } else {
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableUpsertSegmentImpl.java b/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableUpsertSegmentImpl.java
new file mode 100644
index 0000000..a6e7728
--- /dev/null
+++ b/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableUpsertSegmentImpl.java
@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.indexsegment.mutable;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.core.indexsegment.UpsertSegment;
+import org.apache.pinot.core.io.reader.DataFileReader;
+import org.apache.pinot.core.realtime.impl.RealtimeSegmentConfig;
+import org.apache.pinot.core.segment.updater.UpsertWaterMarkManager;
+import org.apache.pinot.core.segment.virtualcolumn.mutable.VirtualColumnLongValueReaderWriter;
+import org.apache.pinot.grigio.common.messages.LogEventType;
+import org.apache.pinot.grigio.common.storageProvider.UpdateLogEntry;
+import org.apache.pinot.grigio.common.storageProvider.UpdateLogEntrySet;
+import org.apache.pinot.grigio.common.storageProvider.UpdateLogStorageProvider;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class MutableUpsertSegmentImpl extends MutableSegmentImpl implements UpsertSegment {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(MutableUpsertSegmentImpl.class);
+  private final UpsertWaterMarkManager upsertWaterMarkManager;
+
+  private final String _kafkaOffsetColumnName;
+
+  private final List<VirtualColumnLongValueReaderWriter> _mutableSegmentReaderWriters = new ArrayList<>();
+  // use map for mapping between kafka offset and docId because we at-most have 1 mutable segment per consumer
+  // will use more memory, but we can update this later
+  private final Map<Long, Integer> _sourceOffsetToDocId = new ConcurrentHashMap<>();
+  // to store the update event that arrive before my current record
+  // TODO remove this in the later version of design if necessary
+  private final Map<Long, UpdateLogEntry> _unmatchedInsertRecords = new ConcurrentHashMap<>();
+  private final Map<Long, UpdateLogEntry> _unmatchedDeleteRecords = new ConcurrentHashMap<>();
+
+  public MutableUpsertSegmentImpl(RealtimeSegmentConfig config) {
+    super(config);
+    _kafkaOffsetColumnName = _schema.getOffsetKey();
+    Preconditions.checkState(_schema.isTableForUpsert(), "table should be upsert");
+    for (Map.Entry<String, DataFileReader> entry: _virtualColumnIndexReader.entrySet()) {
+      String column = entry.getKey();
+      DataFileReader reader = entry.getValue();
+      if (reader instanceof VirtualColumnLongValueReaderWriter) {
+        _logger.info("adding virtual column {} to updatable reader writer list", column);
+        _mutableSegmentReaderWriters.add((VirtualColumnLongValueReaderWriter) reader);
+      }
+    }
+    upsertWaterMarkManager = UpsertWaterMarkManager.getInstance();
+    LOGGER.info("starting upsert segment with {} reader writer", _mutableSegmentReaderWriters.size());
+  }
+
+  @Override
+  public synchronized void updateVirtualColumn(Iterable<UpdateLogEntry> logEntries) {
+    for (UpdateLogEntry logEntry: logEntries) {
+      boolean updated = false;
+      boolean offsetFound = false;
+      Integer docId = _sourceOffsetToDocId.get(logEntry.getOffset());
+      if (docId != null) {
+        offsetFound = true;
+        for (VirtualColumnLongValueReaderWriter readerWriter : _mutableSegmentReaderWriters) {
+          updated = readerWriter.update(docId, logEntry.getValue(), logEntry.getType()) || updated;
+        }
+        if (updated) {
+          // only update high water mark if it indeed updated something
+          upsertWaterMarkManager.processMessage(_tableName, _segmentName, logEntry);
+        }
+      }
+      if (!offsetFound) {
+        putEntryToUnmatchMap(logEntry);
+      }
+    }
+  }
+
+  @Override
+  public String getVirtualColumnInfo(long offset) {
+    Integer docId = _sourceOffsetToDocId.get(offset);
+    StringBuilder result = new StringBuilder("matched: ");
+    if (docId == null) {
+      result = new StringBuilder("no doc id found ");
+    } else {
+      for (VirtualColumnLongValueReaderWriter readerWriter : _mutableSegmentReaderWriters) {
+        result.append(readerWriter.getLong(docId)).append("; ");
+      }
+    }
+    if (_unmatchedInsertRecords.containsKey(offset)) {
+      result.append(" unmatched insert: ").append(_unmatchedInsertRecords.get(offset).getValue());
+    }
+    if (_unmatchedDeleteRecords.containsKey(offset)) {
+      result.append(" unmatched delete: ").append(_unmatchedDeleteRecords.get(offset).getValue());
+    }
+    return result.toString();
+  }
+
+  @Override
+  protected synchronized void postProcessRecords(GenericRow row, int docId) {
+    final Long offset = (Long) row.getValue(_kafkaOffsetColumnName);
+    for (VirtualColumnLongValueReaderWriter readerWriter: _mutableSegmentReaderWriters) {
+      readerWriter.addNewRecord(docId);
+    }
+    _sourceOffsetToDocId.put(offset, docId);
+    checkForOutstandingRecords(_unmatchedDeleteRecords, offset, docId);
+    checkForOutstandingRecords(_unmatchedInsertRecords, offset, docId);
+  }
+
+  @Override
+  public void initVirtualColumn() throws IOException {
+    Preconditions.checkState(_numDocsIndexed == 0, "should init virtual column before ingestion");
+    UpdateLogEntrySet updateLogEntries = UpdateLogStorageProvider.getInstance().getAllMessages(_tableName, _segmentName);
+    LOGGER.info("got {} update log entries for current segment {}", updateLogEntries.size(), _segmentName);
+    // some physical data might have been ingested when we init virtual column, we will go through the normal update
+    // flow to ensure we wont miss records
+    updateVirtualColumn(updateLogEntries);
+  }
+
+  private void checkForOutstandingRecords(Map<Long, UpdateLogEntry> unmatchRecordsMap, Long offset, int docId) {
+    UpdateLogEntry unmatchedEntry = unmatchRecordsMap.remove(offset);
+    if (unmatchedEntry != null) {
+      boolean updated = false;
+      for (VirtualColumnLongValueReaderWriter readerWriter: _mutableSegmentReaderWriters) {
+        updated = readerWriter.update(docId, unmatchedEntry.getValue(), unmatchedEntry.getType()) || updated;
+      }
+      if (updated) {
+        upsertWaterMarkManager.processMessage(_tableName, _segmentName, unmatchedEntry);
+      }
+    }
+  }
+
+  private void putEntryToUnmatchMap(UpdateLogEntry logEntry) {
+    final Map<Long, UpdateLogEntry> unmatchedRecords;
+    if (logEntry.getType() == LogEventType.INSERT) {
+      unmatchedRecords = _unmatchedInsertRecords;
+    } else {
+      unmatchedRecords = _unmatchedDeleteRecords;
+    }
+    unmatchedRecords.put(logEntry.getOffset(), logEntry);
+  }
+
+}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/realtime/impl/RealtimeSegmentConfig.java b/pinot-core/src/main/java/org/apache/pinot/core/realtime/impl/RealtimeSegmentConfig.java
index 9aef2a2..827b0f7 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/realtime/impl/RealtimeSegmentConfig.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/realtime/impl/RealtimeSegmentConfig.java
@@ -27,6 +27,7 @@ import org.apache.pinot.core.io.readerwriter.PinotDataBufferMemoryManager;
 
 
 public class RealtimeSegmentConfig {
+  private final String _tableName;
   private final String _segmentName;
   private final String _streamName;
   private final Schema _schema;
@@ -45,12 +46,13 @@ public class RealtimeSegmentConfig {
   private final boolean _nullHandlingEnabled;
   private final String _consumerDir;
 
-  private RealtimeSegmentConfig(String segmentName, String streamName, Schema schema, int capacity,
+  private RealtimeSegmentConfig(String tableName, String segmentName, String streamName, Schema schema, int capacity,
       int avgNumMultiValues, Set<String> noDictionaryColumns, Set<String> varLengthDictionaryColumns,
       Set<String> invertedIndexColumns, Set<String> textIndexColumns, RealtimeSegmentZKMetadata realtimeSegmentZKMetadata,
       boolean offHeap, PinotDataBufferMemoryManager memoryManager, RealtimeSegmentStatsHistory statsHistory,
       SegmentPartitionConfig segmentPartitionConfig, boolean aggregateMetrics, boolean nullHandlingEnabled,
       String consumerDir) {
+    _tableName = tableName;
     _segmentName = segmentName;
     _streamName = streamName;
     _schema = schema;
@@ -70,6 +72,10 @@ public class RealtimeSegmentConfig {
     _consumerDir = consumerDir;
   }
 
+  public String getTableName() {
+    return _tableName;
+  }
+
   public String getSegmentName() {
     return _segmentName;
   }
@@ -144,6 +150,7 @@ public class RealtimeSegmentConfig {
   }
 
   public static class Builder {
+    private String _tableName;
     private String _segmentName;
     private String _streamName;
     private Schema _schema;
@@ -165,6 +172,11 @@ public class RealtimeSegmentConfig {
     public Builder() {
     }
 
+    public Builder setTableName(String tableName) {
+      _tableName = tableName;
+      return this;
+    }
+
     public Builder setSegmentName(String segmentName) {
       _segmentName = segmentName;
       return this;
@@ -259,7 +271,7 @@ public class RealtimeSegmentConfig {
     }
 
     public RealtimeSegmentConfig build() {
-      return new RealtimeSegmentConfig(_segmentName, _streamName, _schema, _capacity, _avgNumMultiValues,
+      return new RealtimeSegmentConfig(_tableName, _segmentName, _streamName, _schema, _capacity, _avgNumMultiValues,
           _noDictionaryColumns, _varLengthDictionaryColumns, _invertedIndexColumns, _textIndexColumns,
           _realtimeSegmentZKMetadata, _offHeap, _memoryManager, _statsHistory, _segmentPartitionConfig,
           _aggregateMetrics, _nullHandlingEnabled, _consumerDir);
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/index/SegmentMetadataImpl.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/index/SegmentMetadataImpl.java
index 7a3dad3..43d28e5 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/segment/index/SegmentMetadataImpl.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/index/SegmentMetadataImpl.java
@@ -22,6 +22,15 @@ import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.node.ArrayNode;
 import com.fasterxml.jackson.databind.node.ObjectNode;
 import com.google.common.base.Preconditions;
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.commons.configuration.PropertiesConfiguration;
+import org.apache.commons.lang.StringEscapeUtils;
+import org.joda.time.Duration;
+import org.joda.time.Interval;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
 import java.io.DataInputStream;
 import java.io.File;
 import java.io.FileInputStream;
@@ -39,10 +48,6 @@ import java.util.Map;
 import java.util.Set;
 import java.util.TimeZone;
 import java.util.concurrent.TimeUnit;
-import javax.annotation.Nullable;
-import org.apache.commons.configuration.ConfigurationException;
-import org.apache.commons.configuration.PropertiesConfiguration;
-import org.apache.commons.lang.StringEscapeUtils;
 import org.apache.pinot.spi.data.MetricFieldSpec;
 import org.apache.pinot.spi.data.Schema;
 import org.apache.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
@@ -58,10 +63,6 @@ import org.apache.pinot.core.startree.v2.StarTreeV2Constants;
 import org.apache.pinot.core.startree.v2.StarTreeV2Metadata;
 import org.apache.pinot.startree.hll.HllConstants;
 import org.joda.time.DateTimeZone;
-import org.joda.time.Duration;
-import org.joda.time.Interval;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import static org.apache.pinot.core.segment.creator.impl.V1Constants.MetadataKeys.Segment.*;
 import static org.apache.pinot.core.segment.creator.impl.V1Constants.MetadataKeys.StarTree.*;
@@ -108,6 +109,15 @@ public class SegmentMetadataImpl implements SegmentMetadata {
    */
   public SegmentMetadataImpl(File indexDir)
       throws IOException {
+    this(indexDir, null);
+  }
+
+   /**
+   * For segments on disk and segment is upsert, will use the sechema hint to init virtual column
+   * <p>Index directory passed in should be top level segment directory.
+   * <p>If segment metadata file exists in multiple segment version, load the one in highest segment version.
+   */
+  public SegmentMetadataImpl(File indexDir, Schema schemaHint) throws IOException {
     _indexDir = indexDir;
     PropertiesConfiguration segmentMetadataPropertiesConfiguration = getPropertiesConfiguration(indexDir);
     _columnMetadataMap = new HashMap<>();
@@ -115,6 +125,9 @@ public class SegmentMetadataImpl implements SegmentMetadata {
     _schema = new Schema();
 
     init(segmentMetadataPropertiesConfiguration);
+    // adding necessary schema hints for upsert table
+    _schema.withSchemaHint(schemaHint);
+
     File creationMetaFile = SegmentDirectoryPaths.findCreationMetaFile(indexDir);
     if (creationMetaFile != null) {
       loadCreationMeta(creationMetaFile);
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/VirtualColumnContext.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/SegmentDeletionListener.java
similarity index 55%
copy from pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/VirtualColumnContext.java
copy to pinot-core/src/main/java/org/apache/pinot/core/segment/updater/SegmentDeletionListener.java
index bb32320..a29feb7 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/VirtualColumnContext.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/SegmentDeletionListener.java
@@ -16,29 +16,8 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.core.segment.virtualcolumn;
+package org.apache.pinot.core.segment.updater;
 
-import org.apache.pinot.spi.data.FieldSpec;
-
-
-/**
- * Miscellaneous context information about the virtual column.
- * It will be used to build various components (dictionary, reader, etc) in the virtual column provider.
- */
-public class VirtualColumnContext {
-  private FieldSpec _fieldSpec;
-  private int _totalDocCount;
-
-  public VirtualColumnContext(FieldSpec fieldSpec, int totalDocCount) {
-    _fieldSpec = fieldSpec;
-    _totalDocCount = totalDocCount;
-  }
-
-  public FieldSpec getFieldSpec() {
-    return _fieldSpec;
-  }
-
-  public int getTotalDocCount() {
-    return _totalDocCount;
-  }
+public interface SegmentDeletionListener {
+  void onSegmentDeletion(String tableNameWithType, String segmentName);
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdater.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdater.java
new file mode 100644
index 0000000..d0d8c0e
--- /dev/null
+++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdater.java
@@ -0,0 +1,349 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.segment.updater;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.Uninterruptibles;
+import io.netty.util.internal.ConcurrentSet;
+import org.apache.commons.configuration.Configuration;
+import org.apache.pinot.common.config.TableNameBuilder;
+import org.apache.pinot.common.utils.CommonConstants;
+import org.apache.pinot.common.utils.LLCSegmentName;
+import org.apache.pinot.core.data.manager.UpsertSegmentDataManager;
+import org.apache.pinot.grigio.common.messages.LogCoordinatorMessage;
+import org.apache.pinot.grigio.common.metrics.GrigioMeter;
+import org.apache.pinot.grigio.common.metrics.GrigioMetrics;
+import org.apache.pinot.grigio.common.metrics.GrigioTimer;
+import org.apache.pinot.grigio.common.rpcQueue.QueueConsumer;
+import org.apache.pinot.grigio.common.rpcQueue.QueueConsumerRecord;
+import org.apache.pinot.grigio.common.storageProvider.UpdateLogEntry;
+import org.apache.pinot.grigio.common.storageProvider.UpdateLogStorageProvider;
+import org.apache.pinot.grigio.common.storageProvider.retentionManager.UpdateLogRetentionManager;
+import org.apache.pinot.grigio.common.utils.CommonUtils;
+import org.apache.pinot.grigio.servers.SegmentUpdaterProvider;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.ThreadSafe;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * class to perform fetching updates for upsert related information from kafka to local machine and fill in
+ * virtual columns. It should be started after all segments are loaded in current pinot server.
+ */
+@ThreadSafe
+public class SegmentUpdater implements SegmentDeletionListener {
+  private static final Logger LOGGER = LoggerFactory.getLogger(SegmentUpdater.class);
+
+  private static volatile SegmentUpdater _instance = null;
+
+  private static final long NO_MESSAGE_SLEEP_MS = 100;
+  private static final long SHUTDOWN_WAIT_MS = 2000;
+  private static final long LOGGER_TIME_GAP_MS = 5000;
+
+  private final Configuration _conf;
+  private final int _updateSleepMs;
+  private final String _topicPrefix;
+  private final ExecutorService _ingestionExecutorService;
+  private final QueueConsumer _consumer;
+  private final Map<String, Map<String, Set<UpsertSegmentDataManager>>> _tableSegmentMap = new ConcurrentHashMap<>();
+  private final Map<String, Map<Integer, Long>> _tablePartitionCreationTime = new ConcurrentHashMap<>();
+  private final UpdateLogStorageProvider _updateLogStorageProvider;
+  private final UpdateLogRetentionManager _retentionManager;
+  private final GrigioMetrics _metrics;
+
+  private volatile boolean isStarted = true;
+
+  public SegmentUpdater(Configuration conf, SegmentUpdaterProvider provider, UpdateLogRetentionManager retentionManager,
+                        GrigioMetrics metrics) {
+    _conf = conf;
+    _metrics = metrics;
+    _retentionManager = retentionManager;
+    _topicPrefix = conf.getString(SegmentUpdaterConfig.INPUT_TOPIC_PREFIX);
+    _updateSleepMs = conf.getInt(SegmentUpdaterConfig.SEGMENT_UDPATE_SLEEP_MS,
+        SegmentUpdaterConfig.SEGMENT_UDPATE_SLEEP_MS_DEFAULT);
+    UpsertWaterMarkManager.init(metrics);
+    _consumer = provider.getConsumer();
+    _ingestionExecutorService = Executors.newFixedThreadPool(1);
+    _updateLogStorageProvider = UpdateLogStorageProvider.getInstance();
+    _instance = this;
+  }
+
+  public static SegmentUpdater getInstance() {
+    Preconditions.checkState(_instance != null, "there is no instance for segment updater");
+    return _instance;
+  }
+
+
+  public void start() {
+    String listOfTables = Joiner.on(",").join(_tableSegmentMap.keySet());
+    LOGGER.info("starting segment updater main loop with the following table in server: {}", listOfTables);
+    _ingestionExecutorService.submit(this::updateLoop);
+  }
+
+  public void shutdown() {
+    LOGGER.info("closing the segment updater");
+    isStarted = false;
+    _ingestionExecutorService.shutdown();
+    try {
+      _ingestionExecutorService.awaitTermination(SHUTDOWN_WAIT_MS, TimeUnit.MILLISECONDS);
+    } catch (InterruptedException e) {
+      LOGGER.error("failed to wait for shutdown", e);
+    }
+    _ingestionExecutorService.shutdownNow();
+    try {
+      _updateLogStorageProvider.close();
+    } catch (IOException ex) {
+      LOGGER.error("failed to shutdown update log provider", ex);
+    }
+    LOGGER.info("finished shutdown of segment updater service");
+  }
+
+  /**
+   * this method will perform the following:
+   * 1. fetch a list of updates for pinot upsert from kafka consumer
+   * 2. organize the message by table/segment/List<UpdateLogEntry> map
+   * 3. call to save the update data in local file system
+   * 4. apply updates to appropriate data manager
+   * 5. commit consumer offset
+   */
+  private void updateLoop() {
+    try {
+      LOGGER.info("starting update loop");
+      long lastReportedTime = System.currentTimeMillis();
+      while (isStarted) {
+        long startTime = System.currentTimeMillis();
+        long loopStartTime = startTime;
+        final List<QueueConsumerRecord<String, LogCoordinatorMessage>> records = _consumer.getRequests(_updateSleepMs, TimeUnit.MILLISECONDS);
+        _metrics.addTimedValueMs(GrigioTimer.FETCH_MSG_FROM_CONSUMER_TIME, System.currentTimeMillis() - startTime);
+        int eventCount = records.size();
+        _metrics.addMeteredGlobalValue(GrigioMeter.MESSAGE_FETCH_PER_ROUND, eventCount);
+
+        final Map<String, TableUpdateLogs> tableSegmentToUpdateLogs = new HashMap<>();
+        // organize the update logs by {tableName: {segmentName: {list of updatelogs}}}
+        records.iterator().forEachRemaining(consumerRecord -> {
+          TableUpdateLogs tableUpdateLogs = tableSegmentToUpdateLogs.computeIfAbsent(
+              CommonUtils.getTableNameFromKafkaTopic(consumerRecord.getTopic(), _topicPrefix),
+              t -> new TableUpdateLogs());
+          tableUpdateLogs.addUpdateLogEntry(consumerRecord.getRecord(), consumerRecord.getPartition());
+        });
+
+        startTime = System.currentTimeMillis();
+        AtomicLong timeToStoreUpdateLogs = new AtomicLong(0);
+        for (Map.Entry<String, TableUpdateLogs> entry : tableSegmentToUpdateLogs.entrySet()) {
+          String tableName = TableNameBuilder.ensureTableNameWithType(entry.getKey(), CommonConstants.Helix.TableType.REALTIME);
+          int tableMessageCount = 0;
+          if (_tableSegmentMap.containsKey(tableName)) {
+            final Map<String, Set<UpsertSegmentDataManager>> segmentManagersMap = _tableSegmentMap.get(tableName);
+            final TableUpdateLogs segment2UpdateLogsMap = entry.getValue();
+            updateSegmentVirtualColumns(tableName, segmentManagersMap, segment2UpdateLogsMap, timeToStoreUpdateLogs);
+          } else {
+            LOGGER.warn("got messages for table {} not in this server", tableName);
+          }
+          _metrics.addMeteredTableValue(tableName, GrigioMeter.MESSAGE_FETCH_PER_ROUND, tableMessageCount);
+        }
+        _metrics.addTimedValueMs(GrigioTimer.UPDATE_LOCAL_LOG_FILE_TIME, timeToStoreUpdateLogs.get());
+        _metrics.addTimedValueMs(GrigioTimer.UPDATE_DATAMANAGER_TIME, System.currentTimeMillis() - startTime);
+        if (eventCount == 0) {
+          Uninterruptibles.sleepUninterruptibly(NO_MESSAGE_SLEEP_MS, TimeUnit.MILLISECONDS);
+        } else {
+          if (System.currentTimeMillis() - lastReportedTime > LOGGER_TIME_GAP_MS) {
+            lastReportedTime = System.currentTimeMillis();
+            LOGGER.info("processed {} messages in {} ms", eventCount, System.currentTimeMillis() - loopStartTime);
+            LOGGER.info("latest high water mark is {}", UpsertWaterMarkManager.getInstance().toString());
+          }
+          _consumer.ackOffset();
+          _metrics.addTimedValueMs(GrigioTimer.SEGMENT_UPDATER_LOOP_TIME, System.currentTimeMillis() - loopStartTime);
+        }
+      }
+    } catch (Exception ex) {
+      LOGGER.error("failed at segment updates", ex);
+    } finally {
+      LOGGER.info("exiting segment update loop");
+    }
+  }
+
+  private void storeUpdateLogs(String table, String segment, List<UpdateLogEntry> messages,
+                               AtomicLong timeToStoreUpdateLogs) throws IOException {
+    long startTime = System.currentTimeMillis();
+    _updateLogStorageProvider.addDataToFile(table, segment, messages);
+    timeToStoreUpdateLogs.addAndGet(System.currentTimeMillis() - startTime);
+  }
+
+  /**
+   * Update the virtual columns of affected segments of a table.
+   */
+  private void updateSegmentVirtualColumns(String tableName, Map<String, Set<UpsertSegmentDataManager>> segmentManagersMap,
+                                           TableUpdateLogs segment2UpdateLogsMap, AtomicLong timeToStoreUpdateLogs) throws IOException{
+    for (Map.Entry<String, List<UpdateLogEntry>> segmentEntry : segment2UpdateLogsMap.getSegments2UpdateLog().entrySet()) {
+      final String segmentNameStr = segmentEntry.getKey();
+      updateVirtualColumn(tableName, segmentNameStr,
+          segmentManagersMap.computeIfAbsent(segmentNameStr, sn -> new ConcurrentSet<>()),
+          segment2UpdateLogsMap.get(segmentNameStr), timeToStoreUpdateLogs);
+    }
+  }
+
+  /**
+   * in pinot server, there could be multiple segment data managers per table/segment pair during pinot switch a segment
+   * from consuming to online (mutable segment to immutable segment). In most of cases we expect only one segment manager
+   * in this set of UpsertSegmentDataManager
+   */
+  private void updateVirtualColumn(String table, String segment, Set<UpsertSegmentDataManager> segmentDataManagers,
+                                   List<UpdateLogEntry> messages, AtomicLong timeToStoreUpdateLogs) throws IOException {
+    LOGGER.debug("updating segment {} with {} results for {} data managers", segment, messages.size(),
+        segmentDataManagers.size());
+    if (segmentDataManagers.size() > 0 || _retentionManager.getRetentionManagerForTable(table).shouldIngestForSegment(segment)) {
+      storeUpdateLogs(table, segment, messages, timeToStoreUpdateLogs);
+    }
+    try {
+      for (UpsertSegmentDataManager dataManager: segmentDataManagers) {
+        dataManager.updateVirtualColumns(messages);
+      }
+    } catch (Exception ex) {
+      LOGGER.error("failed to update virtual column for key ", ex);
+    }
+  }
+
+  /**
+   * called when we create a new segment data manager, associate this data manager with the given table/segment info
+   * @param tableNameWithType
+   * @param segmentName
+   * @param dataManager the data manager for the current given table/segment combination
+   */
+  public synchronized void addSegmentDataManager(String tableNameWithType, LLCSegmentName segmentName, UpsertSegmentDataManager dataManager) {
+    // TODO get partition assignment from
+    LOGGER.info("segment updater adding table {} segment {}", tableNameWithType, segmentName.getSegmentName());
+    if (!_tableSegmentMap.containsKey(tableNameWithType)) {
+      synchronized (_tableSegmentMap) {
+        _tableSegmentMap.put(tableNameWithType, new ConcurrentHashMap<>());
+      }
+      LOGGER.info("adding table {} to segment updater consumer", tableNameWithType);
+      handleNewTableInServer(tableNameWithType);
+    }
+    _tableSegmentMap.get(tableNameWithType).computeIfAbsent(segmentName.getSegmentName(), sn -> new HashSet<>()).add(dataManager);
+    synchronized (_tablePartitionCreationTime) {
+      long creationTime = _tablePartitionCreationTime.computeIfAbsent(tableNameWithType, t -> new ConcurrentHashMap<>())
+          .computeIfAbsent(segmentName.getPartitionId(), p -> segmentName.getCreationTimeStamp());
+      _tablePartitionCreationTime.get(tableNameWithType)
+          .put(segmentName.getPartitionId(), Long.max(creationTime, segmentName.getCreationTimeStamp()));
+    }
+  }
+
+  public synchronized void removeSegmentDataManager(String tableNameWithType, String segmentName, UpsertSegmentDataManager toDeleteManager) {
+    LOGGER.info("segment updater removing table {} segment {}", tableNameWithType, segmentName);
+    Map<String, Set<UpsertSegmentDataManager>> segmentMap = _tableSegmentMap.get(tableNameWithType);
+    if (segmentMap != null) {
+      Set<UpsertSegmentDataManager> segmentDataManagers = segmentMap.get(segmentName);
+      if (segmentDataManagers != null) {
+        segmentDataManagers.remove(toDeleteManager);
+        if (segmentDataManagers.size() == 0) {
+          segmentMap.remove(segmentName);
+        }
+      }
+    }
+  }
+
+  /**
+   * handle how to read update logs we need to do for adding a new pinot table
+   * need to do the following:
+   * subscribe to table update log kafka topics
+   * @param tableNameWithType the name of the table without
+   */
+  private void handleNewTableInServer(String tableNameWithType) {
+    LOGGER.info("subscribing to new table {}", tableNameWithType);
+    // init the retention manager to ensure we get the first ideal state
+    _retentionManager.getRetentionManagerForTable(tableNameWithType);
+    _consumer.subscribeForTable(TableNameBuilder.extractRawTableName(tableNameWithType), _topicPrefix);
+  }
+
+  /**
+   * handle clean up when a table no longer has any segment in the current server
+   * @param tableNameWithType
+   */
+  private void handleTableRemovalInServer(String tableNameWithType) {
+    LOGGER.info("unsubscribe to old table {}", tableNameWithType);
+    // key coordinator generate message without table name
+    _consumer.unsubscribeForTable(TableNameBuilder.extractRawTableName(tableNameWithType), _topicPrefix);
+  }
+
+  @Override
+  public synchronized void onSegmentDeletion(String tableNameWithType, String segmentName) {
+    LOGGER.info("deleting segment virtual column from local storage for table {} segment {}", tableNameWithType, segmentName);
+    Map<String, Set<UpsertSegmentDataManager>> segmentManagerMap = _tableSegmentMap.get(tableNameWithType);
+    if (segmentManagerMap != null) {
+      if (segmentManagerMap.containsKey(segmentName) && segmentManagerMap.get(segmentName).size() > 0) {
+        LOGGER.error("trying to remove segment storage with {} segment data manager", segmentManagerMap.get(segmentName).size());
+      }
+      try {
+        segmentManagerMap.remove(segmentName);
+        _retentionManager.getRetentionManagerForTable(tableNameWithType).notifySegmentDeletion(tableNameWithType);
+        _updateLogStorageProvider.removeSegment(tableNameWithType, segmentName);
+      } catch (IOException e) {
+        throw new RuntimeException(String.format("failed to delete table %s segment %s", tableNameWithType, segmentName), e);
+      }
+      if (segmentManagerMap.size() == 0) {
+        _tableSegmentMap.remove(tableNameWithType);
+        handleTableRemovalInServer(tableNameWithType);
+      }
+    } else {
+      LOGGER.error("deleting a segment {}:{} from current server but don't have segment map on updater",
+          tableNameWithType, segmentName);
+    }
+  }
+
+  // A table's update logs grouped by segment names.
+  private class TableUpdateLogs {
+    private Map<String, List<UpdateLogEntry>> _segments2UpdateLog;
+
+    public TableUpdateLogs() {
+                                 _segments2UpdateLog = new HashMap<>();
+                                                                       }
+
+    public Map<String, List<UpdateLogEntry>> getSegments2UpdateLog() {
+      return Collections.unmodifiableMap(_segments2UpdateLog);
+    }
+
+    // Partition is the partition where the record appears in the Segment Update Event message queue.
+    public void addUpdateLogEntry(LogCoordinatorMessage record, int partition) {
+      if (record == null || record.getSegmentName() == null) {
+        LOGGER.error("Empty update log or no segment in the entry: {}", record);
+        return;
+      }
+      _segments2UpdateLog.computeIfAbsent(
+              record.getSegmentName(), s -> new ArrayList<>()).add(new UpdateLogEntry(record, partition));
+    }
+
+    public List<UpdateLogEntry> get(String segmentNameStr) {
+      return Collections.unmodifiableList(_segments2UpdateLog.get(segmentNameStr));
+    }
+  }
+}
\ No newline at end of file
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/VirtualColumnContext.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdaterConfig.java
similarity index 55%
copy from pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/VirtualColumnContext.java
copy to pinot-core/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdaterConfig.java
index bb32320..97d48f3 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/VirtualColumnContext.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdaterConfig.java
@@ -16,29 +16,10 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.core.segment.virtualcolumn;
+package org.apache.pinot.core.segment.updater;
 
-import org.apache.pinot.spi.data.FieldSpec;
-
-
-/**
- * Miscellaneous context information about the virtual column.
- * It will be used to build various components (dictionary, reader, etc) in the virtual column provider.
- */
-public class VirtualColumnContext {
-  private FieldSpec _fieldSpec;
-  private int _totalDocCount;
-
-  public VirtualColumnContext(FieldSpec fieldSpec, int totalDocCount) {
-    _fieldSpec = fieldSpec;
-    _totalDocCount = totalDocCount;
-  }
-
-  public FieldSpec getFieldSpec() {
-    return _fieldSpec;
-  }
-
-  public int getTotalDocCount() {
-    return _totalDocCount;
-  }
+public class SegmentUpdaterConfig {
+  public static final String SEGMENT_UDPATE_SLEEP_MS = "sleep.ms";
+  public static final String INPUT_TOPIC_PREFIX  = "input.topic.prefix";
+  public static final int SEGMENT_UDPATE_SLEEP_MS_DEFAULT = 100;
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/UpsertWaterMarkManager.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/UpsertWaterMarkManager.java
new file mode 100644
index 0000000..bbea037
--- /dev/null
+++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/UpsertWaterMarkManager.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.segment.updater;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+import org.apache.pinot.grigio.common.metrics.GrigioGauge;
+import org.apache.pinot.grigio.common.metrics.GrigioMetrics;
+import org.apache.pinot.grigio.common.storageProvider.UpdateLogEntry;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class UpsertWaterMarkManager {
+
+  private final Map<String, Map<Integer, Long>> _highWaterMarkTablePartitionMap = new ConcurrentHashMap<>();
+  private final GrigioMetrics _metrics;
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(UpsertWaterMarkManager.class);
+  private static volatile UpsertWaterMarkManager _instance;
+
+  private UpsertWaterMarkManager(GrigioMetrics metrics) {
+    _metrics = metrics;
+  }
+
+  public static void init(GrigioMetrics metrics) {
+    synchronized (UpsertWaterMarkManager.class) {
+      Preconditions.checkState(_instance == null, "upsert water mark manager is already init");
+      _instance = new UpsertWaterMarkManager(metrics);
+    }
+  }
+
+  public static UpsertWaterMarkManager getInstance() {
+    Preconditions.checkState(_instance != null, "upsert water mark manager is not yet init");
+    return _instance;
+  }
+
+  // TODO(tingchen) Look into the case where Segment Update Messages might arrive before the corresponding physical data.
+  public void processMessage(String table, String segment, UpdateLogEntry logEntry) {
+    if (logEntry == null) {
+      return;
+    }
+    long version = logEntry.getValue();
+    int partition = logEntry.getPartition();
+    processVersionUpdate(table, partition, version);
+  }
+
+  public void processVersionUpdate(String table, int partition, long version) {
+    Preconditions.checkState(partition >= 0, "logEntry has invalid version {} for table {}",
+        version, table);
+    Map<Integer, Long> partitionToHighWaterMark = _highWaterMarkTablePartitionMap.computeIfAbsent(table, t -> new ConcurrentHashMap<>());
+    long currentVersion = partitionToHighWaterMark.getOrDefault(partition, -1L);  // assumes that valid version is non-negative
+    if (version > currentVersion) {
+      partitionToHighWaterMark.put(partition, version);
+      _metrics.setValueOfTableGauge(String.valueOf(partition), GrigioGauge.SERVER_VERSION_CONSUMED, version);
+    }
+  }
+
+  public Map<Integer, Long> getHighWaterMarkForTable(String tableName) {
+    return ImmutableMap.copyOf(_highWaterMarkTablePartitionMap.getOrDefault(tableName, ImmutableMap.of()));
+  }
+
+  public Map<String, Map<Integer, Long>> getHighWaterMarkTablePartitionMap() {
+    return ImmutableMap.copyOf(_highWaterMarkTablePartitionMap);
+  }
+
+  @Override
+  public String toString() {
+    return Joiner.on(",").withKeyValueSeparator("=").join(_highWaterMarkTablePartitionMap);
+  }
+}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/BaseVirtualColumnProvider.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/BaseVirtualColumnProvider.java
index ece8fa7..c1c7c62 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/BaseVirtualColumnProvider.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/BaseVirtualColumnProvider.java
@@ -21,6 +21,9 @@ package org.apache.pinot.core.segment.virtualcolumn;
 import org.apache.pinot.core.segment.index.ColumnMetadata;
 import org.apache.pinot.core.segment.index.column.ColumnIndexContainer;
 import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.core.io.reader.DataFileReader;
+import org.apache.pinot.core.segment.index.readers.Dictionary;
+import org.apache.pinot.core.segment.index.readers.InvertedIndexReader;
 
 
 /**
@@ -39,4 +42,19 @@ public abstract class BaseVirtualColumnProvider implements VirtualColumnProvider
   public ColumnIndexContainer buildColumnIndexContainer(VirtualColumnContext context) {
     return new VirtualColumnIndexContainer(buildReader(context), buildInvertedIndex(context), buildDictionary(context));
   }
+
+  @Override
+  public ColumnIndexContainer buildColumnIndexContainer(VirtualColumnContext context, DataFileReader fileReader,
+                                                        Dictionary dictionary, InvertedIndexReader invertedIndexReader) {
+    if (fileReader == null) {
+      fileReader = buildReader(context);
+    }
+    if (dictionary == null) {
+      dictionary = buildDictionary(context);
+    }
+    if (invertedIndexReader == null) {
+      invertedIndexReader = buildInvertedIndex(context);
+    }
+    return new VirtualColumnIndexContainer(fileReader, invertedIndexReader, dictionary);
+  }
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/VirtualColumnContext.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/VirtualColumnContext.java
index bb32320..abfccf0 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/VirtualColumnContext.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/VirtualColumnContext.java
@@ -28,10 +28,12 @@ import org.apache.pinot.spi.data.FieldSpec;
 public class VirtualColumnContext {
   private FieldSpec _fieldSpec;
   private int _totalDocCount;
+  private boolean _isMutableSegment;
 
-  public VirtualColumnContext(FieldSpec fieldSpec, int totalDocCount) {
+  public VirtualColumnContext(FieldSpec fieldSpec, int totalDocCount, boolean isMutableSegment) {
     _fieldSpec = fieldSpec;
     _totalDocCount = totalDocCount;
+    _isMutableSegment = isMutableSegment;
   }
 
   public FieldSpec getFieldSpec() {
@@ -41,4 +43,8 @@ public class VirtualColumnContext {
   public int getTotalDocCount() {
     return _totalDocCount;
   }
+
+  public boolean isMutableSegment() {
+    return _isMutableSegment;
+  }
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/VirtualColumnProvider.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/VirtualColumnProvider.java
index 8a1e3f5..aed96bd 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/VirtualColumnProvider.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/VirtualColumnProvider.java
@@ -30,6 +30,7 @@ import org.apache.pinot.core.segment.index.readers.InvertedIndexReader;
  * comprise a proper column.
  */
 public interface VirtualColumnProvider {
+
   DataFileReader buildReader(VirtualColumnContext context);
 
   Dictionary buildDictionary(VirtualColumnContext context);
@@ -39,4 +40,7 @@ public interface VirtualColumnProvider {
   InvertedIndexReader buildInvertedIndex(VirtualColumnContext context);
 
   ColumnIndexContainer buildColumnIndexContainer(VirtualColumnContext context);
+
+  ColumnIndexContainer buildColumnIndexContainer(VirtualColumnContext context, DataFileReader reader,
+                                                 Dictionary dictionary, InvertedIndexReader invertedIndexReader);
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/mutable/BaseLongVirtualColumnProvider.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/mutable/BaseLongVirtualColumnProvider.java
new file mode 100644
index 0000000..fc69a9f
--- /dev/null
+++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/mutable/BaseLongVirtualColumnProvider.java
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.segment.virtualcolumn.mutable;
+
+import org.apache.pinot.core.segment.index.ColumnMetadata;
+import org.apache.pinot.core.segment.index.readers.Dictionary;
+import org.apache.pinot.core.segment.index.readers.InvertedIndexReader;
+import org.apache.pinot.core.segment.virtualcolumn.BaseVirtualColumnProvider;
+import org.apache.pinot.core.segment.virtualcolumn.VirtualColumnContext;
+import org.apache.pinot.spi.data.FieldSpec;
+
+/**
+ * base class for upsert related virtual column (validFrom and validUntil virtual column)
+ * this only support the direct access through the reader but not the dictionary or inverted index
+ */
+public abstract class BaseLongVirtualColumnProvider extends BaseVirtualColumnProvider {
+
+  @Override
+  public Dictionary buildDictionary(VirtualColumnContext context) {
+    return null;
+  }
+
+  @Override
+  public ColumnMetadata buildMetadata(VirtualColumnContext context) {
+    ColumnMetadata.Builder columnMetadataBuilder = new ColumnMetadata.Builder()
+        .setCardinality(1) // TODO: double check on this
+        .setHasDictionary(false)
+        .setHasInvertedIndex(false)
+        .setFieldType(FieldSpec.FieldType.DIMENSION)
+        .setDataType(FieldSpec.DataType.LONG)
+        .setSingleValue(true)
+        .setIsSorted(false)
+        .setTotalDocs(context.getTotalDocCount());
+    return columnMetadataBuilder.build();
+  }
+
+  @Override
+  public InvertedIndexReader buildInvertedIndex(VirtualColumnContext context) {
+    return null;
+  }
+}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/VirtualColumnContext.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/mutable/BaseVirtualColumnSingleValueReaderWriter.java
similarity index 56%
copy from pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/VirtualColumnContext.java
copy to pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/mutable/BaseVirtualColumnSingleValueReaderWriter.java
index bb32320..fdcfe25 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/VirtualColumnContext.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/mutable/BaseVirtualColumnSingleValueReaderWriter.java
@@ -16,29 +16,14 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.core.segment.virtualcolumn;
+package org.apache.pinot.core.segment.virtualcolumn.mutable;
 
-import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.core.io.reader.BaseSingleColumnSingleValueReader;
+import org.apache.pinot.core.io.reader.ReaderContext;
 
+public abstract class BaseVirtualColumnSingleValueReaderWriter<E extends ReaderContext> extends BaseSingleColumnSingleValueReader<E> {
 
-/**
- * Miscellaneous context information about the virtual column.
- * It will be used to build various components (dictionary, reader, etc) in the virtual column provider.
- */
-public class VirtualColumnContext {
-  private FieldSpec _fieldSpec;
-  private int _totalDocCount;
-
-  public VirtualColumnContext(FieldSpec fieldSpec, int totalDocCount) {
-    _fieldSpec = fieldSpec;
-    _totalDocCount = totalDocCount;
-  }
-
-  public FieldSpec getFieldSpec() {
-    return _fieldSpec;
-  }
-
-  public int getTotalDocCount() {
-    return _totalDocCount;
+  public void addNewRecord(int docId) {
+    // default do nothing
   }
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/VirtualColumnProvider.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/mutable/ValidFromInMemoryVirtualColumnProvider.java
similarity index 50%
copy from pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/VirtualColumnProvider.java
copy to pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/mutable/ValidFromInMemoryVirtualColumnProvider.java
index 8a1e3f5..ebb45e7 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/VirtualColumnProvider.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/mutable/ValidFromInMemoryVirtualColumnProvider.java
@@ -16,27 +16,30 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.core.segment.virtualcolumn;
+package org.apache.pinot.core.segment.virtualcolumn.mutable;
 
 import org.apache.pinot.core.io.reader.DataFileReader;
-import org.apache.pinot.core.segment.index.ColumnMetadata;
-import org.apache.pinot.core.segment.index.column.ColumnIndexContainer;
-import org.apache.pinot.core.segment.index.readers.Dictionary;
-import org.apache.pinot.core.segment.index.readers.InvertedIndexReader;
+import org.apache.pinot.core.segment.virtualcolumn.VirtualColumnContext;
+import org.apache.pinot.grigio.common.messages.LogEventType;
 
+public class ValidFromInMemoryVirtualColumnProvider extends BaseLongVirtualColumnProvider {
 
-/**
- * Virtual column provider interface, which is used to instantiate the various components (dictionary, reader, etc) that
- * comprise a proper column.
- */
-public interface VirtualColumnProvider {
-  DataFileReader buildReader(VirtualColumnContext context);
-
-  Dictionary buildDictionary(VirtualColumnContext context);
-
-  ColumnMetadata buildMetadata(VirtualColumnContext context);
-
-  InvertedIndexReader buildInvertedIndex(VirtualColumnContext context);
+  @Override
+  public DataFileReader buildReader(VirtualColumnContext context) {
+    return new VirtualColumnLongValueReaderWriter(context) {
+      @Override
+      public boolean update(int docId, int value, LogEventType eventType) {
+        if (eventType == LogEventType.INSERT) {
+          updateValue(docId, value);
+          return true;
+        }
+        return false;
+      }
 
-  ColumnIndexContainer buildColumnIndexContainer(VirtualColumnContext context);
+      @Override
+      public boolean update(int docId, long value, LogEventType eventType) {
+        return update(docId, Math.toIntExact(value), eventType);
+      }
+    };
+  }
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/VirtualColumnProvider.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/mutable/ValidUntilInMemoryVirtualColumnProvider.java
similarity index 50%
copy from pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/VirtualColumnProvider.java
copy to pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/mutable/ValidUntilInMemoryVirtualColumnProvider.java
index 8a1e3f5..9aef94c 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/VirtualColumnProvider.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/mutable/ValidUntilInMemoryVirtualColumnProvider.java
@@ -16,27 +16,30 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.core.segment.virtualcolumn;
+package org.apache.pinot.core.segment.virtualcolumn.mutable;
 
 import org.apache.pinot.core.io.reader.DataFileReader;
-import org.apache.pinot.core.segment.index.ColumnMetadata;
-import org.apache.pinot.core.segment.index.column.ColumnIndexContainer;
-import org.apache.pinot.core.segment.index.readers.Dictionary;
-import org.apache.pinot.core.segment.index.readers.InvertedIndexReader;
+import org.apache.pinot.core.segment.virtualcolumn.VirtualColumnContext;
+import org.apache.pinot.grigio.common.messages.LogEventType;
 
+public class ValidUntilInMemoryVirtualColumnProvider extends BaseLongVirtualColumnProvider {
 
-/**
- * Virtual column provider interface, which is used to instantiate the various components (dictionary, reader, etc) that
- * comprise a proper column.
- */
-public interface VirtualColumnProvider {
-  DataFileReader buildReader(VirtualColumnContext context);
-
-  Dictionary buildDictionary(VirtualColumnContext context);
-
-  ColumnMetadata buildMetadata(VirtualColumnContext context);
-
-  InvertedIndexReader buildInvertedIndex(VirtualColumnContext context);
+  @Override
+  public DataFileReader buildReader(VirtualColumnContext context) {
+    return new VirtualColumnLongValueReaderWriter(context) {
+      @Override
+      public boolean update(int docId, int offset, LogEventType eventType) {
+        if (eventType == LogEventType.DELETE) {
+          this.updateValue(docId, offset);
+          return true;
+        }
+        return false;
+      }
 
-  ColumnIndexContainer buildColumnIndexContainer(VirtualColumnContext context);
+      @Override
+      public boolean update(int docId, long offset, LogEventType eventType) {
+        return update(docId, Math.toIntExact(offset), eventType);
+      }
+    };
+  }
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/mutable/VirtualColumnLongValueReaderWriter.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/mutable/VirtualColumnLongValueReaderWriter.java
new file mode 100644
index 0000000..78f3bda
--- /dev/null
+++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/mutable/VirtualColumnLongValueReaderWriter.java
@@ -0,0 +1,108 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.segment.virtualcolumn.mutable;
+
+import org.apache.pinot.core.io.reader.impl.ChunkReaderContext;
+import org.apache.pinot.core.segment.virtualcolumn.VirtualColumnContext;
+import org.apache.pinot.grigio.common.messages.LogEventType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+public abstract class VirtualColumnLongValueReaderWriter extends BaseVirtualColumnSingleValueReaderWriter<ChunkReaderContext> {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(VirtualColumnLongValueReaderWriter.class);
+
+  private int _totalDocSize;
+  private int _currentMaxDocId;
+  private final int [] _values;
+  private final int DEFAULT_NEW_VALUE = -1;
+
+  public VirtualColumnLongValueReaderWriter(VirtualColumnContext context) {
+    this(context.getTotalDocCount(), context.isMutableSegment());
+  }
+
+  public VirtualColumnLongValueReaderWriter(int totalDocSize, boolean isMutableSegment) {
+    _totalDocSize = totalDocSize;
+    _values = new int[_totalDocSize];
+    if (!isMutableSegment) {
+      Arrays.fill(_values, -1);
+    }
+  }
+
+  @Override
+  public synchronized void addNewRecord(int docId) {
+    if (docId >= _totalDocSize) {
+      throw new RuntimeException(String.format("failed to add docId %s larger than allocated size %s", docId, _totalDocSize));
+    }
+    _currentMaxDocId = docId;
+    _values[docId] = DEFAULT_NEW_VALUE;
+  }
+
+  @Override
+  public ChunkReaderContext createContext() {
+    return null;
+  }
+
+  @Override
+  public long getLong(int row) {
+    return _values[row];
+  }
+
+  @Override
+  public long getLong(int rowId, ChunkReaderContext context) {
+    return _values[rowId];
+  }
+
+  @Override
+  public void readValues(int[] rows, int rowStartPos, int rowSize, int[] values, int valuesStartPos) {
+    System.arraycopy(rows, rowStartPos, values, valuesStartPos, rowSize);
+  }
+
+  @Override
+  public void close() throws IOException {
+  }
+
+  /**
+   * update the internal data to value at a given location
+   * assume the update will be idempotent and we won't check if the existing data have been set
+   *
+   */
+  protected void updateValue(int docId, int value) {
+    if (docId >= _totalDocSize) {
+      throw new RuntimeException(String.format("new record docId %s is larger than capacity %s", docId, _totalDocSize));
+    }
+    if (_values[docId] == 0) {
+      throw new RuntimeException(String.format("failed to update virtual column: with value %s:%s we are trying to " +
+              "update a value that has not been ingested yet, max doc id %s", docId, value, _currentMaxDocId));
+    }
+    _values[docId] = value;
+  }
+
+  // ensure backward compatibility
+  protected void updateValue(int docId, long value) {
+    updateValue(docId, Math.toIntExact(value));
+  }
+
+  public abstract boolean update(int docId, long value, LogEventType eventType);
+
+  public abstract boolean update(int docId, int value, LogEventType eventType);
+}
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManagerTest.java b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManagerTest.java
index 715c833..c4b557b 100644
--- a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManagerTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManagerTest.java
@@ -696,7 +696,7 @@ public class LLRealtimeSegmentDataManagerTest {
     Assert.assertEquals(secondSegmentDataManager.get().getPartitionConsumerSemaphore().availablePermits(), 1);
   }
 
-  public static class FakeLLRealtimeSegmentDataManager extends LLRealtimeSegmentDataManager {
+  public static class FakeLLRealtimeSegmentDataManager extends AppendLLRealtimeSegmentDataManager {
 
     public Field _state;
     public Field _shouldStop;
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/indexsegment/immutable/ImmutableUpsertSegmentImplTest.java b/pinot-core/src/test/java/org/apache/pinot/core/indexsegment/immutable/ImmutableUpsertSegmentImplTest.java
new file mode 100644
index 0000000..817131c
--- /dev/null
+++ b/pinot-core/src/test/java/org/apache/pinot/core/indexsegment/immutable/ImmutableUpsertSegmentImplTest.java
@@ -0,0 +1,127 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.indexsegment.immutable;
+
+import org.apache.pinot.core.segment.updater.UpsertWaterMarkManager;
+import org.apache.pinot.core.segment.virtualcolumn.mutable.VirtualColumnLongValueReaderWriter;
+import org.apache.pinot.grigio.common.messages.LogEventType;
+import org.apache.pinot.grigio.common.storageProvider.UpdateLogEntry;
+import org.apache.pinot.grigio.common.storageProvider.UpdateLogEntrySet;
+import org.apache.pinot.grigio.common.storageProvider.UpdateLogStorageProvider;
+import org.testng.Assert;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+
+public class ImmutableUpsertSegmentImplTest {
+
+  UpdateLogStorageProvider _mockProvider;
+  UpsertWaterMarkManager _mockUpsertWaterMarkManager;
+  List<VirtualColumnLongValueReaderWriter> _readerWriters = new ArrayList<>();
+
+  @BeforeMethod
+  public void init() {
+    _mockProvider = mock(UpdateLogStorageProvider.class);
+    _mockUpsertWaterMarkManager = mock(UpsertWaterMarkManager.class);
+  }
+
+  @Test
+  public void testInitVirtualColumn() throws IOException {
+    long start = System.currentTimeMillis();
+    long minOffset = 5000_000l;
+    int totalDocs = 5_000_000;
+    _readerWriters.add(new VirtualColumnLongValueReaderWriter(totalDocs, false) {
+      @Override
+      public boolean update(int docId, long value, LogEventType eventType) {
+        if (eventType == LogEventType.INSERT) {
+          updateValue(docId, value);
+          return true;
+        }
+        return false;
+      }
+
+      @Override
+      public boolean update(int docId, int value, LogEventType eventType) {
+        return update(docId, value, eventType);
+      }
+    });
+    _readerWriters.add(new VirtualColumnLongValueReaderWriter(totalDocs, false) {
+      @Override
+      public boolean update(int docId, long value, LogEventType eventType) {
+        if (eventType == LogEventType.DELETE) {
+          updateValue(docId, value);
+          return true;
+        }
+        return false;
+      }
+
+      @Override
+      public boolean update(int docId, int value, LogEventType eventType) {
+        return update(docId, value, eventType);
+      }
+    });
+    int[] offsetToDocId = new int[totalDocs];
+    for (int i = 0; i < totalDocs; i++) {
+      offsetToDocId[i] = i;
+    }
+    List<UpdateLogEntry> updateLogEntries = new ArrayList<>(totalDocs * 2);
+    for (int i = 0; i < totalDocs; i++) {
+      updateLogEntries.add(new UpdateLogEntry(minOffset + i, 50, LogEventType.INSERT, i%8));
+      updateLogEntries.add(new UpdateLogEntry(minOffset + i, 100, LogEventType.DELETE, i%8));
+    }
+    UpdateLogEntrySet entrySet = new UpdateLogEntrySet(null, 2) {
+      @Override
+      public Iterator<UpdateLogEntry> iterator() {
+        return updateLogEntries.iterator();
+      }
+    };
+    when(_mockProvider.getAllMessages(anyString(), anyString())).thenReturn(entrySet);
+    System.out.println("run time for set up: " + (System.currentTimeMillis() - start));
+
+    start = System.currentTimeMillis();
+
+    ImmutableUpsertSegmentImpl immutableUpsertSegment = new ImmutableUpsertSegmentImpl(_readerWriters, totalDocs,
+        _mockUpsertWaterMarkManager, _mockProvider, minOffset, offsetToDocId);
+
+    immutableUpsertSegment.initVirtualColumn();
+    long runtime = System.currentTimeMillis() - start;
+    System.out.println("run time is " + runtime);
+    // on regular developer laptop this should take less 1 second, but on integration server this might be longer
+    Assert.assertTrue(runtime < 10_000L, "run time should be less than 10 second");
+
+    VirtualColumnLongValueReaderWriter insertReaderWrite = _readerWriters.get(0);
+    VirtualColumnLongValueReaderWriter deleteReaderWrite = _readerWriters.get(1);
+    for (int i = 0; i < totalDocs; i++) {
+      if (insertReaderWrite.getLong(i) != 50 || deleteReaderWrite.getLong(i) != 100) {
+        System.out.println(String.format("position %d has value %d/%d", i, insertReaderWrite.getLong(i),
+            deleteReaderWrite.getLong(i)));
+        Assert.fail("no correct value");
+      }
+    }
+  }
+}
\ No newline at end of file
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImplTestUtils.java b/pinot-core/src/test/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImplTestUtils.java
index e4df760..26e7eff 100644
--- a/pinot-core/src/test/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImplTestUtils.java
+++ b/pinot-core/src/test/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImplTestUtils.java
@@ -59,6 +59,6 @@ public class MutableSegmentImplTestUtils {
             .setRealtimeSegmentZKMetadata(new RealtimeSegmentZKMetadata())
             .setMemoryManager(new DirectMemoryManager(SEGMENT_NAME)).setStatsHistory(statsHistory)
             .setAggregateMetrics(aggregateMetrics).setNullHandlingEnabled(nullHandlingEnabled).build();
-    return new MutableSegmentImpl(realtimeSegmentConfig);
+    return new MutableAppendSegmentImpl(realtimeSegmentConfig);
   }
 }
diff --git a/pinot-core/src/test/java/org/apache/pinot/queries/InterSegmentAggregationSingleValueQueriesTest.java b/pinot-core/src/test/java/org/apache/pinot/queries/InterSegmentAggregationSingleValueQueriesTest.java
index cca0563..f06ad7d 100644
--- a/pinot-core/src/test/java/org/apache/pinot/queries/InterSegmentAggregationSingleValueQueriesTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/queries/InterSegmentAggregationSingleValueQueriesTest.java
@@ -18,8 +18,6 @@
  */
 package org.apache.pinot.queries;
 
-import java.io.Serializable;
-import java.util.function.Function;
 import org.apache.pinot.common.response.broker.BrokerResponseNative;
 import org.apache.pinot.common.response.broker.SelectionResults;
 import org.apache.pinot.spi.utils.BytesUtils;
@@ -29,6 +27,9 @@ import org.apache.pinot.pql.parsers.Pql2Compiler;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
+import java.io.Serializable;
+import java.util.function.Function;
+
 import static org.testng.Assert.assertFalse;
 import static org.testng.Assert.assertTrue;
 
diff --git a/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixInstanceDataManager.java b/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixInstanceDataManager.java
index 3df1b8f..1af7e29 100644
--- a/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixInstanceDataManager.java
+++ b/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixInstanceDataManager.java
@@ -20,10 +20,7 @@ package org.apache.pinot.server.starter.helix;
 
 import com.google.common.base.Preconditions;
 import java.io.File;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Set;
+import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.locks.Lock;
 import javax.annotation.Nullable;
@@ -48,6 +45,7 @@ import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
 import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
 import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
 import org.apache.pinot.core.segment.index.loader.LoaderUtils;
+import org.apache.pinot.core.segment.updater.UpsertWaterMarkManager;
 import org.apache.pinot.spi.data.Schema;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -304,6 +302,11 @@ public class HelixInstanceDataManager implements InstanceDataManager {
   }
 
   @Override
+  public Map<String, Map<Integer, Long>> getLowWaterMarks() {
+    return UpsertWaterMarkManager.getInstance().getHighWaterMarkTablePartitionMap();
+  }
+
+  @Override
   public String getSegmentDataDirectory() {
     return _instanceDataManagerConfig.getInstanceDataDir();
   }
diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/realtime/provisioning/MemoryEstimator.java b/pinot-tools/src/main/java/org/apache/pinot/tools/realtime/provisioning/MemoryEstimator.java
index 865591b..187b36f 100644
--- a/pinot-tools/src/main/java/org/apache/pinot/tools/realtime/provisioning/MemoryEstimator.java
+++ b/pinot-tools/src/main/java/org/apache/pinot/tools/realtime/provisioning/MemoryEstimator.java
@@ -26,6 +26,7 @@ import java.util.stream.Collectors;
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.io.FileUtils;
 import org.apache.pinot.common.config.TableConfig;
+import org.apache.pinot.core.indexsegment.mutable.MutableAppendSegmentImpl;
 import org.apache.pinot.spi.data.FieldSpec;
 import org.apache.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
 import org.apache.pinot.spi.utils.DataSize;
@@ -135,7 +136,7 @@ public class MemoryEstimator {
             .setStatsHistory(sampleStatsHistory);
 
     // create mutable segment impl
-    MutableSegmentImpl mutableSegmentImpl = new MutableSegmentImpl(realtimeSegmentConfigBuilder.build());
+    MutableSegmentImpl mutableSegmentImpl = new MutableAppendSegmentImpl(realtimeSegmentConfigBuilder.build());
 
     // read all rows and index them
     try (PinotSegmentRecordReader segmentRecordReader = new PinotSegmentRecordReader(_sampleCompletedSegment);) {
@@ -235,7 +236,7 @@ public class MemoryEstimator {
               .setOffHeap(true).setMemoryManager(memoryManager).setStatsHistory(statsHistory);
 
       // create mutable segment impl
-      MutableSegmentImpl mutableSegmentImpl = new MutableSegmentImpl(realtimeSegmentConfigBuilder.build());
+      MutableSegmentImpl mutableSegmentImpl = new MutableAppendSegmentImpl(realtimeSegmentConfigBuilder.build());
       long memoryForConsumingSegmentPerPartition = memoryManager.getTotalAllocatedBytes();
       mutableSegmentImpl.destroy();
       FileUtils.deleteQuietly(statsFileCopy);


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


[incubator-pinot] 08/09: fix another test

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jamesshao pushed a commit to branch upsert-refactor
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git

commit c793323014dea912ef277b997419798dca51e9c6
Author: james Shao <sj...@uber.com>
AuthorDate: Wed Feb 26 11:37:25 2020 -0800

    fix another test
---
 .../MutableSegmentImplAggregateMetricsTest.java    | 46 +++++++++++++++++++++-
 1 file changed, 44 insertions(+), 2 deletions(-)

diff --git a/pinot-core/src/test/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImplAggregateMetricsTest.java b/pinot-core/src/test/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImplAggregateMetricsTest.java
index e08e6b3..e6bc499 100644
--- a/pinot-core/src/test/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImplAggregateMetricsTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImplAggregateMetricsTest.java
@@ -26,6 +26,13 @@ import java.util.Map;
 import java.util.Random;
 import java.util.concurrent.TimeUnit;
 import org.apache.commons.lang.RandomStringUtils;
+import org.apache.pinot.core.io.reader.DataFileReader;
+import org.apache.pinot.core.segment.index.ColumnMetadata;
+import org.apache.pinot.core.segment.index.column.ColumnIndexContainer;
+import org.apache.pinot.core.segment.index.readers.Dictionary;
+import org.apache.pinot.core.segment.index.readers.InvertedIndexReader;
+import org.apache.pinot.core.segment.virtualcolumn.VirtualColumnContext;
+import org.apache.pinot.core.segment.virtualcolumn.VirtualColumnProvider;
 import org.apache.pinot.spi.data.DimensionFieldSpec;
 import org.apache.pinot.spi.data.FieldSpec;
 import org.apache.pinot.spi.data.MetricFieldSpec;
@@ -58,10 +65,10 @@ public class MutableSegmentImplAggregateMetricsTest {
         .build();
     // Add virtual columns, which should not be aggregated
     DimensionFieldSpec virtualDimensionFieldSpec =
-        new DimensionFieldSpec("$virtualDimension", FieldSpec.DataType.INT, true, Object.class);
+        new DimensionFieldSpec("$virtualDimension", FieldSpec.DataType.INT, true, mockVCProvider.class);
     schema.addField(virtualDimensionFieldSpec);
     MetricFieldSpec virtualMetricFieldSpec = new MetricFieldSpec("$virtualMetric", FieldSpec.DataType.INT);
-    virtualMetricFieldSpec.setVirtualColumnProvider("provider.class");
+    virtualMetricFieldSpec.setVirtualColumnProvider(mockVCProvider.class.getName());
     schema.addField(virtualMetricFieldSpec);
 
     _mutableSegmentImpl = MutableSegmentImplTestUtils
@@ -127,4 +134,39 @@ public class MutableSegmentImplAggregateMetricsTest {
   public void tearDown() {
     _mutableSegmentImpl.destroy();
   }
+
+  public static class mockVCProvider implements VirtualColumnProvider {
+
+    public mockVCProvider() {}
+
+    @Override
+    public DataFileReader buildReader(VirtualColumnContext context) {
+      return null;
+    }
+
+    @Override
+    public Dictionary buildDictionary(VirtualColumnContext context) {
+      return null;
+    }
+
+    @Override
+    public ColumnMetadata buildMetadata(VirtualColumnContext context) {
+      return null;
+    }
+
+    @Override
+    public InvertedIndexReader buildInvertedIndex(VirtualColumnContext context) {
+      return null;
+    }
+
+    @Override
+    public ColumnIndexContainer buildColumnIndexContainer(VirtualColumnContext context) {
+      return null;
+    }
+
+    @Override
+    public ColumnIndexContainer buildColumnIndexContainer(VirtualColumnContext context, DataFileReader reader, Dictionary dictionary, InvertedIndexReader invertedIndexReader) {
+      return null;
+    }
+  }
 }


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


[incubator-pinot] 09/09: update to fix unit tests

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jamesshao pushed a commit to branch upsert-refactor
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git

commit 39f5ba8d4770ae0a1b1c23ef02965ff430c7b78b
Author: james Shao <sj...@uber.com>
AuthorDate: Thu Mar 19 14:34:46 2020 -0700

    update to fix unit tests
---
 .../requesthandler/BaseBrokerRequestHandler.java   |  2 +-
 .../broker/upsert/DefaultLowWaterMarkService.java  | 11 +++---
 .../broker/upsert/DefaultUpsertQueryRewriter.java  |  6 ++--
 .../broker/upsert/LowWaterMarkServiceProvider.java | 18 +++++++---
 .../core/data/manager/BaseTableDataManager.java    |  1 +
 .../core/data/manager/InstanceDataManager.java     |  5 ---
 .../realtime/LLRealtimeSegmentDataManager.java     |  5 ++-
 .../manager/realtime/RealtimeTableDataManager.java | 13 ++++----
 .../data/manager/upsert/DataManagerCallback.java   | 37 +++++++++++++++++++-
 .../upsert/DefaultIndexSegmentCallback.java        |  8 ++---
 .../DefaultTableDataManagerCallbackImpl.java       | 14 ++++++--
 .../data/manager/upsert/IndexSegmentCallback.java  | 31 +++++++++++++++++
 .../manager/upsert/TableDataManagerCallback.java   | 39 ++++++++++++++++++++--
 .../upsert/TableDataManagerCallbackProvider.java   | 24 +++++++++----
 .../segment/updater/DefaultWaterMarkManager.java   |  4 ++-
 .../core/segment/updater/LowWaterMarkService.java  | 39 +++++++++++++++-------
 ...UpsertQueryRewriter.java => QueryRewriter.java} | 20 +++++++----
 .../segment/updater/UpsertComponentContainer.java  | 34 +++++++++++++++++++
 .../core/segment/updater/WaterMarkManager.java     | 27 +++++++++++++--
 .../SegmentGenerationWithNullValueVectorTest.java  |  2 ++
 .../pinot/query/executor/QueryExecutorTest.java    |  2 ++
 .../upsert/PollingBasedLowWaterMarkService.java    |  6 ++--
 .../broker/upsert/UpsertQueryRewriterImpl.java     |  6 ++--
 .../upsert/UpsertTableDataManagerCallbackImpl.java | 18 ++++++++--
 .../segment/updater/UpsertWaterMarkManager.java    | 26 ++++++++++++++-
 ...terImplTest.java => QueryRewriterImplTest.java} |  2 +-
 .../starter/helix/HelixInstanceDataManager.java    |  2 +-
 .../upsert/UpsertComponentContainerProvider.java   |  4 ---
 .../apache/pinot/server/api/BaseResourceTest.java  |  4 +++
 29 files changed, 332 insertions(+), 78 deletions(-)

diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java
index cb17740..4b02cd4 100644
--- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java
@@ -300,7 +300,7 @@ public abstract class BaseBrokerRequestHandler implements BrokerRequestHandler {
 
     if (shouldEnableLowWaterMarkRewrite(request)) {
       // Augment the realtime request with LowWaterMark constraints.
-      _lwmService.getQueryRewriter().rewriteQueryForUpsert(realtimeBrokerRequest, rawTableName);
+      _lwmService.getQueryRewriter().maybeRewriteQueryForUpsert(realtimeBrokerRequest, rawTableName);
     }
 
     // Calculate routing table for the query
diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/upsert/DefaultLowWaterMarkService.java b/pinot-broker/src/main/java/org/apache/pinot/broker/upsert/DefaultLowWaterMarkService.java
index 42e1dcb..eb71890 100644
--- a/pinot-broker/src/main/java/org/apache/pinot/broker/upsert/DefaultLowWaterMarkService.java
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/upsert/DefaultLowWaterMarkService.java
@@ -22,13 +22,16 @@ import com.google.common.collect.ImmutableMap;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.pinot.common.metrics.BrokerMetrics;
 import org.apache.pinot.core.segment.updater.LowWaterMarkService;
-import org.apache.pinot.core.segment.updater.UpsertQueryRewriter;
+import org.apache.pinot.core.segment.updater.QueryRewriter;
 
 import java.util.Map;
 
+/**
+ * default class to handle any low watermark operation on pinot broker, mostly no-op
+ */
 public class DefaultLowWaterMarkService implements LowWaterMarkService {
 
-  private UpsertQueryRewriter upsertQueryRewriter = new DefaultUpsertQueryRewriter();
+  private QueryRewriter queryRewriter = new DefaultUpsertQueryRewriter();
 
   @Override
   public void init(HelixDataAccessor helixDataAccessor, String helixClusterName, int serverPollingInterval,
@@ -49,7 +52,7 @@ public class DefaultLowWaterMarkService implements LowWaterMarkService {
   }
 
   @Override
-  public UpsertQueryRewriter getQueryRewriter() {
-    return upsertQueryRewriter;
+  public QueryRewriter getQueryRewriter() {
+    return queryRewriter;
   }
 }
diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/upsert/DefaultUpsertQueryRewriter.java b/pinot-broker/src/main/java/org/apache/pinot/broker/upsert/DefaultUpsertQueryRewriter.java
index d18a56f..97195a3 100644
--- a/pinot-broker/src/main/java/org/apache/pinot/broker/upsert/DefaultUpsertQueryRewriter.java
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/upsert/DefaultUpsertQueryRewriter.java
@@ -1,7 +1,7 @@
 package org.apache.pinot.broker.upsert;
 
 import org.apache.pinot.common.request.BrokerRequest;
-import org.apache.pinot.core.segment.updater.UpsertQueryRewriter;
+import org.apache.pinot.core.segment.updater.QueryRewriter;
 
 /**
  * Licensed to the Apache Software Foundation (ASF) under one
@@ -21,10 +21,10 @@ import org.apache.pinot.core.segment.updater.UpsertQueryRewriter;
  * specific language governing permissions and limitations
  * under the License.
  */
-public class DefaultUpsertQueryRewriter implements UpsertQueryRewriter {
+public class DefaultUpsertQueryRewriter implements QueryRewriter {
 
   @Override
-  public void rewriteQueryForUpsert(BrokerRequest request, String rawTableName) {
+  public void maybeRewriteQueryForUpsert(BrokerRequest request, String rawTableName) {
     // do nothing
   }
 }
diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/upsert/LowWaterMarkServiceProvider.java b/pinot-broker/src/main/java/org/apache/pinot/broker/upsert/LowWaterMarkServiceProvider.java
index f5c06f3..56b3bff 100644
--- a/pinot-broker/src/main/java/org/apache/pinot/broker/upsert/LowWaterMarkServiceProvider.java
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/upsert/LowWaterMarkServiceProvider.java
@@ -18,7 +18,6 @@
  */
 package org.apache.pinot.broker.upsert;
 
-import com.google.common.base.Preconditions;
 import org.apache.commons.configuration.Configuration;
 import org.apache.commons.lang3.exception.ExceptionUtils;
 import org.apache.helix.HelixDataAccessor;
@@ -29,21 +28,28 @@ import org.slf4j.LoggerFactory;
 
 import static org.apache.pinot.common.utils.CommonConstants.Broker.*;
 
-
+/**
+ * provider to initialize LowWaterMarkServer for pinot broker
+ */
 public class LowWaterMarkServiceProvider {
 
   private static final Logger LOGGER = LoggerFactory.getLogger(LowWaterMarkServiceProvider.class);
 
   private LowWaterMarkService _instance;
 
+  /**
+   * create a new provider instance
+   * @param brokerConfig config for this provider to create the actual class reference,
+   *                     refer to {@value CommonConstants.Broker#CONFIG_OF_BROKER_LWMS_CLASS_NAME}
+   * @param dataAccessor helix data access to help low watermark service to find proper server cluster
+   * @param clusterName cluster name for the current pinot cluster
+   */
   public LowWaterMarkServiceProvider(Configuration brokerConfig, HelixDataAccessor dataAccessor, String clusterName) {
     String className = brokerConfig.getString(CommonConstants.Broker.CONFIG_OF_BROKER_LWMS_CLASS_NAME,
         DefaultLowWaterMarkService.class.getName());
     LOGGER.info("creating watermark manager with class {}", className);
     try {
       Class<LowWaterMarkService> comonentContainerClass = (Class<LowWaterMarkService>) Class.forName(className);
-      Preconditions.checkState(comonentContainerClass.isAssignableFrom(LowWaterMarkService.class),
-          "configured class not assignable from LowWaterMarkService class");
       _instance = comonentContainerClass.newInstance();
       _instance.init(dataAccessor, clusterName,
           brokerConfig.getInt(CONFIG_OF_BROKER_POLLING_SERVER_LWMS_INTERVAL_MS,
@@ -57,6 +63,10 @@ public class LowWaterMarkServiceProvider {
     }
   }
 
+  /**
+   * fetch the current instance of low watermark service this provider created
+   * @return
+   */
   public LowWaterMarkService getInstance() {
     return _instance;
   }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java
index cb278b4..6c35b80 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java
@@ -107,6 +107,7 @@ public abstract class BaseTableDataManager implements TableDataManager {
    * <p>The new segment is added with reference count of 1, so that is never removed until a drop command comes through.
    *
    * @param immutableSegment Immutable segment to add
+   * @param dataManagerCallback callback for performing any other necessary operation for other ingestion models
    */
   @Override
   public void addSegment(ImmutableSegment immutableSegment, DataManagerCallback dataManagerCallback) {
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/InstanceDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/InstanceDataManager.java
index eba0689..8f51bd9 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/InstanceDataManager.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/InstanceDataManager.java
@@ -20,7 +20,6 @@ package org.apache.pinot.core.data.manager;
 
 import java.io.File;
 import java.util.List;
-import java.util.Map;
 import java.util.Set;
 import javax.annotation.Nullable;
 import javax.annotation.concurrent.ThreadSafe;
@@ -132,8 +131,4 @@ public interface InstanceDataManager {
    */
   ZkHelixPropertyStore<ZNRecord> getPropertyStore();
 
-//  /**
-//   * Return the mappings from partition -> low water marks of all the tables hosted in this server.
-//   */
-//  Map<String, Map<Integer, Long>> getLowWaterMarks();
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java
index 4f3ae50..dc2f748 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java
@@ -1056,7 +1056,7 @@ public class LLRealtimeSegmentDataManager extends RealtimeSegmentDataManager {
   public LLRealtimeSegmentDataManager(RealtimeSegmentZKMetadata segmentZKMetadata, TableConfig tableConfig,
       RealtimeTableDataManager realtimeTableDataManager, String resourceDataDir, IndexLoadingConfig indexLoadingConfig,
       Schema schema, LLCSegmentName llcSegmentName, Semaphore partitionConsumerSemaphore, ServerMetrics serverMetrics,
-      DataManagerCallback dataManagerCallback) {
+      DataManagerCallback dataManagerCallback) throws IOException {
     _segBuildSemaphore = realtimeTableDataManager.getSegmentBuildSemaphore();
     _segmentZKMetadata = (LLCRealtimeSegmentZKMetadata) segmentZKMetadata;
     _tableConfig = tableConfig;
@@ -1219,6 +1219,9 @@ public class LLRealtimeSegmentDataManager extends RealtimeSegmentDataManager {
 
     _segmentCommitterFactory = new SegmentCommitterFactory(segmentLogger, _indexLoadingConfig, _protocolHandler);
 
+    // init virtual columns
+    _dataManagerCallback.initVirtualColumns();
+
     segmentLogger
         .info("Starting consumption on realtime consuming segment {} maxRowCount {} maxEndTime {}", _llcSegmentName,
             _segmentMaxRowCount, new DateTime(_consumeEndTime, DateTimeZone.UTC).toString());
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java
index bc5316d..fd42627 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java
@@ -39,7 +39,6 @@ import org.apache.pinot.common.metadata.ZKMetadataProvider;
 import org.apache.pinot.common.metadata.instance.InstanceZKMetadata;
 import org.apache.pinot.common.metadata.segment.LLCRealtimeSegmentZKMetadata;
 import org.apache.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
-import org.apache.pinot.common.metrics.ServerMetrics;
 import org.apache.pinot.common.utils.CommonConstants.Segment.Realtime.Status;
 import org.apache.pinot.common.utils.LLCSegmentName;
 import org.apache.pinot.common.utils.NamedThreadFactory;
@@ -229,11 +228,13 @@ public class RealtimeTableDataManager extends BaseTableDataManager {
     // of the index directory and loading segment from it
     LoaderUtils.reloadFailureRecovery(indexDir);
 
+    // tell callback to add segment
+    _tableDataManagerCallback.addSegment(_tableNameWithType, segmentName, tableConfig);
+
     if (indexDir.exists() && (realtimeSegmentZKMetadata.getStatus() == Status.DONE)) {
       // Segment already exists on disk, and metadata has been committed. Treat it like an offline segment
-
-      DataManagerCallback callback = _tableDataManagerCallback
-          .getDataManagerCallback(_tableNameWithType, segmentName, schema, _serverMetrics, false);
+      final DataManagerCallback callback = _tableDataManagerCallback
+          .getImmutableDataManagerCallback(_tableNameWithType, segmentName, schema, _serverMetrics);
       addSegment(loadImmutableSegment(indexDir, indexLoadingConfig, schema, callback), callback);
     } else {
       // Either we don't have the segment on disk or we have not committed in ZK. We should be starting the consumer
@@ -267,7 +268,7 @@ public class RealtimeTableDataManager extends BaseTableDataManager {
         manager = new LLRealtimeSegmentDataManager(realtimeSegmentZKMetadata, tableConfig,
             this, _indexDir.getAbsolutePath(), indexLoadingConfig, schema, llcSegmentName,
             _partitionIdToSemaphoreMap.get(streamPartitionId), _serverMetrics,
-            _tableDataManagerCallback.getDataManagerCallback(_tableNameWithType, segmentName, schema, _serverMetrics, true));
+            _tableDataManagerCallback.getMutableDataManagerCallback(_tableNameWithType, segmentName, schema, _serverMetrics));
       }
       _logger.info("Initialize RealtimeSegmentDataManager - " + segmentName);
       _segmentDataManagerMap.put(segmentName, manager);
@@ -321,7 +322,7 @@ public class RealtimeTableDataManager extends BaseTableDataManager {
     try {
       File indexDir = new File(_indexDir, segmentName);
       DataManagerCallback dataManagerCallback = _tableDataManagerCallback
-          .getDataManagerCallback(_tableNameWithType, segmentName, schema, _serverMetrics, false);
+          .getImmutableDataManagerCallback(_tableNameWithType, segmentName, schema, _serverMetrics);
       addSegment(loadImmutableSegment(indexDir, indexLoadingConfig, schema, dataManagerCallback), dataManagerCallback);
     } catch (Exception e) {
       throw new RuntimeException(e);
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/DataManagerCallback.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/DataManagerCallback.java
index 466553e..91b08d2 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/DataManagerCallback.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/DataManagerCallback.java
@@ -18,26 +18,61 @@
  */
 package org.apache.pinot.core.data.manager.upsert;
 
+import org.apache.pinot.core.data.manager.SegmentDataManager;
 import org.apache.pinot.grigio.common.storageProvider.UpdateLogEntry;
-import org.apache.pinot.spi.data.Schema;
 import org.apache.pinot.spi.data.readers.GenericRow;
 
 import java.io.IOException;
 import java.util.List;
 
+/**
+ * component inject to {@link org.apache.pinot.core.data.manager.SegmentDataManager} for handling extra logics for
+ * other workflows other than regular append-mode ingestion. We are expected to provide appropriate link to class
+ * during run time
+ */
 public interface DataManagerCallback {
 
+  /**
+   * create a callback component for {@link org.apache.pinot.core.indexsegment.IndexSegment} when
+   * {@link org.apache.pinot.core.data.manager.SegmentDataManager} create one.
+   * @return callback associated with the internal index segment this data manager holds
+   */
   IndexSegmentCallback getIndexSegmentCallback();
 
+  /**
+   * process the row after transformation in the ingestion process
+   * @param row the row of newly ingested and transformed data from upstream
+   * @param offset the offset of this particular row
+   */
   void processTransformedRow(GenericRow row, long offset);
 
+  /**
+   * process the row after we have finished the index the current row
+   * @param row the row we just index to the current segment
+   * @param offset the offset associated with the index
+   */
   void postIndexProcessing(GenericRow row, long offset);
 
+  /**
+   * callback for when a realtime segment data manager done with the current consumption loop for all data associated
+   * with it
+   */
   void postConsumeLoop();
 
+  /**
+   * initialize all virtual columns for the current data manager associated with upsert component (if necessary)
+   * @throws IOException
+   */
   void initVirtualColumns() throws IOException;
 
+  /**
+   * update the data in the virtual columns from segment updater loop if necessary
+   * @param messages list of update log entries for the current datamanager
+   */
   void updateVirtualColumns(List<UpdateLogEntry> messages);
 
+  /**
+   * callback when the associated data manager is destroyed by pinot server in call {@link SegmentDataManager#destroy()}
+   */
   void destroy();
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/DefaultIndexSegmentCallback.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/DefaultIndexSegmentCallback.java
index 0299cb7..a0d9acf 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/DefaultIndexSegmentCallback.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/DefaultIndexSegmentCallback.java
@@ -27,6 +27,9 @@ import org.apache.pinot.spi.data.readers.GenericRow;
 
 import java.util.Map;
 
+/**
+ * no-op callback for non-upsert table/pinot server instance
+ */
 public class DefaultIndexSegmentCallback implements IndexSegmentCallback {
 
   public static final DefaultIndexSegmentCallback INSTANCE = new DefaultIndexSegmentCallback();
@@ -35,27 +38,22 @@ public class DefaultIndexSegmentCallback implements IndexSegmentCallback {
 
   @Override
   public void init(SegmentMetadata segmentMetadata, Map<String, DataFileReader> virtualColumnIndexReader) {
-    // do nothing
   }
 
   @Override
   public void initOffsetColumn(ColumnIndexContainer offsetColumnContainer) {
-    // do noting
   }
 
   @Override
   public void postProcessRecords(GenericRow row, int docId) {
-    // do nothing
   }
 
   @Override
   public void initVirtualColumn() {
-    // do nothing
   }
 
   @Override
   public void updateVirtualColumn(Iterable<UpdateLogEntry> logEntries) {
-    // do nothing
   }
 
   @Override
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/DefaultTableDataManagerCallbackImpl.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/DefaultTableDataManagerCallbackImpl.java
index 2e56455..a0f4398 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/DefaultTableDataManagerCallbackImpl.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/DefaultTableDataManagerCallbackImpl.java
@@ -22,6 +22,10 @@ import org.apache.pinot.common.config.TableConfig;
 import org.apache.pinot.common.metrics.ServerMetrics;
 import org.apache.pinot.spi.data.Schema;
 
+/**
+ * Class for no-op callback for pinot cluster/table that don't support upsert
+ * We will also use this for pinot tables that don't configured to use upsert semantic
+ */
 public class DefaultTableDataManagerCallbackImpl implements TableDataManagerCallback {
 
   private static final DefaultDataManagerCallbackImpl DEFAULT_DM_CALLBACK = DefaultDataManagerCallbackImpl.INSTANCE;
@@ -35,8 +39,14 @@ public class DefaultTableDataManagerCallbackImpl implements TableDataManagerCall
   }
 
   @Override
-  public DataManagerCallback getDataManagerCallback(String tableName, String segmentName,
-      Schema schema, ServerMetrics serverMetrics, boolean isMutable) {
+  public DataManagerCallback getMutableDataManagerCallback(String tableName, String segmentName,
+      Schema schema, ServerMetrics serverMetrics) {
+    return DEFAULT_DM_CALLBACK;
+  }
+
+  @Override
+  public DataManagerCallback getImmutableDataManagerCallback(String tableName, String segmentName,
+      Schema schema, ServerMetrics serverMetrics) {
     return DEFAULT_DM_CALLBACK;
   }
 
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/IndexSegmentCallback.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/IndexSegmentCallback.java
index dd198f9..8b3a14d 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/IndexSegmentCallback.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/IndexSegmentCallback.java
@@ -27,17 +27,48 @@ import org.apache.pinot.spi.data.readers.GenericRow;
 import java.io.IOException;
 import java.util.Map;
 
+/**
+ * callback for handling any upsert-related operations in subclass of
+ * {@link org.apache.pinot.core.indexsegment.IndexSegment} if necessary
+ */
 public interface IndexSegmentCallback {
 
+  /**
+   * initialize the callback from {@link org.apache.pinot.core.indexsegment.IndexSegment}
+   * @param segmentMetadata the metadata associated with the curreng segment
+   * @param virtualColumnIndexReader
+   */
   void init(SegmentMetadata segmentMetadata, Map<String, DataFileReader> virtualColumnIndexReader);
 
+  /**
+   * initialize offset column for in-memory access
+   * @param offsetColumnContainer the column that stores the offset data
+   */
   void initOffsetColumn(ColumnIndexContainer offsetColumnContainer);
 
+  /**
+   * perform any operation from the callback for the given row after it has been processed and index
+   * @param row the current pinot row we just indexed into the current IndexSegment
+   * @param docId the docId of this record
+   */
   void postProcessRecords(GenericRow row, int docId);
 
+  /**
+   * initialize set of upsert-related virtual columns if necessary
+   * @throws IOException
+   */
   void initVirtualColumn() throws IOException;
 
+  /**
+   * update upsert-related virtual column from segment updater if necessary
+   * @param logEntries
+   */
   void updateVirtualColumn(Iterable<UpdateLogEntry> logEntries);
 
+  /**
+   * retrieve a information related to an upsert-enable segment virtual column for debug purpose
+   * @param offset the offset of the record we are trying to get the virtual columnn data for
+   * @return string representation of the virtual column data information
+   */
   String getVirtualColumnInfo(long offset);
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/TableDataManagerCallback.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/TableDataManagerCallback.java
index ec6a15f..7501961 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/TableDataManagerCallback.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/TableDataManagerCallback.java
@@ -20,16 +20,49 @@ package org.apache.pinot.core.data.manager.upsert;
 
 import org.apache.pinot.common.config.TableConfig;
 import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
 import org.apache.pinot.spi.data.Schema;
 
+import java.io.File;
+
+/**
+ * component inject to {@link org.apache.pinot.core.data.manager.TableDataManager} for handling extra logics for
+ * other workflows other than regular append-mode ingestion. We are expected to provide appropriate link to class
+ * during run time
+ */
 public interface TableDataManagerCallback {
 
+  /**
+   * initialize the callback object during {@link org.apache.pinot.core.data.manager.TableDataManager#init}
+   * ensure any internal component for this callback is properly created during the start time
+   */
   void init();
 
-  void addSegment(String tableName, String segmentName, TableConfig tableConfig);
+  /**
+   * callback to ensure other components related to the callback are added when
+   * {@link org.apache.pinot.core.data.manager.TableDataManager#addSegment(File, IndexLoadingConfig)}
+   * is executed
+   */
+  void addSegment(String tableNameWithType, String segmentName, TableConfig tableConfig);
+
+  /**
+   * return a callback object for an Immutable segment data manager callback component when a table create a new
+   * immutable {@link org.apache.pinot.core.data.manager.SegmentDataManager}
+   */
+  DataManagerCallback getMutableDataManagerCallback(String tableNameWithType, String segmentName, Schema schema,
+      ServerMetrics serverMetrics);
 
-  DataManagerCallback getDataManagerCallback(String tableName, String segmentName, Schema schema,
-      ServerMetrics serverMetrics, boolean isMutable);
+  /**
+   * return a callback object for a mutable segment data manager callback component when a table create a new
+   * immutable {@link org.apache.pinot.core.data.manager.SegmentDataManager}
+   */
+  DataManagerCallback getImmutableDataManagerCallback(String tableNameWithType, String segmentName, Schema schema,
+      ServerMetrics serverMetrics);
 
+  /**
+   * create a no-op default callback for segmentDataManager that don't support upsert
+   * (eg, offline table, HLL consumers etc)
+   * @return a no-op default callback for data manager
+   */
   DataManagerCallback getDefaultDataManagerCallback();
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/TableDataManagerCallbackProvider.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/TableDataManagerCallbackProvider.java
index 2e868a2..1f08a90 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/TableDataManagerCallbackProvider.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/TableDataManagerCallbackProvider.java
@@ -18,7 +18,6 @@
  */
 package org.apache.pinot.core.data.manager.upsert;
 
-import com.google.common.base.Preconditions;
 import org.apache.commons.configuration.Configuration;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang3.exception.ExceptionUtils;
@@ -27,6 +26,10 @@ import org.apache.pinot.core.data.manager.config.TableDataManagerConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+/**
+ * class for creating appropriate {@link TableDataManagerCallback} depends on the config
+ * allow upsert-enabled pinot server to inject proper logics while keeping append-only pinot server keep the same
+ */
 public class TableDataManagerCallbackProvider {
 
   private static final Logger LOGGER = LoggerFactory.getLogger(TableDataManagerCallbackProvider.class);
@@ -38,6 +41,12 @@ public class TableDataManagerCallbackProvider {
   public static final String DEFAULT_CALLBACK_CLASS_CONFIG_KEY = "append.tableDataManager.callback";
   public static final String CALLBACK_CLASS_CONFIG_DEFAULT = DefaultTableDataManagerCallbackImpl.class.getName();
 
+  /**
+   * initialize table data manager callback provider
+   * the most information config will be {@value UPSERT_CALLBACK_CLASS_CONFIG_KEY} for creating the proper
+   * callback injection for upsert pinot server
+   * @param configuration
+   */
   public TableDataManagerCallbackProvider(Configuration configuration) {
     String appendClassName = configuration.getString(DEFAULT_CALLBACK_CLASS_CONFIG_KEY, CALLBACK_CLASS_CONFIG_DEFAULT);
     String upsertClassName = configuration.getString(UPSERT_CALLBACK_CLASS_CONFIG_KEY);
@@ -47,8 +56,6 @@ public class TableDataManagerCallbackProvider {
       LOGGER.error("failed to load table data manager class {}", appendClassName, e);
       ExceptionUtils.rethrow(e);
     }
-    Preconditions.checkState(defaultTableDataManagerCallBackClass.isAssignableFrom(TableDataManagerCallback.class),
-        "configured class not assignable from Callback class", defaultTableDataManagerCallBackClass);
     if (StringUtils.isNotEmpty(upsertClassName)) {
       try {
         upsertTableDataManagerCallBackClass = (Class<TableDataManagerCallback>) Class.forName(upsertClassName);
@@ -56,11 +63,13 @@ public class TableDataManagerCallbackProvider {
         LOGGER.error("failed to load table data manager class {}", upsertClassName);
         ExceptionUtils.rethrow(e);
       }
-      Preconditions.checkState(upsertTableDataManagerCallBackClass.isAssignableFrom(TableDataManagerCallback.class),
-          "configured class not assignable from Callback class");
     }
   }
 
+  /**
+   * create a proper callback for the table, depends on whether the table is configured for upsert or not
+   * @param tableDataManagerConfig the config for the table
+   */
   public TableDataManagerCallback getTableDataManagerCallback(TableDataManagerConfig tableDataManagerConfig) {
     if (tableDataManagerConfig.getUpdateSemantic() == CommonConstants.UpdateSemantic.UPSERT) {
       return getUpsertTableDataManagerCallback();
@@ -69,7 +78,7 @@ public class TableDataManagerCallbackProvider {
     }
   }
 
-  public TableDataManagerCallback getUpsertTableDataManagerCallback() {
+  private TableDataManagerCallback getUpsertTableDataManagerCallback() {
     try {
       return upsertTableDataManagerCallBackClass.newInstance();
     } catch (Exception ex) {
@@ -79,6 +88,9 @@ public class TableDataManagerCallbackProvider {
     return null;
   }
 
+  /**
+   * create a tabledatamanager for a non-upsert enabled tables, ensure to use the original pinot workflow
+   */
   public TableDataManagerCallback getDefaultTableDataManagerCallback() {
     try {
       return defaultTableDataManagerCallBackClass.newInstance();
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/DefaultWaterMarkManager.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/DefaultWaterMarkManager.java
index b45060c..7c53467 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/DefaultWaterMarkManager.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/DefaultWaterMarkManager.java
@@ -24,13 +24,15 @@ import org.apache.pinot.grigio.common.metrics.GrigioMetrics;
 
 import java.util.Map;
 
+/**
+ * default no-op watermark manager for pinot
+ */
 public class DefaultWaterMarkManager implements WaterMarkManager {
 
   private static final Map<String, Map<Integer, Long>> DEFAULT_MAP = ImmutableMap.of();
 
   @Override
   public void init(Configuration config, GrigioMetrics metrics) {
-
   }
 
   @Override
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/LowWaterMarkService.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/LowWaterMarkService.java
index 0c6756c..e9cc5f9 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/LowWaterMarkService.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/LowWaterMarkService.java
@@ -26,19 +26,34 @@ import java.util.Map;
 /**
  * LowWaterMarkService keeps records of the low water mark (i.e., the stream ingestion progress) for each partition of
  * an input table.
+ * It runs on pinot broker to fetch lwm information from pinot server periodically
+ * and use that to rewrite pinot query periodically
  */
 public interface LowWaterMarkService {
 
-    void init(HelixDataAccessor helixDataAccessor, String helixClusterName, int serverPollingInterval, int serverPort);
-
-    // Return the low water mark mapping from partition id to the corresponding low water mark of a given table.
-    Map<Integer, Long> getLowWaterMarks(String tableName);
-
-    // Shutdown the service.
-    void shutDown();
-
-    // start
-    void start(BrokerMetrics brokerMetrics);
-
-    UpsertQueryRewriter getQueryRewriter();
+  void init(HelixDataAccessor helixDataAccessor, String helixClusterName, int serverPollingInterval, int serverPort);
+
+  /**
+   * the low water mark mapping from partition id to the corresponding low water mark of a given table.
+   * @param tableNameWithType
+   * @return map of partition to lowWatermark
+   */
+  Map<Integer, Long> getLowWaterMarks(String tableNameWithType);
+
+  /**
+   * shutdown low water mark service and its background threads (if any)
+   */
+  void shutDown();
+
+  /**
+   * start the current low watermark service
+   * @param brokerMetrics pinot broker metrics for lwm service to report its status to
+   */
+  void start(BrokerMetrics brokerMetrics);
+
+  /**
+   * get a queryrewriter to ensure that we can rewrite a query if the target table is upsert-enabled table
+   * @return
+   */
+  QueryRewriter getQueryRewriter();
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/UpsertQueryRewriter.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/QueryRewriter.java
similarity index 71%
rename from pinot-core/src/main/java/org/apache/pinot/core/segment/updater/UpsertQueryRewriter.java
rename to pinot-core/src/main/java/org/apache/pinot/core/segment/updater/QueryRewriter.java
index 64a64d0..40abd9a 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/UpsertQueryRewriter.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/QueryRewriter.java
@@ -1,7 +1,3 @@
-package org.apache.pinot.core.segment.updater;
-
-import org.apache.pinot.common.request.BrokerRequest;
-
 /**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -20,8 +16,20 @@ import org.apache.pinot.common.request.BrokerRequest;
  * specific language governing permissions and limitations
  * under the License.
  */
-public interface UpsertQueryRewriter {
+package org.apache.pinot.core.segment.updater;
+
+import org.apache.pinot.common.request.BrokerRequest;
+
+/**
+ * class that rewrite pinot broker sql for upsert or other purpose
+ */
+public interface QueryRewriter {
 
-  void rewriteQueryForUpsert(BrokerRequest request, String rawTableName);
+  /**
+   * rewrite the query for pinot upsert table if necessary
+   * @param request the pinot sql request that pinot broker requests
+   * @param rawTableName the raw
+   */
+  void maybeRewriteQueryForUpsert(BrokerRequest request, String rawTableName);
 
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/UpsertComponentContainer.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/UpsertComponentContainer.java
index eb64285..f1312e7 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/UpsertComponentContainer.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/UpsertComponentContainer.java
@@ -22,21 +22,55 @@ import com.yammer.metrics.core.MetricsRegistry;
 import org.apache.commons.configuration.Configuration;
 import org.apache.helix.HelixManager;
 
+/**
+ * contains all components related to upsert in pinot server
+ */
 public interface UpsertComponentContainer {
 
+  /**
+   * register pinot upsert component metrics to the given registry
+   * @param prefix the prefix of all metrics
+   * @param registry the registry we are going to register the metrics to
+   */
   void registerMetrics(String prefix, MetricsRegistry registry);
 
+  /**
+   * initialize the upsert comonent container with necessary config and information
+   * @param config the configuration for this upsert component
+   * @param helixManager helix manager for the current pinot server helix state
+   * @param clusterName helix cluster name for the current pinot cluster
+   * @param instanceName the name of current pinot instance in this cluster
+   */
   void init(Configuration config, HelixManager helixManager, String clusterName, String instanceName);
 
+  /**
+   * start any necessary background processing for this upsert component
+   */
   void startBackgroundThread();
 
+  /**
+   * stop any necessary background processing for this upsert component
+   */
   void stopBackgroundThread();
 
+  /**
+   * shutdown and clean up any state for this upsert component
+   */
   void shutdown();
 
+  /**
+   * return a segment deletion callback component that should be invoked when pinot server removed a segment
+   * from its internal storage (to DROPPED state)
+   */
   SegmentDeletionHandler getSegmentDeletionHandler();
 
+  /**
+   * return the current watermark manager for this server
+   */
   WaterMarkManager getWatermarkManager();
 
+  /**
+   * check if upsert is enable for the current pinot server
+   */
   boolean isUpsertEnabled();
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/WaterMarkManager.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/WaterMarkManager.java
index acc8479..63ba5c9 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/WaterMarkManager.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/WaterMarkManager.java
@@ -19,16 +19,37 @@
 package org.apache.pinot.core.segment.updater;
 
 import org.apache.commons.configuration.Configuration;
-import org.apache.pinot.grigio.common.metrics.GrigioMeter;
 import org.apache.pinot.grigio.common.metrics.GrigioMetrics;
 
 import java.util.Map;
 
+/**
+ * class run on pinot server to keep track of the low-water-mark of each upsert table
+ * organized by partition
+ */
 public interface WaterMarkManager {
 
+  /**
+   * initialize watermark manager
+   * @param config the configuration subset for waterMarkManager
+   * @param metrics the metrics for watermark manager
+   */
   void init(Configuration config, GrigioMetrics metrics);
 
+  /**
+   * the highest epoch for each partition of each table in this pinot server
+   * @return mapping of {pinot_table_name: {partition_id: high_water_mark}}
+   * example as {
+   *     "table1_REALTIME" : {
+   *       "0" : 1400982,
+   *       "1" : 1400982,
+   *       "2" : 1400982,
+   *       "3" : 1400982
+   *     },
+   *     "table2_REALTIME" : {
+   *       "0" : 1401008,
+   *       "1" : 1401008
+   *     }
+   */
   Map<String, Map<Integer, Long>> getHighWaterMarkTablePartitionMap();
-
-
 }
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/segment/index/creator/SegmentGenerationWithNullValueVectorTest.java b/pinot-core/src/test/java/org/apache/pinot/core/segment/index/creator/SegmentGenerationWithNullValueVectorTest.java
index 4d1d469..a56f897 100644
--- a/pinot-core/src/test/java/org/apache/pinot/core/segment/index/creator/SegmentGenerationWithNullValueVectorTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/core/segment/index/creator/SegmentGenerationWithNullValueVectorTest.java
@@ -40,6 +40,7 @@ import org.apache.pinot.common.request.InstanceRequest;
 import org.apache.pinot.common.utils.DataTable;
 import org.apache.pinot.core.data.manager.InstanceDataManager;
 import org.apache.pinot.core.data.manager.TableDataManager;
+import org.apache.pinot.core.data.manager.config.InstanceDataManagerConfig;
 import org.apache.pinot.core.data.manager.config.TableDataManagerConfig;
 import org.apache.pinot.core.data.manager.offline.TableDataManagerProvider;
 import org.apache.pinot.core.data.manager.upsert.DefaultDataManagerCallbackImpl;
@@ -137,6 +138,7 @@ public class SegmentGenerationWithNullValueVectorTest {
     when(tableDataManagerConfig.getTableDataManagerType()).thenReturn("OFFLINE");
     when(tableDataManagerConfig.getTableName()).thenReturn(TABLE_NAME);
     when(tableDataManagerConfig.getDataDir()).thenReturn(FileUtils.getTempDirectoryPath());
+    TableDataManagerProvider.init(mock(InstanceDataManagerConfig.class));
     @SuppressWarnings("unchecked")
     TableDataManager tableDataManager = TableDataManagerProvider
         .getTableDataManager(tableDataManagerConfig, "testInstance", mock(ZkHelixPropertyStore.class),
diff --git a/pinot-core/src/test/java/org/apache/pinot/query/executor/QueryExecutorTest.java b/pinot-core/src/test/java/org/apache/pinot/query/executor/QueryExecutorTest.java
index 203d2f6..2eda11f 100644
--- a/pinot-core/src/test/java/org/apache/pinot/query/executor/QueryExecutorTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/query/executor/QueryExecutorTest.java
@@ -34,6 +34,7 @@ import org.apache.pinot.common.segment.ReadMode;
 import org.apache.pinot.common.utils.DataTable;
 import org.apache.pinot.core.data.manager.InstanceDataManager;
 import org.apache.pinot.core.data.manager.TableDataManager;
+import org.apache.pinot.core.data.manager.config.InstanceDataManagerConfig;
 import org.apache.pinot.core.data.manager.config.TableDataManagerConfig;
 import org.apache.pinot.core.data.manager.offline.TableDataManagerProvider;
 import org.apache.pinot.core.data.manager.upsert.DefaultDataManagerCallbackImpl;
@@ -98,6 +99,7 @@ public class QueryExecutorTest {
     when(tableDataManagerConfig.getTableDataManagerType()).thenReturn("OFFLINE");
     when(tableDataManagerConfig.getTableName()).thenReturn(TABLE_NAME);
     when(tableDataManagerConfig.getDataDir()).thenReturn(FileUtils.getTempDirectoryPath());
+    TableDataManagerProvider.init(mock(InstanceDataManagerConfig.class));
     @SuppressWarnings("unchecked")
     TableDataManager tableDataManager = TableDataManagerProvider
         .getTableDataManager(tableDataManagerConfig, "testInstance", mock(ZkHelixPropertyStore.class),
diff --git a/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/broker/upsert/PollingBasedLowWaterMarkService.java b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/broker/upsert/PollingBasedLowWaterMarkService.java
index 74e4b41..c92d53e 100644
--- a/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/broker/upsert/PollingBasedLowWaterMarkService.java
+++ b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/broker/upsert/PollingBasedLowWaterMarkService.java
@@ -30,7 +30,7 @@ import org.apache.pinot.common.metrics.BrokerMeter;
 import org.apache.pinot.common.metrics.BrokerMetrics;
 import org.apache.pinot.common.restlet.resources.TableLowWaterMarksInfo;
 import org.apache.pinot.core.segment.updater.LowWaterMarkService;
-import org.apache.pinot.core.segment.updater.UpsertQueryRewriter;
+import org.apache.pinot.core.segment.updater.QueryRewriter;
 import org.glassfish.jersey.client.ClientProperties;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -67,7 +67,7 @@ public class PollingBasedLowWaterMarkService implements LowWaterMarkService {
   private int _serverPort;
   private boolean _shuttingDown;
   private BrokerMetrics _brokerMetrics;
-  private UpsertQueryRewriter _queryRewriter;
+  private QueryRewriter _queryRewriter;
 
   @Override
   public void init(HelixDataAccessor helixDataAccessor, String helixClusterName, int serverPollingInterval, int serverPort) {
@@ -104,7 +104,7 @@ public class PollingBasedLowWaterMarkService implements LowWaterMarkService {
   }
 
   @Override
-  public UpsertQueryRewriter getQueryRewriter() {
+  public QueryRewriter getQueryRewriter() {
     return _queryRewriter;
   }
 
diff --git a/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/broker/upsert/UpsertQueryRewriterImpl.java b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/broker/upsert/UpsertQueryRewriterImpl.java
index 90fce7c..a9cd1f0 100644
--- a/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/broker/upsert/UpsertQueryRewriterImpl.java
+++ b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/broker/upsert/UpsertQueryRewriterImpl.java
@@ -25,7 +25,7 @@ import org.apache.pinot.common.request.FilterQuery;
 import org.apache.pinot.common.request.FilterQueryMap;
 import org.apache.pinot.common.utils.CommonConstants;
 import org.apache.pinot.core.segment.updater.LowWaterMarkService;
-import org.apache.pinot.core.segment.updater.UpsertQueryRewriter;
+import org.apache.pinot.core.segment.updater.QueryRewriter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -34,7 +34,7 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
-public class UpsertQueryRewriterImpl implements UpsertQueryRewriter {
+public class UpsertQueryRewriterImpl implements QueryRewriter {
   private static final Logger LOGGER = LoggerFactory.getLogger(UpsertQueryRewriterImpl.class);
 
   protected final LowWaterMarkService _lwmService;
@@ -48,7 +48,7 @@ public class UpsertQueryRewriterImpl implements UpsertQueryRewriter {
   }
 
   @Override
-  public void rewriteQueryForUpsert(BrokerRequest request, String rawTableName) {
+  public void maybeRewriteQueryForUpsert(BrokerRequest request, String rawTableName) {
     final String realtimeTableName = TableNameBuilder.ensureTableNameWithType(rawTableName,
         CommonConstants.Helix.TableType.REALTIME);
     Map<Integer, Long> lowWaterMarks = _lwmService.getLowWaterMarks(realtimeTableName);
diff --git a/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/data/manager/upsert/UpsertTableDataManagerCallbackImpl.java b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/data/manager/upsert/UpsertTableDataManagerCallbackImpl.java
index dc329e4..efbbf00 100644
--- a/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/data/manager/upsert/UpsertTableDataManagerCallbackImpl.java
+++ b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/data/manager/upsert/UpsertTableDataManagerCallbackImpl.java
@@ -18,7 +18,6 @@
  */
 package org.apache.pinot.core.data.manager.upsert;
 
-
 import org.apache.commons.lang3.NotImplementedException;
 import org.apache.commons.lang3.exception.ExceptionUtils;
 import org.apache.pinot.common.config.TableConfig;
@@ -30,6 +29,10 @@ import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 
+/**
+ * class for handle all upsert related operation for interacting with segments for a given table at
+ * {@link org.apache.pinot.core.data.manager.TableDataManager}
+ */
 public class UpsertTableDataManagerCallbackImpl implements TableDataManagerCallback {
   private static final Logger LOGGER = LoggerFactory.getLogger(UpsertTableDataManagerCallbackImpl.class);
 
@@ -51,7 +54,18 @@ public class UpsertTableDataManagerCallbackImpl implements TableDataManagerCallb
   }
 
   @Override
-  public DataManagerCallback getDataManagerCallback(String tableName, String segmentName, Schema schema,
+  public DataManagerCallback getMutableDataManagerCallback(String tableNameWithType, String segmentName, Schema schema,
+      ServerMetrics serverMetrics) {
+    return getDataManagerCallback(tableNameWithType, segmentName, schema, serverMetrics, true);
+  }
+
+  @Override
+  public DataManagerCallback getImmutableDataManagerCallback(String tableNameWithType, String segmentName,
+      Schema schema, ServerMetrics serverMetrics) {
+    return getDataManagerCallback(tableNameWithType, segmentName, schema, serverMetrics, false);
+  }
+
+  private DataManagerCallback getDataManagerCallback(String tableName, String segmentName, Schema schema,
       ServerMetrics serverMetrics, boolean isMutable) {
     return new UpsertDataManagerCallbackImpl(tableName, segmentName, schema, serverMetrics, isMutable);
   }
diff --git a/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/segment/updater/UpsertWaterMarkManager.java b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/segment/updater/UpsertWaterMarkManager.java
index 266f25d2..4b3d861 100644
--- a/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/segment/updater/UpsertWaterMarkManager.java
+++ b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/segment/updater/UpsertWaterMarkManager.java
@@ -31,6 +31,14 @@ import org.slf4j.LoggerFactory;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
+/**
+ * watermark manager for upsert component to collect the low-water-mark information of each tables in the current
+ * pinot server
+ * watermark is defined as largest version of each partition (segment update event topic partition) for each table
+ * so it stores the data in map of {table_name: {partition_id: highest_water_mark}}
+ * then {@link LowWaterMarkService} will ingest those information from pinot server and calculate the lowest of these
+ * watermark and use it in query to send to server
+ */
 public class UpsertWaterMarkManager implements WaterMarkManager {
 
   private final Map<String, Map<Integer, Long>> _highWaterMarkTablePartitionMap = new ConcurrentHashMap<>();
@@ -55,7 +63,12 @@ public class UpsertWaterMarkManager implements WaterMarkManager {
     return _instance;
   }
 
-  // TODO(tingchen) Look into the case where Segment Update Messages might arrive before the corresponding physical data.
+  /**
+   * process a event message and update the current watermark information for this manager
+   * @param table
+   * @param segment
+   * @param logEntry the message containing the new watermark information
+   */
   public void processMessage(String table, String segment, UpdateLogEntry logEntry) {
     if (logEntry == null) {
       return;
@@ -65,6 +78,12 @@ public class UpsertWaterMarkManager implements WaterMarkManager {
     processVersionUpdate(table, partition, version);
   }
 
+  /**
+   * update the high watermark information associated with the given table/partition
+   * @param table
+   * @param partition
+   * @param version
+   */
   public void processVersionUpdate(String table, int partition, long version) {
     Preconditions.checkState(partition >= 0, "logEntry has invalid version {} for table {}",
         version, table);
@@ -76,6 +95,11 @@ public class UpsertWaterMarkManager implements WaterMarkManager {
     }
   }
 
+  /**
+   * return the highest watermark for each partition of the given table
+   * @param tableName
+   * @return
+   */
   public Map<Integer, Long> getHighWaterMarkForTable(String tableName) {
     return ImmutableMap.copyOf(_highWaterMarkTablePartitionMap.getOrDefault(tableName, ImmutableMap.of()));
   }
diff --git a/pinot-grigio/pinot-grigio-provided/src/test/java/org/apache/pinot/broker/upsert/UpsertQueryRewriterImplTest.java b/pinot-grigio/pinot-grigio-provided/src/test/java/org/apache/pinot/broker/upsert/QueryRewriterImplTest.java
similarity index 99%
rename from pinot-grigio/pinot-grigio-provided/src/test/java/org/apache/pinot/broker/upsert/UpsertQueryRewriterImplTest.java
rename to pinot-grigio/pinot-grigio-provided/src/test/java/org/apache/pinot/broker/upsert/QueryRewriterImplTest.java
index ac6c38d..5e6b703 100644
--- a/pinot-grigio/pinot-grigio-provided/src/test/java/org/apache/pinot/broker/upsert/UpsertQueryRewriterImplTest.java
+++ b/pinot-grigio/pinot-grigio-provided/src/test/java/org/apache/pinot/broker/upsert/QueryRewriterImplTest.java
@@ -33,7 +33,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 
-public class UpsertQueryRewriterImplTest {
+public class QueryRewriterImplTest {
 
     private LowWaterMarkService _lwms;
     private UpsertQueryRewriterImpl rewriter;
diff --git a/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixInstanceDataManager.java b/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixInstanceDataManager.java
index 710e22a..b06000f 100644
--- a/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixInstanceDataManager.java
+++ b/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixInstanceDataManager.java
@@ -221,7 +221,7 @@ public class HelixInstanceDataManager implements InstanceDataManager {
 
       final TableDataManager tableDataManager = _tableDataManagerMap.get(tableNameWithType);
       final DataManagerCallback dataManagerCallback = tableDataManager.getTableDataManagerCallback()
-          .getDataManagerCallback(tableNameWithType, segmentName, schema, _serverMetrics, false);
+          .getImmutableDataManagerCallback(tableNameWithType, segmentName, schema, _serverMetrics);
 
       // Load from index directory
       ImmutableSegment immutableSegment = ImmutableSegmentLoader.load(indexDir,
diff --git a/pinot-server/src/main/java/org/apache/pinot/server/upsert/UpsertComponentContainerProvider.java b/pinot-server/src/main/java/org/apache/pinot/server/upsert/UpsertComponentContainerProvider.java
index 7ae27a6..ab76e36 100644
--- a/pinot-server/src/main/java/org/apache/pinot/server/upsert/UpsertComponentContainerProvider.java
+++ b/pinot-server/src/main/java/org/apache/pinot/server/upsert/UpsertComponentContainerProvider.java
@@ -18,10 +18,8 @@
  */
 package org.apache.pinot.server.upsert;
 
-import com.google.common.base.Preconditions;
 import org.apache.commons.lang3.exception.ExceptionUtils;
 import org.apache.pinot.core.segment.updater.UpsertComponentContainer;
-import org.apache.pinot.core.segment.updater.WaterMarkManager;
 import org.apache.pinot.server.conf.ServerConf;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -37,8 +35,6 @@ public class UpsertComponentContainerProvider {
     LOGGER.info("creating watermark manager with class {}", className);
     try {
       Class<UpsertComponentContainer> comonentContainerClass = (Class<UpsertComponentContainer>) Class.forName(className);
-      Preconditions.checkState(comonentContainerClass.isAssignableFrom(WaterMarkManager.class),
-          "configured class not assignable from Callback class");
       _instance = comonentContainerClass.newInstance();
     } catch (Exception e) {
       LOGGER.error("failed to load watermark manager class", className, e);
diff --git a/pinot-server/src/test/java/org/apache/pinot/server/api/BaseResourceTest.java b/pinot-server/src/test/java/org/apache/pinot/server/api/BaseResourceTest.java
index 60702c4..aec1510 100644
--- a/pinot-server/src/test/java/org/apache/pinot/server/api/BaseResourceTest.java
+++ b/pinot-server/src/test/java/org/apache/pinot/server/api/BaseResourceTest.java
@@ -33,6 +33,7 @@ import org.apache.pinot.common.segment.ReadMode;
 import org.apache.pinot.common.utils.CommonConstants;
 import org.apache.pinot.core.data.manager.InstanceDataManager;
 import org.apache.pinot.core.data.manager.TableDataManager;
+import org.apache.pinot.core.data.manager.config.InstanceDataManagerConfig;
 import org.apache.pinot.core.data.manager.config.TableDataManagerConfig;
 import org.apache.pinot.core.data.manager.offline.TableDataManagerProvider;
 import org.apache.pinot.core.data.manager.upsert.DefaultDataManagerCallbackImpl;
@@ -41,6 +42,7 @@ import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
 import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
 import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver;
 import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.core.segment.updater.DefaultWaterMarkManager;
 import org.apache.pinot.segments.v1.creator.SegmentTestUtils;
 import org.apache.pinot.server.starter.ServerInstance;
 import org.apache.pinot.server.starter.helix.AdminApiApplication;
@@ -83,6 +85,7 @@ public abstract class BaseResourceTest {
 
     // Mock the server instance
     ServerInstance serverInstance = mock(ServerInstance.class);
+    when(serverInstance.getWatermarkManager()).thenReturn(new DefaultWaterMarkManager());
     when(serverInstance.getInstanceDataManager()).thenReturn(instanceDataManager);
 
     // Add the default table and segment
@@ -133,6 +136,7 @@ public abstract class BaseResourceTest {
     when(tableDataManagerConfig.getTableDataManagerType()).thenReturn("OFFLINE");
     when(tableDataManagerConfig.getTableName()).thenReturn(tableName);
     when(tableDataManagerConfig.getDataDir()).thenReturn(FileUtils.getTempDirectoryPath());
+    TableDataManagerProvider.init(mock(InstanceDataManagerConfig.class));
     TableDataManager tableDataManager = TableDataManagerProvider
         .getTableDataManager(tableDataManagerConfig, "testInstance", mock(ZkHelixPropertyStore.class),
             mock(ServerMetrics.class));


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


[incubator-pinot] 06/09: more refactor

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jamesshao pushed a commit to branch upsert-refactor
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git

commit 18a2da4457a75ee34cbb3bb854371706e55f1b51
Author: james Shao <sj...@uber.com>
AuthorDate: Thu Mar 19 10:45:11 2020 -0700

    more refactor
---
 .../requesthandler/BaseBrokerRequestHandler.java   |  14 +--
 .../requesthandler/LowWaterMarkQueryWriter.java    | 109 ---------------------
 .../broker/upsert/DefaultLowWaterMarkService.java  |   8 ++
 ...ervice.java => DefaultUpsertQueryRewriter.java} |  33 ++-----
 .../apache/pinot/common/utils/CommonConstants.java |   2 +-
 .../core/segment/updater/LowWaterMarkService.java  |   2 +
 ...erMarkService.java => UpsertQueryRewriter.java} |  27 ++---
 .../upsert/PollingBasedLowWaterMarkService.java    |   8 ++
 .../broker/upsert/UpsertQueryRewriterImpl.java     |  42 ++++++--
 .../broker/upsert/UpsertQueryRewriterImplTest.java |  21 +++-
 10 files changed, 83 insertions(+), 183 deletions(-)

diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java
index 1d8f97b..cb17740 100644
--- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java
@@ -300,7 +300,7 @@ public abstract class BaseBrokerRequestHandler implements BrokerRequestHandler {
 
     if (shouldEnableLowWaterMarkRewrite(request)) {
       // Augment the realtime request with LowWaterMark constraints.
-      addLowWaterMarkToQuery(realtimeBrokerRequest, rawTableName);
+      _lwmService.getQueryRewriter().rewriteQueryForUpsert(realtimeBrokerRequest, rawTableName);
     }
 
     // Calculate routing table for the query
@@ -786,18 +786,6 @@ public abstract class BaseBrokerRequestHandler implements BrokerRequestHandler {
     }
   }
 
-  private void addLowWaterMarkToQuery(BrokerRequest realtimeBrokerRequest, String rawTableName) {
-    final String realtimeTableName = rawTableName + "_REALTIME";
-    Map<Integer, Long> lowWaterMarks = _lwmService.getLowWaterMarks(realtimeTableName);
-    if (lowWaterMarks == null || lowWaterMarks.size() == 0) {
-      LOGGER.info("No low water marks info found for table {}", realtimeTableName);
-      return;
-    }
-    LOGGER.info("Found low water marks {} for table {}", String.valueOf(lowWaterMarks), realtimeTableName);
-    LowWaterMarkQueryWriter.addLowWaterMarkToQuery(realtimeBrokerRequest, lowWaterMarks);
-    LOGGER.info("Query augmented with LWMS info for table {} : {}", realtimeTableName, realtimeBrokerRequest);
-  }
-
   /**
    * Processes the optimized broker requests for both OFFLINE and REALTIME table.
    */
diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/LowWaterMarkQueryWriter.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/LowWaterMarkQueryWriter.java
index c3285e5..b373243 100644
--- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/LowWaterMarkQueryWriter.java
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/LowWaterMarkQueryWriter.java
@@ -39,113 +39,4 @@ public class LowWaterMarkQueryWriter {
   // Normal Pinot query node uses positive IDs. So lwm query node ids are all negative.
   private static final int QUERY_ID_BASE = -1000;
 
-  /**
-   * For upsert enabled tables, augment the realtime query with low water mark constraints in its filter query of the
-   * form
-   *   ($validFrom <= lwm and $validFrom > -1) AND (lwm < $validUtil OR $validUtil = -1)
-   *
-   * @param realtimeBrokerRequest
-   * @param lowWaterMarks
-   */
-  public static void addLowWaterMarkToQuery(BrokerRequest realtimeBrokerRequest, Map<Integer, Long> lowWaterMarks) {
-    if (lowWaterMarks == null || lowWaterMarks.size() == 0) {
-      LOGGER.warn("No low water mark info found for query: {}", realtimeBrokerRequest);
-      return;
-    }
-
-    // Choose the min lwm among all partitions.
-    long minLwm = Collections.min(lowWaterMarks.values());
-
-    // 1. Build the low water mark query of the form for a table assuming lwm is the min LWM and -1 is used as
-    // uninitialized marker.
-    // ($validFrom <= lwm and $validFrom > -1) AND (lwm < $validUtil OR $validUtil = -1)
-    // -1 is used instead of Long.MAXVALUE because Pinot does not handle long arithmetic correctly.
-    FilterQuery lwmQuery = addSinglePartitionLowWaterMark(QUERY_ID_BASE - 1, realtimeBrokerRequest, minLwm);
-
-    // 2. Attach low water mark filter to the current filters.
-    FilterQuery currentFilterQuery = realtimeBrokerRequest.getFilterQuery();
-    if (currentFilterQuery != null) {
-      // Make an AND query of lwmQuery and the existing query.
-      FilterQuery andFilterQuery = new FilterQuery();
-      // Make sure we do not reuse any query id in lwmQuerys.
-      andFilterQuery.setId(QUERY_ID_BASE);
-      andFilterQuery.setOperator(FilterOperator.AND);
-      List<Integer> nestedFilterQueryIds = new ArrayList<>(2);
-      nestedFilterQueryIds.add(currentFilterQuery.getId());
-      nestedFilterQueryIds.add(lwmQuery.getId());
-      andFilterQuery.setNestedFilterQueryIds(nestedFilterQueryIds);
-
-      realtimeBrokerRequest.setFilterQuery(andFilterQuery);
-      FilterQueryMap filterSubQueryMap = realtimeBrokerRequest.getFilterSubQueryMap();
-      filterSubQueryMap.putToFilterQueryMap(lwmQuery.getId(), lwmQuery);
-      filterSubQueryMap.putToFilterQueryMap(andFilterQuery.getId(), andFilterQuery);
-    } else {
-      realtimeBrokerRequest.getFilterSubQueryMap().putToFilterQueryMap(lwmQuery.getId(), lwmQuery);
-      realtimeBrokerRequest.setFilterQuery(lwmQuery);
-    }
-  }
-
-  /**
-   *
-   * @param queryIdBase The starting id that will be assigned to the first query created in ths method.
-   * @param realtimeBrokerRequest
-   * @param lwm low water mark.
-   * @return a filter query corresponding to the low water mark constraint of a single partition. The general form is:
-   *         ($ValidFrom <= lwm && $validFrom > -1)  AND (lwm < $validUtil OR $validUtil = -1)
-   */
-  private static FilterQuery addSinglePartitionLowWaterMark(int queryIdBase, BrokerRequest realtimeBrokerRequest,
-      Long lwm) {
-    // ValidFromQuery: ($ValidFrom <= lwm && $validFrom > -1)
-    FilterQuery validFromFilterQuery = new FilterQuery();
-    // Important: Always decrement queryIdBase value after use to avoid id conflict.
-    validFromFilterQuery.setId(queryIdBase--);
-    validFromFilterQuery.setOperator(FilterOperator.AND);
-    FilterQuery validFromP1 = getLeafFilterQuery(VALID_FROM, queryIdBase--, "(*\t\t" + lwm + "]", FilterOperator.RANGE, realtimeBrokerRequest);
-    FilterQuery validFromP2 = getLeafFilterQuery(VALID_FROM, queryIdBase--, "(-1\t\t*)", FilterOperator.RANGE, realtimeBrokerRequest);
-    List<Integer> nestedQueriesIdForValidFrom = new ArrayList<>();
-    nestedQueriesIdForValidFrom.add(validFromP1.getId());
-    nestedQueriesIdForValidFrom.add(validFromP2.getId());
-    validFromFilterQuery.setNestedFilterQueryIds(nestedQueriesIdForValidFrom);
-
-    // ValidUtilQuery: (lwm < $validUtil OR $validUtil = -1)
-    FilterQuery validUtilFilterQuery = new FilterQuery();
-    validUtilFilterQuery.setId(queryIdBase--);
-    validUtilFilterQuery.setOperator(FilterOperator.OR);
-
-    FilterQuery validUtilP1 = getLeafFilterQuery(VALID_UNTIL, queryIdBase--, "(" + lwm + "\t\t*)", FilterOperator.RANGE, realtimeBrokerRequest);
-    FilterQuery validUtilP2 = getLeafFilterQuery(VALID_UNTIL, queryIdBase--, "-1", FilterOperator.EQUALITY, realtimeBrokerRequest);
-    List<Integer> nestedQueriesIdForValidUtil = new ArrayList<>();
-    nestedQueriesIdForValidUtil.add(validUtilP1.getId());
-    nestedQueriesIdForValidUtil.add(validUtilP2.getId());
-    validUtilFilterQuery.setNestedFilterQueryIds(nestedQueriesIdForValidUtil);
-
-    // Top level query: ValidFromQuery AND ValidUtilQuery
-    FilterQuery lwmQuery = new FilterQuery();
-    lwmQuery.setId(queryIdBase--);
-    lwmQuery.setOperator(FilterOperator.AND);
-    List<Integer> nestQids = new ArrayList<>();
-    nestQids.add(validFromFilterQuery.getId());
-    nestQids.add(validUtilFilterQuery.getId());
-    lwmQuery.setNestedFilterQueryIds(nestQids);
-
-    // Add all the new created queries to the query map.
-    realtimeBrokerRequest.getFilterSubQueryMap().putToFilterQueryMap(lwmQuery.getId(), lwmQuery);
-    realtimeBrokerRequest.getFilterSubQueryMap().putToFilterQueryMap(validFromFilterQuery.getId(), validFromFilterQuery);
-    realtimeBrokerRequest.getFilterSubQueryMap().putToFilterQueryMap(validUtilFilterQuery.getId(), validUtilFilterQuery);
-    return lwmQuery;
-  }
-
-  private static FilterQuery getLeafFilterQuery(String column, int id, String value, FilterOperator operator,
-      BrokerRequest realtimeBrokerRequest) {
-    FilterQuery filterQuery = new FilterQuery();
-    filterQuery.setColumn(column);
-    filterQuery.setId(id);
-    filterQuery.setValue(Collections.singletonList(value));
-    filterQuery.setOperator(operator);
-    if (realtimeBrokerRequest.getFilterSubQueryMap() == null) {
-      realtimeBrokerRequest.setFilterSubQueryMap(new FilterQueryMap());
-    }
-    realtimeBrokerRequest.getFilterSubQueryMap().putToFilterQueryMap(id, filterQuery);
-    return filterQuery;
-  }
 }
diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/upsert/DefaultLowWaterMarkService.java b/pinot-broker/src/main/java/org/apache/pinot/broker/upsert/DefaultLowWaterMarkService.java
index 94b3be4..42e1dcb 100644
--- a/pinot-broker/src/main/java/org/apache/pinot/broker/upsert/DefaultLowWaterMarkService.java
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/upsert/DefaultLowWaterMarkService.java
@@ -22,11 +22,14 @@ import com.google.common.collect.ImmutableMap;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.pinot.common.metrics.BrokerMetrics;
 import org.apache.pinot.core.segment.updater.LowWaterMarkService;
+import org.apache.pinot.core.segment.updater.UpsertQueryRewriter;
 
 import java.util.Map;
 
 public class DefaultLowWaterMarkService implements LowWaterMarkService {
 
+  private UpsertQueryRewriter upsertQueryRewriter = new DefaultUpsertQueryRewriter();
+
   @Override
   public void init(HelixDataAccessor helixDataAccessor, String helixClusterName, int serverPollingInterval,
       int serverPort){
@@ -44,4 +47,9 @@ public class DefaultLowWaterMarkService implements LowWaterMarkService {
   @Override
   public void start(BrokerMetrics brokerMetrics) {
   }
+
+  @Override
+  public UpsertQueryRewriter getQueryRewriter() {
+    return upsertQueryRewriter;
+  }
 }
diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/upsert/DefaultLowWaterMarkService.java b/pinot-broker/src/main/java/org/apache/pinot/broker/upsert/DefaultUpsertQueryRewriter.java
similarity index 56%
copy from pinot-broker/src/main/java/org/apache/pinot/broker/upsert/DefaultLowWaterMarkService.java
copy to pinot-broker/src/main/java/org/apache/pinot/broker/upsert/DefaultUpsertQueryRewriter.java
index 94b3be4..d18a56f 100644
--- a/pinot-broker/src/main/java/org/apache/pinot/broker/upsert/DefaultLowWaterMarkService.java
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/upsert/DefaultUpsertQueryRewriter.java
@@ -1,3 +1,8 @@
+package org.apache.pinot.broker.upsert;
+
+import org.apache.pinot.common.request.BrokerRequest;
+import org.apache.pinot.core.segment.updater.UpsertQueryRewriter;
+
 /**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -16,32 +21,10 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.broker.upsert;
-
-import com.google.common.collect.ImmutableMap;
-import org.apache.helix.HelixDataAccessor;
-import org.apache.pinot.common.metrics.BrokerMetrics;
-import org.apache.pinot.core.segment.updater.LowWaterMarkService;
-
-import java.util.Map;
-
-public class DefaultLowWaterMarkService implements LowWaterMarkService {
-
-  @Override
-  public void init(HelixDataAccessor helixDataAccessor, String helixClusterName, int serverPollingInterval,
-      int serverPort){
-  }
-
-  @Override
-  public Map<Integer, Long> getLowWaterMarks(String tableName) {
-    return ImmutableMap.of();
-  }
-
-  @Override
-  public void shutDown() {
-  }
+public class DefaultUpsertQueryRewriter implements UpsertQueryRewriter {
 
   @Override
-  public void start(BrokerMetrics brokerMetrics) {
+  public void rewriteQueryForUpsert(BrokerRequest request, String rawTableName) {
+    // do nothing
   }
 }
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/CommonConstants.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/CommonConstants.java
index 0e65779..f5eeb15 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/utils/CommonConstants.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/CommonConstants.java
@@ -179,7 +179,7 @@ public class CommonConstants {
     public static final String CONFIG_OF_BROKER_POLLING_SERVER_LWMS_SERVER_PORT = "pinot.broker.query.polling.server.lwms.port";
     public static final String CONFIG_OF_BROKER_LWMS_CLASS_NAME = "pinot.broker.lwms.classname";
     public static final String CONFIG_OF_BROKER_LWM_REWRITE_ENABLE = "pinot.broker.query.lwm.rewrite";
-    public static final boolean CONFIG_OF_BROKER_LWM_REWRITE_ENABLE_DEFAULT = true;
+    public static final boolean CONFIG_OF_BROKER_LWM_REWRITE_ENABLE_DEFAULT = false;
     public static class Request {
       public static final String PQL = "pql";
       public static final String SQL = "sql";
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/LowWaterMarkService.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/LowWaterMarkService.java
index 2fd7434..0c6756c 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/LowWaterMarkService.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/LowWaterMarkService.java
@@ -39,4 +39,6 @@ public interface LowWaterMarkService {
 
     // start
     void start(BrokerMetrics brokerMetrics);
+
+    UpsertQueryRewriter getQueryRewriter();
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/LowWaterMarkService.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/UpsertQueryRewriter.java
similarity index 54%
copy from pinot-core/src/main/java/org/apache/pinot/core/segment/updater/LowWaterMarkService.java
copy to pinot-core/src/main/java/org/apache/pinot/core/segment/updater/UpsertQueryRewriter.java
index 2fd7434..64a64d0 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/LowWaterMarkService.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/UpsertQueryRewriter.java
@@ -1,3 +1,7 @@
+package org.apache.pinot.core.segment.updater;
+
+import org.apache.pinot.common.request.BrokerRequest;
+
 /**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -16,27 +20,8 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.core.segment.updater;
-
-import org.apache.helix.HelixDataAccessor;
-import org.apache.pinot.common.metrics.BrokerMetrics;
-
-import java.util.Map;
-
-/**
- * LowWaterMarkService keeps records of the low water mark (i.e., the stream ingestion progress) for each partition of
- * an input table.
- */
-public interface LowWaterMarkService {
-
-    void init(HelixDataAccessor helixDataAccessor, String helixClusterName, int serverPollingInterval, int serverPort);
-
-    // Return the low water mark mapping from partition id to the corresponding low water mark of a given table.
-    Map<Integer, Long> getLowWaterMarks(String tableName);
+public interface UpsertQueryRewriter {
 
-    // Shutdown the service.
-    void shutDown();
+  void rewriteQueryForUpsert(BrokerRequest request, String rawTableName);
 
-    // start
-    void start(BrokerMetrics brokerMetrics);
 }
diff --git a/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/broker/upsert/PollingBasedLowWaterMarkService.java b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/broker/upsert/PollingBasedLowWaterMarkService.java
index f7e88c9..74e4b41 100644
--- a/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/broker/upsert/PollingBasedLowWaterMarkService.java
+++ b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/broker/upsert/PollingBasedLowWaterMarkService.java
@@ -30,6 +30,7 @@ import org.apache.pinot.common.metrics.BrokerMeter;
 import org.apache.pinot.common.metrics.BrokerMetrics;
 import org.apache.pinot.common.restlet.resources.TableLowWaterMarksInfo;
 import org.apache.pinot.core.segment.updater.LowWaterMarkService;
+import org.apache.pinot.core.segment.updater.UpsertQueryRewriter;
 import org.glassfish.jersey.client.ClientProperties;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -66,6 +67,7 @@ public class PollingBasedLowWaterMarkService implements LowWaterMarkService {
   private int _serverPort;
   private boolean _shuttingDown;
   private BrokerMetrics _brokerMetrics;
+  private UpsertQueryRewriter _queryRewriter;
 
   @Override
   public void init(HelixDataAccessor helixDataAccessor, String helixClusterName, int serverPollingInterval, int serverPort) {
@@ -75,6 +77,7 @@ public class PollingBasedLowWaterMarkService implements LowWaterMarkService {
     _cacheInstanceConfigsDataAccessor =
         new ZkCacheBaseDataAccessor<>((ZkBaseDataAccessor<ZNRecord>) helixDataAccessor.getBaseDataAccessor(),
             instanceConfigs, null, Collections.singletonList(instanceConfigs));
+    _queryRewriter = new UpsertQueryRewriterImpl(this);
     _tableLowWaterMarks = new ConcurrentHashMap<>();
     _httpClient = ClientBuilder.newClient();
     _httpClient.property(ClientProperties.CONNECT_TIMEOUT, SERVER_CONNENCT_TIMEOUT_MS);
@@ -101,6 +104,11 @@ public class PollingBasedLowWaterMarkService implements LowWaterMarkService {
   }
 
   @Override
+  public UpsertQueryRewriter getQueryRewriter() {
+    return _queryRewriter;
+  }
+
+  @Override
   public Map<Integer, Long> getLowWaterMarks(String tableName) {
     return _tableLowWaterMarks.get(tableName);
   }
diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/LowWaterMarkQueryWriter.java b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/broker/upsert/UpsertQueryRewriterImpl.java
similarity index 80%
copy from pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/LowWaterMarkQueryWriter.java
copy to pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/broker/upsert/UpsertQueryRewriterImpl.java
index c3285e5..90fce7c 100644
--- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/LowWaterMarkQueryWriter.java
+++ b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/broker/upsert/UpsertQueryRewriterImpl.java
@@ -16,29 +16,51 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.broker.requesthandler;
+package org.apache.pinot.broker.upsert;
 
+import org.apache.pinot.common.config.TableNameBuilder;
 import org.apache.pinot.common.request.BrokerRequest;
 import org.apache.pinot.common.request.FilterOperator;
 import org.apache.pinot.common.request.FilterQuery;
 import org.apache.pinot.common.request.FilterQueryMap;
+import org.apache.pinot.common.utils.CommonConstants;
+import org.apache.pinot.core.segment.updater.LowWaterMarkService;
+import org.apache.pinot.core.segment.updater.UpsertQueryRewriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
-// Add a lwm query to the filter query of a Pinot query for upsert enabled table.
-// Thread-Safe
-public class LowWaterMarkQueryWriter {
-  private static final Logger LOGGER = LoggerFactory.getLogger(LowWaterMarkQueryWriter.class);
+public class UpsertQueryRewriterImpl implements UpsertQueryRewriter {
+  private static final Logger LOGGER = LoggerFactory.getLogger(UpsertQueryRewriterImpl.class);
+
+  protected final LowWaterMarkService _lwmService;
   private static final String VALID_FROM = "$validFrom";
   private static final String VALID_UNTIL = "$validUntil";
   // Normal Pinot query node uses positive IDs. So lwm query node ids are all negative.
   private static final int QUERY_ID_BASE = -1000;
 
+  public UpsertQueryRewriterImpl(LowWaterMarkService lwmService) {
+    _lwmService = lwmService;
+  }
+
+  @Override
+  public void rewriteQueryForUpsert(BrokerRequest request, String rawTableName) {
+    final String realtimeTableName = TableNameBuilder.ensureTableNameWithType(rawTableName,
+        CommonConstants.Helix.TableType.REALTIME);
+    Map<Integer, Long> lowWaterMarks = _lwmService.getLowWaterMarks(realtimeTableName);
+    if (lowWaterMarks == null || lowWaterMarks.size() == 0) {
+      LOGGER.info("No low water marks info found for table {}", realtimeTableName);
+      return;
+    }
+    LOGGER.info("Found low water marks {} for table {}", String.valueOf(lowWaterMarks), realtimeTableName);
+    addLowWaterMarkToQuery(request, lowWaterMarks);
+    LOGGER.info("Query augmented with LWMS info for table {} : {}", realtimeTableName, request);
+  }
+
   /**
    * For upsert enabled tables, augment the realtime query with low water mark constraints in its filter query of the
    * form
@@ -47,7 +69,7 @@ public class LowWaterMarkQueryWriter {
    * @param realtimeBrokerRequest
    * @param lowWaterMarks
    */
-  public static void addLowWaterMarkToQuery(BrokerRequest realtimeBrokerRequest, Map<Integer, Long> lowWaterMarks) {
+  public void addLowWaterMarkToQuery(BrokerRequest realtimeBrokerRequest, Map<Integer, Long> lowWaterMarks) {
     if (lowWaterMarks == null || lowWaterMarks.size() == 0) {
       LOGGER.warn("No low water mark info found for query: {}", realtimeBrokerRequest);
       return;
@@ -93,7 +115,7 @@ public class LowWaterMarkQueryWriter {
    * @return a filter query corresponding to the low water mark constraint of a single partition. The general form is:
    *         ($ValidFrom <= lwm && $validFrom > -1)  AND (lwm < $validUtil OR $validUtil = -1)
    */
-  private static FilterQuery addSinglePartitionLowWaterMark(int queryIdBase, BrokerRequest realtimeBrokerRequest,
+  private FilterQuery addSinglePartitionLowWaterMark(int queryIdBase, BrokerRequest realtimeBrokerRequest,
       Long lwm) {
     // ValidFromQuery: ($ValidFrom <= lwm && $validFrom > -1)
     FilterQuery validFromFilterQuery = new FilterQuery();
@@ -135,7 +157,7 @@ public class LowWaterMarkQueryWriter {
     return lwmQuery;
   }
 
-  private static FilterQuery getLeafFilterQuery(String column, int id, String value, FilterOperator operator,
+  private FilterQuery getLeafFilterQuery(String column, int id, String value, FilterOperator operator,
       BrokerRequest realtimeBrokerRequest) {
     FilterQuery filterQuery = new FilterQuery();
     filterQuery.setColumn(column);
diff --git a/pinot-broker/src/test/java/org/apache/pinot/broker/requesthandler/LowWaterMarkQueryWriterTest.java b/pinot-grigio/pinot-grigio-provided/src/test/java/org/apache/pinot/broker/upsert/UpsertQueryRewriterImplTest.java
similarity index 91%
rename from pinot-broker/src/test/java/org/apache/pinot/broker/requesthandler/LowWaterMarkQueryWriterTest.java
rename to pinot-grigio/pinot-grigio-provided/src/test/java/org/apache/pinot/broker/upsert/UpsertQueryRewriterImplTest.java
index 1021477..ac6c38d 100644
--- a/pinot-broker/src/test/java/org/apache/pinot/broker/requesthandler/LowWaterMarkQueryWriterTest.java
+++ b/pinot-grigio/pinot-grigio-provided/src/test/java/org/apache/pinot/broker/upsert/UpsertQueryRewriterImplTest.java
@@ -16,21 +16,34 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.broker.requesthandler;
+package org.apache.pinot.broker.upsert;
 
+import org.apache.pinot.broker.requesthandler.LowWaterMarkQueryWriter;
 import org.apache.pinot.common.request.BrokerRequest;
 import org.apache.pinot.common.request.FilterOperator;
 import org.apache.pinot.common.request.FilterQuery;
+import org.apache.pinot.core.segment.updater.LowWaterMarkService;
 import org.apache.pinot.pql.parsers.Pql2Compiler;
 import org.apache.thrift.TException;
 import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 
-public class LowWaterMarkQueryWriterTest {
+public class UpsertQueryRewriterImplTest {
+
+    private LowWaterMarkService _lwms;
+    private UpsertQueryRewriterImpl rewriter;
+
+    @BeforeClass
+    public void init() {
+        _lwms = new PollingBasedLowWaterMarkService();
+        rewriter = new UpsertQueryRewriterImpl(_lwms);
+    }
+
     @Test
     public void testRewriteQueryWithoutExistingFilters() throws Exception{
         Pql2Compiler pql2Compiler = new Pql2Compiler();
@@ -39,7 +52,7 @@ public class LowWaterMarkQueryWriterTest {
         Map<Integer, Long> lwms = new HashMap<>();
         lwms.put(0, 10L);
         lwms.put(1, 20L);
-        LowWaterMarkQueryWriter.addLowWaterMarkToQuery(req, lwms);
+        rewriter.addLowWaterMarkToQuery(req, lwms);
         Assert.assertTrue(req.isSetFilterQuery());
         try {
             req.validate();
@@ -80,7 +93,7 @@ public class LowWaterMarkQueryWriterTest {
         Map<Integer, Long> lwms = new HashMap<>();
         lwms.put(0, 10L);
         lwms.put(1, 20L);
-        LowWaterMarkQueryWriter.addLowWaterMarkToQuery(req, lwms);
+        rewriter.addLowWaterMarkToQuery(req, lwms);
         Assert.assertTrue(req.isSetFilterQuery());
         try {
             req.validate();


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


[incubator-pinot] 01/09: add coordinator related codes

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jamesshao pushed a commit to branch upsert-refactor
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git

commit 7832c02fa71dab6afcd248a24d15d043ca9cd29f
Author: james Shao <sj...@uber.com>
AuthorDate: Fri Feb 21 18:04:13 2020 -0800

    add coordinator related codes
---
 pinot-grigio/pinot-grigio-common/pom.xml           |  75 ++++
 .../pinot/grigio/common/config/CommonConfig.java   |  33 ++
 .../common/keyValueStore/ByteArrayWrapper.java     |  49 +++
 .../common/keyValueStore/KeyValueStoreDB.java      |  30 ++
 .../common/keyValueStore/KeyValueStoreTable.java   |  32 ++
 .../messages/KeyCoordinatorMessageContext.java     |  92 +++++
 .../common/messages/KeyCoordinatorQueueMsg.java    | 133 +++++++
 .../common/messages/LogCoordinatorMessage.java     |  82 ++++
 .../pinot/grigio/common/messages/LogEventType.java |  48 +++
 .../pinot/grigio/common/metrics/GrigioGauge.java   |  65 ++++
 .../pinot/grigio/common/metrics/GrigioMeter.java   |  66 ++++
 .../pinot/grigio/common/metrics/GrigioMetrics.java |  80 ++++
 .../pinot/grigio/common/metrics/GrigioTimer.java   |  73 ++++
 .../pinot/grigio/common/rpcQueue/ProduceTask.java  | 124 ++++++
 .../grigio/common/rpcQueue/QueueConsumer.java      |  41 ++
 .../common/rpcQueue/QueueConsumerRecord.java       |  66 ++++
 .../grigio/common/rpcQueue/QueueProducer.java      |  37 ++
 .../SegmentUpdateLogStorageProvider.java           | 167 ++++++++
 .../common/storageProvider/UpdateLogEntry.java     | 103 +++++
 .../common/storageProvider/UpdateLogEntrySet.java  |  69 ++++
 .../storageProvider/UpdateLogStorageExplorer.java  |  75 ++++
 .../storageProvider/UpdateLogStorageProvider.java  | 222 +++++++++++
 .../UpdateLogRetentionManager.java                 |  29 ++
 .../UpdateLogRetentionManagerImpl.java             |  45 +++
 .../UpdateLogTableRetentionManager.java            | 129 +++++++
 .../UpdateLogTableRetentionManagerImpl.java        |  57 +++
 .../updateStrategy/MessageResolveStrategy.java     |  25 ++
 .../updateStrategy/MessageTimeResolveStrategy.java |  45 +++
 .../pinot/grigio/common/utils/CommonUtils.java     |  48 +++
 .../grigio/common/utils/IdealStateHelper.java      |  78 ++++
 .../pinot/grigio/servers/GrigioServerMetrics.java  |  45 +++
 .../grigio/servers/KeyCoordinatorProvider.java     |  98 +++++
 .../grigio/servers/SegmentUpdaterProvider.java     |  83 ++++
 .../common/keyValueStore/ByteArrayWrapperTest.java |  45 +++
 .../messages/KeyCoordinatorMessageContextTest.java |  54 +++
 .../messages/KeyCoordinatorQueueMsgTest.java       | 110 ++++++
 .../common/messages/LogCoordinatorMessageTest.java |  43 +++
 .../grigio/common/messages/LogEventTypeTest.java   |  37 ++
 .../common/metrics/MockGrigioServerMetrics.java    |  29 ++
 .../grigio/common/rpcQueue/ProduceTaskTest.java    |  81 ++++
 .../common/rpcQueue/QueueConsumerRecordTest.java   |  37 ++
 .../SegmentUpdateLogStorageProviderTest.java       | 163 ++++++++
 .../common/storageProvider/UpdateLogEntryTest.java |  54 +++
 .../UpdateLogStorageProviderTest.java              | 100 +++++
 .../UpdateLogTableRetentionManagerImplTest.java    | 119 ++++++
 .../grigio/common/utils/IdealStateHelperTest.java  |  74 ++++
 .../grigio/servers/KeyCoordinatorProviderTest.java | 106 +++++
 .../grigio/servers/SegmentUpdaterProviderTest.java |  94 +++++
 pinot-grigio/pinot-grigio-coordinator/pom.xml      | 104 +++++
 .../pinot/grigio/common/CoordinatorConfig.java     |  33 ++
 .../grigio/common/DistributedCommonUtils.java      |  54 +++
 .../FixedPartitionCountBytesPartitioner.java       |  44 +++
 .../common/FixedPartitionCountIntPartitioner.java  |  42 ++
 .../common/FixedPartitionCountPartitioner.java     |  49 +++
 .../org/apache/pinot/grigio/common/OffsetInfo.java |  73 ++++
 .../common/keyValueStore/RocksDBBatchReader.java   |  62 +++
 .../common/keyValueStore/RocksDBBatchWriter.java   |  56 +++
 .../grigio/common/keyValueStore/RocksDBConfig.java |  51 +++
 .../keyValueStore/RocksDBKeyValueStoreDB.java      | 162 ++++++++
 .../keyValueStore/RocksDBKeyValueStoreTable.java   | 110 ++++++
 .../grigio/common/rpcQueue/KafkaQueueConsumer.java | 158 ++++++++
 .../grigio/common/rpcQueue/KafkaQueueProducer.java |  79 ++++
 .../rpcQueue/KeyCoordinatorQueueConsumer.java      |  93 +++++
 .../rpcQueue/KeyCoordinatorQueueProducer.java      |  75 ++++
 .../rpcQueue/LogCoordinatorQueueProducer.java      |  73 ++++
 .../rpcQueue/SegmentUpdateQueueConsumer.java       |  94 +++++
 .../common/rpcQueue/VersionMsgQueueProducer.java   |  74 ++++
 .../KCUpdateLogRetentionManagerImpl.java           |  82 ++++
 .../KCUpdateLogTableRetentionManagerImpl.java      |  86 +++++
 .../GrigioKeyCoordinatorMetrics.java               |  44 +++
 .../api/KeyCoordinatorApiApplication.java          |  90 +++++
 .../keyCoordinator/api/KeyCoordinatorInstance.java |  67 ++++
 .../api/resources/KeyCoordinatorHealthCheck.java   |  59 +++
 .../resources/KeyCoordinatorInstanceResource.java  | 100 +++++
 .../api/resources/KeyCoordinatorStatus.java        |  70 ++++
 .../KeyCoordinatorStatusToggleResource.java        |  61 +++
 .../keyCoordinator/helix/ActiveTableTracker.java   | 109 ++++++
 .../keyCoordinator/helix/HelixSetupUtils.java      |  65 ++++
 .../helix/KeyCoordinatorClusterHelixManager.java   | 104 +++++
 .../helix/KeyCoordinatorLeadershipManager.java     |  53 +++
 .../KeyCoordinatorMessageStateModelFactory.java    |  99 +++++
 .../helix/KeyCoordinatorPinotHelixSpectator.java   |  59 +++
 .../helix/KeyCoordinatorVersionManager.java        | 120 ++++++
 .../pinot/grigio/keyCoordinator/helix/State.java   |  26 ++
 .../grigio/keyCoordinator/helix/TableListener.java |  29 ++
 .../internal/DistributedKeyCoordinatorCore.java    | 148 +++++++
 .../keyCoordinator/internal/MessageFetcher.java    | 186 +++++++++
 .../internal/SegmentEventProcessor.java            | 427 +++++++++++++++++++++
 .../internal/VersionMessageManager.java            | 160 ++++++++
 .../keyCoordinator/starter/KeyCoordinatorConf.java | 168 ++++++++
 .../starter/KeyCoordinatorStarter.java             | 202 ++++++++++
 .../test/java/helix/ActiveTableTrackerTest.java    | 110 ++++++
 .../FixedPartitionCountBytesPartitionerTest.java   |  75 ++++
 .../FixedPartitionCountIntPartitionerTest.java     |  74 ++++
 .../keyValueStore/RocksDBKeyValueStoreDBTest.java  |  87 +++++
 .../KCUpdateLogRetentionManagerImplTest.java       | 114 ++++++
 .../KCUpdateLogTableRetentionManagerImplTest.java  | 148 +++++++
 ...KeyCoordinatorMessageStateModelFactoryTest.java |  28 ++
 .../internal/MessageFetcherTest.java               | 127 ++++++
 .../internal/SegmentEventProcessorTest.java        | 229 +++++++++++
 .../internal/VersionMessageManagerTest.java        |  94 +++++
 .../org.mockito.plugins.MockMaker                  |   1 +
 pinot-grigio/pom.xml                               |  42 ++
 pom.xml                                            |  11 +
 104 files changed, 8830 insertions(+)

diff --git a/pinot-grigio/pinot-grigio-common/pom.xml b/pinot-grigio/pinot-grigio-common/pom.xml
new file mode 100644
index 0000000..6702e5d
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/pom.xml
@@ -0,0 +1,75 @@
+<?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.
+
+-->
+<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>pinot-grigio</artifactId>
+    <groupId>org.apache.pinot</groupId>
+    <version>0.3.0-SNAPSHOT</version>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>pinot-grigio-common</artifactId>
+  <name>Pinot Grigio Common</name>
+  <properties>
+    <pinot.root>${basedir}/../..</pinot.root>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>commons-lang</groupId>
+      <artifactId>commons-lang</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.helix</groupId>
+      <artifactId>helix-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.pinot</groupId>
+      <artifactId>pinot-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>commons-configuration</groupId>
+      <artifactId>commons-configuration</artifactId>
+      <version>1.6</version>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-core</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.testng</groupId>
+      <artifactId>testng</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+
+
+</project>
\ No newline at end of file
diff --git a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/config/CommonConfig.java b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/config/CommonConfig.java
new file mode 100644
index 0000000..35b66c1
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/config/CommonConfig.java
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.config;
+
+public class CommonConfig {
+
+  public static class RPC_QUEUE_CONFIG {
+    public static final String PRODUCER_CONFIG_KEY = "producer";
+    public static final String CONSUMER_CONFIG_KEY = "consumer";
+    public static final String VERSION_MESSAGE_PRODUCER_CONFIG_KEY = "version.message.producer";
+    public static final String TOPIC_KEY = "topic";
+    public static final String HOSTNAME_KEY = "hostname";
+    public static final String CLASS_NAME = "class.name";
+
+  }
+
+}
diff --git a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/keyValueStore/ByteArrayWrapper.java b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/keyValueStore/ByteArrayWrapper.java
new file mode 100644
index 0000000..1cb01f7
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/keyValueStore/ByteArrayWrapper.java
@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.keyValueStore;
+
+import java.util.Arrays;
+
+public class ByteArrayWrapper {
+  private final byte[] _data;
+
+  public ByteArrayWrapper(byte[] data) {
+    if (data == null) {
+      throw new NullPointerException();
+    }
+    _data = data;
+  }
+
+  public byte[] getData() {
+    return _data;
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (!(other instanceof ByteArrayWrapper)) {
+      return false;
+    }
+    return Arrays.equals(this._data, ((ByteArrayWrapper) other)._data);
+  }
+
+  @Override
+  public int hashCode() {
+    return Arrays.hashCode(_data);
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/keyValueStore/KeyValueStoreDB.java b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/keyValueStore/KeyValueStoreDB.java
new file mode 100644
index 0000000..4c0d33f
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/keyValueStore/KeyValueStoreDB.java
@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.keyValueStore;
+
+import org.apache.commons.configuration.Configuration;
+
+public interface KeyValueStoreDB<K, V> {
+
+  void init(Configuration conf);
+
+  KeyValueStoreTable<K, V> getTable(String tableName);
+
+  void deleteTable(String tableName);
+}
diff --git a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/keyValueStore/KeyValueStoreTable.java b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/keyValueStore/KeyValueStoreTable.java
new file mode 100644
index 0000000..d39eae7
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/keyValueStore/KeyValueStoreTable.java
@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.keyValueStore;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+public interface KeyValueStoreTable<K, V> {
+
+  Map<K, V> multiGet(List<K> keys) throws IOException;
+
+  void multiPut(Map<K, V> keyValuePairs) throws IOException;
+
+  void deleteTable() throws IOException;
+}
diff --git a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/messages/KeyCoordinatorMessageContext.java b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/messages/KeyCoordinatorMessageContext.java
new file mode 100644
index 0000000..182e4a9
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/messages/KeyCoordinatorMessageContext.java
@@ -0,0 +1,92 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.messages;
+
+import org.apache.commons.lang.SerializationUtils;
+
+import java.io.Serializable;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * this class is used as the value stored in key-value store
+ * following field:
+ * segmentName: the name of the segment that contains this record
+ * timestamp: the timestamp of the current message (from pinot raw data)
+ * kafkaOffset: the kafka offset of this message
+ */
+public class KeyCoordinatorMessageContext implements Serializable {
+
+  private final String _segmentName;
+  private final long _timestamp;
+  private final long _kafkaOffset;
+
+  public KeyCoordinatorMessageContext(String segmentName, long timestamp, long kafkaOffset) {
+    _segmentName = segmentName;
+    _timestamp = timestamp;
+    _kafkaOffset = kafkaOffset;
+  }
+
+  public byte[] toBytes() {
+    return SerializationUtils.serialize(this);
+  }
+
+  public static Optional<KeyCoordinatorMessageContext> fromBytes(byte[] bytes) {
+    if (bytes == null || bytes.length == 0) {
+      return Optional.empty();
+    }
+    return Optional.ofNullable((KeyCoordinatorMessageContext) SerializationUtils.deserialize(bytes));
+  }
+
+  public String getSegmentName() {
+    return _segmentName;
+  }
+
+  public long getTimestamp() {
+    return _timestamp;
+  }
+
+  public long getKafkaOffset() {
+    return _kafkaOffset;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    KeyCoordinatorMessageContext that = (KeyCoordinatorMessageContext) o;
+    return _timestamp == that._timestamp &&
+        _kafkaOffset == that._kafkaOffset &&
+        Objects.equals(_segmentName, that._segmentName);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(_segmentName, _timestamp, _kafkaOffset);
+  }
+
+  @Override
+  public String toString() {
+    return "KeyCoordinatorMessageContext{" +
+        "_segmentName='" + _segmentName + '\'' +
+        ", _timestamp=" + _timestamp +
+        ", _kafkaOffset=" + _kafkaOffset +
+        '}';
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/messages/KeyCoordinatorQueueMsg.java b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/messages/KeyCoordinatorQueueMsg.java
new file mode 100644
index 0000000..692c083
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/messages/KeyCoordinatorQueueMsg.java
@@ -0,0 +1,133 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.messages;
+
+import com.google.common.base.Preconditions;
+import org.apache.pinot.common.utils.LLCSegmentName;
+
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * class used for the message pass from server to key coordinator
+ * it contains the following field: table name, primary key and other metadata (in the _context field)
+ *
+ * There are two kinds of messages: version message and regular message. Version messages contains only
+ * version field, while regular message contains all other fields but not version field.
+ */
+public class KeyCoordinatorQueueMsg implements Serializable {
+  private static final byte[] KEY_PLACEHOLDER = new byte[0];  // placeholder for key field for version messages
+  private static final String SEGMENT_NAME_PLACEHOLDER = "";  // placeholder for segmentName field for version messages
+  private static final long TIMESTAMP_PLACEHOLDER = -1;  // placeholder for timestamp field for version messages
+  private static final long KAFKA_OFFSET_PLACEHOLDER = -1;  // placeholder for kafka offset field for version messages
+  private static final long VERSION_PLACEHOLDER = -1;  // placeholder for version field for regular messages
+
+  private final byte[] _key;
+  private final String _segmentName;
+  private final long _kafkaOffset;
+  private final long _timestamp;
+  private final long _version;  // positive number when it is a version message, VERSION_PLACEHOLDER (-1) otherwise.
+
+  /**
+   * Constructor for regular messages
+   */
+  public KeyCoordinatorQueueMsg(byte[] key, String segmentName, long timestamp, long kafkaOffset) {
+    this._key = key;
+    this._segmentName = segmentName;
+    this._timestamp = timestamp;
+    this._kafkaOffset = kafkaOffset;
+    this._version = VERSION_PLACEHOLDER;
+  }
+
+  /**
+   * Constructor for version messages
+   */
+  public KeyCoordinatorQueueMsg(long version) {
+    this._key = KEY_PLACEHOLDER;
+    this._segmentName = SEGMENT_NAME_PLACEHOLDER;
+    this._timestamp = TIMESTAMP_PLACEHOLDER;
+    this._kafkaOffset = KAFKA_OFFSET_PLACEHOLDER;
+    this._version = version;
+  }
+
+  public byte[] getKey() {
+    Preconditions.checkState(!isVersionMessage(), "Cannot get key from a version message");
+    return _key;
+  }
+
+  public String getSegmentName() {
+    Preconditions.checkState(!isVersionMessage(), "Cannot get segment name from a version message");
+    return _segmentName;
+  }
+
+  public long getTimestamp() {
+    Preconditions.checkState(!isVersionMessage(), "Cannot get timestamp from a version message");
+    return _timestamp;
+  }
+
+  public long getKafkaOffset() {
+    Preconditions.checkState(!isVersionMessage(), "Cannot get kafka offset from a version message");
+    return _kafkaOffset;
+  }
+
+  public long getVersion() {
+    Preconditions.checkState(isVersionMessage(), "Cannot get version from a regular ingestion upsert event");
+    return _version;
+  }
+
+  public boolean isVersionMessage() {
+    return isValidVersion(_version);
+  }
+
+  // used by deserializer
+  public static boolean isValidVersion(long version) {
+    return version > VERSION_PLACEHOLDER;
+  }
+
+  // used by serializer
+  public Map<String, Object> getDataForSerializer() {
+    Map<String, Object> data = new HashMap<>();
+    data.put("primaryKey", ByteBuffer.wrap(_key));
+    data.put("segmentName", _segmentName);
+    data.put("kafkaOffset", _kafkaOffset);
+    data.put("timestamp", _timestamp);
+    data.put("version", _version);
+    return data;
+  }
+
+  @Override
+  public String toString() {
+    return "key: " + new String(_key, StandardCharsets.UTF_8)
+        + " segment: " + _segmentName
+        + " timestamp: " + _timestamp
+        + " kafkaOffset: " + _kafkaOffset
+        + " version: " + _version;
+  }
+
+  /**
+   * get table name without type info
+   */
+  public String getPinotTableName() {
+    Preconditions.checkState(!isVersionMessage(), "Cannot get Pinot table name from a version message");
+    return new LLCSegmentName(_segmentName).getTableName();
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/messages/LogCoordinatorMessage.java b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/messages/LogCoordinatorMessage.java
new file mode 100644
index 0000000..581dc07
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/messages/LogCoordinatorMessage.java
@@ -0,0 +1,82 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.messages;
+
+import java.util.Objects;
+
+/**
+ * this message contains the following 4 attributes:
+ * 1. segmentName: the name of the segment for the pinot record we are going to update
+ * 2. value: the value to update the virtual column to, could be any value we desired to use (the value of new validFrom/validUntil column)
+ * 3. updateEventType: insert/delete, used to indicate which column to update
+ * 4. kafka offset: the offset of the pinot record we are going to update.
+ *
+ * segment updater will use the segment name & offset to identify the location of the pinot record, and use the
+ * updateEventType to decide which virtual column to update. And it will use value to update the corresponding column.
+ */
+public class LogCoordinatorMessage {
+  private final String _segmentName;
+  private final long _value;
+  private final LogEventType _updateEventType;
+  private long _kafkaOffset;
+
+  public String getSegmentName() {
+    return _segmentName;
+  }
+
+  public long getValue() {
+    return _value;
+  }
+
+  public LogEventType getUpdateEventType() {
+    return _updateEventType;
+  }
+
+  public long getKafkaOffset() {
+    return _kafkaOffset;
+  }
+
+  public LogCoordinatorMessage(String segmentName, long kafkaOffset,
+                               long newValue, LogEventType updateEventType) {
+    this._segmentName = segmentName;
+    this._value = newValue;
+    this._updateEventType = updateEventType;
+    this._kafkaOffset = kafkaOffset;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    LogCoordinatorMessage that = (LogCoordinatorMessage) o;
+    return _value == that._value &&
+        _kafkaOffset == that._kafkaOffset &&
+        Objects.equals(_segmentName, that._segmentName) &&
+        _updateEventType == that._updateEventType;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(_segmentName, _value, _updateEventType, _kafkaOffset);
+  }
+
+  public String toString() {
+    return _segmentName + "|"  + _updateEventType + "|" + _value + "|" + _kafkaOffset;
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/messages/LogEventType.java b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/messages/LogEventType.java
new file mode 100644
index 0000000..b4e9605
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/messages/LogEventType.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
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.messages;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public enum LogEventType {
+  INSERT(0),
+  DELETE(1);
+
+  private final int _uuid;
+  private static final Map<Integer, LogEventType> UUID_MAP = new HashMap<>();
+
+  static {
+    for (LogEventType type: LogEventType.values()) {
+      UUID_MAP.put(type.getUUID(), type);
+    }
+  }
+
+  LogEventType(int uuid) {
+    _uuid = uuid;
+  }
+
+  public int getUUID() {
+    return this._uuid;
+  }
+
+  public static LogEventType getEventType(int uuid) {
+    return UUID_MAP.get(uuid);
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/metrics/GrigioGauge.java b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/metrics/GrigioGauge.java
new file mode 100644
index 0000000..f9838c1
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/metrics/GrigioGauge.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.metrics;
+
+import org.apache.pinot.common.Utils;
+import org.apache.pinot.common.metrics.AbstractMetrics;
+import org.apache.pinot.grigio.common.metrics.GrigioMetrics.MetricsType;
+
+
+public enum GrigioGauge implements AbstractMetrics.Gauge {
+
+  // key coordinator related metrics
+  MESSAGE_PROCESS_QUEUE_SIZE("messages", MetricsType.KC_ONLY),
+  FETCH_MSG_FROM_KV_COUNT("messages", MetricsType.KC_ONLY),
+  VERSION_PRODUCED("versions", MetricsType.KC_ONLY),
+  KC_VERSION_CONSUMED("versions", MetricsType.KC_ONLY),
+  SERVER_VERSION_CONSUMED("versions", MetricsType.SERVER_ONLY),
+  KC_INPUT_MESSAGE_LAG_MS("milliseconds", MetricsType.KC_ONLY)
+  ;
+
+  private final String _gaugeName;
+  private final String _unit;
+  private final MetricsType _type;
+
+  GrigioGauge(String unit, MetricsType type) {
+    this._unit = unit;
+    this._gaugeName = Utils.toCamelCase(name().toLowerCase());
+    this._type = type;
+  }
+
+  @Override
+  public String getGaugeName() {
+    return _gaugeName;
+  }
+
+  @Override
+  public String getUnit() {
+    return _unit;
+  }
+
+  @Override
+  public boolean isGlobal() {
+    return true;
+  }
+
+  public MetricsType getType() {
+    return _type;
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/metrics/GrigioMeter.java b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/metrics/GrigioMeter.java
new file mode 100644
index 0000000..8a738da
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/metrics/GrigioMeter.java
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.metrics;
+
+import org.apache.pinot.common.Utils;
+import org.apache.pinot.common.metrics.AbstractMetrics;
+import org.apache.pinot.grigio.common.metrics.GrigioMetrics.MetricsType;
+
+public enum GrigioMeter implements AbstractMetrics.Meter {
+
+  // metrics for kafka consumer library used in upsert components (server, key coordinator)
+  MESSAGE_INGEST_COUNT_PER_BATCH("message", MetricsType.BOTH),
+
+  // segment updater metrics
+  MESSAGE_FETCH_PER_ROUND("messages", MetricsType.SERVER_ONLY),
+
+  // key coordinator related metrics
+  MESSAGE_PRODUCE_FAILED_COUNT("message", MetricsType.KC_ONLY),
+  MESSAGE_PROCESS_THREAD_FETCH_COUNT("messages", MetricsType.KC_ONLY)
+  ;
+
+  private final String _meterName;
+  private final String _unit;
+  private final MetricsType _type;
+
+  GrigioMeter(String unit, MetricsType type) {
+    this._unit = unit;
+    this._meterName = Utils.toCamelCase(name().toLowerCase());
+    this._type = type;
+  }
+
+  @Override
+  public String getMeterName() {
+    return _meterName;
+  }
+
+  @Override
+  public String getUnit() {
+    return _unit;
+  }
+
+  @Override
+  public boolean isGlobal() {
+    return true;
+  }
+
+  public MetricsType getType() {
+    return _type;
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/metrics/GrigioMetrics.java b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/metrics/GrigioMetrics.java
new file mode 100644
index 0000000..3a6da1a
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/metrics/GrigioMetrics.java
@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.metrics;
+
+import com.yammer.metrics.core.MetricsRegistry;
+import org.apache.pinot.common.metrics.AbstractMetrics;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+public abstract class GrigioMetrics extends AbstractMetrics<AbstractMetrics.QueryPhase, GrigioMeter, GrigioGauge, GrigioTimer> {
+
+  public GrigioMetrics(String prefix, MetricsRegistry metricsRegistry) {
+    super(prefix, metricsRegistry, GrigioMetrics.class);
+  }
+
+  @Override
+  protected QueryPhase[] getQueryPhases() {
+    return new QueryPhase[0];
+  }
+
+  public void addTimedValueMs(GrigioTimer timer, long duration) {
+    addTimedValue(timer, duration, TimeUnit.MILLISECONDS);
+  }
+
+  public void addTimedTableValueMs(String table, GrigioTimer timer, long duration) {
+    addTimedTableValue(table, timer, duration, TimeUnit.MILLISECONDS);
+  }
+
+  protected static GrigioMeter[] filterMeterByTypes(MetricsType... types) {
+    GrigioMeter[] meters = GrigioMeter.values();
+    List<GrigioMeter> matchedMeters = new ArrayList<>();
+    for (GrigioMeter meter : meters) {
+      for (MetricsType type : types) {
+        if (meter.getType() == type) {
+          matchedMeters.add(meter);
+          break;
+        }
+      }
+    }
+    return matchedMeters.toArray(new GrigioMeter[]{});
+  }
+
+  protected static GrigioGauge[] filterGaugeByTypes(MetricsType... types) {
+    GrigioGauge[] gauges = GrigioGauge.values();
+    List<GrigioGauge> matchedGauges = new ArrayList<>();
+    for (GrigioGauge gauge: gauges) {
+      for (MetricsType type : types) {
+        if (gauge.getType() == type) {
+          matchedGauges.add(gauge);
+          break;
+        }
+      }
+    }
+    return matchedGauges.toArray(new GrigioGauge[]{});
+  }
+
+  public enum MetricsType {
+    SERVER_ONLY,
+    KC_ONLY,
+    BOTH
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/metrics/GrigioTimer.java b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/metrics/GrigioTimer.java
new file mode 100644
index 0000000..4b74f30
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/metrics/GrigioTimer.java
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.metrics;
+
+import org.apache.pinot.common.Utils;
+import org.apache.pinot.common.metrics.AbstractMetrics;
+import org.apache.pinot.grigio.common.metrics.GrigioMetrics.MetricsType;
+
+public enum GrigioTimer implements AbstractMetrics.Timer {
+
+  // common metrics for kafka component used in upsert
+  PRODUCER_LAG(MetricsType.BOTH),
+  FLUSH_LAG(MetricsType.BOTH),
+  COMMIT_OFFSET_LAG(MetricsType.BOTH),
+  FETCH_MESSAGE_LAG(MetricsType.BOTH),
+
+  // metrics for segment updater
+  FETCH_MSG_FROM_CONSUMER_TIME(MetricsType.SERVER_ONLY),
+  UPDATE_DATAMANAGER_TIME(MetricsType.SERVER_ONLY),
+  UPDATE_LOCAL_LOG_FILE_TIME(MetricsType.SERVER_ONLY),
+  SEGMENT_UPDATER_LOOP_TIME(MetricsType.SERVER_ONLY),
+
+  // metrics for key coordinator
+  MESSAGE_PROCESS_THREAD_FETCH_DELAY(MetricsType.KC_ONLY),
+  MESSAGE_PROCESS_THREAD_PROCESS_DELAY(MetricsType.KC_ONLY),
+
+  PRODUCE_VERSION_MESSAGE(MetricsType.KC_ONLY),
+
+  FETCH_MSG_FROM_KV_DELAY(MetricsType.KC_ONLY),
+  PROCESS_MSG_UPDATE(MetricsType.KC_ONLY),
+  SEND_MSG_TO_KAFKA(MetricsType.KC_ONLY),
+  STORE_UPDATE_ON_KV(MetricsType.KC_ONLY),
+  STORE_UPDATE_ON_DISK(MetricsType.KC_ONLY),
+  ;
+
+  private final String _timerName;
+  private final MetricsType _type;
+
+  GrigioTimer(MetricsType type) {
+    _timerName = Utils.toCamelCase(name().toLowerCase());
+    _type = type;
+  }
+
+  @Override
+  public String getTimerName() {
+    return _timerName;
+  }
+
+  @Override
+  public boolean isGlobal() {
+    return false;
+  }
+
+  public MetricsType getType() {
+    return _type;
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/rpcQueue/ProduceTask.java b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/rpcQueue/ProduceTask.java
new file mode 100644
index 0000000..35bcbe9
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/rpcQueue/ProduceTask.java
@@ -0,0 +1,124 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.rpcQueue;
+
+import java.util.Objects;
+
+/**
+ * Class created to wrap around the kafka produce task object, so we can make the upsert (grigio) package stream independent
+ * by using this implementation, we don't need to hard code kafka dependency in related package (pinot-server, pinot-core) etc
+ */
+public class ProduceTask<K, V> {
+
+  private volatile boolean _completed = false;
+  private Exception _exception = null;
+  private Callback _callback = null;
+  private String _topic;
+  private K _key;
+  private V _value;
+
+  public ProduceTask(K key, V value) {
+    this._key = key;
+    this._value = value;
+  }
+
+  public ProduceTask(String topic, K key, V value) {
+    this._topic = topic;
+    this._key = key;
+    this._value = value;
+  }
+
+  /**
+   * topic might be null, if that is the case we assume this produce call will rely on default topic in producer
+   * TODO: refactor this part such we ensure non-null return here
+   * @return the name of the topic we are producing to, if there is such topic
+   */
+  public String getTopic() {
+    return _topic;
+  }
+
+  public K getKey() {
+    return _key;
+  }
+
+  public V getValue() {
+    return _value;
+  }
+
+  public void setCallback(Callback callback) {
+    _callback = callback;
+  }
+
+  public boolean isSucceed() {
+    return this._completed && this._exception == null;
+  }
+
+  public Exception getException() {
+    return this._exception;
+  }
+
+  /**
+   * method to be called within native queue producer only, not supposed to be called by us
+   * @param metadata the metadata associated with this call
+   * @param exception any exception associated with this produce, null if no exception happened
+   */
+  public synchronized void markComplete(Object metadata, Exception exception) {
+    if (!_completed) {
+      _completed = true;
+      _exception = exception;
+      if (_callback != null) {
+        if (exception == null) {
+          _callback.onSuccess();
+        } else {
+          _callback.onFailure(exception);
+        }
+      }
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "ProduceTask{" +
+        "_topic='" + _topic + '\'' +
+        ", _key=" + _key +
+        ", _value=" + _value.toString() +
+        '}';
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    ProduceTask<?, ?> that = (ProduceTask<?, ?>) o;
+    return Objects.equals(_topic, that._topic) &&
+        Objects.equals(_key, that._key) &&
+        Objects.equals(_value, that._value);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(_topic, _key, _value);
+  }
+
+  public interface Callback {
+    void onSuccess();
+
+    void onFailure(Exception ex);
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/rpcQueue/QueueConsumer.java b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/rpcQueue/QueueConsumer.java
new file mode 100644
index 0000000..2aace76
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/rpcQueue/QueueConsumer.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.rpcQueue;
+
+
+import org.apache.commons.configuration.Configuration;
+import org.apache.pinot.grigio.common.metrics.GrigioMetrics;
+
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+public interface QueueConsumer<K, V> {
+
+  void init(Configuration conf, GrigioMetrics metrics);
+
+  void subscribeForTable(String tableName, String topicPrefix);
+
+  void unsubscribeForTable(String tableName, String topicPrefix);
+
+  List<QueueConsumerRecord<K, V>> getRequests(long timeout, TimeUnit timeUnit);
+
+  void ackOffset();
+
+  void close();
+}
diff --git a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/rpcQueue/QueueConsumerRecord.java b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/rpcQueue/QueueConsumerRecord.java
new file mode 100644
index 0000000..7e86937
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/rpcQueue/QueueConsumerRecord.java
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.rpcQueue;
+
+/**
+ * data wrapper class for kafka ConsumerRecord data
+ * created to make grigio-common kafka library neutral (prevent kafka dependency)
+ */
+public class QueueConsumerRecord<K, V> {
+
+  private final String _topic;
+  private final int _partition;
+  private final long _offset;
+  private final K _key;
+  private final V _record;
+  private final long _timestamp;
+
+  public QueueConsumerRecord(String topic, int partition, long offset, K key, V record, long timestamp) {
+    this._topic = topic;
+    this._partition = partition;
+    this._offset = offset;
+    this._key = key;
+    this._record = record;
+    this._timestamp = timestamp;
+  }
+
+  public String getTopic() {
+    return _topic;
+  }
+
+  public int getPartition() {
+    return _partition;
+  }
+
+  public long getOffset() {
+    return _offset;
+  }
+
+  public K getKey() {
+    return _key;
+  }
+
+  public V getRecord() {
+    return _record;
+  }
+
+  public long getTimestamp() {
+    return _timestamp;
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/rpcQueue/QueueProducer.java b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/rpcQueue/QueueProducer.java
new file mode 100644
index 0000000..f902ddb
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/rpcQueue/QueueProducer.java
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.rpcQueue;
+
+import org.apache.commons.configuration.Configuration;
+import org.apache.pinot.grigio.common.metrics.GrigioMetrics;
+
+import java.util.List;
+
+public interface QueueProducer<K, V> {
+
+  void init(Configuration conf, GrigioMetrics metrics);
+
+  void produce(ProduceTask<K, V> task);
+
+  void batchProduce(List<ProduceTask<K, V>> tasks);
+
+  void flush();
+
+  void close();
+}
diff --git a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/storageProvider/SegmentUpdateLogStorageProvider.java b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/storageProvider/SegmentUpdateLogStorageProvider.java
new file mode 100644
index 0000000..e7b035f
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/storageProvider/SegmentUpdateLogStorageProvider.java
@@ -0,0 +1,167 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.storageProvider;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.List;
+
+/**
+ * provide the storage abstraction of storing upsert update event logs to a local disk so we can reload it
+ * during server start. This provided the abstraction layer for individual table/segment storage
+ */
+public class SegmentUpdateLogStorageProvider {
+  private static final Logger LOGGER = LoggerFactory.getLogger(SegmentUpdateLogStorageProvider.class);
+
+  @VisibleForTesting
+  protected final File _file;
+  @VisibleForTesting
+  protected final FileOutputStream _outputStream;
+
+  public SegmentUpdateLogStorageProvider(File file)
+      throws IOException {
+    Preconditions.checkState(file != null && file.exists(), "storage file for this virtual column provider does not exist");
+    LOGGER.info("creating segmentUpdateLogProvider at {}", file.getPath());
+    _file = file;
+    openAndLoadDataFromFile(file);
+    _outputStream = new FileOutputStream(_file, true);
+  }
+
+  public synchronized UpdateLogEntrySet readAllMessagesFromFile() throws IOException {
+    int fileLength = (int) _file.length();
+    if (fileLength > 0) {
+      ByteBuffer buffer = ByteBuffer.allocate(fileLength);
+      readFullyFromBeginning(_file, buffer);
+      int messageCount = fileLength / UpdateLogEntry.SIZE;
+      return new UpdateLogEntrySet(buffer, messageCount);
+    } else {
+      return UpdateLogEntrySet.getEmptySet();
+    }
+  }
+
+  public synchronized void addData(List<UpdateLogEntry> messages) throws IOException {
+    final ByteBuffer buffer = ByteBuffer.allocate(messages.size() * UpdateLogEntry.SIZE);
+    for (UpdateLogEntry message: messages) {
+      message.addEntryToBuffer(buffer);
+    }
+    buffer.flip();
+    _outputStream.write(buffer.array());
+    _outputStream.flush();
+
+  }
+
+  public synchronized void destroy() throws IOException {
+    _outputStream.close();
+    if (_file.exists()) {
+      LOGGER.info("deleting file {}", _file.getPath());
+      _file.delete();
+    }
+  }
+
+  public synchronized void close() throws IOException {
+    _outputStream.close();
+  }
+
+  private synchronized void openAndLoadDataFromFile(File segmentUpdateFile) throws IOException {
+    if (segmentUpdateFile == null || !segmentUpdateFile.exists()) {
+      throw new IOException("failed to open segment update file");
+    }
+    FileChannel channel = new RandomAccessFile(segmentUpdateFile, "rw").getChannel();
+    // truncate file if necessary, in case the java process crashed while we have not finished writing out content to
+    // the file. We abandon any unfinished message as we can always read them back from kafka
+    if (segmentUpdateFile.length() > 0 && segmentUpdateFile.length() % UpdateLogEntry.SIZE != 0) {
+      long newSize = segmentUpdateFile.length() / UpdateLogEntry.SIZE * UpdateLogEntry.SIZE;
+      LOGGER.info("truncating {} file from size {} to size {}", segmentUpdateFile.getAbsolutePath(),
+          segmentUpdateFile.length(), newSize);
+      channel.truncate(newSize);
+      channel.force(false);
+    }
+  }
+
+  private synchronized void readFullyFromBeginning(File segmentUpdateFile, ByteBuffer buffer) throws IOException {
+    long start = System.currentTimeMillis();
+    FileChannel channel = new RandomAccessFile(segmentUpdateFile, "r").getChannel();
+    channel.position(0);
+    long position = 0;
+    int byteRead;
+    do {
+      byteRead = channel.read(buffer, position);
+      position += byteRead;
+    } while (byteRead != -1 && buffer.hasRemaining());
+    buffer.flip();
+    LOGGER.info("read all data from segment update file {} to buffer in {} ms", segmentUpdateFile.getName(),
+        System.currentTimeMillis() - start);
+  }
+
+  /**
+   * get the virtual column provider for the consuming segment (won't download from remote)
+   * @param table
+   * @param segment
+   * @param storagePath
+   * @return
+   */
+  public static SegmentUpdateLogStorageProvider getProviderForMutableSegment(String table, String segment,
+                                                                             String storagePath) throws IOException {
+    File file = new File(storagePath);
+    if (!file.exists()) {
+      boolean createResult = file.createNewFile();
+      if (!createResult) {
+        throw new RuntimeException("failed to create file for virtual column storage at path " + storagePath);
+      }
+    }
+    return new SegmentUpdateLogStorageProvider(file);
+  }
+
+  /**
+   * get the virtual column provider for immutable segment (re-use local one or download from remote)
+   * @param table
+   * @param segment
+   * @param storagePath
+   * @return
+   */
+  public static SegmentUpdateLogStorageProvider getProviderForImmutableSegment(String table, String segment,
+                                                                               String storagePath)
+      throws IOException {
+    File file;
+    if (Files.exists(Paths.get(storagePath))) {
+      file = new File(storagePath);
+    } else {
+      file = downloadFileFromRemote(table, segment, storagePath);
+      Preconditions.checkState(file.exists(), "download from remote didn't create the file");
+    }
+    return new SegmentUpdateLogStorageProvider(file);
+  }
+
+  // try to download the update log from remote storage
+  public static File downloadFileFromRemote(String table, String segment, String storagePath) {
+    //TODO implement this logic
+    throw new UnsupportedOperationException("download update log from remote is not supported yet");
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/storageProvider/UpdateLogEntry.java b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/storageProvider/UpdateLogEntry.java
new file mode 100644
index 0000000..2ecc547
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/storageProvider/UpdateLogEntry.java
@@ -0,0 +1,103 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.storageProvider;
+
+import org.apache.pinot.grigio.common.messages.LogCoordinatorMessage;
+import org.apache.pinot.grigio.common.messages.LogEventType;
+
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.util.Objects;
+
+/**
+ * class for the local upsert update log entry,
+ * offset: the offset of the origin message location
+ * value: the value of this record should be updated to
+ * type: the type of the message we are operating
+ * partition: the segment update event stream partition which the message is from. Used for calculating Low water mark
+ * vector.
+ */
+public class UpdateLogEntry implements Serializable {
+  public static final int SIZE = Long.BYTES * 2 + Integer.BYTES * 2;
+  private final long _offset;
+  private final long _value;
+  private final LogEventType _type;
+  private final int _partition;
+
+  public UpdateLogEntry(long offset, long value, LogEventType type, int partition) {
+    _offset = offset;
+    _value = value;
+    _type = type;
+    _partition = partition;
+  }
+
+  public UpdateLogEntry(LogCoordinatorMessage logCoordinatorMessage, int partition) {
+    this(logCoordinatorMessage.getKafkaOffset(), logCoordinatorMessage.getValue(),
+            logCoordinatorMessage.getUpdateEventType(), partition);
+  }
+
+  public long getOffset() {
+    return _offset;
+  }
+
+  public long getValue() {
+    return _value;
+  }
+
+  public LogEventType getType() {
+    return _type;
+  }
+
+  public int getPartition() { return _partition; }
+
+  public void addEntryToBuffer(ByteBuffer buffer) {
+    buffer.putLong(_offset);
+    buffer.putLong(_value);
+    buffer.putInt(_type.getUUID());
+    buffer.putInt(_partition);
+  }
+
+  public static UpdateLogEntry fromBytesBuffer(ByteBuffer buffer) {
+    if (buffer == null) {
+      throw new RuntimeException("trying to get update log event from null buffer");
+    }
+    return new UpdateLogEntry(buffer.getLong(), buffer.getLong(), LogEventType.getEventType(buffer.getInt()), buffer.getInt());
+  }
+
+
+  public String toString() {
+    return "logEventEntry: offset " + _offset + " value " + _value + " type " + _type.toString() + " partition "
+            + _partition;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    UpdateLogEntry logEntry = (UpdateLogEntry) o;
+    return _offset == logEntry._offset &&
+        _value == logEntry._value &&
+        _type == logEntry._type;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(_offset, _value, _type, _partition);
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/storageProvider/UpdateLogEntrySet.java b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/storageProvider/UpdateLogEntrySet.java
new file mode 100644
index 0000000..f756d73
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/storageProvider/UpdateLogEntrySet.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
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.storageProvider;
+
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+
+/**
+ * class for holding the list of update logs we read from files
+ * provide iterator interface for more efficient memory access
+ */
+public class UpdateLogEntrySet implements Iterable<UpdateLogEntry> {
+
+  private final ByteBuffer _buffer;
+  private final int _messageCount;
+  private static final UpdateLogEntrySet EMPTY_LOG_ENTRY_SET = new UpdateLogEntrySet(ByteBuffer.allocate(0),
+      0);
+
+  public UpdateLogEntrySet(ByteBuffer buffer, int messageCount) {
+    _buffer = buffer;
+    _messageCount = messageCount;
+  }
+
+  public int size() {
+    return _messageCount;
+  }
+
+  @Override
+  public Iterator<UpdateLogEntry> iterator() {
+    return new Iterator<UpdateLogEntry>() {
+      @Override
+      public boolean hasNext() {
+        return _buffer != null && _buffer.hasRemaining();
+      }
+
+      @Override
+      public UpdateLogEntry next() {
+        if (!hasNext()) {
+          throw new RuntimeException("no more entries in buffer");
+        }
+        return UpdateLogEntry.fromBytesBuffer(_buffer);
+      }
+    };
+  }
+
+  /**
+   * helper method to create a default empty set in case of invalid/missing input
+   * @return an empty entry set has no data
+   */
+  public static UpdateLogEntrySet getEmptySet() {
+    return EMPTY_LOG_ENTRY_SET;
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/storageProvider/UpdateLogStorageExplorer.java b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/storageProvider/UpdateLogStorageExplorer.java
new file mode 100644
index 0000000..0e38deb
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/storageProvider/UpdateLogStorageExplorer.java
@@ -0,0 +1,75 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.storageProvider;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.apache.commons.configuration.Configuration;
+import org.apache.commons.configuration.PropertiesConfiguration;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Scanner;
+
+/**
+ * command line tools for debug pinot server by allowing us to interatively explore the update log data in pinot server/kc
+ * usage:
+ * $java -cp <pinot-jar-path> org.apache.pinot.grigio.common.storageProvider.UpdateLogStorageExplorer <path to virtual column base path>
+ * you can then input the table name (with _REALTIME postfix) and segment to load data
+ * after this, you can enter the offset you want to explore the update log data at
+ */
+public class UpdateLogStorageExplorer {
+  public static void main(String[] args) throws IOException {
+    Preconditions.checkState(args.length > 0, "need basepath as first parameter");
+    String basePath = args[0];
+
+    Configuration conf = new PropertiesConfiguration();
+    conf.setProperty(UpdateLogStorageProvider.BASE_PATH_CONF_KEY, basePath);
+
+    UpdateLogStorageProvider.init(conf);
+    UpdateLogStorageProvider provider = UpdateLogStorageProvider.getInstance();
+
+    Scanner reader = new Scanner(System.in);
+    System.out.println("please input the tablename and segment name to load");
+    String input = reader.nextLine();
+    String[] inputSplits = input.split(" ");
+    Preconditions.checkState(inputSplits.length == 2, "expect input data to be 'tableName segmentName'");
+    String tableName = inputSplits[0];
+    String segmentName = inputSplits[1];
+
+    provider.loadTable(tableName);
+    UpdateLogEntrySet updateLogEntrySet = provider.getAllMessages(tableName, segmentName);
+    Multimap<Long, UpdateLogEntry> map = ArrayListMultimap.create();
+    System.out.println("update log size: " + updateLogEntrySet.size());
+    updateLogEntrySet.forEach(u -> {
+      map.put(u.getOffset(), u);
+    });
+
+    while (true) {
+      System.out.println("input the offset");
+      long offset = reader.nextLong();
+      Collection<UpdateLogEntry> result = map.get(offset);
+      System.out.println("associated update logs size: " + result.size());
+      for (UpdateLogEntry entry: result) {
+        System.out.println("content: " + entry.toString());
+      }
+    }
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/storageProvider/UpdateLogStorageProvider.java b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/storageProvider/UpdateLogStorageProvider.java
new file mode 100644
index 0000000..b9802c2
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/storageProvider/UpdateLogStorageProvider.java
@@ -0,0 +1,222 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.storageProvider;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import org.apache.commons.configuration.Configuration;
+import org.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class UpdateLogStorageProvider {
+  private static final Logger LOGGER = LoggerFactory.getLogger(UpdateLogStorageProvider.class);
+
+  private final Configuration _conf;
+  private final File _virtualColumnStorageDir;
+  private final Map<String, Map<String, SegmentUpdateLogStorageProvider>> _virtualColumnStorage = new ConcurrentHashMap<>();
+  private volatile boolean _isClosed = false;
+
+  public static final String BASE_PATH_CONF_KEY = "basePath";
+
+  @VisibleForTesting
+  protected static volatile UpdateLogStorageProvider _instance = null;
+
+  public static synchronized void init(Configuration conf) {
+    LOGGER.info("initializing virtual column storage");
+    if (_instance == null) {
+      _instance = new UpdateLogStorageProvider(conf);
+    } else {
+      throw new RuntimeException("validFrom storage has already been inited");
+    }
+  }
+
+  public static UpdateLogStorageProvider getInstance() {
+    if (_instance == null) {
+      throw new RuntimeException("virtual column storage has not been inited");
+    }
+    return _instance;
+  }
+
+  private UpdateLogStorageProvider(Configuration conf) {
+    _conf = conf;
+    final String basePath = conf.getString(BASE_PATH_CONF_KEY);
+    LOGGER.info("use base path {} for virtual column storage", basePath);
+    if (StringUtils.isEmpty(basePath)) {
+      throw new IllegalStateException("base path doesn't exists in config");
+    }
+    _virtualColumnStorageDir = new File(basePath);
+    if (!_virtualColumnStorageDir.exists()) {
+      LOGGER.info("virtual column storage path {} doesn't exist, creating now", basePath);
+      _virtualColumnStorageDir.mkdirs();
+    }
+  }
+
+  public synchronized void addSegment(String tableName, String segmentName) throws IOException {
+    maybeAddTableToMetadata(tableName);
+    Map<String, SegmentUpdateLogStorageProvider> segmentMap = _virtualColumnStorage.get(tableName);
+    if (!segmentMap.containsKey(segmentName)) {
+      File tableDir = new File(_virtualColumnStorageDir, tableName);
+      LOGGER.info("adding local update log storage for table {} segment {}", tableName, segmentName);
+      final File segmentUpdateFile = new File(tableDir, segmentName);
+      if (!segmentUpdateFile.exists()) {
+        LOGGER.info("creating new local update log storage at {}", segmentUpdateFile.getPath());
+        boolean result = segmentUpdateFile.createNewFile();
+        Preconditions.checkState(result, "creating segment path failed " + tableDir);
+      }
+      Preconditions.checkState(segmentUpdateFile.isFile(), "expect segment log location as file");
+      segmentMap.put(segmentName, new SegmentUpdateLogStorageProvider(segmentUpdateFile));
+    }
+  }
+
+  /**
+   * load all segment update logs under a table name in this update log provider
+   * @param tableName the name of the table with type info
+   * @throws IOException
+   */
+  public synchronized void loadTable(String tableName) throws IOException {
+    LOGGER.info("loading table {}", tableName);
+    final File tableDir = new File(_virtualColumnStorageDir, tableName);
+    if (!tableDir.exists()) {
+      LOGGER.warn("table directory does not exist at path {}", tableDir.getPath());
+    } else {
+      Map<String, SegmentUpdateLogStorageProvider> tableUpdateLogs = new ConcurrentHashMap<>();
+      _virtualColumnStorage.put(tableName, tableUpdateLogs);
+      File[] segmentFiles = tableDir.listFiles();
+      if (segmentFiles != null) {
+        for (File segmentFile: segmentFiles) {
+          tableUpdateLogs.put(segmentFile.getName(), new SegmentUpdateLogStorageProvider(segmentFile));
+        }
+        LOGGER.info("loaded {} segment from table", segmentFiles.length);
+      }
+    }
+  }
+
+  public synchronized Set<String> getAllSegments(String tableName) {
+    if (_virtualColumnStorage.containsKey(tableName)) {
+      return ImmutableSet.copyOf(_virtualColumnStorage.get(tableName).keySet());
+    } else {
+      return ImmutableSet.of();
+    }
+  }
+
+  /**
+   * add a table to internal mapping and ensure the local directory exists
+   * @param tableName the name of the table we are adding
+   */
+  private synchronized void maybeAddTableToMetadata(String tableName) {
+    final File tableDir = new File(_virtualColumnStorageDir, tableName);
+    if (!_virtualColumnStorage.containsKey(tableName)) {
+      LOGGER.info("adding virtual column for table {}", tableName);
+      if (!tableDir.exists()) {
+        boolean result = tableDir.mkdir();
+        Preconditions.checkState(result, "creating table path failed " + tableDir);
+      }
+      Preconditions.checkState(tableDir.isDirectory(), "table path is not directory " + tableDir);
+      _virtualColumnStorage.computeIfAbsent(tableName, t -> new ConcurrentHashMap<>());
+    }
+  }
+
+  public UpdateLogEntrySet getAllMessages(String tableName, String segmentName) throws IOException {
+    if (_virtualColumnStorage.containsKey(tableName)) {
+      SegmentUpdateLogStorageProvider provider = _virtualColumnStorage.get(tableName).get(segmentName);
+      if (provider != null) {
+        return provider.readAllMessagesFromFile();
+      } else {
+        LOGGER.warn("don't have data for segment {}", segmentName);
+        return UpdateLogEntrySet.getEmptySet();
+      }
+    } else {
+      LOGGER.error("don't have data for table {}", tableName);
+      return UpdateLogEntrySet.getEmptySet();
+    }
+  }
+
+  public void addDataToFile(String tableName, String segmentName, List<UpdateLogEntry> messages) throws IOException {
+    Preconditions.checkState(!_isClosed, "update log provider has been closed");
+    maybeAddTableToMetadata(tableName);
+    Map<String, SegmentUpdateLogStorageProvider> segmentProviderMap =  _virtualColumnStorage.get(tableName);
+    if (!segmentProviderMap.containsKey(segmentName)) {
+      // TODO fix this part as we are adding all segment metadata
+      // need to work on new design to prevent writing too much data
+      addSegment(tableName, segmentName);
+    }
+    segmentProviderMap.get(segmentName).addData(messages);
+  }
+
+  public synchronized void removeSegment(String tableName, String segmentName) throws IOException {
+    if (_virtualColumnStorage.containsKey(tableName)) {
+      SegmentUpdateLogStorageProvider provider = _virtualColumnStorage.get(tableName).remove(segmentName);
+      if (provider != null) {
+        LOGGER.info("deleting update log for table {} segment {}", tableName, segmentName);
+        provider.destroy();
+      } else {
+        // will also try to delete update log file that are on this server but not loaded due to lazy-loading
+        File segmentUpdateLogFile = new File(new File(_virtualColumnStorageDir, tableName), segmentName);
+        if (segmentUpdateLogFile.exists()) {
+          LOGGER.info("deleting old updates log for table {} segment {}", tableName, segmentName);
+          segmentUpdateLogFile.delete();
+        } else {
+          LOGGER.info("trying to delete table {} segment {} but it doesn't exist", tableName, segmentName);
+        }
+      }
+    } else {
+      LOGGER.info("trying to delete table {} segment {} but table is not in the current server", tableName, segmentName);
+    }
+  }
+
+  public synchronized void removeAllUpdateLogsForTable(String tableName) {
+    LOGGER.info("removing all update log storage for the given table {}", tableName);
+    if (_virtualColumnStorage.containsKey(tableName)) {
+      for (String segmentName : _virtualColumnStorage.get(tableName).keySet()) {
+        try {
+          removeSegment(tableName, segmentName);
+        } catch (IOException ex) {
+          LOGGER.error("failed to remove segment {}:{}", tableName, segmentName);
+        }
+      }
+      _virtualColumnStorage.remove(tableName);
+    }
+    final File tableDir = new File(_virtualColumnStorageDir, tableName);
+    if (tableDir.exists() && tableDir.isDirectory()) {
+      File[] segmentUpdateLogs = tableDir.listFiles();
+      LOGGER.info("remove {} files under table directory {}", segmentUpdateLogs.length, tableDir.getAbsolutePath());
+      for (File segmentUpdateLog : segmentUpdateLogs) {
+        segmentUpdateLog.delete();
+      }
+    }
+  }
+
+  public synchronized void close() throws IOException {
+    _isClosed = true;
+    for (Map<String, SegmentUpdateLogStorageProvider> segmentUpdateLogStorageProviderMap: _virtualColumnStorage.values()) {
+      for (SegmentUpdateLogStorageProvider provider: segmentUpdateLogStorageProviderMap.values()) {
+        provider.close();
+      }
+    }
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/storageProvider/retentionManager/UpdateLogRetentionManager.java b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/storageProvider/retentionManager/UpdateLogRetentionManager.java
new file mode 100644
index 0000000..434c7f5
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/storageProvider/retentionManager/UpdateLogRetentionManager.java
@@ -0,0 +1,29 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.storageProvider.retentionManager;
+
+public interface UpdateLogRetentionManager {
+
+  /**
+   * return an update log retention manager for a given table
+   * @param tableNameWithType the name of the table we are working on, with the type for the table
+   * @return the retention manager
+   */
+  UpdateLogTableRetentionManager getRetentionManagerForTable(String tableNameWithType);
+}
diff --git a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/storageProvider/retentionManager/UpdateLogRetentionManagerImpl.java b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/storageProvider/retentionManager/UpdateLogRetentionManagerImpl.java
new file mode 100644
index 0000000..3852f99
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/storageProvider/retentionManager/UpdateLogRetentionManagerImpl.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.storageProvider.retentionManager;
+
+import org.apache.pinot.grigio.common.utils.IdealStateHelper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class UpdateLogRetentionManagerImpl implements UpdateLogRetentionManager {
+  private static final Logger LOGGER = LoggerFactory.getLogger(UpdateLogRetentionManagerImpl.class);
+
+  private Map<String, UpdateLogTableRetentionManager> _retentionManagerMap = new ConcurrentHashMap<>();
+  private IdealStateHelper _idealStateHelper;
+  private String _instanceName;
+
+  public UpdateLogRetentionManagerImpl(IdealStateHelper idealStateHelper, String instanceName) {
+    _idealStateHelper = idealStateHelper;
+    _instanceName = instanceName;
+  }
+
+  @Override
+  public UpdateLogTableRetentionManager getRetentionManagerForTable(String tableNameWithType) {
+    return _retentionManagerMap.computeIfAbsent(tableNameWithType,
+        t -> new UpdateLogTableRetentionManagerImpl(_idealStateHelper, t, _instanceName));
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/storageProvider/retentionManager/UpdateLogTableRetentionManager.java b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/storageProvider/retentionManager/UpdateLogTableRetentionManager.java
new file mode 100644
index 0000000..85efb47
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/storageProvider/retentionManager/UpdateLogTableRetentionManager.java
@@ -0,0 +1,129 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.storageProvider.retentionManager;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.pinot.common.utils.LLCSegmentName;
+import org.apache.pinot.grigio.common.utils.IdealStateHelper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * class to help decide whether to keep update log for a particular segment or not
+ */
+public abstract class UpdateLogTableRetentionManager {
+
+  protected static final Logger LOGGER = LoggerFactory.getLogger(UpdateLogTableRetentionManagerImpl.class);
+
+  protected String _tableName;
+  protected IdealStateHelper _idealStateHelper;
+  // use the concurrent hashmap as concurrent hashset
+  protected Map<Integer, LLCSegmentName> _partitionToLastSegment = new ConcurrentHashMap<>();
+  protected Map<String, String> _blacklistedSegments = new ConcurrentHashMap<>();
+  protected Map<String, Map<String, String>> _segmentsToInstanceMap;
+
+  public UpdateLogTableRetentionManager(IdealStateHelper idealStateHelper, String tableName) {
+    _idealStateHelper = idealStateHelper;
+    _tableName = tableName;
+  }
+
+  /**
+   * for external components to notify retention manager that physical data for this segment has been deleted
+   * @param segmentName the name of the segment that we are deleting from local storage
+   */
+  public abstract void notifySegmentDeletion(String segmentName);
+
+  /**
+   * for external components to notify us that segments for this tables has been change and we should refresh it
+   */
+  public abstract void notifySegmentsChange();
+
+  /**
+   * check if the the given segment is assigned to current server
+   */
+  protected abstract boolean isSegmentAssignedToCurrentServer(String segmentName);
+
+  /**
+   * update state from helix
+   */
+  protected void updateStateFromHelix() {
+    long start = System.currentTimeMillis();
+    _segmentsToInstanceMap = ImmutableMap.copyOf(_idealStateHelper.getSegmentToInstanceMap(_tableName));
+    if (_segmentsToInstanceMap.size() == 0) {
+      LOGGER.error("failed to get any segment for the current table {}", _tableName);
+    }
+    Map<Integer, LLCSegmentName> partitionLastSeg = new HashMap<>();
+    for (String segmentStr: _segmentsToInstanceMap.keySet()) {
+      LLCSegmentName llcSegmentName = new LLCSegmentName(segmentStr);
+      int partition = llcSegmentName.getPartitionId();
+      if (!partitionLastSeg.containsKey(partition)) {
+        partitionLastSeg.put(partition, llcSegmentName);
+      } else if (compareSegment(llcSegmentName, partitionLastSeg.get(partition))) {
+        partitionLastSeg.put(partition, llcSegmentName);
+      }
+    }
+    _partitionToLastSegment = ImmutableMap.copyOf(partitionLastSeg);
+    LOGGER.info("updated table {} state from helix in {} ms", _tableName, System.currentTimeMillis() - start);
+  }
+
+  /**
+   * decide whether we should ingest the update log for a given segment name
+   * @param segmentName
+   * @return true if we should keep the update log for a particular segment, false otherwise
+   */
+  public boolean shouldIngestForSegment(String segmentName) {
+    if (_segmentsToInstanceMap.containsKey(segmentName)) {
+      return isSegmentAssignedToCurrentServer(segmentName);
+    } else if (_blacklistedSegments.containsKey(segmentName)) {
+      return false;
+    } else {
+      LLCSegmentName llcSegmentName = new LLCSegmentName(segmentName);
+      int partition = llcSegmentName.getPartitionId();
+      if (!_partitionToLastSegment.containsKey(llcSegmentName.getPartitionId())
+          || compareSegment(llcSegmentName, _partitionToLastSegment.get(partition))) {
+        // assume our idealState is out of date
+        updateStateFromHelix();
+        if (_segmentsToInstanceMap.containsKey(segmentName) && isSegmentAssignedToCurrentServer(segmentName)) {
+          LOGGER.info("segment {} matched in ideal state after refresh", segmentName);
+          return true;
+        }
+      }
+      // we most probably got a segment that is from a deleted table or segment assigned to another table
+      // assume we don't do re-balance, we won't do refresh
+      LOGGER.warn("adding segment {} to blacklist", segmentName);
+      _blacklistedSegments.put(segmentName, segmentName);
+      return false;
+    }
+  }
+
+  /**
+   * compare if segment1 is definitely newer segment compared to the segment2
+   * @param segment1
+   * @param segment2
+   * @return true if the segment1 is a "newer" segment
+   */
+  protected boolean compareSegment(LLCSegmentName segment1, LLCSegmentName segment2) {
+    return segment1.getCreationTimeStamp() > segment2.getCreationTimeStamp();
+  }
+
+}
diff --git a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/storageProvider/retentionManager/UpdateLogTableRetentionManagerImpl.java b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/storageProvider/retentionManager/UpdateLogTableRetentionManagerImpl.java
new file mode 100644
index 0000000..91c098a
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/storageProvider/retentionManager/UpdateLogTableRetentionManagerImpl.java
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.storageProvider.retentionManager;
+
+import org.apache.pinot.grigio.common.utils.IdealStateHelper;
+
+import java.util.Arrays;
+
+public class UpdateLogTableRetentionManagerImpl extends UpdateLogTableRetentionManager {
+
+  protected String _instanceId;
+
+  public UpdateLogTableRetentionManagerImpl(IdealStateHelper idealStateHelper, String tableName, String instanceId) {
+    super(idealStateHelper, tableName);
+    _instanceId = instanceId;
+    updateStateFromHelix();
+  }
+
+  @Override
+  public synchronized void notifySegmentDeletion(String segmentName) {
+    LOGGER.info("handling segment deletion in retention manager");
+    updateStateFromHelix();
+    if (isSegmentAssignedToCurrentServer(segmentName)) {
+      LOGGER.error("segment {} data manager has been removed but still in ideal state", segmentName);
+      LOGGER.error("segment ideal state {}", Arrays.toString(_segmentsToInstanceMap.get(segmentName).entrySet().toArray()));
+    }
+  }
+
+  @Override
+  public void notifySegmentsChange() {
+    updateStateFromHelix();
+  }
+
+  @Override
+  protected boolean isSegmentAssignedToCurrentServer(String segmentName) {
+    return _segmentsToInstanceMap.containsKey(segmentName)
+        && _segmentsToInstanceMap.get(segmentName).containsKey(_instanceId)
+        && !"DROPPED".equals(_segmentsToInstanceMap.get(segmentName).get(_instanceId));
+  }
+
+}
diff --git a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/updateStrategy/MessageResolveStrategy.java b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/updateStrategy/MessageResolveStrategy.java
new file mode 100644
index 0000000..9547789
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/updateStrategy/MessageResolveStrategy.java
@@ -0,0 +1,25 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.updateStrategy;
+
+import org.apache.pinot.grigio.common.messages.KeyCoordinatorMessageContext;
+
+public interface MessageResolveStrategy {
+  boolean shouldDeleteFirstMessage(KeyCoordinatorMessageContext oldMessage, KeyCoordinatorMessageContext newMessage);
+}
diff --git a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/updateStrategy/MessageTimeResolveStrategy.java b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/updateStrategy/MessageTimeResolveStrategy.java
new file mode 100644
index 0000000..9ccc35a
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/updateStrategy/MessageTimeResolveStrategy.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.updateStrategy;
+
+import org.apache.pinot.common.utils.LLCSegmentName;
+import org.apache.pinot.grigio.common.messages.KeyCoordinatorMessageContext;
+
+public class MessageTimeResolveStrategy implements MessageResolveStrategy {
+  @Override
+  public boolean shouldDeleteFirstMessage(KeyCoordinatorMessageContext message1, KeyCoordinatorMessageContext message2) {
+    if (message1.getTimestamp() < message2.getTimestamp()) {
+      return true;
+    } else if (message1.getTimestamp() > message2.getTimestamp()) {
+      return false;
+    } else {
+      LLCSegmentName messageSegmentName1 = new LLCSegmentName(message1.getSegmentName());
+      LLCSegmentName messageSegmentName2 = new LLCSegmentName(message2.getSegmentName());
+      // if a message in the later segment, it should delete the same message belong to the earlier segment
+      if (messageSegmentName1.getSequenceNumber() < messageSegmentName2.getSequenceNumber()) {
+        return true;
+      } else if (messageSegmentName1.getSequenceNumber() > messageSegmentName2.getSequenceNumber()) {
+        return false;
+      } else {
+        // if both message in the same segment, the later message should delete the first message
+        return message1.getKafkaOffset() < message2.getKafkaOffset();
+      }
+    }
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/utils/CommonUtils.java b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/utils/CommonUtils.java
new file mode 100644
index 0000000..511b90e
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/utils/CommonUtils.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
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.utils;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.configuration.Configuration;
+import org.apache.commons.configuration.ConfigurationConverter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Properties;
+
+public class CommonUtils {
+  private static final Logger LOGGER = LoggerFactory.getLogger(CommonUtils.class);
+
+  public static void printConfiguration(Configuration configuration, String name) {
+    LOGGER.info("printing {} configuration", name);
+    configuration.getKeys().forEachRemaining(key -> {
+      LOGGER.info("{}: {}", key, configuration.getProperty((String)key));
+    });
+  }
+
+  public static Properties getPropertiesFromConf(Configuration conf) {
+    return ConfigurationConverter.getProperties(conf);
+  }
+
+  public static String getTableNameFromKafkaTopic(String topic, String topicPrefix) {
+    Preconditions.checkState(topic.length() > topicPrefix.length(), "kafka topic is not valid");
+    return topic.substring(topicPrefix.length());
+  }
+
+}
diff --git a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/utils/IdealStateHelper.java b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/utils/IdealStateHelper.java
new file mode 100644
index 0000000..b0c193f
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/utils/IdealStateHelper.java
@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.utils;
+
+import org.apache.helix.HelixAdmin;
+import org.apache.helix.model.IdealState;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * class for getting ideal state from helix, used in update log retention management component to get up-to-date
+ * view of the cluster assignment as helix notification/data manager assignment could be delayed for various reasons
+ */
+public class IdealStateHelper {
+  private static final Logger LOGGER = LoggerFactory.getLogger(IdealStateHelper.class);
+
+  private final String _clusterName;
+  private final HelixAdmin _helixAdmin;
+
+  public IdealStateHelper(HelixAdmin helixAdmin, String clusterName) {
+    _clusterName = clusterName;
+    _helixAdmin = helixAdmin;
+  }
+
+  private IdealState getResourceIdealState(String resourceName) {
+    return _helixAdmin.getResourceIdealState(_clusterName, resourceName);
+  }
+
+  public Set<String> getAllSegmentsForTable(String tableNameWithType) {
+    IdealState idealState = getResourceIdealState(tableNameWithType);
+    if (idealState == null || !idealState.isEnabled()) {
+      LOGGER.warn("ideal state for table {} is not found", tableNameWithType);
+      return new HashSet<>();
+    } else {
+      return idealState.getPartitionSet();
+    }
+  }
+
+  /**
+   * fetch the mapping of {segmentName: {instanceId: helixState}} to the caller for a given table
+   * the information will be the ideal state info stored on helix zk node
+   * @param tableNameWithType the name of the table with type
+   * @return
+   */
+  public Map<String, Map<String, String>> getSegmentToInstanceMap(String tableNameWithType) {
+    Map<String, Map<String, String>> segmentToInstanceMap = new HashMap<>();
+    IdealState idealState = getResourceIdealState(tableNameWithType);
+    if (idealState == null || !idealState.isEnabled()) {
+      LOGGER.warn("ideal state for table {} is not found or disabled", tableNameWithType);
+    } else {
+      for (String partitionName: idealState.getPartitionSet()) {
+        segmentToInstanceMap.put(partitionName, idealState.getInstanceStateMap(partitionName));
+      }
+    }
+    return segmentToInstanceMap;
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/servers/GrigioServerMetrics.java b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/servers/GrigioServerMetrics.java
new file mode 100644
index 0000000..8028d00
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/servers/GrigioServerMetrics.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.servers;
+
+import com.yammer.metrics.core.MetricsRegistry;
+import org.apache.pinot.grigio.common.metrics.GrigioGauge;
+import org.apache.pinot.grigio.common.metrics.GrigioMeter;
+import org.apache.pinot.grigio.common.metrics.GrigioMetrics;
+
+public class GrigioServerMetrics extends GrigioMetrics {
+
+  private GrigioMeter[] meters = filterMeterByTypes(MetricsType.BOTH, MetricsType.SERVER_ONLY);
+  private GrigioGauge[] gauges = filterGaugeByTypes(MetricsType.BOTH, MetricsType.SERVER_ONLY);
+
+  public GrigioServerMetrics(String prefix, MetricsRegistry metricsRegistry) {
+    super(prefix, metricsRegistry);
+  }
+
+  @Override
+  protected GrigioMeter[] getMeters() {
+    return meters;
+  }
+
+  @Override
+  protected GrigioGauge[] getGauges() {
+    return gauges;
+  }
+
+}
diff --git a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/servers/KeyCoordinatorProvider.java b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/servers/KeyCoordinatorProvider.java
new file mode 100644
index 0000000..2ac806d
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/servers/KeyCoordinatorProvider.java
@@ -0,0 +1,98 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.servers;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.commons.configuration.Configuration;
+import org.apache.commons.lang.StringUtils;
+import org.apache.pinot.common.Utils;
+import org.apache.pinot.grigio.common.config.CommonConfig;
+import org.apache.pinot.grigio.common.metrics.GrigioMetrics;
+import org.apache.pinot.grigio.common.rpcQueue.QueueProducer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.pinot.grigio.common.config.CommonConfig.RPC_QUEUE_CONFIG.CLASS_NAME;
+import static org.apache.pinot.grigio.common.config.CommonConfig.RPC_QUEUE_CONFIG.PRODUCER_CONFIG_KEY;
+
+
+public class KeyCoordinatorProvider {
+  private static final Logger LOGGER = LoggerFactory.getLogger(KeyCoordinatorProvider.class);
+
+  @VisibleForTesting
+  protected static KeyCoordinatorProvider _instance = null;
+
+  private String _className;
+  private Configuration _producerConf;
+  private Map<String, QueueProducer> _cachedProducerMap = new HashMap<>();
+  private volatile boolean _isClosed = false;
+  private GrigioMetrics _grigioMetrics;
+
+  public KeyCoordinatorProvider(Configuration conf, String hostname, GrigioMetrics grigioMetrics) {
+    Preconditions.checkState(StringUtils.isNotEmpty(hostname), "host name should not be empty");
+    _producerConf = conf.subset(PRODUCER_CONFIG_KEY);
+    _className = _producerConf.getString(CLASS_NAME);
+    _grigioMetrics = grigioMetrics;
+    Preconditions.checkState(StringUtils.isNotEmpty(_className),
+        "key coordinator producer class should not be empty");
+    _producerConf.addProperty(CommonConfig.RPC_QUEUE_CONFIG.HOSTNAME_KEY, hostname);
+
+    synchronized (KeyCoordinatorProvider.class) {
+      if (_instance == null) {
+        _instance = this;
+      } else {
+        throw new RuntimeException("cannot re-initialize key coordinator provide when there is already one instance");
+      }
+    }
+  }
+
+  public static KeyCoordinatorProvider getInstance() {
+    if (_instance != null) {
+      return _instance;
+    } else {
+      throw new RuntimeException("cannot get instance of key coordinator provider without initializing one before");
+    }
+  }
+
+  public synchronized QueueProducer getCachedProducer(String tableName) {
+    Preconditions.checkState(!_isClosed, "provider should not be closed");
+    return _cachedProducerMap.computeIfAbsent(tableName, t -> createNewProducer());
+  }
+
+  private synchronized QueueProducer createNewProducer() {
+    QueueProducer producer = null;
+    try {
+      producer = (QueueProducer) Class.forName(_className).newInstance();
+      producer.init(_producerConf, _grigioMetrics);
+    } catch (ClassNotFoundException | InstantiationException | IllegalAccessException ex) {
+      LOGGER.info("failed to load/create class for key coordinator producer for class {}", _className);
+      Utils.rethrowException(ex);
+    }
+    return producer;
+  }
+
+  public synchronized void close() {
+    _isClosed = true;
+    _cachedProducerMap.values().forEach(QueueProducer::close);
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/servers/SegmentUpdaterProvider.java b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/servers/SegmentUpdaterProvider.java
new file mode 100644
index 0000000..4a662df
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/servers/SegmentUpdaterProvider.java
@@ -0,0 +1,83 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.servers;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.configuration.Configuration;
+import org.apache.commons.lang.StringUtils;
+import org.apache.pinot.common.Utils;
+import org.apache.pinot.grigio.common.config.CommonConfig;
+import org.apache.pinot.grigio.common.metrics.GrigioMetrics;
+import org.apache.pinot.grigio.common.rpcQueue.QueueConsumer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Component to provide the segment updater for related components
+ * Right now it provide the consumer to ingest data from key coordinator output queue
+ */
+public class SegmentUpdaterProvider {
+  private static final Logger LOGGER = LoggerFactory.getLogger(SegmentUpdaterProvider.class);
+  private static SegmentUpdaterProvider _instance = null;
+
+  private Configuration _conf;
+  private QueueConsumer _consumer;
+
+  public SegmentUpdaterProvider(Configuration conf, String hostName, GrigioMetrics metrics) {
+    Preconditions.checkState(StringUtils.isNotEmpty(hostName), "host name should not be empty");
+    _conf = conf;
+    Configuration consumerConfig = conf.subset(CommonConfig.RPC_QUEUE_CONFIG.CONSUMER_CONFIG_KEY);
+    String className = consumerConfig.getString(CommonConfig.RPC_QUEUE_CONFIG.CLASS_NAME);
+    Preconditions.checkState(StringUtils.isNotEmpty(className),
+        "segment updater class path should not be empty");
+    try {
+      consumerConfig.setProperty(CommonConfig.RPC_QUEUE_CONFIG.HOSTNAME_KEY, hostName);
+      _consumer = (QueueConsumer) Class.forName(className).newInstance();
+      _consumer.init(consumerConfig, metrics);
+      synchronized (SegmentUpdaterProvider.class) {
+        if (_instance == null) {
+          _instance = this;
+        } else {
+          throw new RuntimeException("cannot re-initialize segment updater provide when there is already one instance");
+        }
+      }
+    } catch (ClassNotFoundException | InstantiationException | IllegalAccessException ex) {
+      LOGGER.info("failed to load/create class for segment updater consumer: {} ", className);
+      Utils.rethrowException(ex);
+    }
+  }
+
+  public static SegmentUpdaterProvider getInstance() {
+    if (_instance != null) {
+      return _instance;
+    } else {
+      throw new RuntimeException("cannot get instance of segment updater provider without initializing one before");
+    }
+
+  }
+
+  public QueueConsumer getConsumer() {
+    return _consumer;
+  }
+
+  public void close() {
+    //TODO close producer and what not
+    _consumer.close();
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/keyValueStore/ByteArrayWrapperTest.java b/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/keyValueStore/ByteArrayWrapperTest.java
new file mode 100644
index 0000000..d033bc0
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/keyValueStore/ByteArrayWrapperTest.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.keyValueStore;
+
+import org.testng.Assert;
+import org.testng.annotations.BeforeTest;
+import org.testng.annotations.Test;
+
+public class ByteArrayWrapperTest {
+
+  private byte[] data = new byte[]{0, 1};
+  private ByteArrayWrapper wrapper = new ByteArrayWrapper(data);
+
+  @BeforeTest
+  public void setup() {
+  }
+
+  @Test
+  public void testGetData() {
+    Assert.assertEquals(wrapper.getData(), data);
+  }
+
+  @Test
+  public void testEquals1() {
+    Assert.assertEquals(wrapper, new ByteArrayWrapper(new byte[]{0, 1}));
+    Assert.assertNotEquals(wrapper, new ByteArrayWrapper(new byte[]{}));
+    Assert.assertNotEquals(wrapper, new ByteArrayWrapper(new byte[]{1, 1}));
+  }
+}
\ No newline at end of file
diff --git a/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/messages/KeyCoordinatorMessageContextTest.java b/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/messages/KeyCoordinatorMessageContextTest.java
new file mode 100644
index 0000000..f8ff937
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/messages/KeyCoordinatorMessageContextTest.java
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.messages;
+
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+public class KeyCoordinatorMessageContextTest {
+
+  private KeyCoordinatorMessageContext context;
+
+  @BeforeClass
+  public void setup() {
+    context = new KeyCoordinatorMessageContext("name", 123, 100);
+  }
+
+  @Test
+  public void testGetMembers() {
+    Assert.assertEquals(context.getSegmentName(), "name");
+    Assert.assertEquals(context.getTimestamp(), 123);
+    Assert.assertEquals(context.getKafkaOffset(), 100);
+  }
+
+  @Test
+  public void testSerialDeserial() {
+    Assert.assertEquals(KeyCoordinatorMessageContext.fromBytes(context.toBytes()).get(), context);
+  }
+
+  @Test
+  public void testEquals() {
+    Assert.assertEquals(context, new KeyCoordinatorMessageContext("name", 123, 100));
+    Assert.assertNotEquals(context, new KeyCoordinatorMessageContext("name1", 123, 100));
+    Assert.assertNotEquals(context, new KeyCoordinatorMessageContext("name", 12, 100));
+    Assert.assertNotEquals(context, new KeyCoordinatorMessageContext("name", 123, 0));
+  }
+
+}
\ No newline at end of file
diff --git a/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/messages/KeyCoordinatorQueueMsgTest.java b/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/messages/KeyCoordinatorQueueMsgTest.java
new file mode 100644
index 0000000..a2fdbaf
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/messages/KeyCoordinatorQueueMsgTest.java
@@ -0,0 +1,110 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.messages;
+
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import java.nio.charset.StandardCharsets;
+
+import static org.testng.Assert.*;
+
+public class KeyCoordinatorQueueMsgTest {
+
+  private KeyCoordinatorQueueMsg msg;
+  private KeyCoordinatorQueueMsg versionMsg;
+  private String key = "abc";
+  private byte[] keyBytes = key.getBytes(StandardCharsets.UTF_8);
+  private String segmentName = "table_name__15__9__20190718T1930Z";
+  private long offset = 1000;
+  private long timestamp = 1000;
+  private long version = 100;
+
+  @BeforeClass
+  public void setup() {
+    msg = new KeyCoordinatorQueueMsg(keyBytes, segmentName, offset, timestamp);
+    versionMsg = new KeyCoordinatorQueueMsg(version);
+  }
+
+  @Test
+  public void testIsVersionMessage() {
+    assertFalse(msg.isVersionMessage());
+    assertTrue(versionMsg.isVersionMessage());
+  }
+
+  @Test
+  public void testGetKey() {
+    assertEquals(msg.getKey(), keyBytes);
+  }
+
+  @Test
+  public void testGetSegmentName() {
+    assertEquals(msg.getSegmentName(), segmentName);
+  }
+
+  @Test
+  public void testGetTimestamp() {
+    assertEquals(msg.getTimestamp(), timestamp);
+  }
+
+  @Test
+  public void testGetKafkaOffset() {
+    assertEquals(msg.getKafkaOffset(), offset);
+  }
+
+  @Test(expectedExceptions = IllegalStateException.class)
+  public void testGetVersionFails() {
+    msg.getVersion();
+  }
+
+  @Test
+  public void testGetPinotTable() {
+    assertEquals(msg.getPinotTableName(), "table_name");
+  }
+
+  @Test(expectedExceptions = IllegalStateException.class)
+  public void testGetKeyFails() {
+    versionMsg.getKey();
+  }
+
+  @Test(expectedExceptions = IllegalStateException.class)
+  public void testGetSegmentNameFails() {
+    versionMsg.getSegmentName();
+  }
+
+  @Test(expectedExceptions = IllegalStateException.class)
+  public void testGetTimestampFails() {
+    versionMsg.getTimestamp();
+  }
+
+  @Test(expectedExceptions = IllegalStateException.class)
+  public void testGetKafkaOffsetFails() {
+    versionMsg.getKafkaOffset();
+  }
+
+  @Test
+  public void testGetVersion() {
+    assertEquals(version, versionMsg.getVersion());
+  }
+
+  @Test(expectedExceptions = IllegalStateException.class)
+  public void testGetPinotTableFails() {
+    versionMsg.getPinotTableName();
+  }
+}
\ No newline at end of file
diff --git a/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/messages/LogCoordinatorMessageTest.java b/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/messages/LogCoordinatorMessageTest.java
new file mode 100644
index 0000000..576b566
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/messages/LogCoordinatorMessageTest.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.messages;
+
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+public class LogCoordinatorMessageTest {
+
+  private LogCoordinatorMessage message;
+
+  @BeforeClass
+  public void setup() {
+    message = new LogCoordinatorMessage("name", 123, 1, LogEventType.INSERT);
+  }
+
+  @Test
+  public void testGets() {
+    Assert.assertEquals(message.getSegmentName(), "name");
+    Assert.assertEquals(message.getKafkaOffset(), 123);
+    Assert.assertEquals(message.getValue(), 1);
+    Assert.assertEquals(message.getUpdateEventType(), LogEventType.INSERT);
+
+  }
+
+}
\ No newline at end of file
diff --git a/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/messages/LogEventTypeTest.java b/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/messages/LogEventTypeTest.java
new file mode 100644
index 0000000..aa9103f
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/messages/LogEventTypeTest.java
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.messages;
+
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+public class LogEventTypeTest {
+
+  @Test
+  public void testGetUUID() {
+    Assert.assertEquals(0, LogEventType.INSERT.getUUID());
+    Assert.assertEquals(1, LogEventType.DELETE.getUUID());
+  }
+
+  @Test
+  public void testGetEventType() {
+    Assert.assertEquals(LogEventType.getEventType(0), LogEventType.INSERT);
+    Assert.assertEquals(LogEventType.getEventType(1), LogEventType.DELETE);
+  }
+}
\ No newline at end of file
diff --git a/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/metrics/MockGrigioServerMetrics.java b/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/metrics/MockGrigioServerMetrics.java
new file mode 100644
index 0000000..1123c62
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/metrics/MockGrigioServerMetrics.java
@@ -0,0 +1,29 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.metrics;
+
+import com.yammer.metrics.core.MetricsRegistry;
+import org.apache.pinot.grigio.servers.GrigioServerMetrics;
+
+public class MockGrigioServerMetrics extends GrigioServerMetrics {
+
+  public MockGrigioServerMetrics() {
+    super("", new MetricsRegistry());
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/rpcQueue/ProduceTaskTest.java b/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/rpcQueue/ProduceTaskTest.java
new file mode 100644
index 0000000..652944f
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/rpcQueue/ProduceTaskTest.java
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.rpcQueue;
+
+import org.testng.Assert;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class ProduceTaskTest {
+
+  private ProduceTask<String, String> task;
+
+  @BeforeMethod
+  public void setup() {
+    this.task = new ProduceTask<>("topic", "key", "value");
+  }
+
+  @Test
+  public void testGets() {
+    Assert.assertEquals(task.getTopic(), "topic");
+    Assert.assertEquals(task.getKey(), "key");
+    Assert.assertEquals(task.getValue(), "value");
+  }
+
+  @Test
+  public void testSetCallback() {
+
+    final AtomicInteger successCount = new AtomicInteger(0);
+    final AtomicInteger failureCount = new AtomicInteger(0);
+    ProduceTask.Callback callback = new ProduceTask.Callback() {
+      @Override
+      public void onSuccess() {
+        successCount.incrementAndGet();
+      }
+
+      @Override
+      public void onFailure(Exception ex) {
+        failureCount.incrementAndGet();
+      }
+    };
+    task.setCallback(callback);
+    task.markComplete(new Object(), null);
+    Assert.assertEquals(successCount.get(), 1);
+    Assert.assertEquals(failureCount.get(), 0);
+    task = new ProduceTask<>("key", "value");
+    task.setCallback(callback);
+    task.markComplete(null, new Exception());
+    Assert.assertEquals(successCount.get(), 1);
+    Assert.assertEquals(failureCount.get(), 1);
+  }
+
+  @Test
+  public void testMarkComplete() {
+    task.markComplete(new Object(), null);
+    Assert.assertTrue(task.isSucceed());
+  }
+
+  @Test
+  public void testMarkException() {
+    task.markComplete(new Object(), new Exception());
+    Assert.assertTrue(!task.isSucceed());
+  }
+}
\ No newline at end of file
diff --git a/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/rpcQueue/QueueConsumerRecordTest.java b/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/rpcQueue/QueueConsumerRecordTest.java
new file mode 100644
index 0000000..b129d08
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/rpcQueue/QueueConsumerRecordTest.java
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.rpcQueue;
+
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+public class QueueConsumerRecordTest {
+
+  @Test
+  public void testGets() {
+    QueueConsumerRecord<String, String> record = new QueueConsumerRecord<>("topic", 1, 2, "key", "record", 123);
+    Assert.assertEquals(record.getTopic(), "topic");
+    Assert.assertEquals(record.getPartition(), 1);
+    Assert.assertEquals(record.getOffset(), 2);
+    Assert.assertEquals(record.getKey(), "key");
+    Assert.assertEquals(record.getRecord(), "record");
+    Assert.assertEquals(record.getTimestamp(), 123);
+
+  }
+}
\ No newline at end of file
diff --git a/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/storageProvider/SegmentUpdateLogStorageProviderTest.java b/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/storageProvider/SegmentUpdateLogStorageProviderTest.java
new file mode 100644
index 0000000..1eba023
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/storageProvider/SegmentUpdateLogStorageProviderTest.java
@@ -0,0 +1,163 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.storageProvider;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.pinot.grigio.common.messages.LogEventType;
+import org.testng.Assert;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+public class SegmentUpdateLogStorageProviderTest {
+
+  protected volatile UpdateLogEntry entryHolder;
+  SegmentUpdateLogStorageProvider provider;
+  List<UpdateLogEntry> inputDataList = ImmutableList.of(
+      new UpdateLogEntry(1, 2, LogEventType.INSERT, 0),
+      new UpdateLogEntry(2, 3, LogEventType.DELETE, 0),
+      new UpdateLogEntry(4,10, LogEventType.DELETE, 0));
+
+  @BeforeMethod
+  public void setup() throws IOException {
+    File file = File.createTempFile("test", "postFix");
+    file.deleteOnExit();
+    this.provider = new SegmentUpdateLogStorageProvider(file);
+  }
+
+  @Test
+  public void testWriteAndReadData() throws IOException {
+    UpdateLogEntrySet logEntrySet = provider.readAllMessagesFromFile();
+    // new file should have no data
+    Assert.assertEquals(logEntrySet.size(), 0);
+    provider.addData(inputDataList);
+
+    SegmentUpdateLogStorageProvider provider1= new SegmentUpdateLogStorageProvider(provider._file);
+    logEntrySet = provider1.readAllMessagesFromFile();
+    Iterator<UpdateLogEntry> it = logEntrySet.iterator();
+    Assert.assertEquals(logEntrySet.size(), inputDataList.size());
+    Assert.assertEquals(it.next(), inputDataList.get(0));
+    Assert.assertEquals(it.next(), inputDataList.get(1));
+    Assert.assertEquals(it.next(), inputDataList.get(2));
+    Assert.assertFalse(it.hasNext());
+    provider.addData(inputDataList);
+    Assert.assertEquals(provider.readAllMessagesFromFile().size(), inputDataList.size() * 2);
+  }
+
+  @Test
+  public void testReadPartialData() throws IOException {
+    provider.addData(inputDataList);
+    // write some more data to channel and persist it
+    provider._outputStream.write(new byte[]{1,2,3});
+    provider._outputStream.flush();
+
+    SegmentUpdateLogStorageProvider provider1 = new SegmentUpdateLogStorageProvider(provider._file);
+    UpdateLogEntrySet logEntrySet = provider1.readAllMessagesFromFile();
+    Iterator<UpdateLogEntry> it = logEntrySet.iterator();
+    Assert.assertEquals(logEntrySet.size(), inputDataList.size());
+    Assert.assertEquals(it.next(), inputDataList.get(0));
+    Assert.assertEquals(it.next(), inputDataList.get(1));
+    Assert.assertEquals(it.next(), inputDataList.get(2));
+    Assert.assertFalse(it.hasNext());
+  }
+
+  @Test
+  public void testMultiThreadReadAndWrite() throws InterruptedException, IOException {
+
+    ExecutorService service = Executors.newFixedThreadPool(2);
+    final long writeIterationCount = 1000;
+    final long readIterationCount = 100;
+    List<Callable<Object>> tasks = new ArrayList<>();
+    tasks.add(() -> {
+      for (int i = 0; i < writeIterationCount; i++) {
+        try {
+          provider.addData(inputDataList);
+        } catch (IOException e) {
+          Assert.fail();
+        }
+      }
+      return null;
+    });
+    tasks.add(() -> {
+      for (int i = 0; i < readIterationCount; i++) {
+        try {
+          provider.readAllMessagesFromFile();
+        } catch (IOException e) {
+          Assert.fail();
+        }
+      }
+      return null;
+    });
+    service.invokeAll(tasks);
+    service.shutdownNow();
+    UpdateLogEntrySet updateLogEntries = provider.readAllMessagesFromFile();
+    Assert.assertEquals(updateLogEntries.size(), writeIterationCount * inputDataList.size());
+    Iterator<UpdateLogEntry> it = updateLogEntries.iterator();
+    for (int i = 0; i < writeIterationCount; i++) {
+      Assert.assertEquals(it.next(), inputDataList.get(0));
+      Assert.assertEquals(it.next(), inputDataList.get(1));
+      Assert.assertEquals(it.next(), inputDataList.get(2));
+    }
+    Assert.assertFalse(it.hasNext());
+  }
+
+  @Test
+  public void testReadMesssagePerf() throws IOException {
+    int totalMessageCount = 5_000_000;
+    // write a lot of data to file
+    List<UpdateLogEntry> inputMessages = new ArrayList<>(totalMessageCount * 2);
+    for (int i = 0; i < totalMessageCount; i++) {
+      inputMessages.add(new UpdateLogEntry(i, 50, LogEventType.INSERT, i%8));
+      inputMessages.add(new UpdateLogEntry(i, 100, LogEventType.DELETE, i%8));
+    }
+    long start = System.currentTimeMillis();
+    provider.addData(inputMessages);
+    System.out.println("write data takes ms: " + (System.currentTimeMillis() - start));
+
+    // load data from file to temp object, we don't measure this performance as it depends on disk/computer
+    start = System.currentTimeMillis();
+    UpdateLogEntrySet entrySet = provider.readAllMessagesFromFile();
+    long loadTime = System.currentTimeMillis() - start;
+    System.out.println("load data takes ms: " + loadTime);
+    Assert.assertTrue(entrySet.size() == totalMessageCount * 2);
+
+    // old implementation where we hold the data in array list will take 1000 - 2000 seconds for the data loading
+    // using iterator (current implementation) should make this code finished within 300 - 600 ms.
+    // test accessing those object
+    start = System.currentTimeMillis();
+    for (UpdateLogEntry entry: entrySet) {
+      // ensure we hold them in volatile member to force JVM allocate the object and
+      // prevent JIT optimize this part of code away
+      entryHolder = entry;
+    }
+    long readTime = System.currentTimeMillis() - start;
+    System.out.println("read data takes ms: " + readTime);
+    // this should be relatively fast (under 1 seconds), but on integration server it might take longer
+    // TODO: make sure this runs fine on integration server as well
+    Assert.assertTrue(readTime < 10_000L);
+  }
+}
\ No newline at end of file
diff --git a/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/storageProvider/UpdateLogEntryTest.java b/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/storageProvider/UpdateLogEntryTest.java
new file mode 100644
index 0000000..49ba020
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/storageProvider/UpdateLogEntryTest.java
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.storageProvider;
+
+import org.apache.pinot.grigio.common.messages.LogEventType;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import java.nio.ByteBuffer;
+
+public class UpdateLogEntryTest {
+
+  @Test
+  public void testGets() {
+    UpdateLogEntry entry = new UpdateLogEntry(123, 12, LogEventType.INSERT, 0);
+    Assert.assertEquals(entry.getOffset(), 123);
+    Assert.assertEquals(entry.getValue(), 12);
+    Assert.assertEquals(entry.getType(), LogEventType.INSERT);
+  }
+
+  @Test
+  public void testEquals() {
+    UpdateLogEntry entry = new UpdateLogEntry(123, 12, LogEventType.INSERT, 0);
+    Assert.assertEquals(entry, new UpdateLogEntry(123, 12, LogEventType.INSERT, 0));
+    Assert.assertNotEquals(entry, new UpdateLogEntry(12, 12, LogEventType.INSERT, 0));
+    Assert.assertNotEquals(entry, new UpdateLogEntry(123, 121, LogEventType.INSERT, 0));
+    Assert.assertNotEquals(entry, new UpdateLogEntry(123, 12, LogEventType.DELETE, 0));
+  }
+
+  @Test
+  public void testSerialDeserial() {
+    ByteBuffer buffer = ByteBuffer.allocate(UpdateLogEntry.SIZE);
+    UpdateLogEntry entry = new UpdateLogEntry(1, 2, LogEventType.INSERT, 0);
+    entry.addEntryToBuffer(buffer);
+    buffer.flip();
+    Assert.assertEquals(UpdateLogEntry.fromBytesBuffer(buffer), entry);
+  }
+}
\ No newline at end of file
diff --git a/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/storageProvider/UpdateLogStorageProviderTest.java b/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/storageProvider/UpdateLogStorageProviderTest.java
new file mode 100644
index 0000000..2ca3daf
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/storageProvider/UpdateLogStorageProviderTest.java
@@ -0,0 +1,100 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.storageProvider;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.io.Files;
+import org.apache.commons.configuration.Configuration;
+import org.apache.commons.configuration.PropertiesConfiguration;
+import org.apache.pinot.grigio.common.messages.LogEventType;
+import org.testng.Assert;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+
+public class UpdateLogStorageProviderTest {
+
+  private Configuration conf;
+  private File tempDir;
+
+  @BeforeMethod
+  public void setup() {
+    conf = new PropertiesConfiguration();
+    tempDir = Files.createTempDir();
+    tempDir.deleteOnExit();
+    conf.setProperty(UpdateLogStorageProvider.BASE_PATH_CONF_KEY, tempDir.getAbsolutePath());
+
+    UpdateLogStorageProvider._instance = null;
+    UpdateLogStorageProvider.init(conf);
+  }
+
+  @Test
+  public void testAddSegment() throws IOException {
+    UpdateLogStorageProvider provider = UpdateLogStorageProvider.getInstance();
+    provider.addSegment("table", "segment");
+    File segmentFile = new File(new File(tempDir, "table"), "segment");
+    Assert.assertTrue(segmentFile.exists());
+  }
+
+  @Test
+  public void testAddAndGetData() throws IOException {
+    UpdateLogStorageProvider provider = UpdateLogStorageProvider.getInstance();
+    List<UpdateLogEntry> inputDataList1 = ImmutableList.of(
+        new UpdateLogEntry(1, 2, LogEventType.INSERT, 0),
+        new UpdateLogEntry(2, 3, LogEventType.DELETE, 0),
+        new UpdateLogEntry(4,10, LogEventType.DELETE, 0));
+    List<UpdateLogEntry> inputDataList2 = ImmutableList.of(
+        new UpdateLogEntry(10, 11, LogEventType.INSERT, 0));
+    List<UpdateLogEntry> inputDataList3 = ImmutableList.of(
+        new UpdateLogEntry(100, 110, LogEventType.DELETE, 0));
+    List<UpdateLogEntry> inputDataList4 = ImmutableList.of();
+
+    provider.addSegment("table", "segment");
+    provider.addSegment("table2", "segment");
+
+    provider.addDataToFile("table", "segment", inputDataList1);
+    provider.addDataToFile("table", "segment1", inputDataList2);
+    provider.addDataToFile("table2", "segment", inputDataList3);
+    provider.addDataToFile("table2", "segment1", inputDataList4);
+
+    Assert.assertEquals(provider.getAllMessages("table", "segment"), inputDataList1);
+    Assert.assertEquals(provider.getAllMessages("table", "segment1"), inputDataList2);
+    Assert.assertEquals(provider.getAllMessages("table2", "segment"), inputDataList3);
+    Assert.assertEquals(provider.getAllMessages("table2", "segment1"), inputDataList4);
+    Assert.assertEquals(provider.getAllMessages("table2", "segment2"), inputDataList4);
+  }
+
+  @Test
+  public void testRemoveSegment() throws IOException {
+    UpdateLogStorageProvider provider = UpdateLogStorageProvider.getInstance();
+    provider.addSegment("table", "segment");
+    provider.addSegment("table", "segment1");
+    provider.addSegment("table1", "segment1");
+
+    provider.removeSegment("table", "segment");
+
+    Assert.assertFalse(new File(new File(tempDir, "table"), "segment").exists());
+    Assert.assertTrue(new File(new File(tempDir, "table"), "segment1").exists());
+    Assert.assertTrue(new File(new File(tempDir, "table1"), "segment1").exists());
+
+  }
+}
\ No newline at end of file
diff --git a/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/storageProvider/retentionManager/UpdateLogTableRetentionManagerImplTest.java b/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/storageProvider/retentionManager/UpdateLogTableRetentionManagerImplTest.java
new file mode 100644
index 0000000..83fba68
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/storageProvider/retentionManager/UpdateLogTableRetentionManagerImplTest.java
@@ -0,0 +1,119 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.storageProvider.retentionManager;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.pinot.grigio.common.utils.IdealStateHelper;
+import org.testng.Assert;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+import static org.mockito.internal.verification.VerificationModeFactory.times;
+
+public class UpdateLogTableRetentionManagerImplTest {
+
+  private IdealStateHelper mockIdealStateHelper;
+  private Map<String, Map<String, String>> segmentsInstanceMap;
+  private UpdateLogTableRetentionManagerImpl updateLogTableRetentionManager;
+
+  @BeforeMethod
+  public void init() {
+    segmentsInstanceMap = new HashMap<>();
+
+    segmentsInstanceMap.put("table__0__10__20191027T2041Z",
+        ImmutableMap.of("server1", "ONLINE", "server2", "ONLINE"));
+    segmentsInstanceMap.put("table__0__11__20191028T2041Z",
+        ImmutableMap.of("server1", "ONLINE", "server3", "ONLINE"));
+    segmentsInstanceMap.put("table__1__10__20191027T2041Z",
+        ImmutableMap.of("server1", "ONLINE", "server2", "ONLINE"));
+    segmentsInstanceMap.put("table__2__10__20191027T2041Z",
+        ImmutableMap.of("server2", "ONLINE", "server3", "ONLINE"));
+
+    mockIdealStateHelper = mock(IdealStateHelper.class);
+    when(mockIdealStateHelper.getSegmentToInstanceMap("table")).thenReturn(segmentsInstanceMap);
+
+    updateLogTableRetentionManager = new UpdateLogTableRetentionManagerImpl(mockIdealStateHelper, "table",
+        "server1");
+  }
+
+  @Test
+  public void testShouldIngestForSegment() {
+    // test the happy paths
+    Assert.assertTrue(updateLogTableRetentionManager.shouldIngestForSegment("table__0__10__20191027T2041Z"));
+    Assert.assertTrue(updateLogTableRetentionManager.shouldIngestForSegment("table__1__10__20191027T2041Z"));
+    verify(mockIdealStateHelper, times(1)).getSegmentToInstanceMap("table");
+
+    // test for segments that has lower seq
+    Assert.assertFalse(updateLogTableRetentionManager.shouldIngestForSegment("table__1__5__20191027T2041Z"));
+    verify(mockIdealStateHelper, times(1)).getSegmentToInstanceMap("table");
+
+    //test for segment that are older and not in existing segment list
+    Assert.assertFalse(updateLogTableRetentionManager.shouldIngestForSegment("table__1__20__20191027T2041Z"));
+    verify(mockIdealStateHelper, times(1)).getSegmentToInstanceMap("table");
+
+    // test for partition that has no assignment on host
+    Assert.assertFalse(updateLogTableRetentionManager.shouldIngestForSegment("table__2__10__20191027T2041Z"));
+    verify(mockIdealStateHelper, times(1)).getSegmentToInstanceMap("table");
+
+    //test for older timestamp but higher seq
+    Assert.assertFalse(updateLogTableRetentionManager.shouldIngestForSegment("table__0__20__20191027T2041Z"));
+    verify(mockIdealStateHelper, times(1)).getSegmentToInstanceMap("table");
+
+    // test for newer segments, and there is update in newer ideal state
+    segmentsInstanceMap.put("table__0__12__20191029T2041Z",
+        ImmutableMap.of("server1", "ONLINE", "server3", "ONLINE"));
+    Assert.assertTrue(updateLogTableRetentionManager.shouldIngestForSegment("table__0__12__20191029T2041Z"));
+    verify(mockIdealStateHelper, times(2)).getSegmentToInstanceMap("table");
+
+    // test for newer segments and there is no update in newer ideal state
+    Assert.assertFalse(updateLogTableRetentionManager.shouldIngestForSegment("table__0__13__20191029T2141Z"));
+    verify(mockIdealStateHelper, times(3)).getSegmentToInstanceMap("table");
+
+    // multiple attempts trying to fetch for segment should not recheck the ideal state often
+    Assert.assertFalse(updateLogTableRetentionManager.shouldIngestForSegment("table__0__13__20191029T2141Z"));
+    verify(mockIdealStateHelper, times(3)).getSegmentToInstanceMap("table");
+
+    // for unknown partition, we should also refresh data
+    Assert.assertFalse(updateLogTableRetentionManager.shouldIngestForSegment("table__2__13__20191029T2141Z"));
+    verify(mockIdealStateHelper, times(4)).getSegmentToInstanceMap("table");
+
+    Assert.assertFalse(updateLogTableRetentionManager.shouldIngestForSegment("table__3__13__20191029T2141Z"));
+    verify(mockIdealStateHelper, times(5)).getSegmentToInstanceMap("table");
+  }
+
+  @Test
+  public void testNotifySegmentDeletion() {
+    verify(mockIdealStateHelper, times(1)).getSegmentToInstanceMap("table");
+    updateLogTableRetentionManager.notifySegmentDeletion("table__0__10__20191027T2041Z");
+    verify(mockIdealStateHelper, times(2)).getSegmentToInstanceMap("table");
+  }
+
+  @Test
+  public void testNotifySegmentsChange() {
+    verify(mockIdealStateHelper, times(1)).getSegmentToInstanceMap("table");
+    updateLogTableRetentionManager.notifySegmentDeletion("table__0__10__20191027T2041Z");
+    verify(mockIdealStateHelper, times(2)).getSegmentToInstanceMap("table");
+  }
+}
\ No newline at end of file
diff --git a/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/utils/IdealStateHelperTest.java b/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/utils/IdealStateHelperTest.java
new file mode 100644
index 0000000..4245e5b
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/common/utils/IdealStateHelperTest.java
@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.utils;
+
+import org.apache.helix.HelixAdmin;
+import org.apache.helix.model.IdealState;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import java.util.Map;
+import java.util.Set;
+
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class IdealStateHelperTest {
+
+  private HelixAdmin mockAmin;
+  private IdealStateHelper idealStateHelper;
+  private IdealState mockIdealState;
+
+  @BeforeClass
+  public void init() {
+    mockIdealState = new IdealState("resource");
+    mockIdealState.enable(true);
+    mockIdealState.setRebalanceMode(IdealState.RebalanceMode.CUSTOMIZED);
+    mockIdealState.setPartitionState("seg1", "host1", "ONLINE");
+    mockIdealState.setPartitionState("seg1", "host2", "ONLINE");
+    mockIdealState.setPartitionState("seg2", "host2", "ONLINE");
+    mockIdealState.setPartitionState("seg2", "host3", "OFFLINE");
+
+    mockAmin = mock(HelixAdmin.class);
+    when(mockAmin.getResourceIdealState(anyString(), anyString())).thenReturn(mockIdealState);
+    idealStateHelper = new IdealStateHelper(mockAmin, "clusterName");
+  }
+
+  @Test
+  public void testGetAllSegmentsForTable() {
+    Set<String> segments = idealStateHelper.getAllSegmentsForTable("table");
+    Assert.assertEquals(segments.size(), 2);
+    Assert.assertTrue(segments.contains("seg1"));
+    Assert.assertTrue(segments.contains("seg2"));
+  }
+
+  @Test
+  public void testGetSegmentToInstanceMap() {
+    Map<String, Map<String, String>> resultMap = idealStateHelper.getSegmentToInstanceMap("table");
+    Assert.assertEquals(resultMap.size(), 2);
+    Assert.assertEquals(resultMap.get("seg1").size(), 2);
+    Assert.assertEquals(resultMap.get("seg1").get("host1"), "ONLINE");
+    Assert.assertEquals(resultMap.get("seg1").get("host2"), "ONLINE");
+    Assert.assertEquals(resultMap.get("seg2").size(), 2);
+    Assert.assertEquals(resultMap.get("seg2").get("host2"), "ONLINE");
+    Assert.assertEquals(resultMap.get("seg2").get("host3"), "OFFLINE");
+  }
+}
\ No newline at end of file
diff --git a/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/servers/KeyCoordinatorProviderTest.java b/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/servers/KeyCoordinatorProviderTest.java
new file mode 100644
index 0000000..a5803b9
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/servers/KeyCoordinatorProviderTest.java
@@ -0,0 +1,106 @@
+package org.apache.pinot.grigio.servers;
+/**
+ * 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.
+ */
+
+import org.apache.commons.configuration.Configuration;
+import org.apache.commons.configuration.PropertiesConfiguration;
+import org.apache.pinot.grigio.common.metrics.MockGrigioServerMetrics;
+import org.apache.pinot.grigio.common.metrics.GrigioMetrics;
+import org.apache.pinot.grigio.common.rpcQueue.ProduceTask;
+import org.apache.pinot.grigio.common.rpcQueue.QueueProducer;
+import org.testng.Assert;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import java.util.List;
+
+import static org.apache.pinot.grigio.common.config.CommonConfig.RPC_QUEUE_CONFIG.CLASS_NAME;
+import static org.apache.pinot.grigio.common.config.CommonConfig.RPC_QUEUE_CONFIG.HOSTNAME_KEY;
+import static org.apache.pinot.grigio.common.config.CommonConfig.RPC_QUEUE_CONFIG.PRODUCER_CONFIG_KEY;
+
+public class KeyCoordinatorProviderTest {
+
+  private Configuration conf = new PropertiesConfiguration();
+
+  @BeforeMethod
+  public void setup() {
+    conf.addProperty(PRODUCER_CONFIG_KEY + "." + CLASS_NAME, MockProducer.class.getName());
+    KeyCoordinatorProvider._instance = null;
+  }
+
+  @Test
+  public void testCreteProducer() {
+    KeyCoordinatorProvider provider = new KeyCoordinatorProvider(conf, "host_name_sample", new MockGrigioServerMetrics());
+
+    MockProducer producer1 = (MockProducer) provider.getCachedProducer("table1");
+    MockProducer producer2 = (MockProducer) provider.getCachedProducer("table2");
+    Configuration producerConfig = producer1._conf;
+
+    Assert.assertEquals(producerConfig.getString(HOSTNAME_KEY), "host_name_sample");
+    Assert.assertEquals(producerConfig.getString(CLASS_NAME), MockProducer.class.getName());
+    Assert.assertEquals(KeyCoordinatorProvider.getInstance(), provider);
+
+    Assert.assertNotEquals(producer1, producer2);
+    Assert.assertEquals(producer1, provider.getCachedProducer("table1"));
+  }
+
+  @Test
+  public void testClose() {
+    KeyCoordinatorProvider provider = new KeyCoordinatorProvider(conf, "host_name_sample", new MockGrigioServerMetrics());
+    MockProducer producer1 = (MockProducer) provider.getCachedProducer("table1");
+    MockProducer producer2 = (MockProducer) provider.getCachedProducer("table2");
+    // verify close logic
+    Assert.assertEquals(producer1._isClosed, false);
+    Assert.assertEquals(producer2._isClosed, false);
+    provider.close();
+    Assert.assertEquals(producer1._isClosed, true);
+    Assert.assertEquals(producer2._isClosed, true);
+  }
+
+  static class MockProducer implements QueueProducer {
+    protected Configuration _conf;
+    protected boolean _isClosed = false;
+
+    @Override
+    public void init(Configuration conf, GrigioMetrics metrics) {
+      _conf = conf;
+    }
+
+    @Override
+    public void produce(ProduceTask task) {
+
+    }
+
+    @Override
+    public void flush() {
+
+    }
+
+    @Override
+    public void close() {
+      _isClosed = true;
+
+    }
+
+    @Override
+    public void batchProduce(List list) {
+
+    }
+  }
+}
\ No newline at end of file
diff --git a/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/servers/SegmentUpdaterProviderTest.java b/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/servers/SegmentUpdaterProviderTest.java
new file mode 100644
index 0000000..f3d97ce
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-common/src/test/java/org/apache/pinot/grigio/servers/SegmentUpdaterProviderTest.java
@@ -0,0 +1,94 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.servers;
+
+import org.apache.commons.configuration.Configuration;
+import org.apache.commons.configuration.PropertiesConfiguration;
+import org.apache.pinot.grigio.common.metrics.MockGrigioServerMetrics;
+import org.apache.pinot.grigio.common.metrics.GrigioMetrics;
+import org.apache.pinot.grigio.common.rpcQueue.QueueConsumer;
+import org.apache.pinot.grigio.common.rpcQueue.QueueConsumerRecord;
+import org.testng.Assert;
+import org.testng.annotations.BeforeTest;
+import org.testng.annotations.Test;
+
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.pinot.grigio.common.config.CommonConfig.RPC_QUEUE_CONFIG.CLASS_NAME;
+import static org.apache.pinot.grigio.common.config.CommonConfig.RPC_QUEUE_CONFIG.CONSUMER_CONFIG_KEY;
+import static org.apache.pinot.grigio.common.config.CommonConfig.RPC_QUEUE_CONFIG.HOSTNAME_KEY;
+
+public class SegmentUpdaterProviderTest {
+
+  private Configuration conf = new PropertiesConfiguration();
+
+  @BeforeTest
+  public void setup() {
+    conf.addProperty(CONSUMER_CONFIG_KEY + "." + CLASS_NAME, MockConsumer.class.getName());
+  }
+
+  @Test
+  public void testGetConsumer() {
+    SegmentUpdaterProvider provider = new SegmentUpdaterProvider(conf, "host_name_sample", new MockGrigioServerMetrics());
+    Configuration conf = ((MockConsumer) provider.getConsumer())._conf;
+
+    Assert.assertEquals(conf.getString(HOSTNAME_KEY), "host_name_sample");
+    Assert.assertEquals(conf.getString(CLASS_NAME), MockConsumer.class.getName());
+    Assert.assertEquals(SegmentUpdaterProvider.getInstance(), provider);
+
+    // verify close logic
+    Assert.assertEquals(((MockConsumer)provider.getConsumer())._isClosed, false);
+    provider.close();
+    Assert.assertEquals(((MockConsumer)provider.getConsumer())._isClosed, true);
+  }
+
+  static class MockConsumer implements QueueConsumer {
+    protected Configuration _conf;
+    protected boolean _isClosed = false;
+
+    @Override
+    public void init(Configuration conf, GrigioMetrics metrics) {
+      _conf = conf;
+    }
+
+    @Override
+    public void subscribeForTable(String tableName, String topicPrefix) {
+    }
+
+    @Override
+    public void unsubscribeForTable(String tableName, String topicPrefix) {
+    }
+
+    @Override
+    public List<QueueConsumerRecord> getRequests(long timeout, TimeUnit timeUnit) {
+      return null;
+    }
+
+    @Override
+    public void ackOffset() {
+    }
+
+    @Override
+    public void close() {
+      _isClosed = true;
+
+    }
+  }
+}
\ No newline at end of file
diff --git a/pinot-grigio/pinot-grigio-coordinator/pom.xml b/pinot-grigio/pinot-grigio-coordinator/pom.xml
new file mode 100644
index 0000000..dbe2916
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/pom.xml
@@ -0,0 +1,104 @@
+<?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.
+
+-->
+<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>pinot-grigio</artifactId>
+    <groupId>org.apache.pinot</groupId>
+    <version>0.3.0-SNAPSHOT</version>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+  <artifactId>pinot-grigio-coordinator</artifactId>
+  <name>Pinot Grigio Coordinator</name>
+  <properties>
+    <pinot.root>${basedir}/../..</pinot.root>
+  </properties>
+  <dependencies>
+    <dependency>
+      <groupId>org.glassfish.jersey.containers</groupId>
+      <artifactId>jersey-container-grizzly2-http</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.helix</groupId>
+      <artifactId>helix-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.glassfish.jersey.media</groupId>
+      <artifactId>jersey-media-json-jackson</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.glassfish.jersey.core</groupId>
+      <artifactId>jersey-server</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-annotations</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.rocksdb</groupId>
+      <artifactId>rocksdbjni</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.pinot</groupId>
+      <artifactId>pinot-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>commons-configuration</groupId>
+      <artifactId>commons-configuration</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>io.swagger</groupId>
+      <artifactId>swagger-jaxrs</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.pinot</groupId>
+      <artifactId>pinot-grigio-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.pinot</groupId>
+      <artifactId>pinot-kafka-${kafka.version}</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-core</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.testng</groupId>
+      <artifactId>testng</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-core</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+</project>
\ No newline at end of file
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/CoordinatorConfig.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/CoordinatorConfig.java
new file mode 100644
index 0000000..c8764d3
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/CoordinatorConfig.java
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common;
+
+public class CoordinatorConfig {
+  public static class KAFKA_CONFIG {
+    public static final String KAFKA_CONFIG_KEY = "kafka.conf";
+
+    public static final int PRODUCER_LINGER_MS = 50;
+    public static final String COMPRESS_TYPE = "snappy";
+    public static final int PRODUCER_RETRIES = 3;
+    public static final int PRODUCER_MAX_BLOCK_TIME_MS = 60000;
+
+    public static final String CONSUMER_GROUP_PREFIX_KEY = "group.prefix";
+  }
+
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/DistributedCommonUtils.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/DistributedCommonUtils.java
new file mode 100644
index 0000000..ffe7b99
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/DistributedCommonUtils.java
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common;
+
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.pinot.grigio.keyCoordinator.starter.KeyCoordinatorConf;
+
+import java.util.Properties;
+
+public class DistributedCommonUtils {
+
+  public static String getKafkaTopicFromTableName(String tableName, String topicPrefix) {
+    return topicPrefix + tableName;
+  }
+
+  public static String getClientId(String hostName) {
+    return KeyCoordinatorConf.KAFKA_CLIENT_ID_PREFIX + hostName;
+  }
+
+  public static void setKakfaLosslessProducerConfig(Properties kafkaProducerConfig, String hostname) {
+    kafkaProducerConfig.put(ProducerConfig.ACKS_CONFIG, "all");
+    if (!kafkaProducerConfig.containsKey(ProducerConfig.RETRIES_CONFIG)) {
+      kafkaProducerConfig.put(ProducerConfig.RETRIES_CONFIG, CoordinatorConfig.KAFKA_CONFIG.PRODUCER_RETRIES);
+    }
+    if (!kafkaProducerConfig.containsKey(ProducerConfig.COMPRESSION_TYPE_CONFIG)) {
+      kafkaProducerConfig.put(ProducerConfig.COMPRESSION_TYPE_CONFIG, CoordinatorConfig.KAFKA_CONFIG.COMPRESS_TYPE);
+    }
+    if (!kafkaProducerConfig.containsKey(ProducerConfig.LINGER_MS_CONFIG)) {
+      kafkaProducerConfig.put(ProducerConfig.LINGER_MS_CONFIG, CoordinatorConfig.KAFKA_CONFIG.PRODUCER_LINGER_MS);
+    }
+    if (!kafkaProducerConfig.containsKey(ProducerConfig.MAX_BLOCK_MS_CONFIG)) {
+      kafkaProducerConfig.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, CoordinatorConfig.KAFKA_CONFIG.PRODUCER_MAX_BLOCK_TIME_MS);
+    }
+    if (!kafkaProducerConfig.containsKey(ProducerConfig.CLIENT_ID_CONFIG)) {
+      kafkaProducerConfig.put(ProducerConfig.CLIENT_ID_CONFIG, DistributedCommonUtils.getClientId(hostname));
+    }
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/FixedPartitionCountBytesPartitioner.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/FixedPartitionCountBytesPartitioner.java
new file mode 100644
index 0000000..9b1de08
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/FixedPartitionCountBytesPartitioner.java
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common;
+
+import org.apache.kafka.common.Cluster;
+import org.apache.kafka.common.utils.Utils;
+
+
+/**
+ * Fixed partition count partitioner that partition with the bytes of the primary key
+ */
+public class FixedPartitionCountBytesPartitioner extends FixedPartitionCountPartitioner {
+
+  @Override
+  public int partition(String topic, Object key, byte[] keyBytes, Object value, byte[] valueBytes, Cluster cluster) {
+    if (keyBytes == null) {
+      throw new IllegalArgumentException("Cannot partition without a key");
+    }
+    int numPartitions = cluster.partitionCountForTopic(topic);
+    int partitionCount = getPartitionCount();
+    if (partitionCount > numPartitions) {
+      throw new IllegalArgumentException(String
+          .format("Cannot partition to %d partitions for records in topic %s, which has only %d partitions.",
+              partitionCount, topic, numPartitions));
+    }
+    return Utils.toPositive(Utils.murmur2(keyBytes)) % partitionCount;
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/FixedPartitionCountIntPartitioner.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/FixedPartitionCountIntPartitioner.java
new file mode 100644
index 0000000..3969098
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/FixedPartitionCountIntPartitioner.java
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common;
+
+import com.google.common.base.Preconditions;
+import org.apache.kafka.common.Cluster;
+
+
+/**
+ * Fixed partition count partitioner that assumes that the primary key is integer and use it as the partition directly
+ */
+public class FixedPartitionCountIntPartitioner extends FixedPartitionCountPartitioner {
+
+  @Override
+  public int partition(String topic, Object key, byte[] keyBytes, Object value, byte[] valueBytes, Cluster cluster) {
+    Preconditions.checkState(key instanceof Integer, "expect key to be an integer");
+    int numPartitions = cluster.partitionCountForTopic(topic);
+    int partitionCount = getPartitionCount();
+    if (partitionCount > numPartitions) {
+      throw new IllegalArgumentException(String
+          .format("Cannot partition to %d partitions for records in topic %s, which has only %d partitions.",
+              partitionCount, topic, numPartitions));
+    }
+    return (Integer) key % partitionCount;
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/FixedPartitionCountPartitioner.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/FixedPartitionCountPartitioner.java
new file mode 100644
index 0000000..7e36abb
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/FixedPartitionCountPartitioner.java
@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common;
+
+import com.google.common.base.Preconditions;
+import java.util.Map;
+import org.apache.kafka.clients.producer.Partitioner;
+
+
+/**
+ * Kafka partitioner that partition records to a fixed number of partitions.
+ * i.e., partition results will not change even when more partitions are added to the Kafka topic
+ */
+public abstract class FixedPartitionCountPartitioner implements Partitioner {
+
+  private static final String PARTITION_COUNT = "partition.count";
+  private int partitionCount;
+
+  int getPartitionCount() {
+    return partitionCount;
+  }
+
+  @Override
+  public void close() {
+
+  }
+
+  @Override
+  public void configure(Map<String, ?> configs) {
+    partitionCount = Integer.parseInt((String)configs.get(PARTITION_COUNT));
+    Preconditions.checkState(partitionCount > 0, "Partition count must be greater than 0");
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/OffsetInfo.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/OffsetInfo.java
new file mode 100644
index 0000000..84b213f
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/OffsetInfo.java
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.pinot.grigio.common.rpcQueue.QueueConsumerRecord;
+
+import javax.annotation.concurrent.NotThreadSafe;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * class wrap around the queue offset information for pinot internal services to use without hardcoding kafka dep
+ */
+@NotThreadSafe
+public class OffsetInfo {
+  private Map<TopicPartition, Long> _offsetMap;
+
+  public OffsetInfo() {
+    _offsetMap = new HashMap<>();
+  }
+
+  @VisibleForTesting
+  public OffsetInfo(Map<TopicPartition, Long> offsets) {
+    _offsetMap = new HashMap<>(offsets);
+  }
+
+  /**
+   * update the largest offset for the given partition with the current record,
+   * if it is already larger than current highest record
+   * @param record the current consumption record
+   */
+  public void updateOffsetIfNecessary(QueueConsumerRecord record) {
+    TopicPartition tp = getTopicPartitionFromRecord(record);
+    long offset = record.getOffset() + 1;
+    if (!_offsetMap.containsKey(tp) || _offsetMap.get(tp) < offset) {
+      _offsetMap.put(tp, offset);
+    }
+  }
+
+  /**
+   * return the mapping of partition to the largest offset
+   */
+  public Map<TopicPartition, OffsetAndMetadata> getOffsetMap() {
+    Map<TopicPartition, OffsetAndMetadata> offsetAndMetadataMap = new HashMap<>();
+    for (Map.Entry<TopicPartition, Long> entry: _offsetMap.entrySet()) {
+      offsetAndMetadataMap.put(entry.getKey(), new OffsetAndMetadata(entry.getValue()));
+    }
+    return offsetAndMetadataMap;
+  }
+
+  private TopicPartition getTopicPartitionFromRecord(QueueConsumerRecord record) {
+    return new TopicPartition(record.getTopic(), record.getPartition());
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/keyValueStore/RocksDBBatchReader.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/keyValueStore/RocksDBBatchReader.java
new file mode 100644
index 0000000..3512414
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/keyValueStore/RocksDBBatchReader.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.keyValueStore;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class RocksDBBatchReader implements Callable<Boolean> {
+  static final int MAX_RETRY_ATTEMPTS = 3;
+  static final long RETRY_WAIT_MS = 1000L;
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(RocksDBBatchReader.class);
+
+  private final RocksDB _rocksDB;
+  private final List<byte[]> _byteKeys;
+  private Map<byte[], byte[]> _result;
+
+  RocksDBBatchReader(RocksDB rocksDB, List<byte[]> byteKeys) {
+    _rocksDB = rocksDB;
+    _byteKeys = byteKeys;
+  }
+
+  Map<byte[], byte[]> getResult() {
+    if (_result == null) {
+      throw new RuntimeException("No data got from RocksDB yet!");
+    }
+    return _result;
+  }
+
+  @Override
+  public Boolean call() {
+    try {
+      _result = _rocksDB.multiGet(_byteKeys);
+      return true;
+    } catch (RocksDBException e) {
+      LOGGER.warn("Failed to read from RocksDB: ", e);
+      return false;
+    }
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/keyValueStore/RocksDBBatchWriter.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/keyValueStore/RocksDBBatchWriter.java
new file mode 100644
index 0000000..a910c02
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/keyValueStore/RocksDBBatchWriter.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.keyValueStore;
+
+import java.util.concurrent.Callable;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.rocksdb.WriteBatch;
+import org.rocksdb.WriteOptions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class RocksDBBatchWriter implements Callable<Boolean> {
+  static final int MAX_RETRY_ATTEMPTS = 3;
+  static final long RETRY_WAIT_MS = 1000L;
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(RocksDBBatchWriter.class);
+
+  private final RocksDB _rocksDB;
+  private final WriteOptions _writeOptions;
+  private final WriteBatch _batch;
+
+  public RocksDBBatchWriter(RocksDB rocksDB, WriteOptions writeOptions, WriteBatch batch) {
+    _rocksDB = rocksDB;
+    _writeOptions = writeOptions;
+    _batch = batch;
+  }
+
+  @Override
+  public Boolean call() {
+    try {
+      _rocksDB.write(_writeOptions, _batch);
+      return true;
+    } catch (RocksDBException e) {
+      LOGGER.warn("Failed to write to RocksDB: ", e);
+      return false;
+    }
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/keyValueStore/RocksDBConfig.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/keyValueStore/RocksDBConfig.java
new file mode 100644
index 0000000..24b95fb
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/keyValueStore/RocksDBConfig.java
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.keyValueStore;
+
+import org.rocksdb.util.SizeUnit;
+
+public class RocksDBConfig {
+  public static final String USE_MEMORY_CONFIG = "rocksdb.env.memory";
+  public static final String KEY_SIZE = "rocksdb.key.size";
+  public static final String WRITE_BUFFER_SIZE = "rocksdb.write.buffer.size";
+  public static final String MAX_WRITE_BUFFER_NUMBER = "rocksdb.write.max.buffer";
+  public static final String MAX_BACKGROUND_COMPACTION_THREADS = "rocksdb.compaction.max";
+  public static final String MAX_BACKGROUND_FLUSH = "rocksdb.background.flush.max";
+  public static final String MAX_BACKGROUND_JOBS = "rocksdb.background.jobs.max";
+  public static final String MAX_OPEN_FILES = "rocksdb.files.open.max";
+  public static final String USE_FSYNC = "rocksdb.fsync.enable";
+  public static final String DELETE_OBSOLETE_FILES_PERIOD = "rocksdb.obsolete.file.delete.micro";
+  public static final String TABLE_NUM_SHARD = "rocksdb.table.shard.num";
+  public static final String MMAP_READ = "rocksdb.mmap.reads";
+  public static final String MMAP_WRITE = "rocksdb.mmap.writes";
+  public static final String ADVICE_ON_RANDOM = "rocksdb.advice.random";
+  public static final String NUM_LEVELS = "rocksdb.num.levels";
+
+  public static final String DATABASE_DIR = "rocksdb.database.dir";
+
+  public static final String WRITE_SYNC = "rocksdb.writes.sync";
+  public static final String WRITE_DISABLE_WAL = "rocksdb.writes.wal.disable";
+
+  public static final String READ_VERIFY_CHECKSUM = "rocksdb.reads.checksum";
+  public static final String READ_USE_TAILING = "rocksdb.reads.tailing";
+
+  // default values
+  public static final long  DEFAULT_WRITE_BUFFER_SIZE = 4 * SizeUnit.MB;
+  public static final int DEFAULT_WRITE_BUFFER_NUM = 2;
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/keyValueStore/RocksDBKeyValueStoreDB.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/keyValueStore/RocksDBKeyValueStoreDB.java
new file mode 100644
index 0000000..e93c155
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/keyValueStore/RocksDBKeyValueStoreDB.java
@@ -0,0 +1,162 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.keyValueStore;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.commons.configuration.Configuration;
+import org.apache.commons.lang.StringUtils;
+import org.apache.pinot.grigio.common.messages.KeyCoordinatorMessageContext;
+import org.apache.pinot.grigio.common.utils.CommonUtils;
+import org.rocksdb.BlockBasedTableConfig;
+import org.rocksdb.Options;
+import org.rocksdb.ReadOptions;
+import org.rocksdb.RocksMemEnv;
+import org.rocksdb.SkipListMemTableConfig;
+import org.rocksdb.WriteOptions;
+import org.rocksdb.util.SizeUnit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+
+public class RocksDBKeyValueStoreDB implements KeyValueStoreDB<ByteArrayWrapper, KeyCoordinatorMessageContext> {
+  private static final Logger LOGGER = LoggerFactory.getLogger(RocksDBKeyValueStoreDB.class);
+
+  private String _DBBasePath;
+  private Options _rocksDBOptions;
+  private WriteOptions _writeOptions;
+  private ReadOptions _readOptions;
+  private final ConcurrentMap<String, RocksDBKeyValueStoreTable> _rocksDBTables = new ConcurrentHashMap<>();
+
+  @Override
+  public void init(Configuration configuration) {
+    CommonUtils.printConfiguration(configuration, "key value store");
+    _DBBasePath = configuration.getString(RocksDBConfig.DATABASE_DIR);
+    LOGGER.info("rocksdb config {}", _DBBasePath);
+    Preconditions.checkState(StringUtils.isNotEmpty(_DBBasePath), "db path should not be empty");
+    File dbDir = new File(_DBBasePath);
+    if (!dbDir.exists()) {
+      LOGGER.info("db directory {} does not exist, creating one now", _DBBasePath);
+      dbDir.mkdirs();
+    }
+    _rocksDBOptions = getDBOptions(configuration);
+    _writeOptions = getWriteOptions(configuration);
+    _readOptions = getReadOptions(configuration);
+  }
+
+  @Override
+  public KeyValueStoreTable<ByteArrayWrapper, KeyCoordinatorMessageContext> getTable(String tableName) {
+    return _rocksDBTables.computeIfAbsent(tableName, t -> {
+      LOGGER.info("adding table {}", tableName);
+      String path = getPathForTable(t);
+      try {
+        return new RocksDBKeyValueStoreTable(path, _rocksDBOptions, _readOptions, _writeOptions);
+      } catch (IOException e) {
+        throw new RuntimeException("failed to open rocksdb for path " + path, e);
+      }
+    });
+  }
+
+  @Override
+  public void deleteTable(String tableName) {
+    LOGGER.info("dropping table {}", tableName);
+    RocksDBKeyValueStoreTable table = _rocksDBTables.remove(tableName);
+    try {
+      table.deleteTable();
+    } catch (IOException e) {
+      LOGGER.error("failed to delete/move files", e);
+    }
+  }
+
+  private String getPathForTable(String table) {
+    return Paths.get(_DBBasePath, table).toString();
+  }
+
+  @VisibleForTesting
+  protected static Options getDBOptions(Configuration configuration) {
+    Options options = new Options();
+    options.setCreateIfMissing(true);
+    if (configuration.getBoolean(RocksDBConfig.USE_MEMORY_CONFIG, false)) {
+      options.setEnv(new RocksMemEnv());
+    }
+    options.setMemTableConfig(new SkipListMemTableConfig());
+    BlockBasedTableConfig tableConfig = new BlockBasedTableConfig();
+    tableConfig.setBlockCacheSize(-1).setCacheNumShardBits(-1);
+    options.setTableFormatConfig(tableConfig);
+
+    options.setWriteBufferSize(configuration.getLong(RocksDBConfig.WRITE_BUFFER_SIZE, 4L * SizeUnit.MB));
+    options.setMaxWriteBufferNumber(configuration.getInt(RocksDBConfig.MAX_WRITE_BUFFER_NUMBER, 2));
+    options.setMaxBackgroundCompactions(configuration.getInt(RocksDBConfig.MAX_BACKGROUND_COMPACTION_THREADS,
+        options.maxBackgroundCompactions()));
+    options.getEnv().setBackgroundThreads(configuration.getInt(RocksDBConfig.MAX_BACKGROUND_COMPACTION_THREADS,
+        options.maxBackgroundCompactions()));
+    options.setMaxBackgroundFlushes(configuration.getInt(RocksDBConfig.MAX_BACKGROUND_FLUSH,
+        options.maxBackgroundFlushes()));
+    options.setMaxBackgroundJobs(configuration.getInt(RocksDBConfig.MAX_OPEN_FILES,
+        options.maxOpenFiles()));
+    options.setUseFsync(configuration.getBoolean(RocksDBConfig.USE_FSYNC, false));
+    options.setDeleteObsoleteFilesPeriodMicros(configuration.getLong(RocksDBConfig.DELETE_OBSOLETE_FILES_PERIOD,
+        0));
+    options.setTableCacheNumshardbits(configuration.getInt(RocksDBConfig.TABLE_NUM_SHARD, 4));
+    options.setAllowMmapReads(configuration.getBoolean(RocksDBConfig.MMAP_READ, false));
+    options.setAllowMmapWrites(configuration.getBoolean(RocksDBConfig.MMAP_WRITE, false));
+    options.setAdviseRandomOnOpen(configuration.getBoolean(RocksDBConfig.ADVICE_ON_RANDOM, false));
+    options.setNumLevels(configuration.getInt(RocksDBConfig.NUM_LEVELS, 7));
+
+    // other default options, can be add to config later
+    // from rocks db performance benchmark suit
+    options.setBloomLocality(0);
+    options.setTargetFileSizeBase(10 * 1048576);
+    options.setMaxBytesForLevelBase(10 * 1048576);
+    options.setLevelZeroStopWritesTrigger(12);
+    options.setLevelZeroSlowdownWritesTrigger(8);
+    options.setLevelZeroFileNumCompactionTrigger(4);
+    options.setMaxCompactionBytes(0);
+    options.setDisableAutoCompactions(false);
+    options.setMaxSuccessiveMerges(0);
+    options.setWalTtlSeconds(0);
+    options.setWalSizeLimitMB(0);
+
+    LOGGER.info("starting with options {}", options.toString());
+    return options;
+  }
+
+  @VisibleForTesting
+  protected static WriteOptions getWriteOptions(Configuration configuration) {
+    WriteOptions writeOptions = new WriteOptions();
+    writeOptions.setSync(configuration.getBoolean(RocksDBConfig.WRITE_SYNC, true));
+    writeOptions.setDisableWAL(configuration.getBoolean(RocksDBConfig.WRITE_DISABLE_WAL, false));
+    return writeOptions;
+  }
+
+  @VisibleForTesting
+  protected static ReadOptions getReadOptions(Configuration configuration) {
+    ReadOptions readOptions = new ReadOptions();
+    readOptions.setVerifyChecksums(configuration.getBoolean(RocksDBConfig.READ_VERIFY_CHECKSUM, false));
+    readOptions.setVerifyChecksums(configuration.getBoolean(RocksDBConfig.READ_USE_TAILING, false));
+    return readOptions;
+  }
+
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/keyValueStore/RocksDBKeyValueStoreTable.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/keyValueStore/RocksDBKeyValueStoreTable.java
new file mode 100644
index 0000000..2534fd6
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/keyValueStore/RocksDBKeyValueStoreTable.java
@@ -0,0 +1,110 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.keyValueStore;
+
+import org.apache.pinot.grigio.common.messages.KeyCoordinatorMessageContext;
+import org.apache.pinot.spi.utils.retry.RetryPolicies;
+import org.rocksdb.Options;
+import org.rocksdb.ReadOptions;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.rocksdb.WriteBatch;
+import org.rocksdb.WriteOptions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+public class RocksDBKeyValueStoreTable implements KeyValueStoreTable<ByteArrayWrapper, KeyCoordinatorMessageContext> {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(RocksDBKeyValueStoreTable.class);
+  private final String _path;
+  private final Options _options;
+  private final ReadOptions _readOptions;
+  private final WriteOptions _writeOptions;
+  private final RocksDB _db;
+
+  public RocksDBKeyValueStoreTable(String path, Options options, ReadOptions readOptions, WriteOptions writeOptions)
+      throws IOException {
+    _path = path;
+    _options = options;
+    _readOptions = readOptions;
+    _writeOptions = writeOptions;
+    try {
+      _db = RocksDB.open(path);
+    } catch (RocksDBException e) {
+      throw new IOException("failed to open rocksdb db", e);
+    }
+  }
+
+  @Override
+  public Map<ByteArrayWrapper, KeyCoordinatorMessageContext> multiGet(List<ByteArrayWrapper> keys) throws IOException {
+    try {
+      List<byte[]> byteKeys = keys.stream().map(ByteArrayWrapper::getData).collect(Collectors.toList());
+      RocksDBBatchReader batchReader = new RocksDBBatchReader(_db, byteKeys);
+      RetryPolicies.fixedDelayRetryPolicy(RocksDBBatchReader.MAX_RETRY_ATTEMPTS, RocksDBBatchReader.RETRY_WAIT_MS).attempt(batchReader);
+      Map<byte[], byte[]> rocksdbResult = batchReader.getResult();
+      Map<ByteArrayWrapper, KeyCoordinatorMessageContext> result = new HashMap<>(rocksdbResult.size());
+      for (Map.Entry<byte[], byte[]> entry : rocksdbResult.entrySet()) {
+        Optional<KeyCoordinatorMessageContext> value = KeyCoordinatorMessageContext.fromBytes(entry.getValue());
+        if (!value.isPresent()) {
+          LOGGER.warn("failed to parse value in kv for key {} and value {}", entry.getKey(), entry.getValue());
+        } else {
+          result.put(new ByteArrayWrapper(entry.getKey()), value.get());
+        }
+      }
+      return result;
+    } catch (Exception e) {
+      throw new IOException("failed to get keys from rocksdb " + _path, e);
+    }
+  }
+
+  @Override
+  public void multiPut(Map<ByteArrayWrapper, KeyCoordinatorMessageContext> keyValuePairs) throws IOException {
+    if (keyValuePairs.size() == 0) {
+      return;
+    }
+    final WriteBatch batch = new WriteBatch();
+    try {
+      for (Map.Entry<ByteArrayWrapper, KeyCoordinatorMessageContext> entry: keyValuePairs.entrySet()) {
+        batch.put(entry.getKey().getData(), entry.getValue().toBytes());
+      }
+      RocksDBBatchWriter batchWriter = new RocksDBBatchWriter(_db, _writeOptions, batch);
+      RetryPolicies.fixedDelayRetryPolicy(RocksDBBatchWriter.MAX_RETRY_ATTEMPTS, RocksDBBatchWriter.RETRY_WAIT_MS).attempt(batchWriter);
+    } catch (Exception e) {
+      throw new IOException("failed to put data to rocksdb table " + _path, e);
+    }
+
+  }
+
+  @Override
+  public void deleteTable() throws IOException {
+    String backupPath = _path + ".bak";
+    Files.delete(Paths.get(backupPath));
+    Files.move(Paths.get(_path), Paths.get(backupPath), StandardCopyOption.COPY_ATTRIBUTES);
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/rpcQueue/KafkaQueueConsumer.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/rpcQueue/KafkaQueueConsumer.java
new file mode 100644
index 0000000..ab00e39
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/rpcQueue/KafkaQueueConsumer.java
@@ -0,0 +1,158 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.rpcQueue;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.pinot.grigio.common.OffsetInfo;
+import org.apache.pinot.grigio.common.metrics.GrigioMeter;
+import org.apache.pinot.grigio.common.metrics.GrigioMetrics;
+import org.apache.pinot.grigio.common.metrics.GrigioTimer;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+
+public abstract class KafkaQueueConsumer<K, V> implements QueueConsumer<K, V> {
+
+  protected abstract KafkaConsumer<K, V> getConsumer();
+  protected abstract GrigioMetrics getMetrics();
+
+  /**
+   * Subscribe to the topic specified
+   * @param topic topic to subscribe
+   */
+  public synchronized void subscribe(String topic) {
+    getLogger().info("Trying to subscribe to kafka topic {}", topic);
+    List<PartitionInfo> partitionInfos = getConsumer().partitionsFor(topic);
+    Preconditions.checkState(partitionInfos != null && partitionInfos.size() > 0, "topic doesn't have any partitions");
+    Set<TopicPartition> subscribedTopicPartitions = new HashSet<>(getConsumer().assignment());
+    partitionInfos.forEach(pi -> subscribedTopicPartitions.add(new TopicPartition(topic, pi.partition())));
+    getLogger().info("Total subscribed topic partitions count: {}", partitionInfos.size());
+    getConsumer().assign(subscribedTopicPartitions);
+  }
+
+  /**
+   * Subscribe to the topic and partition specified
+   * @param topic topic to subscribe
+   * @param partition partition to subscribe
+   */
+  public synchronized void subscribe(String topic, Integer partition) {
+    getLogger().info("Trying to subscribe to kafka topic: {}, partition: {}", topic, partition);
+    Set<TopicPartition> subscribedTopicPartitions = new HashSet<>(getConsumer().assignment());
+    TopicPartition topicPartition = new TopicPartition(topic, partition);
+    if (subscribedTopicPartitions.contains(topicPartition)) {
+      getLogger().error("Already subscribed to topic: {}, partition: {} previously, skipping", topic, partition);
+    } else {
+      subscribedTopicPartitions.add(topicPartition);
+      getLogger().info("Total subscribed topic partitions count: {}", subscribedTopicPartitions.size());
+      getConsumer().assign(subscribedTopicPartitions);
+      getLogger().info("Successfully subscribed to topic: {}, partition: {}", topic, partition);
+    }
+  }
+
+  /**
+   * Unsubscribe to the topic specified
+   * @param topic topic to unsubscribe
+   */
+  public synchronized void unsubscribe(String topic) {
+    getLogger().info("Trying to unsubscribe to kafka topic {}", topic);
+    Set<TopicPartition> resultTopicPartitions =
+        getConsumer().assignment().stream().filter(tp -> !tp.topic().equals(topic)).collect(Collectors.toSet());
+    getLogger().info("Total subscribed topic partitions count: {}", resultTopicPartitions.size());
+    getConsumer().assign(resultTopicPartitions);
+  }
+
+  /**
+   * Unsubscribe to the topic and partition specified
+   * @param topic topic to unsubscribe
+   * @param partition partition to unsubscribe
+   */
+  public synchronized void unsubscribe(String topic, Integer partition) {
+    getLogger().info("Trying to unsubscribe to kafka topic: {}, partition: {}", topic, partition);
+    Set<TopicPartition> resultTopicPartitions =
+        getConsumer().assignment().stream().filter(tp -> !(tp.topic().equals(topic) && tp.partition() == partition)).collect(
+            Collectors.toSet());
+    getLogger().info("Total subscribed topic partitions count: {}", resultTopicPartitions.size());
+    getConsumer().assign(resultTopicPartitions);
+  }
+
+  public Set<TopicPartition> getListOfSubscribedTopicPartitions() {
+    return getConsumer().assignment();
+  }
+
+  public abstract Logger getLogger();
+
+  @Override
+  public synchronized List<QueueConsumerRecord<K, V>> getRequests(long timeout, TimeUnit timeUnit) {
+    long start = System.currentTimeMillis();
+    List<QueueConsumerRecord<K, V>> msgList;
+    if (getConsumer().assignment().size() == 0) {
+      msgList = ImmutableList.of();
+    } else {
+      ConsumerRecords<K, V> records = getConsumerRecords(timeout, timeUnit);
+      msgList = new ArrayList<>(records.count());
+      for (ConsumerRecord<K, V> record : records) {
+        msgList.add(
+            new QueueConsumerRecord<>(record.topic(), record.partition(), record.offset(), record.key(), record.value(),
+                record.timestamp()));
+      }
+    }
+    getMetrics().addMeteredGlobalValue(GrigioMeter.MESSAGE_INGEST_COUNT_PER_BATCH, msgList.size());
+    getMetrics().addTimedValueMs(GrigioTimer.FETCH_MESSAGE_LAG, System.currentTimeMillis() - start);
+    return msgList;
+
+  }
+
+  private synchronized ConsumerRecords<K, V> getConsumerRecords(long timeout, TimeUnit timeUnit) {
+    return getConsumer().poll(timeUnit.toMillis(timeout));
+  }
+
+  @Override
+  public synchronized void ackOffset() {
+    getConsumer().commitSync();
+  }
+
+  public synchronized void ackOffset(OffsetInfo offsetInfo) {
+    getLogger().info("committing offset for consumer");
+    for (Map.Entry<TopicPartition, OffsetAndMetadata> entry : offsetInfo.getOffsetMap().entrySet()) {
+      getLogger().info("topic {} partition {} offset {}", entry.getKey().topic(), entry.getKey().partition(),
+          entry.getValue().offset());
+    }
+    long start = System.currentTimeMillis();
+    getConsumer().commitSync(offsetInfo.getOffsetMap());
+    getMetrics().addTimedValueMs(GrigioTimer.COMMIT_OFFSET_LAG, System.currentTimeMillis() - start);
+  }
+
+  public void close() {
+    getConsumer().close();
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/rpcQueue/KafkaQueueProducer.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/rpcQueue/KafkaQueueProducer.java
new file mode 100644
index 0000000..72e5547
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/rpcQueue/KafkaQueueProducer.java
@@ -0,0 +1,79 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.rpcQueue;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.pinot.grigio.common.metrics.GrigioMetrics;
+import org.apache.pinot.grigio.common.metrics.GrigioTimer;
+
+import java.util.List;
+
+public abstract class KafkaQueueProducer<K, V> implements QueueProducer<K, V> {
+
+  protected abstract KafkaProducer<K, V> getKafkaNativeProducer();
+
+  protected abstract String getDefaultTopic();
+
+  protected abstract GrigioMetrics getMetrics();
+
+  @Override
+  public void produce(ProduceTask<K, V> produceTask) {
+    long startTime = System.currentTimeMillis();
+    getKafkaNativeProducer().send(new ProducerRecord<>(getTopic(produceTask), produceTask.getKey(),
+        produceTask.getValue()), produceTask::markComplete);
+    long produceLag = System.currentTimeMillis() - startTime;
+    getMetrics().addTimedValueMs(GrigioTimer.PRODUCER_LAG, produceLag);
+    getMetrics().addTimedTableValueMs(produceTask.getTopic(), GrigioTimer.PRODUCER_LAG, produceLag);
+  }
+
+  public String getTopic(ProduceTask<K, V> produceTask) {
+    if (StringUtils.isNotEmpty(produceTask.getTopic())) {
+      return produceTask.getTopic();
+    }
+    return getDefaultTopic();
+  }
+
+  @Override
+  public void batchProduce(List<ProduceTask<K, V>> produceTasks) {
+    for (ProduceTask<K, V> task: produceTasks) {
+      produce(task);
+    }
+  }
+
+  /**
+   * this method will wait for the necessary long time to ensure all messages are delivered to kafka (including retries and other)
+   * in order to control the latency of this method, you will need to config the kafka producer config such as retries count and timeout
+   * we should also monitor the metrics to ensure the flush is not taking too long
+   */
+  @Override
+  public void flush() {
+    long startTime = System.currentTimeMillis();
+    getKafkaNativeProducer().flush();
+    long flushLag = System.currentTimeMillis() - startTime;
+    getMetrics().addTimedValueMs(GrigioTimer.FLUSH_LAG, flushLag);
+  }
+
+  @Override
+  public void close() {
+    getKafkaNativeProducer().close();
+  }
+
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/rpcQueue/KeyCoordinatorQueueConsumer.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/rpcQueue/KeyCoordinatorQueueConsumer.java
new file mode 100644
index 0000000..ddece45
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/rpcQueue/KeyCoordinatorQueueConsumer.java
@@ -0,0 +1,93 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.rpcQueue;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.configuration.Configuration;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.pinot.grigio.common.CoordinatorConfig;
+import org.apache.pinot.grigio.common.DistributedCommonUtils;
+import org.apache.pinot.grigio.common.config.CommonConfig;
+import org.apache.pinot.grigio.common.messages.KeyCoordinatorQueueMsg;
+import org.apache.pinot.grigio.common.metrics.GrigioMetrics;
+import org.apache.pinot.grigio.common.utils.CommonUtils;
+import org.apache.pinot.grigio.keyCoordinator.starter.KeyCoordinatorConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Properties;
+
+
+/**
+ * Consumer to ingest data from key coordinator messages produced by pinot servers into key coordinators
+ */
+public class KeyCoordinatorQueueConsumer extends KafkaQueueConsumer<byte[], KeyCoordinatorQueueMsg> {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(KeyCoordinatorQueueConsumer.class);
+
+  private KafkaConsumer<byte[], KeyCoordinatorQueueMsg> _consumer;
+  private GrigioMetrics _metrics;
+
+  /**
+   * @param conf configuration of the kafka key coordinator queue consumer
+   */
+  @Override
+  public void init(Configuration conf, GrigioMetrics metrics) {
+    this._metrics = metrics;
+    String _consumerGroupPrefix = conf.getString(CoordinatorConfig.KAFKA_CONFIG.CONSUMER_GROUP_PREFIX_KEY, KeyCoordinatorConf.KAFKA_CONSUMER_GROUP_ID_PREFIX);
+    String hostname = conf.getString(CommonConfig.RPC_QUEUE_CONFIG.HOSTNAME_KEY);
+
+    Properties kafkaProperties = CommonUtils.getPropertiesFromConf(conf.subset(CoordinatorConfig.KAFKA_CONFIG.KAFKA_CONFIG_KEY));
+    // right now key coordinator version message still generate message with integer as the key.
+    // However, because we don't read key for any real purpose, it is fine to have wrong key decoding scheme here
+    kafkaProperties.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
+    kafkaProperties.put(ConsumerConfig.GROUP_ID_CONFIG, _consumerGroupPrefix + hostname);
+    kafkaProperties.put(ConsumerConfig.CLIENT_ID_CONFIG, DistributedCommonUtils.getClientId(hostname));
+    kafkaProperties.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false);
+    _consumer = new KafkaConsumer<>(kafkaProperties);
+  }
+
+  @Override
+  public void subscribeForTable(String tableName, String topicPrefix) {
+    // nothing as key coordinator don't subscribe for table
+  }
+
+  @Override
+  public void unsubscribeForTable(String tableName, String topicPrefix) {
+    // nothing as key coordinator don't subscribe for table
+  }
+
+  @Override
+  protected KafkaConsumer<byte[], KeyCoordinatorQueueMsg> getConsumer() {
+    Preconditions.checkState(_consumer != null, "consumer is not initialized yet");
+    return _consumer;
+  }
+
+  @Override
+  protected GrigioMetrics getMetrics() {
+    return _metrics;
+  }
+
+  @Override
+  public Logger getLogger() {
+    return LOGGER;
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/rpcQueue/KeyCoordinatorQueueProducer.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/rpcQueue/KeyCoordinatorQueueProducer.java
new file mode 100644
index 0000000..ee417f2
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/rpcQueue/KeyCoordinatorQueueProducer.java
@@ -0,0 +1,75 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.rpcQueue;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.configuration.Configuration;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.common.serialization.ByteArraySerializer;
+import org.apache.pinot.grigio.common.CoordinatorConfig;
+import org.apache.pinot.grigio.common.DistributedCommonUtils;
+import org.apache.pinot.grigio.common.FixedPartitionCountBytesPartitioner;
+import org.apache.pinot.grigio.common.config.CommonConfig;
+import org.apache.pinot.grigio.common.messages.KeyCoordinatorQueueMsg;
+import org.apache.pinot.grigio.common.metrics.GrigioMetrics;
+import org.apache.pinot.grigio.common.utils.CommonUtils;
+
+import java.util.Properties;
+
+public class KeyCoordinatorQueueProducer extends KafkaQueueProducer<byte[], KeyCoordinatorQueueMsg> {
+
+  private Configuration _conf;
+  private String _topic;
+  private GrigioMetrics _grigioMetrics;
+  private KafkaProducer<byte[], KeyCoordinatorQueueMsg> _kafkaProducer;
+
+  @Override
+  protected KafkaProducer<byte[], KeyCoordinatorQueueMsg> getKafkaNativeProducer() {
+    Preconditions.checkState(_kafkaProducer != null, "Producer has not been initialized yet");
+    return _kafkaProducer;
+  }
+
+  @Override
+  protected String getDefaultTopic() {
+    Preconditions.checkState(_topic != null, "Producer has not been initialized yet");
+    return _topic;
+  }
+
+  @Override
+  protected GrigioMetrics getMetrics() {
+    return _grigioMetrics;
+  }
+
+  @Override
+  public void init(Configuration conf, GrigioMetrics grigioMetrics) {
+    _conf = conf;
+    _grigioMetrics = grigioMetrics;
+    _topic = _conf.getString(CommonConfig.RPC_QUEUE_CONFIG.TOPIC_KEY);
+    String hostname = conf.getString(CommonConfig.RPC_QUEUE_CONFIG.HOSTNAME_KEY);
+    final Properties kafkaProducerConfig = CommonUtils.getPropertiesFromConf(
+        conf.subset(CoordinatorConfig.KAFKA_CONFIG.KAFKA_CONFIG_KEY));
+
+    kafkaProducerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
+    kafkaProducerConfig.put(ProducerConfig.PARTITIONER_CLASS_CONFIG, FixedPartitionCountBytesPartitioner.class.getName());
+    DistributedCommonUtils.setKakfaLosslessProducerConfig(kafkaProducerConfig, hostname);
+
+    _kafkaProducer = new KafkaProducer<>(kafkaProducerConfig);
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/rpcQueue/LogCoordinatorQueueProducer.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/rpcQueue/LogCoordinatorQueueProducer.java
new file mode 100644
index 0000000..9d2b0b7
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/rpcQueue/LogCoordinatorQueueProducer.java
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.rpcQueue;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.configuration.Configuration;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.common.serialization.IntegerSerializer;
+import org.apache.pinot.grigio.common.config.CommonConfig;
+import org.apache.pinot.grigio.common.messages.LogCoordinatorMessage;
+import org.apache.pinot.grigio.common.metrics.GrigioMetrics;
+import org.apache.pinot.grigio.common.utils.CommonUtils;
+import org.apache.pinot.grigio.common.CoordinatorConfig;
+import org.apache.pinot.grigio.common.DistributedCommonUtils;
+import org.apache.pinot.grigio.common.FixedPartitionCountIntPartitioner;
+
+import java.util.Properties;
+
+public class LogCoordinatorQueueProducer extends KafkaQueueProducer<Integer, LogCoordinatorMessage> {
+
+  private KafkaProducer<Integer, LogCoordinatorMessage> _kafkaProducer;
+  private String _topic;
+  private GrigioMetrics _grigioMetrics;
+
+  @Override
+  protected KafkaProducer<Integer, LogCoordinatorMessage> getKafkaNativeProducer() {
+    Preconditions.checkState(_kafkaProducer != null, "Producer has not been initialized yet");
+    return _kafkaProducer;
+  }
+
+  @Override
+  protected String getDefaultTopic() {
+    Preconditions.checkState(_topic != null, "Producer has not been initialized yet");
+    return _topic;
+  }
+
+  @Override
+  protected GrigioMetrics getMetrics() {
+    return _grigioMetrics;
+  }
+
+  @Override
+  public void init(Configuration conf, GrigioMetrics grigioMetrics) {
+    final Properties kafkaProducerConfig = CommonUtils.getPropertiesFromConf(
+        conf.subset(CoordinatorConfig.KAFKA_CONFIG.KAFKA_CONFIG_KEY));
+    String hostName = conf.getString(CommonConfig.RPC_QUEUE_CONFIG.HOSTNAME_KEY);
+    _topic = conf.getString(CommonConfig.RPC_QUEUE_CONFIG.TOPIC_KEY);
+    _grigioMetrics = grigioMetrics;
+
+    kafkaProducerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, IntegerSerializer.class.getName());
+    kafkaProducerConfig.put(ProducerConfig.PARTITIONER_CLASS_CONFIG, FixedPartitionCountIntPartitioner.class.getName());
+    DistributedCommonUtils.setKakfaLosslessProducerConfig(kafkaProducerConfig, hostName);
+
+    this._kafkaProducer = new KafkaProducer<>(kafkaProducerConfig);
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/rpcQueue/SegmentUpdateQueueConsumer.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/rpcQueue/SegmentUpdateQueueConsumer.java
new file mode 100644
index 0000000..4f96ba6
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/rpcQueue/SegmentUpdateQueueConsumer.java
@@ -0,0 +1,94 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.rpcQueue;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.configuration.Configuration;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.serialization.IntegerDeserializer;
+import org.apache.pinot.grigio.common.config.CommonConfig;
+import org.apache.pinot.grigio.common.CoordinatorConfig;
+import org.apache.pinot.grigio.common.messages.LogCoordinatorMessage;
+import org.apache.pinot.grigio.common.metrics.GrigioMetrics;
+import org.apache.pinot.grigio.common.utils.CommonUtils;
+import org.apache.pinot.grigio.common.DistributedCommonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Properties;
+
+/**
+ * consumer to ingest data from key coordinator output kafka topic, run in segment updater to fetch all update events
+ */
+public class SegmentUpdateQueueConsumer extends KafkaQueueConsumer<String, LogCoordinatorMessage> {
+  private static final Logger LOGGER = LoggerFactory.getLogger(SegmentUpdateQueueConsumer.class);
+
+  public static final String DEFAULT_CONSUMER_GROUP_ID_PREFIX = "pinot_upsert_updater_";
+
+  private KafkaConsumer<String, LogCoordinatorMessage> _consumer;
+  private GrigioMetrics _metrics;
+
+  @Override
+  public void init(Configuration conf, GrigioMetrics metrics) {
+    this._metrics = metrics;
+    String hostName = conf.getString(CommonConfig.RPC_QUEUE_CONFIG.HOSTNAME_KEY);
+    String groupid = conf.getString(CoordinatorConfig.KAFKA_CONFIG.CONSUMER_GROUP_PREFIX_KEY,
+        DEFAULT_CONSUMER_GROUP_ID_PREFIX) + hostName;
+
+    LOGGER.info("creating segment updater kafka consumer with group id {}", groupid);
+    Properties kafkaProperties = CommonUtils.getPropertiesFromConf(conf.subset(CoordinatorConfig.KAFKA_CONFIG.KAFKA_CONFIG_KEY));
+    kafkaProperties.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, IntegerDeserializer.class.getName());
+    kafkaProperties.put(ConsumerConfig.GROUP_ID_CONFIG, groupid);
+    kafkaProperties.put(ConsumerConfig.CLIENT_ID_CONFIG, DistributedCommonUtils.getClientId(hostName));
+    kafkaProperties.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false);
+    _consumer = new KafkaConsumer<>(kafkaProperties);
+  }
+
+  @Override
+  public void subscribeForTable(String tableName, String topicPrefix) {
+    String topicName = DistributedCommonUtils.getKafkaTopicFromTableName(tableName, topicPrefix);
+    LOGGER.info("subscribing for table {}, kafka topic {}", tableName, topicName);
+    this.subscribe(topicName);
+  }
+
+  @Override
+  public void unsubscribeForTable(String tableName, String topicPrefix) {
+    String topicName = DistributedCommonUtils.getKafkaTopicFromTableName(tableName, topicPrefix);
+    LOGGER.info("unsubscribing for table {}, kafka topic {}", tableName, topicName);
+    this.unsubscribe(topicName);
+  }
+
+  @Override
+  protected KafkaConsumer<String, LogCoordinatorMessage> getConsumer() {
+    Preconditions.checkState(_consumer != null, "consumer is not initialized yet");
+    return _consumer;
+  }
+
+  @Override
+  protected GrigioMetrics getMetrics() {
+    return _metrics;
+  }
+
+  @Override
+  public Logger getLogger() {
+    return LOGGER;
+  }
+
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/rpcQueue/VersionMsgQueueProducer.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/rpcQueue/VersionMsgQueueProducer.java
new file mode 100644
index 0000000..edacd33
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/rpcQueue/VersionMsgQueueProducer.java
@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.rpcQueue;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.configuration.Configuration;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.common.serialization.IntegerSerializer;
+import org.apache.pinot.grigio.common.CoordinatorConfig;
+import org.apache.pinot.grigio.common.DistributedCommonUtils;
+import org.apache.pinot.grigio.common.FixedPartitionCountIntPartitioner;
+import org.apache.pinot.grigio.common.config.CommonConfig;
+import org.apache.pinot.grigio.common.messages.KeyCoordinatorQueueMsg;
+import org.apache.pinot.grigio.common.metrics.GrigioMetrics;
+import org.apache.pinot.grigio.common.utils.CommonUtils;
+
+import java.util.Properties;
+
+public class VersionMsgQueueProducer extends KafkaQueueProducer<Integer, KeyCoordinatorQueueMsg> {
+
+  private Configuration _conf;
+  private GrigioMetrics _grigioMetrics;
+  private String _topic;
+  private KafkaProducer<Integer, KeyCoordinatorQueueMsg> _kafkaProducer;
+
+  @Override
+  protected KafkaProducer<Integer, KeyCoordinatorQueueMsg> getKafkaNativeProducer() {
+    Preconditions.checkState(_kafkaProducer != null, "Producer has not been initialized yet");
+    return _kafkaProducer;
+  }
+
+  @Override
+  protected String getDefaultTopic() {
+    return _topic;
+  }
+
+  @Override
+  protected GrigioMetrics getMetrics() {
+    return _grigioMetrics;
+  }
+
+  @Override
+  public void init(Configuration conf, GrigioMetrics metrics) {
+    _conf = conf;
+    _topic = _conf.getString(CommonConfig.RPC_QUEUE_CONFIG.TOPIC_KEY);
+    _grigioMetrics = metrics;
+    String hostname = conf.getString(CommonConfig.RPC_QUEUE_CONFIG.HOSTNAME_KEY);
+    final Properties kafkaProducerConfig = CommonUtils.getPropertiesFromConf(
+        conf.subset(CoordinatorConfig.KAFKA_CONFIG.KAFKA_CONFIG_KEY));
+
+    kafkaProducerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, IntegerSerializer.class.getName());
+    kafkaProducerConfig.put(ProducerConfig.PARTITIONER_CLASS_CONFIG, FixedPartitionCountIntPartitioner.class.getName());
+    DistributedCommonUtils.setKakfaLosslessProducerConfig(kafkaProducerConfig, hostname);
+
+    _kafkaProducer = new KafkaProducer<>(kafkaProducerConfig);
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/storageProvider/retentionManager/KCUpdateLogRetentionManagerImpl.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/storageProvider/retentionManager/KCUpdateLogRetentionManagerImpl.java
new file mode 100644
index 0000000..8c846b3
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/storageProvider/retentionManager/KCUpdateLogRetentionManagerImpl.java
@@ -0,0 +1,82 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.storageProvider.retentionManager;
+
+import org.apache.helix.NotificationContext;
+import org.apache.helix.api.listeners.IdealStateChangeListener;
+import org.apache.helix.model.IdealState;
+import org.apache.pinot.common.Utils;
+import org.apache.pinot.grigio.common.utils.IdealStateHelper;
+import org.apache.pinot.grigio.common.storageProvider.UpdateLogStorageProvider;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class KCUpdateLogRetentionManagerImpl implements UpdateLogRetentionManager, IdealStateChangeListener {
+  private static final Logger LOGGER = LoggerFactory.getLogger(UpdateLogRetentionManagerImpl.class);
+
+  private Map<String, UpdateLogTableRetentionManager> _retentionManagerMap = new ConcurrentHashMap<>();
+  private Map<String, IdealState> _idealStateCache = new ConcurrentHashMap<>();
+  private IdealStateHelper _idealStateHelper;
+  private String _instanceName;
+  private UpdateLogStorageProvider _provider;
+
+  public KCUpdateLogRetentionManagerImpl(IdealStateHelper idealStateHelper, UpdateLogStorageProvider updateLogStorageProvider, String instanceName) {
+    _idealStateHelper = idealStateHelper;
+    _instanceName = instanceName;
+    _provider = updateLogStorageProvider;
+  }
+
+  @Override
+  public void onIdealStateChange(List<IdealState> idealState, NotificationContext changeContext)
+      throws InterruptedException {
+    Map<String, IdealState> tablesInChange = new HashMap<>();
+    idealState.forEach(is -> tablesInChange.put(is.getResourceName(), is));
+    for (Map.Entry<String, IdealState> entry: tablesInChange.entrySet()) {
+      String tableNameWithType = entry.getKey();
+      IdealState newIdealState = entry.getValue();
+      IdealState oldIdealState = _idealStateCache.get(tableNameWithType);
+      if (_retentionManagerMap.containsKey(tableNameWithType) && !newIdealState.equals(oldIdealState)) {
+        LOGGER.info("updating table {} due to ideal state change notification", tableNameWithType);
+        _retentionManagerMap.get(tableNameWithType).notifySegmentsChange();
+      }
+      _idealStateCache.put(tableNameWithType, newIdealState);
+    }
+  }
+
+  @Override
+  public UpdateLogTableRetentionManager getRetentionManagerForTable(String tableNameWithType) {
+    return _retentionManagerMap.computeIfAbsent(tableNameWithType,
+        t -> {
+          try {
+            return new KCUpdateLogTableRetentionManagerImpl(_idealStateHelper, t, _provider);
+          } catch (IOException e) {
+            LOGGER.error("failed to get retention manager for table {}", tableNameWithType, e);
+            Utils.rethrowException(e);
+          }
+          // won't reach here
+          return null;
+        });
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/storageProvider/retentionManager/KCUpdateLogTableRetentionManagerImpl.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/storageProvider/retentionManager/KCUpdateLogTableRetentionManagerImpl.java
new file mode 100644
index 0000000..8e89fb9
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/common/storageProvider/retentionManager/KCUpdateLogTableRetentionManagerImpl.java
@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.storageProvider.retentionManager;
+
+import com.google.common.collect.Sets;
+import org.apache.pinot.grigio.common.storageProvider.UpdateLogStorageProvider;
+import org.apache.pinot.grigio.common.utils.IdealStateHelper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Set;
+
+public class KCUpdateLogTableRetentionManagerImpl extends UpdateLogTableRetentionManager {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(KCUpdateLogTableRetentionManagerImpl.class);
+
+  private Set<String> _segments;
+  private final UpdateLogStorageProvider _provider;
+
+  public KCUpdateLogTableRetentionManagerImpl(IdealStateHelper idealStateHelper, String tableName, UpdateLogStorageProvider provider) throws IOException {
+    super(idealStateHelper, tableName);
+    _provider = provider;
+    // load the current update log on this server and match it with helix stored state,
+    // so we can remove any unused update logs
+    _provider.loadTable(tableName);
+    _segments = _provider.getAllSegments(tableName);
+    updateStateFromHelix();
+  }
+
+  public synchronized void updateSegmentsAndRemoveOldFiles(Set<String> newSegmentSet) {
+    Set<String> segmentDeleted = Sets.difference(_segments, newSegmentSet);
+    _segments = newSegmentSet;
+    if (segmentDeleted.size() > 0) {
+      LOGGER.info("deleting table {} segments {} from KC", _tableName, Arrays.toString(segmentDeleted.toArray()));
+      segmentDeleted.forEach(segmentName -> {
+        try {
+          _provider.removeSegment(_tableName, segmentName);
+        } catch (IOException e) {
+          LOGGER.error("failed to remove segment for table {} segment {}", _tableName, segmentName);
+        }
+      });
+    }
+  }
+
+  @Override
+  protected boolean isSegmentAssignedToCurrentServer(String segmentName) {
+    // always return true as key coordinator should store whatever segment sent to it
+    return true;
+  }
+
+  @Override
+  protected void updateStateFromHelix() {
+    long start = System.currentTimeMillis();
+    super.updateStateFromHelix();
+    updateSegmentsAndRemoveOldFiles(_segmentsToInstanceMap.keySet());
+    LOGGER.info("updated table {} state from helix in {} ms", _tableName, System.currentTimeMillis() - start);
+  }
+
+  @Override
+  public void notifySegmentDeletion(String segmentName) {
+    // do nothing, kc don't auto delete physical data
+  }
+
+  @Override
+  public void notifySegmentsChange() {
+    updateStateFromHelix();
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/GrigioKeyCoordinatorMetrics.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/GrigioKeyCoordinatorMetrics.java
new file mode 100644
index 0000000..b1f651f
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/GrigioKeyCoordinatorMetrics.java
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.keyCoordinator;
+
+import com.yammer.metrics.core.MetricsRegistry;
+import org.apache.pinot.grigio.common.metrics.GrigioGauge;
+import org.apache.pinot.grigio.common.metrics.GrigioMeter;
+import org.apache.pinot.grigio.common.metrics.GrigioMetrics;
+
+public class GrigioKeyCoordinatorMetrics extends GrigioMetrics {
+
+  private GrigioMeter[] meters = filterMeterByTypes(MetricsType.BOTH, MetricsType.KC_ONLY);
+  private GrigioGauge[] gauges = filterGaugeByTypes(MetricsType.BOTH, MetricsType.KC_ONLY);
+
+  public GrigioKeyCoordinatorMetrics(String prefix, MetricsRegistry metricsRegistry) {
+    super(prefix, metricsRegistry);
+  }
+
+  @Override
+  protected GrigioMeter[] getMeters() {
+    return meters;
+  }
+
+  @Override
+  protected GrigioGauge[] getGauges() {
+    return gauges;
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/api/KeyCoordinatorApiApplication.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/api/KeyCoordinatorApiApplication.java
new file mode 100644
index 0000000..0461405
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/api/KeyCoordinatorApiApplication.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.keyCoordinator.api;
+
+import com.google.common.base.Preconditions;
+import io.swagger.jaxrs.config.BeanConfig;
+import org.apache.pinot.grigio.keyCoordinator.helix.KeyCoordinatorClusterHelixManager;
+import org.apache.pinot.grigio.keyCoordinator.starter.KeyCoordinatorStarter;
+import org.glassfish.grizzly.http.server.CLStaticHttpHandler;
+import org.glassfish.grizzly.http.server.HttpHandler;
+import org.glassfish.grizzly.http.server.HttpServer;
+import org.glassfish.jersey.grizzly2.httpserver.GrizzlyHttpServerFactory;
+import org.glassfish.jersey.internal.inject.AbstractBinder;
+import org.glassfish.jersey.jackson.JacksonFeature;
+import org.glassfish.jersey.server.ResourceConfig;
+
+import java.net.URI;
+import java.net.URL;
+import java.net.URLClassLoader;
+
+public class KeyCoordinatorApiApplication extends ResourceConfig {
+  private static final String RESOURCE_PACKAGE = "org.apache.pinot.grigio.keyCoordinator.api.resources";
+
+  private URI _baseUri;
+  private HttpServer _httpServer;
+
+  public KeyCoordinatorApiApplication(KeyCoordinatorStarter keyCoordinatorStarter) {
+    packages(RESOURCE_PACKAGE);
+    register(JacksonFeature.class);
+    register(new AbstractBinder() {
+      @Override
+      protected void configure() {
+        bind(keyCoordinatorStarter).to(KeyCoordinatorStarter.class);
+        bind(keyCoordinatorStarter.getKeyCoordinatorClusterHelixManager()).to(KeyCoordinatorClusterHelixManager.class);
+      }
+    });
+    registerClasses(io.swagger.jaxrs.listing.ApiListingResource.class);
+    registerClasses(io.swagger.jaxrs.listing.SwaggerSerializers.class);
+  }
+
+  public void start(int httpPort) {
+    Preconditions.checkArgument(httpPort > 0);
+    _baseUri = URI.create("http://0.0.0.0:" + httpPort + "/");
+    _httpServer = GrizzlyHttpServerFactory.createHttpServer(_baseUri, this);
+    setupSwagger();
+  }
+
+  private void setupSwagger() {
+    BeanConfig beanConfig = new BeanConfig();
+    beanConfig.setTitle("Pinot key coordinator API");
+    beanConfig.setDescription("APIs for accessing Pinot key coordinator information");
+    beanConfig.setContact("https://github.com/linkedin/pinot");
+    beanConfig.setVersion("1.0");
+    beanConfig.setSchemes(new String[]{"http"});
+    beanConfig.setBasePath(_baseUri.getPath());
+    beanConfig.setResourcePackage(RESOURCE_PACKAGE);
+    beanConfig.setScan(true);
+
+    HttpHandler httpHandler = new CLStaticHttpHandler(KeyCoordinatorApiApplication.class.getClassLoader(), "/api/");
+    // map both /api and /help to swagger docs. /api because it looks nice. /help for backward compatibility
+    _httpServer.getServerConfiguration().addHttpHandler(httpHandler, "/api", "/help");
+
+    URL swaggerDistLocation =
+        KeyCoordinatorApiApplication.class.getClassLoader().getResource("META-INF/resources/webjars/swagger-ui/2.2.2/");
+    CLStaticHttpHandler swaggerDist = new CLStaticHttpHandler(new URLClassLoader(new URL[]{swaggerDistLocation}));
+    _httpServer.getServerConfiguration().addHttpHandler(swaggerDist, "/swaggerui-dist/");
+  }
+
+  public void stop() {
+    if (_httpServer != null) {
+      _httpServer.shutdownNow();
+    }
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/api/KeyCoordinatorInstance.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/api/KeyCoordinatorInstance.java
new file mode 100644
index 0000000..fa21f07
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/api/KeyCoordinatorInstance.java
@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.keyCoordinator.api;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.pinot.common.utils.CommonConstants;
+
+
+public class KeyCoordinatorInstance {
+  private final String _host;
+  private final String _port;
+
+  @JsonCreator
+  public KeyCoordinatorInstance(
+      @JsonProperty(value = "host", required = true) String host,
+      @JsonProperty(value = "port", required = true) String port
+  ) {
+    _host = host;
+    _port = port;
+  }
+
+  public String getHost() {
+    return _host;
+  }
+
+  public String getPort() {
+    return _port;
+  }
+
+  public String toInstanceId() {
+    return CommonConstants.Helix.PREFIX_OF_KEY_COORDINATOR_INSTANCE + _host + "_" + _port;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder bld = new StringBuilder();
+    bld.append("host : " + _host + "\n");
+    bld.append("port : " + _port + "\n");
+    return bld.toString();
+  }
+
+  public InstanceConfig toInstanceConfig() {
+    final InstanceConfig iConfig = new InstanceConfig(toInstanceId());
+    iConfig.setHostName(_host);
+    iConfig.setPort(_port);
+    iConfig.setInstanceEnabled(true);
+    return iConfig;
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/api/resources/KeyCoordinatorHealthCheck.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/api/resources/KeyCoordinatorHealthCheck.java
new file mode 100644
index 0000000..0cd7559
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/api/resources/KeyCoordinatorHealthCheck.java
@@ -0,0 +1,59 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.keyCoordinator.api.resources;
+
+import org.apache.pinot.grigio.keyCoordinator.starter.KeyCoordinatorStarter;
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiOperation;
+import io.swagger.annotations.ApiResponse;
+import io.swagger.annotations.ApiResponses;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+
+@Api(tags = "Health")
+@Path("/")
+public class KeyCoordinatorHealthCheck {
+
+  @Inject
+  private KeyCoordinatorStarter keyCoordinatorStarter;
+
+  @GET
+  @Produces(MediaType.TEXT_PLAIN)
+  @Path("health")
+  @ApiOperation(value = "Checking key coordinator health")
+  @ApiResponses(value = {
+      @ApiResponse(code = 200, message = "KC is healthy"),
+      @ApiResponse(code = 503, message = "KC is disabled")
+  })
+  public String getKCHealth() {
+
+    if (keyCoordinatorStarter != null && keyCoordinatorStarter.isRunning()) {
+      return "OK";
+    } else {
+      throw new WebApplicationException("Pinot key coordinator is disabled", Response.Status.SERVICE_UNAVAILABLE);
+    }
+  }
+
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/api/resources/KeyCoordinatorInstanceResource.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/api/resources/KeyCoordinatorInstanceResource.java
new file mode 100644
index 0000000..d648ceb
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/api/resources/KeyCoordinatorInstanceResource.java
@@ -0,0 +1,100 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.keyCoordinator.api.resources;
+
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiOperation;
+import io.swagger.annotations.ApiResponse;
+import io.swagger.annotations.ApiResponses;
+import org.apache.pinot.grigio.keyCoordinator.api.KeyCoordinatorInstance;
+import org.apache.pinot.grigio.keyCoordinator.helix.KeyCoordinatorClusterHelixManager;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.inject.Inject;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.MediaType;
+import java.util.List;
+
+
+@Api(tags = "Instance")
+@Path("/")
+public class KeyCoordinatorInstanceResource {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(KeyCoordinatorInstanceResource.class);
+
+  @Inject
+  private KeyCoordinatorClusterHelixManager _keyCoordinatorClusterHelixManager;
+
+  @GET
+  @Path("/instances")
+  @Produces(MediaType.APPLICATION_JSON)
+  @ApiOperation(value = "List all instances", produces = MediaType.APPLICATION_JSON)
+  @ApiResponses(value = {
+      @ApiResponse(code = 200, message = "Success"),
+      @ApiResponse(code = 500, message = "Internal error")
+  })
+  public String getAllInstances() {
+    LOGGER.info("Trying to get all key coordinator instances");
+    List<String> instances = _keyCoordinatorClusterHelixManager.getAllInstances();
+    return JsonUtils.objectToJsonNode(instances).toString();
+  }
+
+  @POST
+  @Path("/instances")
+  @Consumes(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.TEXT_PLAIN)
+  @ApiOperation(value = "Create a new instance", consumes = MediaType.APPLICATION_JSON, produces = MediaType.TEXT_PLAIN,
+      notes = "Create a new instance with given instance config")
+  @ApiResponses(value = {
+      @ApiResponse(code = 200, message = "Success"),
+      @ApiResponse(code = 409, message = "Instance already exists"),
+      @ApiResponse(code = 500, message = "Internal error")
+  })
+  public String addInstance(KeyCoordinatorInstance keyCoordinatorInstance) {
+    LOGGER.info("Trying to add a new key coordinator instance");
+    _keyCoordinatorClusterHelixManager.addInstance(keyCoordinatorInstance);
+    return "Successfully created instance";
+  }
+
+  @DELETE
+  @Path("/instances")
+  @Consumes(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.TEXT_PLAIN)
+  @ApiOperation(value = "Drop an instance", consumes = MediaType.APPLICATION_JSON, produces = MediaType.TEXT_PLAIN,
+      notes = "Drop an instance")
+  @ApiResponses(value = {
+      @ApiResponse(code = 200, message = "Success"),
+      @ApiResponse(code = 404, message = "Instance not found"),
+      @ApiResponse(code = 409, message = "Instance cannot be dropped"),
+      @ApiResponse(code = 500, message = "Internal error")
+  })
+  public String dropInstance(KeyCoordinatorInstance keyCoordinatorInstance) {
+    LOGGER.info("Trying to drop a key coordinator instance");
+    _keyCoordinatorClusterHelixManager.dropInstance(keyCoordinatorInstance);
+    return "Successfully dropped instance";
+  }
+}
+
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/api/resources/KeyCoordinatorStatus.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/api/resources/KeyCoordinatorStatus.java
new file mode 100644
index 0000000..e436bde
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/api/resources/KeyCoordinatorStatus.java
@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.keyCoordinator.api.resources;
+
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiOperation;
+import io.swagger.annotations.ApiResponse;
+import io.swagger.annotations.ApiResponses;
+import org.apache.pinot.grigio.keyCoordinator.starter.KeyCoordinatorStarter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.util.stream.Collectors;
+
+@Api(tags = "Status")
+@Path("/")
+public class KeyCoordinatorStatus {
+  private static final Logger LOGGER = LoggerFactory.getLogger(KeyCoordinatorStatus.class);
+
+  @Inject
+  private KeyCoordinatorStarter keyCoordinatorStarter;
+
+  @GET
+  @Produces(MediaType.TEXT_PLAIN)
+  @Path("status")
+  @ApiOperation(value = "Checking log coordinator status")
+  @ApiResponses(value = {
+      @ApiResponse(code = 200, message = "success"),
+  })
+  public String getKCStatus() {
+    try {
+      if (keyCoordinatorStarter != null && keyCoordinatorStarter.isRunning()) {
+        String result = keyCoordinatorStarter.getConsumer().getListOfSubscribedTopicPartitions()
+            .stream()
+            .map(tp -> String.format("%s:%d", tp.topic(), tp.partition()))
+            .collect(Collectors.joining(","));
+        return "list of subscribed topic partitions: " + result;
+      } else {
+        throw new WebApplicationException("Pinot key coordinator is disabled", Response.Status.SERVICE_UNAVAILABLE);
+      }
+    } catch (Exception ex) {
+      LOGGER.error("failed to get kc status", ex);
+      throw new WebApplicationException(ex);
+    }
+  }
+
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/api/resources/KeyCoordinatorStatusToggleResource.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/api/resources/KeyCoordinatorStatusToggleResource.java
new file mode 100644
index 0000000..583022f
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/api/resources/KeyCoordinatorStatusToggleResource.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.keyCoordinator.api.resources;
+
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiOperation;
+import io.swagger.annotations.ApiResponse;
+import io.swagger.annotations.ApiResponses;
+import javax.inject.Inject;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.MediaType;
+import org.apache.pinot.grigio.keyCoordinator.helix.KeyCoordinatorClusterHelixManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * APIs to start/stop consuming key coordinator messages after initialization of key coordinator cluster
+ */
+@Api(tags = "Status")
+@Path("/")
+public class KeyCoordinatorStatusToggleResource {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(KeyCoordinatorStatusToggleResource.class);
+
+  @Inject
+  private KeyCoordinatorClusterHelixManager _keyCoordinatorClusterHelixManager;
+
+  @POST
+  @Path("/start")
+  @Produces(MediaType.TEXT_PLAIN)
+  @ApiOperation(value = "Start consuming key coordinator messages", produces = MediaType.TEXT_PLAIN,
+      notes = "Start consuming key coordinator messages")
+  @ApiResponses(value = {
+      @ApiResponse(code = 200, message = "Success"),
+      @ApiResponse(code = 500, message = "Internal error")
+  })
+  public String startConsuming() {
+    LOGGER.info("Trying to start consuming key coordinator messages");
+    _keyCoordinatorClusterHelixManager.rebalance();
+    return "Successfully started consuming key coordinator messages";
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/helix/ActiveTableTracker.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/helix/ActiveTableTracker.java
new file mode 100644
index 0000000..4b49e51
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/helix/ActiveTableTracker.java
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.keyCoordinator.helix;
+
+import com.google.common.collect.Sets;
+import org.apache.helix.NotificationContext;
+import org.apache.helix.api.listeners.ExternalViewChangeListener;
+import org.apache.helix.model.ExternalView;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.pinot.common.utils.CommonConstants.Helix.BROKER_RESOURCE_INSTANCE;
+import static org.apache.pinot.common.utils.CommonConstants.Helix.TableType.REALTIME;
+
+/**
+ * Tracks all the changes happening in the Pinot cluster for all external views
+ * Keeps a track of existing tables within the KeyCoordinator and uses this
+ * to notify which tables are being newly added and which tables are being
+ * deleted.
+ */
+public class ActiveTableTracker implements ExternalViewChangeListener {
+    private static final Logger LOGGER = LoggerFactory.getLogger(ActiveTableTracker.class);
+
+    private final Set<String> _activeTables = new HashSet<>();
+    private final TableListener _listener;
+
+    /**
+     * Initialize with any existing tables. Any table added / deleted is then
+     * notified after comparing to these tables.
+     *
+     * @param existingTables specifies tables that already exist in the Key Coordinator
+     * @param listener register a listener for the table add/delete callbacks
+     */
+    public ActiveTableTracker(Set<String> existingTables, TableListener listener) {
+        if (existingTables != null) {
+            _activeTables.addAll(existingTables);
+        }
+        _listener = listener;
+    }
+
+    public Set<String> getCurrentActiveTables() {
+        return Collections.unmodifiableSet(_activeTables);
+    }
+
+    @Override
+    public void onExternalViewChange(List<ExternalView> externalViewList, NotificationContext notificationContext) {
+        Set<String> externalViewTables = new HashSet<>();
+
+        // First, create a set of all table names seen in the external view
+        for (ExternalView view : externalViewList) {
+
+            // For table creation / deletion its sufficient to listen to external
+            // view for 'brokerResource'. Other views contain segment information
+            // which is not needed at this point.
+            if (BROKER_RESOURCE_INSTANCE.equalsIgnoreCase(view.getResourceName())) {
+                for (String tableName : view.getPartitionSet()) {
+                    // We're only interested in tracking realtime tables
+                    if (!tableName.endsWith(REALTIME.name())) {
+                        continue;
+                    }
+                    LOGGER.debug("Found table in external view: {}", tableName);
+                    externalViewTables.add(tableName);
+                }
+                break;
+            }
+        }
+
+        Set<String> newTablesAdded = new HashSet<>();
+        Set<String> tablesDeleted = new HashSet<>();
+        // Now compute the sets of tableNames being newly added and deleted
+        newTablesAdded.addAll(Sets.difference(externalViewTables, _activeTables));
+        tablesDeleted.addAll(Sets.difference(_activeTables, externalViewTables));
+
+        // Finally, track these changes in the internal set and send
+        // notifications as needed
+        newTablesAdded.forEach(tableName -> {
+            LOGGER.info("Adding new table: {}", tableName);
+            _activeTables.add(tableName);
+            _listener.onNewTableCreated(tableName);
+        });
+
+        tablesDeleted.forEach(tableName -> {
+            LOGGER.info("Deleting table: {}", tableName);
+            _activeTables.remove(tableName);
+            _listener.onTableDeleted(tableName);
+        });
+    }
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/helix/HelixSetupUtils.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/helix/HelixSetupUtils.java
new file mode 100644
index 0000000..845f7c1
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/helix/HelixSetupUtils.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.keyCoordinator.helix;
+
+import org.apache.helix.HelixAdmin;
+import org.apache.helix.HelixManager;
+import org.apache.helix.controller.HelixControllerMain;
+import org.apache.helix.manager.zk.ZKHelixAdmin;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class HelixSetupUtils {
+  private static final Logger LOGGER = LoggerFactory.getLogger(HelixSetupUtils.class);
+
+  public static synchronized HelixManager setup(String helixClusterName, String zkPath, String instanceId) {
+    HelixManager helixManager;
+    try {
+      createHelixClusterIfNeeded(helixClusterName, zkPath);
+    } catch (final Exception ex) {
+      LOGGER.error("failed to set up helix for key coordinator", ex);
+      return null;
+    }
+
+    try {
+      helixManager = startHelixController(helixClusterName, zkPath, instanceId);
+    } catch (final Exception ex) {
+      LOGGER.error("failed to start up helix controller for key coordinator", ex);
+      return null;
+    }
+
+    return helixManager;
+  }
+
+  private static void createHelixClusterIfNeeded(String helixClusterName, String zkPath) {
+    final HelixAdmin admin = new ZKHelixAdmin(zkPath);
+    if (admin.getClusters().contains(helixClusterName)) {
+      LOGGER.info("cluster {} already exists", helixClusterName);
+      return;
+    }
+    admin.addCluster(helixClusterName);
+  }
+
+  private static HelixManager startHelixController(String helixClusterName, String zkUrl,
+      String instanceId) {
+    LOGGER.info("Starting Helix Standalone Controller ... ");
+    return HelixControllerMain.startHelixController(zkUrl, helixClusterName, instanceId,
+        HelixControllerMain.STANDALONE);
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/helix/KeyCoordinatorClusterHelixManager.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/helix/KeyCoordinatorClusterHelixManager.java
new file mode 100644
index 0000000..e5b1573
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/helix/KeyCoordinatorClusterHelixManager.java
@@ -0,0 +1,104 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.keyCoordinator.helix;
+
+import org.apache.helix.HelixAdmin;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.HelixManagerFactory;
+import org.apache.helix.InstanceType;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.OnlineOfflineSMD;
+import org.apache.pinot.common.utils.CommonConstants;
+import org.apache.pinot.grigio.common.rpcQueue.KeyCoordinatorQueueConsumer;
+import org.apache.pinot.grigio.keyCoordinator.api.KeyCoordinatorInstance;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import java.util.List;
+
+
+/**
+ * This manages the key coordinator cluster (key coordinators as controller-participant)
+ */
+public class KeyCoordinatorClusterHelixManager {
+  private static final Logger LOGGER = LoggerFactory.getLogger(KeyCoordinatorClusterHelixManager.class);
+
+  private final String _helixZkURL;
+  private final String _keyCoordinatorClusterName;
+  private final String _keyCoordinatorId;
+  private final HelixManager _controllerHelixManager;
+  private final HelixManager _participantHelixManager;
+  private final HelixAdmin _helixAdmin;
+
+  public KeyCoordinatorClusterHelixManager(@Nonnull String zkURL, @Nonnull String keyCoordinatorClusterName,
+                                           @Nonnull String keyCoordinatorId, @Nonnull KeyCoordinatorQueueConsumer keyCoordinatorQueueConsumer,
+                                           @Nonnull String keyCoordinatorMessageTopic, int keyCoordinatorMessagePartitionCount)
+      throws Exception {
+    _helixZkURL = zkURL;
+    _keyCoordinatorClusterName = keyCoordinatorClusterName;
+    _keyCoordinatorId = keyCoordinatorId;
+
+    _controllerHelixManager = HelixSetupUtils.setup(_keyCoordinatorClusterName, _helixZkURL, _keyCoordinatorId);
+    _helixAdmin = _controllerHelixManager.getClusterManagmentTool();
+
+    IdealState keyCoordinatorMessageResourceIdealState = _helixAdmin
+        .getResourceIdealState(_keyCoordinatorClusterName, CommonConstants.Helix.KEY_COORDINATOR_MESSAGE_RESOURCE_NAME);
+    if (keyCoordinatorMessageResourceIdealState == null) {
+      _helixAdmin.addResource(_keyCoordinatorClusterName, CommonConstants.Helix.KEY_COORDINATOR_MESSAGE_RESOURCE_NAME,
+          keyCoordinatorMessagePartitionCount, OnlineOfflineSMD.name, IdealState.RebalanceMode.CUSTOMIZED.name());
+    }
+
+    try {
+      _helixAdmin.addInstance(_keyCoordinatorClusterName, new InstanceConfig(_keyCoordinatorId));
+    } catch (final HelixException ex) {
+      LOGGER.info("key coordinator instance {} already exist in helix cluster {}", _keyCoordinatorId,
+          _keyCoordinatorClusterName);
+    }
+
+    _participantHelixManager = HelixManagerFactory
+        .getZKHelixManager(_keyCoordinatorClusterName, _keyCoordinatorId, InstanceType.PARTICIPANT, _helixZkURL);
+    _participantHelixManager.getStateMachineEngine().registerStateModelFactory(OnlineOfflineSMD.name,
+        new KeyCoordinatorMessageStateModelFactory(keyCoordinatorQueueConsumer, keyCoordinatorMessageTopic));
+    _participantHelixManager.connect();
+  }
+
+  public HelixManager getControllerHelixManager() {
+    return _controllerHelixManager;
+  }
+
+  public List<String> getAllInstances() {
+    return _helixAdmin.getInstancesInCluster(_keyCoordinatorClusterName);
+  }
+
+  public void addInstance(KeyCoordinatorInstance keyCoordinatorInstance) {
+    _helixAdmin.addInstance(_keyCoordinatorClusterName, keyCoordinatorInstance.toInstanceConfig());
+  }
+
+  public void dropInstance(KeyCoordinatorInstance keyCoordinatorInstance) {
+    _helixAdmin.dropInstance(_keyCoordinatorClusterName, keyCoordinatorInstance.toInstanceConfig());
+  }
+
+  public void rebalance() {
+    _helixAdmin.rebalance(_keyCoordinatorClusterName, CommonConstants.Helix.KEY_COORDINATOR_MESSAGE_RESOURCE_NAME,
+        CommonConstants.Helix.KEY_COORDINATOR_MESSAGE_RESOURCE_REPLICA_COUNT);
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/helix/KeyCoordinatorLeadershipManager.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/helix/KeyCoordinatorLeadershipManager.java
new file mode 100644
index 0000000..65adfdc
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/helix/KeyCoordinatorLeadershipManager.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.keyCoordinator.helix;
+
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.listeners.ControllerChangeListener;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Manager for leadership status of key coordinator controllers. This listens to leadership changes
+ * and updates local cache for the leadership status.
+ */
+public class KeyCoordinatorLeadershipManager {
+  private static final Logger LOGGER = LoggerFactory.getLogger(KeyCoordinatorLeadershipManager.class);
+
+  private final HelixManager _controllerHelixManager;
+
+  private volatile boolean _isLeader = false;
+
+  public KeyCoordinatorLeadershipManager(HelixManager controllerHelixManager) {
+    _controllerHelixManager = controllerHelixManager;
+    _controllerHelixManager
+        .addControllerListener((ControllerChangeListener) notificationContext -> onControllerChange());
+  }
+
+  public synchronized boolean isLeader() {
+    return _isLeader;
+  }
+
+  private synchronized void onControllerChange() {
+    boolean newIsLeader = _controllerHelixManager.isLeader();
+    LOGGER.info("Key coordinator controller isLeader status changed from {} to {}", _isLeader, newIsLeader);
+    _isLeader = newIsLeader;
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/helix/KeyCoordinatorMessageStateModelFactory.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/helix/KeyCoordinatorMessageStateModelFactory.java
new file mode 100644
index 0000000..e1215eb
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/helix/KeyCoordinatorMessageStateModelFactory.java
@@ -0,0 +1,99 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.keyCoordinator.helix;
+
+import com.google.common.base.Preconditions;
+import org.apache.helix.NotificationContext;
+import org.apache.helix.model.Message;
+import org.apache.helix.participant.statemachine.StateModel;
+import org.apache.helix.participant.statemachine.StateModelFactory;
+import org.apache.helix.participant.statemachine.StateModelInfo;
+import org.apache.helix.participant.statemachine.Transition;
+import org.apache.pinot.grigio.common.rpcQueue.KafkaQueueConsumer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * State model for key coordinator to handle:
+ * 1. start of the key coordinator cluster (initial assignment of key coordinator message segments)
+ * 2. todo: fail over of a key coordinator instance
+ */
+
+public class KeyCoordinatorMessageStateModelFactory extends StateModelFactory<StateModel> {
+  private static final Logger LOGGER = LoggerFactory.getLogger(KeyCoordinatorMessageStateModelFactory.class);
+
+  private final KafkaQueueConsumer _keyCoordinatorQueueConsumer;
+  private final String _keyCoordinatorMessageTopic;
+
+  private static final String HELIX_PARTITION_SEPARATOR = "_";
+
+  public KeyCoordinatorMessageStateModelFactory(KafkaQueueConsumer keyCoordinatorQueueConsumer,
+      String keyCoordinatorMessageTopic) {
+    _keyCoordinatorQueueConsumer = keyCoordinatorQueueConsumer;
+    _keyCoordinatorMessageTopic = keyCoordinatorMessageTopic;
+  }
+
+  @Override
+  public StateModel createNewStateModel(String resourceName, String partitionName) {
+    LOGGER.info("creating new state model with resource {} and partition {}", resourceName, partitionName);
+    return new KeyCoordinatorMessageStateModel(partitionName);
+  }
+
+  @StateModelInfo(states = "{'OFFLINE', 'ONLINE'}", initialState = "OFFLINE")
+  public class KeyCoordinatorMessageStateModel extends StateModel {
+
+    private final String _partitionName;
+
+    public KeyCoordinatorMessageStateModel(String partitionName) {
+      LOGGER.info("Creating a Key coordinator message state model with partition: {}", partitionName);
+      _partitionName = partitionName;
+    }
+
+    @Transition(from = "OFFLINE", to = "ONLINE")
+    public void onBecomeOnlineFromOffline(Message message, NotificationContext context) {
+      LOGGER.info("Key coordinator message onBecomeOnlineFromOffline with partition: {}", _partitionName);
+      _keyCoordinatorQueueConsumer.subscribe(_keyCoordinatorMessageTopic,
+          getKafkaPartitionNumberFromHelixPartition(_partitionName));
+    }
+
+    @Transition(from = "ONLINE", to = "OFFLINE")
+    public void onBecomeOfflineFromOnline(Message message, NotificationContext context) {
+      LOGGER.info("Key coordinator message onBecomeOfflineFromOnline with partition: {}", _partitionName);
+      _keyCoordinatorQueueConsumer.unsubscribe(_keyCoordinatorMessageTopic,
+          getKafkaPartitionNumberFromHelixPartition(_partitionName));
+    }
+  }
+
+  /** helix partitions name as something like keyCoordinatorMessageResource_3
+   * parse this string to get the correct numeric value for partition
+   * @return the numeric value of this partition
+   */
+  protected int getKafkaPartitionNumberFromHelixPartition(String helixPartition) {
+    String[] partitionNameComponents = helixPartition.split(HELIX_PARTITION_SEPARATOR);
+    Preconditions.checkState(partitionNameComponents.length > 1,
+        "partition name should have more than 1 parts: " + helixPartition);
+    try {
+      return Integer.parseInt(partitionNameComponents[partitionNameComponents.length - 1]);
+    } catch (NumberFormatException ex) {
+      LOGGER.error("failed to parse numeric partition value from helix message {}", helixPartition);
+      throw new RuntimeException(ex);
+    }
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/helix/KeyCoordinatorPinotHelixSpectator.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/helix/KeyCoordinatorPinotHelixSpectator.java
new file mode 100644
index 0000000..5157835
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/helix/KeyCoordinatorPinotHelixSpectator.java
@@ -0,0 +1,59 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.keyCoordinator.helix;
+
+import org.apache.helix.HelixManager;
+import org.apache.helix.HelixManagerFactory;
+import org.apache.helix.InstanceType;
+import org.apache.helix.api.listeners.IdealStateChangeListener;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+public class KeyCoordinatorPinotHelixSpectator {
+  private static final Logger LOGGER = LoggerFactory.getLogger(KeyCoordinatorPinotHelixSpectator.class);
+
+  private final String _pinotHelixZkURL;
+  private final String _pinotHelixClusterName;
+  private final String _keyCoordinatorId;
+  private final HelixManager _spectatorHelixManager;
+
+  private HelixManager _helixZkManager;
+
+  public KeyCoordinatorPinotHelixSpectator(@Nonnull String zkURL, @Nonnull String helixClusterName,
+                                           @Nonnull String keyCoordinatorId) throws Exception {
+    _pinotHelixZkURL = zkURL;
+    _pinotHelixClusterName = helixClusterName;
+    _keyCoordinatorId = keyCoordinatorId;
+
+    _spectatorHelixManager =
+        HelixManagerFactory.getZKHelixManager(_pinotHelixClusterName, _keyCoordinatorId, InstanceType.SPECTATOR, _pinotHelixZkURL);
+    _spectatorHelixManager.connect();
+  }
+
+  public HelixManager getHelixManager() {
+    return _spectatorHelixManager;
+  }
+
+  public void addListener(IdealStateChangeListener listener) throws Exception {
+    _spectatorHelixManager.addIdealStateChangeListener(listener);
+  }
+
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/helix/KeyCoordinatorVersionManager.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/helix/KeyCoordinatorVersionManager.java
new file mode 100644
index 0000000..6d59d99
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/helix/KeyCoordinatorVersionManager.java
@@ -0,0 +1,120 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.keyCoordinator.helix;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.helix.AccessOption;
+import org.apache.helix.HelixManager;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.store.zk.ZkHelixPropertyStore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Manager for version numbers. This handles getting/setting version produced/consumed from/to zookeeper
+ * property store. Utilizes cache built in ZkHelixPropertyStore to reduce load on zookeeper.
+ */
+public class KeyCoordinatorVersionManager {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(KeyCoordinatorVersionManager.class);
+
+  private static final String VERSION_PRODUCED_ZN_PATH = "/VERSION_PRODUCED";
+  private static final String VERSION_PRODUCED_KEY = "VERSION_PRODUCED";
+  private static final String VERSION_CONSUMED_ZN_PATH = "/VERSION_CONSUMED";
+
+  private final ZkHelixPropertyStore<ZNRecord> _propertyStore;
+  private final String _instanceName;
+
+  public KeyCoordinatorVersionManager(HelixManager helixManager) {
+    _propertyStore = helixManager.getHelixPropertyStore();
+    _instanceName = helixManager.getInstanceName();
+  }
+
+  /**
+   * Get version produced for the key coordinator cluster.
+   *
+   * There is only one version produced for the whole key coordinator cluster.
+   */
+  public long getVersionProducedFromPropertyStore() {
+    ZNRecord record = _propertyStore.get(VERSION_PRODUCED_ZN_PATH, null, AccessOption.PERSISTENT);
+    if (record == null) {
+      // new cluster
+      return 0L;
+    }
+    return Long.parseLong(record.getSimpleField(VERSION_PRODUCED_KEY));
+  }
+
+  /**
+   * Set version produced for the key coordinator cluster.
+   *
+   * There is only one version produced for the whole key coordinator cluster.
+   *
+   * @return true if the version produced is saved to the property store successfully, false otherwise.
+   */
+  public boolean setVersionProducedToPropertyStore(long versionProduced) {
+    ZNRecord record = new ZNRecord(VERSION_PRODUCED_ZN_PATH);
+    record.setLongField(VERSION_PRODUCED_KEY, versionProduced);
+    return _propertyStore.set(VERSION_PRODUCED_ZN_PATH, record, AccessOption.PERSISTENT);
+  }
+
+  /**
+   * Get version consumed for the current key coordinator instance.
+   *
+   * There is a map of version consumed for each key coordinator instance, with the partition as key and version as value.
+   */
+  public Map<Integer, Long> getVersionConsumedFromPropertyStore() {
+    Map<Integer, Long> versionConsumed = new HashMap<>();
+    ZNRecord record = _propertyStore.get(VERSION_CONSUMED_ZN_PATH, null, AccessOption.PERSISTENT);
+    if (record == null) {
+      // new cluster
+      return versionConsumed;
+    }
+    Map<String, String> versionConsumedStr = record.getMapField(_instanceName);
+    if (versionConsumedStr == null) {
+      // new instance
+      return versionConsumed;
+    }
+    for (Map.Entry<String, String> entry : versionConsumedStr.entrySet()) {
+      versionConsumed.put(Integer.parseInt(entry.getKey()), Long.parseLong(entry.getValue()));
+    }
+    return versionConsumed;
+  }
+
+  /**
+   * Set the version consumed for the current key coordinator instance.
+   *
+   * There is a map of version consumed for each key coordinator instance, with the partition as key and version as value.
+   *
+   * @return true if the version consumed is saved to the property store successfully, false otherwise.
+   */
+  public boolean setVersionConsumedToPropertyStore(Map<Integer, Long> versionConsumed) {
+    ZNRecord record = _propertyStore.get(VERSION_CONSUMED_ZN_PATH, null, AccessOption.PERSISTENT);
+    if (record == null) {
+      record = new ZNRecord(VERSION_CONSUMED_ZN_PATH);
+    }
+    Map<String, String> versionConsumedStr = new HashMap<>();
+    for (Map.Entry<Integer, Long> entry : versionConsumed.entrySet()) {
+      versionConsumedStr.put(entry.getKey().toString(), entry.getValue().toString());
+    }
+    record.setMapField(_instanceName, versionConsumedStr);
+    return _propertyStore.set(VERSION_CONSUMED_ZN_PATH, record, AccessOption.PERSISTENT);
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/helix/State.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/helix/State.java
new file mode 100644
index 0000000..f29f05b
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/helix/State.java
@@ -0,0 +1,26 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.keyCoordinator.helix;
+
+public enum State {
+  INIT,
+  RUNNING,
+  SHUTTING_DOWN,
+  SHUTDOWN
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/helix/TableListener.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/helix/TableListener.java
new file mode 100644
index 0000000..12abe35
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/helix/TableListener.java
@@ -0,0 +1,29 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.keyCoordinator.helix;
+
+/**
+ * Listener interface to listen on Pinot tables being added / deleted
+ */
+public interface TableListener {
+
+    public void onNewTableCreated(String tableName);
+
+    public void onTableDeleted(String tableName);
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/internal/DistributedKeyCoordinatorCore.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/internal/DistributedKeyCoordinatorCore.java
new file mode 100644
index 0000000..33790cd
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/internal/DistributedKeyCoordinatorCore.java
@@ -0,0 +1,148 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.keyCoordinator.internal;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.pinot.grigio.common.messages.KeyCoordinatorQueueMsg;
+import org.apache.pinot.grigio.common.metrics.GrigioMeter;
+import org.apache.pinot.grigio.common.metrics.GrigioTimer;
+import org.apache.pinot.grigio.common.rpcQueue.QueueConsumerRecord;
+import org.apache.pinot.grigio.common.utils.CommonUtils;
+import org.apache.pinot.grigio.keyCoordinator.GrigioKeyCoordinatorMetrics;
+import org.apache.pinot.grigio.keyCoordinator.helix.State;
+import org.apache.pinot.grigio.keyCoordinator.starter.KeyCoordinatorConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+
+public class DistributedKeyCoordinatorCore {
+  private static final Logger LOGGER = LoggerFactory.getLogger(DistributedKeyCoordinatorCore.class);
+  private static final long TERMINATION_WAIT_MS = 10000;
+
+  protected KeyCoordinatorConf _conf;
+  protected ExecutorService _messageProcessThread;
+  protected GrigioKeyCoordinatorMetrics _metrics;
+  protected int _fetchMsgMaxDelayMs;
+
+  // sub manager for other stuff
+  protected SegmentEventProcessor _segmentEventProcessor;
+  protected MessageFetcher _messageFetcher;
+  protected VersionMessageManager _versionMessageManager;
+
+  protected volatile State _state = State.SHUTDOWN;
+
+  public DistributedKeyCoordinatorCore() {}
+
+  public void init(KeyCoordinatorConf conf, SegmentEventProcessor segmentEventProcessor,
+                   MessageFetcher fetcher, VersionMessageManager versionMessageManager,
+                   GrigioKeyCoordinatorMetrics metrics) {
+    init(conf, Executors.newSingleThreadExecutor(), segmentEventProcessor, fetcher, versionMessageManager, metrics);
+  }
+
+  @VisibleForTesting
+  public void init(KeyCoordinatorConf conf, ExecutorService coreThread, SegmentEventProcessor segmentEventProcessor,
+                   MessageFetcher fetcher, VersionMessageManager versionMessageManager,
+                   GrigioKeyCoordinatorMetrics metrics) {
+    CommonUtils.printConfiguration(conf, "distributed key coordinator core");
+    Preconditions.checkState(_state == State.SHUTDOWN, "can only init if it is not running yet");
+    _conf = conf;
+    _messageProcessThread = coreThread;
+    _versionMessageManager = versionMessageManager;
+    _segmentEventProcessor = segmentEventProcessor;
+    _messageFetcher = fetcher;
+    _metrics = metrics;
+
+    _fetchMsgMaxDelayMs = conf.getInt(KeyCoordinatorConf.FETCH_MSG_MAX_DELAY_MS,
+        KeyCoordinatorConf.FETCH_MSG_MAX_DELAY_MS_DEFAULT);
+
+    _state = State.INIT;
+  }
+
+  public void start() {
+    Preconditions.checkState(_state == State.INIT, "key coordinate is not in correct state");
+    LOGGER.info("starting key coordinator message process loop");
+    _state = State.RUNNING;
+    _messageFetcher.start();
+    _versionMessageManager.start();
+    _segmentEventProcessor.start();
+    _messageProcessThread.submit(this::messageProcessLoop);
+  }
+
+  private void messageProcessLoop() {
+    try {
+      long deadline = System.currentTimeMillis() + _fetchMsgMaxDelayMs;
+      while (_state == State.RUNNING) {
+        LOGGER.info("starting new loop");
+        long start = System.currentTimeMillis();
+        // process message when we got max message count or reach max delay ms
+        MessageFetcher.MessageAndOffset<QueueConsumerRecord<byte[], KeyCoordinatorQueueMsg>> messageAndOffset =
+            _messageFetcher.getMessages(deadline);
+        List<QueueConsumerRecord<byte[], KeyCoordinatorQueueMsg>> messages = messageAndOffset.getMessages();
+        deadline = System.currentTimeMillis() + _fetchMsgMaxDelayMs;
+
+        _metrics.addMeteredGlobalValue(GrigioMeter.MESSAGE_PROCESS_THREAD_FETCH_COUNT, messages.size());
+        _metrics.addTimedValueMs(GrigioTimer.MESSAGE_PROCESS_THREAD_FETCH_DELAY, System.currentTimeMillis() - start);
+
+        if (messages.size() > 0) {
+          _segmentEventProcessor.processMessages(messages);
+          // todo: make ackOffset and setVersionConsumed as one transaction
+          _messageFetcher.ackOffset(messageAndOffset);
+          _versionMessageManager.setVersionConsumedToPropertyStore();
+          LOGGER.info("kc processed {} messages in this loop for {} ms", messages.size(),
+              System.currentTimeMillis() - start);
+        } else {
+          LOGGER.info("no message received in the current loop");
+        }
+      }
+    } catch (Exception ex) {
+      LOGGER.warn("key coordinator is exiting due to exception", ex);
+    } finally {
+      _state = State.SHUTTING_DOWN;
+      LOGGER.info("exiting key coordinator loop");
+    }
+    LOGGER.info("existing key coordinator core /procthread");
+  }
+
+  public void stop() {
+    _state = State.SHUTTING_DOWN;
+    _messageFetcher.stop();
+    _versionMessageManager.stop();
+    _segmentEventProcessor.stop();
+    _messageProcessThread.shutdown();
+    try {
+      _messageProcessThread.awaitTermination(TERMINATION_WAIT_MS, TimeUnit.MILLISECONDS);
+    } catch (InterruptedException ex) {
+      LOGGER.error("failed to wait for key coordinator thread to shutdown", ex);
+    }
+    _messageProcessThread.shutdownNow();
+    _state = State.SHUTDOWN;
+  }
+
+  public State getState() {
+    return _state;
+  }
+
+
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/internal/MessageFetcher.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/internal/MessageFetcher.java
new file mode 100644
index 0000000..4d19a18
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/internal/MessageFetcher.java
@@ -0,0 +1,186 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.keyCoordinator.internal;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.Uninterruptibles;
+import org.apache.pinot.grigio.common.OffsetInfo;
+import org.apache.pinot.grigio.common.messages.KeyCoordinatorQueueMsg;
+import org.apache.pinot.grigio.common.metrics.GrigioGauge;
+import org.apache.pinot.grigio.common.rpcQueue.KeyCoordinatorQueueConsumer;
+import org.apache.pinot.grigio.common.rpcQueue.QueueConsumerRecord;
+import org.apache.pinot.grigio.keyCoordinator.GrigioKeyCoordinatorMetrics;
+import org.apache.pinot.grigio.keyCoordinator.helix.State;
+import org.apache.pinot.grigio.keyCoordinator.starter.KeyCoordinatorConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * class to handle fetching messages from the given input message queue and allow other components to get list of
+ * messages from its internal buffer
+ */
+public class MessageFetcher {
+  private static final Logger LOGGER = LoggerFactory.getLogger(MessageFetcher.class);
+  private static final long TERMINATION_WAIT_MS = 10000;
+
+  protected int _fetchMsgDelayMs;
+  protected int _fetchMsgMaxDelayMs;
+  protected int _fetchMsgMaxCount;
+
+  protected BlockingQueue<QueueConsumerRecord<byte[], KeyCoordinatorQueueMsg>> _consumerRecordBlockingQueue;
+  protected KeyCoordinatorQueueConsumer _inputKafkaConsumer;
+  protected GrigioKeyCoordinatorMetrics _metrics;
+  protected ExecutorService _consumerThread;
+
+  protected volatile State _state;
+
+  public MessageFetcher(KeyCoordinatorConf conf, KeyCoordinatorQueueConsumer consumer,
+                        GrigioKeyCoordinatorMetrics metrics) {
+    this(conf, consumer, Executors.newSingleThreadExecutor(), metrics);
+  }
+
+  @VisibleForTesting
+  protected MessageFetcher(KeyCoordinatorConf conf, KeyCoordinatorQueueConsumer consumer,
+                        ExecutorService service, GrigioKeyCoordinatorMetrics metrics) {
+    _inputKafkaConsumer = consumer;
+    _metrics = metrics;
+    _consumerThread = service;
+    _consumerRecordBlockingQueue = new ArrayBlockingQueue<>(conf.getConsumerBlockingQueueSize());
+
+    _fetchMsgDelayMs = conf.getInt(KeyCoordinatorConf.FETCH_MSG_DELAY_MS,
+        KeyCoordinatorConf.FETCH_MSG_DELAY_MS_DEFAULT);
+    _fetchMsgMaxDelayMs = conf.getInt(KeyCoordinatorConf.FETCH_MSG_MAX_DELAY_MS,
+        KeyCoordinatorConf.FETCH_MSG_MAX_DELAY_MS_DEFAULT);
+    _fetchMsgMaxCount = conf.getInt(KeyCoordinatorConf.FETCH_MSG_MAX_BATCH_SIZE,
+        KeyCoordinatorConf.FETCH_MSG_MAX_BATCH_SIZE_DEFAULT);
+
+    _state = State.INIT;
+    LOGGER.info("starting with fetch delay: {} max delay: {}, fetch max count: {}", _fetchMsgDelayMs, _fetchMsgMaxDelayMs,
+        _fetchMsgMaxCount);
+  }
+
+  public void start() {
+    Preconditions.checkState(_state == State.INIT, "key coordinate is not in correct state");
+    _state = State.RUNNING;
+    _consumerThread.submit(this::consumerIngestLoop);
+
+  }
+
+  /**
+   * get a list of messages read by the consumer ingestion thread
+   * @param deadlineInMs linux epoch time we should stop the ingestion and return it to caller with the data we have so far
+   * @return list of messages to be processed
+  */
+  public MessageAndOffset<QueueConsumerRecord<byte[], KeyCoordinatorQueueMsg>> getMessages(long deadlineInMs) {
+    List<QueueConsumerRecord<byte[], KeyCoordinatorQueueMsg>> buffer = new ArrayList<>(_fetchMsgMaxCount);
+    while(System.currentTimeMillis() < deadlineInMs && buffer.size() < _fetchMsgMaxCount) {
+      _consumerRecordBlockingQueue.drainTo(buffer, _fetchMsgMaxCount - buffer.size());
+      if (buffer.size() < _fetchMsgMaxCount) {
+        Uninterruptibles.sleepUninterruptibly(_fetchMsgDelayMs, TimeUnit.MILLISECONDS);
+      }
+    }
+    OffsetInfo offsetInfo = new OffsetInfo();
+    for (QueueConsumerRecord<byte[], KeyCoordinatorQueueMsg> record: buffer) {
+      offsetInfo.updateOffsetIfNecessary(record);
+    }
+    return new MessageAndOffset<>(buffer, offsetInfo);
+  }
+
+  /**
+   * commit the current ingestion progress to internal offset storage
+   * @param messageAndOffset
+   */
+  public void ackOffset(MessageAndOffset messageAndOffset) {
+    _inputKafkaConsumer.ackOffset(messageAndOffset.getOffsetInfo());
+  }
+
+  public void stop() {
+    _state = State.SHUTTING_DOWN;
+    _consumerThread.shutdown();
+    try {
+      _consumerThread.awaitTermination(TERMINATION_WAIT_MS, TimeUnit.MILLISECONDS);
+    } catch (InterruptedException ex) {
+      LOGGER.error("failed to wait for key coordinator thread to shutdown", ex);
+    }
+    _consumerThread.shutdownNow();
+  }
+
+  private void consumerIngestLoop() {
+    while (_state == State.RUNNING) {
+      try {
+        _metrics.setValueOfGlobalGauge(GrigioGauge.MESSAGE_PROCESS_QUEUE_SIZE, _consumerRecordBlockingQueue.size());
+        List<QueueConsumerRecord<byte[], KeyCoordinatorQueueMsg>> records =
+            _inputKafkaConsumer.getRequests(_fetchMsgMaxDelayMs, TimeUnit.MILLISECONDS);
+        if (records.size() == 0) {
+          LOGGER.info("no message found in kafka consumer, sleep and wait for next batch");
+          Uninterruptibles.sleepUninterruptibly(_fetchMsgDelayMs, TimeUnit.MILLISECONDS);
+        } else {
+          records.forEach(c -> {
+            try {
+              _consumerRecordBlockingQueue.put(c);
+            } catch (InterruptedException e) {
+              LOGGER.warn("exception while trying to put message to queue", e);
+            }
+          });
+          _metrics.setValueOfGlobalGauge(GrigioGauge.KC_INPUT_MESSAGE_LAG_MS,
+              System.currentTimeMillis() - records.get(records.size() - 1).getTimestamp());
+        }
+      } catch (Exception ex) {
+        LOGGER.error("encountered exception in consumer ingest loop, will retry", ex);
+      }
+    }
+    LOGGER.info("exiting consumer ingest loop");
+  }
+
+  /**
+   * class wrap around the message and offset associated information
+   * @param <K>
+   */
+  public static class MessageAndOffset<K> {
+    private List<K> _messages;
+    private OffsetInfo _offsetInfo;
+
+    /**
+     * @param messages list of messages for the current batch
+     * @param offsetInfo the largest offset for each partition in current set of messages
+     */
+    public MessageAndOffset(List<K> messages, OffsetInfo offsetInfo) {
+      _messages = messages;
+      _offsetInfo  = offsetInfo;
+    }
+
+    public List<K> getMessages() {
+      return _messages;
+    }
+
+    public OffsetInfo getOffsetInfo() {
+      return _offsetInfo;
+    }
+  }
+
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/internal/SegmentEventProcessor.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/internal/SegmentEventProcessor.java
new file mode 100644
index 0000000..cf6f762
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/internal/SegmentEventProcessor.java
@@ -0,0 +1,427 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.keyCoordinator.internal;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.commons.configuration.Configuration;
+import org.apache.pinot.common.config.TableNameBuilder;
+import org.apache.pinot.common.utils.CommonConstants;
+import org.apache.pinot.grigio.common.DistributedCommonUtils;
+import org.apache.pinot.grigio.common.keyValueStore.ByteArrayWrapper;
+import org.apache.pinot.grigio.common.keyValueStore.KeyValueStoreDB;
+import org.apache.pinot.grigio.common.keyValueStore.KeyValueStoreTable;
+import org.apache.pinot.grigio.common.keyValueStore.RocksDBKeyValueStoreDB;
+import org.apache.pinot.grigio.common.messages.KeyCoordinatorMessageContext;
+import org.apache.pinot.grigio.common.messages.KeyCoordinatorQueueMsg;
+import org.apache.pinot.grigio.common.messages.LogCoordinatorMessage;
+import org.apache.pinot.grigio.common.messages.LogEventType;
+import org.apache.pinot.grigio.common.metrics.GrigioGauge;
+import org.apache.pinot.grigio.common.metrics.GrigioMeter;
+import org.apache.pinot.grigio.common.metrics.GrigioTimer;
+import org.apache.pinot.grigio.common.rpcQueue.LogCoordinatorQueueProducer;
+import org.apache.pinot.grigio.common.rpcQueue.ProduceTask;
+import org.apache.pinot.grigio.common.rpcQueue.QueueConsumerRecord;
+import org.apache.pinot.grigio.common.storageProvider.UpdateLogEntry;
+import org.apache.pinot.grigio.common.storageProvider.UpdateLogStorageProvider;
+import org.apache.pinot.grigio.common.storageProvider.retentionManager.UpdateLogRetentionManager;
+import org.apache.pinot.grigio.common.storageProvider.retentionManager.UpdateLogTableRetentionManager;
+import org.apache.pinot.grigio.common.updateStrategy.MessageResolveStrategy;
+import org.apache.pinot.grigio.keyCoordinator.GrigioKeyCoordinatorMetrics;
+import org.apache.pinot.grigio.keyCoordinator.helix.State;
+import org.apache.pinot.grigio.keyCoordinator.starter.KeyCoordinatorConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+/**
+ * process a list of input messages for a key coordinator, sending out updates events and update internal data storage
+ */
+public class SegmentEventProcessor {
+  private static final Logger LOGGER = LoggerFactory.getLogger(SegmentEventProcessor.class);
+
+  private static final long TERMINATION_WAIT_MS = 10_000L;
+  public static final String PROCESS_THREAD_COUNT = "kc.processor.threadCount";
+  public static final int PROCESS_THREAD_COUNT_DEFAULT = 5;
+
+  // constructor provided param
+  protected KeyCoordinatorConf _conf;
+  protected LogCoordinatorQueueProducer _outputKafkaProducer;
+  protected MessageResolveStrategy _messageResolveStrategy;
+  protected KeyValueStoreDB<ByteArrayWrapper, KeyCoordinatorMessageContext> _keyValueStoreDB;
+  protected UpdateLogStorageProvider _storageProvider;
+  protected UpdateLogRetentionManager _retentionManager;
+  protected GrigioKeyCoordinatorMetrics _metrics;
+  protected VersionMessageManager _versionMessageManager;
+
+  // config provided param
+  protected String _topicPrefix;
+  protected ExecutorService _service;
+
+  protected volatile State _state;
+
+  public SegmentEventProcessor(KeyCoordinatorConf conf, LogCoordinatorQueueProducer keyCoordinatorProducer,
+                               MessageResolveStrategy messageResolveStrategy,
+                               UpdateLogRetentionManager updateLogRetentionManager,
+                               VersionMessageManager versionMessageManager,
+                               GrigioKeyCoordinatorMetrics metrics) {
+    this(conf, keyCoordinatorProducer, messageResolveStrategy,
+        getKeyValueStore(conf.subset(KeyCoordinatorConf.KEY_COORDINATOR_KV_STORE)),
+        UpdateLogStorageProvider.getInstance(),
+        updateLogRetentionManager,
+        versionMessageManager,
+        metrics);
+  }
+
+  @VisibleForTesting
+  public SegmentEventProcessor(KeyCoordinatorConf conf, LogCoordinatorQueueProducer keyCoordinatorProducer,
+                               MessageResolveStrategy messageResolveStrategy,
+                               KeyValueStoreDB<ByteArrayWrapper, KeyCoordinatorMessageContext> keyValueStoreDB,
+                               UpdateLogStorageProvider storageProvider,
+                               UpdateLogRetentionManager updateLogRetentionManager,
+                               VersionMessageManager versionMessageManager,
+                               GrigioKeyCoordinatorMetrics metrics) {
+    _conf = conf;
+    _outputKafkaProducer = keyCoordinatorProducer;
+    _messageResolveStrategy = messageResolveStrategy;
+    _keyValueStoreDB = keyValueStoreDB;
+    _storageProvider = storageProvider;
+    _retentionManager = updateLogRetentionManager;
+    _versionMessageManager = versionMessageManager;
+    _metrics = metrics;
+    // get config
+    _topicPrefix = conf.getTopicPrefix();
+    _service = Executors.newFixedThreadPool(conf.getInt(PROCESS_THREAD_COUNT, PROCESS_THREAD_COUNT_DEFAULT));
+
+    _state = State.INIT;
+  }
+
+  public void start() {
+    _state = State.RUNNING;
+  }
+
+  public void stop() {
+    _state = State.SHUTTING_DOWN;
+    _service.shutdown();
+    try  {
+      _service.awaitTermination(TERMINATION_WAIT_MS, TimeUnit.MILLISECONDS);
+    } catch (InterruptedException ex) {
+      LOGGER.error("failed to wait for message processing thread to shutdown", ex);
+    }
+    _service.shutdownNow();
+    _state = State.SHUTDOWN;
+  }
+
+  /**
+   * process a list of update logs (update kv, send to downstream kafka, etc)
+   * @param messages list of the segment ingestion event for us to use in updates
+   */
+  public void processMessages(List<QueueConsumerRecord<byte[], KeyCoordinatorQueueMsg>> messages) {
+    long start = System.currentTimeMillis();
+    Map<String, List<MessageWithPartitionAndVersion>> tableMsgMap = new HashMap<>();
+    for (QueueConsumerRecord<byte[], KeyCoordinatorQueueMsg> msg: messages) {
+      if (msg.getRecord().isVersionMessage()) {
+        // update _currentVersionConsumed for each version message
+        _versionMessageManager.maybeUpdateVersionConsumed(msg.getPartition(), msg.getRecord().getVersion());
+        _metrics.setValueOfTableGauge(String.valueOf(msg.getPartition()), GrigioGauge.KC_VERSION_CONSUMED, msg.getRecord().getVersion());
+      } else {
+        // filter out version messages, attach the current version (and partition) to each regular messages
+        tableMsgMap.computeIfAbsent(msg.getRecord().getPinotTableName(), t -> new ArrayList<>()).add(
+            new MessageWithPartitionAndVersion(
+                msg.getRecord().getKey(),
+                msg.getRecord().getSegmentName(),
+                msg.getRecord().getKafkaOffset(),
+                msg.getRecord().getTimestamp(),
+                msg.getPartition(),
+                _versionMessageManager.getVersionConsumed(msg.getPartition())
+            )
+        );
+      }
+    }
+    for (Map.Entry<String, List<MessageWithPartitionAndVersion>> perTableUpdates: tableMsgMap.entrySet()) {
+      processMessagesForTable(perTableUpdates.getKey(), perTableUpdates.getValue());
+    }
+    _metrics.addTimedValueMs(GrigioTimer.MESSAGE_PROCESS_THREAD_PROCESS_DELAY, System.currentTimeMillis() - start);
+  }
+
+  /**
+   * process update for a specific table
+   * @param tableName the name of the table we are processing (with type)
+   * @param msgList the list of message associated with it
+   */
+  protected void processMessagesForTable(String tableName, List<MessageWithPartitionAndVersion> msgList) {
+    Preconditions.checkState(_state == State.RUNNING, "segment update processor is not running");
+    try {
+      long start = System.currentTimeMillis();
+      if (msgList == null || msgList.size() == 0) {
+        LOGGER.warn("trying to process topic message with empty list {}", tableName);
+        return;
+      }
+      List<ProduceTask<Integer, LogCoordinatorMessage>> tasks = new ArrayList<>();
+      Map<ByteArrayWrapper, KeyCoordinatorMessageContext> primaryKeyToValueMap = fetchDataFromKVStore(tableName, msgList);
+      processMessageUpdates(tableName, msgList, primaryKeyToValueMap, tasks);
+      List<ProduceTask<Integer, LogCoordinatorMessage>> failedTasks = sendMessagesToLogCoordinator(tasks, 10, TimeUnit.SECONDS);
+      if (failedTasks.size() > 0) {
+        LOGGER.error("send to producer failed: {}", failedTasks.size());
+      }
+      storeMessageToLocal(tableName, tasks);
+      storeMessageToKVStore(tableName, primaryKeyToValueMap);
+      _metrics.addTimedTableValueMs(tableName, GrigioTimer.MESSAGE_PROCESS_THREAD_PROCESS_DELAY, System.currentTimeMillis() - start);
+    } catch (IOException e) {
+      throw new RuntimeException("failed to interact with rocksdb", e);
+    } catch (RuntimeException e) {
+      throw new RuntimeException("failed to interact with key value store", e);
+    }
+  }
+
+  /**
+   * create a message to be sent to output kafka topic based on a list of metadata
+   * partition is the partition number of the ingestion upsert event. Note that the partition of records with primary
+   * key will be the same across ingestion upsert event and segment update event topics.
+   * TODO: remove name referring log coordinator
+   */
+  protected ProduceTask<Integer, LogCoordinatorMessage> createMessageToLogCoordinator(String tableName, String segmentName,
+                                                                                    long oldKafkaOffset, long value,
+                                                                                    LogEventType eventType, int partition) {
+    return new ProduceTask<>(DistributedCommonUtils.getKafkaTopicFromTableName(tableName, _topicPrefix),
+        partition, new LogCoordinatorMessage(segmentName, oldKafkaOffset, value, eventType));
+  }
+
+  /**
+   * fetch all available data from kv from the primary key associated with the messages in the given message list
+   * @param tableName the name of table we are processing
+   * @param msgList list of ingestion update messages
+   * @return map of primary key and their associated state in key-value store
+   * @throws IOException
+   */
+  protected Map<ByteArrayWrapper, KeyCoordinatorMessageContext> fetchDataFromKVStore(String tableName,
+                                                                                   List<MessageWithPartitionAndVersion> msgList)
+      throws IOException {
+    long start = System.currentTimeMillis();
+    KeyValueStoreTable<ByteArrayWrapper, KeyCoordinatorMessageContext> table = _keyValueStoreDB.getTable(tableName);
+    // get a set of unique primary key and retrieve its corresponding value in kv store
+    Set<ByteArrayWrapper> primaryKeys = msgList.stream()
+        .filter(msg -> msg.getKey() != null && msg.getKey().length > 0)
+        .map(msg -> new ByteArrayWrapper(msg.getKey()))
+        .collect(Collectors.toCollection(HashSet::new));
+    Map<ByteArrayWrapper, KeyCoordinatorMessageContext> primaryKeyToValueMap = new HashMap<>(table.multiGet(new ArrayList<>(primaryKeys)));
+    _metrics.setValueOfGlobalGauge(GrigioGauge.FETCH_MSG_FROM_KV_COUNT, primaryKeyToValueMap.size());
+    _metrics.addTimedValueMs(GrigioTimer.FETCH_MSG_FROM_KV_DELAY, System.currentTimeMillis() - start);
+    _metrics.addTimedTableValueMs(tableName, GrigioTimer.FETCH_MSG_FROM_KV_DELAY, System.currentTimeMillis() - start);
+    LOGGER.info("input keys got {} results from kv store in {} ms", primaryKeyToValueMap.size(), System.currentTimeMillis() - start);
+    return primaryKeyToValueMap;
+  }
+
+  /**
+   * process whether a list of messages should be treated as update based on the existing data from kv store
+   * @param tableName the name of the table
+   * @param msgList a list of ingestion update messages to be processed
+   * @param primaryKeyToValueMap the current kv store state associated with this list of messages, will be updated to
+   *                             reflect the new state after these updates are done
+   * @param tasks the generated producer task to be sent to segment update entry queue (kafka topic)
+   */
+  protected void processMessageUpdates(String tableName, List<MessageWithPartitionAndVersion> msgList,
+                                     Map<ByteArrayWrapper, KeyCoordinatorMessageContext> primaryKeyToValueMap,
+                                     List<ProduceTask<Integer, LogCoordinatorMessage>> tasks) {
+    // TODO: add a unit test
+    long start = System.currentTimeMillis();
+    UpdateLogTableRetentionManager tableRetentionManager = _retentionManager.getRetentionManagerForTable(
+        TableNameBuilder.ensureTableNameWithType(tableName, CommonConstants.Helix.TableType.REALTIME));
+    for (MessageWithPartitionAndVersion msg: msgList) {
+      KeyCoordinatorMessageContext currentContext = msg.getContext();
+      ByteArrayWrapper key = new ByteArrayWrapper(msg.getKey());
+      if (primaryKeyToValueMap.containsKey(key)) {
+        // key conflicts, should resolve which one to delete
+        KeyCoordinatorMessageContext oldContext = primaryKeyToValueMap.get(key);
+        if (oldContext.equals(currentContext)) {
+          // message are equals, it is from another replica and should skip
+          continue;
+        }
+        if (_messageResolveStrategy.shouldDeleteFirstMessage(oldContext, currentContext)) {
+          // the existing message we have is older than the message we just processed, create delete for it
+          if (tableRetentionManager.shouldIngestForSegment(oldContext.getSegmentName())) {
+            // only generate delete event if the segment is still valid
+            tasks.add(createMessageToLogCoordinator(tableName, oldContext.getSegmentName(), oldContext.getKafkaOffset(),
+                msg.getVersion(), LogEventType.DELETE, msg.getPartition()));
+          }
+          // update the local cache to the latest message, so message within the same batch can override each other
+          primaryKeyToValueMap.put(key, currentContext);
+          tasks.add(createMessageToLogCoordinator(tableName, currentContext.getSegmentName(), currentContext.getKafkaOffset(),
+              msg.getVersion(), LogEventType.INSERT, msg.getPartition()));
+        }
+      } else {
+        // no key in the existing map, adding this key to the running set
+        primaryKeyToValueMap.put(key, currentContext);
+        tasks.add(createMessageToLogCoordinator(tableName, currentContext.getSegmentName(), currentContext.getKafkaOffset(),
+            msg.getVersion(), LogEventType.INSERT, msg.getPartition()));
+      }
+    }
+    _metrics.addTimedValueMs(GrigioTimer.PROCESS_MSG_UPDATE, System.currentTimeMillis() - start);
+    LOGGER.info("processed all messages in {} ms", System.currentTimeMillis() - start);
+  }
+
+
+  /**
+   * send the list of the message to downstream kafka topic
+   * @param tasks the list of the tasks we are going to produce to downstream kafka
+   * @param timeout how much time we wait for producer to send the messages
+   * @param timeUnit the timeunit for waiting for the producers
+   * @return a list of the tasks we failed to produce to downstream
+   */
+  protected List<ProduceTask<Integer, LogCoordinatorMessage>> sendMessagesToLogCoordinator(
+      List<ProduceTask<Integer, LogCoordinatorMessage>> tasks, long timeout, TimeUnit timeUnit) {
+    long startTime = System.currentTimeMillis();
+    // send all and wait for result, batch for better perf
+    CountDownLatch countDownLatch = new CountDownLatch(tasks.size());
+    tasks.forEach(t -> t.setCallback(new ProduceTask.Callback() {
+      @Override
+      public void onSuccess() {
+        countDownLatch.countDown();
+      }
+      @Override
+      public void onFailure(Exception ex) {
+        countDownLatch.countDown();
+      }
+    }));
+    _outputKafkaProducer.batchProduce(tasks);
+    _outputKafkaProducer.flush();
+    try {
+      countDownLatch.await(timeout, timeUnit);
+      // right now we only set up a metrics for recording produce fails
+      // TODO: design a better way to handle kafka failure
+      List<ProduceTask<Integer, LogCoordinatorMessage>> failedOrTimeoutTasks = tasks.stream().filter(t -> !t.isSucceed()).collect(Collectors.toList());
+      _metrics.addMeteredGlobalValue(GrigioMeter.MESSAGE_PRODUCE_FAILED_COUNT, failedOrTimeoutTasks.size());
+      return failedOrTimeoutTasks;
+    } catch (InterruptedException e) {
+      throw new RuntimeException("encountered run time exception while waiting for the loop to finish");
+    } finally {
+      _metrics.addTimedValueMs(GrigioTimer.SEND_MSG_TO_KAFKA, System.currentTimeMillis() - startTime);
+      LOGGER.info("send to producer take {} ms", System.currentTimeMillis() - startTime);
+    }
+  }
+
+  protected static KeyValueStoreDB<ByteArrayWrapper, KeyCoordinatorMessageContext> getKeyValueStore(Configuration conf) {
+    RocksDBKeyValueStoreDB keyValueStore = new RocksDBKeyValueStoreDB();
+    keyValueStore.init(conf);
+    return keyValueStore;
+  }
+
+  /**
+   * store updates to local update log file, organized by segments
+   * @param tableName
+   * @param tasks
+   * @throws IOException
+   */
+  protected void storeMessageToLocal(String tableName, List<ProduceTask<Integer, LogCoordinatorMessage>> tasks) throws IOException {
+    String tableNameWithType = TableNameBuilder.ensureTableNameWithType(tableName, CommonConstants.Helix.TableType.REALTIME);
+    long start = System.currentTimeMillis();
+    Map<String, List<UpdateLogEntry>> segmentUpdateLogs = new HashMap<>();
+    for (ProduceTask<Integer, LogCoordinatorMessage> task: tasks) {
+      String segmentName = task.getValue().getSegmentName();
+      UpdateLogEntry entry = new UpdateLogEntry(task.getValue(), task.getKey());
+      segmentUpdateLogs.computeIfAbsent(segmentName, s -> new ArrayList<>()).add(entry);
+    }
+    for (Map.Entry<String, List<UpdateLogEntry>> segmentEntry: segmentUpdateLogs.entrySet()) {
+      _storageProvider.addDataToFile(tableNameWithType, segmentEntry.getKey(), segmentEntry.getValue());
+    }
+    long duration = System.currentTimeMillis() - start;
+    _metrics.addTimedValueMs(GrigioTimer.STORE_UPDATE_ON_DISK, duration);
+    _metrics.addTimedTableValueMs(tableName, GrigioTimer.STORE_UPDATE_ON_DISK, duration);
+    LOGGER.info("stored all data to files in {} ms", System.currentTimeMillis() - start);
+  }
+
+  /**
+   * store updates to local kv store
+   * @param tableName the name of the table
+   * @param primaryKeyToValueMap the mapping between the primary-key and the their associated state
+   * @throws IOException
+   */
+  protected void storeMessageToKVStore(String tableName, Map<ByteArrayWrapper, KeyCoordinatorMessageContext> primaryKeyToValueMap)
+      throws IOException {
+    long start = System.currentTimeMillis();
+    // update kv store
+    KeyValueStoreTable<ByteArrayWrapper, KeyCoordinatorMessageContext> table = _keyValueStoreDB.getTable(tableName);
+    table.multiPut(primaryKeyToValueMap);
+    _metrics.addTimedValueMs(GrigioTimer.STORE_UPDATE_ON_KV, System.currentTimeMillis() - start);
+    LOGGER.info("updated {} message to key value store in {} ms", primaryKeyToValueMap.size(), System.currentTimeMillis() - start);
+  }
+
+  /**
+   * Partially processed ingestion upsert messages, with partition and version added
+   */
+  protected static class MessageWithPartitionAndVersion {
+    private final byte[] _key;
+    private final String _segmentName;
+    private final long _kafkaOffset;
+    private final long _timestamp;
+    private final int _partition;
+    private final long _version;
+
+    public MessageWithPartitionAndVersion(byte[] key, String segmentName, long kafkaOffset, long timestamp,
+                                          int partition, long version) {
+      _key = key;
+      _segmentName = segmentName;
+      _kafkaOffset = kafkaOffset;
+      _timestamp = timestamp;
+      _partition = partition;
+      _version = version;
+    }
+
+    public byte[] getKey() {
+      return _key;
+    }
+
+    public int getPartition() {
+      return _partition;
+    }
+
+    public long getVersion() {
+      return _version;
+    }
+
+    public KeyCoordinatorMessageContext getContext() {
+      return new KeyCoordinatorMessageContext(_segmentName, _timestamp, _kafkaOffset);
+    }
+
+    @Override
+    public String toString() {
+      return "MessageWithPartitionAndVersion{" +
+          "_key=" + Arrays.toString(_key) +
+          ", _segmentName='" + _segmentName + '\'' +
+          ", _kafkaOffset=" + _kafkaOffset +
+          ", _timestamp=" + _timestamp +
+          ", _partition=" + _partition +
+          ", _version=" + _version +
+          '}';
+    }
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/internal/VersionMessageManager.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/internal/VersionMessageManager.java
new file mode 100644
index 0000000..859bc0b
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/internal/VersionMessageManager.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
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.keyCoordinator.internal;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableMap;
+import org.apache.helix.HelixManager;
+import org.apache.pinot.grigio.common.messages.KeyCoordinatorQueueMsg;
+import org.apache.pinot.grigio.common.metrics.GrigioGauge;
+import org.apache.pinot.grigio.common.metrics.GrigioTimer;
+import org.apache.pinot.grigio.common.rpcQueue.ProduceTask;
+import org.apache.pinot.grigio.common.rpcQueue.VersionMsgQueueProducer;
+import org.apache.pinot.grigio.keyCoordinator.GrigioKeyCoordinatorMetrics;
+import org.apache.pinot.grigio.keyCoordinator.helix.KeyCoordinatorLeadershipManager;
+import org.apache.pinot.grigio.keyCoordinator.helix.KeyCoordinatorVersionManager;
+import org.apache.pinot.grigio.keyCoordinator.helix.State;
+import org.apache.pinot.grigio.keyCoordinator.starter.KeyCoordinatorConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.Timer;
+import java.util.TimerTask;
+
+/**
+ * class to handle version information in key coordinator, it does the following two tasks:
+ * 1. send out version message to kc input queue if it is leader at the moment
+ * 2. store and manage the current version for each input partition.
+ */
+public class VersionMessageManager {
+  private static final Logger LOGGER = LoggerFactory.getLogger(VersionMessageManager.class);
+
+  protected KeyCoordinatorConf _conf;
+  protected Timer _versionMessageTimer;
+  protected TimerTask _versionMessageProduceTask;
+  protected VersionMsgQueueProducer _versionMessageKafkaProducer;
+  protected KeyCoordinatorVersionManager _keyCoordinatorVersionManager;
+  protected KeyCoordinatorLeadershipManager _keyCoordinatorLeadershipManager;
+  protected GrigioKeyCoordinatorMetrics _metrics;
+  protected long _versionMessageIntervalMs;
+  protected Map<Integer, Long> _currentVersionConsumed;
+
+  protected volatile State _state;
+
+  public VersionMessageManager(KeyCoordinatorConf conf, VersionMsgQueueProducer versionMsgQueueProducer,
+                               HelixManager controllerHelixManager, GrigioKeyCoordinatorMetrics metrics) {
+    this(conf,
+        versionMsgQueueProducer,
+        new Timer(),
+        new KeyCoordinatorVersionManager(controllerHelixManager),
+        new KeyCoordinatorLeadershipManager(controllerHelixManager),
+        metrics);
+  }
+
+  @VisibleForTesting
+  public VersionMessageManager(KeyCoordinatorConf conf, VersionMsgQueueProducer versionProducer,
+                               Timer versionMessageTimer, KeyCoordinatorVersionManager versionManager,
+                               KeyCoordinatorLeadershipManager leadershipManager, GrigioKeyCoordinatorMetrics metrics) {
+    _conf = conf;
+    _versionMessageTimer = versionMessageTimer;
+    _versionMessageKafkaProducer = versionProducer;
+    _keyCoordinatorVersionManager = versionManager;
+    _keyCoordinatorLeadershipManager = leadershipManager;
+    _metrics = metrics;
+    _versionMessageProduceTask = new TimerTask() {
+      @Override
+      public void run() {
+        produceVersionMessage();
+      }
+    };
+    _versionMessageIntervalMs = conf.getLong(KeyCoordinatorConf.VERSION_MESSAGE_INTERVAL_MS,
+        KeyCoordinatorConf.VERSION_MESSAGE_INTERVAL_MS_DEFAULT);
+    _currentVersionConsumed = _keyCoordinatorVersionManager.getVersionConsumedFromPropertyStore();
+
+    _state = State.INIT;
+  }
+
+  public void start() {
+    _state = State.RUNNING;
+    _versionMessageTimer.schedule(_versionMessageProduceTask, 0L, _versionMessageIntervalMs);
+  }
+
+  public void stop() {
+    _versionMessageTimer.cancel();
+  }
+
+  private void produceVersionMessage() {
+    long start = System.currentTimeMillis();
+    if (_state != State.RUNNING) {
+      LOGGER.info("Key coordinator not running, skip producing version messages");
+      return;
+    }
+    if (!_keyCoordinatorLeadershipManager.isLeader()) {
+      LOGGER.debug("Not controller leader, skip producing version messages");
+      return;
+    }
+    try {
+      long versionProduced = _keyCoordinatorVersionManager.getVersionProducedFromPropertyStore();
+      long versionToProduce = versionProduced + 1;
+      // produce to all partitions
+      for (int partition = 0; partition < _conf.getKeyCoordinatorMessagePartitionCount(); partition++) {
+        ProduceTask<Integer, KeyCoordinatorQueueMsg> produceTask =
+            new ProduceTask<>(_conf.getKeyCoordinatorMessageTopic(), partition,
+                new KeyCoordinatorQueueMsg(versionToProduce));
+        _versionMessageKafkaProducer.produce(produceTask);
+      }
+      // todo: make producing version messages and setting versions to property store as one transaction
+      _keyCoordinatorVersionManager.setVersionProducedToPropertyStore(versionToProduce);
+      long duration = System.currentTimeMillis() - start;
+      _metrics.addTimedValueMs(GrigioTimer.PRODUCE_VERSION_MESSAGE, duration);
+      _metrics.setValueOfGlobalGauge(GrigioGauge.VERSION_PRODUCED, versionToProduce);
+      LOGGER.info("Produced version messages to all partitions with version {} in {} ms", versionToProduce, duration);
+    } catch (Exception ex) {
+      LOGGER.error("Failed to produce version message", ex);
+    }
+  }
+
+  /**
+   * store the current version for all known partition to property store
+   */
+  public synchronized void setVersionConsumedToPropertyStore() {
+    _keyCoordinatorVersionManager.setVersionConsumedToPropertyStore(ImmutableMap.copyOf(_currentVersionConsumed));
+  }
+
+  /**
+   * update the version for a given partition if it is larger than current version value
+   * @param partition the partition we are updating
+   * @param version the version we are updating to
+   */
+  public synchronized void maybeUpdateVersionConsumed(int partition, long version) {
+    if (!_currentVersionConsumed.containsKey(partition) || _currentVersionConsumed.get(partition) < version) {
+      _currentVersionConsumed.put(partition, version);
+    }
+  }
+
+  /**
+   * get the current version for a given partition
+   * @param partition the partition we are trying to fetch the version from
+   * @return the version associated with the current partition
+   */
+  public synchronized long getVersionConsumed(int partition) {
+    return _currentVersionConsumed.getOrDefault(partition, 0L);
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/starter/KeyCoordinatorConf.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/starter/KeyCoordinatorConf.java
new file mode 100644
index 0000000..d4ccf02
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/starter/KeyCoordinatorConf.java
@@ -0,0 +1,168 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.keyCoordinator.starter;
+
+import org.apache.commons.configuration.Configuration;
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.commons.configuration.PropertiesConfiguration;
+import org.apache.pinot.common.utils.CommonConstants;
+import org.apache.pinot.common.utils.StringUtil;
+import org.apache.pinot.grigio.common.config.CommonConfig;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+
+public class KeyCoordinatorConf extends PropertiesConfiguration {
+
+  public static final String FETCH_MSG_DELAY_MS = "kc.queue.fetch.delay.ms";
+  public static final int FETCH_MSG_DELAY_MS_DEFAULT = 100;
+
+  public static final String FETCH_MSG_MAX_DELAY_MS = "kc.queue.fetch.delay.max.ms";
+  public static final int FETCH_MSG_MAX_DELAY_MS_DEFAULT = 5000;
+
+  public static final String FETCH_MSG_MAX_BATCH_SIZE = "kc.queue.fetch.size";
+  public static final int FETCH_MSG_MAX_BATCH_SIZE_DEFAULT = 10000;
+
+  public static final String CONSUMER_BLOCKING_QUEUE_SIZE = "consumer.queue.size";
+  public static final int CONSUMER_BLOCKING_QUEUE_SIZE_DEFAULT = 10000;
+
+  public static final String VERSION_MESSAGE_INTERVAL_MS = "version.message.interval.ms";
+  public static final long VERSION_MESSAGE_INTERVAL_MS_DEFAULT = 1000;
+
+  public static final String KEY_COORDINATOR_KV_STORE = "kvstore";
+
+  // server related config
+  public static final String SERVER_CONFIG = "web.server";
+  public static final String PORT = "jersey.port";
+  public static final int PORT_DEFAULT = 8092;
+  public static final String HOST_NAME = "hostname";
+
+  // storage provider config
+  public static final String STORAGE_PROVIDER_CONFIG = "updatelog.storage";
+
+  // kafka prefix
+  public static final String KAFKA_CLIENT_ID_PREFIX = "pinot_upsert_client_";
+  public static final String KAFKA_CONSUMER_GROUP_ID_PREFIX = "pinot_upsert_kc_consumerGroup_";
+
+  private static final String KC_MESSAGE_TOPIC = "kc.message.topic";
+  private static final String KC_MESSAGE_PARTITION_COUNT = "kc.message.partition.count";
+
+  public static final String KC_OUTPUT_TOPIC_PREFIX_KEY = "kc.output.topic.prefix";
+
+  // helix related cofig
+  private static final String HELIX_CLUSTER_NAME = "helix.cluster.name";
+  private static final String ZK_STR = "zk.str";
+  private static final String KC_CLUSTER_NAME = "kc.cluster.name";
+
+  private static final String PINOT_HELIX_CLUSTER_NAME = "pinot.helix.cluster.name";
+  private static final String PINOT_HELIX_ZK_STR = "pinot.zk.str";
+
+  // metrics related config
+  public static final String METRICS_CONFIG = "metrics";
+
+  public KeyCoordinatorConf(File file) throws ConfigurationException {
+    super(file);
+  }
+
+  public KeyCoordinatorConf() {
+    super();
+  }
+
+  public int getConsumerBlockingQueueSize() {
+    return getInt(CONSUMER_BLOCKING_QUEUE_SIZE, CONSUMER_BLOCKING_QUEUE_SIZE_DEFAULT);
+  }
+
+  public Configuration getProducerConf() {
+    return this.subset(CommonConfig.RPC_QUEUE_CONFIG.PRODUCER_CONFIG_KEY);
+  }
+
+  public Configuration getConsumerConf() {
+    return this.subset(CommonConfig.RPC_QUEUE_CONFIG.CONSUMER_CONFIG_KEY);
+  }
+
+  public Configuration getVersionMessageProducerConf() {
+    return this.subset(CommonConfig.RPC_QUEUE_CONFIG.VERSION_MESSAGE_PRODUCER_CONFIG_KEY);
+  }
+
+  public Configuration getMetricsConf() {
+    return this.subset(METRICS_CONFIG);
+  }
+
+  public Configuration getServerConf() {
+    return this.subset(SERVER_CONFIG);
+  }
+
+  public Configuration getStorageProviderConf() {
+    return this.subset(STORAGE_PROVIDER_CONFIG);
+  }
+
+  public int getPort() {
+    return this.subset(SERVER_CONFIG).getInt(KeyCoordinatorConf.PORT, KeyCoordinatorConf.PORT_DEFAULT);
+  }
+
+  public String getTopicPrefix() {
+    return this.getString(KC_OUTPUT_TOPIC_PREFIX_KEY);
+  }
+
+  public String getKeyCoordinatorClusterName() {
+    return this.getString(KC_CLUSTER_NAME);
+  }
+
+  public String getZkStr() {
+    return convertConfigToZkString(getProperty(ZK_STR));
+  }
+
+  public String getPinotClusterZkStr() {
+    return convertConfigToZkString(getProperty(PINOT_HELIX_ZK_STR));
+  }
+
+  public String getPinotClusterName() {
+    return this.getString(PINOT_HELIX_CLUSTER_NAME);
+  }
+
+  // convert the config value to zk string
+  private String convertConfigToZkString(Object zkAddressObj) {
+    // The set method converted comma separated string into ArrayList, so need to convert back to String here.
+    if (zkAddressObj instanceof ArrayList) {
+      List<String> zkAddressList = (ArrayList<String>) zkAddressObj;
+      String[] zkAddress = zkAddressList.toArray(new String[0]);
+      return StringUtil.join(",", zkAddress);
+    } else if (zkAddressObj instanceof String) {
+      return (String) zkAddressObj;
+    } else {
+      throw new RuntimeException(
+          "Unexpected data type for zkAddress PropertiesConfiguration, expecting String but got " + zkAddressObj
+              .getClass().getName());
+    }
+
+  }
+
+  public String getKeyCoordinatorMessageTopic() {
+    return this.getString(KC_MESSAGE_TOPIC);
+  }
+
+  public int getKeyCoordinatorMessagePartitionCount() {
+    return this.getInt(KC_MESSAGE_PARTITION_COUNT);
+  }
+
+  public String getMetricsPrefix() {
+    return getString(CommonConstants.Grigio.CONFIG_OF_METRICS_PREFIX_KEY, CommonConstants.Grigio.DEFAULT_METRICS_PREFIX);
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/starter/KeyCoordinatorStarter.java b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/starter/KeyCoordinatorStarter.java
new file mode 100644
index 0000000..6db84d9
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/main/java/org/apache/pinot/grigio/keyCoordinator/starter/KeyCoordinatorStarter.java
@@ -0,0 +1,202 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.keyCoordinator.starter;
+
+import com.google.common.base.Preconditions;
+import com.yammer.metrics.core.MetricsRegistry;
+import org.apache.commons.configuration.Configuration;
+import org.apache.commons.lang.StringUtils;
+import org.apache.pinot.common.metrics.MetricsHelper;
+import org.apache.pinot.common.utils.CommonConstants;
+import org.apache.pinot.grigio.common.utils.IdealStateHelper;
+import org.apache.pinot.grigio.common.config.CommonConfig;
+import org.apache.pinot.grigio.common.rpcQueue.KeyCoordinatorQueueConsumer;
+import org.apache.pinot.grigio.common.rpcQueue.LogCoordinatorQueueProducer;
+import org.apache.pinot.grigio.common.rpcQueue.VersionMsgQueueProducer;
+import org.apache.pinot.grigio.common.storageProvider.UpdateLogStorageProvider;
+import org.apache.pinot.grigio.common.storageProvider.retentionManager.KCUpdateLogRetentionManagerImpl;
+import org.apache.pinot.grigio.common.updateStrategy.MessageResolveStrategy;
+import org.apache.pinot.grigio.common.updateStrategy.MessageTimeResolveStrategy;
+import org.apache.pinot.grigio.keyCoordinator.GrigioKeyCoordinatorMetrics;
+import org.apache.pinot.grigio.keyCoordinator.api.KeyCoordinatorApiApplication;
+import org.apache.pinot.grigio.keyCoordinator.helix.KeyCoordinatorClusterHelixManager;
+import org.apache.pinot.grigio.keyCoordinator.helix.KeyCoordinatorPinotHelixSpectator;
+import org.apache.pinot.grigio.keyCoordinator.helix.State;
+import org.apache.pinot.grigio.keyCoordinator.internal.DistributedKeyCoordinatorCore;
+import org.apache.pinot.grigio.keyCoordinator.internal.MessageFetcher;
+import org.apache.pinot.grigio.keyCoordinator.internal.SegmentEventProcessor;
+import org.apache.pinot.grigio.keyCoordinator.internal.VersionMessageManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.Iterator;
+
+public class KeyCoordinatorStarter {
+  private static final Logger LOGGER = LoggerFactory.getLogger(KeyCoordinatorStarter.class);
+
+  private KeyCoordinatorConf _keyCoordinatorConf;
+  private GrigioKeyCoordinatorMetrics _metrics;
+  private KeyCoordinatorQueueConsumer _consumer;
+  private LogCoordinatorQueueProducer _producer;
+  private VersionMsgQueueProducer _versionMessageProducer;
+  private MessageResolveStrategy _messageResolveStrategy;
+  private DistributedKeyCoordinatorCore _keyCoordinatorCore;
+  private KeyCoordinatorApiApplication _application;
+  private String _hostName;
+  private int _port;
+  private String _instanceId;
+  private KCUpdateLogRetentionManagerImpl _retentionManager;
+  private KeyCoordinatorClusterHelixManager _keyCoordinatorClusterHelixManager;
+  private KeyCoordinatorPinotHelixSpectator _keyCoordinatorPinotHelixSpectator;
+
+  public KeyCoordinatorStarter(KeyCoordinatorConf conf) throws Exception {
+    _keyCoordinatorConf = conf;
+    initMetrics(_keyCoordinatorConf.getMetricsConf(), _keyCoordinatorConf.getMetricsPrefix());
+    _hostName = conf.getString(KeyCoordinatorConf.HOST_NAME);
+    Preconditions.checkState(StringUtils.isNotEmpty(_hostName), "expect host name in configuration");
+    _port = conf.getPort();
+    _instanceId = CommonConstants.Helix.PREFIX_OF_KEY_COORDINATOR_INSTANCE + _hostName + "_" + _port;
+    _consumer = getConsumer(_keyCoordinatorConf.getConsumerConf());
+    _producer = getProducer(_keyCoordinatorConf.getProducerConf());
+    _versionMessageProducer = getVersionMessageProducer(_keyCoordinatorConf.getVersionMessageProducerConf());
+    _keyCoordinatorClusterHelixManager = new KeyCoordinatorClusterHelixManager(
+        _keyCoordinatorConf.getZkStr(),
+        _keyCoordinatorConf.getKeyCoordinatorClusterName(),
+        _instanceId,
+        _consumer,
+        conf.getKeyCoordinatorMessageTopic(),
+        conf.getKeyCoordinatorMessagePartitionCount()
+    );
+    UpdateLogStorageProvider.init(_keyCoordinatorConf.getStorageProviderConf());
+    _keyCoordinatorPinotHelixSpectator = new KeyCoordinatorPinotHelixSpectator(
+        _keyCoordinatorConf.getPinotClusterZkStr(), _keyCoordinatorConf.getPinotClusterName(), _instanceId);
+    _retentionManager = new KCUpdateLogRetentionManagerImpl(
+        new IdealStateHelper(_keyCoordinatorPinotHelixSpectator.getHelixManager().getClusterManagmentTool(), _keyCoordinatorConf.getPinotClusterName()),
+        UpdateLogStorageProvider.getInstance(), _instanceId);
+    _keyCoordinatorPinotHelixSpectator.addListener(_retentionManager);
+    _messageResolveStrategy = new MessageTimeResolveStrategy();
+    _keyCoordinatorCore = new DistributedKeyCoordinatorCore();
+    _application = new KeyCoordinatorApiApplication(this);
+  }
+
+  private void initMetrics(Configuration conf, String prefix) {
+    MetricsHelper.initializeMetrics(conf);
+    MetricsRegistry registry = new MetricsRegistry();
+    MetricsHelper.registerMetricsRegistry(registry);
+    _metrics = new GrigioKeyCoordinatorMetrics(prefix, registry);
+    _metrics.initializeGlobalMeters();
+  }
+
+  private KeyCoordinatorQueueConsumer getConsumer(Configuration consumerConfig) {
+    consumerConfig.setProperty(CommonConfig.RPC_QUEUE_CONFIG.HOSTNAME_KEY, _hostName);
+    KeyCoordinatorQueueConsumer consumer = new KeyCoordinatorQueueConsumer();
+    consumer.init(consumerConfig, _metrics);
+    return consumer;
+  }
+
+  private LogCoordinatorQueueProducer getProducer(Configuration producerConfig) {
+    producerConfig.setProperty(CommonConfig.RPC_QUEUE_CONFIG.HOSTNAME_KEY, _hostName);
+    LogCoordinatorQueueProducer producer = new LogCoordinatorQueueProducer();
+    producer.init(producerConfig, _metrics);
+    return producer;
+  }
+
+  private VersionMsgQueueProducer getVersionMessageProducer(Configuration configuration) {
+    configuration.setProperty(CommonConfig.RPC_QUEUE_CONFIG.HOSTNAME_KEY, _hostName);
+    VersionMsgQueueProducer versionMessageProducer = new VersionMsgQueueProducer();
+    versionMessageProducer.init(configuration, _metrics);
+    return versionMessageProducer;
+  }
+
+  public KeyCoordinatorClusterHelixManager getKeyCoordinatorClusterHelixManager() {
+    return _keyCoordinatorClusterHelixManager;
+  }
+
+  public KeyCoordinatorQueueConsumer getConsumer() {
+    return _consumer;
+  }
+
+  public void start() {
+    LOGGER.info("starting key coordinator instance");
+    final MessageFetcher fetcher = new MessageFetcher(_keyCoordinatorConf, _consumer, _metrics);
+    final VersionMessageManager versionMessageManager = new VersionMessageManager(_keyCoordinatorConf,
+        _versionMessageProducer, _keyCoordinatorClusterHelixManager.getControllerHelixManager(), _metrics);
+    final SegmentEventProcessor segmentEventProcessor = new SegmentEventProcessor(_keyCoordinatorConf, _producer,
+        _messageResolveStrategy, _retentionManager, versionMessageManager, _metrics);
+    _keyCoordinatorCore.init(_keyCoordinatorConf, segmentEventProcessor, fetcher, versionMessageManager, _metrics);
+    LOGGER.info("finished init key coordinator instance, starting loop");
+    _keyCoordinatorCore.start();
+    LOGGER.info("starting web service");
+    _application.start(_port);
+  }
+
+  public void shutdown() {
+    LOGGER.info("shutting down key coordinator instance");
+    _keyCoordinatorCore.stop();
+    LOGGER.info("finished shutdown key coordinator instance");
+    _producer.close();
+    LOGGER.info("finished shutdown producer");
+    _consumer.close();
+    LOGGER.info("finished shutdown consumer");
+    _versionMessageProducer.close();
+    LOGGER.info("finished shutdown version message producer");
+  }
+
+  public boolean isRunning() {
+    return _keyCoordinatorCore != null && _keyCoordinatorCore.getState() == State.RUNNING;
+  }
+
+  public static KeyCoordinatorStarter startDefault(KeyCoordinatorConf conf) throws Exception {
+    KeyCoordinatorStarter starter = new KeyCoordinatorStarter(conf);
+    starter.start();
+    return starter;
+  }
+
+  public static void main(String[] args) throws Exception {
+    if (args.length == 0) {
+      System.out.println("need path to file in props");
+    }
+    File confFile = new File(args[0]);
+    if (!confFile.exists()) {
+      System.out.println("conf file does not exist");
+    }
+    KeyCoordinatorConf properties = new KeyCoordinatorConf(confFile);
+    LOGGER.info(properties.toString());
+    Iterator<String> iterators = properties.getKeys();
+    while (iterators.hasNext()) {
+      String key = iterators.next();
+      LOGGER.info("grigio kc Prop: key= " + key + ", value= " + properties.getString(key));
+    }
+    KeyCoordinatorStarter starter = startDefault(properties);
+
+    Runtime.getRuntime().addShutdownHook(new Thread() {
+      public void run() {
+        try {
+          LOGGER.info("received shutdown event from shutdown hook");
+          starter.shutdown();
+        } catch (Exception e) {
+          e.printStackTrace();
+          LOGGER.error("error shutting down key coordinator: ", e);
+        }
+      }
+    });
+
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/test/java/helix/ActiveTableTrackerTest.java b/pinot-grigio/pinot-grigio-coordinator/src/test/java/helix/ActiveTableTrackerTest.java
new file mode 100644
index 0000000..b671313
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/test/java/helix/ActiveTableTrackerTest.java
@@ -0,0 +1,110 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package helix;
+
+import org.apache.pinot.grigio.keyCoordinator.helix.ActiveTableTracker;
+import org.apache.pinot.grigio.keyCoordinator.helix.TableListener;
+import org.apache.helix.model.ExternalView;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+
+import static org.apache.pinot.common.utils.CommonConstants.Helix.BROKER_RESOURCE_INSTANCE;
+import static org.testng.Assert.assertEquals;
+import static org.testng.AssertJUnit.fail;
+
+public class ActiveTableTrackerTest {
+
+    private final String _tableName = "dummy_table_REALTIME";
+    private final String _brokerInstanceId = "broker_instance_1";
+    private ExternalView _brokerResource;
+    private ExternalView _brokerResourceEmpty;
+
+    @BeforeClass
+    public void setUp() {
+        _brokerResource = new ExternalView(BROKER_RESOURCE_INSTANCE);
+        _brokerResource.setState(_tableName, _brokerInstanceId, "ONLINE");
+        _brokerResourceEmpty = new ExternalView(BROKER_RESOURCE_INSTANCE);
+    }
+
+    @Test
+    public void testNewTableCreation() throws Exception {
+        final boolean[] success = {false};
+        ActiveTableTracker tracker = new ActiveTableTracker(new HashSet<>(), new TableListener() {
+            @Override
+            public void onNewTableCreated(String tableName) {
+                assertEquals(tableName, _tableName);
+                success[0] = true;
+            }
+
+            @Override
+            public void onTableDeleted(String tableName) {
+            }
+        });
+        List<ExternalView> externalViewList = Arrays.asList(_brokerResource);
+        tracker.onExternalViewChange(externalViewList, null);
+        if (!checkResultWithSleep(success)) {
+            fail("Did not get a callback .");
+        }
+    }
+
+    @Test
+    public void testTableDeletion() throws Exception {
+        final boolean[] success = {false};
+        ActiveTableTracker tracker = new ActiveTableTracker(new HashSet<>(), new TableListener() {
+            @Override
+            public void onNewTableCreated(String tableName) {
+                assertEquals(tableName, _tableName);
+            }
+
+            @Override
+            public void onTableDeleted(String tableName) {
+                assertEquals(tableName, _tableName);
+                success[0] = true;
+            }
+        });
+
+        // Lets first add a table
+        List<ExternalView> externalViewList = Arrays.asList(_brokerResource);
+        tracker.onExternalViewChange(externalViewList, null);
+
+        // Now send a deletion event
+        List<ExternalView> externalViewListDeletion = Arrays.asList(_brokerResourceEmpty);
+        tracker.onExternalViewChange(externalViewListDeletion, null);
+
+        if (!checkResultWithSleep(success)) {
+            fail("Did not get a callback .");
+        }
+    }
+
+    private boolean checkResultWithSleep(final boolean[] success) {
+        try {
+            for(int i=0;i<10;i++) {
+                Thread.sleep(1000);
+                if (success[0]) {
+                    return true;
+                }
+            }
+        } catch (Exception e) {}
+        return false;
+    }
+}
\ No newline at end of file
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/test/java/org/apache/pinot/grigio/common/FixedPartitionCountBytesPartitionerTest.java b/pinot-grigio/pinot-grigio-coordinator/src/test/java/org/apache/pinot/grigio/common/FixedPartitionCountBytesPartitionerTest.java
new file mode 100644
index 0000000..567f5df
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/test/java/org/apache/pinot/grigio/common/FixedPartitionCountBytesPartitionerTest.java
@@ -0,0 +1,75 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.kafka.common.Cluster;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import static org.testng.Assert.assertEquals;
+
+
+public class FixedPartitionCountBytesPartitionerTest {
+
+  private Map<String, String> configs;
+
+  @BeforeClass
+  public void setUp() {
+    configs = new HashMap<>();
+    configs.put("partition.count", "4");
+  }
+
+  @Test
+  public void testPartition() {
+    FixedPartitionCountBytesPartitioner partitioner = new FixedPartitionCountBytesPartitioner();
+    partitioner.configure(configs);
+
+    String topic1 = "test-topic1";
+    String topic2 = "test-topic2";
+    Cluster cluster = mock(Cluster.class);
+    when(cluster.partitionCountForTopic(topic1)).thenReturn(4);
+    when(cluster.partitionCountForTopic(topic2)).thenReturn(8);
+
+    String key = "test-key";
+    byte[] keyBytes = key.getBytes();
+
+    int partitionResult1 = partitioner.partition(topic1, key, keyBytes, null, null, cluster);
+    int partitionResult2 = partitioner.partition(topic2, key, keyBytes, null, null, cluster);
+    assertEquals(partitionResult1, partitionResult2);
+  }
+
+  @Test(expectedExceptions = IllegalArgumentException.class)
+  public void testPartitionFailed() {
+    FixedPartitionCountBytesPartitioner partitioner = new FixedPartitionCountBytesPartitioner();
+    partitioner.configure(configs);
+
+    String topic = "test-topic";
+    Cluster cluster = mock(Cluster.class);
+    when(cluster.partitionCountForTopic(topic)).thenReturn(2);
+
+    String key = "test-key";
+    byte[] keyBytes = key.getBytes();
+
+    partitioner.partition(topic, key, keyBytes, null, null, cluster);
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/test/java/org/apache/pinot/grigio/common/FixedPartitionCountIntPartitionerTest.java b/pinot-grigio/pinot-grigio-coordinator/src/test/java/org/apache/pinot/grigio/common/FixedPartitionCountIntPartitionerTest.java
new file mode 100644
index 0000000..6581ebb
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/test/java/org/apache/pinot/grigio/common/FixedPartitionCountIntPartitionerTest.java
@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.kafka.common.Cluster;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import static org.testng.Assert.assertEquals;
+
+
+public class FixedPartitionCountIntPartitionerTest {
+
+  private Map<String, String> configs;
+
+  @BeforeClass
+  public void setUp() {
+    configs = new HashMap<>();
+    configs.put("partition.count", "4");
+  }
+
+  @Test
+  public void testPartition() {
+    FixedPartitionCountIntPartitioner partitioner = new FixedPartitionCountIntPartitioner();
+    partitioner.configure(configs);
+
+    String topic1 = "test-topic1";
+    String topic2 = "test-topic2";
+    Cluster cluster = mock(Cluster.class);
+    when(cluster.partitionCountForTopic(topic1)).thenReturn(4);
+    when(cluster.partitionCountForTopic(topic2)).thenReturn(8);
+
+    Integer key = 6;
+
+    int partitionResult1 = partitioner.partition(topic1, key, null, null, null, cluster);
+    int partitionResult2 = partitioner.partition(topic2, key, null, null, null, cluster);
+    assertEquals(partitionResult1, 2);
+    assertEquals(partitionResult2, 2);
+  }
+
+  @Test(expectedExceptions = IllegalArgumentException.class)
+  public void testPartitionFailed() {
+    FixedPartitionCountIntPartitioner partitioner = new FixedPartitionCountIntPartitioner();
+    partitioner.configure(configs);
+
+    String topic = "test-topic";
+    Cluster cluster = mock(Cluster.class);
+    when(cluster.partitionCountForTopic(topic)).thenReturn(2);
+
+    Integer key = 6;
+
+    partitioner.partition(topic, key, null, null, null, cluster);
+  }
+}
\ No newline at end of file
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/test/java/org/apache/pinot/grigio/common/keyValueStore/RocksDBKeyValueStoreDBTest.java b/pinot-grigio/pinot-grigio-coordinator/src/test/java/org/apache/pinot/grigio/common/keyValueStore/RocksDBKeyValueStoreDBTest.java
new file mode 100644
index 0000000..8c4d33d
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/test/java/org/apache/pinot/grigio/common/keyValueStore/RocksDBKeyValueStoreDBTest.java
@@ -0,0 +1,87 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.keyValueStore;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.io.Files;
+import org.apache.commons.configuration.Configuration;
+import org.apache.commons.configuration.PropertiesConfiguration;
+import org.apache.pinot.grigio.common.messages.KeyCoordinatorMessageContext;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Map;
+
+
+public class RocksDBKeyValueStoreDBTest {
+
+  private File tempDir;
+  private Configuration configuration;
+
+  @BeforeClass
+  public void init() {
+    tempDir = Files.createTempDir();
+    tempDir.deleteOnExit();
+    configuration = new PropertiesConfiguration();
+    configuration.setProperty("rocksdb.database.dir", tempDir.getAbsolutePath());
+
+  }
+
+  @Test
+  public void testGetTable() throws IOException {
+    File table1Folder = new File(tempDir, "abc");
+    File table2Folder = new File(tempDir, "dec");
+
+    RocksDBKeyValueStoreDB db = new RocksDBKeyValueStoreDB();
+    db.init(configuration);
+    KeyValueStoreTable<ByteArrayWrapper, KeyCoordinatorMessageContext> abcTable = db.getTable("abc");
+
+    // only create the abc table but not the other
+    Assert.assertTrue(table1Folder.exists());
+    Assert.assertTrue(table1Folder.isDirectory());
+    Assert.assertFalse(table2Folder.exists());
+
+    KeyValueStoreTable<ByteArrayWrapper, KeyCoordinatorMessageContext> decTable = db.getTable("dec");
+
+    // dec table is also created
+    Assert.assertTrue(table2Folder.exists());
+    Assert.assertTrue(table2Folder.isDirectory());
+
+    // test putting & getting data from two different tables and ensure they store correct values
+    ByteArrayWrapper key = new ByteArrayWrapper(new byte[]{1, 3, 4});
+    KeyCoordinatorMessageContext value1 = new KeyCoordinatorMessageContext("a", 1, 2);
+    KeyCoordinatorMessageContext value2 = new KeyCoordinatorMessageContext("b", 2, 3);
+
+    abcTable.multiPut(ImmutableMap.of(key, value1));
+    decTable.multiPut(ImmutableMap.of(key, value2));
+
+    Map<ByteArrayWrapper, KeyCoordinatorMessageContext> abcResult = abcTable.multiGet(ImmutableList.of(key));
+    Map<ByteArrayWrapper, KeyCoordinatorMessageContext> decResult = decTable.multiGet(ImmutableList.of(key));
+
+    // make sure each table has their own values store in it and they don't interfere with each other
+    Assert.assertEquals(abcResult.size(), 1);
+    Assert.assertEquals(decResult.size(), 1);
+    Assert.assertEquals(abcResult.get(key), value1);
+    Assert.assertEquals(decResult.get(key), value2);
+  }
+}
\ No newline at end of file
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/test/java/org/apache/pinot/grigio/common/storageProvider/retentionManager/KCUpdateLogRetentionManagerImplTest.java b/pinot-grigio/pinot-grigio-coordinator/src/test/java/org/apache/pinot/grigio/common/storageProvider/retentionManager/KCUpdateLogRetentionManagerImplTest.java
new file mode 100644
index 0000000..9f9f4de
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/test/java/org/apache/pinot/grigio/common/storageProvider/retentionManager/KCUpdateLogRetentionManagerImplTest.java
@@ -0,0 +1,114 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.storageProvider.retentionManager;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.helix.NotificationContext;
+import org.apache.helix.model.IdealState;
+import org.apache.pinot.grigio.common.storageProvider.UpdateLogStorageProvider;
+import org.apache.pinot.grigio.common.utils.IdealStateHelper;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+import static org.mockito.internal.verification.VerificationModeFactory.times;
+
+public class KCUpdateLogRetentionManagerImplTest {
+  private KCUpdateLogRetentionManagerImpl kcUpdateLogRetentionManager;
+  private IdealStateHelper mockIdealStateHelper;
+  private UpdateLogStorageProvider mockUpdateLogStorageProvider;
+  private Map<String, Map<String, String>> table1SegmentMap;
+  private Map<String, Map<String, String>> table2SegmentMap;
+
+  @BeforeMethod
+  public void setUp() {
+    table1SegmentMap = new HashMap<>();
+
+    table1SegmentMap.put("table1__0__10__20191027T2041Z",
+        ImmutableMap.of("server1", "ONLINE", "server2", "ONLINE"));
+    table1SegmentMap.put("table1__0__11__20191028T2041Z",
+        ImmutableMap.of("server1", "ONLINE", "server3", "ONLINE"));
+    table1SegmentMap.put("table1__1__10__20191027T2041Z",
+        ImmutableMap.of("server1", "ONLINE", "server2", "ONLINE"));
+    table1SegmentMap.put("table1__2__10__20191027T2041Z",
+        ImmutableMap.of("server2", "ONLINE", "server3", "ONLINE"));
+
+    table2SegmentMap = new HashMap<>();
+    table2SegmentMap.put("table2__0__10__20191027T2041Z",
+        ImmutableMap.of("server1", "ONLINE", "server2", "ONLINE"));
+
+    mockIdealStateHelper = mock(IdealStateHelper.class);
+    when(mockIdealStateHelper.getSegmentToInstanceMap("table1")).thenReturn(table1SegmentMap);
+    when(mockIdealStateHelper.getSegmentToInstanceMap("table2")).thenReturn(table1SegmentMap);
+
+    mockUpdateLogStorageProvider = mock(UpdateLogStorageProvider.class);
+    kcUpdateLogRetentionManager = new KCUpdateLogRetentionManagerImpl(mockIdealStateHelper, mockUpdateLogStorageProvider,
+        "kc1");
+  }
+
+  @Test
+  public void testOnIdealStateChange() throws InterruptedException {
+    IdealState table1MockIdealState = getMockIdealState("table1_REALTIME",
+        ImmutableList.of("seg1", "host1"), ImmutableList.of("seg1", "host2"));
+
+    IdealState table2MockIdealState = getMockIdealState("table2_REALTIME",
+        ImmutableList.of("seg1", "host1"), ImmutableList.of("seg1", "host2"));
+
+    UpdateLogTableRetentionManager table1RetentionManager = kcUpdateLogRetentionManager.
+        getRetentionManagerForTable("table1_REALTIME");
+    UpdateLogTableRetentionManager table2RetentionManager = kcUpdateLogRetentionManager.
+        getRetentionManagerForTable("table2_REALTIME");
+
+    verify(mockIdealStateHelper, times(1)).getSegmentToInstanceMap("table1_REALTIME");
+    verify(mockIdealStateHelper, times(1)).getSegmentToInstanceMap("table2_REALTIME");
+
+    kcUpdateLogRetentionManager.onIdealStateChange(ImmutableList.of(table1MockIdealState), mock(NotificationContext.class));
+    verify(mockIdealStateHelper, times(2)).getSegmentToInstanceMap("table1_REALTIME");
+    verify(mockIdealStateHelper, times(1)).getSegmentToInstanceMap("table2_REALTIME");
+
+    // table1 idealstate is the same as last time
+    kcUpdateLogRetentionManager.onIdealStateChange(ImmutableList.of(table1MockIdealState, table2MockIdealState), mock(NotificationContext.class));
+    verify(mockIdealStateHelper, times(2)).getSegmentToInstanceMap("table1_REALTIME");
+    verify(mockIdealStateHelper, times(2)).getSegmentToInstanceMap("table2_REALTIME");
+
+    kcUpdateLogRetentionManager.onIdealStateChange(ImmutableList.of(
+        getMockIdealState("table1_REALTIME", ImmutableList.of("seg1", "host1"), ImmutableList.of("seg2", "host2")),
+        getMockIdealState("table2_REALTIME", ImmutableList.of("seg1", "host1"), ImmutableList.of("seg2", "host2"))
+        ), mock(NotificationContext.class));
+    verify(mockIdealStateHelper, times(3)).getSegmentToInstanceMap("table1_REALTIME");
+    verify(mockIdealStateHelper, times(3)).getSegmentToInstanceMap("table2_REALTIME");
+
+  }
+
+  private IdealState getMockIdealState(String resourceName, List<String>... partitionInstance) {
+    IdealState mockIdealState = new IdealState(resourceName);
+    mockIdealState.enable(true);
+    mockIdealState.setRebalanceMode(IdealState.RebalanceMode.CUSTOMIZED);
+    Arrays.stream(partitionInstance).forEach(partition -> mockIdealState.setPartitionState(partition.get(0), partition.get(1), "ONLINE"));
+    return mockIdealState;
+  }
+}
\ No newline at end of file
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/test/java/org/apache/pinot/grigio/common/storageProvider/retentionManager/KCUpdateLogTableRetentionManagerImplTest.java b/pinot-grigio/pinot-grigio-coordinator/src/test/java/org/apache/pinot/grigio/common/storageProvider/retentionManager/KCUpdateLogTableRetentionManagerImplTest.java
new file mode 100644
index 0000000..c36ff1f
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/test/java/org/apache/pinot/grigio/common/storageProvider/retentionManager/KCUpdateLogTableRetentionManagerImplTest.java
@@ -0,0 +1,148 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.common.storageProvider.retentionManager;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.pinot.grigio.common.storageProvider.UpdateLogStorageProvider;
+import org.apache.pinot.grigio.common.utils.IdealStateHelper;
+import org.testng.Assert;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class KCUpdateLogTableRetentionManagerImplTest {
+  private IdealStateHelper mockIdealStateHelper;
+  private UpdateLogStorageProvider mockStorageProvider;
+  private Map<String, Map<String, String>> segmentsInstanceMap;
+  private Set<String> updateLogStorageSegments;
+  private KCUpdateLogTableRetentionManagerImpl updateLogTableRetentionManager;
+
+  @BeforeMethod
+  public void setUp() throws IOException {
+    segmentsInstanceMap = new HashMap<>();
+
+    segmentsInstanceMap.put("table__0__10__20191027T2041Z",
+        ImmutableMap.of("server1", "ONLINE", "server2", "ONLINE"));
+    segmentsInstanceMap.put("table__0__11__20191028T2041Z",
+        ImmutableMap.of("server1", "ONLINE", "server3", "ONLINE"));
+    segmentsInstanceMap.put("table__1__10__20191027T2041Z",
+        ImmutableMap.of("server1", "ONLINE", "server2", "ONLINE"));
+    segmentsInstanceMap.put("table__2__10__20191027T2041Z",
+        ImmutableMap.of("server2", "ONLINE", "server3", "ONLINE"));
+
+    updateLogStorageSegments = new HashSet<>();
+    updateLogStorageSegments.add("table__0__10__20191027T2041Z");
+    updateLogStorageSegments.add("table__0__11__20191028T2041Z");
+    updateLogStorageSegments.add("table__1__10__20191027T2041Z");
+    updateLogStorageSegments.add("table__2__10__20191027T2041Z");
+
+    mockIdealStateHelper = mock(IdealStateHelper.class);
+    when(mockIdealStateHelper.getSegmentToInstanceMap("table")).thenReturn(segmentsInstanceMap);
+
+    mockStorageProvider = mock(UpdateLogStorageProvider.class);
+    when(mockStorageProvider.getAllSegments(anyString())).thenReturn(updateLogStorageSegments);
+
+  }
+
+  @Test
+  public void testInit() throws IOException {
+    updateLogStorageSegments.add("table__3__10__20191027T2041Z");
+    updateLogTableRetentionManager = new KCUpdateLogTableRetentionManagerImpl(mockIdealStateHelper, "table",
+        mockStorageProvider);
+
+    // verify that we update helix and performed deletion for expired segments
+    verify(mockStorageProvider, times(1))
+        .removeSegment("table", "table__3__10__20191027T2041Z");
+    verify(mockIdealStateHelper, times(1)).getSegmentToInstanceMap("table");
+  }
+
+  @Test
+  public void testShouldIngestForSegment() throws IOException {
+    updateLogTableRetentionManager = new KCUpdateLogTableRetentionManagerImpl(mockIdealStateHelper, "table",
+        mockStorageProvider);
+
+    // test the happy paths
+    Assert.assertTrue(updateLogTableRetentionManager.shouldIngestForSegment("table__0__10__20191027T2041Z"));
+    Assert.assertTrue(updateLogTableRetentionManager.shouldIngestForSegment("table__0__11__20191028T2041Z"));
+    Assert.assertTrue(updateLogTableRetentionManager.shouldIngestForSegment("table__1__10__20191027T2041Z"));
+    Assert.assertTrue(updateLogTableRetentionManager.shouldIngestForSegment("table__2__10__20191027T2041Z"));
+    verify(mockIdealStateHelper, times(1)).getSegmentToInstanceMap("table");
+
+    // test for segments that has lower seq
+    Assert.assertFalse(updateLogTableRetentionManager.shouldIngestForSegment("table__1__5__20191027T2041Z"));
+    verify(mockIdealStateHelper, times(1)).getSegmentToInstanceMap("table");
+
+    //test for segment that are older and not in existing segment list
+    Assert.assertFalse(updateLogTableRetentionManager.shouldIngestForSegment("table__1__20__20191027T2041Z"));
+    verify(mockIdealStateHelper, times(1)).getSegmentToInstanceMap("table");
+
+    //test for older timestamp but higher seq
+    Assert.assertFalse(updateLogTableRetentionManager.shouldIngestForSegment("table__0__20__20191027T2041Z"));
+    verify(mockIdealStateHelper, times(1)).getSegmentToInstanceMap("table");
+
+    // test for newer segments, and there is update in newer ideal state
+    segmentsInstanceMap.put("table__0__12__20191029T2041Z",
+        ImmutableMap.of("server1", "ONLINE", "server3", "ONLINE"));
+    Assert.assertTrue(updateLogTableRetentionManager.shouldIngestForSegment("table__0__12__20191029T2041Z"));
+    verify(mockIdealStateHelper, times(2)).getSegmentToInstanceMap("table");
+
+    // test for newer segments and there is no update in newer ideal state
+    Assert.assertFalse(updateLogTableRetentionManager.shouldIngestForSegment("table__0__13__20191029T2141Z"));
+    verify(mockIdealStateHelper, times(3)).getSegmentToInstanceMap("table");
+
+    // multiple attempts trying to fetch for segment should not recheck the ideal state often
+    Assert.assertFalse(updateLogTableRetentionManager.shouldIngestForSegment("table__0__13__20191029T2141Z"));
+    verify(mockIdealStateHelper, times(3)).getSegmentToInstanceMap("table");
+
+    // for unknown partition, we should also refresh data
+    Assert.assertFalse(updateLogTableRetentionManager.shouldIngestForSegment("table__2__13__20191029T2141Z"));
+    verify(mockIdealStateHelper, times(4)).getSegmentToInstanceMap("table");
+
+    Assert.assertFalse(updateLogTableRetentionManager.shouldIngestForSegment("table__3__13__20191029T2141Z"));
+    verify(mockIdealStateHelper, times(5)).getSegmentToInstanceMap("table");
+
+  }
+
+  @Test
+  public void testNotifySegmentsChange() throws IOException {
+    updateLogTableRetentionManager = new KCUpdateLogTableRetentionManagerImpl(mockIdealStateHelper, "table",
+        mockStorageProvider);
+    verify(mockIdealStateHelper, times(1)).getSegmentToInstanceMap("table");
+    updateLogTableRetentionManager.notifySegmentsChange();
+    verify(mockIdealStateHelper, times(2)).getSegmentToInstanceMap("table");
+    verify(mockStorageProvider, never()).removeSegment(anyString(), anyString());
+
+    segmentsInstanceMap.remove("table__0__10__20191027T2041Z");
+    updateLogTableRetentionManager.notifySegmentsChange();
+    verify(mockIdealStateHelper, times(3)).getSegmentToInstanceMap("table");
+    verify(mockStorageProvider, times(1)).removeSegment("table", "table__0__10__20191027T2041Z");
+  }
+}
\ No newline at end of file
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/test/java/org/apache/pinot/grigio/keyCoordinator/helix/KeyCoordinatorMessageStateModelFactoryTest.java b/pinot-grigio/pinot-grigio-coordinator/src/test/java/org/apache/pinot/grigio/keyCoordinator/helix/KeyCoordinatorMessageStateModelFactoryTest.java
new file mode 100644
index 0000000..90830f0
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/test/java/org/apache/pinot/grigio/keyCoordinator/helix/KeyCoordinatorMessageStateModelFactoryTest.java
@@ -0,0 +1,28 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.keyCoordinator.helix;
+
+import org.testng.annotations.Test;
+
+public class KeyCoordinatorMessageStateModelFactoryTest {
+
+  @Test
+  public void testGetKafkaPartitionNumberFromHelixPartition() {
+  }
+}
\ No newline at end of file
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/test/java/org/apache/pinot/grigio/keyCoordinator/internal/MessageFetcherTest.java b/pinot-grigio/pinot-grigio-coordinator/src/test/java/org/apache/pinot/grigio/keyCoordinator/internal/MessageFetcherTest.java
new file mode 100644
index 0000000..dfacd3b
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/test/java/org/apache/pinot/grigio/keyCoordinator/internal/MessageFetcherTest.java
@@ -0,0 +1,127 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.keyCoordinator.internal;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.pinot.grigio.common.OffsetInfo;
+import org.apache.pinot.grigio.common.messages.KeyCoordinatorQueueMsg;
+import org.apache.pinot.grigio.common.rpcQueue.KeyCoordinatorQueueConsumer;
+import org.apache.pinot.grigio.common.rpcQueue.QueueConsumerRecord;
+import org.apache.pinot.grigio.keyCoordinator.GrigioKeyCoordinatorMetrics;
+import org.apache.pinot.grigio.keyCoordinator.starter.KeyCoordinatorConf;
+import org.testng.Assert;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyLong;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+import static org.mockito.internal.verification.VerificationModeFactory.times;
+
+public class MessageFetcherTest {
+
+  private MessageFetcher messageFetcher;
+  private KeyCoordinatorQueueConsumer mockConsumer;
+  private GrigioKeyCoordinatorMetrics mockMetrics;
+  private BlockingQueue<QueueConsumerRecord<byte[], KeyCoordinatorQueueMsg>> consumerRecords;
+  private int invocationCount;
+
+  @BeforeMethod
+  public void init() {
+    KeyCoordinatorConf conf = new KeyCoordinatorConf();
+
+    mockConsumer = mock(KeyCoordinatorQueueConsumer.class);
+    mockMetrics = mock(GrigioKeyCoordinatorMetrics.class);
+    consumerRecords = new ArrayBlockingQueue<>(100);
+    invocationCount = 0;
+    when(mockConsumer.getRequests(anyLong(), any())).thenAnswer((invocationOnMock) -> {
+      invocationCount++;
+      List<QueueConsumerRecord<byte[], KeyCoordinatorQueueMsg>> result = new ArrayList<>();
+      consumerRecords.drainTo(result);
+      return result;
+    });
+    messageFetcher = new MessageFetcher(conf, mockConsumer, Executors.newFixedThreadPool(1), mockMetrics);
+  }
+
+  @Test
+  public void testGetMessages() throws InterruptedException {
+    List<QueueConsumerRecord<byte[], KeyCoordinatorQueueMsg>> msgList = ImmutableList.of(
+        new QueueConsumerRecord<>("topic1", 1, 123, new byte[]{123},
+          new KeyCoordinatorQueueMsg(new byte[]{123}, "segment1", 456, 900), 123),
+        new QueueConsumerRecord<>("topic1", 2, 156, new byte[]{123},
+          new KeyCoordinatorQueueMsg(new byte[]{123}, "segment2", 456, 901),123),
+        new QueueConsumerRecord<>("topic1", 1, 140, new byte[]{123},
+          new KeyCoordinatorQueueMsg(new byte[]{123}, "segment1", 470, 901), 123));
+    msgList.forEach(consumerRecords::offer);
+    messageFetcher.start();
+    // wait necessary time for ingestion loop to start and processing the data
+    // TODO: make the wait smarter so we can ensure the messages are fetched after a certain time
+    TimeUnit.MILLISECONDS.sleep(100);
+    MessageFetcher.MessageAndOffset<QueueConsumerRecord<byte[], KeyCoordinatorQueueMsg>> result = messageFetcher.getMessages(System.currentTimeMillis() + 500);
+
+    // ensure the invocation is not too much:
+    Assert.assertTrue(invocationCount < 10);
+
+    // ensure the offset are handled properly
+    Map<TopicPartition, OffsetAndMetadata> offsetMap = result.getOffsetInfo().getOffsetMap();
+    Assert.assertEquals(offsetMap.size(), 2);
+    Assert.assertEquals(offsetMap.get(new TopicPartition("topic1", 1)).offset(), 141);
+    Assert.assertEquals(offsetMap.get(new TopicPartition("topic1", 2)).offset(), 157);
+
+    // ensure the data fetched are correct
+    Assert.assertEquals(result.getMessages().size(), 3);
+    Assert.assertEquals(result.getMessages().get(0), msgList.get(0));
+    Assert.assertEquals(result.getMessages().get(1), msgList.get(1));
+    Assert.assertEquals(result.getMessages().get(2), msgList.get(2));
+
+    // test if we fetch message again
+    msgList.forEach(consumerRecords::offer);
+    TimeUnit.MILLISECONDS.sleep(100);
+    result = messageFetcher.getMessages(System.currentTimeMillis() + 100);
+    Assert.assertEquals(result.getMessages().size(), 3);
+
+    Assert.assertEquals(result.getMessages().get(0), msgList.get(0));
+    Assert.assertEquals(result.getMessages().get(1), msgList.get(1));
+    Assert.assertEquals(result.getMessages().get(2), msgList.get(2));
+  }
+
+  @Test
+  public void testAckOffset() {
+    OffsetInfo offsetInfo = new OffsetInfo(
+      (Map) ImmutableMap.of(
+          new TopicPartition("topic1", 1), 141,
+          new TopicPartition("topic1", 2), 150)
+    );
+    messageFetcher.ackOffset(new MessageFetcher.MessageAndOffset(ImmutableList.of(), offsetInfo));
+    verify(mockConsumer, times(1)).ackOffset(offsetInfo);
+  }
+}
\ No newline at end of file
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/test/java/org/apache/pinot/grigio/keyCoordinator/internal/SegmentEventProcessorTest.java b/pinot-grigio/pinot-grigio-coordinator/src/test/java/org/apache/pinot/grigio/keyCoordinator/internal/SegmentEventProcessorTest.java
new file mode 100644
index 0000000..9482617
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/test/java/org/apache/pinot/grigio/keyCoordinator/internal/SegmentEventProcessorTest.java
@@ -0,0 +1,229 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.keyCoordinator.internal;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.pinot.grigio.common.keyValueStore.ByteArrayWrapper;
+import org.apache.pinot.grigio.common.keyValueStore.KeyValueStoreDB;
+import org.apache.pinot.grigio.common.keyValueStore.KeyValueStoreTable;
+import org.apache.pinot.grigio.common.messages.KeyCoordinatorMessageContext;
+import org.apache.pinot.grigio.common.messages.KeyCoordinatorQueueMsg;
+import org.apache.pinot.grigio.common.messages.LogCoordinatorMessage;
+import org.apache.pinot.grigio.common.messages.LogEventType;
+import org.apache.pinot.grigio.common.rpcQueue.LogCoordinatorQueueProducer;
+import org.apache.pinot.grigio.common.rpcQueue.ProduceTask;
+import org.apache.pinot.grigio.common.rpcQueue.QueueConsumerRecord;
+import org.apache.pinot.grigio.common.storageProvider.UpdateLogEntry;
+import org.apache.pinot.grigio.common.storageProvider.UpdateLogStorageProvider;
+import org.apache.pinot.grigio.common.storageProvider.retentionManager.UpdateLogRetentionManager;
+import org.apache.pinot.grigio.common.storageProvider.retentionManager.UpdateLogTableRetentionManager;
+import org.apache.pinot.grigio.common.updateStrategy.MessageResolveStrategy;
+import org.apache.pinot.grigio.common.updateStrategy.MessageTimeResolveStrategy;
+import org.apache.pinot.grigio.keyCoordinator.GrigioKeyCoordinatorMetrics;
+import org.apache.pinot.grigio.keyCoordinator.starter.KeyCoordinatorConf;
+import org.testng.Assert;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.pinot.grigio.keyCoordinator.starter.KeyCoordinatorConf.KC_OUTPUT_TOPIC_PREFIX_KEY;
+import static org.mockito.ArgumentMatchers.anyInt;
+import static org.mockito.ArgumentMatchers.anyList;
+import static org.mockito.ArgumentMatchers.anyLong;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class SegmentEventProcessorTest {
+
+  private KeyValueStoreTable mockTable1;
+  private KeyValueStoreTable mockTable2;
+  private LogCoordinatorQueueProducer mockProducer;
+  private MessageResolveStrategy messageResolveStrategy;
+  private KeyValueStoreDB<ByteArrayWrapper, KeyCoordinatorMessageContext> mockDB;
+  private UpdateLogStorageProvider mockStorageProvider;
+  private UpdateLogRetentionManager mockRetentionManager;
+  private VersionMessageManager mockVersionManager;
+  private GrigioKeyCoordinatorMetrics mockMetrics;
+
+  private Map<String, List<ProduceTask>> capturedTasks;
+
+  private SegmentEventProcessor processor;
+
+  @BeforeMethod
+  public void init() throws IOException {
+    KeyCoordinatorConf conf = new KeyCoordinatorConf();
+    conf.addProperty(KC_OUTPUT_TOPIC_PREFIX_KEY, "prefix_");
+
+    // all mocks
+    mockProducer = mock(LogCoordinatorQueueProducer.class);
+    messageResolveStrategy = new MessageTimeResolveStrategy();
+    mockDB = mock(KeyValueStoreDB.class);
+    mockStorageProvider = mock(UpdateLogStorageProvider.class);
+    mockRetentionManager = mock(UpdateLogRetentionManager.class);
+    mockVersionManager = mock(VersionMessageManager.class);
+    mockMetrics = mock(GrigioKeyCoordinatorMetrics.class);
+
+    // inner mock for retentionManager
+    UpdateLogTableRetentionManager mockTableRetentionManager = mock(UpdateLogTableRetentionManager.class);
+    when(mockRetentionManager.getRetentionManagerForTable(anyString())).thenReturn(mockTableRetentionManager);
+    when(mockTableRetentionManager.shouldIngestForSegment(anyString())).thenReturn(true);
+
+    // inner mock for db
+    mockTable1 = mock(KeyValueStoreTable.class);
+    mockTable2 = mock(KeyValueStoreTable.class);
+    when(mockDB.getTable("table1")).thenReturn(mockTable1);
+    when(mockDB.getTable("table2")).thenReturn(mockTable2);
+    when(mockTable1.multiGet(anyList())).thenReturn(ImmutableMap.of());
+    when(mockTable2.multiGet(anyList())).thenReturn(
+        ImmutableMap.of(new ByteArrayWrapper(new byte[]{13}),
+            new KeyCoordinatorMessageContext("table2__0__9__20191027T2041Z", 90, 300)));
+
+    // mock version
+    Map<Integer, Long> versionMapping = new HashMap<>();
+    when(mockVersionManager.getVersionConsumed(anyInt())).thenAnswer(invocationOnMock -> {
+      int partition = invocationOnMock.getArgument(0);
+      return versionMapping.getOrDefault(partition, 9l);
+    });
+    doAnswer(invocationOnMock -> {
+      int partition = invocationOnMock.getArgument(0);
+      long version = invocationOnMock.getArgument(1);
+      versionMapping.put(partition, version);
+      return null;
+    }).when(mockVersionManager).maybeUpdateVersionConsumed(anyInt(), anyLong());
+
+
+    capturedTasks = new HashMap<>();
+    // mock producer
+    doAnswer(invocationOnMock -> {
+      List<ProduceTask> produceTasks = invocationOnMock.getArgument(0);
+      for (ProduceTask produceTask : produceTasks) {
+        produceTask.markComplete(null, null);
+        capturedTasks.computeIfAbsent(produceTask.getTopic(), t -> new ArrayList<>()).add(produceTask);
+      }
+      return null;
+    }).when(mockProducer).batchProduce(anyList());
+
+    processor = new SegmentEventProcessor(conf, mockProducer, messageResolveStrategy, mockDB, mockStorageProvider,
+        mockRetentionManager, mockVersionManager, mockMetrics);
+  }
+
+  @Test
+  public void testProcessMessages() throws IOException {
+    ImmutableList msgList = ImmutableList.copyOf(new QueueConsumerRecord[]{
+        // side effect: generate 1 insert for key 13
+        new QueueConsumerRecord<>("topic", 1, 45, new byte[]{13},
+            new KeyCoordinatorQueueMsg(new byte[]{13}, "table1__0__10__20191027T2041Z", 100, 500), 123),
+        // side effect: generate 1 insert & 1 delete for key 13
+        new QueueConsumerRecord<>("topic", 1, 45, new byte[]{13},
+            new KeyCoordinatorQueueMsg(new byte[]{13}, "table1__0__10__20191027T2041Z", 120, 600), 123),
+        // side effect: generate 1 insert for key 13 at table 2, verify different tables works
+        // also 1 delete message on existing message in kv store
+        new QueueConsumerRecord<>("topic", 1, 45, new byte[]{13},
+            new KeyCoordinatorQueueMsg(new byte[]{13}, "table2__0__10__20191027T2041Z", 120, 600), 123),
+        // side effect: version message should update versions
+        new QueueConsumerRecord<>("topic", 1, 45, new byte[]{12},
+            new KeyCoordinatorQueueMsg(10), 123),
+        // side effect: generate 1 insert for new key 14
+        new QueueConsumerRecord<>("topic", 1, 45, new byte[]{14},
+            new KeyCoordinatorQueueMsg(new byte[]{14}, "table1__0__10__20191027T2041Z", 120, 700), 123),
+        // side effect: generate no insert/delete message due to older timestamp
+        new QueueConsumerRecord<>("topic", 1, 45, new byte[]{13},
+            new KeyCoordinatorQueueMsg(new byte[]{13}, "table1__0__10__20191027T2041Z", 90, 800), 123),
+        // side effect: generate no update message as we reprocess same message
+        new QueueConsumerRecord<>("topic", 1, 45, new byte[]{13},
+            new KeyCoordinatorQueueMsg(new byte[]{13}, "table2__0__10__20191027T2041Z", 120, 600), 123),
+        // side effect: generate 1 insert & 1 delete for key 13 at table 2
+        new QueueConsumerRecord<>("topic", 1, 45, new byte[]{45},
+            new KeyCoordinatorQueueMsg(new byte[]{13}, "table2__0__11__20191027T2041Z", 140, 800), 123),
+        }
+    );
+    processor.start();
+    processor.processMessages(msgList);
+
+    // verify kafka output
+    List<ProduceTask> table1Tasks = capturedTasks.get("prefix_table1");
+    Assert.assertEquals(table1Tasks.size(), 4);
+    Assert.assertEquals(table1Tasks.get(0), new ProduceTask<>("prefix_table1", 1,
+        new LogCoordinatorMessage("table1__0__10__20191027T2041Z", 500, 9, LogEventType.INSERT)));
+    Assert.assertEquals(table1Tasks.get(1), new ProduceTask<>("prefix_table1", 1,
+        new LogCoordinatorMessage("table1__0__10__20191027T2041Z", 500, 9, LogEventType.DELETE)));
+    Assert.assertEquals(table1Tasks.get(2), new ProduceTask<>("prefix_table1", 1,
+        new LogCoordinatorMessage("table1__0__10__20191027T2041Z", 600, 9, LogEventType.INSERT)));
+    Assert.assertEquals(table1Tasks.get(3), new ProduceTask<>("prefix_table1", 1,
+        new LogCoordinatorMessage("table1__0__10__20191027T2041Z", 700, 10, LogEventType.INSERT)));
+
+    List<ProduceTask> table2Tasks = capturedTasks.get("prefix_table2");
+    Assert.assertEquals(table2Tasks.size(), 4);
+    Assert.assertEquals(table2Tasks.get(0), new ProduceTask<>("prefix_table2", 1,
+        new LogCoordinatorMessage("table2__0__9__20191027T2041Z", 300, 9, LogEventType.DELETE)));
+    Assert.assertEquals(table2Tasks.get(1), new ProduceTask<>("prefix_table2", 1,
+        new LogCoordinatorMessage("table2__0__10__20191027T2041Z", 600, 9, LogEventType.INSERT)));
+    Assert.assertEquals(table2Tasks.get(2), new ProduceTask<>("prefix_table2", 1,
+        new LogCoordinatorMessage("table2__0__10__20191027T2041Z", 600, 10, LogEventType.DELETE)));
+    Assert.assertEquals(table2Tasks.get(3), new ProduceTask<>("prefix_table2", 1,
+        new LogCoordinatorMessage("table2__0__11__20191027T2041Z", 800, 10, LogEventType.INSERT)));
+
+    // verify kv storage
+    verify(mockTable1).multiPut(ImmutableMap.of(
+        new ByteArrayWrapper(new byte[]{13}),
+        new KeyCoordinatorMessageContext("table1__0__10__20191027T2041Z", 120, 600),
+        new ByteArrayWrapper(new byte[]{14}),
+        new KeyCoordinatorMessageContext("table1__0__10__20191027T2041Z", 120, 700)
+    ));
+    verify(mockTable2).multiPut(ImmutableMap.of(
+        new ByteArrayWrapper(new byte[]{13}),
+        new KeyCoordinatorMessageContext("table2__0__11__20191027T2041Z", 140, 800)
+    ));
+
+    // verify local log storage
+    verify(mockStorageProvider).addDataToFile("table1_REALTIME", "table1__0__10__20191027T2041Z",
+        ImmutableList.of(
+            new UpdateLogEntry(500, 9, LogEventType.INSERT, 1),
+            new UpdateLogEntry(500, 9, LogEventType.DELETE, 1),
+            new UpdateLogEntry(600, 9, LogEventType.INSERT, 1),
+            new UpdateLogEntry(700, 10, LogEventType.INSERT, 1)
+        )
+    );
+    verify(mockStorageProvider).addDataToFile("table2_REALTIME", "table2__0__9__20191027T2041Z",
+        ImmutableList.of(
+            new UpdateLogEntry(300, 9, LogEventType.DELETE, 1)
+        )
+    );
+    verify(mockStorageProvider).addDataToFile("table2_REALTIME", "table2__0__10__20191027T2041Z",
+        ImmutableList.of(
+            new UpdateLogEntry(600, 9, LogEventType.INSERT, 1),
+            new UpdateLogEntry(600, 10, LogEventType.DELETE, 1)
+        )
+    );
+    verify(mockStorageProvider).addDataToFile("table2_REALTIME", "table2__0__11__20191027T2041Z",
+        ImmutableList.of(
+            new UpdateLogEntry(800, 10, LogEventType.INSERT, 1)
+        )
+    );
+  }
+}
\ No newline at end of file
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/test/java/org/apache/pinot/grigio/keyCoordinator/internal/VersionMessageManagerTest.java b/pinot-grigio/pinot-grigio-coordinator/src/test/java/org/apache/pinot/grigio/keyCoordinator/internal/VersionMessageManagerTest.java
new file mode 100644
index 0000000..2e653d0
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/test/java/org/apache/pinot/grigio/keyCoordinator/internal/VersionMessageManagerTest.java
@@ -0,0 +1,94 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.grigio.keyCoordinator.internal;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.pinot.grigio.common.rpcQueue.VersionMsgQueueProducer;
+import org.apache.pinot.grigio.keyCoordinator.GrigioKeyCoordinatorMetrics;
+import org.apache.pinot.grigio.keyCoordinator.helix.KeyCoordinatorLeadershipManager;
+import org.apache.pinot.grigio.keyCoordinator.helix.KeyCoordinatorVersionManager;
+import org.apache.pinot.grigio.keyCoordinator.starter.KeyCoordinatorConf;
+import org.testng.Assert;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import java.util.Timer;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+
+
+public class VersionMessageManagerTest {
+
+  private VersionMsgQueueProducer mockProducer;
+  private KeyCoordinatorVersionManager mockVersionManager;
+  private KeyCoordinatorLeadershipManager mockLeadershipManager;
+  private GrigioKeyCoordinatorMetrics mockMetrics;
+
+  private VersionMessageManager versionMessageManager;
+
+  @BeforeMethod
+  public void init() {
+    mockProducer = mock(VersionMsgQueueProducer.class);
+    mockVersionManager = mock(KeyCoordinatorVersionManager.class);
+    mockLeadershipManager = mock(KeyCoordinatorLeadershipManager.class);
+    mockMetrics = mock(GrigioKeyCoordinatorMetrics.class);
+
+    KeyCoordinatorConf conf = new KeyCoordinatorConf();
+    Timer timer = new Timer();
+
+    versionMessageManager = new VersionMessageManager(conf, mockProducer, timer, mockVersionManager,
+        mockLeadershipManager, mockMetrics);
+  }
+
+  @Test
+  public void testSetVersionConsumedToPropertyStore() {
+
+    versionMessageManager.setVersionConsumedToPropertyStore();
+    verify(mockVersionManager).setVersionConsumedToPropertyStore(ImmutableMap.of());
+
+    versionMessageManager.maybeUpdateVersionConsumed(1, 2l);
+    versionMessageManager.maybeUpdateVersionConsumed(2, 3l);
+    versionMessageManager.maybeUpdateVersionConsumed(2, 4l);
+    versionMessageManager.setVersionConsumedToPropertyStore();
+
+    verify(mockVersionManager).setVersionConsumedToPropertyStore(ImmutableMap.of(1, 2l, 2, 4l));
+  }
+
+  @Test
+  public void testGetAndUpdateVersionConsumed() {
+    Assert.assertEquals(versionMessageManager.getVersionConsumed(0), 0);
+
+    versionMessageManager.maybeUpdateVersionConsumed(1, 2);
+    Assert.assertEquals(versionMessageManager.getVersionConsumed(0), 0);
+    Assert.assertEquals(versionMessageManager.getVersionConsumed(1), 2);
+
+    versionMessageManager.maybeUpdateVersionConsumed(0, 1);
+    Assert.assertEquals(versionMessageManager.getVersionConsumed(0), 1);
+    Assert.assertEquals(versionMessageManager.getVersionConsumed(1), 2);
+
+    versionMessageManager.maybeUpdateVersionConsumed(0, 4);
+    Assert.assertEquals(versionMessageManager.getVersionConsumed(0), 4);
+    Assert.assertEquals(versionMessageManager.getVersionConsumed(1), 2);
+
+    versionMessageManager.maybeUpdateVersionConsumed(0, 3);
+    Assert.assertEquals(versionMessageManager.getVersionConsumed(0), 4);
+    Assert.assertEquals(versionMessageManager.getVersionConsumed(1), 2);
+  }
+}
\ No newline at end of file
diff --git a/pinot-grigio/pinot-grigio-coordinator/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/pinot-grigio/pinot-grigio-coordinator/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker
new file mode 100644
index 0000000..ca6ee9c
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-coordinator/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker
@@ -0,0 +1 @@
+mock-maker-inline
\ No newline at end of file
diff --git a/pinot-grigio/pom.xml b/pinot-grigio/pom.xml
new file mode 100644
index 0000000..e53fa72
--- /dev/null
+++ b/pinot-grigio/pom.xml
@@ -0,0 +1,42 @@
+<?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.
+
+-->
+<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>pinot</artifactId>
+    <groupId>org.apache.pinot</groupId>
+    <version>0.3.0-SNAPSHOT</version>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+  <artifactId>pinot-grigio</artifactId>
+    <packaging>pom</packaging>
+    <name>Pinot Grigio</name>
+  <url>http://maven.apache.org</url>
+    <modules>
+      <module>pinot-grigio-common</module>
+      <module>pinot-grigio-coordinator</module>
+    </modules>
+    <properties>
+    <pinot.root>${basedir}/..</pinot.root>
+  </properties>
+</project>
\ No newline at end of file
diff --git a/pom.xml b/pom.xml
index a6c187d..29e5d16 100644
--- a/pom.xml
+++ b/pom.xml
@@ -51,6 +51,7 @@
     <module>pinot-perf</module>
     <module>pinot-integration-tests</module>
     <module>pinot-distribution</module>
+    <module>pinot-grigio</module>
   </modules>
 
   <licenses>
@@ -314,6 +315,16 @@
         <version>${project.version}</version>
       </dependency>
       <dependency>
+        <groupId>org.apache.pinot</groupId>
+        <artifactId>pinot-grigio-common</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.pinot</groupId>
+        <artifactId>pinot-grigio-coordinator</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
         <groupId>nl.jqno.equalsverifier</groupId>
         <artifactId>equalsverifier</artifactId>
         <version>1.7.2</version>


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


[incubator-pinot] 04/09: basic split of logics

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jamesshao pushed a commit to branch upsert-refactor
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git

commit 63e52fe517621accc74a99d5a24e6246c8fc8cb0
Author: james Shao <sj...@uber.com>
AuthorDate: Tue Mar 17 17:46:31 2020 -0700

    basic split of logics
---
 .../pinot/broker/broker/BrokerServerBuilder.java   |  12 +-
 .../broker/broker/helix/HelixBrokerStarter.java    |  35 +++-
 .../requesthandler/BaseBrokerRequestHandler.java   |  47 ++++-
 .../requesthandler/LowWaterMarkQueryWriter.java    | 151 ++++++++++++++
 .../SingleConnectionBrokerRequestHandler.java      |   8 +-
 .../broker/upsert/DefaultLowWaterMarkService.java  |  28 ++-
 .../broker/upsert/LowWaterMarkServiceProvider.java |  39 ++--
 .../LowWaterMarkQueryWriterTest.java               | 130 ++++++++++++
 .../apache/pinot/common/utils/CommonConstants.java |   2 +
 .../helix/core/PinotHelixResourceManager.java      |  12 ++
 ...rkManager.java => DefaultWaterMarkManager.java} |  15 +-
 ...ermarkManager.java => LowWaterMarkService.java} |  21 +-
 .../segment/updater}/SegmentDeletionHandler.java   |   3 +-
 .../segment/updater}/UpsertComponentContainer.java |  19 +-
 ...WatermarkManager.java => WaterMarkManager.java} |   3 +-
 pinot-grigio/pinot-grigio-provided/pom.xml         |  38 +++-
 .../upsert/PollingBasedLowWaterMarkService.java    | 224 +++++++++++++++++++++
 .../UpsertImmutableIndexSegmentCallback.java       |   8 +-
 .../upsert/UpsertMutableIndexSegmentCallback.java  |   6 +-
 .../pinot/core/segment/updater/SegmentUpdater.java |   4 +-
 .../updater/UpsertComponentContainerImpl.java      | 151 ++++++++++++++
 ...arkManager.java => UpsertWaterMarkManager.java} |  14 +-
 .../PollingBasedLowWaterMarkServiceTest.java       | 215 ++++++++++++++++++++
 .../UpsertImmutableIndexSegmentCallbackTest.java   |   6 +-
 .../tests/ClusterIntegrationTestUtils.java         |   2 +
 .../api/resources/LowWatermarksResource.java       |  62 ++++++
 .../server/api/resources/UpsertDebugResource.java  |  84 ++++++++
 .../org/apache/pinot/server/conf/ServerConf.java   |  15 +-
 .../pinot/server/starter/ServerInstance.java       |  39 +++-
 .../starter/helix/HelixInstanceDataManager.java    |   7 -
 .../server/starter/helix/HelixServerStarter.java   |  67 ++++--
 .../SegmentOnlineOfflineStateModelFactory.java     |  14 +-
 .../upsert/DefaultUpsertComponentContainer.java    |  36 +++-
 .../upsert/UpsertComponentContainerProvider.java   |  18 +-
 .../api/resources/LowWatermarksResourceTest.java   |  23 ++-
 .../realtime/provisioning/MemoryEstimator.java     |  17 +-
 36 files changed, 1424 insertions(+), 151 deletions(-)

diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/broker/BrokerServerBuilder.java b/pinot-broker/src/main/java/org/apache/pinot/broker/broker/BrokerServerBuilder.java
index 103d469..c30dd51 100644
--- a/pinot-broker/src/main/java/org/apache/pinot/broker/broker/BrokerServerBuilder.java
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/broker/BrokerServerBuilder.java
@@ -19,8 +19,8 @@
 package org.apache.pinot.broker.broker;
 
 import com.google.common.base.Preconditions;
+import org.apache.pinot.core.segment.updater.LowWaterMarkService;
 import com.yammer.metrics.core.MetricsRegistry;
-import java.util.concurrent.atomic.AtomicReference;
 import org.apache.commons.configuration.Configuration;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.store.zk.ZkHelixPropertyStore;
@@ -36,6 +36,8 @@ import org.apache.pinot.common.utils.CommonConstants.Helix;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.concurrent.atomic.AtomicReference;
+
 
 public class BrokerServerBuilder {
   private static final Logger LOGGER = LoggerFactory.getLogger(BrokerServerBuilder.class);
@@ -57,9 +59,11 @@ public class BrokerServerBuilder {
   private final BrokerMetrics _brokerMetrics;
   private final BrokerRequestHandler _brokerRequestHandler;
   private final BrokerAdminApiApplication _brokerAdminApplication;
+  private final LowWaterMarkService _lwmService;
 
   public BrokerServerBuilder(Configuration config, RoutingTable routingTable, TimeBoundaryService timeBoundaryService,
-      QueryQuotaManager queryQuotaManager, ZkHelixPropertyStore<ZNRecord> propertyStore) {
+      QueryQuotaManager queryQuotaManager, ZkHelixPropertyStore<ZNRecord> propertyStore,
+                             LowWaterMarkService lowWaterMarkService) {
     _config = config;
     _delayedShutdownTimeMs =
         config.getLong(Broker.CONFIG_OF_DELAY_SHUTDOWN_TIME_MS, Broker.DEFAULT_DELAY_SHUTDOWN_TIME_MS);
@@ -77,8 +81,10 @@ public class BrokerServerBuilder {
     _brokerMetrics.initializeGlobalMeters();
     _brokerRequestHandler =
         new SingleConnectionBrokerRequestHandler(_config, _routingTable, _timeBoundaryService, _accessControlFactory,
-            queryQuotaManager, _brokerMetrics, _propertyStore);
+            queryQuotaManager, _brokerMetrics, _propertyStore, lowWaterMarkService);
     _brokerAdminApplication = new BrokerAdminApiApplication(this);
+
+    _lwmService = lowWaterMarkService;
   }
 
   public void start() {
diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/broker/helix/HelixBrokerStarter.java b/pinot-broker/src/main/java/org/apache/pinot/broker/broker/helix/HelixBrokerStarter.java
index 6986182..0e127ad 100644
--- a/pinot-broker/src/main/java/org/apache/pinot/broker/broker/helix/HelixBrokerStarter.java
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/broker/helix/HelixBrokerStarter.java
@@ -20,11 +20,6 @@ package org.apache.pinot.broker.broker.helix;
 
 import com.google.common.collect.ImmutableList;
 import com.yammer.metrics.core.MetricsRegistry;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import javax.annotation.Nullable;
 import org.apache.commons.configuration.BaseConfiguration;
 import org.apache.commons.configuration.Configuration;
 import org.apache.helix.ConfigAccessor;
@@ -49,6 +44,8 @@ import org.apache.pinot.broker.broker.BrokerServerBuilder;
 import org.apache.pinot.broker.queryquota.HelixExternalViewBasedQueryQuotaManager;
 import org.apache.pinot.broker.requesthandler.BrokerRequestHandler;
 import org.apache.pinot.broker.routing.HelixExternalViewBasedRouting;
+import org.apache.pinot.core.segment.updater.LowWaterMarkService;
+import org.apache.pinot.broker.upsert.LowWaterMarkServiceProvider;
 import org.apache.pinot.common.Utils;
 import org.apache.pinot.common.config.TagNameUtils;
 import org.apache.pinot.common.metadata.ZKMetadataProvider;
@@ -62,6 +59,12 @@ import org.apache.pinot.common.utils.ServiceStatus;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
 
 @SuppressWarnings("unused")
 public class HelixBrokerStarter {
@@ -93,6 +96,8 @@ public class HelixBrokerStarter {
   private HelixManager _participantHelixManager;
   private TimeboundaryRefreshMessageHandlerFactory _tbiMessageHandler;
 
+  private LowWaterMarkService _lwmService;
+
   public HelixBrokerStarter(Configuration brokerConf, String clusterName, String zkServer)
       throws Exception {
     this(brokerConf, clusterName, zkServer, null);
@@ -168,6 +173,11 @@ public class HelixBrokerStarter {
     _helixDataAccessor = _spectatorHelixManager.getHelixDataAccessor();
     ConfigAccessor configAccessor = _spectatorHelixManager.getConfigAccessor();
 
+    // start lwm service
+    LowWaterMarkServiceProvider provider = new LowWaterMarkServiceProvider(_brokerConf,
+        _spectatorHelixManager.getHelixDataAccessor(), _clusterName);
+    _lwmService = provider.getInstance();
+
     // Set up the broker server builder
     LOGGER.info("Setting up broker server builder");
     _helixExternalViewBasedRouting =
@@ -184,13 +194,17 @@ public class HelixBrokerStarter {
     String enableQueryLimitOverride = configAccessor.get(helixConfigScope, Broker.CONFIG_OF_ENABLE_QUERY_LIMIT_OVERRIDE);
     _brokerConf.setProperty(Broker.CONFIG_OF_ENABLE_QUERY_LIMIT_OVERRIDE, Boolean.valueOf(enableQueryLimitOverride));
     _brokerServerBuilder = new BrokerServerBuilder(_brokerConf, _helixExternalViewBasedRouting,
-        _helixExternalViewBasedRouting.getTimeBoundaryService(), _helixExternalViewBasedQueryQuotaManager, _propertyStore);
+        _helixExternalViewBasedRouting.getTimeBoundaryService(), _helixExternalViewBasedQueryQuotaManager,
+        _propertyStore, _lwmService);
     BrokerRequestHandler brokerRequestHandler = _brokerServerBuilder.getBrokerRequestHandler();
     BrokerMetrics brokerMetrics = _brokerServerBuilder.getBrokerMetrics();
     _helixExternalViewBasedRouting.setBrokerMetrics(brokerMetrics);
     _helixExternalViewBasedQueryQuotaManager.setBrokerMetrics(brokerMetrics);
     _brokerServerBuilder.start();
 
+    // start lwm service
+    _lwmService.start(brokerMetrics);
+
     // Initialize the cluster change mediator
     LOGGER.info("Initializing cluster change mediator");
     for (ClusterChangeHandler externalViewChangeHandler : _externalViewChangeHandlers) {
@@ -241,6 +255,7 @@ public class HelixBrokerStarter {
     _participantHelixManager
         .addPreConnectCallback(() -> brokerMetrics.addMeteredGlobalValue(BrokerMeter.HELIX_ZOOKEEPER_RECONNECTS, 1L));
 
+
     // Register the service status handler
     registerServiceStatusHandler();
 
@@ -315,6 +330,10 @@ public class HelixBrokerStarter {
       _spectatorHelixManager.disconnect();
     }
 
+    if (_lwmService != null) {
+      LOGGER.info("Shutting down low water mark service");
+      _lwmService.shutDown();
+    }
     LOGGER.info("Finish shutting down Pinot broker");
   }
 
@@ -347,6 +366,10 @@ public class HelixBrokerStarter {
     return new HelixBrokerStarter(brokerConf, "quickstart", "localhost:2122");
   }
 
+  public LowWaterMarkService getLwmService() {
+    return _lwmService;
+  }
+
   public static void main(String[] args)
       throws Exception {
     getDefault().start();
diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java
index 7667e61..1d8f97b 100644
--- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java
@@ -38,6 +38,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import javax.annotation.Nullable;
 import javax.annotation.concurrent.ThreadSafe;
+import org.apache.pinot.core.segment.updater.LowWaterMarkService;
 import org.apache.commons.configuration.Configuration;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.helix.ZNRecord;
@@ -76,6 +77,10 @@ import org.apache.pinot.pql.parsers.pql2.ast.FunctionCallAstNode;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.pinot.common.utils.CommonConstants.Broker.CONFIG_OF_BROKER_LWM_REWRITE_ENABLE;
+import static org.apache.pinot.common.utils.CommonConstants.Broker.CONFIG_OF_BROKER_LWM_REWRITE_ENABLE_DEFAULT;
+import static org.apache.pinot.common.utils.CommonConstants.Broker.Request.DISABLE_REWRITE;
+
 
 @ThreadSafe
 public abstract class BaseBrokerRequestHandler implements BrokerRequestHandler {
@@ -87,6 +92,7 @@ public abstract class BaseBrokerRequestHandler implements BrokerRequestHandler {
   protected final AccessControlFactory _accessControlFactory;
   protected final QueryQuotaManager _queryQuotaManager;
   protected final BrokerMetrics _brokerMetrics;
+  protected final LowWaterMarkService _lwmService;
 
   protected final AtomicLong _requestIdGenerator = new AtomicLong();
   protected final BrokerRequestOptimizer _brokerRequestOptimizer = new BrokerRequestOptimizer();
@@ -96,6 +102,7 @@ public abstract class BaseBrokerRequestHandler implements BrokerRequestHandler {
   protected final long _brokerTimeoutMs;
   protected final int _queryResponseLimit;
   protected final int _queryLogLength;
+  protected final boolean _enableQueryRewrite;
 
   private final RateLimiter _queryLogRateLimiter;
 
@@ -108,13 +115,15 @@ public abstract class BaseBrokerRequestHandler implements BrokerRequestHandler {
 
   public BaseBrokerRequestHandler(Configuration config, RoutingTable routingTable,
       TimeBoundaryService timeBoundaryService, AccessControlFactory accessControlFactory,
-      QueryQuotaManager queryQuotaManager, BrokerMetrics brokerMetrics, ZkHelixPropertyStore<ZNRecord> propertyStore) {
+      QueryQuotaManager queryQuotaManager, BrokerMetrics brokerMetrics, ZkHelixPropertyStore<ZNRecord> propertyStore,
+      LowWaterMarkService lowWaterMarkService) {
     _config = config;
     _routingTable = routingTable;
     _timeBoundaryService = timeBoundaryService;
     _accessControlFactory = accessControlFactory;
     _queryQuotaManager = queryQuotaManager;
     _brokerMetrics = brokerMetrics;
+    _lwmService = lowWaterMarkService;
 
     _enableCaseInsensitivePql = _config.getBoolean(CommonConstants.Helix.ENABLE_CASE_INSENSITIVE_PQL_KEY, false);
     if (_enableCaseInsensitivePql) {
@@ -125,6 +134,10 @@ public abstract class BaseBrokerRequestHandler implements BrokerRequestHandler {
 
     _enableQueryLimitOverride = _config.getBoolean(Broker.CONFIG_OF_ENABLE_QUERY_LIMIT_OVERRIDE, false);
 
+    // query rewrite for upsert feature
+    _enableQueryRewrite = config.getBoolean(CONFIG_OF_BROKER_LWM_REWRITE_ENABLE,
+        CONFIG_OF_BROKER_LWM_REWRITE_ENABLE_DEFAULT);
+
     _brokerId = config.getString(Broker.CONFIG_OF_BROKER_ID, getDefaultBrokerId());
     _brokerTimeoutMs = config.getLong(Broker.CONFIG_OF_BROKER_TIMEOUT_MS, Broker.DEFAULT_BROKER_TIMEOUT_MS);
     _queryResponseLimit =
@@ -285,6 +298,11 @@ public abstract class BaseBrokerRequestHandler implements BrokerRequestHandler {
       requestStatistics.setFanoutType(RequestStatistics.FanoutType.REALTIME);
     }
 
+    if (shouldEnableLowWaterMarkRewrite(request)) {
+      // Augment the realtime request with LowWaterMark constraints.
+      addLowWaterMarkToQuery(realtimeBrokerRequest, rawTableName);
+    }
+
     // Calculate routing table for the query
     long routingStartTimeNs = System.nanoTime();
     Map<ServerInstance, List<String>> offlineRoutingTable = null;
@@ -368,6 +386,21 @@ public abstract class BaseBrokerRequestHandler implements BrokerRequestHandler {
     return brokerResponse;
   }
 
+  private boolean shouldEnableLowWaterMarkRewrite(JsonNode request) {
+    if (_enableQueryRewrite) {
+      try {
+        if (request.has(DISABLE_REWRITE)) {
+          return !request.get(DISABLE_REWRITE).asBoolean();
+        } else {
+          return true;
+        }
+      } catch (Exception ex) {
+        LOGGER.warn("cannot parse the disable rewrite option: [{}] to boolean from request json", DISABLE_REWRITE, ex);
+      }
+    }
+    return false;
+  }
+
   /**
    * Reset limit for selection query if it exceeds maxQuerySelectionLimit.
    * @param brokerRequest
@@ -753,6 +786,18 @@ public abstract class BaseBrokerRequestHandler implements BrokerRequestHandler {
     }
   }
 
+  private void addLowWaterMarkToQuery(BrokerRequest realtimeBrokerRequest, String rawTableName) {
+    final String realtimeTableName = rawTableName + "_REALTIME";
+    Map<Integer, Long> lowWaterMarks = _lwmService.getLowWaterMarks(realtimeTableName);
+    if (lowWaterMarks == null || lowWaterMarks.size() == 0) {
+      LOGGER.info("No low water marks info found for table {}", realtimeTableName);
+      return;
+    }
+    LOGGER.info("Found low water marks {} for table {}", String.valueOf(lowWaterMarks), realtimeTableName);
+    LowWaterMarkQueryWriter.addLowWaterMarkToQuery(realtimeBrokerRequest, lowWaterMarks);
+    LOGGER.info("Query augmented with LWMS info for table {} : {}", realtimeTableName, realtimeBrokerRequest);
+  }
+
   /**
    * Processes the optimized broker requests for both OFFLINE and REALTIME table.
    */
diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/LowWaterMarkQueryWriter.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/LowWaterMarkQueryWriter.java
new file mode 100644
index 0000000..c3285e5
--- /dev/null
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/LowWaterMarkQueryWriter.java
@@ -0,0 +1,151 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.broker.requesthandler;
+
+import org.apache.pinot.common.request.BrokerRequest;
+import org.apache.pinot.common.request.FilterOperator;
+import org.apache.pinot.common.request.FilterQuery;
+import org.apache.pinot.common.request.FilterQueryMap;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+// Add a lwm query to the filter query of a Pinot query for upsert enabled table.
+// Thread-Safe
+public class LowWaterMarkQueryWriter {
+  private static final Logger LOGGER = LoggerFactory.getLogger(LowWaterMarkQueryWriter.class);
+  private static final String VALID_FROM = "$validFrom";
+  private static final String VALID_UNTIL = "$validUntil";
+  // Normal Pinot query node uses positive IDs. So lwm query node ids are all negative.
+  private static final int QUERY_ID_BASE = -1000;
+
+  /**
+   * For upsert enabled tables, augment the realtime query with low water mark constraints in its filter query of the
+   * form
+   *   ($validFrom <= lwm and $validFrom > -1) AND (lwm < $validUtil OR $validUtil = -1)
+   *
+   * @param realtimeBrokerRequest
+   * @param lowWaterMarks
+   */
+  public static void addLowWaterMarkToQuery(BrokerRequest realtimeBrokerRequest, Map<Integer, Long> lowWaterMarks) {
+    if (lowWaterMarks == null || lowWaterMarks.size() == 0) {
+      LOGGER.warn("No low water mark info found for query: {}", realtimeBrokerRequest);
+      return;
+    }
+
+    // Choose the min lwm among all partitions.
+    long minLwm = Collections.min(lowWaterMarks.values());
+
+    // 1. Build the low water mark query of the form for a table assuming lwm is the min LWM and -1 is used as
+    // uninitialized marker.
+    // ($validFrom <= lwm and $validFrom > -1) AND (lwm < $validUtil OR $validUtil = -1)
+    // -1 is used instead of Long.MAXVALUE because Pinot does not handle long arithmetic correctly.
+    FilterQuery lwmQuery = addSinglePartitionLowWaterMark(QUERY_ID_BASE - 1, realtimeBrokerRequest, minLwm);
+
+    // 2. Attach low water mark filter to the current filters.
+    FilterQuery currentFilterQuery = realtimeBrokerRequest.getFilterQuery();
+    if (currentFilterQuery != null) {
+      // Make an AND query of lwmQuery and the existing query.
+      FilterQuery andFilterQuery = new FilterQuery();
+      // Make sure we do not reuse any query id in lwmQuerys.
+      andFilterQuery.setId(QUERY_ID_BASE);
+      andFilterQuery.setOperator(FilterOperator.AND);
+      List<Integer> nestedFilterQueryIds = new ArrayList<>(2);
+      nestedFilterQueryIds.add(currentFilterQuery.getId());
+      nestedFilterQueryIds.add(lwmQuery.getId());
+      andFilterQuery.setNestedFilterQueryIds(nestedFilterQueryIds);
+
+      realtimeBrokerRequest.setFilterQuery(andFilterQuery);
+      FilterQueryMap filterSubQueryMap = realtimeBrokerRequest.getFilterSubQueryMap();
+      filterSubQueryMap.putToFilterQueryMap(lwmQuery.getId(), lwmQuery);
+      filterSubQueryMap.putToFilterQueryMap(andFilterQuery.getId(), andFilterQuery);
+    } else {
+      realtimeBrokerRequest.getFilterSubQueryMap().putToFilterQueryMap(lwmQuery.getId(), lwmQuery);
+      realtimeBrokerRequest.setFilterQuery(lwmQuery);
+    }
+  }
+
+  /**
+   *
+   * @param queryIdBase The starting id that will be assigned to the first query created in ths method.
+   * @param realtimeBrokerRequest
+   * @param lwm low water mark.
+   * @return a filter query corresponding to the low water mark constraint of a single partition. The general form is:
+   *         ($ValidFrom <= lwm && $validFrom > -1)  AND (lwm < $validUtil OR $validUtil = -1)
+   */
+  private static FilterQuery addSinglePartitionLowWaterMark(int queryIdBase, BrokerRequest realtimeBrokerRequest,
+      Long lwm) {
+    // ValidFromQuery: ($ValidFrom <= lwm && $validFrom > -1)
+    FilterQuery validFromFilterQuery = new FilterQuery();
+    // Important: Always decrement queryIdBase value after use to avoid id conflict.
+    validFromFilterQuery.setId(queryIdBase--);
+    validFromFilterQuery.setOperator(FilterOperator.AND);
+    FilterQuery validFromP1 = getLeafFilterQuery(VALID_FROM, queryIdBase--, "(*\t\t" + lwm + "]", FilterOperator.RANGE, realtimeBrokerRequest);
+    FilterQuery validFromP2 = getLeafFilterQuery(VALID_FROM, queryIdBase--, "(-1\t\t*)", FilterOperator.RANGE, realtimeBrokerRequest);
+    List<Integer> nestedQueriesIdForValidFrom = new ArrayList<>();
+    nestedQueriesIdForValidFrom.add(validFromP1.getId());
+    nestedQueriesIdForValidFrom.add(validFromP2.getId());
+    validFromFilterQuery.setNestedFilterQueryIds(nestedQueriesIdForValidFrom);
+
+    // ValidUtilQuery: (lwm < $validUtil OR $validUtil = -1)
+    FilterQuery validUtilFilterQuery = new FilterQuery();
+    validUtilFilterQuery.setId(queryIdBase--);
+    validUtilFilterQuery.setOperator(FilterOperator.OR);
+
+    FilterQuery validUtilP1 = getLeafFilterQuery(VALID_UNTIL, queryIdBase--, "(" + lwm + "\t\t*)", FilterOperator.RANGE, realtimeBrokerRequest);
+    FilterQuery validUtilP2 = getLeafFilterQuery(VALID_UNTIL, queryIdBase--, "-1", FilterOperator.EQUALITY, realtimeBrokerRequest);
+    List<Integer> nestedQueriesIdForValidUtil = new ArrayList<>();
+    nestedQueriesIdForValidUtil.add(validUtilP1.getId());
+    nestedQueriesIdForValidUtil.add(validUtilP2.getId());
+    validUtilFilterQuery.setNestedFilterQueryIds(nestedQueriesIdForValidUtil);
+
+    // Top level query: ValidFromQuery AND ValidUtilQuery
+    FilterQuery lwmQuery = new FilterQuery();
+    lwmQuery.setId(queryIdBase--);
+    lwmQuery.setOperator(FilterOperator.AND);
+    List<Integer> nestQids = new ArrayList<>();
+    nestQids.add(validFromFilterQuery.getId());
+    nestQids.add(validUtilFilterQuery.getId());
+    lwmQuery.setNestedFilterQueryIds(nestQids);
+
+    // Add all the new created queries to the query map.
+    realtimeBrokerRequest.getFilterSubQueryMap().putToFilterQueryMap(lwmQuery.getId(), lwmQuery);
+    realtimeBrokerRequest.getFilterSubQueryMap().putToFilterQueryMap(validFromFilterQuery.getId(), validFromFilterQuery);
+    realtimeBrokerRequest.getFilterSubQueryMap().putToFilterQueryMap(validUtilFilterQuery.getId(), validUtilFilterQuery);
+    return lwmQuery;
+  }
+
+  private static FilterQuery getLeafFilterQuery(String column, int id, String value, FilterOperator operator,
+      BrokerRequest realtimeBrokerRequest) {
+    FilterQuery filterQuery = new FilterQuery();
+    filterQuery.setColumn(column);
+    filterQuery.setId(id);
+    filterQuery.setValue(Collections.singletonList(value));
+    filterQuery.setOperator(operator);
+    if (realtimeBrokerRequest.getFilterSubQueryMap() == null) {
+      realtimeBrokerRequest.setFilterSubQueryMap(new FilterQueryMap());
+    }
+    realtimeBrokerRequest.getFilterSubQueryMap().putToFilterQueryMap(id, filterQuery);
+    return filterQuery;
+  }
+}
diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java
index 844a200..95f943f 100644
--- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java
@@ -23,6 +23,8 @@ import java.util.List;
 import java.util.Map;
 import javax.annotation.Nullable;
 import javax.annotation.concurrent.ThreadSafe;
+
+import org.apache.pinot.core.segment.updater.LowWaterMarkService;
 import org.apache.commons.configuration.Configuration;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.store.zk.ZkHelixPropertyStore;
@@ -57,8 +59,10 @@ public class SingleConnectionBrokerRequestHandler extends BaseBrokerRequestHandl
 
   public SingleConnectionBrokerRequestHandler(Configuration config, RoutingTable routingTable,
       TimeBoundaryService timeBoundaryService, AccessControlFactory accessControlFactory,
-      QueryQuotaManager queryQuotaManager, BrokerMetrics brokerMetrics, ZkHelixPropertyStore<ZNRecord> propertyStore) {
-    super(config, routingTable, timeBoundaryService, accessControlFactory, queryQuotaManager, brokerMetrics, propertyStore);
+      QueryQuotaManager queryQuotaManager, BrokerMetrics brokerMetrics, ZkHelixPropertyStore<ZNRecord> propertyStore,
+      LowWaterMarkService lowWaterMarkService) {
+    super(config, routingTable, timeBoundaryService, accessControlFactory, queryQuotaManager, brokerMetrics,
+            propertyStore, lowWaterMarkService);
     _queryRouter = new QueryRouter(_brokerId, brokerMetrics);
   }
 
diff --git a/pinot-server/src/main/java/org/apache/pinot/server/upsert/DefaultUpsertComponentContainer.java b/pinot-broker/src/main/java/org/apache/pinot/broker/upsert/DefaultLowWaterMarkService.java
similarity index 57%
copy from pinot-server/src/main/java/org/apache/pinot/server/upsert/DefaultUpsertComponentContainer.java
copy to pinot-broker/src/main/java/org/apache/pinot/broker/upsert/DefaultLowWaterMarkService.java
index 63424ac..94b3be4 100644
--- a/pinot-server/src/main/java/org/apache/pinot/server/upsert/DefaultUpsertComponentContainer.java
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/upsert/DefaultLowWaterMarkService.java
@@ -16,34 +16,32 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.server.upsert;
+package org.apache.pinot.broker.upsert;
 
-import com.codahale.metrics.MetricRegistry;
-import org.apache.commons.configuration.Configuration;
+import com.google.common.collect.ImmutableMap;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.core.segment.updater.LowWaterMarkService;
 
-public class DefaultUpsertComponentContainer implements UpsertComponentContainer {
+import java.util.Map;
 
-  private final SegmentDeletionHandler deletionHandler = new SegmentDeletionHandler();
+public class DefaultLowWaterMarkService implements LowWaterMarkService {
 
   @Override
-  public void registerMetrics(MetricRegistry registry) {
-
-  }
-
-  @Override
-  public void init(Configuration config) {
+  public void init(HelixDataAccessor helixDataAccessor, String helixClusterName, int serverPollingInterval,
+      int serverPort){
   }
 
   @Override
-  public SegmentDeletionHandler getSegmentDeletionHandler() {
-    return deletionHandler;
+  public Map<Integer, Long> getLowWaterMarks(String tableName) {
+    return ImmutableMap.of();
   }
 
   @Override
-  public void start() {
+  public void shutDown() {
   }
 
   @Override
-  public void stop() {
+  public void start(BrokerMetrics brokerMetrics) {
   }
 }
diff --git a/pinot-server/src/main/java/org/apache/pinot/server/upsert/UpsertComponentContainerProvider.java b/pinot-broker/src/main/java/org/apache/pinot/broker/upsert/LowWaterMarkServiceProvider.java
similarity index 50%
copy from pinot-server/src/main/java/org/apache/pinot/server/upsert/UpsertComponentContainerProvider.java
copy to pinot-broker/src/main/java/org/apache/pinot/broker/upsert/LowWaterMarkServiceProvider.java
index fe4af07..f5c06f3 100644
--- a/pinot-server/src/main/java/org/apache/pinot/server/upsert/UpsertComponentContainerProvider.java
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/upsert/LowWaterMarkServiceProvider.java
@@ -16,43 +16,48 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.server.upsert;
+package org.apache.pinot.broker.upsert;
 
 import com.google.common.base.Preconditions;
 import org.apache.commons.configuration.Configuration;
 import org.apache.commons.lang3.exception.ExceptionUtils;
-import org.apache.pinot.core.segment.updater.WatermarkManager;
-import org.apache.pinot.grigio.common.metrics.GrigioMetrics;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.pinot.common.utils.CommonConstants;
+import org.apache.pinot.core.segment.updater.LowWaterMarkService;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class UpsertComponentContainerProvider {
+import static org.apache.pinot.common.utils.CommonConstants.Broker.*;
 
-  private static final Logger LOGGER = LoggerFactory.getLogger(UpsertComponentContainerProvider.class);
 
-  public static final String UPSERT_COMPONENT_CONFIG_KEY = "watermarkManager.class";
-  public static final String UPSERT_COMPONENT_CONFIG_DEFAULT = DefaultUpsertComponentContainer.class.getName();
+public class LowWaterMarkServiceProvider {
 
-  private final Configuration _conf;
-  private UpsertComponentContainer _instance;
+  private static final Logger LOGGER = LoggerFactory.getLogger(LowWaterMarkServiceProvider.class);
 
-  public UpsertComponentContainerProvider(Configuration conf, GrigioMetrics metrics) {
-    _conf = conf;
-    String className = _conf.getString(UPSERT_COMPONENT_CONFIG_KEY, UPSERT_COMPONENT_CONFIG_DEFAULT);
+  private LowWaterMarkService _instance;
+
+  public LowWaterMarkServiceProvider(Configuration brokerConfig, HelixDataAccessor dataAccessor, String clusterName) {
+    String className = brokerConfig.getString(CommonConstants.Broker.CONFIG_OF_BROKER_LWMS_CLASS_NAME,
+        DefaultLowWaterMarkService.class.getName());
     LOGGER.info("creating watermark manager with class {}", className);
     try {
-      Class<UpsertComponentContainer> comonentContainerClass = (Class<UpsertComponentContainer>) Class.forName(className);
-      Preconditions.checkState(comonentContainerClass.isAssignableFrom(WatermarkManager.class),
-          "configured class not assignable from Callback class");
+      Class<LowWaterMarkService> comonentContainerClass = (Class<LowWaterMarkService>) Class.forName(className);
+      Preconditions.checkState(comonentContainerClass.isAssignableFrom(LowWaterMarkService.class),
+          "configured class not assignable from LowWaterMarkService class");
       _instance = comonentContainerClass.newInstance();
-      _instance.init(_conf);
+      _instance.init(dataAccessor, clusterName,
+          brokerConfig.getInt(CONFIG_OF_BROKER_POLLING_SERVER_LWMS_INTERVAL_MS,
+              DEFAULT_OF_BROKER_POLLING_SERVER_LWMS_INTERVAL_MS),
+          brokerConfig.getInt(CONFIG_OF_BROKER_POLLING_SERVER_LWMS_SERVER_PORT,
+              CommonConstants.Server.DEFAULT_ADMIN_API_PORT));
     } catch (Exception e) {
       LOGGER.error("failed to load watermark manager class", className, e);
+      _instance = null;
       ExceptionUtils.rethrow(e);
     }
   }
 
-  public UpsertComponentContainer getInstance() {
+  public LowWaterMarkService getInstance() {
     return _instance;
   }
 }
diff --git a/pinot-broker/src/test/java/org/apache/pinot/broker/requesthandler/LowWaterMarkQueryWriterTest.java b/pinot-broker/src/test/java/org/apache/pinot/broker/requesthandler/LowWaterMarkQueryWriterTest.java
new file mode 100644
index 0000000..1021477
--- /dev/null
+++ b/pinot-broker/src/test/java/org/apache/pinot/broker/requesthandler/LowWaterMarkQueryWriterTest.java
@@ -0,0 +1,130 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.broker.requesthandler;
+
+import org.apache.pinot.common.request.BrokerRequest;
+import org.apache.pinot.common.request.FilterOperator;
+import org.apache.pinot.common.request.FilterQuery;
+import org.apache.pinot.pql.parsers.Pql2Compiler;
+import org.apache.thrift.TException;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+public class LowWaterMarkQueryWriterTest {
+    @Test
+    public void testRewriteQueryWithoutExistingFilters() throws Exception{
+        Pql2Compiler pql2Compiler = new Pql2Compiler();
+        BrokerRequest req = pql2Compiler.compileToBrokerRequest("SELECT * FROM T");
+        Assert.assertFalse(req.isSetFilterQuery());
+        Map<Integer, Long> lwms = new HashMap<>();
+        lwms.put(0, 10L);
+        lwms.put(1, 20L);
+        LowWaterMarkQueryWriter.addLowWaterMarkToQuery(req, lwms);
+        Assert.assertTrue(req.isSetFilterQuery());
+        try {
+            req.validate();
+        } catch (TException e)   {
+            Assert.fail("Query after low water mark query is not valid: ", e);
+        }
+        // Verify there are in total 7 filter query nodes in the filter query tree.
+        Map<Integer,FilterQuery> filterSubQueryMap = req.getFilterSubQueryMap().getFilterQueryMap();
+        Assert.assertEquals(filterSubQueryMap.size(), 7);
+
+        Integer lwmQueryId = req.getFilterQuery().getId();
+        // 1. Verify the low water mark query.
+        FilterQuery lwmQuery = filterSubQueryMap.get(lwmQueryId);
+        verifyNoneTerminalFilterQuery(lwmQuery, FilterOperator.AND, 2);
+        FilterQuery validFrom1Query = filterSubQueryMap.get(lwmQuery.getNestedFilterQueryIds().get(0));
+        FilterQuery validTo1Query = filterSubQueryMap.get(lwmQuery.getNestedFilterQueryIds().get(1));
+
+        // Verify the subtree (i.e., an AND with two nodes) for the $validFrom column.
+        verifyNoneTerminalFilterQuery(validFrom1Query, FilterOperator.AND, 2);
+        verifyTerminalFilterQuery(filterSubQueryMap.get(validFrom1Query.getNestedFilterQueryIds().get(0)),
+            "$validFrom", "(*\t\t10]", FilterOperator.RANGE);
+        verifyTerminalFilterQuery(filterSubQueryMap.get(validFrom1Query.getNestedFilterQueryIds().get(1)),
+            "$validFrom", "(-1\t\t*)", FilterOperator.RANGE);
+
+        // Verify the subtree (i.e., an OR with two nodes) for the $validutil column.
+        verifyNoneTerminalFilterQuery(validTo1Query, FilterOperator.OR, 2);
+        verifyTerminalFilterQuery(filterSubQueryMap.get(validTo1Query.getNestedFilterQueryIds().get(0)),
+            "$validUntil", "(10\t\t*)", FilterOperator.RANGE);
+        verifyTerminalFilterQuery(filterSubQueryMap.get(validTo1Query.getNestedFilterQueryIds().get(1)),
+            "$validUntil", "-1", FilterOperator.EQUALITY);
+    }
+
+    @Test
+    public void testRewriteQueryWithExistingFilters() {
+        Pql2Compiler pql2Compiler = new Pql2Compiler();
+        BrokerRequest req = pql2Compiler.compileToBrokerRequest("SELECT * FROM T WHERE A < 4");
+        Assert.assertTrue(req.isSetFilterQuery());
+        Map<Integer, Long> lwms = new HashMap<>();
+        lwms.put(0, 10L);
+        lwms.put(1, 20L);
+        LowWaterMarkQueryWriter.addLowWaterMarkToQuery(req, lwms);
+        Assert.assertTrue(req.isSetFilterQuery());
+        try {
+            req.validate();
+        } catch (TException e) {
+            Assert.fail("Query after low water mark query is not valid: ", e);
+        }
+        // Verify there are in total 9 filter query nodes in the filter query tree.
+        Map<Integer,FilterQuery> filterSubQueryMap = req.getFilterSubQueryMap().getFilterQueryMap();
+        Assert.assertEquals(filterSubQueryMap.size(), 9);
+        // 0. Verify there are one top level filter of operator OR with two sub filter queries.
+        FilterQuery rootFilterQuery = req.getFilterQuery();
+        verifyNoneTerminalFilterQuery(rootFilterQuery, FilterOperator.AND, 2);
+        // 1. Verify the existing filter query A < 4 is not affected.
+        verifyTerminalFilterQuery(filterSubQueryMap.get(rootFilterQuery.getNestedFilterQueryIds().get(0)), "A", "(*\t\t4)", FilterOperator.RANGE);
+
+        FilterQuery lowWaterMarkQuery = filterSubQueryMap.get(rootFilterQuery.getNestedFilterQueryIds().get(1));
+        // Verify the lwm query
+        verifyNoneTerminalFilterQuery(lowWaterMarkQuery, FilterOperator.AND, 2);
+        FilterQuery validFrom1Query = filterSubQueryMap.get(lowWaterMarkQuery.getNestedFilterQueryIds().get(0));
+        FilterQuery validTo1Query = filterSubQueryMap.get(lowWaterMarkQuery.getNestedFilterQueryIds().get(1));
+
+        // Verify the subtree (i.e., an AND with two nodes) for the $validFrom column.
+        verifyNoneTerminalFilterQuery(validFrom1Query, FilterOperator.AND, 2);
+        verifyTerminalFilterQuery(filterSubQueryMap.get(validFrom1Query.getNestedFilterQueryIds().get(0)),
+            "$validFrom", "(*\t\t10]", FilterOperator.RANGE);
+        verifyTerminalFilterQuery(filterSubQueryMap.get(validFrom1Query.getNestedFilterQueryIds().get(1)),
+            "$validFrom", "(-1\t\t*)", FilterOperator.RANGE);
+
+        // Verify the subtree (i.e., an OR with two nodes) for the $validutil column.
+        verifyNoneTerminalFilterQuery(validTo1Query, FilterOperator.OR, 2);
+        verifyTerminalFilterQuery(filterSubQueryMap.get(validTo1Query.getNestedFilterQueryIds().get(0)),
+            "$validUntil", "(10\t\t*)", FilterOperator.RANGE);
+        verifyTerminalFilterQuery(filterSubQueryMap.get(validTo1Query.getNestedFilterQueryIds().get(1)),
+            "$validUntil", "-1", FilterOperator.EQUALITY);
+    }
+
+    private void verifyTerminalFilterQuery(FilterQuery filterQuery, String column, String value, FilterOperator op) {
+        Assert.assertEquals(filterQuery.getColumn(), column);
+        Assert.assertEquals(filterQuery.getValue(), Collections.singletonList(value));
+        Assert.assertEquals(filterQuery.getOperator(), op);
+    }
+
+    private void verifyNoneTerminalFilterQuery(FilterQuery filterQuery, FilterOperator op, int numOfChildQueries) {
+        Assert.assertEquals(filterQuery.getOperator(), op);
+        Assert.assertEquals(filterQuery.getNestedFilterQueryIdsSize(), numOfChildQueries);
+    }
+}
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/CommonConstants.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/CommonConstants.java
index a6f242c..0e65779 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/utils/CommonConstants.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/CommonConstants.java
@@ -175,7 +175,9 @@ public class CommonConstants {
     public static final String CONFIG_OF_ENABLE_QUERY_LIMIT_OVERRIDE = "pinot.broker.enable.query.limit.override";
 
     public static final String CONFIG_OF_BROKER_POLLING_SERVER_LWMS_INTERVAL_MS = "pinot.broker.query.polling.server.lwms.interval.ms";
+    public static final int DEFAULT_OF_BROKER_POLLING_SERVER_LWMS_INTERVAL_MS = 5 * 1_000;
     public static final String CONFIG_OF_BROKER_POLLING_SERVER_LWMS_SERVER_PORT = "pinot.broker.query.polling.server.lwms.port";
+    public static final String CONFIG_OF_BROKER_LWMS_CLASS_NAME = "pinot.broker.lwms.classname";
     public static final String CONFIG_OF_BROKER_LWM_REWRITE_ENABLE = "pinot.broker.query.lwm.rewrite";
     public static final boolean CONFIG_OF_BROKER_LWM_REWRITE_ENABLE_DEFAULT = true;
     public static class Request {
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java
index a494e3f..9cce40d 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java
@@ -112,6 +112,18 @@ import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
 
 public class PinotHelixResourceManager {
   private static final Logger LOGGER = LoggerFactory.getLogger(PinotHelixResourceManager.class);
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/WatermarkManager.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/DefaultWaterMarkManager.java
similarity index 71%
copy from pinot-core/src/main/java/org/apache/pinot/core/segment/updater/WatermarkManager.java
copy to pinot-core/src/main/java/org/apache/pinot/core/segment/updater/DefaultWaterMarkManager.java
index 01579e7..b45060c 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/WatermarkManager.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/DefaultWaterMarkManager.java
@@ -18,16 +18,23 @@
  */
 package org.apache.pinot.core.segment.updater;
 
+import com.google.common.collect.ImmutableMap;
 import org.apache.commons.configuration.Configuration;
-import org.apache.pinot.grigio.common.metrics.GrigioMeter;
 import org.apache.pinot.grigio.common.metrics.GrigioMetrics;
 
 import java.util.Map;
 
-public interface WatermarkManager {
+public class DefaultWaterMarkManager implements WaterMarkManager {
 
-  void init(Configuration config, GrigioMetrics metrics);
+  private static final Map<String, Map<Integer, Long>> DEFAULT_MAP = ImmutableMap.of();
 
-  Map<String, Map<Integer, Long>> getHighWaterMarkTablePartitionMap();
+  @Override
+  public void init(Configuration config, GrigioMetrics metrics) {
 
+  }
+
+  @Override
+  public Map<String, Map<Integer, Long>> getHighWaterMarkTablePartitionMap() {
+    return DEFAULT_MAP;
+  }
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/WatermarkManager.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/LowWaterMarkService.java
similarity index 56%
copy from pinot-core/src/main/java/org/apache/pinot/core/segment/updater/WatermarkManager.java
copy to pinot-core/src/main/java/org/apache/pinot/core/segment/updater/LowWaterMarkService.java
index 01579e7..2fd7434 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/WatermarkManager.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/LowWaterMarkService.java
@@ -18,16 +18,25 @@
  */
 package org.apache.pinot.core.segment.updater;
 
-import org.apache.commons.configuration.Configuration;
-import org.apache.pinot.grigio.common.metrics.GrigioMeter;
-import org.apache.pinot.grigio.common.metrics.GrigioMetrics;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.pinot.common.metrics.BrokerMetrics;
 
 import java.util.Map;
 
-public interface WatermarkManager {
+/**
+ * LowWaterMarkService keeps records of the low water mark (i.e., the stream ingestion progress) for each partition of
+ * an input table.
+ */
+public interface LowWaterMarkService {
+
+    void init(HelixDataAccessor helixDataAccessor, String helixClusterName, int serverPollingInterval, int serverPort);
 
-  void init(Configuration config, GrigioMetrics metrics);
+    // Return the low water mark mapping from partition id to the corresponding low water mark of a given table.
+    Map<Integer, Long> getLowWaterMarks(String tableName);
 
-  Map<String, Map<Integer, Long>> getHighWaterMarkTablePartitionMap();
+    // Shutdown the service.
+    void shutDown();
 
+    // start
+    void start(BrokerMetrics brokerMetrics);
 }
diff --git a/pinot-server/src/main/java/org/apache/pinot/server/upsert/SegmentDeletionHandler.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/SegmentDeletionHandler.java
similarity index 94%
rename from pinot-server/src/main/java/org/apache/pinot/server/upsert/SegmentDeletionHandler.java
rename to pinot-core/src/main/java/org/apache/pinot/core/segment/updater/SegmentDeletionHandler.java
index 1aba06b..1db6f84 100644
--- a/pinot-server/src/main/java/org/apache/pinot/server/upsert/SegmentDeletionHandler.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/SegmentDeletionHandler.java
@@ -16,10 +16,9 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.server.upsert;
+package org.apache.pinot.core.segment.updater;
 
 import com.google.common.collect.ImmutableList;
-import org.apache.pinot.core.segment.updater.SegmentDeletionListener;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/pinot-server/src/main/java/org/apache/pinot/server/upsert/UpsertComponentContainer.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/UpsertComponentContainer.java
similarity index 67%
rename from pinot-server/src/main/java/org/apache/pinot/server/upsert/UpsertComponentContainer.java
rename to pinot-core/src/main/java/org/apache/pinot/core/segment/updater/UpsertComponentContainer.java
index 7f636fc..eb64285 100644
--- a/pinot-server/src/main/java/org/apache/pinot/server/upsert/UpsertComponentContainer.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/UpsertComponentContainer.java
@@ -16,20 +16,27 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.server.upsert;
+package org.apache.pinot.core.segment.updater;
 
-import com.codahale.metrics.MetricRegistry;
+import com.yammer.metrics.core.MetricsRegistry;
 import org.apache.commons.configuration.Configuration;
+import org.apache.helix.HelixManager;
 
 public interface UpsertComponentContainer {
 
-  void registerMetrics(MetricRegistry registry);
+  void registerMetrics(String prefix, MetricsRegistry registry);
 
-  void init(Configuration config);
+  void init(Configuration config, HelixManager helixManager, String clusterName, String instanceName);
+
+  void startBackgroundThread();
+
+  void stopBackgroundThread();
+
+  void shutdown();
 
   SegmentDeletionHandler getSegmentDeletionHandler();
 
-  void start();
+  WaterMarkManager getWatermarkManager();
 
-  void stop();
+  boolean isUpsertEnabled();
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/WatermarkManager.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/WaterMarkManager.java
similarity index 96%
copy from pinot-core/src/main/java/org/apache/pinot/core/segment/updater/WatermarkManager.java
copy to pinot-core/src/main/java/org/apache/pinot/core/segment/updater/WaterMarkManager.java
index 01579e7..acc8479 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/WatermarkManager.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/WaterMarkManager.java
@@ -24,10 +24,11 @@ import org.apache.pinot.grigio.common.metrics.GrigioMetrics;
 
 import java.util.Map;
 
-public interface WatermarkManager {
+public interface WaterMarkManager {
 
   void init(Configuration config, GrigioMetrics metrics);
 
   Map<String, Map<Integer, Long>> getHighWaterMarkTablePartitionMap();
 
+
 }
diff --git a/pinot-grigio/pinot-grigio-provided/pom.xml b/pinot-grigio/pinot-grigio-provided/pom.xml
index 5643070..ec7b4ca 100644
--- a/pinot-grigio/pinot-grigio-provided/pom.xml
+++ b/pinot-grigio/pinot-grigio-provided/pom.xml
@@ -52,8 +52,40 @@
             <artifactId>mockito-core</artifactId>
             <scope>test</scope>
         </dependency>
-    </dependencies>
-
-
 
+        <dependency>
+            <groupId>com.github.tomakehurst</groupId>
+            <artifactId>wiremock-jre8</artifactId>
+            <scope>test</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.ow2.asm</groupId>
+                    <artifactId>asm</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>junit</groupId>
+                    <artifactId>junit</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.pinot</groupId>
+            <artifactId>pinot-broker</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.pinot</groupId>
+            <artifactId>pinot-controller</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.pinot</groupId>
+            <artifactId>pinot-controller</artifactId>
+            <version>${project.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
 </project>
\ No newline at end of file
diff --git a/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/broker/upsert/PollingBasedLowWaterMarkService.java b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/broker/upsert/PollingBasedLowWaterMarkService.java
new file mode 100644
index 0000000..f7e88c9
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/broker/upsert/PollingBasedLowWaterMarkService.java
@@ -0,0 +1,224 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.broker.upsert;
+
+import org.apache.helix.AccessOption;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.PropertyPathBuilder;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.manager.zk.ZkBaseDataAccessor;
+import org.apache.helix.manager.zk.ZkCacheBaseDataAccessor;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.pinot.common.metrics.BrokerGauge;
+import org.apache.pinot.common.metrics.BrokerMeter;
+import org.apache.pinot.common.metrics.BrokerMetrics;
+import org.apache.pinot.common.restlet.resources.TableLowWaterMarksInfo;
+import org.apache.pinot.core.segment.updater.LowWaterMarkService;
+import org.glassfish.jersey.client.ClientProperties;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.ws.rs.client.Client;
+import javax.ws.rs.client.ClientBuilder;
+import javax.ws.rs.client.WebTarget;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+// A low water mark service which polls various Pinot servers periodically to get the low water marks for partitions of
+// servers.
+public class PollingBasedLowWaterMarkService implements LowWaterMarkService {
+  private static final Logger LOGGER = LoggerFactory.getLogger(PollingBasedLowWaterMarkService.class);
+
+  private static final String LWMS_PATH = "lwms";
+  private static final String HTTP = "http";
+
+  private static final int SERVER_CONNENCT_TIMEOUT_MS = 10000;
+  private static final int SERVER_READ_TIMEOUT = 10000;
+  private static final String SERVER_PREFIX = "Server_";
+
+  // A map from table_name to its partition->lwm mapping.
+  private Map<String, Map<Integer, Long>> _tableLowWaterMarks;
+  private ZkCacheBaseDataAccessor<ZNRecord> _cacheInstanceConfigsDataAccessor;
+  private Client _httpClient;
+  // We can tune this polling interval to make sure we get the fresh snapshot of server low water marks.
+  private int _serverPollingInterval;
+  private int _serverPort;
+  private boolean _shuttingDown;
+  private BrokerMetrics _brokerMetrics;
+
+  @Override
+  public void init(HelixDataAccessor helixDataAccessor, String helixClusterName, int serverPollingInterval, int serverPort) {
+    // Construct the zk path to get the server instances.
+    String instanceConfigs = PropertyPathBuilder.instanceConfig(helixClusterName);
+    // Build a zk data reader.
+    _cacheInstanceConfigsDataAccessor =
+        new ZkCacheBaseDataAccessor<>((ZkBaseDataAccessor<ZNRecord>) helixDataAccessor.getBaseDataAccessor(),
+            instanceConfigs, null, Collections.singletonList(instanceConfigs));
+    _tableLowWaterMarks = new ConcurrentHashMap<>();
+    _httpClient = ClientBuilder.newClient();
+    _httpClient.property(ClientProperties.CONNECT_TIMEOUT, SERVER_CONNENCT_TIMEOUT_MS);
+    _httpClient.property(ClientProperties.READ_TIMEOUT, SERVER_READ_TIMEOUT);
+    _serverPollingInterval = serverPollingInterval;
+    _serverPort = serverPort;
+    Runtime.getRuntime().addShutdownHook(new Thread() {
+      @Override
+      public void run() {
+        try {
+          shutDown();
+        } catch (final Exception e) {
+          LOGGER.error("Caught exception while running shutdown hook", e);
+        }
+      }
+    });
+  }
+
+  @Override
+  public void start(BrokerMetrics brokerMetrics) {
+    _brokerMetrics = brokerMetrics;
+    Thread serverPollingThread = new Thread(new PinotServerPollingExecutor());
+    serverPollingThread.start();
+  }
+
+  @Override
+  public Map<Integer, Long> getLowWaterMarks(String tableName) {
+    return _tableLowWaterMarks.get(tableName);
+  }
+
+  @Override
+  public void shutDown() {
+    _shuttingDown = true;
+  }
+
+  // Poll all the servers periodically to find out the Low Water Mark info.
+  private class PinotServerPollingExecutor implements Runnable {
+    @Override
+    public void run() {
+      while (!_shuttingDown) {
+        try {
+          Map<String, Map<Integer, Long>> latestLowWaterMarks = new ConcurrentHashMap<>();
+          // 1. Find out all the alive servers.
+          List<String> serverInstances = _cacheInstanceConfigsDataAccessor.getChildNames("/", AccessOption.PERSISTENT);
+          List<ZNRecord> instances = _cacheInstanceConfigsDataAccessor.getChildren("/", null, AccessOption.PERSISTENT);
+          for (ZNRecord r : instances) {
+            LOGGER.info("Instance info for lwms: {}", r.toString());
+          }
+          // 2. Ask each server for its low water mark info.
+          for (String serverIntanceId : serverInstances) {
+            // Check the instance is in fact a server.
+            if (!serverIntanceId.startsWith(SERVER_PREFIX) && !serverIntanceId.startsWith("server_"))
+              continue;
+            InstanceConfig serverConfig = InstanceConfig.toInstanceConfig(serverIntanceId.substring(
+                SERVER_PREFIX.length()));
+            try {
+              // (TODO) Fixing this. Hardcode using the default server admin port for now.
+              WebTarget webTarget = _httpClient.target(getURI(serverConfig.getHostName(), _serverPort));
+              TableLowWaterMarksInfo lwms = webTarget.path(PollingBasedLowWaterMarkService.LWMS_PATH).request().
+                  get(TableLowWaterMarksInfo.class);
+              LOGGER.info("Found low water mark info for server {}: {}", serverIntanceId, lwms.getTableLowWaterMarks());
+              // 3. Update the low water marks.
+              LwmMerger.updateLowWaterMarks(latestLowWaterMarks, lwms.getTableLowWaterMarks());
+            } catch (Exception e) {
+              // TODO(tingchen) Handle server failures. We could keep the last known lwms of a server.
+              LOGGER.warn("Error during getting low water marks from server {}", serverIntanceId, e);
+            }
+          }
+          // 4. Replace the broker's low water marks table with the latest low water mark info.
+          if (validate(latestLowWaterMarks)) {
+            _tableLowWaterMarks = latestLowWaterMarks;
+          }
+          // 5. Sleep for some interval.
+          Thread.sleep(_serverPollingInterval);
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          // It is OK for us to break out the loop early because the Low Water Mark refresh is best effort.
+          break;
+        }
+      }
+    }
+
+    // Validate the low water mark info polled from all the servers are right. For now, return true.
+    // (TODO tingchen) figure out the right checks.
+    private boolean validate(Map<String, Map<Integer, Long>> latestLowWaterMarks) {
+      if (latestLowWaterMarks == null) {
+        _brokerMetrics.addMeteredGlobalValue(BrokerMeter.LOW_WATER_MARK_QUERY_FAILURES, 1);
+        return false;
+      }
+      for(String tableName : latestLowWaterMarks.keySet()) {
+        Map<Integer, Long> partitionLWMs = latestLowWaterMarks.get(tableName);
+        _brokerMetrics.addValueToTableGauge(tableName, BrokerGauge.TABLE_MIN_LOW_WATER_MARK,
+            Collections.min(partitionLWMs.values()));
+      }
+      return true;
+    }
+
+    private URI getURI(String host, int port) throws URISyntaxException {
+      LOGGER.info("requesting host {} and port {}", host, port);
+      return new URI(PollingBasedLowWaterMarkService.HTTP, null, host, port, null
+          , null, null);
+    }
+  }
+
+  static class LwmMerger {
+    // Update an existing map currentLwmsMap of tableName->low_water_marks with a new map of the same type.
+    // If an entry in the new map does not exist in currentLwmsMap, insert it to currentLwmsMap.
+    // otherwise merge the entry with the existing entry in currentLwmsMap using mergeTablePartitionLwms().
+    static void updateLowWaterMarks(Map<String, Map<Integer, Long>> currentLwmsMap,
+                                    final Map<String, Map<Integer, Long>> serverLwmsMap) {
+      for (Map.Entry<String, Map<Integer, Long>> serverMap : serverLwmsMap.entrySet()) {
+        String tableName = serverMap.getKey();
+        Map<Integer, Long> tableLwms = serverMap.getValue();
+        if (currentLwmsMap.containsKey(tableName)) {
+          currentLwmsMap.put(tableName,
+              LwmMerger.mergeTablePartitionLwms(Collections.unmodifiableMap(currentLwmsMap.get(tableName)),
+                  tableLwms));
+        } else {
+          currentLwmsMap.put(tableName, tableLwms);
+        }
+      }
+    }
+
+    // Merge all the entries in the two input maps of partition_id->lwm.
+    // If an entry exists only in a map, put it in the combined map.
+    // If an entry exists in both maps, use the entry with the smaller low water marks.
+    static Map<Integer, Long> mergeTablePartitionLwms(final Map<Integer, Long> m1, final Map<Integer, Long> m2) {
+      if (m1 == null || m1.size() == 0) {
+        return m2;
+      }
+      if (m2 == null || m2.size() == 0) {
+        return m1;
+      }
+      Map<Integer, Long> result = new HashMap<>(m1);
+      for (Map.Entry<Integer, Long> entry : m2.entrySet()) {
+        Integer partitionNo = entry.getKey();
+        Long lwm = entry.getValue();
+        if (result.containsKey(partitionNo)) {
+          result.put(partitionNo, Math.min(lwm, result.get(partitionNo)));
+        } else {
+          result.put(partitionNo, lwm);
+        }
+      }
+      return result;
+    }
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/data/manager/upsert/UpsertImmutableIndexSegmentCallback.java b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/data/manager/upsert/UpsertImmutableIndexSegmentCallback.java
index 1ddc963..b7fcdb8 100644
--- a/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/data/manager/upsert/UpsertImmutableIndexSegmentCallback.java
+++ b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/data/manager/upsert/UpsertImmutableIndexSegmentCallback.java
@@ -29,7 +29,7 @@ import org.apache.pinot.core.io.reader.BaseSingleColumnSingleValueReader;
 import org.apache.pinot.core.io.reader.DataFileReader;
 import org.apache.pinot.core.segment.index.column.ColumnIndexContainer;
 import org.apache.pinot.core.segment.index.readers.Dictionary;
-import org.apache.pinot.core.segment.updater.UpsertWatermarkManager;
+import org.apache.pinot.core.segment.updater.UpsertWaterMarkManager;
 import org.apache.pinot.core.segment.virtualcolumn.mutable.VirtualColumnLongValueReaderWriter;
 import org.apache.pinot.grigio.common.storageProvider.UpdateLogEntry;
 import org.apache.pinot.grigio.common.storageProvider.UpdateLogEntrySet;
@@ -50,7 +50,7 @@ public class UpsertImmutableIndexSegmentCallback implements IndexSegmentCallback
   private String _segmentName;
   private int _totalDoc;
   private long _minSourceOffset;
-  private UpsertWatermarkManager _upsertWatermarkManager;
+  private UpsertWaterMarkManager _upsertWatermarkManager;
   private UpdateLogStorageProvider _updateLogStorageProvider;
   // use array for mapping bewteen offset to docId, where actual offset = min_offset + array_index
   // use 4 bytes per record
@@ -67,7 +67,7 @@ public class UpsertImmutableIndexSegmentCallback implements IndexSegmentCallback
         CommonConstants.Helix.TableType.REALTIME);
     _segmentName = segmentMetadata.getName();
     _totalDoc = segmentMetadata.getTotalDocs();
-    _upsertWatermarkManager = UpsertWatermarkManager.getInstance();
+    _upsertWatermarkManager = UpsertWaterMarkManager.getInstance();
     _updateLogStorageProvider = UpdateLogStorageProvider.getInstance();
     _virtualColumnsReaderWriter = new ArrayList<>();
     for (DataFileReader reader: virtualColumnIndexReader.values()) {
@@ -80,7 +80,7 @@ public class UpsertImmutableIndexSegmentCallback implements IndexSegmentCallback
    */
   @VisibleForTesting
   protected void init(List<VirtualColumnLongValueReaderWriter> readerWriters,
-      int totalDoc, UpsertWatermarkManager manager,
+      int totalDoc, UpsertWaterMarkManager manager,
       UpdateLogStorageProvider updateLogStorageProvider,
       long minSourceOffset, int[] offsetToDocId) {
     _tableNameWithType = "testTable";
diff --git a/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/data/manager/upsert/UpsertMutableIndexSegmentCallback.java b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/data/manager/upsert/UpsertMutableIndexSegmentCallback.java
index eeda5a6..6eae5fb 100644
--- a/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/data/manager/upsert/UpsertMutableIndexSegmentCallback.java
+++ b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/data/manager/upsert/UpsertMutableIndexSegmentCallback.java
@@ -24,7 +24,7 @@ import org.apache.pinot.common.segment.SegmentMetadata;
 import org.apache.pinot.common.utils.CommonConstants;
 import org.apache.pinot.core.io.reader.DataFileReader;
 import org.apache.pinot.core.segment.index.column.ColumnIndexContainer;
-import org.apache.pinot.core.segment.updater.UpsertWatermarkManager;
+import org.apache.pinot.core.segment.updater.UpsertWaterMarkManager;
 import org.apache.pinot.core.segment.virtualcolumn.mutable.VirtualColumnLongValueReaderWriter;
 import org.apache.pinot.grigio.common.messages.LogEventType;
 import org.apache.pinot.grigio.common.storageProvider.UpdateLogEntry;
@@ -48,7 +48,7 @@ public class UpsertMutableIndexSegmentCallback implements IndexSegmentCallback {
   private String _tableName;
   private String _segmentName;
   private Schema _schema;
-  private UpsertWatermarkManager _upsertWatermarkManager;
+  private UpsertWaterMarkManager _upsertWatermarkManager;
   private String _offsetColumnName;
   private final List<VirtualColumnLongValueReaderWriter> _mutableSegmentReaderWriters = new ArrayList<>();
   // use map for mapping between kafka offset and docId because we at-most have 1 mutable segment per consumer
@@ -75,7 +75,7 @@ public class UpsertMutableIndexSegmentCallback implements IndexSegmentCallback {
         _mutableSegmentReaderWriters.add((VirtualColumnLongValueReaderWriter) reader);
       }
     }
-    _upsertWatermarkManager = UpsertWatermarkManager.getInstance();
+    _upsertWatermarkManager = UpsertWaterMarkManager.getInstance();
     LOGGER.info("starting upsert segment with {} reader writer", _mutableSegmentReaderWriters.size());
   }
 
diff --git a/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdater.java b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdater.java
index 530cd0d..5f0aa43 100644
--- a/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdater.java
+++ b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdater.java
@@ -91,7 +91,7 @@ public class SegmentUpdater implements SegmentDeletionListener {
     _topicPrefix = conf.getString(SegmentUpdaterConfig.INPUT_TOPIC_PREFIX);
     _updateSleepMs = conf.getInt(SegmentUpdaterConfig.SEGMENT_UDPATE_SLEEP_MS,
         SegmentUpdaterConfig.SEGMENT_UDPATE_SLEEP_MS_DEFAULT);
-    UpsertWatermarkManager.init(metrics);
+    UpsertWaterMarkManager.init(metrics);
     _consumer = provider.getConsumer();
     _ingestionExecutorService = Executors.newFixedThreadPool(1);
     _updateLogStorageProvider = UpdateLogStorageProvider.getInstance();
@@ -179,7 +179,7 @@ public class SegmentUpdater implements SegmentDeletionListener {
           if (System.currentTimeMillis() - lastReportedTime > LOGGER_TIME_GAP_MS) {
             lastReportedTime = System.currentTimeMillis();
             LOGGER.info("processed {} messages in {} ms", eventCount, System.currentTimeMillis() - loopStartTime);
-            LOGGER.info("latest high water mark is {}", UpsertWatermarkManager.getInstance().toString());
+            LOGGER.info("latest high water mark is {}", UpsertWaterMarkManager.getInstance().toString());
           }
           _consumer.ackOffset();
           _metrics.addTimedValueMs(GrigioTimer.SEGMENT_UPDATER_LOOP_TIME, System.currentTimeMillis() - loopStartTime);
diff --git a/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/segment/updater/UpsertComponentContainerImpl.java b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/segment/updater/UpsertComponentContainerImpl.java
new file mode 100644
index 0000000..20758f0
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/segment/updater/UpsertComponentContainerImpl.java
@@ -0,0 +1,151 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.segment.updater;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.yammer.metrics.core.MetricsRegistry;
+import org.apache.commons.configuration.Configuration;
+import org.apache.helix.HelixManager;
+import org.apache.pinot.grigio.common.metrics.GrigioMetrics;
+import org.apache.pinot.grigio.common.storageProvider.UpdateLogStorageProvider;
+import org.apache.pinot.grigio.common.storageProvider.retentionManager.UpdateLogRetentionManager;
+import org.apache.pinot.grigio.common.storageProvider.retentionManager.UpdateLogRetentionManagerImpl;
+import org.apache.pinot.grigio.common.utils.IdealStateHelper;
+import org.apache.pinot.grigio.servers.GrigioServerMetrics;
+import org.apache.pinot.grigio.servers.KeyCoordinatorProvider;
+import org.apache.pinot.grigio.servers.SegmentUpdaterProvider;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.pinot.common.utils.CommonConstants.Grigio.PINOT_UPSERT_SERVER_COMPONENT_PREFIX;
+
+public class UpsertComponentContainerImpl implements UpsertComponentContainer {
+  private static final Logger LOGGER = LoggerFactory.getLogger(UpsertComponentContainerImpl.class);
+
+  // config keys
+  public static final String ENABLED_CONFIG_KEY = "enabled";
+  public static final String STORAGE_CONFIG_KEY = "storage";
+  public static final String HOST_NAME_CONFIG_KEY = "hostname";
+  public static final String KC_CONFIG_KEY = "kc";
+  public static final String UPDATER_CONFIG_KEY = "updater";
+
+  private volatile boolean _isUpsertEnabled = false;
+  private AtomicBoolean _inited = new AtomicBoolean(false);
+
+  // members of related upsert components
+  private Configuration _conf;
+  private String _hostName;
+  private GrigioMetrics _grigioMetrics;
+  private KeyCoordinatorProvider _keyCoordinatorProvider;
+  private SegmentUpdaterProvider _segmentUpdaterProvider;
+  private SegmentUpdater _segmentUpdater;
+  private UpdateLogRetentionManager _retentionManager;
+  private SegmentDeletionHandler _segmentDeletionHandler;
+  private WaterMarkManager _waterMarkManager;
+
+  @Override
+  public void registerMetrics(String prefix, MetricsRegistry registry) {
+    _grigioMetrics = new GrigioServerMetrics(prefix + PINOT_UPSERT_SERVER_COMPONENT_PREFIX, registry);
+    _grigioMetrics.initializeGlobalMeters();
+  }
+
+  @Override
+  public void init(Configuration config, HelixManager helixManager, String clusterName, String instanceName) {
+    Preconditions.checkState(!_inited.getAndSet(true), "cannot initialize upsert component twice");
+    _isUpsertEnabled = config.getBoolean(ENABLED_CONFIG_KEY, false);
+    if (_isUpsertEnabled) {
+      LOGGER.info("initializing upsert components");
+      _conf = config;
+      _hostName = _conf.getString(HOST_NAME_CONFIG_KEY);
+      initVirtualColumnStorageProvider(config);
+      _keyCoordinatorProvider = buildKeyCoordinatorProvider(config, _hostName);
+      _segmentUpdaterProvider = buildSegmentUpdaterProvider(config, _hostName);
+      _retentionManager = new UpdateLogRetentionManagerImpl(
+          new IdealStateHelper(helixManager.getClusterManagmentTool(), clusterName), instanceName
+      );
+      _segmentUpdater = buildSegmentUpdater(config, _segmentUpdaterProvider, _retentionManager);
+      UpsertWaterMarkManager.init(_grigioMetrics);
+      _waterMarkManager = UpsertWaterMarkManager.getInstance();
+      _segmentDeletionHandler = new SegmentDeletionHandler(ImmutableList.of(_segmentUpdater));
+    } else {
+      _waterMarkManager = new DefaultWaterMarkManager();
+      _segmentDeletionHandler = new SegmentDeletionHandler();
+    }
+    _inited.set(true);
+  }
+
+  @Override
+  public SegmentDeletionHandler getSegmentDeletionHandler() {
+    Preconditions.checkState(_inited.get(), "upsert container is not initialized yet");
+    return _segmentDeletionHandler;
+  }
+
+  @Override
+  public WaterMarkManager getWatermarkManager() {
+    return _waterMarkManager;
+  }
+
+  @Override
+  public synchronized void startBackgroundThread() {
+    if (_segmentUpdater != null) {
+      _segmentUpdater.start();
+    }
+  }
+
+  @Override
+  public synchronized void stopBackgroundThread() {
+    if (_segmentUpdater != null) {
+      LOGGER.info("closing segment updater");
+      _segmentUpdater.shutdown();
+    }
+  }
+
+  @Override
+  public boolean isUpsertEnabled() {
+    return _isUpsertEnabled;
+  }
+
+  public synchronized void shutdown() {
+    if (_keyCoordinatorProvider != null) {
+      LOGGER.info("shutting down key coordinator provider");
+      _keyCoordinatorProvider.close();
+    }
+  }
+
+  private void initVirtualColumnStorageProvider(Configuration conf) {
+    UpdateLogStorageProvider.init(conf.subset(STORAGE_CONFIG_KEY));
+  }
+
+  public KeyCoordinatorProvider buildKeyCoordinatorProvider(Configuration conf, String hostname) {
+    return new KeyCoordinatorProvider(conf.subset(KC_CONFIG_KEY), hostname, _grigioMetrics);
+  }
+
+  public SegmentUpdaterProvider buildSegmentUpdaterProvider(Configuration conf, String hostname) {
+    return new SegmentUpdaterProvider(conf.subset(UPDATER_CONFIG_KEY), hostname, _grigioMetrics);
+  }
+
+  public SegmentUpdater buildSegmentUpdater(Configuration conf, SegmentUpdaterProvider updateProvider,
+                                            UpdateLogRetentionManager updateLogRetentionManager) {
+    return new SegmentUpdater(conf.subset(UPDATER_CONFIG_KEY), updateProvider, updateLogRetentionManager,
+            _grigioMetrics);
+  }
+}
diff --git a/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/segment/updater/UpsertWatermarkManager.java b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/segment/updater/UpsertWaterMarkManager.java
similarity index 90%
rename from pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/segment/updater/UpsertWatermarkManager.java
rename to pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/segment/updater/UpsertWaterMarkManager.java
index faad27b..266f25d2 100644
--- a/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/segment/updater/UpsertWatermarkManager.java
+++ b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/segment/updater/UpsertWaterMarkManager.java
@@ -31,26 +31,26 @@ import org.slf4j.LoggerFactory;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
-public class UpsertWatermarkManager implements WatermarkManager {
+public class UpsertWaterMarkManager implements WaterMarkManager {
 
   private final Map<String, Map<Integer, Long>> _highWaterMarkTablePartitionMap = new ConcurrentHashMap<>();
   private final GrigioMetrics _metrics;
 
-  private static final Logger LOGGER = LoggerFactory.getLogger(UpsertWatermarkManager.class);
-  private static volatile UpsertWatermarkManager _instance;
+  private static final Logger LOGGER = LoggerFactory.getLogger(UpsertWaterMarkManager.class);
+  private static volatile UpsertWaterMarkManager _instance;
 
-  private UpsertWatermarkManager(GrigioMetrics metrics) {
+  private UpsertWaterMarkManager(GrigioMetrics metrics) {
     _metrics = metrics;
   }
 
   public static void init(GrigioMetrics metrics) {
-    synchronized (UpsertWatermarkManager.class) {
+    synchronized (UpsertWaterMarkManager.class) {
       Preconditions.checkState(_instance == null, "upsert water mark manager is already init");
-      _instance = new UpsertWatermarkManager(metrics);
+      _instance = new UpsertWaterMarkManager(metrics);
     }
   }
 
-  public static UpsertWatermarkManager getInstance() {
+  public static UpsertWaterMarkManager getInstance() {
     Preconditions.checkState(_instance != null, "upsert water mark manager is not yet init");
     return _instance;
   }
diff --git a/pinot-grigio/pinot-grigio-provided/src/test/java/org/apache/pinot/broker/upsert/PollingBasedLowWaterMarkServiceTest.java b/pinot-grigio/pinot-grigio-provided/src/test/java/org/apache/pinot/broker/upsert/PollingBasedLowWaterMarkServiceTest.java
new file mode 100644
index 0000000..611f257
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-provided/src/test/java/org/apache/pinot/broker/upsert/PollingBasedLowWaterMarkServiceTest.java
@@ -0,0 +1,215 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.broker.upsert;
+
+import com.github.tomakehurst.wiremock.WireMockServer;
+import com.github.tomakehurst.wiremock.client.WireMock;
+import org.apache.commons.configuration.PropertiesConfiguration;
+import org.apache.helix.HelixManager;
+import org.apache.helix.HelixManagerFactory;
+import org.apache.helix.InstanceType;
+import org.apache.helix.examples.MasterSlaveStateModelFactory;
+import org.apache.helix.model.MasterSlaveSMD;
+import org.apache.helix.participant.StateMachineEngine;
+import org.apache.helix.participant.statemachine.StateModelFactory;
+import org.apache.pinot.broker.broker.helix.HelixBrokerStarter;
+import org.apache.pinot.common.config.TagNameUtils;
+import org.apache.pinot.common.restlet.resources.ResourceUtils;
+import org.apache.pinot.common.restlet.resources.TableLowWaterMarksInfo;
+import org.apache.pinot.common.utils.CommonConstants;
+import org.apache.pinot.common.utils.ZkStarter;
+import org.apache.pinot.controller.helix.ControllerTest;
+import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
+import org.apache.commons.configuration.Configuration;
+import org.apache.helix.HelixAdmin;
+import org.testng.Assert;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static com.github.tomakehurst.wiremock.client.WireMock.aResponse;
+import static com.github.tomakehurst.wiremock.client.WireMock.get;
+import static com.github.tomakehurst.wiremock.client.WireMock.getRequestedFor;
+import static com.github.tomakehurst.wiremock.client.WireMock.urlEqualTo;
+import static org.apache.pinot.common.utils.CommonConstants.Helix.UNTAGGED_BROKER_INSTANCE;
+
+public class PollingBasedLowWaterMarkServiceTest extends ControllerTest {
+  private PinotHelixResourceManager _pinotResourceManager;
+  private static final String HELIX_CLUSTER_NAME = "TestLowWaterMarksPolling";
+  private final Configuration _pinotHelixBrokerProperties = new PropertiesConfiguration();
+
+  private HelixAdmin _helixAdmin;
+  private HelixBrokerStarter _helixBrokerStarter;
+
+//  @Test
+  public void testBrokerCallServersCorrectly()
+      throws Exception {
+    ZkStarter.startLocalZkServer();
+    final String instanceId = "localhost_helixController";
+    _pinotResourceManager =
+        new PinotHelixResourceManager(ZkStarter.DEFAULT_ZK_STR, HELIX_CLUSTER_NAME, null, 10000L,
+            true, /*isUpdateStateModel=*/false, true);
+    HelixManager helixManager = registerAndConnectAsHelixParticipant(HELIX_CLUSTER_NAME, instanceId, ZkStarter.DEFAULT_ZK_STR);
+    _pinotResourceManager.start(helixManager);
+    _helixAdmin = _pinotResourceManager.getHelixAdmin();
+
+    // Set up a cluster with one controller and 2 servers.
+    addFakeBrokerInstancesToAutoJoinHelixCluster(1, true);
+    addFakeServerInstancesToAutoJoinHelixCluster(2, true);
+
+
+    _pinotHelixBrokerProperties.addProperty(CommonConstants.Helix.KEY_OF_BROKER_QUERY_PORT, 8943);
+    _pinotHelixBrokerProperties
+        .addProperty(CommonConstants.Broker.CONFIG_OF_BROKER_REFRESH_TIMEBOUNDARY_INFO_SLEEP_INTERVAL, 100L);
+
+
+    // Set the two servers' lwms info.
+    Map<Integer, Long> table1Map = new HashMap<>();
+    table1Map.put(0, 10L);
+    table1Map.put(1, 20L);
+    Map<Integer, Long> table2Map = new HashMap<>();
+    table2Map.put(0, 11L);
+    Map<String, Map<Integer, Long>> server1LwmsMap = new ConcurrentHashMap<>();
+    server1LwmsMap.put("Table1", table1Map);
+    server1LwmsMap.put("Table2", table2Map);
+
+    Map<Integer, Long> newTable1Map = new HashMap<>();
+    newTable1Map.put(0, 15L);
+    newTable1Map.put(1, 18L);
+    Map<Integer, Long> table3Map = new HashMap<>();
+    table3Map.put(0, 17L);
+    Map<String, Map<Integer, Long>> server2LwmsMap = new HashMap<>();
+    server2LwmsMap.put("Table1", newTable1Map);
+    server2LwmsMap.put("Table3", table3Map);
+
+
+    WireMockServer mockServer1 = new WireMockServer(1);
+    mockServer1.start();
+    mockServer1.stubFor(WireMock.get(WireMock.urlEqualTo("/lwms")).willReturn(WireMock.aResponse()
+        .withBody(ResourceUtils.convertToJsonString(new TableLowWaterMarksInfo(server1LwmsMap)))
+        .withHeader("Content-Type", "application/json")
+        .withStatus(200)));
+    WireMockServer mockServer2 = new WireMockServer(2);
+    mockServer2.start();
+    mockServer2.stubFor(WireMock.get(WireMock.urlEqualTo("/lwms")).willReturn(WireMock.aResponse()
+        .withBody(ResourceUtils.convertToJsonString(new TableLowWaterMarksInfo(server2LwmsMap)))
+        .withHeader("Content-Type", "application/json")
+        .withStatus(200)));
+
+    _helixBrokerStarter =
+        new HelixBrokerStarter(_pinotHelixBrokerProperties, HELIX_CLUSTER_NAME, ZkStarter.DEFAULT_ZK_STR);
+
+
+    while (_helixAdmin.getInstancesInClusterWithTag(HELIX_CLUSTER_NAME, "DefaultTenant_OFFLINE").size() == 0
+        || _helixAdmin.getInstancesInClusterWithTag(HELIX_CLUSTER_NAME, "DefaultTenant_BROKER").size() == 0) {
+      Thread.sleep(100);
+    }
+
+    Thread.sleep(1000);
+
+    // Verify the low water mark service behaviors.
+    mockServer1.verify(1, WireMock.getRequestedFor(WireMock.urlEqualTo("/lwms")));
+    mockServer2.verify(1, WireMock.getRequestedFor(WireMock.urlEqualTo("/lwms")));
+
+    Assert.assertNotNull(_helixBrokerStarter.getLwmService().getLowWaterMarks("Table1"));
+    Assert.assertNotNull(_helixBrokerStarter.getLwmService().getLowWaterMarks("Table2"));
+    Assert.assertNotNull(_helixBrokerStarter.getLwmService().getLowWaterMarks("Table3"));
+
+    // Table 1 verification.
+    Assert.assertEquals(_helixBrokerStarter.getLwmService().getLowWaterMarks("Table1").size(), 2);
+    Assert.assertTrue(_helixBrokerStarter.getLwmService().getLowWaterMarks("Table1").get(Integer.parseInt("0")) == 10L);
+    Assert.assertTrue(_helixBrokerStarter.getLwmService().getLowWaterMarks("Table1").get(Integer.parseInt("1")) == 18L);
+    // Table 1 verification.
+    Assert.assertEquals(_helixBrokerStarter.getLwmService().getLowWaterMarks("Table2").size(), 1);
+    Assert.assertTrue(_helixBrokerStarter.getLwmService().getLowWaterMarks("Table2").get(Integer.parseInt("0")) == 11L);
+    // Table 1 verification.
+    Assert.assertEquals(_helixBrokerStarter.getLwmService().getLowWaterMarks("Table3").size(), 1);
+    Assert.assertTrue(_helixBrokerStarter.getLwmService().getLowWaterMarks("Table3").get(Integer.parseInt("0")) == 17L);
+  }
+
+//  @Test
+  public void testLowWaterMarksMerge() {
+    Map<Integer, Long> table1Map = new HashMap<>();
+    table1Map.put(0, 10L);
+    table1Map.put(1, 20L);
+    Map<Integer, Long> table2Map = new HashMap<>();
+    table2Map.put(0, 11L);
+    Map<String, Map<Integer, Long>> currentLwmsMap = new ConcurrentHashMap<>();
+    currentLwmsMap.put("Table1", table1Map);
+    currentLwmsMap.put("Table2", table2Map);
+
+    Map<Integer, Long> newTable1Map = new HashMap<>();
+    newTable1Map.put(0, 15L);
+    newTable1Map.put(1, 18L);
+    Map<Integer, Long> table3Map = new HashMap<>();
+    table3Map.put(0, 17L);
+    Map<String, Map<Integer, Long>> serverLwms = new HashMap<>();
+    serverLwms.put("Table1", newTable1Map);
+    serverLwms.put("Table3", table3Map);
+
+    PollingBasedLowWaterMarkService.LwmMerger.updateLowWaterMarks(currentLwmsMap, serverLwms);
+
+    Assert.assertEquals(currentLwmsMap.size(), 3);
+
+    // Verify Table1 content.
+    Assert.assertTrue(currentLwmsMap.containsKey("Table1"));
+    Map<Integer, Long> lwmsMap1 = currentLwmsMap.get("Table1");
+    Assert.assertEquals(lwmsMap1.size(), 2);
+    // Verify that the lower LWM value is chosen in the combined results.
+    Assert.assertTrue(lwmsMap1.get(0) == 10L);
+    Assert.assertTrue(lwmsMap1.get(1) == 18L);
+
+    // Verify Table2 content.
+    Assert.assertTrue(currentLwmsMap.containsKey("Table2"));
+    Map<Integer, Long> lwmsMap2 = currentLwmsMap.get("Table2");
+    Assert.assertEquals(lwmsMap2.size(), 1);
+    // Verify that the lower LWM value is chosen in the combined results.
+    Assert.assertTrue(lwmsMap2.get(0) == 11L);
+
+    // Verify Table3 content.
+    Assert.assertTrue(currentLwmsMap.containsKey("Table3"));
+    Map<Integer, Long> lwmsMap3 = currentLwmsMap.get("Table3");
+    Assert.assertEquals(lwmsMap3.size(), 1);
+    // Verify that the lower LWM value is chosen in the combined results.
+    Assert.assertTrue(lwmsMap3.get(0) == 17L);
+  }
+
+  /**
+   * Register and connect to Helix cluster as PARTICIPANT role.
+   */
+  private HelixManager registerAndConnectAsHelixParticipant(String helixClusterName, String instanceId, String helixZkURL) {
+    HelixManager helixManager =
+            HelixManagerFactory.getZKHelixManager(helixClusterName, instanceId, InstanceType.PARTICIPANT, helixZkURL);
+
+    // Registers Master-Slave state model to state machine engine, which is for calculating participant assignment in lead controller resource.
+    helixManager.getStateMachineEngine()
+            .registerStateModelFactory(MasterSlaveSMD.name, new MasterSlaveStateModelFactory());
+
+    try {
+      helixManager.connect();
+      return helixManager;
+    } catch (Exception e) {
+      String errorMsg =
+              String.format("Exception when connecting the instance %s as Participant to Helix.", instanceId);
+      throw new RuntimeException(errorMsg);
+    }
+  }
+
+}
diff --git a/pinot-grigio/pinot-grigio-provided/src/test/java/org/apache/pinot/core/data/manager/upsert/UpsertImmutableIndexSegmentCallbackTest.java b/pinot-grigio/pinot-grigio-provided/src/test/java/org/apache/pinot/core/data/manager/upsert/UpsertImmutableIndexSegmentCallbackTest.java
index 979e5c1..e2e270d 100644
--- a/pinot-grigio/pinot-grigio-provided/src/test/java/org/apache/pinot/core/data/manager/upsert/UpsertImmutableIndexSegmentCallbackTest.java
+++ b/pinot-grigio/pinot-grigio-provided/src/test/java/org/apache/pinot/core/data/manager/upsert/UpsertImmutableIndexSegmentCallbackTest.java
@@ -18,7 +18,7 @@
  */
 package org.apache.pinot.core.data.manager.upsert;
 
-import org.apache.pinot.core.segment.updater.UpsertWatermarkManager;
+import org.apache.pinot.core.segment.updater.UpsertWaterMarkManager;
 import org.apache.pinot.core.segment.virtualcolumn.mutable.VirtualColumnLongValueReaderWriter;
 import org.apache.pinot.grigio.common.messages.LogEventType;
 import org.apache.pinot.grigio.common.storageProvider.UpdateLogEntry;
@@ -40,13 +40,13 @@ import static org.mockito.Mockito.when;
 
 public class UpsertImmutableIndexSegmentCallbackTest {
   UpdateLogStorageProvider _mockProvider;
-  UpsertWatermarkManager _mockUpsertWatermarkManager;
+  UpsertWaterMarkManager _mockUpsertWatermarkManager;
   List<VirtualColumnLongValueReaderWriter> _readerWriters = new ArrayList<>();
 
   @BeforeMethod
   public void init() {
     _mockProvider = mock(UpdateLogStorageProvider.class);
-    _mockUpsertWatermarkManager = mock(UpsertWatermarkManager.class);
+    _mockUpsertWatermarkManager = mock(UpsertWaterMarkManager.class);
   }
 
   @Test
diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/ClusterIntegrationTestUtils.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/ClusterIntegrationTestUtils.java
index 6a626cc..8f30ca8 100644
--- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/ClusterIntegrationTestUtils.java
+++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/ClusterIntegrationTestUtils.java
@@ -332,6 +332,8 @@ public class ClusterIntegrationTestUtils {
     properties.put("serializer.class", "kafka.serializer.DefaultEncoder");
     properties.put("request.required.acks", "1");
     properties.put("partitioner.class", "kafka.producer.ByteArrayPartitioner");
+    properties.put("max.request.size", "300000000");
+    properties.put("buffer.memory", "300000000");
 
     StreamDataProducer producer =
         StreamDataProvider.getStreamDataProducer(KafkaStarterUtils.KAFKA_PRODUCER_CLASS_NAME, properties);
diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/LowWatermarksResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/LowWatermarksResource.java
new file mode 100644
index 0000000..b346066
--- /dev/null
+++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/LowWatermarksResource.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.server.api.resources;
+
+import org.apache.pinot.common.restlet.resources.ResourceUtils;
+import org.apache.pinot.common.restlet.resources.TableLowWaterMarksInfo;
+import org.apache.pinot.core.segment.updater.WaterMarkManager;
+import org.apache.pinot.server.starter.ServerInstance;
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiOperation;
+import io.swagger.annotations.ApiResponse;
+import io.swagger.annotations.ApiResponses;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+
+@Api(tags = "LowWaterMarks")
+@Path("/")
+public class LowWatermarksResource {
+
+  @Inject
+  ServerInstance serverInstance;
+
+  @GET
+  @Produces(MediaType.APPLICATION_JSON)
+  @Path("/lwms")
+  @ApiOperation(value = "Show the lwms of tables ", notes = "Returns the lwms of all upsert enable tables in this server")
+  @ApiResponses(value = {
+      @ApiResponse(code = 200, message = "Success"),
+      @ApiResponse(code = 500, message = "Internal server error"),
+  })
+  public String getLowWaterMarks() {
+    WaterMarkManager watermarkManager = serverInstance.getWatermarkManager();
+
+    if (watermarkManager == null) {
+      throw new WebApplicationException("Invalid server initialization", Response.Status.INTERNAL_SERVER_ERROR);
+    }
+    return ResourceUtils.convertToJsonString(
+        new TableLowWaterMarksInfo(watermarkManager.getHighWaterMarkTablePartitionMap()));
+  }
+}
diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/UpsertDebugResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/UpsertDebugResource.java
new file mode 100644
index 0000000..13a7612
--- /dev/null
+++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/UpsertDebugResource.java
@@ -0,0 +1,84 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.server.api.resources;
+
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiOperation;
+import io.swagger.annotations.ApiParam;
+import io.swagger.annotations.ApiResponse;
+import io.swagger.annotations.ApiResponses;
+import org.apache.pinot.core.data.manager.InstanceDataManager;
+import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.manager.TableDataManager;
+import org.apache.pinot.core.data.manager.UpsertSegmentDataManager;
+import org.apache.pinot.server.starter.ServerInstance;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.MediaType;
+
+@Api(tags = "UpsertDebug")
+@Path("/")
+public class UpsertDebugResource {
+
+  @Inject
+  ServerInstance serverInstance;
+
+  @GET
+  @Produces(MediaType.TEXT_PLAIN)
+  @Path("/upsert/{tableName}/{segmentName}/{offset}")
+  @ApiOperation(value = "$validFrom and $validUntil value", notes = "")
+  @ApiResponses(value = {
+      @ApiResponse(code = 200, message = "Success", response = String.class),
+      @ApiResponse(code = 500, message = "Internal server error"),
+  })
+  public String getUpsertDataAtOffset(
+      @ApiParam(value = "Table name including type", required = true, example = "myTable_REALTIME") @PathParam("tableName") String tableName,
+      @ApiParam(value = "segment name", required = true, example = "eats_supply_update__0__0__20190923T0700Z") @PathParam("segmentName") String segmentName,
+      @ApiParam(value = "offset", required = true, example = "100") @PathParam("offset") String offsetStr
+  ) {
+    if (!serverInstance.isUpsertEnabled()) {
+      return "not an upsert server";
+    }
+    InstanceDataManager instanceDataManager = serverInstance.getInstanceDataManager();
+    TableDataManager tableDataManager = instanceDataManager.getTableDataManager(tableName);
+    if (tableDataManager == null) {
+      return "no table for " + tableName;
+    }
+    SegmentDataManager segmentDataManager = null;
+    try {
+      segmentDataManager = tableDataManager.acquireSegment(segmentName);
+      if (segmentDataManager == null) {
+        return "cannot find associate segment for segment " + segmentName;
+      }
+      if (!(segmentDataManager instanceof UpsertSegmentDataManager)) {
+        return "it is not an upsert table";
+      } else {
+        return ((UpsertSegmentDataManager) segmentDataManager).getVirtualColumnInfo(Long.parseLong(offsetStr));
+      }
+    } finally {
+      if (segmentDataManager != null) {
+        tableDataManager.releaseSegment(segmentDataManager);
+      }
+    }
+  }
+}
diff --git a/pinot-server/src/main/java/org/apache/pinot/server/conf/ServerConf.java b/pinot-server/src/main/java/org/apache/pinot/server/conf/ServerConf.java
index 5610a4a..d2adcb3 100644
--- a/pinot-server/src/main/java/org/apache/pinot/server/conf/ServerConf.java
+++ b/pinot-server/src/main/java/org/apache/pinot/server/conf/ServerConf.java
@@ -20,8 +20,8 @@ package org.apache.pinot.server.conf;
 
 import org.apache.commons.configuration.Configuration;
 import org.apache.commons.configuration.ConfigurationException;
-import org.apache.pinot.common.metrics.ServerMetrics;
 import org.apache.pinot.common.utils.CommonConstants;
+import org.apache.pinot.server.upsert.DefaultUpsertComponentContainer;
 
 
 /**
@@ -45,6 +45,11 @@ public class ServerConf {
 
   private static final String PINOT_QUERY_SCHEDULER_PREFIX = "pinot.query.scheduler";
 
+  public static final String UPSERT_CONFIG_PARENT = "pinot.server.upsert";
+  public static final String UPSERT_COMPONENT_CONFIG_KEY = "pinot.server.upsertComponent.class";
+  public static final String UPSERT_COMPONENT_CONFIG_DEFAULT = DefaultUpsertComponentContainer.class.getName();
+
+
   private Configuration _serverConf;
 
   public ServerConf(Configuration serverConfig) {
@@ -92,6 +97,14 @@ public class ServerConf {
     return _serverConf.subset(PINOT_QUERY_SCHEDULER_PREFIX);
   }
 
+  public String getUpsertComponentContainerClassName() {
+    return _serverConf.getString(UPSERT_COMPONENT_CONFIG_KEY, UPSERT_COMPONENT_CONFIG_DEFAULT);
+  }
+
+  public Configuration getUpsertConfig() {
+    return _serverConf.subset(UPSERT_CONFIG_PARENT);
+  }
+
   /**
    * Returns an array of transform function names as defined in the config
    * @return String array of transform functions
diff --git a/pinot-server/src/main/java/org/apache/pinot/server/starter/ServerInstance.java b/pinot-server/src/main/java/org/apache/pinot/server/starter/ServerInstance.java
index 11e2326..845a90b 100644
--- a/pinot-server/src/main/java/org/apache/pinot/server/starter/ServerInstance.java
+++ b/pinot-server/src/main/java/org/apache/pinot/server/starter/ServerInstance.java
@@ -26,14 +26,18 @@ import java.util.concurrent.atomic.LongAccumulator;
 import org.apache.helix.HelixManager;
 import org.apache.pinot.common.metrics.MetricsHelper;
 import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.core.segment.updater.SegmentDeletionHandler;
 import org.apache.pinot.core.data.manager.InstanceDataManager;
 import org.apache.pinot.core.operator.transform.function.TransformFunction;
 import org.apache.pinot.core.operator.transform.function.TransformFunctionFactory;
 import org.apache.pinot.core.query.executor.QueryExecutor;
 import org.apache.pinot.core.query.scheduler.QueryScheduler;
 import org.apache.pinot.core.query.scheduler.QuerySchedulerFactory;
+import org.apache.pinot.core.segment.updater.WaterMarkManager;
 import org.apache.pinot.core.transport.QueryServer;
 import org.apache.pinot.server.conf.ServerConf;
+import org.apache.pinot.core.segment.updater.UpsertComponentContainer;
+import org.apache.pinot.server.upsert.UpsertComponentContainerProvider;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -45,6 +49,7 @@ import org.slf4j.LoggerFactory;
 public class ServerInstance {
   private static final Logger LOGGER = LoggerFactory.getLogger(ServerInstance.class);
 
+  private final ServerConf _serverConf;
   private final ServerMetrics _serverMetrics;
   private final InstanceDataManager _instanceDataManager;
   private final QueryExecutor _queryExecutor;
@@ -52,12 +57,16 @@ public class ServerInstance {
   private final QueryScheduler _queryScheduler;
   private final QueryServer _queryServer;
 
+  // upsert related component, only initialize if necessary
+  private UpsertComponentContainer _upsertComponentContainer;
+
   private boolean _started = false;
 
-  public ServerInstance(ServerConf serverConf, HelixManager helixManager)
+  public ServerInstance(ServerConf serverConf, HelixManager helixManager, String clusterName, String instanceName)
       throws Exception {
     LOGGER.info("Initializing server instance");
 
+    _serverConf = serverConf;
     LOGGER.info("Initializing server metrics");
     MetricsHelper.initializeMetrics(serverConf.getMetricsConfig());
     MetricsRegistry metricsRegistry = new MetricsRegistry();
@@ -97,12 +106,19 @@ public class ServerInstance {
     }
     TransformFunctionFactory.init(transformFunctionClasses);
 
+
+    final UpsertComponentContainerProvider upsertComponentContainerProvider = new UpsertComponentContainerProvider(serverConf);
+    _upsertComponentContainer = upsertComponentContainerProvider.getInstance();
+    _upsertComponentContainer.registerMetrics(_serverConf.getMetricsPrefix(), metricsRegistry);
+    _upsertComponentContainer.init(_serverConf.getUpsertConfig(), helixManager, clusterName, instanceName);
+
     LOGGER.info("Finish initializing server instance");
   }
 
   public synchronized void start() {
     // This method is called when Helix starts a new ZK session, and can be called multiple times. We only need to start
     // the server instance once, and simply ignore the following invocations.
+    LOGGER.info("Starting server instance");
     if (_started) {
       LOGGER.info("Server instance is already running, skipping the start");
       return;
@@ -127,12 +143,16 @@ public class ServerInstance {
     Preconditions.checkState(_started, "Server instance is not running");
     LOGGER.info("Shutting down server instance");
 
+    _upsertComponentContainer.stopBackgroundThread();
     LOGGER.info("Shutting down query server");
     _queryServer.shutDown();
     LOGGER.info("Shutting down query scheduler");
     _queryScheduler.stop();
     LOGGER.info("Shutting down query executor");
     _queryExecutor.shutDown();
+    LOGGER.info("Shutting down upsert components if necessary");
+    _upsertComponentContainer.shutdown();
+
     LOGGER.info("Shutting down instance data manager");
     _instanceDataManager.shutDown();
 
@@ -148,6 +168,23 @@ public class ServerInstance {
     return _instanceDataManager;
   }
 
+  public SegmentDeletionHandler getSegmentDeletionHandler() {
+    return _upsertComponentContainer.getSegmentDeletionHandler();
+  }
+
+  public WaterMarkManager getWatermarkManager() {
+    return _upsertComponentContainer.getWatermarkManager();
+  }
+
+  public void maybeStartUpsertBackgroundThread() {
+    LOGGER.info("starting upsert component background thread");
+    _upsertComponentContainer.startBackgroundThread();
+  }
+
+  public boolean isUpsertEnabled() {
+    return _upsertComponentContainer.isUpsertEnabled();
+  }
+
   public long getLatestQueryTime() {
     return _latestQueryTime.get();
   }
diff --git a/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixInstanceDataManager.java b/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixInstanceDataManager.java
index 920b422..710e22a 100644
--- a/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixInstanceDataManager.java
+++ b/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixInstanceDataManager.java
@@ -307,13 +307,6 @@ public class HelixInstanceDataManager implements InstanceDataManager {
     }
   }
 
-  /*
-  @Override
-  public Map<String, Map<Integer, Long>> getLowWaterMarks() {
-    return UpsertWaterMarkManager.getInstance().getHighWaterMarkTablePartitionMap();
-  }
-   */
-
   @Override
   public String getSegmentDataDirectory() {
     return _instanceDataManagerConfig.getInstanceDataDir();
diff --git a/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixServerStarter.java b/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixServerStarter.java
index c1ea182..402bb6b 100644
--- a/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixServerStarter.java
+++ b/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixServerStarter.java
@@ -21,14 +21,6 @@ package org.apache.pinot.server.starter.helix;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
 import org.apache.commons.configuration.BaseConfiguration;
 import org.apache.commons.configuration.Configuration;
 import org.apache.commons.configuration.ConfigurationUtils;
@@ -64,14 +56,57 @@ import org.apache.pinot.server.conf.ServerConf;
 import org.apache.pinot.server.realtime.ControllerLeaderLocator;
 import org.apache.pinot.server.realtime.ServerSegmentCompletionProtocolHandler;
 import org.apache.pinot.server.starter.ServerInstance;
-import org.apache.pinot.server.upsert.UpsertComponentContainer;
-import org.apache.pinot.server.upsert.UpsertComponentContainerProvider;
 import org.apache.pinot.spi.filesystem.PinotFSFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.pinot.common.utils.CommonConstants.Helix.*;
-import static org.apache.pinot.common.utils.CommonConstants.Server.*;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.pinot.common.utils.CommonConstants.Helix.CONFIG_OF_SERVER_FLAPPING_TIME_WINDOW_MS;
+import static org.apache.pinot.common.utils.CommonConstants.Helix.DEFAULT_FLAPPING_TIME_WINDOW_MS;
+import static org.apache.pinot.common.utils.CommonConstants.Helix.DEFAULT_SERVER_NETTY_PORT;
+import static org.apache.pinot.common.utils.CommonConstants.Helix.INSTANCE_CONNECTED_METRIC_NAME;
+import static org.apache.pinot.common.utils.CommonConstants.Helix.IS_SHUTDOWN_IN_PROGRESS;
+import static org.apache.pinot.common.utils.CommonConstants.Helix.Instance;
+import static org.apache.pinot.common.utils.CommonConstants.Helix.KEY_OF_SERVER_NETTY_HOST;
+import static org.apache.pinot.common.utils.CommonConstants.Helix.KEY_OF_SERVER_NETTY_PORT;
+import static org.apache.pinot.common.utils.CommonConstants.Helix.PREFIX_OF_SERVER_INSTANCE;
+import static org.apache.pinot.common.utils.CommonConstants.Helix.StateModel;
+import static org.apache.pinot.common.utils.CommonConstants.Helix.UNTAGGED_SERVER_INSTANCE;
+import static org.apache.pinot.common.utils.CommonConstants.Server.CONFIG_OF_ADMIN_API_PORT;
+import static org.apache.pinot.common.utils.CommonConstants.Server.CONFIG_OF_INSTANCE_DATA_DIR;
+import static org.apache.pinot.common.utils.CommonConstants.Server.CONFIG_OF_INSTANCE_ID;
+import static org.apache.pinot.common.utils.CommonConstants.Server.CONFIG_OF_INSTANCE_SEGMENT_TAR_DIR;
+import static org.apache.pinot.common.utils.CommonConstants.Server.CONFIG_OF_QUERY_EXECUTOR_TIMEOUT;
+import static org.apache.pinot.common.utils.CommonConstants.Server.CONFIG_OF_SERVER_MIN_RESOURCE_PERCENT_FOR_START;
+import static org.apache.pinot.common.utils.CommonConstants.Server.CONFIG_OF_SHUTDOWN_ENABLE_QUERY_CHECK;
+import static org.apache.pinot.common.utils.CommonConstants.Server.CONFIG_OF_SHUTDOWN_ENABLE_RESOURCE_CHECK;
+import static org.apache.pinot.common.utils.CommonConstants.Server.CONFIG_OF_SHUTDOWN_NO_QUERY_THRESHOLD_MS;
+import static org.apache.pinot.common.utils.CommonConstants.Server.CONFIG_OF_SHUTDOWN_RESOURCE_CHECK_INTERVAL_MS;
+import static org.apache.pinot.common.utils.CommonConstants.Server.CONFIG_OF_SHUTDOWN_TIMEOUT_MS;
+import static org.apache.pinot.common.utils.CommonConstants.Server.CONFIG_OF_STARTUP_ENABLE_SERVICE_STATUS_CHECK;
+import static org.apache.pinot.common.utils.CommonConstants.Server.CONFIG_OF_STARTUP_REALTIME_CONSUMPTION_CATCHUP_WAIT_MS;
+import static org.apache.pinot.common.utils.CommonConstants.Server.CONFIG_OF_STARTUP_SERVICE_STATUS_CHECK_INTERVAL_MS;
+import static org.apache.pinot.common.utils.CommonConstants.Server.CONFIG_OF_STARTUP_TIMEOUT_MS;
+import static org.apache.pinot.common.utils.CommonConstants.Server.DEFAULT_ADMIN_API_PORT;
+import static org.apache.pinot.common.utils.CommonConstants.Server.DEFAULT_QUERY_EXECUTOR_TIMEOUT_MS;
+import static org.apache.pinot.common.utils.CommonConstants.Server.DEFAULT_SERVER_MIN_RESOURCE_PERCENT_FOR_START;
+import static org.apache.pinot.common.utils.CommonConstants.Server.DEFAULT_SHUTDOWN_ENABLE_QUERY_CHECK;
+import static org.apache.pinot.common.utils.CommonConstants.Server.DEFAULT_SHUTDOWN_ENABLE_RESOURCE_CHECK;
+import static org.apache.pinot.common.utils.CommonConstants.Server.DEFAULT_SHUTDOWN_RESOURCE_CHECK_INTERVAL_MS;
+import static org.apache.pinot.common.utils.CommonConstants.Server.DEFAULT_SHUTDOWN_TIMEOUT_MS;
+import static org.apache.pinot.common.utils.CommonConstants.Server.DEFAULT_STARTUP_ENABLE_SERVICE_STATUS_CHECK;
+import static org.apache.pinot.common.utils.CommonConstants.Server.DEFAULT_STARTUP_REALTIME_CONSUMPTION_CATCHUP_WAIT_MS;
+import static org.apache.pinot.common.utils.CommonConstants.Server.DEFAULT_STARTUP_SERVICE_STATUS_CHECK_INTERVAL_MS;
+import static org.apache.pinot.common.utils.CommonConstants.Server.DEFAULT_STARTUP_TIMEOUT_MS;
+import static org.apache.pinot.common.utils.CommonConstants.Server.SegmentCompletionProtocol;
 
 
 /**
@@ -138,11 +173,11 @@ public class HelixServerStarter {
     ServerSegmentCompletionProtocolHandler
         .init(_serverConf.subset(SegmentCompletionProtocol.PREFIX_OF_CONFIG_OF_SEGMENT_UPLOADER));
     ServerConf serverInstanceConfig = DefaultHelixStarterServerConfig.getDefaultHelixServerConfig(_serverConf);
-    _serverInstance = new ServerInstance(serverInstanceConfig, _helixManager);
+    _serverInstance = new ServerInstance(serverInstanceConfig, _helixManager, _helixClusterName, _instanceId);
     InstanceDataManager instanceDataManager = _serverInstance.getInstanceDataManager();
     SegmentFetcherAndLoader fetcherAndLoader = new SegmentFetcherAndLoader(_serverConf, instanceDataManager);
-    StateModelFactory<?> stateModelFactory =
-        new SegmentOnlineOfflineStateModelFactory(_instanceId, instanceDataManager, fetcherAndLoader);
+    StateModelFactory<?> stateModelFactory = new SegmentOnlineOfflineStateModelFactory(_instanceId, instanceDataManager,
+            fetcherAndLoader, _serverInstance.getSegmentDeletionHandler());
     _helixManager.getStateMachineEngine()
         .registerStateModelFactory(SegmentOnlineOfflineStateModelFactory.getStateModelName(), stateModelFactory);
     // Start the server instance as a pre-connect callback so that it starts after connecting to the ZK in order to
@@ -181,6 +216,8 @@ public class HelixServerStarter {
       long endTimeMs = startTimeMs + _serverConf.getLong(CONFIG_OF_STARTUP_TIMEOUT_MS, DEFAULT_STARTUP_TIMEOUT_MS);
       startupServiceStatusCheck(endTimeMs);
     }
+    _serverInstance.maybeStartUpsertBackgroundThread();
+
     setShuttingDownStatus(false);
     LOGGER.info("Pinot server ready");
 
diff --git a/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/SegmentOnlineOfflineStateModelFactory.java b/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/SegmentOnlineOfflineStateModelFactory.java
index 145e378..1eab7f5 100644
--- a/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/SegmentOnlineOfflineStateModelFactory.java
+++ b/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/SegmentOnlineOfflineStateModelFactory.java
@@ -19,8 +19,6 @@
 package org.apache.pinot.server.starter.helix;
 
 import com.google.common.base.Preconditions;
-import java.io.File;
-import java.util.concurrent.locks.Lock;
 import org.apache.commons.io.FileUtils;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.model.Message;
@@ -39,10 +37,13 @@ import org.apache.pinot.core.data.manager.InstanceDataManager;
 import org.apache.pinot.core.data.manager.SegmentDataManager;
 import org.apache.pinot.core.data.manager.TableDataManager;
 import org.apache.pinot.core.data.manager.realtime.LLRealtimeSegmentDataManager;
-import org.apache.pinot.server.upsert.SegmentDeletionHandler;
+import org.apache.pinot.core.segment.updater.SegmentDeletionHandler;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.File;
+import java.util.concurrent.locks.Lock;
+
 
 /**
  * Data Server layer state model to take over how to operate on:
@@ -56,17 +57,18 @@ public class SegmentOnlineOfflineStateModelFactory extends StateModelFactory<Sta
   private final SegmentFetcherAndLoader _fetcherAndLoader;
   private final SegmentDeletionHandler _segmentDeletionHandler;
 
+
   public SegmentOnlineOfflineStateModelFactory(String instanceId, InstanceDataManager instanceDataManager,
       SegmentFetcherAndLoader fetcherAndLoader) {
-    this(instanceId, instanceDataManager, fetcherAndLoader, new SegmentDeletionHandler());
+      this(instanceId, instanceDataManager, fetcherAndLoader, new SegmentDeletionHandler());
   }
 
   public SegmentOnlineOfflineStateModelFactory(String instanceId, InstanceDataManager instanceDataManager,
-      SegmentFetcherAndLoader fetcherAndLoader, SegmentDeletionHandler segmentDeletionHandler) {
+      SegmentFetcherAndLoader fetcherAndLoader, SegmentDeletionHandler deletionHandler) {
     _instanceId = instanceId;
     _instanceDataManager = instanceDataManager;
     _fetcherAndLoader = fetcherAndLoader;
-    _segmentDeletionHandler = segmentDeletionHandler;
+    _segmentDeletionHandler = deletionHandler;
   }
 
   public static String getStateModelName() {
diff --git a/pinot-server/src/main/java/org/apache/pinot/server/upsert/DefaultUpsertComponentContainer.java b/pinot-server/src/main/java/org/apache/pinot/server/upsert/DefaultUpsertComponentContainer.java
index 63424ac..3841a2e 100644
--- a/pinot-server/src/main/java/org/apache/pinot/server/upsert/DefaultUpsertComponentContainer.java
+++ b/pinot-server/src/main/java/org/apache/pinot/server/upsert/DefaultUpsertComponentContainer.java
@@ -18,20 +18,25 @@
  */
 package org.apache.pinot.server.upsert;
 
-import com.codahale.metrics.MetricRegistry;
+import com.yammer.metrics.core.MetricsRegistry;
 import org.apache.commons.configuration.Configuration;
+import org.apache.helix.HelixManager;
+import org.apache.pinot.core.segment.updater.DefaultWaterMarkManager;
+import org.apache.pinot.core.segment.updater.SegmentDeletionHandler;
+import org.apache.pinot.core.segment.updater.UpsertComponentContainer;
+import org.apache.pinot.core.segment.updater.WaterMarkManager;
 
 public class DefaultUpsertComponentContainer implements UpsertComponentContainer {
 
   private final SegmentDeletionHandler deletionHandler = new SegmentDeletionHandler();
+  private final WaterMarkManager watermarkManager = new DefaultWaterMarkManager();
 
   @Override
-  public void registerMetrics(MetricRegistry registry) {
-
+  public void registerMetrics(String prefix, MetricsRegistry registry) {
   }
 
   @Override
-  public void init(Configuration config) {
+  public void init(Configuration config, HelixManager helixManager, String clusterName, String instanceName) {
   }
 
   @Override
@@ -40,10 +45,29 @@ public class DefaultUpsertComponentContainer implements UpsertComponentContainer
   }
 
   @Override
-  public void start() {
+  public WaterMarkManager getWatermarkManager() {
+    return watermarkManager;
+  }
+
+  @Override
+  public boolean isUpsertEnabled() {
+    return false;
   }
 
   @Override
-  public void stop() {
+  public void startBackgroundThread() {
+
   }
+
+  @Override
+  public void stopBackgroundThread() {
+
+  }
+
+  @Override
+  public void shutdown() {
+
+  }
+
+
 }
diff --git a/pinot-server/src/main/java/org/apache/pinot/server/upsert/UpsertComponentContainerProvider.java b/pinot-server/src/main/java/org/apache/pinot/server/upsert/UpsertComponentContainerProvider.java
index fe4af07..7ae27a6 100644
--- a/pinot-server/src/main/java/org/apache/pinot/server/upsert/UpsertComponentContainerProvider.java
+++ b/pinot-server/src/main/java/org/apache/pinot/server/upsert/UpsertComponentContainerProvider.java
@@ -19,10 +19,10 @@
 package org.apache.pinot.server.upsert;
 
 import com.google.common.base.Preconditions;
-import org.apache.commons.configuration.Configuration;
 import org.apache.commons.lang3.exception.ExceptionUtils;
-import org.apache.pinot.core.segment.updater.WatermarkManager;
-import org.apache.pinot.grigio.common.metrics.GrigioMetrics;
+import org.apache.pinot.core.segment.updater.UpsertComponentContainer;
+import org.apache.pinot.core.segment.updater.WaterMarkManager;
+import org.apache.pinot.server.conf.ServerConf;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -30,22 +30,16 @@ public class UpsertComponentContainerProvider {
 
   private static final Logger LOGGER = LoggerFactory.getLogger(UpsertComponentContainerProvider.class);
 
-  public static final String UPSERT_COMPONENT_CONFIG_KEY = "watermarkManager.class";
-  public static final String UPSERT_COMPONENT_CONFIG_DEFAULT = DefaultUpsertComponentContainer.class.getName();
-
-  private final Configuration _conf;
   private UpsertComponentContainer _instance;
 
-  public UpsertComponentContainerProvider(Configuration conf, GrigioMetrics metrics) {
-    _conf = conf;
-    String className = _conf.getString(UPSERT_COMPONENT_CONFIG_KEY, UPSERT_COMPONENT_CONFIG_DEFAULT);
+  public UpsertComponentContainerProvider(ServerConf serverConf) {
+    String className = serverConf.getUpsertComponentContainerClassName();
     LOGGER.info("creating watermark manager with class {}", className);
     try {
       Class<UpsertComponentContainer> comonentContainerClass = (Class<UpsertComponentContainer>) Class.forName(className);
-      Preconditions.checkState(comonentContainerClass.isAssignableFrom(WatermarkManager.class),
+      Preconditions.checkState(comonentContainerClass.isAssignableFrom(WaterMarkManager.class),
           "configured class not assignable from Callback class");
       _instance = comonentContainerClass.newInstance();
-      _instance.init(_conf);
     } catch (Exception e) {
       LOGGER.error("failed to load watermark manager class", className, e);
       ExceptionUtils.rethrow(e);
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/WatermarkManager.java b/pinot-server/src/test/java/org/apache/pinot/server/api/resources/LowWatermarksResourceTest.java
similarity index 60%
rename from pinot-core/src/main/java/org/apache/pinot/core/segment/updater/WatermarkManager.java
rename to pinot-server/src/test/java/org/apache/pinot/server/api/resources/LowWatermarksResourceTest.java
index 01579e7..b007c90 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/WatermarkManager.java
+++ b/pinot-server/src/test/java/org/apache/pinot/server/api/resources/LowWatermarksResourceTest.java
@@ -1,3 +1,4 @@
+
 /**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -16,18 +17,18 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.core.segment.updater;
-
-import org.apache.commons.configuration.Configuration;
-import org.apache.pinot.grigio.common.metrics.GrigioMeter;
-import org.apache.pinot.grigio.common.metrics.GrigioMetrics;
-
-import java.util.Map;
-
-public interface WatermarkManager {
+package org.apache.pinot.server.api.resources;
 
-  void init(Configuration config, GrigioMetrics metrics);
+import org.apache.pinot.common.restlet.resources.TableLowWaterMarksInfo;
+import org.apache.pinot.server.api.BaseResourceTest;
+import org.testng.Assert;
+import org.testng.annotations.Test;
 
-  Map<String, Map<Integer, Long>> getHighWaterMarkTablePartitionMap();
 
+public class LowWatermarksResourceTest extends BaseResourceTest {
+    @Test
+    public void testHappyPath() {
+        TableLowWaterMarksInfo lwms = _webTarget.path("lwms").request().get(TableLowWaterMarksInfo.class);
+        Assert.assertEquals(lwms.getTableLowWaterMarks().size(), 0);
+    }
 }
diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/realtime/provisioning/MemoryEstimator.java b/pinot-tools/src/main/java/org/apache/pinot/tools/realtime/provisioning/MemoryEstimator.java
index ac989bb..65cb9e7 100644
--- a/pinot-tools/src/main/java/org/apache/pinot/tools/realtime/provisioning/MemoryEstimator.java
+++ b/pinot-tools/src/main/java/org/apache/pinot/tools/realtime/provisioning/MemoryEstimator.java
@@ -18,11 +18,6 @@
  */
 package org.apache.pinot.tools.realtime.provisioning;
 
-import java.io.File;
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.Set;
-import java.util.stream.Collectors;
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.io.FileUtils;
 import org.apache.pinot.common.config.TableConfig;
@@ -40,6 +35,12 @@ import org.apache.pinot.core.realtime.impl.RealtimeSegmentConfig;
 import org.apache.pinot.core.realtime.impl.RealtimeSegmentStatsHistory;
 import org.apache.pinot.core.segment.index.SegmentMetadataImpl;
 
+import java.io.File;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.stream.Collectors;
+
 
 /**
  * Given a sample segment, this class can estimate how much memory would be used per host, for various combinations of numHostsToProvision and numHoursToConsume
@@ -126,7 +127,8 @@ public class MemoryEstimator {
 
     // create a config
     RealtimeSegmentConfig.Builder realtimeSegmentConfigBuilder =
-        new RealtimeSegmentConfig.Builder().setSegmentName(_segmentMetadata.getName())
+        new RealtimeSegmentConfig.Builder().setTableName(_tableConfig.getTableName())
+            .setSegmentName(_segmentMetadata.getName())
             .setStreamName(_tableNameWithType).setSchema(_segmentMetadata.getSchema())
             .setCapacity(_segmentMetadata.getTotalDocs()).setAvgNumMultiValues(_avgMultiValues)
             .setNoDictionaryColumns(_noDictionaryColumns)
@@ -228,7 +230,8 @@ public class MemoryEstimator {
       RealtimeSegmentZKMetadata segmentZKMetadata = getRealtimeSegmentZKMetadata(_segmentMetadata, totalDocs);
 
       RealtimeSegmentConfig.Builder realtimeSegmentConfigBuilder =
-          new RealtimeSegmentConfig.Builder().setSegmentName(_segmentMetadata.getName())
+          new RealtimeSegmentConfig.Builder().setTableName(_tableConfig.getTableName())
+              .setSegmentName(_segmentMetadata.getName())
               .setStreamName(_tableNameWithType).setSchema(_segmentMetadata.getSchema())
               .setCapacity(totalDocs).setAvgNumMultiValues(_avgMultiValues)
               .setNoDictionaryColumns(_noDictionaryColumns)


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


[incubator-pinot] 05/09: test build

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jamesshao pushed a commit to branch upsert-refactor
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git

commit 592fb5f2fdd9ce5c6d3e3275de522c20a4e96f0c
Author: Uber Jenkins <pa...@uber.com>
AuthorDate: Thu Feb 13 23:55:51 2020 +0000

    test build
    
    Summary: test unit test
    
    Differential Revision: https://code.uberinternal.com/D4105485
---
 .../VirtualColumnLongValueReaderWriter.java        |   7 +-
 .../SegmentUpdateLogStorageProvider.java           |   9 +-
 .../storageProvider/UpdateLogStorageExplorer.java  |  25 ++-
 .../UpsertImmutableIndexSegmentCallback.java       |  44 +++--
 .../pinot/core/segment/updater/SegmentUpdater.java |  74 ++++-----
 .../updater/SegmentUpdaterDataManagerHolder.java   | 131 +++++++++++++++
 .../SegmentUpdaterDataManagerHolderTest.java       | 184 +++++++++++++++++++++
 .../server/api/resources/UpsertDebugResource.java  |  17 +-
 8 files changed, 421 insertions(+), 70 deletions(-)

diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/mutable/VirtualColumnLongValueReaderWriter.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/mutable/VirtualColumnLongValueReaderWriter.java
index 78f3bda..dea717c 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/mutable/VirtualColumnLongValueReaderWriter.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/virtualcolumn/mutable/VirtualColumnLongValueReaderWriter.java
@@ -64,7 +64,12 @@ public abstract class VirtualColumnLongValueReaderWriter extends BaseVirtualColu
 
   @Override
   public long getLong(int row) {
-    return _values[row];
+    if (row >= 0 && row < _totalDocSize) {
+      return _values[row];
+    } else {
+      throw new RuntimeException(String.format("trying to fetch row %d while we only have total row count %d", row,
+          _totalDocSize));
+    }
   }
 
   @Override
diff --git a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/storageProvider/SegmentUpdateLogStorageProvider.java b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/storageProvider/SegmentUpdateLogStorageProvider.java
index e7b035f..fa045d1 100644
--- a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/storageProvider/SegmentUpdateLogStorageProvider.java
+++ b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/storageProvider/SegmentUpdateLogStorageProvider.java
@@ -32,6 +32,7 @@ import java.nio.channels.FileChannel;
 import java.nio.file.Files;
 import java.nio.file.Paths;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
 
 /**
  * provide the storage abstraction of storing upsert update event logs to a local disk so we can reload it
@@ -44,6 +45,7 @@ public class SegmentUpdateLogStorageProvider {
   protected final File _file;
   @VisibleForTesting
   protected final FileOutputStream _outputStream;
+  private AtomicInteger messageCountInFile = new AtomicInteger(0);
 
   public SegmentUpdateLogStorageProvider(File file)
       throws IOException {
@@ -60,6 +62,7 @@ public class SegmentUpdateLogStorageProvider {
       ByteBuffer buffer = ByteBuffer.allocate(fileLength);
       readFullyFromBeginning(_file, buffer);
       int messageCount = fileLength / UpdateLogEntry.SIZE;
+      LOGGER.info("read {} messages from file {}", messageCount, _file.getName());
       return new UpdateLogEntrySet(buffer, messageCount);
     } else {
       return UpdateLogEntrySet.getEmptySet();
@@ -74,7 +77,8 @@ public class SegmentUpdateLogStorageProvider {
     buffer.flip();
     _outputStream.write(buffer.array());
     _outputStream.flush();
-
+    messageCountInFile.getAndAdd(messages.size());
+    LOGGER.debug("file {} message count {}", _file.getName(), messageCountInFile.get());
   }
 
   public synchronized void destroy() throws IOException {
@@ -102,6 +106,7 @@ public class SegmentUpdateLogStorageProvider {
           segmentUpdateFile.length(), newSize);
       channel.truncate(newSize);
       channel.force(false);
+      messageCountInFile.set(Math.toIntExact(newSize / UpdateLogEntry.SIZE));
     }
   }
 
@@ -116,7 +121,7 @@ public class SegmentUpdateLogStorageProvider {
       position += byteRead;
     } while (byteRead != -1 && buffer.hasRemaining());
     buffer.flip();
-    LOGGER.info("read all data from segment update file {} to buffer in {} ms", segmentUpdateFile.getName(),
+    LOGGER.info("read {} bytes from segment update file {} to buffer in {} ms", position, segmentUpdateFile.getName(),
         System.currentTimeMillis() - start);
   }
 
diff --git a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/storageProvider/UpdateLogStorageExplorer.java b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/storageProvider/UpdateLogStorageExplorer.java
index 0e38deb..a2b53ee 100644
--- a/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/storageProvider/UpdateLogStorageExplorer.java
+++ b/pinot-grigio/pinot-grigio-common/src/main/java/org/apache/pinot/grigio/common/storageProvider/UpdateLogStorageExplorer.java
@@ -27,6 +27,7 @@ import org.apache.commons.configuration.PropertiesConfiguration;
 import java.io.IOException;
 import java.util.Collection;
 import java.util.Scanner;
+import java.util.concurrent.atomic.AtomicInteger;
 
 /**
  * command line tools for debug pinot server by allowing us to interatively explore the update log data in pinot server/kc
@@ -52,24 +53,38 @@ public class UpdateLogStorageExplorer {
     String[] inputSplits = input.split(" ");
     Preconditions.checkState(inputSplits.length == 2, "expect input data to be 'tableName segmentName'");
     String tableName = inputSplits[0];
+    if (!tableName.endsWith("_REALTIME")) {
+      tableName = tableName + "_REALTIME";
+    }
     String segmentName = inputSplits[1];
 
     provider.loadTable(tableName);
     UpdateLogEntrySet updateLogEntrySet = provider.getAllMessages(tableName, segmentName);
-    Multimap<Long, UpdateLogEntry> map = ArrayListMultimap.create();
+    Multimap<Long, UpdateLogAndPos> map = ArrayListMultimap.create();
     System.out.println("update log size: " + updateLogEntrySet.size());
+    AtomicInteger pos = new AtomicInteger(0);
     updateLogEntrySet.forEach(u -> {
-      map.put(u.getOffset(), u);
+      map.put(u.getOffset(), new UpdateLogAndPos(u, pos.getAndIncrement()));
     });
 
     while (true) {
       System.out.println("input the offset");
       long offset = reader.nextLong();
-      Collection<UpdateLogEntry> result = map.get(offset);
+      Collection<UpdateLogAndPos> result = map.get(offset);
       System.out.println("associated update logs size: " + result.size());
-      for (UpdateLogEntry entry: result) {
-        System.out.println("content: " + entry.toString());
+      for (UpdateLogAndPos entry: result) {
+        System.out.println("content: " + entry.logEntry.toString() + " pos " + entry.pos);
       }
     }
   }
+
+  static class UpdateLogAndPos {
+    public UpdateLogEntry logEntry;
+    public int pos;
+
+    public UpdateLogAndPos(UpdateLogEntry entry, int pos) {
+      this.logEntry = entry;
+      this.pos = pos;
+    }
+  }
 }
diff --git a/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/data/manager/upsert/UpsertImmutableIndexSegmentCallback.java b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/data/manager/upsert/UpsertImmutableIndexSegmentCallback.java
index b7fcdb8..d0667b8 100644
--- a/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/data/manager/upsert/UpsertImmutableIndexSegmentCallback.java
+++ b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/data/manager/upsert/UpsertImmutableIndexSegmentCallback.java
@@ -130,9 +130,9 @@ public class UpsertImmutableIndexSegmentCallback implements IndexSegmentCallback
       throw new RuntimeException("unexpected forward reader type for kafka offset column " + reader.getClass());
     }
     LOGGER.info("built offset to DocId map for segment {} with {} documents in {} ms", _segmentName, _totalDoc, System.currentTimeMillis() - start);
+    LOGGER.info("immutable segment {} built offset map with minOffset {} and maxOffset {}", _segmentName, minOffset, maxOffset);
   }
 
-  @Override
   public void postProcessRecords(GenericRow row, int docId) {
     // do nothing
   }
@@ -150,7 +150,7 @@ public class UpsertImmutableIndexSegmentCallback implements IndexSegmentCallback
         updateLogEntries.size(), _segmentName, System.currentTimeMillis() - start);
 
     start = System.currentTimeMillis();
-    final long maxOffset = _totalDoc + _minSourceOffset;
+    final long maxOffset = _minSourceOffset + _sourceOffsetToDocIdArray.length;
     int unmatchedLogEntryCount = 0;
     try {
       Map<Integer, Long> partitionToHighestWatermark = new HashMap<>();
@@ -193,13 +193,19 @@ public class UpsertImmutableIndexSegmentCallback implements IndexSegmentCallback
   @Override
   public void updateVirtualColumn(Iterable<UpdateLogEntry> logEntries) {
     for (UpdateLogEntry logEntry: logEntries) {
-      boolean updated = false;
-      int docId = getDocIdFromSourceOffset(logEntry.getOffset());
-      for (VirtualColumnLongValueReaderWriter readerWriter : _virtualColumnsReaderWriter) {
-        updated = readerWriter.update(docId, logEntry.getValue(), logEntry.getType()) || updated;
-      }
-      if (updated) {
-        _upsertWatermarkManager.processMessage(_tableNameWithType, _segmentName, logEntry);
+      try {
+        boolean updated = false;
+        int docId = getDocIdFromSourceOffset(logEntry.getOffset());
+        if (docId != DEFAULT_DOC_ID_FOR_MISSING_ENTRY) {
+          for (VirtualColumnLongValueReaderWriter readerWriter : _virtualColumnsReaderWriter) {
+            updated = readerWriter.update(docId, logEntry.getValue(), logEntry.getType()) || updated;
+          }
+          if (updated) {
+            _upsertWatermarkManager.processMessage(_tableNameWithType, _segmentName, logEntry);
+          }
+        }
+      } catch (Exception ex) {
+        LOGGER.warn("failed to update virtual column, skipping the current record {}", logEntries.toString());
       }
     }
   }
@@ -207,11 +213,15 @@ public class UpsertImmutableIndexSegmentCallback implements IndexSegmentCallback
   @Override
   public String getVirtualColumnInfo(long offset) {
     int docId = getDocIdFromSourceOffset(offset);
-    StringBuilder result = new StringBuilder("matched: ");
-    for (VirtualColumnLongValueReaderWriter readerWriter : _virtualColumnsReaderWriter) {
-      result.append(readerWriter.getInt(docId)).append("; ");
+    if (docId != DEFAULT_DOC_ID_FOR_MISSING_ENTRY) {
+      StringBuilder result = new StringBuilder("matched: ");
+      for (VirtualColumnLongValueReaderWriter readerWriter : _virtualColumnsReaderWriter) {
+        result.append(readerWriter.getLong(docId)).append("; ");
+      }
+      return result.toString();
+    } else {
+      return "cannot found doc matching offset";
     }
-    return result.toString();
   }
 
   /**
@@ -222,14 +232,14 @@ public class UpsertImmutableIndexSegmentCallback implements IndexSegmentCallback
    */
   private int getDocIdFromSourceOffset(long offset) throws RuntimeException {
     if (offset < _minSourceOffset || offset - _minSourceOffset >= _sourceOffsetToDocIdArray.length) {
-      LOGGER.error("offset {} is outside range for current segment {} start offset {} size {}",
+      LOGGER.warn("offset {} is outside of range for current segment {} start offset {} size {}",
           offset, _segmentName, _minSourceOffset, _sourceOffsetToDocIdArray.length);
-      throw new RuntimeException("offset outside range");
+      return DEFAULT_DOC_ID_FOR_MISSING_ENTRY;
     } else {
       int position = Math.toIntExact(offset - _minSourceOffset);
       if (_sourceOffsetToDocIdArray[position] == DEFAULT_DOC_ID_FOR_MISSING_ENTRY) {
-        LOGGER.error("no docId associated with offset {} for segment {}", offset, _segmentName);
-        throw new RuntimeException("docId not found");
+        LOGGER.warn("no docId associated with offset {} for segment {}", offset, _segmentName);
+        return DEFAULT_DOC_ID_FOR_MISSING_ENTRY;
       } else {
         return _sourceOffsetToDocIdArray[position];
       }
diff --git a/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdater.java b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdater.java
index 5f0aa43..f380afa 100644
--- a/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdater.java
+++ b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdater.java
@@ -21,7 +21,6 @@ package org.apache.pinot.core.segment.updater;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.util.concurrent.Uninterruptibles;
-import io.netty.util.internal.ConcurrentSet;
 import org.apache.commons.configuration.Configuration;
 import org.apache.pinot.common.config.TableNameBuilder;
 import org.apache.pinot.common.utils.CommonConstants;
@@ -36,6 +35,7 @@ import org.apache.pinot.grigio.common.rpcQueue.QueueConsumerRecord;
 import org.apache.pinot.grigio.common.storageProvider.UpdateLogEntry;
 import org.apache.pinot.grigio.common.storageProvider.UpdateLogStorageProvider;
 import org.apache.pinot.grigio.common.storageProvider.retentionManager.UpdateLogRetentionManager;
+import org.apache.pinot.grigio.common.storageProvider.retentionManager.UpdateLogTableRetentionManager;
 import org.apache.pinot.grigio.common.utils.CommonUtils;
 import org.apache.pinot.grigio.servers.SegmentUpdaterProvider;
 import org.slf4j.Logger;
@@ -46,7 +46,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -75,7 +74,7 @@ public class SegmentUpdater implements SegmentDeletionListener {
   private final String _topicPrefix;
   private final ExecutorService _ingestionExecutorService;
   private final QueueConsumer _consumer;
-  private final Map<String, Map<String, Set<DataManagerCallback>>> _tableSegmentMap = new ConcurrentHashMap<>();
+  private final SegmentUpdaterDataManagerHolder _dataManagersHolder = new SegmentUpdaterDataManagerHolder();
   private final Map<String, Map<Integer, Long>> _tablePartitionCreationTime = new ConcurrentHashMap<>();
   private final UpdateLogStorageProvider _updateLogStorageProvider;
   private final UpdateLogRetentionManager _retentionManager;
@@ -105,7 +104,7 @@ public class SegmentUpdater implements SegmentDeletionListener {
 
 
   public void start() {
-    String listOfTables = Joiner.on(",").join(_tableSegmentMap.keySet());
+    String listOfTables = Joiner.on(",").join(_dataManagersHolder.getAllTables());
     LOGGER.info("starting segment updater main loop with the following table in server: {}", listOfTables);
     _ingestionExecutorService.submit(this::updateLoop);
   }
@@ -162,10 +161,9 @@ public class SegmentUpdater implements SegmentDeletionListener {
         for (Map.Entry<String, TableUpdateLogs> entry : tableSegmentToUpdateLogs.entrySet()) {
           String tableName = TableNameBuilder.ensureTableNameWithType(entry.getKey(), CommonConstants.Helix.TableType.REALTIME);
           int tableMessageCount = 0;
-          if (_tableSegmentMap.containsKey(tableName)) {
-            final Map<String, Set<DataManagerCallback>> segmentManagersMap = _tableSegmentMap.get(tableName);
+          if (_dataManagersHolder.hasTable(tableName)) {
             final TableUpdateLogs segment2UpdateLogsMap = entry.getValue();
-            updateSegmentVirtualColumns(tableName, segmentManagersMap, segment2UpdateLogsMap, timeToStoreUpdateLogs);
+            updateSegmentVirtualColumns(tableName, segment2UpdateLogsMap, timeToStoreUpdateLogs);
           } else {
             LOGGER.warn("got messages for table {} not in this server", tableName);
           }
@@ -202,12 +200,11 @@ public class SegmentUpdater implements SegmentDeletionListener {
   /**
    * Update the virtual columns of affected segments of a table.
    */
-  private void updateSegmentVirtualColumns(String tableName, Map<String, Set<DataManagerCallback>> segmentManagersMap,
-                                           TableUpdateLogs segment2UpdateLogsMap, AtomicLong timeToStoreUpdateLogs) throws IOException{
+  private void updateSegmentVirtualColumns(String tableName, TableUpdateLogs segment2UpdateLogsMap,
+                                           AtomicLong timeToStoreUpdateLogs) throws IOException{
     for (Map.Entry<String, List<UpdateLogEntry>> segmentEntry : segment2UpdateLogsMap.getSegments2UpdateLog().entrySet()) {
       final String segmentNameStr = segmentEntry.getKey();
       updateVirtualColumn(tableName, segmentNameStr,
-          segmentManagersMap.computeIfAbsent(segmentNameStr, sn -> new ConcurrentSet<>()),
           segment2UpdateLogsMap.get(segmentNameStr), timeToStoreUpdateLogs);
     }
   }
@@ -217,19 +214,21 @@ public class SegmentUpdater implements SegmentDeletionListener {
    * from consuming to online (mutable segment to immutable segment). In most of cases we expect only one segment manager
    * in this set of UpsertSegmentDataManager
    */
-  private void updateVirtualColumn(String table, String segment, Set<DataManagerCallback> segmentDataManagers,
+  private void updateVirtualColumn(String table, String segment,
                                    List<UpdateLogEntry> messages, AtomicLong timeToStoreUpdateLogs) throws IOException {
+    Set<DataManagerCallback> dataManagers = _dataManagersHolder.getDataManagers(table, segment);
     LOGGER.debug("updating segment {} with {} results for {} data managers", segment, messages.size(),
-        segmentDataManagers.size());
-    if (segmentDataManagers.size() > 0 || _retentionManager.getRetentionManagerForTable(table).shouldIngestForSegment(segment)) {
+        dataManagers.size());
+    if (dataManagers.size() > 0 || _retentionManager.getRetentionManagerForTable(table).shouldIngestForSegment(segment)) {
       storeUpdateLogs(table, segment, messages, timeToStoreUpdateLogs);
     }
     try {
-      for (DataManagerCallback dataManager: segmentDataManagers) {
+      // refetch the data managers from holder in case there are updates
+      for (DataManagerCallback dataManager: _dataManagersHolder.getDataManagers(table, segment)) {
         dataManager.updateVirtualColumns(messages);
       }
     } catch (Exception ex) {
-      LOGGER.error("failed to update virtual column for key ", ex);
+      LOGGER.error("failed to update virtual column for key", ex);
     }
   }
 
@@ -243,14 +242,11 @@ public class SegmentUpdater implements SegmentDeletionListener {
       DataManagerCallback dataManager) {
     // TODO get partition assignment from
     LOGGER.info("segment updater adding table {} segment {}", tableNameWithType, segmentName.getSegmentName());
-    if (!_tableSegmentMap.containsKey(tableNameWithType)) {
-      synchronized (_tableSegmentMap) {
-        _tableSegmentMap.put(tableNameWithType, new ConcurrentHashMap<>());
-      }
+    if (!_dataManagersHolder.hasTable(tableNameWithType)) {
       LOGGER.info("adding table {} to segment updater consumer", tableNameWithType);
       handleNewTableInServer(tableNameWithType);
     }
-    _tableSegmentMap.get(tableNameWithType).computeIfAbsent(segmentName.getSegmentName(), sn -> new HashSet<>()).add(dataManager);
+    _dataManagersHolder.addDataManager(tableNameWithType, segmentName.getSegmentName(), dataManager);
     synchronized (_tablePartitionCreationTime) {
       long creationTime = _tablePartitionCreationTime.computeIfAbsent(tableNameWithType, t -> new ConcurrentHashMap<>())
           .computeIfAbsent(segmentName.getPartitionId(), p -> segmentName.getCreationTimeStamp());
@@ -262,16 +258,7 @@ public class SegmentUpdater implements SegmentDeletionListener {
   public synchronized void removeSegmentDataManager(String tableNameWithType, String segmentName,
       DataManagerCallback toDeleteManager) {
     LOGGER.info("segment updater removing table {} segment {}", tableNameWithType, segmentName);
-    Map<String, Set<DataManagerCallback>> segmentMap = _tableSegmentMap.get(tableNameWithType);
-    if (segmentMap != null) {
-      Set<DataManagerCallback> segmentDataManagers = segmentMap.get(segmentName);
-      if (segmentDataManagers != null) {
-        segmentDataManagers.remove(toDeleteManager);
-        if (segmentDataManagers.size() == 0) {
-          segmentMap.remove(segmentName);
-        }
-      }
-    }
+    _dataManagersHolder.removeDataManager(tableNameWithType, segmentName, toDeleteManager);
   }
 
   /**
@@ -300,22 +287,23 @@ public class SegmentUpdater implements SegmentDeletionListener {
   @Override
   public synchronized void onSegmentDeletion(String tableNameWithType, String segmentName) {
     LOGGER.info("deleting segment virtual column from local storage for table {} segment {}", tableNameWithType, segmentName);
-    Map<String, Set<DataManagerCallback>> segmentManagerMap = _tableSegmentMap.get(tableNameWithType);
-    if (segmentManagerMap != null) {
-      if (segmentManagerMap.containsKey(segmentName) && segmentManagerMap.get(segmentName).size() > 0) {
-        LOGGER.error("trying to remove segment storage with {} segment data manager", segmentManagerMap.get(segmentName).size());
-      }
-      try {
-        segmentManagerMap.remove(segmentName);
-        _retentionManager.getRetentionManagerForTable(tableNameWithType).notifySegmentDeletion(tableNameWithType);
-        _updateLogStorageProvider.removeSegment(tableNameWithType, segmentName);
-      } catch (IOException e) {
-        throw new RuntimeException(String.format("failed to delete table %s segment %s", tableNameWithType, segmentName), e);
+    if (_dataManagersHolder.hasTable(tableNameWithType)) {
+      boolean result = _dataManagersHolder.removeAllDataManagerForSegment(tableNameWithType, segmentName);
+      if (result) {
+        try {
+          UpdateLogTableRetentionManager retentionManager = _retentionManager.getRetentionManagerForTable(tableNameWithType);
+          if (retentionManager != null) {
+            retentionManager.notifySegmentDeletion(segmentName);
+          }
+          _updateLogStorageProvider.removeSegment(tableNameWithType, segmentName);
+        } catch (IOException e) {
+          throw new RuntimeException(String.format("failed to delete table %s segment %s", tableNameWithType, segmentName), e);
+        }
       }
-      if (segmentManagerMap.size() == 0) {
-        _tableSegmentMap.remove(tableNameWithType);
+      if (_dataManagersHolder.maybeRemoveTable(tableNameWithType)) {
         handleTableRemovalInServer(tableNameWithType);
       }
+
     } else {
       LOGGER.error("deleting a segment {}:{} from current server but don't have segment map on updater",
           tableNameWithType, segmentName);
diff --git a/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdaterDataManagerHolder.java b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdaterDataManagerHolder.java
new file mode 100644
index 0000000..4f6bd50
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdaterDataManagerHolder.java
@@ -0,0 +1,131 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.segment.updater;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.pinot.core.data.manager.upsert.DataManagerCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.ThreadSafe;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * class to manage list of data managers and their associated table/segment for segment updater
+ */
+@ThreadSafe
+public class SegmentUpdaterDataManagerHolder {
+  private static final Logger LOGGER = LoggerFactory.getLogger(SegmentUpdaterDataManagerHolder.class);
+
+  private final Map<String, Map<String, Set<DataManagerCallback>>> _tableSegmentMap = new ConcurrentHashMap<>();
+
+  public SegmentUpdaterDataManagerHolder() {}
+
+  /**
+   * fetch all tables containing at least one data manager on this server
+   * @return list of all pinot tables name for the current segment updater
+   */
+  public Set<String> getAllTables() {
+    return ImmutableSet.copyOf(_tableSegmentMap.keySet());
+  }
+
+  /**
+   * check if there is any data manager associated with the given table
+   */
+  public boolean hasTable(String tableName) {
+    return _tableSegmentMap.containsKey(tableName);
+  }
+
+  /**
+   * get a set of data manager for the given table name and segment name
+   */
+  public synchronized Set<DataManagerCallback> getDataManagers(String tableName, String segmentName) {
+    if (!_tableSegmentMap.containsKey(tableName)) {
+      LOGGER.error("try to fetch data manager for non-existing table {} segment {}", tableName, segmentName);
+    } else {
+      final Map<String, Set<DataManagerCallback>> segmentDataManagerMap = _tableSegmentMap.get(tableName);
+      if (segmentDataManagerMap.containsKey(segmentName)) {
+        return ImmutableSet.copyOf(segmentDataManagerMap.get(segmentName));
+      }
+    }
+    return ImmutableSet.of();
+  }
+
+  /**
+   * add a data manager for a given table and segment name
+   */
+  public synchronized void addDataManager(String tableName, String segmentName, DataManagerCallback dataManager) {
+    LOGGER.info("adding new data manager to updater for table {}, segment {}", tableName, segmentName);
+    if (!_tableSegmentMap.containsKey(tableName)) {
+      _tableSegmentMap.put(tableName, new ConcurrentHashMap<>());
+    }
+    _tableSegmentMap.get(tableName).computeIfAbsent(segmentName, sn -> ConcurrentHashMap.newKeySet()).add(dataManager);
+  }
+
+  /**
+   * remove a specific data manager for a given table and segment name.
+   * do nothing if there is no such data manager for the given table/segment name
+   */
+  public synchronized void removeDataManager(String tableName, String segmentName,
+                                             DataManagerCallback toDeleteManager) {
+    Map<String, Set<DataManagerCallback>> segmentMap = _tableSegmentMap.get(tableName);
+    if (segmentMap != null) {
+      Set<DataManagerCallback> segmentDataManagers = segmentMap.get(segmentName);
+      if (segmentDataManagers != null) {
+        segmentDataManagers.remove(toDeleteManager);
+        LOGGER.info("removing data manager for table {} segment {}", tableName, segmentName);
+        if (segmentDataManagers.size() == 0) {
+          segmentMap.remove(segmentName);
+        }
+      }
+    }
+  }
+
+  /**
+   * remove all data managers for a table and segment
+   * @return true if we indeed remove any data manager, false otherwise
+   */
+  public synchronized boolean removeAllDataManagerForSegment(String tableName, String segmentName) {
+    Map<String, Set<DataManagerCallback>> segmentManagerMap = _tableSegmentMap.get(tableName);
+    if (segmentManagerMap != null) {
+      if (segmentManagerMap.containsKey(segmentName)) {
+        LOGGER.error("trying to remove segment storage with {} segment data manager", segmentManagerMap.get(segmentName).size());
+      }
+      Set<DataManagerCallback> result = segmentManagerMap.remove(segmentName);
+      return result != null;
+    }
+    return false;
+  }
+
+  /**
+   * check if the table still has any associated data manager. If there is no data managers, then removed it from cached
+   * @return true if the given table is removed, false otherwise
+   */
+  public synchronized boolean maybeRemoveTable(String tableName) {
+    Map<String, Set<DataManagerCallback>> segmentManagerMap = _tableSegmentMap.get(tableName);
+    if (segmentManagerMap != null && segmentManagerMap.size() == 0) {
+      _tableSegmentMap.remove(tableName);
+      return true;
+    }
+    return false;
+  }
+
+}
diff --git a/pinot-grigio/pinot-grigio-provided/src/test/java/org/apache/pinot/core/segment/updater/SegmentUpdaterDataManagerHolderTest.java b/pinot-grigio/pinot-grigio-provided/src/test/java/org/apache/pinot/core/segment/updater/SegmentUpdaterDataManagerHolderTest.java
new file mode 100644
index 0000000..ca7aceb
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-provided/src/test/java/org/apache/pinot/core/segment/updater/SegmentUpdaterDataManagerHolderTest.java
@@ -0,0 +1,184 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.segment.updater;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.pinot.core.data.manager.upsert.DataManagerCallback;
+import org.testng.Assert;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import java.util.Set;
+
+import static org.mockito.Mockito.mock;
+
+public class SegmentUpdaterDataManagerHolderTest {
+  private SegmentUpdaterDataManagerHolder emptyHolder;
+  private SegmentUpdaterDataManagerHolder dataManagerHolder;
+
+  private DataManagerCallback dummyManager1;
+  private DataManagerCallback dummyManager2;
+  private DataManagerCallback dummyManager3;
+
+  @BeforeMethod
+  public void setUp() {
+    emptyHolder = new SegmentUpdaterDataManagerHolder();
+    dataManagerHolder = new SegmentUpdaterDataManagerHolder();
+    dummyManager1 = mock(DataManagerCallback.class);
+    dummyManager2 = mock(DataManagerCallback.class);
+    dummyManager3 = mock(DataManagerCallback.class);
+    dataManagerHolder.addDataManager("table", "segment1", dummyManager1);
+    dataManagerHolder.addDataManager("table", "segment2", dummyManager2);
+    dataManagerHolder.addDataManager("table2", "segment3", dummyManager3);
+  }
+
+  @Test
+  public void testGetAllTables() {
+    Set<String> tables = dataManagerHolder.getAllTables();
+    ensureSetEqual(tables, ImmutableSet.of("table", "table2"));
+
+    Assert.assertEquals(emptyHolder.getAllTables().size(), 0);
+  }
+
+  @Test
+  public void testHasTable() {
+    Assert.assertFalse(emptyHolder.hasTable("table"));
+
+    Assert.assertTrue(dataManagerHolder.hasTable("table"));
+    Assert.assertTrue(dataManagerHolder.hasTable("table2"));
+    Assert.assertFalse(dataManagerHolder.hasTable("table3"));
+  }
+
+  @Test
+  public void testGetDataManagers() {
+    Set<DataManagerCallback> dataManagers = dataManagerHolder.getDataManagers("table", "segment1");
+    ensureSetEqual(dataManagers, ImmutableSet.of(dummyManager1));
+
+    dataManagers = dataManagerHolder.getDataManagers("table", "segment2");
+    ensureSetEqual(dataManagers, ImmutableSet.of(dummyManager2));
+
+    dataManagers = dataManagerHolder.getDataManagers("table2", "segment3");
+    ensureSetEqual(dataManagers, ImmutableSet.of(dummyManager3));
+
+    // non exist tables/segments
+    dataManagers = dataManagerHolder.getDataManagers("table2", "segment1");
+    ensureSetEqual(dataManagers, ImmutableSet.of());
+
+    dataManagers = dataManagerHolder.getDataManagers("table3", "segment1");
+    ensureSetEqual(dataManagers, ImmutableSet.of());
+  }
+
+  @Test
+  public void testAddDataManager() {
+    DataManagerCallback dummyManager4 = mock(DataManagerCallback.class);
+    DataManagerCallback dummyManager5 = mock(DataManagerCallback.class);
+    DataManagerCallback dummyManager6 = mock(DataManagerCallback.class);
+    dataManagerHolder.addDataManager("table", "segment1", dummyManager4);
+    dataManagerHolder.addDataManager("table", "segment2", dummyManager5);
+    dataManagerHolder.addDataManager("table2", "segment1", dummyManager6);
+
+    Set<DataManagerCallback> tableSegmentDMs = dataManagerHolder.getDataManagers("table", "segment1");
+    ensureSetEqual(tableSegmentDMs, ImmutableSet.of(dummyManager1, dummyManager4));
+
+    tableSegmentDMs = dataManagerHolder.getDataManagers("table", "segment2");
+    ensureSetEqual(tableSegmentDMs, ImmutableSet.of(dummyManager2, dummyManager5));
+
+    tableSegmentDMs = dataManagerHolder.getDataManagers("table2", "segment1");
+    ensureSetEqual(tableSegmentDMs, ImmutableSet.of(dummyManager6));
+  }
+
+  @Test
+  public void testRemoveDataManager() {
+    DataManagerCallback dummyManager4 = mock(DataManagerCallback.class);
+    DataManagerCallback dummyManager5 = mock(DataManagerCallback.class);
+    DataManagerCallback dummyManager6 = mock(DataManagerCallback.class);
+    dataManagerHolder.addDataManager("table", "segment1", dummyManager4);
+    dataManagerHolder.addDataManager("table", "segment2", dummyManager5);
+    dataManagerHolder.addDataManager("table2", "segment1", dummyManager6);
+    Set<DataManagerCallback> tableSegmentDMs;
+
+    // start deleting
+    dataManagerHolder.removeDataManager("table", "segment1", dummyManager1);
+    dataManagerHolder.removeDataManager("table", "segment1", dummyManager2);
+    dataManagerHolder.removeDataManager("table", "segment", dummyManager2);
+    tableSegmentDMs = dataManagerHolder.getDataManagers("table", "segment1");
+    ensureSetEqual(tableSegmentDMs, ImmutableSet.of(dummyManager4));
+
+    // delete all segment
+    dataManagerHolder.removeDataManager("table", "segment1", dummyManager4);
+    tableSegmentDMs = dataManagerHolder.getDataManagers("table", "segment1");
+    ensureSetEqual(tableSegmentDMs, ImmutableSet.of());
+
+    // delete some more
+    dataManagerHolder.removeDataManager("table", "segment1", dummyManager4);
+    tableSegmentDMs = dataManagerHolder.getDataManagers("table", "segment1");
+    ensureSetEqual(tableSegmentDMs, ImmutableSet.of());
+
+    // add some back and delete
+    dataManagerHolder.addDataManager("table", "segment1", dummyManager4);
+    ensureSetEqual(dataManagerHolder.getDataManagers("table", "segment1"), ImmutableSet.of(dummyManager4));
+
+    dataManagerHolder.removeDataManager("table", "segment1", dummyManager4);
+    ensureSetEqual(dataManagerHolder.getDataManagers("table", "segment1"), ImmutableSet.of());
+  }
+
+  @Test
+  public void testRemoveAllDataManagerForSegment() {
+    DataManagerCallback dummyManager4 = mock(DataManagerCallback.class);
+    DataManagerCallback dummyManager5 = mock(DataManagerCallback.class);
+    DataManagerCallback dummyManager6 = mock(DataManagerCallback.class);
+    dataManagerHolder.addDataManager("table", "segment1", dummyManager4);
+    dataManagerHolder.addDataManager("table", "segment2", dummyManager5);
+    dataManagerHolder.addDataManager("table2", "segment1", dummyManager6);
+
+    boolean result = dataManagerHolder.removeAllDataManagerForSegment("table", "segment1");
+    ensureSetEqual(dataManagerHolder.getDataManagers("table", "segment1"), ImmutableSet.of());
+    Assert.assertTrue(result);
+
+    result = dataManagerHolder.removeAllDataManagerForSegment("table", "segment1");
+    ensureSetEqual(dataManagerHolder.getDataManagers("table", "segment1"), ImmutableSet.of());
+    Assert.assertFalse(result);
+
+    result = dataManagerHolder.removeAllDataManagerForSegment("table3", "segment1");
+    Assert.assertFalse(result);
+  }
+
+  @Test
+  public void testMaybeRemoveTable() {
+
+    Assert.assertFalse(dataManagerHolder.maybeRemoveTable("table"));
+    Assert.assertFalse(dataManagerHolder.maybeRemoveTable("table2"));
+    Assert.assertFalse(dataManagerHolder.maybeRemoveTable("table4"));
+
+    dataManagerHolder.removeAllDataManagerForSegment("table2", "segment3");
+    Assert.assertTrue(dataManagerHolder.maybeRemoveTable("table2"));
+
+    dataManagerHolder.removeAllDataManagerForSegment("table", "segment1");
+    Assert.assertFalse(dataManagerHolder.maybeRemoveTable("table"));
+    dataManagerHolder.removeAllDataManagerForSegment("table", "segment2");
+    Assert.assertTrue(dataManagerHolder.maybeRemoveTable("table"));
+  }
+
+  private <T> void ensureSetEqual(Set<T> set1, Set<T> set2) {
+    Assert.assertEquals(set2.size(), set1.size());
+    for (T o: set1) {
+      Assert.assertTrue(set2.contains(o));
+    }
+  }
+}
\ No newline at end of file
diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/UpsertDebugResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/UpsertDebugResource.java
index 13a7612..eb1f3ea 100644
--- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/UpsertDebugResource.java
+++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/UpsertDebugResource.java
@@ -26,19 +26,23 @@ import io.swagger.annotations.ApiResponses;
 import org.apache.pinot.core.data.manager.InstanceDataManager;
 import org.apache.pinot.core.data.manager.SegmentDataManager;
 import org.apache.pinot.core.data.manager.TableDataManager;
-import org.apache.pinot.core.data.manager.UpsertSegmentDataManager;
 import org.apache.pinot.server.starter.ServerInstance;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import javax.inject.Inject;
 import javax.ws.rs.GET;
 import javax.ws.rs.Path;
 import javax.ws.rs.PathParam;
 import javax.ws.rs.Produces;
+import javax.ws.rs.WebApplicationException;
 import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
 
 @Api(tags = "UpsertDebug")
 @Path("/")
 public class UpsertDebugResource {
+  private static final Logger LOGGER = LoggerFactory.getLogger(UpsertDebugResource.class);
 
   @Inject
   ServerInstance serverInstance;
@@ -64,6 +68,7 @@ public class UpsertDebugResource {
     if (tableDataManager == null) {
       return "no table for " + tableName;
     }
+    /*
     SegmentDataManager segmentDataManager = null;
     try {
       segmentDataManager = tableDataManager.acquireSegment(segmentName);
@@ -73,12 +78,20 @@ public class UpsertDebugResource {
       if (!(segmentDataManager instanceof UpsertSegmentDataManager)) {
         return "it is not an upsert table";
       } else {
-        return ((UpsertSegmentDataManager) segmentDataManager).getVirtualColumnInfo(Long.parseLong(offsetStr));
+        long offset = Long.parseLong(offsetStr);
+        LOGGER.info("getting virtual column for table {} segment {} offset {}", tableName, segmentName, offset);
+        return ( segmentDataManager).getVirtualColumnInfo(Long.parseLong(offsetStr));
       }
+    } catch (Exception ex) {
+      LOGGER.error("failed to fetch virtual column info", ex);
+      throw new WebApplicationException("Failed to fetch virtual column info" + ex.getMessage(),
+          Response.Status.INTERNAL_SERVER_ERROR);
     } finally {
       if (segmentDataManager != null) {
         tableDataManager.releaseSegment(segmentDataManager);
       }
     }
+    */
+    return "";
   }
 }


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


[incubator-pinot] 03/09: temp update for adding bunch of stuff

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jamesshao pushed a commit to branch upsert-refactor
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git

commit 92bba12ca1bf14207e8eec2b46ee1c7cc50946a4
Author: james Shao <sj...@uber.com>
AuthorDate: Mon Mar 16 11:58:46 2020 -0700

    temp update for adding bunch of stuff
---
 .../core/data/manager/BaseTableDataManager.java    |  10 +-
 .../core/data/manager/InstanceDataManager.java     |   8 +-
 .../pinot/core/data/manager/TableDataManager.java  |  10 +-
 .../offline/ImmutableSegmentDataManager.java       |  17 +++
 .../manager/offline/OfflineTableDataManager.java   |  18 ++-
 .../manager/offline/TableDataManagerProvider.java  |  26 +++--
 .../offline/UpsertImmutableSegmentDataManager.java |  67 -----------
 .../AppendLLRealtimeSegmentDataManager.java        |  58 ---------
 .../realtime/AppendRealtimeTableDataManager.java   |  55 ---------
 .../realtime/HLRealtimeSegmentDataManager.java     |   5 +-
 .../realtime/LLRealtimeSegmentDataManager.java     |  26 +++--
 .../manager/realtime/RealtimeTableDataManager.java |  48 +++++---
 .../realtime/UpsertRealtimeTableDataManager.java   |  82 -------------
 .../manager/upsert/DataManagerCallback.java}       |  32 ++---
 .../upsert/DefaultDataManagerCallbackImpl.java     |  71 +++++++++++
 .../upsert/DefaultIndexSegmentCallback.java        |  65 +++++++++++
 .../DefaultTableDataManagerCallbackImpl.java}      |  36 +++---
 .../manager/upsert/IndexSegmentCallback.java}      |  32 ++---
 .../manager/upsert/TableDataManagerCallback.java}  |  20 +++-
 .../upsert/TableDataManagerCallbackProvider.java   |  91 +++++++++++++++
 .../immutable/ImmutableSegmentImpl.java            |  32 +++--
 .../immutable/ImmutableSegmentLoader.java          |  26 ++---
 .../indexsegment/mutable/MutableSegmentImpl.java   |  24 ++--
 ...entUpdaterConfig.java => WatermarkManager.java} |  16 ++-
 .../data/manager/BaseTableDataManagerTest.java     |  20 ++--
 .../realtime/LLRealtimeSegmentDataManagerTest.java |   5 +-
 .../mutable/MutableSegmentImplTestUtils.java       |   3 +-
 .../SegmentGenerationWithNullValueVectorTest.java  |   3 +-
 .../core/segment/index/loader/LoaderTest.java      |  10 +-
 .../pinot/queries/SerializedBytesQueriesTest.java  |   2 +
 .../pinot/query/executor/QueryExecutorTest.java    |   3 +-
 pinot-grigio/{ => pinot-grigio-provided}/pom.xml   |  49 +++++---
 .../upsert/UpsertDataManagerCallbackImpl.java      | 106 +++++++++--------
 .../UpsertImmutableIndexSegmentCallback.java       | 130 ++++++++++-----------
 .../upsert/UpsertMutableIndexSegmentCallback.java  |  55 +++++----
 .../upsert/UpsertTableDataManagerCallbackImpl.java |  63 ++++++++++
 .../pinot/core/segment/updater/SegmentUpdater.java |  28 ++---
 .../core/segment/updater/SegmentUpdaterConfig.java |   0
 .../segment/updater/UpsertWatermarkManager.java    |  20 ++--
 .../UpsertImmutableIndexSegmentCallbackTest.java   |  17 ++-
 pinot-grigio/pom.xml                               |   1 +
 .../starter/helix/HelixInstanceDataManager.java    |  16 ++-
 .../server/starter/helix/HelixServerStarter.java   |   2 +
 .../SegmentOnlineOfflineStateModelFactory.java     |   9 ++
 .../upsert/DefaultUpsertComponentContainer.java    |  34 ++++--
 .../server/upsert/SegmentDeletionHandler.java      |  55 +++++++++
 .../server/upsert/UpsertComponentContainer.java    |  20 +++-
 .../upsert/UpsertComponentContainerProvider.java   |  58 +++++++++
 .../apache/pinot/server/api/BaseResourceTest.java  |   3 +-
 .../realtime/provisioning/MemoryEstimator.java     |   9 +-
 50 files changed, 961 insertions(+), 635 deletions(-)

diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java
index a39bdfc..cb278b4 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java
@@ -33,6 +33,7 @@ import org.apache.pinot.common.metrics.ServerMeter;
 import org.apache.pinot.common.metrics.ServerMetrics;
 import org.apache.pinot.core.data.manager.config.TableDataManagerConfig;
 import org.apache.pinot.core.data.manager.offline.ImmutableSegmentDataManager;
+import org.apache.pinot.core.data.manager.upsert.DataManagerCallback;
 import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
 import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
 import org.slf4j.Logger;
@@ -108,14 +109,14 @@ public abstract class BaseTableDataManager implements TableDataManager {
    * @param immutableSegment Immutable segment to add
    */
   @Override
-  public void addSegment(ImmutableSegment immutableSegment) {
+  public void addSegment(ImmutableSegment immutableSegment, DataManagerCallback dataManagerCallback) {
     String segmentName = immutableSegment.getSegmentName();
     _logger.info("Adding immutable segment: {} to table: {}", segmentName, _tableNameWithType);
     _serverMetrics.addValueToTableGauge(_tableNameWithType, ServerGauge.DOCUMENT_COUNT,
         immutableSegment.getSegmentMetadata().getTotalRawDocs());
     _serverMetrics.addValueToTableGauge(_tableNameWithType, ServerGauge.SEGMENT_COUNT, 1L);
 
-    ImmutableSegmentDataManager newSegmentManager = getImmutableSegmentDataManager(immutableSegment);
+    ImmutableSegmentDataManager newSegmentManager = getImmutableSegmentDataManager(immutableSegment, dataManagerCallback);
     SegmentDataManager oldSegmentManager = _segmentDataManagerMap.put(segmentName, newSegmentManager);
     if (oldSegmentManager == null) {
       _logger.info("Added new immutable segment: {} to table: {}", segmentName, _tableNameWithType);
@@ -212,7 +213,8 @@ public abstract class BaseTableDataManager implements TableDataManager {
     return _tableNameWithType;
   }
 
-  protected ImmutableSegmentDataManager getImmutableSegmentDataManager(ImmutableSegment immutableSegment) {
-    return new ImmutableSegmentDataManager(immutableSegment);
+  protected ImmutableSegmentDataManager getImmutableSegmentDataManager(ImmutableSegment immutableSegment,
+      DataManagerCallback callback) {
+    return new ImmutableSegmentDataManager(immutableSegment, callback);
   }
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/InstanceDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/InstanceDataManager.java
index 175f5a7..eba0689 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/InstanceDataManager.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/InstanceDataManager.java
@@ -132,8 +132,8 @@ public interface InstanceDataManager {
    */
   ZkHelixPropertyStore<ZNRecord> getPropertyStore();
 
-  /**
-   * Return the mappings from partition -> low water marks of all the tables hosted in this server.
-   */
-  Map<String, Map<Integer, Long>> getLowWaterMarks();
+//  /**
+//   * Return the mappings from partition -> low water marks of all the tables hosted in this server.
+//   */
+//  Map<String, Map<Integer, Long>> getLowWaterMarks();
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/TableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/TableDataManager.java
index 7b1c79c..46aa745 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/TableDataManager.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/TableDataManager.java
@@ -27,6 +27,8 @@ import org.apache.helix.store.zk.ZkHelixPropertyStore;
 import org.apache.pinot.common.config.TableConfig;
 import org.apache.pinot.common.metrics.ServerMetrics;
 import org.apache.pinot.core.data.manager.config.TableDataManagerConfig;
+import org.apache.pinot.core.data.manager.upsert.DataManagerCallback;
+import org.apache.pinot.core.data.manager.upsert.TableDataManagerCallback;
 import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
 import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
 
@@ -58,7 +60,7 @@ public interface TableDataManager {
   /**
    * Adds a loaded immutable segment into the table.
    */
-  void addSegment(ImmutableSegment immutableSegment);
+  void addSegment(ImmutableSegment immutableSegment, DataManagerCallback dataManagerCallback);
 
   /**
    * Adds a segment from local disk into the OFFLINE table.
@@ -116,4 +118,10 @@ public interface TableDataManager {
    * Returns the table name managed by this instance.
    */
   String getTableName();
+
+  /**
+   * get the callback object for this table, either no-op object for the regular table or upsert callback for
+   * upsert-enabled table
+   */
+  TableDataManagerCallback getTableDataManagerCallback();
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/ImmutableSegmentDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/ImmutableSegmentDataManager.java
index 0401f61..949affe 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/ImmutableSegmentDataManager.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/ImmutableSegmentDataManager.java
@@ -18,9 +18,14 @@
  */
 package org.apache.pinot.core.data.manager.offline;
 
+import org.apache.commons.lang3.exception.ExceptionUtils;
 import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.manager.upsert.DataManagerCallback;
+import org.apache.pinot.core.data.manager.upsert.DefaultDataManagerCallbackImpl;
 import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
 
+import java.io.IOException;
+
 
 /**
  * Segment data manager for immutable segment.
@@ -28,9 +33,20 @@ import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
 public class ImmutableSegmentDataManager extends SegmentDataManager {
 
   protected final ImmutableSegment _immutableSegment;
+  protected final DataManagerCallback _dataManagerCallback;
 
   public ImmutableSegmentDataManager(ImmutableSegment immutableSegment) {
+    this(immutableSegment, DefaultDataManagerCallbackImpl.INSTANCE);
+  }
+
+  public ImmutableSegmentDataManager(ImmutableSegment immutableSegment, DataManagerCallback dataManagerCallback) {
     _immutableSegment = immutableSegment;
+    _dataManagerCallback = dataManagerCallback;
+    try {
+      _dataManagerCallback.initVirtualColumns();
+    } catch (IOException ex) {
+      ExceptionUtils.rethrow(ex);
+    }
   }
 
   @Override
@@ -45,6 +61,7 @@ public class ImmutableSegmentDataManager extends SegmentDataManager {
 
   @Override
   public void destroy() {
+    _dataManagerCallback.destroy();
     _immutableSegment.destroy();
   }
 
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/OfflineTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/OfflineTableDataManager.java
index c2c9351..61230a4 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/OfflineTableDataManager.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/OfflineTableDataManager.java
@@ -20,6 +20,9 @@ package org.apache.pinot.core.data.manager.offline;
 
 import java.io.File;
 import javax.annotation.concurrent.ThreadSafe;
+
+import org.apache.pinot.core.data.manager.upsert.DataManagerCallback;
+import org.apache.pinot.core.data.manager.upsert.TableDataManagerCallback;
 import org.apache.pinot.spi.data.Schema;
 import org.apache.pinot.common.metadata.ZKMetadataProvider;
 import org.apache.pinot.core.data.manager.BaseTableDataManager;
@@ -33,6 +36,12 @@ import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
 @ThreadSafe
 public class OfflineTableDataManager extends BaseTableDataManager {
 
+  private TableDataManagerCallback _tableDataManagerCallback;
+
+  public OfflineTableDataManager(TableDataManagerCallback tableDataManagerCallback) {
+    _tableDataManagerCallback = tableDataManagerCallback;
+  }
+
   @Override
   protected void doInit() {
   }
@@ -49,6 +58,13 @@ public class OfflineTableDataManager extends BaseTableDataManager {
   public void addSegment(File indexDir, IndexLoadingConfig indexLoadingConfig)
       throws Exception {
     Schema schema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
-    addSegment(ImmutableSegmentLoader.load(indexDir, indexLoadingConfig, schema));
+    DataManagerCallback callback = _tableDataManagerCallback.getDefaultDataManagerCallback();
+    addSegment(ImmutableSegmentLoader.load(indexDir, indexLoadingConfig,
+        callback, schema), callback);
+  }
+
+  @Override
+  public TableDataManagerCallback getTableDataManagerCallback() {
+    return _tableDataManagerCallback;
   }
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/TableDataManagerProvider.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/TableDataManagerProvider.java
index 3ec1bbd..f5b009f 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/TableDataManagerProvider.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/TableDataManagerProvider.java
@@ -18,6 +18,8 @@
  */
 package org.apache.pinot.core.data.manager.offline;
 
+import com.google.common.base.Preconditions;
+import org.apache.commons.configuration.PropertiesConfiguration;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.store.zk.ZkHelixPropertyStore;
 import org.apache.pinot.common.metrics.ServerMetrics;
@@ -25,8 +27,8 @@ import org.apache.pinot.common.utils.CommonConstants;
 import org.apache.pinot.core.data.manager.TableDataManager;
 import org.apache.pinot.core.data.manager.config.InstanceDataManagerConfig;
 import org.apache.pinot.core.data.manager.config.TableDataManagerConfig;
-import org.apache.pinot.core.data.manager.realtime.AppendRealtimeTableDataManager;
-import org.apache.pinot.core.data.manager.realtime.UpsertRealtimeTableDataManager;
+import org.apache.pinot.core.data.manager.realtime.RealtimeTableDataManager;
+import org.apache.pinot.core.data.manager.upsert.TableDataManagerCallbackProvider;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -41,11 +43,18 @@ public class TableDataManagerProvider {
   private static final Logger LOGGER = LoggerFactory.getLogger(TableDataManagerProvider.class);
 
   private static Semaphore _segmentBuildSemaphore;
+  private static TableDataManagerCallbackProvider _tableDataManagerCallbackProvider;
 
   private TableDataManagerProvider() {
   }
 
   public static void init(InstanceDataManagerConfig instanceDataManagerConfig) {
+    init(instanceDataManagerConfig, new TableDataManagerCallbackProvider(new PropertiesConfiguration()));
+  }
+
+  public static void init(InstanceDataManagerConfig instanceDataManagerConfig,
+                          TableDataManagerCallbackProvider callbackProvider) {
+    _tableDataManagerCallbackProvider = callbackProvider;
     int maxParallelBuilds = instanceDataManagerConfig.getMaxParallelSegmentBuilds();
     if (maxParallelBuilds > 0) {
       _segmentBuildSemaphore = new Semaphore(maxParallelBuilds, true);
@@ -55,17 +64,16 @@ public class TableDataManagerProvider {
   public static TableDataManager getTableDataManager(@Nonnull TableDataManagerConfig tableDataManagerConfig,
       @Nonnull String instanceId, @Nonnull ZkHelixPropertyStore<ZNRecord> propertyStore,
       @Nonnull ServerMetrics serverMetrics) {
-    TableDataManager tableDataManager;
+    Preconditions.checkNotNull(_tableDataManagerCallbackProvider, "callback provider is not init");
+    final TableDataManager tableDataManager;
     switch (CommonConstants.Helix.TableType.valueOf(tableDataManagerConfig.getTableDataManagerType())) {
       case OFFLINE:
-        tableDataManager = new OfflineTableDataManager();
+        tableDataManager = new OfflineTableDataManager(
+            _tableDataManagerCallbackProvider.getDefaultTableDataManagerCallback());
         break;
       case REALTIME:
-        if (tableDataManagerConfig.getUpdateSemantic() == CommonConstants.UpdateSemantic.UPSERT) {
-          tableDataManager = new UpsertRealtimeTableDataManager(_segmentBuildSemaphore);
-        } else {
-          tableDataManager = new AppendRealtimeTableDataManager(_segmentBuildSemaphore);
-        }
+        tableDataManager = new RealtimeTableDataManager(_segmentBuildSemaphore,
+            _tableDataManagerCallbackProvider.getTableDataManagerCallback(tableDataManagerConfig));
         break;
       default:
         throw new IllegalStateException();
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/UpsertImmutableSegmentDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/UpsertImmutableSegmentDataManager.java
deleted file mode 100644
index 790e912..0000000
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/UpsertImmutableSegmentDataManager.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.pinot.core.data.manager.offline;
-
-import org.apache.pinot.common.config.TableNameBuilder;
-import org.apache.pinot.common.utils.CommonConstants;
-import org.apache.pinot.common.utils.LLCSegmentName;
-import org.apache.pinot.core.data.manager.UpsertSegmentDataManager;
-import org.apache.pinot.core.indexsegment.UpsertSegment;
-import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
-import org.apache.pinot.core.segment.updater.SegmentUpdater;
-import org.apache.pinot.grigio.common.storageProvider.UpdateLogEntry;
-
-import java.io.IOException;
-import java.util.List;
-
-public class UpsertImmutableSegmentDataManager extends ImmutableSegmentDataManager implements UpsertSegmentDataManager {
-
-  private String _tableNameWithType;
-
-  public UpsertImmutableSegmentDataManager(ImmutableSegment immutableSegment) throws IOException {
-    super(immutableSegment);
-    _tableNameWithType = TableNameBuilder.ensureTableNameWithType(immutableSegment.getSegmentMetadata().getTableName(),
-        CommonConstants.Helix.TableType.REALTIME);
-    initVirtualColumns();
-  }
-
-  @Override
-  public void updateVirtualColumns(List<UpdateLogEntry> messages) {
-    ((UpsertSegment) _immutableSegment).updateVirtualColumn(messages);
-  }
-
-  @Override
-  public String getVirtualColumnInfo(long offset) {
-    return ((UpsertSegment) _immutableSegment).getVirtualColumnInfo(offset);
-  }
-
-  @Override
-  public void destroy() {
-    SegmentUpdater.getInstance().removeSegmentDataManager(_tableNameWithType, getSegmentName(), this);
-    super.destroy();
-  }
-
-  private void initVirtualColumns() throws IOException {
-    // 1. add listener for update events
-    // 2. load all existing messages
-    // ensure the above orders so we can ensure all events are received by this data manager
-    SegmentUpdater.getInstance().addSegmentDataManager(_tableNameWithType, new LLCSegmentName(getSegmentName()), this);
-    ((UpsertSegment) _immutableSegment).initVirtualColumn();
-  }
-}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/AppendLLRealtimeSegmentDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/AppendLLRealtimeSegmentDataManager.java
deleted file mode 100644
index f725cc8..0000000
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/AppendLLRealtimeSegmentDataManager.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.pinot.core.data.manager.realtime;
-
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
-import org.apache.pinot.common.metrics.ServerMetrics;
-import org.apache.pinot.common.utils.LLCSegmentName;
-import org.apache.pinot.core.indexsegment.mutable.MutableAppendSegmentImpl;
-import org.apache.pinot.core.indexsegment.mutable.MutableSegmentImpl;
-import org.apache.pinot.core.realtime.impl.RealtimeSegmentConfig;
-import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
-import org.apache.pinot.spi.data.Schema;
-import org.apache.pinot.spi.data.readers.GenericRow;
-
-import java.util.concurrent.Semaphore;
-
-public class AppendLLRealtimeSegmentDataManager extends LLRealtimeSegmentDataManager {
-
-  public AppendLLRealtimeSegmentDataManager(RealtimeSegmentZKMetadata segmentZKMetadata, TableConfig tableConfig,
-      RealtimeTableDataManager realtimeTableDataManager, String resourceDataDir, IndexLoadingConfig indexLoadingConfig,
-      Schema schema, LLCSegmentName llcSegmentName, Semaphore partitionConsumerSemaphore, ServerMetrics serverMetrics) {
-    super(segmentZKMetadata, tableConfig, realtimeTableDataManager, resourceDataDir, indexLoadingConfig, schema,
-            llcSegmentName, partitionConsumerSemaphore, serverMetrics);
-  }
-
-  @Override
-  protected MutableSegmentImpl createMutableSegment(RealtimeSegmentConfig config) {
-    return new MutableAppendSegmentImpl(config);
-  }
-
-  @Override
-  protected void processTransformedRow(GenericRow row, long offset) {
-    // do nothing
-  }
-
-  @Override
-  protected void postIndexProcessing(GenericRow row, long offset) {
-    // do nothing
-  }
-
-}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/AppendRealtimeTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/AppendRealtimeTableDataManager.java
deleted file mode 100644
index 741fb08..0000000
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/AppendRealtimeTableDataManager.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.pinot.core.data.manager.realtime;
-
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.metadata.instance.InstanceZKMetadata;
-import org.apache.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
-import org.apache.pinot.common.metrics.ServerMetrics;
-import org.apache.pinot.common.utils.LLCSegmentName;
-import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
-import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
-import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
-import org.apache.pinot.spi.data.Schema;
-
-import java.io.File;
-import java.util.concurrent.Semaphore;
-
-public class AppendRealtimeTableDataManager extends RealtimeTableDataManager {
-
-  public AppendRealtimeTableDataManager(Semaphore segmentBuildSemaphore) {
-    super(segmentBuildSemaphore);
-  }
-
-  @Override
-  protected LLRealtimeSegmentDataManager getLLRealtimeSegmentDataManager(
-          RealtimeSegmentZKMetadata realtimeSegmentZKMetadata, TableConfig tableConfig,
-          RealtimeTableDataManager realtimeTableDataManager, String indexDirPath, IndexLoadingConfig indexLoadingConfig,
-          Schema schema, LLCSegmentName llcSegmentName, Semaphore partitionConsumerSemaphore, ServerMetrics serverMetrics)
-      throws Exception {
-    return new AppendLLRealtimeSegmentDataManager(realtimeSegmentZKMetadata, tableConfig, realtimeTableDataManager,
-            indexDirPath, indexLoadingConfig, schema, llcSegmentName, partitionConsumerSemaphore, serverMetrics);
-  }
-
-  @Override
-  protected ImmutableSegment loadImmutableSegment(File indexDir, IndexLoadingConfig indexLoadingConfig, Schema schema)
-      throws Exception {
-    return ImmutableSegmentLoader.load(indexDir, indexLoadingConfig, schema);
-  }
-}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/HLRealtimeSegmentDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/HLRealtimeSegmentDataManager.java
index 086812e..d767e98 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/HLRealtimeSegmentDataManager.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/HLRealtimeSegmentDataManager.java
@@ -30,7 +30,7 @@ import java.util.concurrent.atomic.AtomicLong;
 import org.apache.commons.io.FileUtils;
 import org.apache.pinot.common.config.IndexingConfig;
 import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.core.indexsegment.mutable.MutableAppendSegmentImpl;
+import org.apache.pinot.core.data.manager.upsert.DefaultDataManagerCallbackImpl;
 import org.apache.pinot.spi.data.Schema;
 import org.apache.pinot.common.metadata.instance.InstanceZKMetadata;
 import org.apache.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
@@ -195,7 +195,8 @@ public class HLRealtimeSegmentDataManager extends RealtimeSegmentDataManager {
                 indexLoadingConfig.isDirectRealtimeOffheapAllocation(), serverMetrics))
             .setStatsHistory(realtimeTableDataManager.getStatsHistory())
             .setNullHandlingEnabled(indexingConfig.isNullHandlingEnabled()).build();
-    realtimeSegment = new MutableAppendSegmentImpl(realtimeSegmentConfig);
+    realtimeSegment = new MutableSegmentImpl(realtimeSegmentConfig,
+        DefaultDataManagerCallbackImpl.INSTANCE.getIndexSegmentCallback());
 
 
     notifier = realtimeTableDataManager;
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java
index 7765e9e..4f3ae50 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java
@@ -50,6 +50,7 @@ import org.apache.pinot.common.protocols.SegmentCompletionProtocol;
 import org.apache.pinot.common.utils.CommonConstants.Segment.Realtime.CompletionMode;
 import org.apache.pinot.common.utils.LLCSegmentName;
 import org.apache.pinot.common.utils.TarGzCompressionUtils;
+import org.apache.pinot.core.data.manager.upsert.DataManagerCallback;
 import org.apache.pinot.core.data.recordtransformer.CompositeTransformer;
 import org.apache.pinot.core.data.recordtransformer.RecordTransformer;
 import org.apache.pinot.core.indexsegment.generator.SegmentVersion;
@@ -83,7 +84,7 @@ import org.slf4j.LoggerFactory;
 /**
  * Segment data manager for low level consumer realtime segments, which manages consumption and segment completion.
  */
-public abstract class LLRealtimeSegmentDataManager extends RealtimeSegmentDataManager {
+public class LLRealtimeSegmentDataManager extends RealtimeSegmentDataManager {
   protected enum State {
     // The state machine starts off with this state. While in this state we consume stream events
     // and index them in memory. We continue to be in this state until the end criteria is satisfied
@@ -272,6 +273,9 @@ public abstract class LLRealtimeSegmentDataManager extends RealtimeSegmentDataMa
   private final boolean _nullHandlingEnabled;
   private final SegmentCommitterFactory _segmentCommitterFactory;
 
+  // upsert/append related components
+  private final DataManagerCallback _dataManagerCallback;
+
   // TODO each time this method is called, we print reason for stop. Good to print only once.
   private boolean endCriteriaReached() {
     Preconditions.checkState(_state.shouldConsume(), "Incorrect state %s", _state);
@@ -416,6 +420,7 @@ public abstract class LLRealtimeSegmentDataManager extends RealtimeSegmentDataMa
       _serverMetrics.addMeteredTableValue(_metricKeyName, ServerMeter.ROWS_WITH_ERRORS, _numRowsErrored);
       _serverMetrics.addMeteredTableValue(_tableStreamName, ServerMeter.ROWS_WITH_ERRORS, _numRowsErrored);
     }
+    _dataManagerCallback.postConsumeLoop();
     return true;
   }
 
@@ -467,14 +472,14 @@ public abstract class LLRealtimeSegmentDataManager extends RealtimeSegmentDataMa
           GenericRow transformedRow = _recordTransformer.transform(decodedRow);
 
           if (transformedRow != null) {
-            processTransformedRow(transformedRow, _currentOffset);
+            _dataManagerCallback.processTransformedRow(transformedRow, _currentOffset);
             realtimeRowsConsumedMeter =
                 _serverMetrics.addMeteredTableValue(_metricKeyName, ServerMeter.REALTIME_ROWS_CONSUMED, 1,
                     realtimeRowsConsumedMeter);
             indexedMessageCount++;
 
             canTakeMore = _realtimeSegment.index(transformedRow, msgMetadata);
-            postIndexProcessing(transformedRow, _currentOffset);
+            _dataManagerCallback.postIndexProcessing(transformedRow, _currentOffset);
           } else {
             realtimeRowsDroppedMeter = _serverMetrics
                 .addMeteredTableValue(_metricKeyName, ServerMeter.INVALID_REALTIME_ROWS_DROPPED, 1,
@@ -505,11 +510,6 @@ public abstract class LLRealtimeSegmentDataManager extends RealtimeSegmentDataMa
     }
   }
 
-  protected abstract void processTransformedRow(GenericRow row, long offset);
-
-
-  protected abstract void postIndexProcessing(GenericRow row, long offset);
-
   public class PartitionConsumer implements Runnable {
     public void run() {
       long initialConsumptionEnd = 0L;
@@ -1017,6 +1017,7 @@ public abstract class LLRealtimeSegmentDataManager extends RealtimeSegmentDataMa
   }
 
   public void destroy() {
+    _dataManagerCallback.destroy();
     try {
       stop();
     } catch (InterruptedException e) {
@@ -1054,7 +1055,8 @@ public abstract class LLRealtimeSegmentDataManager extends RealtimeSegmentDataMa
   // If the transition is OFFLINE to ONLINE, the caller should have downloaded the segment and we don't reach here.
   public LLRealtimeSegmentDataManager(RealtimeSegmentZKMetadata segmentZKMetadata, TableConfig tableConfig,
       RealtimeTableDataManager realtimeTableDataManager, String resourceDataDir, IndexLoadingConfig indexLoadingConfig,
-      Schema schema, LLCSegmentName llcSegmentName, Semaphore partitionConsumerSemaphore, ServerMetrics serverMetrics) {
+      Schema schema, LLCSegmentName llcSegmentName, Semaphore partitionConsumerSemaphore, ServerMetrics serverMetrics,
+      DataManagerCallback dataManagerCallback) {
     _segBuildSemaphore = realtimeTableDataManager.getSegmentBuildSemaphore();
     _segmentZKMetadata = (LLCRealtimeSegmentZKMetadata) segmentZKMetadata;
     _tableConfig = tableConfig;
@@ -1064,6 +1066,7 @@ public abstract class LLRealtimeSegmentDataManager extends RealtimeSegmentDataMa
     _indexLoadingConfig = indexLoadingConfig;
     _schema = schema;
     _serverMetrics = serverMetrics;
+    _dataManagerCallback = dataManagerCallback;
     _segmentVersion = indexLoadingConfig.getSegmentVersion();
     _instanceId = _realtimeTableDataManager.getServerInstance();
     _leaseExtender = SegmentBuildTimeLeaseExtender.getLeaseExtender(_instanceId);
@@ -1185,7 +1188,8 @@ public abstract class LLRealtimeSegmentDataManager extends RealtimeSegmentDataMa
       }
     }
 
-    _realtimeSegment = createMutableSegment(realtimeSegmentConfigBuilder.build());
+    _realtimeSegment = new MutableSegmentImpl(realtimeSegmentConfigBuilder.build(),
+        _dataManagerCallback.getIndexSegmentCallback());
     _startOffset = _segmentZKMetadata.getStartOffset();
     _currentOffset = _startOffset;
     _resourceTmpDir = new File(resourceDataDir, "_tmp");
@@ -1221,8 +1225,6 @@ public abstract class LLRealtimeSegmentDataManager extends RealtimeSegmentDataMa
     start();
   }
 
-  protected abstract MutableSegmentImpl createMutableSegment(RealtimeSegmentConfig config);
-
   /**
    * Creates a new stream consumer
    * @param reason
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java
index 2be3ca1..bc5316d 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java
@@ -48,6 +48,8 @@ import org.apache.pinot.common.utils.TarGzCompressionUtils;
 import org.apache.pinot.common.utils.fetcher.SegmentFetcherFactory;
 import org.apache.pinot.core.data.manager.BaseTableDataManager;
 import org.apache.pinot.core.data.manager.SegmentDataManager;
+import org.apache.pinot.core.data.manager.upsert.DataManagerCallback;
+import org.apache.pinot.core.data.manager.upsert.TableDataManagerCallback;
 import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
 import org.apache.pinot.core.realtime.impl.RealtimeSegmentStatsHistory;
 import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
@@ -59,12 +61,13 @@ import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
 
 
 @ThreadSafe
-public abstract class RealtimeTableDataManager extends BaseTableDataManager {
+public class RealtimeTableDataManager extends BaseTableDataManager {
   private final ExecutorService _segmentAsyncExecutorService =
       Executors.newSingleThreadExecutor(new NamedThreadFactory("SegmentAsyncExecutorService"));
   private SegmentBuildTimeLeaseExtender _leaseExtender;
   private RealtimeSegmentStatsHistory _statsHistory;
   private final Semaphore _segmentBuildSemaphore;
+  private final TableDataManagerCallback _tableDataManagerCallback;
   // Maintains a map of partitionIds to semaphores.
   // The semaphore ensures that exactly one PartitionConsumer instance consumes from any stream partition.
   // In some streams, it's possible that having multiple consumers (with the same consumer name on the same host) consuming from the same stream partition can lead to bugs.
@@ -90,12 +93,14 @@ public abstract class RealtimeTableDataManager extends BaseTableDataManager {
   // likely that we get fresh data each time instead of multiple copies of roughly same data.
   private static final int MIN_INTERVAL_BETWEEN_STATS_UPDATES_MINUTES = 30;
 
-  public RealtimeTableDataManager(Semaphore segmentBuildSemaphore) {
+  public RealtimeTableDataManager(Semaphore segmentBuildSemaphore, TableDataManagerCallback tableDataManagerCallback) {
     _segmentBuildSemaphore = segmentBuildSemaphore;
+    _tableDataManagerCallback = tableDataManagerCallback;
   }
 
   @Override
   protected void doInit() {
+    _tableDataManagerCallback.init();
     _leaseExtender = SegmentBuildTimeLeaseExtender.create(_instanceId, _serverMetrics, _tableNameWithType);
 
     File statsFile = new File(_tableDataDir, STATS_FILE_NAME);
@@ -227,7 +232,9 @@ public abstract class RealtimeTableDataManager extends BaseTableDataManager {
     if (indexDir.exists() && (realtimeSegmentZKMetadata.getStatus() == Status.DONE)) {
       // Segment already exists on disk, and metadata has been committed. Treat it like an offline segment
 
-      addSegment(loadImmutableSegment(indexDir, indexLoadingConfig, schema));
+      DataManagerCallback callback = _tableDataManagerCallback
+          .getDataManagerCallback(_tableNameWithType, segmentName, schema, _serverMetrics, false);
+      addSegment(loadImmutableSegment(indexDir, indexLoadingConfig, schema, callback), callback);
     } else {
       // Either we don't have the segment on disk or we have not committed in ZK. We should be starting the consumer
       // for realtime segment here. If we wrote it on disk but could not get to commit to zk yet, we should replace the
@@ -257,15 +264,21 @@ public abstract class RealtimeTableDataManager extends BaseTableDataManager {
         LLCSegmentName llcSegmentName = new LLCSegmentName(segmentName);
         int streamPartitionId = llcSegmentName.getPartitionId();
         _partitionIdToSemaphoreMap.putIfAbsent(streamPartitionId, new Semaphore(1));
-        manager = getLLRealtimeSegmentDataManager(realtimeSegmentZKMetadata, tableConfig, this,
-                _indexDir.getAbsolutePath(), indexLoadingConfig, schema, llcSegmentName,
-                _partitionIdToSemaphoreMap.get(streamPartitionId), _serverMetrics);
+        manager = new LLRealtimeSegmentDataManager(realtimeSegmentZKMetadata, tableConfig,
+            this, _indexDir.getAbsolutePath(), indexLoadingConfig, schema, llcSegmentName,
+            _partitionIdToSemaphoreMap.get(streamPartitionId), _serverMetrics,
+            _tableDataManagerCallback.getDataManagerCallback(_tableNameWithType, segmentName, schema, _serverMetrics, true));
       }
       _logger.info("Initialize RealtimeSegmentDataManager - " + segmentName);
       _segmentDataManagerMap.put(segmentName, manager);
     }
   }
 
+  @Override
+  public TableDataManagerCallback getTableDataManagerCallback() {
+    return _tableDataManagerCallback;
+  }
+
   public void downloadAndReplaceSegment(String segmentName, LLCRealtimeSegmentZKMetadata llcSegmentMetadata,
       IndexLoadingConfig indexLoadingConfig, Schema schema) {
     final String uri = llcSegmentMetadata.getDownloadUrl();
@@ -297,7 +310,8 @@ public abstract class RealtimeTableDataManager extends BaseTableDataManager {
     ZKMetadataProvider.setRealtimeSegmentZKMetadata(_propertyStore, _tableNameWithType, segmentZKMetadata);
     File indexDir = new File(_indexDir, segmentZKMetadata.getSegmentName());
     Schema schema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
-    addSegment(ImmutableSegmentLoader.load(indexDir, indexLoadingConfig, schema));
+    addSegment(ImmutableSegmentLoader.load(indexDir, indexLoadingConfig),
+        _tableDataManagerCallback.getDefaultDataManagerCallback());
   }
 
   /**
@@ -306,7 +320,9 @@ public abstract class RealtimeTableDataManager extends BaseTableDataManager {
   public void replaceLLSegment(String segmentName, IndexLoadingConfig indexLoadingConfig, Schema schema) {
     try {
       File indexDir = new File(_indexDir, segmentName);
-      addSegment(loadImmutableSegment(indexDir, indexLoadingConfig, schema));
+      DataManagerCallback dataManagerCallback = _tableDataManagerCallback
+          .getDataManagerCallback(_tableNameWithType, segmentName, schema, _serverMetrics, false);
+      addSegment(loadImmutableSegment(indexDir, indexLoadingConfig, schema, dataManagerCallback), dataManagerCallback);
     } catch (Exception e) {
       throw new RuntimeException(e);
     }
@@ -316,17 +332,11 @@ public abstract class RealtimeTableDataManager extends BaseTableDataManager {
     return _instanceId;
   }
 
-  /**
-   *  allow {@link UpsertRealtimeTableDataManager} to override this method
-   */
-  protected abstract LLRealtimeSegmentDataManager getLLRealtimeSegmentDataManager(
-          RealtimeSegmentZKMetadata realtimeSegmentZKMetadata, TableConfig tableConfig,
-          RealtimeTableDataManager realtimeTableDataManager, String indexDirPath, IndexLoadingConfig indexLoadingConfig,
-          Schema schema, LLCSegmentName llcSegmentName, Semaphore partitionConsumerSemaphore, ServerMetrics serverMetrics)
-      throws Exception;
-
-  protected abstract ImmutableSegment loadImmutableSegment(File indexDir, IndexLoadingConfig indexLoadingConfig, Schema schema)
-      throws Exception;
+  protected ImmutableSegment loadImmutableSegment(File indexDir, IndexLoadingConfig indexLoadingConfig, Schema schema,
+                                                  DataManagerCallback dataManagerCallback)
+      throws Exception {
+    return ImmutableSegmentLoader.load(indexDir, indexLoadingConfig, dataManagerCallback, schema);
+  }
 
   /**
    * Validate a schema against the table config for real-time record consumption.
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/UpsertRealtimeTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/UpsertRealtimeTableDataManager.java
deleted file mode 100644
index 12569db..0000000
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/UpsertRealtimeTableDataManager.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.pinot.core.data.manager.realtime;
-
-import org.apache.pinot.common.utils.LLCSegmentName;
-import org.apache.pinot.core.data.manager.offline.UpsertImmutableSegmentDataManager;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.metadata.instance.InstanceZKMetadata;
-import org.apache.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
-import org.apache.pinot.common.metrics.ServerMetrics;
-import org.apache.pinot.core.data.manager.offline.ImmutableSegmentDataManager;
-import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
-import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
-import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
-import org.apache.pinot.grigio.common.storageProvider.UpdateLogStorageProvider;
-import org.apache.pinot.spi.data.Schema;
-
-import javax.annotation.Nonnull;
-import java.io.File;
-import java.io.IOException;
-import java.util.concurrent.Semaphore;
-
-public class UpsertRealtimeTableDataManager extends RealtimeTableDataManager {
-  private UpdateLogStorageProvider _updateLogStorageProvider;
-
-  public UpsertRealtimeTableDataManager(Semaphore segmentBuildSemaphore) {
-    super(segmentBuildSemaphore);
-    _updateLogStorageProvider = UpdateLogStorageProvider.getInstance();
-  }
-
-  @Override
-  public void addSegment(@Nonnull String segmentName, @Nonnull TableConfig tableConfig,
-                         @Nonnull IndexLoadingConfig indexLoadingConfig) throws Exception {
-    _updateLogStorageProvider.addSegment(_tableNameWithType, segmentName);
-    super.addSegment(segmentName, tableConfig, indexLoadingConfig);
-  }
-
-  @Override
-  protected LLRealtimeSegmentDataManager getLLRealtimeSegmentDataManager(
-          RealtimeSegmentZKMetadata realtimeSegmentZKMetadata, TableConfig tableConfig,
-          RealtimeTableDataManager realtimeTableDataManager, String indexDirPath, IndexLoadingConfig indexLoadingConfig,
-          Schema schema, LLCSegmentName llcSegmentName, Semaphore partitionConsumerSemaphore, ServerMetrics serverMetrics)
-      throws Exception {
-    return new UpsertLLRealtimeSegmentDataManager(realtimeSegmentZKMetadata, tableConfig, realtimeTableDataManager,
-            indexDirPath, indexLoadingConfig, schema, llcSegmentName, partitionConsumerSemaphore, serverMetrics);
-  }
-
-  @Override
-  protected ImmutableSegmentDataManager getImmutableSegmentDataManager(ImmutableSegment immutableSegment) {
-    try {
-      return new UpsertImmutableSegmentDataManager(immutableSegment);
-    } catch (IOException e) {
-      throw new RuntimeException("failed to init the upsert immutable segment", e);
-    }
-  }
-
-  @Override
-  protected ImmutableSegment loadImmutableSegment(File indexDir, IndexLoadingConfig indexLoadingConfig, Schema schema) {
-    try {
-      return ImmutableSegmentLoader.loadUpsertSegment(indexDir, indexLoadingConfig, schema);
-    } catch (Exception e) {
-      throw new RuntimeException("failed to load immutable segment", e);
-    }
-  }
-
-}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableAppendSegmentImpl.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/DataManagerCallback.java
similarity index 60%
copy from pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableAppendSegmentImpl.java
copy to pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/DataManagerCallback.java
index 1c9effc..466553e 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableAppendSegmentImpl.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/DataManagerCallback.java
@@ -16,22 +16,28 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.core.indexsegment.mutable;
+package org.apache.pinot.core.data.manager.upsert;
 
-import org.apache.pinot.core.realtime.impl.RealtimeSegmentConfig;
+import org.apache.pinot.grigio.common.storageProvider.UpdateLogEntry;
+import org.apache.pinot.spi.data.Schema;
 import org.apache.pinot.spi.data.readers.GenericRow;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
-public class MutableAppendSegmentImpl extends MutableSegmentImpl {
-  private static final Logger LOGGER = LoggerFactory.getLogger(MutableUpsertSegmentImpl.class);
+import java.io.IOException;
+import java.util.List;
 
-  public MutableAppendSegmentImpl(RealtimeSegmentConfig config) {
-    super(config);
-  }
+public interface DataManagerCallback {
 
-  @Override
-  protected void postProcessRecords(GenericRow row, int docId) {
-    // nothing
-  }
+  IndexSegmentCallback getIndexSegmentCallback();
+
+  void processTransformedRow(GenericRow row, long offset);
+
+  void postIndexProcessing(GenericRow row, long offset);
+
+  void postConsumeLoop();
+
+  void initVirtualColumns() throws IOException;
+
+  void updateVirtualColumns(List<UpdateLogEntry> messages);
+
+  void destroy();
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/DefaultDataManagerCallbackImpl.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/DefaultDataManagerCallbackImpl.java
new file mode 100644
index 0000000..347c541
--- /dev/null
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/DefaultDataManagerCallbackImpl.java
@@ -0,0 +1,71 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.data.manager.upsert;
+
+import org.apache.pinot.grigio.common.storageProvider.UpdateLogEntry;
+import org.apache.pinot.spi.data.readers.GenericRow;
+
+import java.io.IOException;
+import java.util.List;
+
+public class DefaultDataManagerCallbackImpl implements DataManagerCallback {
+
+  // expect to use the cached instance  for append mode to reduce memory usage
+  public static final DefaultDataManagerCallbackImpl INSTANCE = new DefaultDataManagerCallbackImpl();
+
+  private final IndexSegmentCallback _indexSegmentCallback;
+
+  private DefaultDataManagerCallbackImpl() {
+    _indexSegmentCallback = DefaultIndexSegmentCallback.INSTANCE;
+  }
+
+  public IndexSegmentCallback getIndexSegmentCallback() {
+    return _indexSegmentCallback;
+  }
+
+  @Override
+  public void processTransformedRow(GenericRow row, long offset) {
+    // do nothing
+  }
+
+  @Override
+  public void postIndexProcessing(GenericRow row, long offset) {
+    // do nothing
+  }
+
+  @Override
+  public void postConsumeLoop() {
+    // do nothing
+  }
+
+  @Override
+  public void initVirtualColumns() throws IOException {
+    // do nothing
+  }
+
+  @Override
+  public void updateVirtualColumns(List<UpdateLogEntry> messages) {
+    // do nothing
+  }
+
+  @Override
+  public void destroy() {
+    // do nothing
+  }
+}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/DefaultIndexSegmentCallback.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/DefaultIndexSegmentCallback.java
new file mode 100644
index 0000000..0299cb7
--- /dev/null
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/DefaultIndexSegmentCallback.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.data.manager.upsert;
+
+import joptsimple.internal.Strings;
+import org.apache.pinot.common.segment.SegmentMetadata;
+import org.apache.pinot.core.io.reader.DataFileReader;
+import org.apache.pinot.core.segment.index.column.ColumnIndexContainer;
+import org.apache.pinot.grigio.common.storageProvider.UpdateLogEntry;
+import org.apache.pinot.spi.data.readers.GenericRow;
+
+import java.util.Map;
+
+public class DefaultIndexSegmentCallback implements IndexSegmentCallback {
+
+  public static final DefaultIndexSegmentCallback INSTANCE = new DefaultIndexSegmentCallback();
+
+  private DefaultIndexSegmentCallback(){}
+
+  @Override
+  public void init(SegmentMetadata segmentMetadata, Map<String, DataFileReader> virtualColumnIndexReader) {
+    // do nothing
+  }
+
+  @Override
+  public void initOffsetColumn(ColumnIndexContainer offsetColumnContainer) {
+    // do noting
+  }
+
+  @Override
+  public void postProcessRecords(GenericRow row, int docId) {
+    // do nothing
+  }
+
+  @Override
+  public void initVirtualColumn() {
+    // do nothing
+  }
+
+  @Override
+  public void updateVirtualColumn(Iterable<UpdateLogEntry> logEntries) {
+    // do nothing
+  }
+
+  @Override
+  public String getVirtualColumnInfo(long offset) {
+    return Strings.EMPTY;
+  }
+}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/ImmutableSegmentDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/DefaultTableDataManagerCallbackImpl.java
similarity index 52%
copy from pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/ImmutableSegmentDataManager.java
copy to pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/DefaultTableDataManagerCallbackImpl.java
index 0401f61..2e56455 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/ImmutableSegmentDataManager.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/DefaultTableDataManagerCallbackImpl.java
@@ -16,40 +16,34 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.core.data.manager.offline;
+package org.apache.pinot.core.data.manager.upsert;
 
-import org.apache.pinot.core.data.manager.SegmentDataManager;
-import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
+import org.apache.pinot.common.config.TableConfig;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.spi.data.Schema;
 
+public class DefaultTableDataManagerCallbackImpl implements TableDataManagerCallback {
 
-/**
- * Segment data manager for immutable segment.
- */
-public class ImmutableSegmentDataManager extends SegmentDataManager {
-
-  protected final ImmutableSegment _immutableSegment;
-
-  public ImmutableSegmentDataManager(ImmutableSegment immutableSegment) {
-    _immutableSegment = immutableSegment;
-  }
+  private static final DefaultDataManagerCallbackImpl DEFAULT_DM_CALLBACK = DefaultDataManagerCallbackImpl.INSTANCE;
 
   @Override
-  public String getSegmentName() {
-    return _immutableSegment.getSegmentName();
+  public void init() {
   }
 
   @Override
-  public ImmutableSegment getSegment() {
-    return _immutableSegment;
+  public void addSegment(String tableName, String segmentName, TableConfig tableConfig) {
   }
 
   @Override
-  public void destroy() {
-    _immutableSegment.destroy();
+  public DataManagerCallback getDataManagerCallback(String tableName, String segmentName,
+      Schema schema, ServerMetrics serverMetrics, boolean isMutable) {
+    return DEFAULT_DM_CALLBACK;
   }
 
   @Override
-  public String toString() {
-    return "ImmutableSegmentDataManager(" + _immutableSegment.getSegmentName() + ")";
+  public DataManagerCallback getDefaultDataManagerCallback() {
+    return DEFAULT_DM_CALLBACK;
   }
+
+
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableAppendSegmentImpl.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/IndexSegmentCallback.java
similarity index 52%
copy from pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableAppendSegmentImpl.java
copy to pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/IndexSegmentCallback.java
index 1c9effc..dd198f9 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableAppendSegmentImpl.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/IndexSegmentCallback.java
@@ -16,22 +16,28 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.core.indexsegment.mutable;
+package org.apache.pinot.core.data.manager.upsert;
 
-import org.apache.pinot.core.realtime.impl.RealtimeSegmentConfig;
+import org.apache.pinot.common.segment.SegmentMetadata;
+import org.apache.pinot.core.io.reader.DataFileReader;
+import org.apache.pinot.core.segment.index.column.ColumnIndexContainer;
+import org.apache.pinot.grigio.common.storageProvider.UpdateLogEntry;
 import org.apache.pinot.spi.data.readers.GenericRow;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
-public class MutableAppendSegmentImpl extends MutableSegmentImpl {
-  private static final Logger LOGGER = LoggerFactory.getLogger(MutableUpsertSegmentImpl.class);
+import java.io.IOException;
+import java.util.Map;
 
-  public MutableAppendSegmentImpl(RealtimeSegmentConfig config) {
-    super(config);
-  }
+public interface IndexSegmentCallback {
 
-  @Override
-  protected void postProcessRecords(GenericRow row, int docId) {
-    // nothing
-  }
+  void init(SegmentMetadata segmentMetadata, Map<String, DataFileReader> virtualColumnIndexReader);
+
+  void initOffsetColumn(ColumnIndexContainer offsetColumnContainer);
+
+  void postProcessRecords(GenericRow row, int docId);
+
+  void initVirtualColumn() throws IOException;
+
+  void updateVirtualColumn(Iterable<UpdateLogEntry> logEntries);
+
+  String getVirtualColumnInfo(long offset);
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdaterConfig.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/TableDataManagerCallback.java
similarity index 59%
copy from pinot-core/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdaterConfig.java
copy to pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/TableDataManagerCallback.java
index 97d48f3..ec6a15f 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdaterConfig.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/TableDataManagerCallback.java
@@ -16,10 +16,20 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.core.segment.updater;
+package org.apache.pinot.core.data.manager.upsert;
 
-public class SegmentUpdaterConfig {
-  public static final String SEGMENT_UDPATE_SLEEP_MS = "sleep.ms";
-  public static final String INPUT_TOPIC_PREFIX  = "input.topic.prefix";
-  public static final int SEGMENT_UDPATE_SLEEP_MS_DEFAULT = 100;
+import org.apache.pinot.common.config.TableConfig;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.spi.data.Schema;
+
+public interface TableDataManagerCallback {
+
+  void init();
+
+  void addSegment(String tableName, String segmentName, TableConfig tableConfig);
+
+  DataManagerCallback getDataManagerCallback(String tableName, String segmentName, Schema schema,
+      ServerMetrics serverMetrics, boolean isMutable);
+
+  DataManagerCallback getDefaultDataManagerCallback();
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/TableDataManagerCallbackProvider.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/TableDataManagerCallbackProvider.java
new file mode 100644
index 0000000..2e868a2
--- /dev/null
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/upsert/TableDataManagerCallbackProvider.java
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.data.manager.upsert;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.configuration.Configuration;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.exception.ExceptionUtils;
+import org.apache.pinot.common.utils.CommonConstants;
+import org.apache.pinot.core.data.manager.config.TableDataManagerConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TableDataManagerCallbackProvider {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(TableDataManagerCallbackProvider.class);
+
+  private Class<TableDataManagerCallback> defaultTableDataManagerCallBackClass;
+  private Class<TableDataManagerCallback> upsertTableDataManagerCallBackClass;
+
+  public static final String UPSERT_CALLBACK_CLASS_CONFIG_KEY = "upsert.tableDataManager.callback";
+  public static final String DEFAULT_CALLBACK_CLASS_CONFIG_KEY = "append.tableDataManager.callback";
+  public static final String CALLBACK_CLASS_CONFIG_DEFAULT = DefaultTableDataManagerCallbackImpl.class.getName();
+
+  public TableDataManagerCallbackProvider(Configuration configuration) {
+    String appendClassName = configuration.getString(DEFAULT_CALLBACK_CLASS_CONFIG_KEY, CALLBACK_CLASS_CONFIG_DEFAULT);
+    String upsertClassName = configuration.getString(UPSERT_CALLBACK_CLASS_CONFIG_KEY);
+    try {
+      defaultTableDataManagerCallBackClass = (Class<TableDataManagerCallback>) Class.forName(appendClassName);
+    } catch (ClassNotFoundException e) {
+      LOGGER.error("failed to load table data manager class {}", appendClassName, e);
+      ExceptionUtils.rethrow(e);
+    }
+    Preconditions.checkState(defaultTableDataManagerCallBackClass.isAssignableFrom(TableDataManagerCallback.class),
+        "configured class not assignable from Callback class", defaultTableDataManagerCallBackClass);
+    if (StringUtils.isNotEmpty(upsertClassName)) {
+      try {
+        upsertTableDataManagerCallBackClass = (Class<TableDataManagerCallback>) Class.forName(upsertClassName);
+      } catch (ClassNotFoundException e) {
+        LOGGER.error("failed to load table data manager class {}", upsertClassName);
+        ExceptionUtils.rethrow(e);
+      }
+      Preconditions.checkState(upsertTableDataManagerCallBackClass.isAssignableFrom(TableDataManagerCallback.class),
+          "configured class not assignable from Callback class");
+    }
+  }
+
+  public TableDataManagerCallback getTableDataManagerCallback(TableDataManagerConfig tableDataManagerConfig) {
+    if (tableDataManagerConfig.getUpdateSemantic() == CommonConstants.UpdateSemantic.UPSERT) {
+      return getUpsertTableDataManagerCallback();
+    } else {
+      return getDefaultTableDataManagerCallback();
+    }
+  }
+
+  public TableDataManagerCallback getUpsertTableDataManagerCallback() {
+    try {
+      return upsertTableDataManagerCallBackClass.newInstance();
+    } catch (Exception ex) {
+      LOGGER.error("failed to initialize new table data manager callback {}", upsertTableDataManagerCallBackClass.getName());
+      ExceptionUtils.rethrow(ex);
+    }
+    return null;
+  }
+
+  public TableDataManagerCallback getDefaultTableDataManagerCallback() {
+    try {
+      return defaultTableDataManagerCallBackClass.newInstance();
+    } catch (Exception ex) {
+      LOGGER.error("failed to initialize new table data manager callback {}", upsertTableDataManagerCallBackClass.getName());
+      ExceptionUtils.rethrow(ex);
+    }
+    return null;
+  }
+}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/immutable/ImmutableSegmentImpl.java b/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/immutable/ImmutableSegmentImpl.java
index 21f056c..5e070ba 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/immutable/ImmutableSegmentImpl.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/immutable/ImmutableSegmentImpl.java
@@ -19,10 +19,11 @@
 package org.apache.pinot.core.indexsegment.immutable;
 
 import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
+import java.util.*;
 import javax.annotation.Nullable;
+
+import org.apache.pinot.core.data.manager.upsert.IndexSegmentCallback;
+import org.apache.pinot.core.segment.virtualcolumn.mutable.VirtualColumnLongValueReaderWriter;
 import org.apache.pinot.spi.data.FieldSpec;
 import org.apache.pinot.spi.data.Schema;
 import org.apache.pinot.spi.data.readers.GenericRow;
@@ -43,18 +44,33 @@ import org.slf4j.LoggerFactory;
 public class ImmutableSegmentImpl implements ImmutableSegment {
   private static final Logger LOGGER = LoggerFactory.getLogger(ImmutableSegmentImpl.class);
 
-  protected final SegmentDirectory _segmentDirectory;
-  protected final SegmentMetadataImpl _segmentMetadata;
-  protected final Map<String, ColumnIndexContainer> _indexContainerMap;
-  protected final StarTreeIndexContainer _starTreeIndexContainer;
+  private final SegmentDirectory _segmentDirectory;
+  private final SegmentMetadataImpl _segmentMetadata;
+  private final Map<String, ColumnIndexContainer> _indexContainerMap;
+  private final StarTreeIndexContainer _starTreeIndexContainer;
+  private final IndexSegmentCallback _segmentCallback;
+
 
   public ImmutableSegmentImpl(SegmentDirectory segmentDirectory, SegmentMetadataImpl segmentMetadata,
       Map<String, ColumnIndexContainer> columnIndexContainerMap,
-      @Nullable StarTreeIndexContainer starTreeIndexContainer) {
+      @Nullable StarTreeIndexContainer starTreeIndexContainer, IndexSegmentCallback segmentCallback) {
     _segmentDirectory = segmentDirectory;
     _segmentMetadata = segmentMetadata;
     _indexContainerMap = columnIndexContainerMap;
     _starTreeIndexContainer = starTreeIndexContainer;
+    _segmentCallback = segmentCallback;
+
+    // create virtualColumnReaderWriter
+    Map<String, DataFileReader> virtualColumnsReaderWriter = new HashMap<>();
+    for (Map.Entry<String, ColumnIndexContainer> entry: columnIndexContainerMap.entrySet()) {
+      String columnName = entry.getKey();
+      ColumnIndexContainer container = entry.getValue();
+      if (segmentMetadata.getSchema().isVirtualColumn(columnName) && (container.getForwardIndex() instanceof VirtualColumnLongValueReaderWriter)) {
+        virtualColumnsReaderWriter.put(columnName, container.getForwardIndex());
+      }
+    }
+    _segmentCallback.init(segmentMetadata, virtualColumnsReaderWriter);
+    _segmentCallback.initOffsetColumn(columnIndexContainerMap.get(segmentMetadata.getSchema().getOffsetKey()));
   }
 
   @Override
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/immutable/ImmutableSegmentLoader.java b/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/immutable/ImmutableSegmentLoader.java
index 49d7729..198a164 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/immutable/ImmutableSegmentLoader.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/immutable/ImmutableSegmentLoader.java
@@ -24,6 +24,8 @@ import java.util.HashMap;
 import java.util.Map;
 import javax.annotation.Nullable;
 import org.apache.pinot.common.segment.ReadMode;
+import org.apache.pinot.core.data.manager.upsert.DefaultDataManagerCallbackImpl;
+import org.apache.pinot.core.data.manager.upsert.DataManagerCallback;
 import org.apache.pinot.core.indexsegment.generator.SegmentVersion;
 import org.apache.pinot.core.segment.index.ColumnMetadata;
 import org.apache.pinot.core.segment.index.SegmentMetadataImpl;
@@ -44,8 +46,6 @@ import org.apache.pinot.spi.data.Schema;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.annotation.Nonnull;
-
 
 public class ImmutableSegmentLoader {
   private ImmutableSegmentLoader() {
@@ -60,7 +60,7 @@ public class ImmutableSegmentLoader {
       throws Exception {
     IndexLoadingConfig defaultIndexLoadingConfig = new IndexLoadingConfig();
     defaultIndexLoadingConfig.setReadMode(readMode);
-    return load(indexDir, defaultIndexLoadingConfig, null);
+    return load(indexDir, defaultIndexLoadingConfig, DefaultDataManagerCallbackImpl.INSTANCE, null);
   }
 
   /**
@@ -68,25 +68,18 @@ public class ImmutableSegmentLoader {
    */
   public static ImmutableSegment load(File indexDir, IndexLoadingConfig indexLoadingConfig)
       throws Exception {
-    return load(indexDir, indexLoadingConfig, null);
+    return load(indexDir, indexLoadingConfig, DefaultDataManagerCallbackImpl.INSTANCE, null);
   }
 
-  public static ImmutableSegment load(File indexDir, IndexLoadingConfig indexLoadingConfig, @Nullable Schema schema)
+  public static ImmutableSegment load(File indexDir, IndexLoadingConfig indexLoadingConfig,
+                                      DataManagerCallback dataManagerCallback, @Nullable Schema schema)
           throws Exception {
-    return loadHelper(indexDir, indexLoadingConfig, schema);
+    return loadHelper(indexDir, indexLoadingConfig, dataManagerCallback, schema);
   }
 
-  /**
-   * to load upsert related segment
-   */
-  public static ImmutableUpsertSegmentImpl loadUpsertSegment(File indexDir, IndexLoadingConfig indexLoadingConfig,
-                                                             Schema schema) throws Exception {
-    ImmutableSegmentImpl segment = loadHelper(indexDir, indexLoadingConfig, schema);
-    return ImmutableUpsertSegmentImpl.copyOf(segment);
-  }
 
   private static ImmutableSegmentImpl loadHelper(File indexDir, IndexLoadingConfig indexLoadingConfig,
-                                                 @Nullable Schema schema) throws Exception {
+                                                 DataManagerCallback dataManagerCallback, @Nullable Schema schema) throws Exception {
     Preconditions.checkArgument(indexDir.isDirectory(), "Index directory: {} does not exist or is not a directory",
         indexDir);
 
@@ -155,7 +148,8 @@ public class ImmutableSegmentLoader {
               indexContainerMap, readMode);
     }
 
-    return new ImmutableSegmentImpl(segmentDirectory, segmentMetadata, indexContainerMap, starTreeIndexContainer);
+    return new ImmutableSegmentImpl(segmentDirectory, segmentMetadata, indexContainerMap, starTreeIndexContainer,
+        dataManagerCallback.getIndexSegmentCallback());
   }
 
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImpl.java b/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImpl.java
index b80ac64..12fcd94 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImpl.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImpl.java
@@ -34,6 +34,7 @@ import javax.annotation.Nullable;
 
 import org.apache.pinot.common.config.SegmentPartitionConfig;
 import org.apache.pinot.common.segment.SegmentMetadata;
+import org.apache.pinot.core.data.manager.upsert.IndexSegmentCallback;
 import org.apache.pinot.core.indexsegment.IndexSegmentUtils;
 import org.apache.pinot.core.io.reader.DataFileReader;
 import org.apache.pinot.core.io.readerwriter.BaseSingleColumnSingleValueReaderWriter;
@@ -76,7 +77,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 
-public abstract class MutableSegmentImpl implements MutableSegment {
+public class MutableSegmentImpl implements MutableSegment {
   // For multi-valued column, forward-index.
   // Maximum number of multi-values per row. We assert on this.
   private static final int MAX_MULTI_VALUES_PER_ROW = 1000;
@@ -124,9 +125,9 @@ public abstract class MutableSegmentImpl implements MutableSegment {
   private final Collection<MetricFieldSpec> _physicalMetricFieldSpecs;
 
   // Cache some virtual columns
-  protected final Map<String, VirtualColumnProvider> _virtualColumnProviderMap = new HashMap<>();
-  protected final Map<String, Dictionary> _virtualColumnDictionary = new HashMap<>();
-  protected final Map<String, DataFileReader> _virtualColumnIndexReader = new HashMap<>();
+  private final Map<String, VirtualColumnProvider> _virtualColumnProviderMap = new HashMap<>();
+  private final Map<String, Dictionary> _virtualColumnDictionary = new HashMap<>();
+  private final Map<String, DataFileReader> _virtualColumnIndexReader = new HashMap<>();
 
   // default message metadata
   private volatile long _lastIndexedTimeMs = Long.MIN_VALUE;
@@ -134,7 +135,10 @@ public abstract class MutableSegmentImpl implements MutableSegment {
 
   private RealtimeLuceneReaders _realtimeLuceneReaders;
 
-  public MutableSegmentImpl(RealtimeSegmentConfig config) {
+  // upsert/append related components
+  private final IndexSegmentCallback _indexSegmentCallback;
+
+  public MutableSegmentImpl(RealtimeSegmentConfig config, IndexSegmentCallback indexSegmentCallback) {
     _tableName = config.getTableName();
     _segmentName = config.getSegmentName();
     _schema = config.getSchema();
@@ -167,6 +171,7 @@ public abstract class MutableSegmentImpl implements MutableSegment {
     _statsHistory = config.getStatsHistory();
     _segmentPartitionConfig = config.getSegmentPartitionConfig();
     _nullHandlingEnabled = config.isNullHandlingEnabled();
+    _indexSegmentCallback = indexSegmentCallback;
 
     Collection<FieldSpec> allFieldSpecs = _schema.getAllFieldSpecs();
     List<FieldSpec> physicalFieldSpecs = new ArrayList<>(allFieldSpecs.size());
@@ -302,7 +307,7 @@ public abstract class MutableSegmentImpl implements MutableSegment {
       }
     }
 
-    for (FieldSpec fieldSpec : _physicalFieldSpecs) {
+    for (FieldSpec fieldSpec : virtualFieldSpecs) {
       String column = fieldSpec.getName();
       VirtualColumnContext virtualColumnContext = new VirtualColumnContext(fieldSpec, _capacity, true);
       final VirtualColumnProvider provider =
@@ -317,6 +322,8 @@ public abstract class MutableSegmentImpl implements MutableSegment {
       _virtualColumnDictionary.put(column, dictionary);
     }
 
+    _indexSegmentCallback.init(_segmentMetadata, _virtualColumnIndexReader);
+
     if (_realtimeLuceneReaders != null) {
       // add the realtime lucene index readers to the global queue for refresh task to pick up
       RealtimeLuceneIndexRefreshState realtimeLuceneIndexRefreshState = RealtimeLuceneIndexRefreshState.getInstance();
@@ -377,7 +384,7 @@ public abstract class MutableSegmentImpl implements MutableSegment {
 
       // Update number of document indexed at last to make the latest record queryable
       canTakeMore = _numDocsIndexed++ < _capacity;
-      postProcessRecords(row, docId);
+      _indexSegmentCallback.postProcessRecords(row, docId);
     } else {
       Preconditions
           .checkState(_aggregateMetrics, "Invalid document-id during indexing: " + docId + " expected: " + numDocs);
@@ -522,8 +529,6 @@ public abstract class MutableSegmentImpl implements MutableSegment {
     }
   }
 
-  protected abstract void postProcessRecords(GenericRow row, int docId);
-
   private boolean aggregateMetrics(GenericRow row, int docId) {
     for (MetricFieldSpec metricFieldSpec : _physicalMetricFieldSpecs) {
       String column = metricFieldSpec.getName();
@@ -589,7 +594,6 @@ public abstract class MutableSegmentImpl implements MutableSegment {
   public ColumnDataSource getDataSource(String columnName) {
     FieldSpec fieldSpec = _schema.getFieldSpecFor(columnName);
     if (fieldSpec.isVirtualColumn()) {
-      // FIXME
       VirtualColumnContext virtualColumnContext = new VirtualColumnContext(fieldSpec, _numDocsIndexed, true);
       VirtualColumnProvider virtualColumnProvider = _virtualColumnProviderMap.get(columnName);
       return new ColumnDataSource(virtualColumnProvider.buildColumnIndexContainer(virtualColumnContext),
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdaterConfig.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/WatermarkManager.java
similarity index 70%
copy from pinot-core/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdaterConfig.java
copy to pinot-core/src/main/java/org/apache/pinot/core/segment/updater/WatermarkManager.java
index 97d48f3..01579e7 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdaterConfig.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/WatermarkManager.java
@@ -18,8 +18,16 @@
  */
 package org.apache.pinot.core.segment.updater;
 
-public class SegmentUpdaterConfig {
-  public static final String SEGMENT_UDPATE_SLEEP_MS = "sleep.ms";
-  public static final String INPUT_TOPIC_PREFIX  = "input.topic.prefix";
-  public static final int SEGMENT_UDPATE_SLEEP_MS_DEFAULT = 100;
+import org.apache.commons.configuration.Configuration;
+import org.apache.pinot.grigio.common.metrics.GrigioMeter;
+import org.apache.pinot.grigio.common.metrics.GrigioMetrics;
+
+import java.util.Map;
+
+public interface WatermarkManager {
+
+  void init(Configuration config, GrigioMetrics metrics);
+
+  Map<String, Map<Integer, Long>> getHighWaterMarkTablePartitionMap();
+
 }
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/BaseTableDataManagerTest.java b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/BaseTableDataManagerTest.java
index 9b28f5a..9626be2 100644
--- a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/BaseTableDataManagerTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/BaseTableDataManagerTest.java
@@ -36,6 +36,9 @@ import org.apache.pinot.common.segment.SegmentMetadata;
 import org.apache.pinot.core.data.manager.config.TableDataManagerConfig;
 import org.apache.pinot.core.data.manager.offline.ImmutableSegmentDataManager;
 import org.apache.pinot.core.data.manager.offline.OfflineTableDataManager;
+import org.apache.pinot.core.data.manager.upsert.DefaultDataManagerCallbackImpl;
+import org.apache.pinot.core.data.manager.upsert.DefaultIndexSegmentCallback;
+import org.apache.pinot.core.data.manager.upsert.DefaultTableDataManagerCallbackImpl;
 import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
 import org.testng.Assert;
 import org.testng.annotations.AfterSuite;
@@ -100,7 +103,7 @@ public class BaseTableDataManagerTest {
 
   private TableDataManager makeTestableManager()
       throws Exception {
-    TableDataManager tableDataManager = new OfflineTableDataManager();
+    TableDataManager tableDataManager = new OfflineTableDataManager(new DefaultTableDataManagerCallbackImpl());
     TableDataManagerConfig config;
     {
       config = mock(TableDataManagerConfig.class);
@@ -139,7 +142,7 @@ public class BaseTableDataManagerTest {
     // Add the segment, get it for use, remove the segment, and then return it.
     // Make sure that the segment is not destroyed before return.
     ImmutableSegment immutableSegment = makeImmutableSegment(segmentName, totalDocs);
-    tableDataManager.addSegment(immutableSegment);
+    tableDataManager.addSegment(immutableSegment, DefaultDataManagerCallbackImpl.INSTANCE);
     SegmentDataManager segmentDataManager = tableDataManager.acquireSegment(segmentName);
     Assert.assertEquals(segmentDataManager.getReferenceCount(), 2);
     tableDataManager.removeSegment(segmentName);
@@ -161,7 +164,7 @@ public class BaseTableDataManagerTest {
     // Add a new segment and remove it in order this time.
     final String anotherSeg = "AnotherSegment";
     ImmutableSegment ix1 = makeImmutableSegment(anotherSeg, totalDocs);
-    tableDataManager.addSegment(ix1);
+    tableDataManager.addSegment(ix1, DefaultDataManagerCallbackImpl.INSTANCE);
     SegmentDataManager sdm1 = tableDataManager.acquireSegment(anotherSeg);
     Assert.assertNotNull(sdm1);
     Assert.assertEquals(sdm1.getReferenceCount(), 2);
@@ -178,7 +181,7 @@ public class BaseTableDataManagerTest {
     Assert.assertEquals(sdm1.getReferenceCount(), 1);
     // Now replace the segment with another one.
     ImmutableSegment ix2 = makeImmutableSegment(anotherSeg, totalDocs + 1);
-    tableDataManager.addSegment(ix2);
+    tableDataManager.addSegment(ix2, DefaultDataManagerCallbackImpl.INSTANCE);
     // Now the previous one should have been destroyed, and
     Assert.assertEquals(sdm1.getReferenceCount(), 0);
     verify(ix1, times(1)).destroy();
@@ -222,7 +225,8 @@ public class BaseTableDataManagerTest {
 
     for (int i = _lo; i <= _hi; i++) {
       final String segName = SEGMENT_PREFIX + i;
-      tableDataManager.addSegment(makeImmutableSegment(segName, random.nextInt()));
+      tableDataManager.addSegment(makeImmutableSegment(segName, random.nextInt()),
+          DefaultDataManagerCallbackImpl.INSTANCE);
       _allSegManagers.add(_internalSegMap.get(segName));
     }
 
@@ -409,7 +413,8 @@ public class BaseTableDataManagerTest {
     private void addSegment() {
       final int segmentToAdd = _hi + 1;
       final String segName = SEGMENT_PREFIX + segmentToAdd;
-      _tableDataManager.addSegment(makeImmutableSegment(segName, _random.nextInt()));
+      _tableDataManager.addSegment(makeImmutableSegment(segName, _random.nextInt()),
+          DefaultDataManagerCallbackImpl.INSTANCE);
       _allSegManagers.add(_internalSegMap.get(segName));
       _hi = segmentToAdd;
     }
@@ -418,7 +423,8 @@ public class BaseTableDataManagerTest {
     private void replaceSegment() {
       int segToReplace = _random.nextInt(_hi - _lo + 1) + _lo;
       final String segName = SEGMENT_PREFIX + segToReplace;
-      _tableDataManager.addSegment(makeImmutableSegment(segName, _random.nextInt()));
+      _tableDataManager.addSegment(makeImmutableSegment(segName, _random.nextInt()),
+          DefaultDataManagerCallbackImpl.INSTANCE);
       _allSegManagers.add(_internalSegMap.get(segName));
     }
 
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManagerTest.java b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManagerTest.java
index c4b557b..2f8bc3f 100644
--- a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManagerTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManagerTest.java
@@ -31,6 +31,7 @@ import java.util.concurrent.Semaphore;
 import java.util.concurrent.atomic.AtomicReference;
 import org.apache.commons.io.FileUtils;
 import org.apache.pinot.common.config.TableConfig;
+import org.apache.pinot.core.data.manager.upsert.DefaultDataManagerCallbackImpl;
 import org.apache.pinot.spi.data.Schema;
 import org.apache.pinot.common.metadata.instance.InstanceZKMetadata;
 import org.apache.pinot.common.metadata.segment.LLCRealtimeSegmentZKMetadata;
@@ -696,7 +697,7 @@ public class LLRealtimeSegmentDataManagerTest {
     Assert.assertEquals(secondSegmentDataManager.get().getPartitionConsumerSemaphore().availablePermits(), 1);
   }
 
-  public static class FakeLLRealtimeSegmentDataManager extends AppendLLRealtimeSegmentDataManager {
+  public static class FakeLLRealtimeSegmentDataManager extends LLRealtimeSegmentDataManager {
 
     public Field _state;
     public Field _shouldStop;
@@ -729,7 +730,7 @@ public class LLRealtimeSegmentDataManagerTest {
         throws Exception {
       super(segmentZKMetadata, tableConfig, realtimeTableDataManager, resourceDataDir,
           new IndexLoadingConfig(makeInstanceDataManagerConfig(), tableConfig), schema, llcSegmentName,
-          semaphoreMap.get(llcSegmentName.getPartitionId()), serverMetrics);
+          semaphoreMap.get(llcSegmentName.getPartitionId()), serverMetrics, DefaultDataManagerCallbackImpl.INSTANCE);
       _state = LLRealtimeSegmentDataManager.class.getDeclaredField("_state");
       _state.setAccessible(true);
       _shouldStop = LLRealtimeSegmentDataManager.class.getDeclaredField("_shouldStop");
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImplTestUtils.java b/pinot-core/src/test/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImplTestUtils.java
index 26e7eff..4ccf197 100644
--- a/pinot-core/src/test/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImplTestUtils.java
+++ b/pinot-core/src/test/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImplTestUtils.java
@@ -21,6 +21,7 @@ package org.apache.pinot.core.indexsegment.mutable;
 import java.util.Set;
 import javax.annotation.Nonnull;
 import org.apache.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
+import org.apache.pinot.core.data.manager.upsert.DefaultIndexSegmentCallback;
 import org.apache.pinot.core.io.writer.impl.DirectMemoryManager;
 import org.apache.pinot.core.realtime.impl.RealtimeSegmentConfig;
 import org.apache.pinot.core.realtime.impl.RealtimeSegmentStatsHistory;
@@ -59,6 +60,6 @@ public class MutableSegmentImplTestUtils {
             .setRealtimeSegmentZKMetadata(new RealtimeSegmentZKMetadata())
             .setMemoryManager(new DirectMemoryManager(SEGMENT_NAME)).setStatsHistory(statsHistory)
             .setAggregateMetrics(aggregateMetrics).setNullHandlingEnabled(nullHandlingEnabled).build();
-    return new MutableAppendSegmentImpl(realtimeSegmentConfig);
+    return new MutableSegmentImpl(realtimeSegmentConfig, DefaultIndexSegmentCallback.INSTANCE);
   }
 }
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/segment/index/creator/SegmentGenerationWithNullValueVectorTest.java b/pinot-core/src/test/java/org/apache/pinot/core/segment/index/creator/SegmentGenerationWithNullValueVectorTest.java
index a4ee984..4d1d469 100644
--- a/pinot-core/src/test/java/org/apache/pinot/core/segment/index/creator/SegmentGenerationWithNullValueVectorTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/core/segment/index/creator/SegmentGenerationWithNullValueVectorTest.java
@@ -42,6 +42,7 @@ import org.apache.pinot.core.data.manager.InstanceDataManager;
 import org.apache.pinot.core.data.manager.TableDataManager;
 import org.apache.pinot.core.data.manager.config.TableDataManagerConfig;
 import org.apache.pinot.core.data.manager.offline.TableDataManagerProvider;
+import org.apache.pinot.core.data.manager.upsert.DefaultDataManagerCallbackImpl;
 import org.apache.pinot.core.query.executor.QueryExecutor;
 import org.apache.pinot.core.query.executor.ServerQueryExecutorV1Impl;
 import org.apache.pinot.core.query.request.ServerQueryRequest;
@@ -141,7 +142,7 @@ public class SegmentGenerationWithNullValueVectorTest {
         .getTableDataManager(tableDataManagerConfig, "testInstance", mock(ZkHelixPropertyStore.class),
             mock(ServerMetrics.class));
     tableDataManager.start();
-    tableDataManager.addSegment(_segment);
+    tableDataManager.addSegment(_segment, DefaultDataManagerCallbackImpl.INSTANCE);
     _instanceDataManager = mock(InstanceDataManager.class);
     when(_instanceDataManager.getTableDataManager(TABLE_NAME)).thenReturn(tableDataManager);
 
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/segment/index/loader/LoaderTest.java b/pinot-core/src/test/java/org/apache/pinot/core/segment/index/loader/LoaderTest.java
index 59342a3..d71896c 100644
--- a/pinot-core/src/test/java/org/apache/pinot/core/segment/index/loader/LoaderTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/core/segment/index/loader/LoaderTest.java
@@ -25,6 +25,7 @@ import java.util.ArrayList;
 import java.util.List;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.filefilter.FileFileFilter;
+import org.apache.pinot.core.data.manager.upsert.DefaultDataManagerCallbackImpl;
 import org.apache.pinot.core.segment.creator.impl.inv.text.LuceneTextIndexCreator;
 import org.apache.pinot.spi.data.DimensionFieldSpec;
 import org.apache.pinot.spi.data.FieldSpec;
@@ -225,12 +226,14 @@ public class LoaderTest {
     schema.addField(new DimensionFieldSpec("SVString", FieldSpec.DataType.STRING, true, ""));
     schema.addField(new DimensionFieldSpec("MVString", FieldSpec.DataType.STRING, false, ""));
 
-    IndexSegment indexSegment = ImmutableSegmentLoader.load(_indexDir, _v1IndexLoadingConfig, schema);
+    IndexSegment indexSegment = ImmutableSegmentLoader
+        .load(_indexDir, _v1IndexLoadingConfig, DefaultDataManagerCallbackImpl.INSTANCE, schema);
     Assert.assertEquals(indexSegment.getDataSource("SVString").getDictionary().get(0), "");
     Assert.assertEquals(indexSegment.getDataSource("MVString").getDictionary().get(0), "");
     indexSegment.destroy();
 
-    indexSegment = ImmutableSegmentLoader.load(_indexDir, _v3IndexLoadingConfig, schema);
+    indexSegment = ImmutableSegmentLoader
+        .load(_indexDir, _v3IndexLoadingConfig, DefaultDataManagerCallbackImpl.INSTANCE, schema);
     Assert.assertEquals(indexSegment.getDataSource("SVString").getDictionary().get(0), "");
     Assert.assertEquals(indexSegment.getDataSource("MVString").getDictionary().get(0), "");
     indexSegment.destroy();
@@ -245,7 +248,8 @@ public class LoaderTest {
 
     FieldSpec byteMetric = new MetricFieldSpec(newColumnName, FieldSpec.DataType.BYTES, defaultValue);
     schema.addField(byteMetric);
-    IndexSegment indexSegment = ImmutableSegmentLoader.load(_indexDir, _v3IndexLoadingConfig, schema);
+    IndexSegment indexSegment = ImmutableSegmentLoader
+        .load(_indexDir, _v3IndexLoadingConfig, DefaultDataManagerCallbackImpl.INSTANCE, schema);
     Assert
         .assertEquals(BytesUtils.toHexString((byte[]) indexSegment.getDataSource(newColumnName).getDictionary().get(0)),
             defaultValue);
diff --git a/pinot-core/src/test/java/org/apache/pinot/queries/SerializedBytesQueriesTest.java b/pinot-core/src/test/java/org/apache/pinot/queries/SerializedBytesQueriesTest.java
index c8e688b..b0f5a55 100644
--- a/pinot-core/src/test/java/org/apache/pinot/queries/SerializedBytesQueriesTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/queries/SerializedBytesQueriesTest.java
@@ -29,6 +29,8 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Random;
 import org.apache.commons.io.FileUtils;
+import org.apache.pinot.core.data.manager.upsert.DefaultDataManagerCallbackImpl;
+import org.apache.pinot.core.data.manager.upsert.DefaultIndexSegmentCallback;
 import org.apache.pinot.spi.data.FieldSpec.DataType;
 import org.apache.pinot.spi.data.Schema;
 import org.apache.pinot.common.response.broker.AggregationResult;
diff --git a/pinot-core/src/test/java/org/apache/pinot/query/executor/QueryExecutorTest.java b/pinot-core/src/test/java/org/apache/pinot/query/executor/QueryExecutorTest.java
index b6ce1e2..203d2f6 100644
--- a/pinot-core/src/test/java/org/apache/pinot/query/executor/QueryExecutorTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/query/executor/QueryExecutorTest.java
@@ -36,6 +36,7 @@ import org.apache.pinot.core.data.manager.InstanceDataManager;
 import org.apache.pinot.core.data.manager.TableDataManager;
 import org.apache.pinot.core.data.manager.config.TableDataManagerConfig;
 import org.apache.pinot.core.data.manager.offline.TableDataManagerProvider;
+import org.apache.pinot.core.data.manager.upsert.DefaultDataManagerCallbackImpl;
 import org.apache.pinot.core.indexsegment.IndexSegment;
 import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
 import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
@@ -103,7 +104,7 @@ public class QueryExecutorTest {
             mock(ServerMetrics.class));
     tableDataManager.start();
     for (ImmutableSegment indexSegment : _indexSegments) {
-      tableDataManager.addSegment(indexSegment);
+      tableDataManager.addSegment(indexSegment, DefaultDataManagerCallbackImpl.INSTANCE);
     }
     InstanceDataManager instanceDataManager = mock(InstanceDataManager.class);
     when(instanceDataManager.getTableDataManager(TABLE_NAME)).thenReturn(tableDataManager);
diff --git a/pinot-grigio/pom.xml b/pinot-grigio/pinot-grigio-provided/pom.xml
similarity index 51%
copy from pinot-grigio/pom.xml
copy to pinot-grigio/pinot-grigio-provided/pom.xml
index e53fa72..5643070 100644
--- a/pinot-grigio/pom.xml
+++ b/pinot-grigio/pinot-grigio-provided/pom.xml
@@ -22,21 +22,38 @@
 <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>pinot</artifactId>
-    <groupId>org.apache.pinot</groupId>
-    <version>0.3.0-SNAPSHOT</version>
-  </parent>
-  <modelVersion>4.0.0</modelVersion>
-  <artifactId>pinot-grigio</artifactId>
-    <packaging>pom</packaging>
-    <name>Pinot Grigio</name>
-  <url>http://maven.apache.org</url>
-    <modules>
-      <module>pinot-grigio-common</module>
-      <module>pinot-grigio-coordinator</module>
-    </modules>
+    <parent>
+        <artifactId>pinot-grigio</artifactId>
+        <groupId>org.apache.pinot</groupId>
+        <version>0.3.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>pinot-grigio-provided</artifactId>
+    <name>Pinot Grigio Provided components</name>
     <properties>
-    <pinot.root>${basedir}/..</pinot.root>
-  </properties>
+        <pinot.root>${basedir}/../..</pinot.root>
+    </properties>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.pinot</groupId>
+            <artifactId>pinot-common</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.pinot</groupId>
+            <artifactId>pinot-core</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.testng</groupId>
+            <artifactId>testng</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-core</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+
+
+
 </project>
\ No newline at end of file
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/UpsertLLRealtimeSegmentDataManager.java b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/data/manager/upsert/UpsertDataManagerCallbackImpl.java
similarity index 57%
rename from pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/UpsertLLRealtimeSegmentDataManager.java
rename to pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/data/manager/upsert/UpsertDataManagerCallbackImpl.java
index f08037a..24731f2 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/UpsertLLRealtimeSegmentDataManager.java
+++ b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/data/manager/upsert/UpsertDataManagerCallbackImpl.java
@@ -16,20 +16,14 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.core.data.manager.realtime;
+package org.apache.pinot.core.data.manager.upsert;
 
 import com.google.common.base.Preconditions;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
+import org.apache.pinot.common.config.TableNameBuilder;
 import org.apache.pinot.common.metrics.ServerMeter;
 import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.common.utils.CommonConstants;
 import org.apache.pinot.common.utils.LLCSegmentName;
-import org.apache.pinot.core.data.manager.UpsertSegmentDataManager;
-import org.apache.pinot.core.indexsegment.UpsertSegment;
-import org.apache.pinot.core.indexsegment.mutable.MutableSegmentImpl;
-import org.apache.pinot.core.indexsegment.mutable.MutableUpsertSegmentImpl;
-import org.apache.pinot.core.realtime.impl.RealtimeSegmentConfig;
-import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
 import org.apache.pinot.core.segment.updater.SegmentUpdater;
 import org.apache.pinot.grigio.common.messages.KeyCoordinatorQueueMsg;
 import org.apache.pinot.grigio.common.rpcQueue.ProduceTask;
@@ -40,71 +34,74 @@ import org.apache.pinot.spi.data.DimensionFieldSpec;
 import org.apache.pinot.spi.data.Schema;
 import org.apache.pinot.spi.data.TimeFieldSpec;
 import org.apache.pinot.spi.data.readers.GenericRow;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.util.List;
-import java.util.concurrent.Semaphore;
 
-/**
- * class design is pretty bad right now, need to rework inheritance to abstract the base class or use composition instead
- */
-public class UpsertLLRealtimeSegmentDataManager extends LLRealtimeSegmentDataManager implements UpsertSegmentDataManager {
+public class UpsertDataManagerCallbackImpl implements DataManagerCallback {
 
-  private final QueueProducer _keyCoordinatorQueueProducer;
+  private static final Logger LOGGER = LoggerFactory.getLogger(UpsertDataManagerCallbackImpl.class);
 
-  public UpsertLLRealtimeSegmentDataManager(RealtimeSegmentZKMetadata segmentZKMetadata, TableConfig tableConfig,
-      RealtimeTableDataManager realtimeTableDataManager, String resourceDataDir, IndexLoadingConfig indexLoadingConfig,
-      Schema schema, LLCSegmentName llcSegmentName, Semaphore partitionConsumerSemaphore, ServerMetrics serverMetrics) throws Exception {
-    super(segmentZKMetadata, tableConfig, realtimeTableDataManager, resourceDataDir, indexLoadingConfig, schema,
-            llcSegmentName, partitionConsumerSemaphore, serverMetrics);
-    Preconditions.checkState(_schema.getPrimaryKeyFieldSpec() != null, "primary key not found");
-    Preconditions.checkState(_schema.getOffsetKeyFieldSpec() != null, "offset key not found");
+  private final String _tableNameWithType;
+  private final QueueProducer _keyCoordinatorQueueProducer;
+  private final String _segmentName;
+  private final Schema _schema;
+  private final ServerMetrics _serverMetrics;
+  private IndexSegmentCallback _indexSegmentCallback;
+
+  public UpsertDataManagerCallbackImpl(String tableNameWithType, String segmentName, Schema schema,
+      ServerMetrics serverMetrics, boolean isMutable) {
+    Preconditions.checkState(schema.getPrimaryKeyFieldSpec() != null, "primary key not found");
+    Preconditions.checkState(schema.getOffsetKeyFieldSpec() != null, "offset key not found");
+    _tableNameWithType = TableNameBuilder.ensureTableNameWithType(tableNameWithType,
+        CommonConstants.Helix.TableType.REALTIME);
+    _segmentName = segmentName;
+    _schema = schema;
+    _serverMetrics = serverMetrics;
     _keyCoordinatorQueueProducer = KeyCoordinatorProvider.getInstance().getCachedProducer(_tableNameWithType);
-    initVirtualColumns();
-  }
-
-  public String getSegmentName() {
-    return _segmentNameStr;
+    if (isMutable) {
+      _indexSegmentCallback = new UpsertMutableIndexSegmentCallback();
+    } else {
+      _indexSegmentCallback = new UpsertImmutableIndexSegmentCallback();
+    }
   }
 
   @Override
-  public void updateVirtualColumns(List<UpdateLogEntry> messages) {
-    ((UpsertSegment) _realtimeSegment).updateVirtualColumn(messages);
+  public synchronized IndexSegmentCallback getIndexSegmentCallback() {
+    return _indexSegmentCallback;
   }
 
   @Override
-  public String getVirtualColumnInfo(long offset) {
-    return ((UpsertSegment) _realtimeSegment).getVirtualColumnInfo(offset);
+  public void processTransformedRow(GenericRow row, long offset) {
+    row.putValue(_schema.getOffsetKey(), offset);
   }
 
   @Override
-  public void destroy() {
-    SegmentUpdater.getInstance().removeSegmentDataManager(_tableNameWithType, _llcSegmentName.getSegmentName(), this);
-    super.destroy();
+  public void postIndexProcessing(GenericRow row, long offset) {
+    emitEventToKeyCoordinator(row, offset);
   }
 
   @Override
-  protected MutableSegmentImpl createMutableSegment(RealtimeSegmentConfig config) {
-    return new MutableUpsertSegmentImpl(config);
+  public void postConsumeLoop() {
+    _keyCoordinatorQueueProducer.flush();
   }
 
-  @Override
-  protected void processTransformedRow(GenericRow row, long offset) {
-    row.putField(_schema.getOffsetKey(), offset);
+  public void updateVirtualColumns(List<UpdateLogEntry> messages) {
+    _indexSegmentCallback.updateVirtualColumn(messages);
   }
 
-  @Override
-  protected void postIndexProcessing(GenericRow row, long offset) {
-    emitEventToKeyCoordinator(row, offset);
+  public String getVirtualColumnInfo(long offset) {
+    return _indexSegmentCallback.getVirtualColumnInfo(offset);
   }
 
-  @Override
-  protected boolean consumeLoop() throws Exception {
-    boolean result = super.consumeLoop();
-    segmentLogger.info("flushing kafka producer");
-    _keyCoordinatorQueueProducer.flush();
-    segmentLogger.info("done flushing kafka producer");
-    return result;
+  public void destroy() {
+    try {
+      SegmentUpdater.getInstance().removeSegmentDataManager(_tableNameWithType, _segmentName, this);
+    } catch (Exception ex) {
+      LOGGER.error("failed to destroy data manager callback");
+    }
   }
 
   /**
@@ -116,7 +113,7 @@ public class UpsertLLRealtimeSegmentDataManager extends LLRealtimeSegmentDataMan
     final byte[] primaryKeyBytes = getPrimaryKeyBytesFromRow(row);
     final long timestampMillis = getTimestampFromRow(row);
     ProduceTask<byte[], KeyCoordinatorQueueMsg> task = new ProduceTask<>(primaryKeyBytes,
-        new KeyCoordinatorQueueMsg(primaryKeyBytes, _segmentNameStr, timestampMillis, offset));
+        new KeyCoordinatorQueueMsg(primaryKeyBytes, _segmentName, timestampMillis, offset));
     task.setCallback(new ProduceTask.Callback() {
       @Override
       public void onSuccess() {
@@ -143,11 +140,12 @@ public class UpsertLLRealtimeSegmentDataManager extends LLRealtimeSegmentDataMan
     return spec.getIncomingGranularitySpec().toMillis(row.getValue(spec.getIncomingTimeColumnName()));
   }
 
-
-  private void initVirtualColumns() throws IOException {
+  @Override
+  public void initVirtualColumns() throws IOException {
     // 1. ensure the data manager can capture all update events
     // 2. load all existing messages
-    SegmentUpdater.getInstance().addSegmentDataManager(_tableNameWithType, _llcSegmentName, this);
-    ((UpsertSegment) _realtimeSegment).initVirtualColumn();
+    SegmentUpdater.getInstance().addSegmentDataManager(_tableNameWithType, new LLCSegmentName(_segmentName), this);
+    _indexSegmentCallback.initVirtualColumn();
   }
+
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/immutable/ImmutableUpsertSegmentImpl.java b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/data/manager/upsert/UpsertImmutableIndexSegmentCallback.java
similarity index 74%
rename from pinot-core/src/main/java/org/apache/pinot/core/indexsegment/immutable/ImmutableUpsertSegmentImpl.java
rename to pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/data/manager/upsert/UpsertImmutableIndexSegmentCallback.java
index 5ab42f5..1ddc963 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/immutable/ImmutableUpsertSegmentImpl.java
+++ b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/data/manager/upsert/UpsertImmutableIndexSegmentCallback.java
@@ -16,97 +16,86 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.core.indexsegment.immutable;
+package org.apache.pinot.core.data.manager.upsert;
+
 
 import com.clearspring.analytics.util.Preconditions;
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.pinot.common.Utils;
 import org.apache.pinot.common.config.TableNameBuilder;
+import org.apache.pinot.common.segment.SegmentMetadata;
 import org.apache.pinot.common.utils.CommonConstants;
-import org.apache.pinot.core.indexsegment.UpsertSegment;
 import org.apache.pinot.core.io.reader.BaseSingleColumnSingleValueReader;
 import org.apache.pinot.core.io.reader.DataFileReader;
-import org.apache.pinot.core.segment.index.SegmentMetadataImpl;
 import org.apache.pinot.core.segment.index.column.ColumnIndexContainer;
 import org.apache.pinot.core.segment.index.readers.Dictionary;
-import org.apache.pinot.core.segment.store.SegmentDirectory;
-import org.apache.pinot.core.segment.updater.UpsertWaterMarkManager;
+import org.apache.pinot.core.segment.updater.UpsertWatermarkManager;
 import org.apache.pinot.core.segment.virtualcolumn.mutable.VirtualColumnLongValueReaderWriter;
-import org.apache.pinot.core.startree.v2.store.StarTreeIndexContainer;
 import org.apache.pinot.grigio.common.storageProvider.UpdateLogEntry;
 import org.apache.pinot.grigio.common.storageProvider.UpdateLogEntrySet;
 import org.apache.pinot.grigio.common.storageProvider.UpdateLogStorageProvider;
+import org.apache.pinot.spi.data.readers.GenericRow;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.annotation.Nullable;
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
+import java.util.*;
 
-public class ImmutableUpsertSegmentImpl extends ImmutableSegmentImpl implements UpsertSegment {
+public class UpsertImmutableIndexSegmentCallback implements IndexSegmentCallback {
 
-  private static final Logger LOGGER = LoggerFactory.getLogger(ImmutableUpsertSegmentImpl.class);
+  private static final Logger LOGGER = LoggerFactory.getLogger(UpsertImmutableIndexSegmentCallback.class);
 
-  private final List<VirtualColumnLongValueReaderWriter> _virtualColumnsReaderWriter;
-  private final String _tableNameWithType;
-  private final String _segmentName;
-  private final int _totalDoc;
+  private List<VirtualColumnLongValueReaderWriter> _virtualColumnsReaderWriter;
+  private String _tableNameWithType;
+  private String _segmentName;
+  private int _totalDoc;
   private long _minSourceOffset;
-  private final UpsertWaterMarkManager _upsertWaterMarkManager;
-  private final UpdateLogStorageProvider _updateLogStorageProvider;
+  private UpsertWatermarkManager _upsertWatermarkManager;
+  private UpdateLogStorageProvider _updateLogStorageProvider;
   // use array for mapping bewteen offset to docId, where actual offset = min_offset + array_index
   // use 4 bytes per record
   private int[] _sourceOffsetToDocIdArray;
 
   private static final int DEFAULT_DOC_ID_FOR_MISSING_ENTRY = -1;
 
-  public ImmutableUpsertSegmentImpl(SegmentDirectory segmentDirectory,
-                                    SegmentMetadataImpl segmentMetadata,
-                                    Map<String, ColumnIndexContainer> columnIndexContainerMap,
-                                    @Nullable StarTreeIndexContainer starTreeIndexContainer) {
-    super(segmentDirectory, segmentMetadata, columnIndexContainerMap, starTreeIndexContainer);
+  public UpsertImmutableIndexSegmentCallback() {}
+
+  @Override
+  public void init(SegmentMetadata segmentMetadata, Map<String, DataFileReader> virtualColumnIndexReader) {
     Preconditions.checkState(segmentMetadata.getSchema().isTableForUpsert(), "table should be upsert but it is not");
     _tableNameWithType = TableNameBuilder.ensureTableNameWithType(segmentMetadata.getTableName(),
         CommonConstants.Helix.TableType.REALTIME);
     _segmentName = segmentMetadata.getName();
     _totalDoc = segmentMetadata.getTotalDocs();
-    _upsertWaterMarkManager = UpsertWaterMarkManager.getInstance();
+    _upsertWatermarkManager = UpsertWatermarkManager.getInstance();
     _updateLogStorageProvider = UpdateLogStorageProvider.getInstance();
     _virtualColumnsReaderWriter = new ArrayList<>();
-    for (Map.Entry<String, ColumnIndexContainer> entry: columnIndexContainerMap.entrySet()) {
-      String columnName = entry.getKey();
-      ColumnIndexContainer container = entry.getValue();
-      if (segmentMetadata.getSchema().isVirtualColumn(columnName) && (container.getForwardIndex() instanceof VirtualColumnLongValueReaderWriter)) {
-        _virtualColumnsReaderWriter.add((VirtualColumnLongValueReaderWriter) container.getForwardIndex());
-      }
+    for (DataFileReader reader: virtualColumnIndexReader.values()) {
+      _virtualColumnsReaderWriter.add((VirtualColumnLongValueReaderWriter) reader);
     }
-     buildOffsetToDocIdMap(columnIndexContainerMap.get(segmentMetadata.getSchema().getOffsetKey()));
   }
 
-  /** constructor used for creating instance in test cases
-   * should not be used for creating regular segment
+  /**
+   * method for testing purpose only
    */
   @VisibleForTesting
-  protected ImmutableUpsertSegmentImpl(List<VirtualColumnLongValueReaderWriter> readerWriters,
-                                       int totalDoc, UpsertWaterMarkManager manager,
-                                       UpdateLogStorageProvider updateLogStorageProvider,
-                                       long minSourceOffset, int[] offsetToDocId) {
-    super(null, null, null, null);
+  protected void init(List<VirtualColumnLongValueReaderWriter> readerWriters,
+      int totalDoc, UpsertWatermarkManager manager,
+      UpdateLogStorageProvider updateLogStorageProvider,
+      long minSourceOffset, int[] offsetToDocId) {
     _tableNameWithType = "testTable";
     _segmentName = "testSegment";
     _virtualColumnsReaderWriter = readerWriters;
     _totalDoc = totalDoc;
-    _upsertWaterMarkManager = manager;
+    _upsertWatermarkManager = manager;
     _updateLogStorageProvider = updateLogStorageProvider;
     _minSourceOffset = minSourceOffset;
     _sourceOffsetToDocIdArray = offsetToDocId;
-
   }
 
-  private void buildOffsetToDocIdMap(ColumnIndexContainer offsetColumnIndexContainer) {
+
+  @Override
+  public void initOffsetColumn(ColumnIndexContainer offsetColumnIndexContainer) {
     long start = System.currentTimeMillis();
     final DataFileReader reader = offsetColumnIndexContainer.getForwardIndex();
     final Dictionary dictionary = offsetColumnIndexContainer.getDictionary();
@@ -143,34 +132,9 @@ public class ImmutableUpsertSegmentImpl extends ImmutableSegmentImpl implements
     LOGGER.info("built offset to DocId map for segment {} with {} documents in {} ms", _segmentName, _totalDoc, System.currentTimeMillis() - start);
   }
 
-  public static ImmutableUpsertSegmentImpl copyOf(ImmutableSegmentImpl immutableSegment) {
-
-    return new ImmutableUpsertSegmentImpl(immutableSegment._segmentDirectory, immutableSegment._segmentMetadata,
-        immutableSegment._indexContainerMap, immutableSegment._starTreeIndexContainer);
-  }
-
-  @Override
-  public void updateVirtualColumn(Iterable<UpdateLogEntry> logEntries) {
-    for (UpdateLogEntry logEntry: logEntries) {
-      boolean updated = false;
-      int docId = getDocIdFromSourceOffset(logEntry.getOffset());
-      for (VirtualColumnLongValueReaderWriter readerWriter : _virtualColumnsReaderWriter) {
-        updated = readerWriter.update(docId, logEntry.getValue(), logEntry.getType()) || updated;
-      }
-      if (updated) {
-        _upsertWaterMarkManager.processMessage(_tableNameWithType, _segmentName, logEntry);
-      }
-    }
-  }
-
   @Override
-  public String getVirtualColumnInfo(long offset) {
-    int docId = getDocIdFromSourceOffset(offset);
-    StringBuilder result = new StringBuilder("matched: ");
-    for (VirtualColumnLongValueReaderWriter readerWriter : _virtualColumnsReaderWriter) {
-      result.append(readerWriter.getInt(docId)).append("; ");
-    }
-    return result.toString();
+  public void postProcessRecords(GenericRow row, int docId) {
+    // do nothing
   }
 
   /**
@@ -216,7 +180,7 @@ public class ImmutableUpsertSegmentImpl extends ImmutableSegmentImpl implements
             unmatchedLogEntryCount);
       }
       partitionToHighestWatermark.forEach((partition, value) ->
-          _upsertWaterMarkManager.processVersionUpdate(_tableNameWithType, partition, value));
+          _upsertWatermarkManager.processVersionUpdate(_tableNameWithType, partition, value));
 
     } catch (Exception e) {
       LOGGER.error("failed to load the offset with thread pool");
@@ -226,6 +190,30 @@ public class ImmutableUpsertSegmentImpl extends ImmutableSegmentImpl implements
         _segmentName, System.currentTimeMillis() - start);
   }
 
+  @Override
+  public void updateVirtualColumn(Iterable<UpdateLogEntry> logEntries) {
+    for (UpdateLogEntry logEntry: logEntries) {
+      boolean updated = false;
+      int docId = getDocIdFromSourceOffset(logEntry.getOffset());
+      for (VirtualColumnLongValueReaderWriter readerWriter : _virtualColumnsReaderWriter) {
+        updated = readerWriter.update(docId, logEntry.getValue(), logEntry.getType()) || updated;
+      }
+      if (updated) {
+        _upsertWatermarkManager.processMessage(_tableNameWithType, _segmentName, logEntry);
+      }
+    }
+  }
+
+  @Override
+  public String getVirtualColumnInfo(long offset) {
+    int docId = getDocIdFromSourceOffset(offset);
+    StringBuilder result = new StringBuilder("matched: ");
+    for (VirtualColumnLongValueReaderWriter readerWriter : _virtualColumnsReaderWriter) {
+      result.append(readerWriter.getInt(docId)).append("; ");
+    }
+    return result.toString();
+  }
+
   /**
    * given a offset from source kafka topic, return the docId associated with it
    * throw exception if there is no docId for the associated kafka offset (because kafka offset might not be continuous)
@@ -246,6 +234,6 @@ public class ImmutableUpsertSegmentImpl extends ImmutableSegmentImpl implements
         return _sourceOffsetToDocIdArray[position];
       }
     }
-
   }
+
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableUpsertSegmentImpl.java b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/data/manager/upsert/UpsertMutableIndexSegmentCallback.java
similarity index 76%
rename from pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableUpsertSegmentImpl.java
rename to pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/data/manager/upsert/UpsertMutableIndexSegmentCallback.java
index a6e7728..eeda5a6 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableUpsertSegmentImpl.java
+++ b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/data/manager/upsert/UpsertMutableIndexSegmentCallback.java
@@ -16,18 +16,21 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.core.indexsegment.mutable;
+package org.apache.pinot.core.data.manager.upsert;
 
 import com.google.common.base.Preconditions;
-import org.apache.pinot.core.indexsegment.UpsertSegment;
+import org.apache.pinot.common.config.TableNameBuilder;
+import org.apache.pinot.common.segment.SegmentMetadata;
+import org.apache.pinot.common.utils.CommonConstants;
 import org.apache.pinot.core.io.reader.DataFileReader;
-import org.apache.pinot.core.realtime.impl.RealtimeSegmentConfig;
-import org.apache.pinot.core.segment.updater.UpsertWaterMarkManager;
+import org.apache.pinot.core.segment.index.column.ColumnIndexContainer;
+import org.apache.pinot.core.segment.updater.UpsertWatermarkManager;
 import org.apache.pinot.core.segment.virtualcolumn.mutable.VirtualColumnLongValueReaderWriter;
 import org.apache.pinot.grigio.common.messages.LogEventType;
 import org.apache.pinot.grigio.common.storageProvider.UpdateLogEntry;
 import org.apache.pinot.grigio.common.storageProvider.UpdateLogEntrySet;
 import org.apache.pinot.grigio.common.storageProvider.UpdateLogStorageProvider;
+import org.apache.pinot.spi.data.Schema;
 import org.apache.pinot.spi.data.readers.GenericRow;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -38,13 +41,15 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
-public class MutableUpsertSegmentImpl extends MutableSegmentImpl implements UpsertSegment {
+public class UpsertMutableIndexSegmentCallback implements IndexSegmentCallback {
 
-  private static final Logger LOGGER = LoggerFactory.getLogger(MutableUpsertSegmentImpl.class);
-  private final UpsertWaterMarkManager upsertWaterMarkManager;
-
-  private final String _kafkaOffsetColumnName;
+  private static final Logger LOGGER = LoggerFactory.getLogger(UpsertMutableIndexSegmentCallback.class);
 
+  private String _tableName;
+  private String _segmentName;
+  private Schema _schema;
+  private UpsertWatermarkManager _upsertWatermarkManager;
+  private String _offsetColumnName;
   private final List<VirtualColumnLongValueReaderWriter> _mutableSegmentReaderWriters = new ArrayList<>();
   // use map for mapping between kafka offset and docId because we at-most have 1 mutable segment per consumer
   // will use more memory, but we can update this later
@@ -54,23 +59,31 @@ public class MutableUpsertSegmentImpl extends MutableSegmentImpl implements Upse
   private final Map<Long, UpdateLogEntry> _unmatchedInsertRecords = new ConcurrentHashMap<>();
   private final Map<Long, UpdateLogEntry> _unmatchedDeleteRecords = new ConcurrentHashMap<>();
 
-  public MutableUpsertSegmentImpl(RealtimeSegmentConfig config) {
-    super(config);
-    _kafkaOffsetColumnName = _schema.getOffsetKey();
+  @Override
+  public void init(SegmentMetadata segmentMetadata, Map<String, DataFileReader> virtualColumnIndexReader) {
+    _tableName = TableNameBuilder.ensureTableNameWithType(segmentMetadata.getTableName(),
+        CommonConstants.Helix.TableType.REALTIME);
+    _segmentName = segmentMetadata.getName();
+    _schema = segmentMetadata.getSchema();
+    _offsetColumnName = _schema.getOffsetKey();
     Preconditions.checkState(_schema.isTableForUpsert(), "table should be upsert");
-    for (Map.Entry<String, DataFileReader> entry: _virtualColumnIndexReader.entrySet()) {
+    for (Map.Entry<String, DataFileReader> entry: virtualColumnIndexReader.entrySet()) {
       String column = entry.getKey();
       DataFileReader reader = entry.getValue();
       if (reader instanceof VirtualColumnLongValueReaderWriter) {
-        _logger.info("adding virtual column {} to updatable reader writer list", column);
+        LOGGER.info("adding virtual column {} to updatable reader writer list", column);
         _mutableSegmentReaderWriters.add((VirtualColumnLongValueReaderWriter) reader);
       }
     }
-    upsertWaterMarkManager = UpsertWaterMarkManager.getInstance();
+    _upsertWatermarkManager = UpsertWatermarkManager.getInstance();
     LOGGER.info("starting upsert segment with {} reader writer", _mutableSegmentReaderWriters.size());
   }
 
   @Override
+  public void initOffsetColumn(ColumnIndexContainer offsetColumnContainer) {
+    // do nothing
+  }
+
   public synchronized void updateVirtualColumn(Iterable<UpdateLogEntry> logEntries) {
     for (UpdateLogEntry logEntry: logEntries) {
       boolean updated = false;
@@ -83,7 +96,7 @@ public class MutableUpsertSegmentImpl extends MutableSegmentImpl implements Upse
         }
         if (updated) {
           // only update high water mark if it indeed updated something
-          upsertWaterMarkManager.processMessage(_tableName, _segmentName, logEntry);
+          _upsertWatermarkManager.processMessage(_tableName, _segmentName, logEntry);
         }
       }
       if (!offsetFound) {
@@ -92,7 +105,6 @@ public class MutableUpsertSegmentImpl extends MutableSegmentImpl implements Upse
     }
   }
 
-  @Override
   public String getVirtualColumnInfo(long offset) {
     Integer docId = _sourceOffsetToDocId.get(offset);
     StringBuilder result = new StringBuilder("matched: ");
@@ -113,8 +125,8 @@ public class MutableUpsertSegmentImpl extends MutableSegmentImpl implements Upse
   }
 
   @Override
-  protected synchronized void postProcessRecords(GenericRow row, int docId) {
-    final Long offset = (Long) row.getValue(_kafkaOffsetColumnName);
+  public synchronized void postProcessRecords(GenericRow row, int docId) {
+    final Long offset = (Long) row.getValue(_offsetColumnName);
     for (VirtualColumnLongValueReaderWriter readerWriter: _mutableSegmentReaderWriters) {
       readerWriter.addNewRecord(docId);
     }
@@ -123,9 +135,7 @@ public class MutableUpsertSegmentImpl extends MutableSegmentImpl implements Upse
     checkForOutstandingRecords(_unmatchedInsertRecords, offset, docId);
   }
 
-  @Override
   public void initVirtualColumn() throws IOException {
-    Preconditions.checkState(_numDocsIndexed == 0, "should init virtual column before ingestion");
     UpdateLogEntrySet updateLogEntries = UpdateLogStorageProvider.getInstance().getAllMessages(_tableName, _segmentName);
     LOGGER.info("got {} update log entries for current segment {}", updateLogEntries.size(), _segmentName);
     // some physical data might have been ingested when we init virtual column, we will go through the normal update
@@ -133,6 +143,7 @@ public class MutableUpsertSegmentImpl extends MutableSegmentImpl implements Upse
     updateVirtualColumn(updateLogEntries);
   }
 
+
   private void checkForOutstandingRecords(Map<Long, UpdateLogEntry> unmatchRecordsMap, Long offset, int docId) {
     UpdateLogEntry unmatchedEntry = unmatchRecordsMap.remove(offset);
     if (unmatchedEntry != null) {
@@ -141,7 +152,7 @@ public class MutableUpsertSegmentImpl extends MutableSegmentImpl implements Upse
         updated = readerWriter.update(docId, unmatchedEntry.getValue(), unmatchedEntry.getType()) || updated;
       }
       if (updated) {
-        upsertWaterMarkManager.processMessage(_tableName, _segmentName, unmatchedEntry);
+        _upsertWatermarkManager.processMessage(_tableName, _segmentName, unmatchedEntry);
       }
     }
   }
diff --git a/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/data/manager/upsert/UpsertTableDataManagerCallbackImpl.java b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/data/manager/upsert/UpsertTableDataManagerCallbackImpl.java
new file mode 100644
index 0000000..dc329e4
--- /dev/null
+++ b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/data/manager/upsert/UpsertTableDataManagerCallbackImpl.java
@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.data.manager.upsert;
+
+
+import org.apache.commons.lang3.NotImplementedException;
+import org.apache.commons.lang3.exception.ExceptionUtils;
+import org.apache.pinot.common.config.TableConfig;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.grigio.common.storageProvider.UpdateLogStorageProvider;
+import org.apache.pinot.spi.data.Schema;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+public class UpsertTableDataManagerCallbackImpl implements TableDataManagerCallback {
+  private static final Logger LOGGER = LoggerFactory.getLogger(UpsertTableDataManagerCallbackImpl.class);
+
+  private UpdateLogStorageProvider _updateLogStorageProvider;
+
+  @Override
+  public void init() {
+    _updateLogStorageProvider = UpdateLogStorageProvider.getInstance();
+  }
+
+  @Override
+  public void addSegment(String tableName, String segmentName, TableConfig tableConfig) {
+    try {
+      _updateLogStorageProvider.addSegment(tableName, segmentName);
+    } catch (IOException e) {
+      LOGGER.error("failed to add update log for segment {} {}", tableName, segmentName);
+      ExceptionUtils.rethrow(e);
+    }
+  }
+
+  @Override
+  public DataManagerCallback getDataManagerCallback(String tableName, String segmentName, Schema schema,
+      ServerMetrics serverMetrics, boolean isMutable) {
+    return new UpsertDataManagerCallbackImpl(tableName, segmentName, schema, serverMetrics, isMutable);
+  }
+
+  @Override
+  public DataManagerCallback getDefaultDataManagerCallback() {
+    throw new NotImplementedException("cannot create DefaultDataManagerCallback from UpsertTableDataManagerCallback");
+  }
+}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdater.java b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdater.java
similarity index 94%
rename from pinot-core/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdater.java
rename to pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdater.java
index d0d8c0e..530cd0d 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdater.java
+++ b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdater.java
@@ -26,7 +26,7 @@ import org.apache.commons.configuration.Configuration;
 import org.apache.pinot.common.config.TableNameBuilder;
 import org.apache.pinot.common.utils.CommonConstants;
 import org.apache.pinot.common.utils.LLCSegmentName;
-import org.apache.pinot.core.data.manager.UpsertSegmentDataManager;
+import org.apache.pinot.core.data.manager.upsert.DataManagerCallback;
 import org.apache.pinot.grigio.common.messages.LogCoordinatorMessage;
 import org.apache.pinot.grigio.common.metrics.GrigioMeter;
 import org.apache.pinot.grigio.common.metrics.GrigioMetrics;
@@ -75,7 +75,7 @@ public class SegmentUpdater implements SegmentDeletionListener {
   private final String _topicPrefix;
   private final ExecutorService _ingestionExecutorService;
   private final QueueConsumer _consumer;
-  private final Map<String, Map<String, Set<UpsertSegmentDataManager>>> _tableSegmentMap = new ConcurrentHashMap<>();
+  private final Map<String, Map<String, Set<DataManagerCallback>>> _tableSegmentMap = new ConcurrentHashMap<>();
   private final Map<String, Map<Integer, Long>> _tablePartitionCreationTime = new ConcurrentHashMap<>();
   private final UpdateLogStorageProvider _updateLogStorageProvider;
   private final UpdateLogRetentionManager _retentionManager;
@@ -91,7 +91,7 @@ public class SegmentUpdater implements SegmentDeletionListener {
     _topicPrefix = conf.getString(SegmentUpdaterConfig.INPUT_TOPIC_PREFIX);
     _updateSleepMs = conf.getInt(SegmentUpdaterConfig.SEGMENT_UDPATE_SLEEP_MS,
         SegmentUpdaterConfig.SEGMENT_UDPATE_SLEEP_MS_DEFAULT);
-    UpsertWaterMarkManager.init(metrics);
+    UpsertWatermarkManager.init(metrics);
     _consumer = provider.getConsumer();
     _ingestionExecutorService = Executors.newFixedThreadPool(1);
     _updateLogStorageProvider = UpdateLogStorageProvider.getInstance();
@@ -163,7 +163,7 @@ public class SegmentUpdater implements SegmentDeletionListener {
           String tableName = TableNameBuilder.ensureTableNameWithType(entry.getKey(), CommonConstants.Helix.TableType.REALTIME);
           int tableMessageCount = 0;
           if (_tableSegmentMap.containsKey(tableName)) {
-            final Map<String, Set<UpsertSegmentDataManager>> segmentManagersMap = _tableSegmentMap.get(tableName);
+            final Map<String, Set<DataManagerCallback>> segmentManagersMap = _tableSegmentMap.get(tableName);
             final TableUpdateLogs segment2UpdateLogsMap = entry.getValue();
             updateSegmentVirtualColumns(tableName, segmentManagersMap, segment2UpdateLogsMap, timeToStoreUpdateLogs);
           } else {
@@ -179,7 +179,7 @@ public class SegmentUpdater implements SegmentDeletionListener {
           if (System.currentTimeMillis() - lastReportedTime > LOGGER_TIME_GAP_MS) {
             lastReportedTime = System.currentTimeMillis();
             LOGGER.info("processed {} messages in {} ms", eventCount, System.currentTimeMillis() - loopStartTime);
-            LOGGER.info("latest high water mark is {}", UpsertWaterMarkManager.getInstance().toString());
+            LOGGER.info("latest high water mark is {}", UpsertWatermarkManager.getInstance().toString());
           }
           _consumer.ackOffset();
           _metrics.addTimedValueMs(GrigioTimer.SEGMENT_UPDATER_LOOP_TIME, System.currentTimeMillis() - loopStartTime);
@@ -202,7 +202,7 @@ public class SegmentUpdater implements SegmentDeletionListener {
   /**
    * Update the virtual columns of affected segments of a table.
    */
-  private void updateSegmentVirtualColumns(String tableName, Map<String, Set<UpsertSegmentDataManager>> segmentManagersMap,
+  private void updateSegmentVirtualColumns(String tableName, Map<String, Set<DataManagerCallback>> segmentManagersMap,
                                            TableUpdateLogs segment2UpdateLogsMap, AtomicLong timeToStoreUpdateLogs) throws IOException{
     for (Map.Entry<String, List<UpdateLogEntry>> segmentEntry : segment2UpdateLogsMap.getSegments2UpdateLog().entrySet()) {
       final String segmentNameStr = segmentEntry.getKey();
@@ -217,7 +217,7 @@ public class SegmentUpdater implements SegmentDeletionListener {
    * from consuming to online (mutable segment to immutable segment). In most of cases we expect only one segment manager
    * in this set of UpsertSegmentDataManager
    */
-  private void updateVirtualColumn(String table, String segment, Set<UpsertSegmentDataManager> segmentDataManagers,
+  private void updateVirtualColumn(String table, String segment, Set<DataManagerCallback> segmentDataManagers,
                                    List<UpdateLogEntry> messages, AtomicLong timeToStoreUpdateLogs) throws IOException {
     LOGGER.debug("updating segment {} with {} results for {} data managers", segment, messages.size(),
         segmentDataManagers.size());
@@ -225,7 +225,7 @@ public class SegmentUpdater implements SegmentDeletionListener {
       storeUpdateLogs(table, segment, messages, timeToStoreUpdateLogs);
     }
     try {
-      for (UpsertSegmentDataManager dataManager: segmentDataManagers) {
+      for (DataManagerCallback dataManager: segmentDataManagers) {
         dataManager.updateVirtualColumns(messages);
       }
     } catch (Exception ex) {
@@ -239,7 +239,8 @@ public class SegmentUpdater implements SegmentDeletionListener {
    * @param segmentName
    * @param dataManager the data manager for the current given table/segment combination
    */
-  public synchronized void addSegmentDataManager(String tableNameWithType, LLCSegmentName segmentName, UpsertSegmentDataManager dataManager) {
+  public synchronized void addSegmentDataManager(String tableNameWithType, LLCSegmentName segmentName,
+      DataManagerCallback dataManager) {
     // TODO get partition assignment from
     LOGGER.info("segment updater adding table {} segment {}", tableNameWithType, segmentName.getSegmentName());
     if (!_tableSegmentMap.containsKey(tableNameWithType)) {
@@ -258,11 +259,12 @@ public class SegmentUpdater implements SegmentDeletionListener {
     }
   }
 
-  public synchronized void removeSegmentDataManager(String tableNameWithType, String segmentName, UpsertSegmentDataManager toDeleteManager) {
+  public synchronized void removeSegmentDataManager(String tableNameWithType, String segmentName,
+      DataManagerCallback toDeleteManager) {
     LOGGER.info("segment updater removing table {} segment {}", tableNameWithType, segmentName);
-    Map<String, Set<UpsertSegmentDataManager>> segmentMap = _tableSegmentMap.get(tableNameWithType);
+    Map<String, Set<DataManagerCallback>> segmentMap = _tableSegmentMap.get(tableNameWithType);
     if (segmentMap != null) {
-      Set<UpsertSegmentDataManager> segmentDataManagers = segmentMap.get(segmentName);
+      Set<DataManagerCallback> segmentDataManagers = segmentMap.get(segmentName);
       if (segmentDataManagers != null) {
         segmentDataManagers.remove(toDeleteManager);
         if (segmentDataManagers.size() == 0) {
@@ -298,7 +300,7 @@ public class SegmentUpdater implements SegmentDeletionListener {
   @Override
   public synchronized void onSegmentDeletion(String tableNameWithType, String segmentName) {
     LOGGER.info("deleting segment virtual column from local storage for table {} segment {}", tableNameWithType, segmentName);
-    Map<String, Set<UpsertSegmentDataManager>> segmentManagerMap = _tableSegmentMap.get(tableNameWithType);
+    Map<String, Set<DataManagerCallback>> segmentManagerMap = _tableSegmentMap.get(tableNameWithType);
     if (segmentManagerMap != null) {
       if (segmentManagerMap.containsKey(segmentName) && segmentManagerMap.get(segmentName).size() > 0) {
         LOGGER.error("trying to remove segment storage with {} segment data manager", segmentManagerMap.get(segmentName).size());
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdaterConfig.java b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdaterConfig.java
similarity index 100%
copy from pinot-core/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdaterConfig.java
copy to pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdaterConfig.java
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/UpsertWaterMarkManager.java b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/segment/updater/UpsertWatermarkManager.java
similarity index 86%
rename from pinot-core/src/main/java/org/apache/pinot/core/segment/updater/UpsertWaterMarkManager.java
rename to pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/segment/updater/UpsertWatermarkManager.java
index bbea037..faad27b 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/UpsertWaterMarkManager.java
+++ b/pinot-grigio/pinot-grigio-provided/src/main/java/org/apache/pinot/core/segment/updater/UpsertWatermarkManager.java
@@ -21,6 +21,7 @@ package org.apache.pinot.core.segment.updater;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableMap;
+import org.apache.commons.configuration.Configuration;
 import org.apache.pinot.grigio.common.metrics.GrigioGauge;
 import org.apache.pinot.grigio.common.metrics.GrigioMetrics;
 import org.apache.pinot.grigio.common.storageProvider.UpdateLogEntry;
@@ -30,26 +31,26 @@ import org.slf4j.LoggerFactory;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
-public class UpsertWaterMarkManager {
+public class UpsertWatermarkManager implements WatermarkManager {
 
   private final Map<String, Map<Integer, Long>> _highWaterMarkTablePartitionMap = new ConcurrentHashMap<>();
   private final GrigioMetrics _metrics;
 
-  private static final Logger LOGGER = LoggerFactory.getLogger(UpsertWaterMarkManager.class);
-  private static volatile UpsertWaterMarkManager _instance;
+  private static final Logger LOGGER = LoggerFactory.getLogger(UpsertWatermarkManager.class);
+  private static volatile UpsertWatermarkManager _instance;
 
-  private UpsertWaterMarkManager(GrigioMetrics metrics) {
+  private UpsertWatermarkManager(GrigioMetrics metrics) {
     _metrics = metrics;
   }
 
   public static void init(GrigioMetrics metrics) {
-    synchronized (UpsertWaterMarkManager.class) {
+    synchronized (UpsertWatermarkManager.class) {
       Preconditions.checkState(_instance == null, "upsert water mark manager is already init");
-      _instance = new UpsertWaterMarkManager(metrics);
+      _instance = new UpsertWatermarkManager(metrics);
     }
   }
 
-  public static UpsertWaterMarkManager getInstance() {
+  public static UpsertWatermarkManager getInstance() {
     Preconditions.checkState(_instance != null, "upsert water mark manager is not yet init");
     return _instance;
   }
@@ -79,6 +80,11 @@ public class UpsertWaterMarkManager {
     return ImmutableMap.copyOf(_highWaterMarkTablePartitionMap.getOrDefault(tableName, ImmutableMap.of()));
   }
 
+  @Override
+  public void init(Configuration config, GrigioMetrics metrics) {
+
+  }
+
   public Map<String, Map<Integer, Long>> getHighWaterMarkTablePartitionMap() {
     return ImmutableMap.copyOf(_highWaterMarkTablePartitionMap);
   }
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/indexsegment/immutable/ImmutableUpsertSegmentImplTest.java b/pinot-grigio/pinot-grigio-provided/src/test/java/org/apache/pinot/core/data/manager/upsert/UpsertImmutableIndexSegmentCallbackTest.java
similarity index 89%
rename from pinot-core/src/test/java/org/apache/pinot/core/indexsegment/immutable/ImmutableUpsertSegmentImplTest.java
rename to pinot-grigio/pinot-grigio-provided/src/test/java/org/apache/pinot/core/data/manager/upsert/UpsertImmutableIndexSegmentCallbackTest.java
index 817131c..979e5c1 100644
--- a/pinot-core/src/test/java/org/apache/pinot/core/indexsegment/immutable/ImmutableUpsertSegmentImplTest.java
+++ b/pinot-grigio/pinot-grigio-provided/src/test/java/org/apache/pinot/core/data/manager/upsert/UpsertImmutableIndexSegmentCallbackTest.java
@@ -16,9 +16,9 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.core.indexsegment.immutable;
+package org.apache.pinot.core.data.manager.upsert;
 
-import org.apache.pinot.core.segment.updater.UpsertWaterMarkManager;
+import org.apache.pinot.core.segment.updater.UpsertWatermarkManager;
 import org.apache.pinot.core.segment.virtualcolumn.mutable.VirtualColumnLongValueReaderWriter;
 import org.apache.pinot.grigio.common.messages.LogEventType;
 import org.apache.pinot.grigio.common.storageProvider.UpdateLogEntry;
@@ -38,16 +38,15 @@ import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 
-public class ImmutableUpsertSegmentImplTest {
-
+public class UpsertImmutableIndexSegmentCallbackTest {
   UpdateLogStorageProvider _mockProvider;
-  UpsertWaterMarkManager _mockUpsertWaterMarkManager;
+  UpsertWatermarkManager _mockUpsertWatermarkManager;
   List<VirtualColumnLongValueReaderWriter> _readerWriters = new ArrayList<>();
 
   @BeforeMethod
   public void init() {
     _mockProvider = mock(UpdateLogStorageProvider.class);
-    _mockUpsertWaterMarkManager = mock(UpsertWaterMarkManager.class);
+    _mockUpsertWatermarkManager = mock(UpsertWatermarkManager.class);
   }
 
   @Test
@@ -105,10 +104,10 @@ public class ImmutableUpsertSegmentImplTest {
 
     start = System.currentTimeMillis();
 
-    ImmutableUpsertSegmentImpl immutableUpsertSegment = new ImmutableUpsertSegmentImpl(_readerWriters, totalDocs,
-        _mockUpsertWaterMarkManager, _mockProvider, minOffset, offsetToDocId);
+    UpsertImmutableIndexSegmentCallback callback = new UpsertImmutableIndexSegmentCallback();
+    callback.init(_readerWriters, totalDocs, _mockUpsertWatermarkManager, _mockProvider, minOffset, offsetToDocId);
+    callback.initVirtualColumn();
 
-    immutableUpsertSegment.initVirtualColumn();
     long runtime = System.currentTimeMillis() - start;
     System.out.println("run time is " + runtime);
     // on regular developer laptop this should take less 1 second, but on integration server this might be longer
diff --git a/pinot-grigio/pom.xml b/pinot-grigio/pom.xml
index e53fa72..47ad035 100644
--- a/pinot-grigio/pom.xml
+++ b/pinot-grigio/pom.xml
@@ -35,6 +35,7 @@
     <modules>
       <module>pinot-grigio-common</module>
       <module>pinot-grigio-coordinator</module>
+      <module>pinot-grigio-provided</module>
     </modules>
     <properties>
     <pinot.root>${basedir}/..</pinot.root>
diff --git a/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixInstanceDataManager.java b/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixInstanceDataManager.java
index 1af7e29..920b422 100644
--- a/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixInstanceDataManager.java
+++ b/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixInstanceDataManager.java
@@ -41,11 +41,11 @@ import org.apache.pinot.core.data.manager.SegmentDataManager;
 import org.apache.pinot.core.data.manager.TableDataManager;
 import org.apache.pinot.core.data.manager.config.TableDataManagerConfig;
 import org.apache.pinot.core.data.manager.offline.TableDataManagerProvider;
+import org.apache.pinot.core.data.manager.upsert.DataManagerCallback;
 import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
 import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
 import org.apache.pinot.core.segment.index.loader.IndexLoadingConfig;
 import org.apache.pinot.core.segment.index.loader.LoaderUtils;
-import org.apache.pinot.core.segment.updater.UpsertWaterMarkManager;
 import org.apache.pinot.spi.data.Schema;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -219,12 +219,18 @@ public class HelixInstanceDataManager implements InstanceDataManager {
       // Copy from segment backup directory back to index directory
       FileUtils.copyDirectory(segmentBackupDir, indexDir);
 
+      final TableDataManager tableDataManager = _tableDataManagerMap.get(tableNameWithType);
+      final DataManagerCallback dataManagerCallback = tableDataManager.getTableDataManagerCallback()
+          .getDataManagerCallback(tableNameWithType, segmentName, schema, _serverMetrics, false);
+
       // Load from index directory
-      ImmutableSegment immutableSegment = ImmutableSegmentLoader
-          .load(indexDir, new IndexLoadingConfig(_instanceDataManagerConfig, tableConfig), schema);
+      ImmutableSegment immutableSegment = ImmutableSegmentLoader.load(indexDir,
+          new IndexLoadingConfig(_instanceDataManagerConfig, tableConfig),
+          dataManagerCallback,
+          schema);
 
       // Replace the old segment in memory
-      _tableDataManagerMap.get(tableNameWithType).addSegment(immutableSegment);
+      _tableDataManagerMap.get(tableNameWithType).addSegment(immutableSegment, dataManagerCallback);
 
       // Rename segment backup directory to segment temporary directory (atomic)
       // The reason to first rename then delete is that, renaming is an atomic operation, but deleting is not. When we
@@ -301,10 +307,12 @@ public class HelixInstanceDataManager implements InstanceDataManager {
     }
   }
 
+  /*
   @Override
   public Map<String, Map<Integer, Long>> getLowWaterMarks() {
     return UpsertWaterMarkManager.getInstance().getHighWaterMarkTablePartitionMap();
   }
+   */
 
   @Override
   public String getSegmentDataDirectory() {
diff --git a/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixServerStarter.java b/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixServerStarter.java
index f46023d..c1ea182 100644
--- a/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixServerStarter.java
+++ b/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixServerStarter.java
@@ -64,6 +64,8 @@ import org.apache.pinot.server.conf.ServerConf;
 import org.apache.pinot.server.realtime.ControllerLeaderLocator;
 import org.apache.pinot.server.realtime.ServerSegmentCompletionProtocolHandler;
 import org.apache.pinot.server.starter.ServerInstance;
+import org.apache.pinot.server.upsert.UpsertComponentContainer;
+import org.apache.pinot.server.upsert.UpsertComponentContainerProvider;
 import org.apache.pinot.spi.filesystem.PinotFSFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
diff --git a/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/SegmentOnlineOfflineStateModelFactory.java b/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/SegmentOnlineOfflineStateModelFactory.java
index 910a6b9..145e378 100644
--- a/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/SegmentOnlineOfflineStateModelFactory.java
+++ b/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/SegmentOnlineOfflineStateModelFactory.java
@@ -39,6 +39,7 @@ import org.apache.pinot.core.data.manager.InstanceDataManager;
 import org.apache.pinot.core.data.manager.SegmentDataManager;
 import org.apache.pinot.core.data.manager.TableDataManager;
 import org.apache.pinot.core.data.manager.realtime.LLRealtimeSegmentDataManager;
+import org.apache.pinot.server.upsert.SegmentDeletionHandler;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -53,12 +54,19 @@ public class SegmentOnlineOfflineStateModelFactory extends StateModelFactory<Sta
   private final String _instanceId;
   private final InstanceDataManager _instanceDataManager;
   private final SegmentFetcherAndLoader _fetcherAndLoader;
+  private final SegmentDeletionHandler _segmentDeletionHandler;
 
   public SegmentOnlineOfflineStateModelFactory(String instanceId, InstanceDataManager instanceDataManager,
       SegmentFetcherAndLoader fetcherAndLoader) {
+    this(instanceId, instanceDataManager, fetcherAndLoader, new SegmentDeletionHandler());
+  }
+
+  public SegmentOnlineOfflineStateModelFactory(String instanceId, InstanceDataManager instanceDataManager,
+      SegmentFetcherAndLoader fetcherAndLoader, SegmentDeletionHandler segmentDeletionHandler) {
     _instanceId = instanceId;
     _instanceDataManager = instanceDataManager;
     _fetcherAndLoader = fetcherAndLoader;
+    _segmentDeletionHandler = segmentDeletionHandler;
   }
 
   public static String getStateModelName() {
@@ -203,6 +211,7 @@ public class SegmentOnlineOfflineStateModelFactory extends StateModelFactory<Sta
           FileUtils.deleteQuietly(segmentDir);
           _logger.info("Deleted segment directory {}", segmentDir);
         }
+        _segmentDeletionHandler.deleteSegmentFromLocalStorage(tableNameWithType, segmentName);
       } catch (final Exception e) {
         _logger.error("Cannot delete the segment : " + segmentName + " from local directory!\n" + e.getMessage(), e);
         Utils.rethrowException(e);
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableAppendSegmentImpl.java b/pinot-server/src/main/java/org/apache/pinot/server/upsert/DefaultUpsertComponentContainer.java
similarity index 57%
rename from pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableAppendSegmentImpl.java
rename to pinot-server/src/main/java/org/apache/pinot/server/upsert/DefaultUpsertComponentContainer.java
index 1c9effc..63424ac 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableAppendSegmentImpl.java
+++ b/pinot-server/src/main/java/org/apache/pinot/server/upsert/DefaultUpsertComponentContainer.java
@@ -16,22 +16,34 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.core.indexsegment.mutable;
+package org.apache.pinot.server.upsert;
 
-import org.apache.pinot.core.realtime.impl.RealtimeSegmentConfig;
-import org.apache.pinot.spi.data.readers.GenericRow;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import com.codahale.metrics.MetricRegistry;
+import org.apache.commons.configuration.Configuration;
 
-public class MutableAppendSegmentImpl extends MutableSegmentImpl {
-  private static final Logger LOGGER = LoggerFactory.getLogger(MutableUpsertSegmentImpl.class);
+public class DefaultUpsertComponentContainer implements UpsertComponentContainer {
 
-  public MutableAppendSegmentImpl(RealtimeSegmentConfig config) {
-    super(config);
+  private final SegmentDeletionHandler deletionHandler = new SegmentDeletionHandler();
+
+  @Override
+  public void registerMetrics(MetricRegistry registry) {
+
+  }
+
+  @Override
+  public void init(Configuration config) {
+  }
+
+  @Override
+  public SegmentDeletionHandler getSegmentDeletionHandler() {
+    return deletionHandler;
+  }
+
+  @Override
+  public void start() {
   }
 
   @Override
-  protected void postProcessRecords(GenericRow row, int docId) {
-    // nothing
+  public void stop() {
   }
 }
diff --git a/pinot-server/src/main/java/org/apache/pinot/server/upsert/SegmentDeletionHandler.java b/pinot-server/src/main/java/org/apache/pinot/server/upsert/SegmentDeletionHandler.java
new file mode 100644
index 0000000..1aba06b
--- /dev/null
+++ b/pinot-server/src/main/java/org/apache/pinot/server/upsert/SegmentDeletionHandler.java
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.server.upsert;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.pinot.core.segment.updater.SegmentDeletionListener;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+public class SegmentDeletionHandler {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(SegmentDeletionHandler.class);
+
+  private List<SegmentDeletionListener> _deleteListeners;
+
+
+  public SegmentDeletionHandler() {
+    this(ImmutableList.of());
+  }
+
+  public SegmentDeletionHandler(List<SegmentDeletionListener> listeners) {
+    _deleteListeners = listeners;
+  }
+
+  public void deleteSegmentFromLocalStorage(String tableNameWithType, String segmentName) {
+    //TODO move the deletion from file system logic to here
+    LOGGER.info("trying to perform deletion for {}:{} on {} deletion handler", tableNameWithType, segmentName,
+        _deleteListeners.size());
+    for (SegmentDeletionListener deletionListener : _deleteListeners) {
+      try {
+        deletionListener.onSegmentDeletion(tableNameWithType, segmentName);
+      } catch (Exception ex) {
+        LOGGER.error("failed to delete table {} segment {} with exception", tableNameWithType, segmentName, ex);
+      }
+    }
+  }
+}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdaterConfig.java b/pinot-server/src/main/java/org/apache/pinot/server/upsert/UpsertComponentContainer.java
similarity index 69%
rename from pinot-core/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdaterConfig.java
rename to pinot-server/src/main/java/org/apache/pinot/server/upsert/UpsertComponentContainer.java
index 97d48f3..7f636fc 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/segment/updater/SegmentUpdaterConfig.java
+++ b/pinot-server/src/main/java/org/apache/pinot/server/upsert/UpsertComponentContainer.java
@@ -16,10 +16,20 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.core.segment.updater;
+package org.apache.pinot.server.upsert;
 
-public class SegmentUpdaterConfig {
-  public static final String SEGMENT_UDPATE_SLEEP_MS = "sleep.ms";
-  public static final String INPUT_TOPIC_PREFIX  = "input.topic.prefix";
-  public static final int SEGMENT_UDPATE_SLEEP_MS_DEFAULT = 100;
+import com.codahale.metrics.MetricRegistry;
+import org.apache.commons.configuration.Configuration;
+
+public interface UpsertComponentContainer {
+
+  void registerMetrics(MetricRegistry registry);
+
+  void init(Configuration config);
+
+  SegmentDeletionHandler getSegmentDeletionHandler();
+
+  void start();
+
+  void stop();
 }
diff --git a/pinot-server/src/main/java/org/apache/pinot/server/upsert/UpsertComponentContainerProvider.java b/pinot-server/src/main/java/org/apache/pinot/server/upsert/UpsertComponentContainerProvider.java
new file mode 100644
index 0000000..fe4af07
--- /dev/null
+++ b/pinot-server/src/main/java/org/apache/pinot/server/upsert/UpsertComponentContainerProvider.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.server.upsert;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.configuration.Configuration;
+import org.apache.commons.lang3.exception.ExceptionUtils;
+import org.apache.pinot.core.segment.updater.WatermarkManager;
+import org.apache.pinot.grigio.common.metrics.GrigioMetrics;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class UpsertComponentContainerProvider {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(UpsertComponentContainerProvider.class);
+
+  public static final String UPSERT_COMPONENT_CONFIG_KEY = "watermarkManager.class";
+  public static final String UPSERT_COMPONENT_CONFIG_DEFAULT = DefaultUpsertComponentContainer.class.getName();
+
+  private final Configuration _conf;
+  private UpsertComponentContainer _instance;
+
+  public UpsertComponentContainerProvider(Configuration conf, GrigioMetrics metrics) {
+    _conf = conf;
+    String className = _conf.getString(UPSERT_COMPONENT_CONFIG_KEY, UPSERT_COMPONENT_CONFIG_DEFAULT);
+    LOGGER.info("creating watermark manager with class {}", className);
+    try {
+      Class<UpsertComponentContainer> comonentContainerClass = (Class<UpsertComponentContainer>) Class.forName(className);
+      Preconditions.checkState(comonentContainerClass.isAssignableFrom(WatermarkManager.class),
+          "configured class not assignable from Callback class");
+      _instance = comonentContainerClass.newInstance();
+      _instance.init(_conf);
+    } catch (Exception e) {
+      LOGGER.error("failed to load watermark manager class", className, e);
+      ExceptionUtils.rethrow(e);
+    }
+  }
+
+  public UpsertComponentContainer getInstance() {
+    return _instance;
+  }
+}
diff --git a/pinot-server/src/test/java/org/apache/pinot/server/api/BaseResourceTest.java b/pinot-server/src/test/java/org/apache/pinot/server/api/BaseResourceTest.java
index 08ab8c3..60702c4 100644
--- a/pinot-server/src/test/java/org/apache/pinot/server/api/BaseResourceTest.java
+++ b/pinot-server/src/test/java/org/apache/pinot/server/api/BaseResourceTest.java
@@ -35,6 +35,7 @@ import org.apache.pinot.core.data.manager.InstanceDataManager;
 import org.apache.pinot.core.data.manager.TableDataManager;
 import org.apache.pinot.core.data.manager.config.TableDataManagerConfig;
 import org.apache.pinot.core.data.manager.offline.TableDataManagerProvider;
+import org.apache.pinot.core.data.manager.upsert.DefaultDataManagerCallbackImpl;
 import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
 import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment;
 import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
@@ -122,7 +123,7 @@ public abstract class BaseResourceTest {
     ImmutableSegment immutableSegment =
         ImmutableSegmentLoader.load(new File(INDEX_DIR, driver.getSegmentName()), ReadMode.mmap);
     _indexSegments.add(immutableSegment);
-    _tableDataManagerMap.get(TABLE_NAME).addSegment(immutableSegment);
+    _tableDataManagerMap.get(TABLE_NAME).addSegment(immutableSegment, DefaultDataManagerCallbackImpl.INSTANCE);
     return immutableSegment;
   }
 
diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/realtime/provisioning/MemoryEstimator.java b/pinot-tools/src/main/java/org/apache/pinot/tools/realtime/provisioning/MemoryEstimator.java
index 187b36f..ac989bb 100644
--- a/pinot-tools/src/main/java/org/apache/pinot/tools/realtime/provisioning/MemoryEstimator.java
+++ b/pinot-tools/src/main/java/org/apache/pinot/tools/realtime/provisioning/MemoryEstimator.java
@@ -26,7 +26,8 @@ import java.util.stream.Collectors;
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.io.FileUtils;
 import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.core.indexsegment.mutable.MutableAppendSegmentImpl;
+import org.apache.pinot.core.data.manager.upsert.DefaultDataManagerCallbackImpl;
+import org.apache.pinot.core.data.manager.upsert.DefaultIndexSegmentCallback;
 import org.apache.pinot.spi.data.FieldSpec;
 import org.apache.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
 import org.apache.pinot.spi.utils.DataSize;
@@ -136,7 +137,8 @@ public class MemoryEstimator {
             .setStatsHistory(sampleStatsHistory);
 
     // create mutable segment impl
-    MutableSegmentImpl mutableSegmentImpl = new MutableAppendSegmentImpl(realtimeSegmentConfigBuilder.build());
+    MutableSegmentImpl mutableSegmentImpl = new MutableSegmentImpl(realtimeSegmentConfigBuilder.build(),
+        DefaultIndexSegmentCallback.INSTANCE);
 
     // read all rows and index them
     try (PinotSegmentRecordReader segmentRecordReader = new PinotSegmentRecordReader(_sampleCompletedSegment);) {
@@ -236,7 +238,8 @@ public class MemoryEstimator {
               .setOffHeap(true).setMemoryManager(memoryManager).setStatsHistory(statsHistory);
 
       // create mutable segment impl
-      MutableSegmentImpl mutableSegmentImpl = new MutableAppendSegmentImpl(realtimeSegmentConfigBuilder.build());
+      MutableSegmentImpl mutableSegmentImpl = new MutableSegmentImpl(realtimeSegmentConfigBuilder.build(),
+          DefaultIndexSegmentCallback.INSTANCE);
       long memoryForConsumingSegmentPerPartition = memoryManager.getTotalAllocatedBytes();
       mutableSegmentImpl.destroy();
       FileUtils.deleteQuietly(statsFileCopy);


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


[incubator-pinot] 07/09: fix bug

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jamesshao pushed a commit to branch upsert-refactor
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git

commit 32899a4f81418b1512c3edba496b7b17bfa0bd55
Author: james Shao <sj...@uber.com>
AuthorDate: Wed Feb 26 10:51:30 2020 -0800

    fix bug
    
    Differential Revision: https://code.uberinternal.com/D3998717
---
 .../org/apache/pinot/core/indexsegment/mutable/MutableSegmentImpl.java   | 1 -
 1 file changed, 1 deletion(-)

diff --git a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImpl.java b/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImpl.java
index 12fcd94..4bd67e3 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImpl.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImpl.java
@@ -191,7 +191,6 @@ public class MutableSegmentImpl implements MutableSegment {
         }
       } else {
         virtualFieldSpecs.add(fieldSpec);
-
       }
     }
     _physicalFieldSpecs = Collections.unmodifiableCollection(physicalFieldSpecs);


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