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:31 UTC

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

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