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 2019/10/10 19:31:40 UTC

[incubator-pinot] 01/01: [Resource Assignment] Plug in resource assignment to LLC REALTIME table

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

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

commit 171a0dbb024e12aba51ecdfad71b4ef78ca9acee
Author: Jackie (Xiaotian) Jiang <xa...@linkedin.com>
AuthorDate: Thu Oct 10 12:08:34 2019 -0700

    [Resource Assignment] Plug in resource assignment to LLC REALTIME table
    
    This is the last PR to plug in the new resource assignment, after this PR, all the segment assignment/rebalance will use the new strategy.
    For LLC REALTIME table, to keep it backward-compatible, allow assigning new segment to disabled instances
    
    Re-write most of the PinotLLCRealtimeSegmentManager to use the new assignment strategy:
    - Remove the boolean return and change it to throw exception when operation failed (old code can throw exception and also return false, which is hard to manage)
    - Remove the controller leadership check inside the PinotLLCRealtimeSegmentManager to prevent responding FAILED in case of NOT_LEADER. The check is already performed in SegmentCompletionManager.
    - Enhance the segment repair logic to correctly skip OFFLINE segments
    - Introduce the LLC_ZOOKEEPER_FETCH_FAILURES in controller metrics to track ZK exceptions
    
    Re-write the PinotLLCRealtimeSegmentManagerTest to test all the changes:
    - testCommittingSegmentIfDisconnected is removed because we don't explicitly check disconnection but fail with the ZK exception
    - testIdealStateAlreadyUpdated is covered by testCommitSegment committing the segment again
---
 .../PartitionAwareOfflineRoutingTableBuilder.java  |   23 +-
 ...rtitionAwareOfflineRoutingTableBuilderTest.java |   44 +-
 .../common/assignment/InstancePartitionsUtils.java |   32 +-
 .../apache/pinot/common/config/IndexingConfig.java |   11 -
 .../common/config/StreamConsumptionConfig.java     |   35 -
 .../apache/pinot/common/config/TableConfig.java    |   10 -
 .../apache/pinot/common/config/TagNameUtils.java   |   15 -
 .../pinot/common/metrics/ControllerMeter.java      |    4 +-
 .../common/partition/PartitionAssignment.java      |  109 -
 ...roupBasedStreamPartitionAssignmentStrategy.java |   83 -
 .../partition/ReplicaGroupPartitionAssignment.java |  124 --
 .../ReplicaGroupPartitionAssignmentGenerator.java  |  128 --
 .../StreamPartitionAssignmentGenerator.java        |  139 --
 .../StreamPartitionAssignmentStrategy.java         |   40 -
 .../StreamPartitionAssignmentStrategyFactory.java  |   45 -
 .../UniformStreamPartitionAssignmentStrategy.java  |   66 -
 .../pinot/common/config/TableConfigTest.java       |   23 -
 .../common/partition/IdealStateBuilderUtil.java    |  179 --
 ...icaGroupBasedStreamPartitionAssignmentTest.java |  136 --
 .../StreamPartitionAssignmentGeneratorTest.java    |  364 ----
 .../UniformStreamPartitionAssignmentTest.java      |   93 -
 .../apache/pinot/controller/ControllerStarter.java |    2 +-
 .../helix/core/PinotHelixResourceManager.java      |   74 +-
 .../helix/core/PinotTableIdealStateBuilder.java    |    7 +-
 .../assignment/segment/SegmentAssignmentUtils.java |    3 +-
 .../realtime/PinotLLCRealtimeSegmentManager.java   | 1383 +++++--------
 .../core/realtime/SegmentCompletionManager.java    |   61 +-
 .../helix/core/realtime/TableConfigCache.java      |   12 +-
 .../segment/DefaultFlushThresholdUpdater.java      |   41 +-
 .../segment/FlushThresholdUpdateManager.java       |   31 +-
 .../realtime/segment/FlushThresholdUpdater.java    |   19 +-
 .../SegmentSizeBasedFlushThresholdUpdater.java     |   11 +-
 .../RealtimeSegmentValidationManager.java          |    8 +-
 .../PinotLLCRealtimeSegmentManagerTest.java        | 2114 +++++++-------------
 .../helix/core/realtime/SegmentCompletionTest.java |   32 +-
 .../segment/FlushThresholdUpdaterTest.java         |  394 ++--
 .../ConsumingSegmentAssignmentStrategy.java        |   61 -
 .../segment/RealtimeSegmentAssignmentStrategy.java |   41 -
 .../stream/PartitionLevelStreamConfig.java         |   70 +-
 .../pinot/core/realtime/stream/StreamConfig.java   |   88 +-
 .../ConsumingSegmentAssignmentStrategyTest.java    |  285 ---
 41 files changed, 1635 insertions(+), 4805 deletions(-)

diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/builder/PartitionAwareOfflineRoutingTableBuilder.java b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/builder/PartitionAwareOfflineRoutingTableBuilder.java
index 19305ea..d8df632 100644
--- a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/builder/PartitionAwareOfflineRoutingTableBuilder.java
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/builder/PartitionAwareOfflineRoutingTableBuilder.java
@@ -18,6 +18,7 @@
  */
 package org.apache.pinot.broker.routing.builder;
 
+import com.google.common.base.Preconditions;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -28,14 +29,15 @@ import org.apache.helix.ZNRecord;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.store.zk.ZkHelixPropertyStore;
+import org.apache.pinot.common.assignment.InstancePartitions;
+import org.apache.pinot.common.assignment.InstancePartitionsType;
+import org.apache.pinot.common.assignment.InstancePartitionsUtils;
 import org.apache.pinot.common.config.TableConfig;
 import org.apache.pinot.common.metadata.ZKMetadataProvider;
 import org.apache.pinot.common.metadata.segment.ColumnPartitionMetadata;
 import org.apache.pinot.common.metadata.segment.SegmentPartitionMetadata;
 import org.apache.pinot.common.metadata.segment.SegmentZKMetadata;
 import org.apache.pinot.common.metrics.BrokerMetrics;
-import org.apache.pinot.common.partition.ReplicaGroupPartitionAssignment;
-import org.apache.pinot.common.partition.ReplicaGroupPartitionAssignmentGenerator;
 import org.apache.pinot.common.utils.CommonConstants;
 
 
@@ -88,14 +90,16 @@ public class PartitionAwareOfflineRoutingTableBuilder extends BasePartitionAware
     RoutingTableInstancePruner instancePruner = new RoutingTableInstancePruner(instanceConfigs);
     Set<String> segmentSet = externalView.getPartitionSet();
 
-    // Fetch the partition to replica group mapping table from the property store
-    ReplicaGroupPartitionAssignmentGenerator partitionAssignmentGenerator =
-        new ReplicaGroupPartitionAssignmentGenerator(_propertyStore);
-    ReplicaGroupPartitionAssignment partitionAssignment =
-        partitionAssignmentGenerator.getReplicaGroupPartitionAssignment(tableName);
+    // Fetch the instance partitions from the property store
+    String instancePartitionsName =
+        InstancePartitionsUtils.getInstancePartitionsName(tableName, InstancePartitionsType.OFFLINE);
+    InstancePartitions instancePartitions =
+        InstancePartitionsUtils.fetchInstancePartitions(_propertyStore, instancePartitionsName);
+    Preconditions
+        .checkState(instancePartitions != null, "Failed to find instance partitions: %s", instancePartitionsName);
 
     // Update numReplicas if the replica group partition assignment has been changed.
-    int numReplicas = partitionAssignment.getNumReplicaGroups();
+    int numReplicas = instancePartitions.getNumReplicaGroups();
     if (_numReplicas != numReplicas) {
       _numReplicas = numReplicas;
     }
@@ -121,8 +125,7 @@ public class PartitionAwareOfflineRoutingTableBuilder extends BasePartitionAware
     Map<Integer, Map<String, Integer>> partitionToServerToReplicaMap = new HashMap<>();
     for (Integer partitionId : partitionIds) {
       for (int replicaId = 0; replicaId < _numReplicas; replicaId++) {
-        List<String> serversForPartitionAndReplica =
-            partitionAssignment.getInstancesFromReplicaGroup(partitionId, replicaId);
+        List<String> serversForPartitionAndReplica = instancePartitions.getInstances(partitionId, replicaId);
         for (String serverName : serversForPartitionAndReplica) {
           Map<String, Integer> serverToReplicaMap =
               partitionToServerToReplicaMap.computeIfAbsent(partitionId, k -> new HashMap<>());
diff --git a/pinot-broker/src/test/java/org/apache/pinot/broker/routing/builder/PartitionAwareOfflineRoutingTableBuilderTest.java b/pinot-broker/src/test/java/org/apache/pinot/broker/routing/builder/PartitionAwareOfflineRoutingTableBuilderTest.java
index e71dd8a..52b2d82 100644
--- a/pinot-broker/src/test/java/org/apache/pinot/broker/routing/builder/PartitionAwareOfflineRoutingTableBuilderTest.java
+++ b/pinot-broker/src/test/java/org/apache/pinot/broker/routing/builder/PartitionAwareOfflineRoutingTableBuilderTest.java
@@ -30,6 +30,9 @@ import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.pinot.broker.routing.RoutingTableLookupRequest;
 import org.apache.pinot.broker.util.FakePropertyStore;
+import org.apache.pinot.common.assignment.InstancePartitions;
+import org.apache.pinot.common.assignment.InstancePartitionsType;
+import org.apache.pinot.common.assignment.InstancePartitionsUtils;
 import org.apache.pinot.common.config.ReplicaGroupStrategyConfig;
 import org.apache.pinot.common.config.RoutingConfig;
 import org.apache.pinot.common.config.TableConfig;
@@ -38,8 +41,6 @@ import org.apache.pinot.common.metadata.segment.ColumnPartitionMetadata;
 import org.apache.pinot.common.metadata.segment.OfflineSegmentZKMetadata;
 import org.apache.pinot.common.metadata.segment.SegmentPartitionMetadata;
 import org.apache.pinot.common.metadata.segment.SegmentZKMetadata;
-import org.apache.pinot.common.partition.ReplicaGroupPartitionAssignment;
-import org.apache.pinot.common.partition.ReplicaGroupPartitionAssignmentGenerator;
 import org.apache.pinot.common.utils.CommonConstants;
 import org.apache.pinot.pql.parsers.Pql2Compiler;
 import org.testng.Assert;
@@ -97,7 +98,7 @@ public class PartitionAwareOfflineRoutingTableBuilderTest {
       }
 
       // Update replica group mapping zk metadata
-      updateReplicaGroupPartitionAssignment(OFFLINE_TABLE_NAME, fakePropertyStore);
+      updateInstancePartitions(OFFLINE_TABLE_NAME, fakePropertyStore);
 
       // Create the fake external view
       ExternalView externalView = buildExternalView(OFFLINE_TABLE_NAME, replicaToServerMapping);
@@ -182,7 +183,7 @@ public class PartitionAwareOfflineRoutingTableBuilderTest {
     }
 
     // Update replica group mapping zk metadata
-    updateReplicaGroupPartitionAssignment(OFFLINE_TABLE_NAME, fakePropertyStore);
+    updateInstancePartitions(OFFLINE_TABLE_NAME, fakePropertyStore);
 
     // Create the fake external view
     ExternalView externalView = buildExternalView(OFFLINE_TABLE_NAME, replicaToServerMapping);
@@ -207,7 +208,7 @@ public class PartitionAwareOfflineRoutingTableBuilderTest {
     instanceConfigs.add(new InstanceConfig(newServerName));
 
     // Update replica group partition assignment
-    updateReplicaGroupPartitionAssignment(OFFLINE_TABLE_NAME, fakePropertyStore);
+    updateInstancePartitions(OFFLINE_TABLE_NAME, fakePropertyStore);
 
     // Update external view
     Map<Integer, List<String>> newReplicaToServerMapping = buildReplicaGroupMapping();
@@ -230,7 +231,8 @@ public class PartitionAwareOfflineRoutingTableBuilderTest {
   }
 
   @Test
-  public void testRoutingAfterOneServerDown() throws Exception {
+  public void testRoutingAfterOneServerDown()
+      throws Exception {
     NUM_REPLICA = 3;
     NUM_PARTITION = 1;
     NUM_SERVERS = 3;
@@ -255,8 +257,8 @@ public class PartitionAwareOfflineRoutingTableBuilderTest {
               metadata.toZNRecord());
     }
 
-    // Update replica group mapping zk metadata
-    updateReplicaGroupPartitionAssignment(OFFLINE_TABLE_NAME, fakePropertyStore);
+    // Update instance partitions
+    updateInstancePartitions(OFFLINE_TABLE_NAME, fakePropertyStore);
 
     // Create instance Configs
     List<InstanceConfig> instanceConfigs = new ArrayList<>();
@@ -289,22 +291,21 @@ public class PartitionAwareOfflineRoutingTableBuilderTest {
     Assert.assertEquals(servers.size(), 2);
   }
 
-  private void updateReplicaGroupPartitionAssignment(String tableNameWithType, FakePropertyStore propertyStore) {
-    // Create partition assignment mapping table.
-    ReplicaGroupPartitionAssignment replicaGroupPartitionAssignment =
-        new ReplicaGroupPartitionAssignment(tableNameWithType);
+  private void updateInstancePartitions(String tableNameWithType, FakePropertyStore propertyStore) {
+    InstancePartitions instancePartitions = new InstancePartitions(
+        InstancePartitionsUtils.getInstancePartitionsName(tableNameWithType, InstancePartitionsType.OFFLINE));
 
     int partitionId = 0;
-    for (int serverId = 0; serverId < NUM_SERVERS; serverId++) {
-      String serverName = "Server_localhost_" + serverId;
-      int replicaGroupId = serverId / (NUM_SERVERS / NUM_REPLICA);
-      replicaGroupPartitionAssignment.addInstanceToReplicaGroup(partitionId, replicaGroupId, serverName);
+    int numInstancesPerReplicaGroup = NUM_SERVERS / NUM_REPLICA;
+    for (int replicaGroupId = 0; replicaGroupId < NUM_REPLICA; replicaGroupId++) {
+      List<String> instances = new ArrayList<>(numInstancesPerReplicaGroup);
+      for (int instanceId = 0; instanceId < numInstancesPerReplicaGroup; instanceId++) {
+        instances.add("Server_localhost_" + (replicaGroupId * numInstancesPerReplicaGroup + instanceId));
+      }
+      instancePartitions.setInstances(partitionId, replicaGroupId, instances);
     }
 
-    // Write partition assignment to the property store.
-    ReplicaGroupPartitionAssignmentGenerator partitionAssignmentGenerator =
-        new ReplicaGroupPartitionAssignmentGenerator(propertyStore);
-    partitionAssignmentGenerator.writeReplicaGroupPartitionAssignment(replicaGroupPartitionAssignment);
+    InstancePartitionsUtils.persistInstancePartitions(propertyStore, instancePartitions);
   }
 
   private RoutingTableBuilder buildPartitionAwareOfflineRoutingTableBuilder(FakePropertyStore propertyStore,
@@ -332,7 +333,8 @@ public class PartitionAwareOfflineRoutingTableBuilderTest {
   }
 
   private ExternalView buildExternalViewWithDownServer(String tableName, Map<Integer, List<String>> replicaGroupServers,
-      String downServer) throws Exception {
+      String downServer)
+      throws Exception {
     // Create External View
     ExternalView externalView = new ExternalView(tableName);
     for (int i = 0; i < NUM_SEGMENTS; i++) {
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/assignment/InstancePartitionsUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/assignment/InstancePartitionsUtils.java
index 8a4861f..e9cfade 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/assignment/InstancePartitionsUtils.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/assignment/InstancePartitionsUtils.java
@@ -25,11 +25,13 @@ import org.I0Itec.zkclient.exception.ZkException;
 import org.apache.helix.AccessOption;
 import org.apache.helix.HelixManager;
 import org.apache.helix.ZNRecord;
+import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.store.HelixPropertyStore;
 import org.apache.helix.store.zk.ZkHelixPropertyStore;
+import org.apache.pinot.common.config.OfflineTagConfig;
+import org.apache.pinot.common.config.RealtimeTagConfig;
 import org.apache.pinot.common.config.TableConfig;
 import org.apache.pinot.common.config.TableNameBuilder;
-import org.apache.pinot.common.config.TagNameUtils;
 import org.apache.pinot.common.metadata.ZKMetadataProvider;
 import org.apache.pinot.common.utils.helix.HelixHelper;
 
@@ -89,18 +91,32 @@ public class InstancePartitionsUtils {
   }
 
   /**
-   * Computes the default instance partitions.
-   * <p>For backward-compatibility, sort all enabled instances with the server tag, rotate the list based on the table
-   * name name to prevent creating hotspot servers.
+   * Computes the default instance partitions. Sort all qualified instances and rotate the list based on the table name
+   * to prevent creating hotspot servers.
+   * <p>For backward-compatibility, choose only enabled instances with the server tag for OFFLINE table; choose both
+   * enabled and disabled instances with the server tag for REALTIME table.
    */
   public static InstancePartitions computeDefaultInstancePartitions(HelixManager helixManager, TableConfig tableConfig,
       InstancePartitionsType instancePartitionsType) {
-    String tableNameWithType = tableConfig.getTableName();
-    String serverTag =
-        TagNameUtils.getServerTagFromTableConfigAndInstancePartitionsType(tableConfig, instancePartitionsType);
-    List<String> instances = HelixHelper.getEnabledInstancesWithTag(helixManager, serverTag);
+    List<InstanceConfig> instanceConfigs = HelixHelper.getInstanceConfigs(helixManager);
+    List<String> instances;
+    if (instancePartitionsType == InstancePartitionsType.OFFLINE) {
+      // For backward-compatibility, choose enabled instances with the server tag for OFFLINE table
+      String serverTag = new OfflineTagConfig(tableConfig).getOfflineServerTag();
+      instances = HelixHelper.getEnabledInstancesWithTag(instanceConfigs, serverTag);
+    } else {
+      // For backward-compatibility, choose both enabled and disabled instances with the server tag for REALTIME table
+      RealtimeTagConfig realtimeTagConfig = new RealtimeTagConfig(tableConfig);
+      String serverTag =
+          instancePartitionsType == InstancePartitionsType.CONSUMING ? realtimeTagConfig.getConsumingServerTag()
+              : realtimeTagConfig.getCompletedServerTag();
+      instances = HelixHelper.getInstancesWithTag(instanceConfigs, serverTag);
+    }
+
+    // Sort the instances and rotate the list based on the table name
     instances.sort(null);
     int numInstances = instances.size();
+    String tableNameWithType = tableConfig.getTableName();
     Collections.rotate(instances, -(Math.abs(tableNameWithType.hashCode()) % numInstances));
     InstancePartitions instancePartitions =
         new InstancePartitions(getInstancePartitionsName(tableNameWithType, instancePartitionsType));
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/config/IndexingConfig.java b/pinot-common/src/main/java/org/apache/pinot/common/config/IndexingConfig.java
index 6172f52..6c74e15 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/config/IndexingConfig.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/config/IndexingConfig.java
@@ -56,9 +56,6 @@ public class IndexingConfig {
   @UseChildKeyHandler(SimpleMapChildKeyHandler.class)
   private Map<String, String> _streamConfigs = new HashMap<>();
 
-  @ConfigKey("streamConsumptionConfig")
-  private StreamConsumptionConfig _streamConsumptionConfig;
-
   @ConfigKey("segmentFormatVersion")
   private String _segmentFormatVersion;
 
@@ -151,14 +148,6 @@ public class IndexingConfig {
     _streamConfigs = streamConfigs;
   }
 
-  public StreamConsumptionConfig getStreamConsumptionConfig() {
-    return _streamConsumptionConfig;
-  }
-
-  public void setStreamConsumptionConfig(StreamConsumptionConfig streamConsumptionConfig) {
-    _streamConsumptionConfig = streamConsumptionConfig;
-  }
-
   public String getSegmentFormatVersion() {
     return _segmentFormatVersion;
   }
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/config/StreamConsumptionConfig.java b/pinot-common/src/main/java/org/apache/pinot/common/config/StreamConsumptionConfig.java
deleted file mode 100644
index e9f5e1b..0000000
--- a/pinot-common/src/main/java/org/apache/pinot/common/config/StreamConsumptionConfig.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.pinot.common.config;
-
-import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
-
-
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class StreamConsumptionConfig {
-  private String _streamPartitionAssignmentStrategy;
-
-  public String getStreamPartitionAssignmentStrategy() {
-    return _streamPartitionAssignmentStrategy;
-  }
-
-  public void setStreamPartitionAssignmentStrategy(String streamPartitionAssignmentStrategy) {
-    _streamPartitionAssignmentStrategy = streamPartitionAssignmentStrategy;
-  }
-}
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/config/TableConfig.java b/pinot-common/src/main/java/org/apache/pinot/common/config/TableConfig.java
index 0d8e136..3eca666 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/config/TableConfig.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/config/TableConfig.java
@@ -474,7 +474,6 @@ public class TableConfig {
     private static final String DEFAULT_SEGMENT_PUSH_TYPE = "APPEND";
     private static final String REFRESH_SEGMENT_PUSH_TYPE = "REFRESH";
     private static final String DEFAULT_SEGMENT_ASSIGNMENT_STRATEGY = "BalanceNumSegmentAssignmentStrategy";
-    private static final String DEFAULT_STREAM_PARTITION_ASSIGNMENT_STRATEGY = "UniformStreamPartitionAssignment";
     private static final String DEFAULT_NUM_REPLICAS = "1";
     private static final String DEFAULT_LOAD_MODE = "HEAP";
     private static final String MMAP_LOAD_MODE = "MMAP";
@@ -508,7 +507,6 @@ public class TableConfig {
     private List<String> _onHeapDictionaryColumns;
     private List<String> _bloomFilterColumns;
     private Map<String, String> _streamConfigs;
-    private String _streamPartitionAssignmentStrategy = DEFAULT_STREAM_PARTITION_ASSIGNMENT_STRATEGY;
     private SegmentPartitionConfig _segmentPartitionConfig;
 
     private TableCustomConfig _customConfig;
@@ -642,11 +640,6 @@ public class TableConfig {
       return this;
     }
 
-    public Builder setStreamPartitionAssignmentStrategy(String streamPartitionAssignmentStrategy) {
-      _streamPartitionAssignmentStrategy = streamPartitionAssignmentStrategy;
-      return this;
-    }
-
     public Builder setSegmentPartitionConfig(SegmentPartitionConfig segmentPartitionConfig) {
       _segmentPartitionConfig = segmentPartitionConfig;
       return this;
@@ -712,9 +705,6 @@ public class TableConfig {
       indexingConfig.setOnHeapDictionaryColumns(_onHeapDictionaryColumns);
       indexingConfig.setBloomFilterColumns(_bloomFilterColumns);
       indexingConfig.setStreamConfigs(_streamConfigs);
-      StreamConsumptionConfig streamConsumptionConfig = new StreamConsumptionConfig();
-      streamConsumptionConfig.setStreamPartitionAssignmentStrategy(_streamPartitionAssignmentStrategy);
-      indexingConfig.setStreamConsumptionConfig(streamConsumptionConfig);
       indexingConfig.setSegmentPartitionConfig(_segmentPartitionConfig);
 
       if (_customConfig == null) {
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/config/TagNameUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/config/TagNameUtils.java
index 689ecd0..1702f53 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/config/TagNameUtils.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/config/TagNameUtils.java
@@ -18,7 +18,6 @@
  */
 package org.apache.pinot.common.config;
 
-import org.apache.pinot.common.assignment.InstancePartitionsType;
 import org.apache.pinot.common.utils.ServerType;
 import org.apache.pinot.common.utils.TenantRole;
 
@@ -102,18 +101,4 @@ public class TagNameUtils {
     }
     return tag;
   }
-
-  public static String getServerTagFromTableConfigAndInstancePartitionsType(TableConfig tableConfig,
-      InstancePartitionsType instancePartitionsType) {
-    switch (instancePartitionsType) {
-      case OFFLINE:
-        return new OfflineTagConfig(tableConfig).getOfflineServerTag();
-      case CONSUMING:
-        return new RealtimeTagConfig(tableConfig).getConsumingServerTag();
-      case COMPLETED:
-        return new RealtimeTagConfig(tableConfig).getCompletedServerTag();
-      default:
-        throw new IllegalArgumentException();
-    }
-  }
 }
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/metrics/ControllerMeter.java b/pinot-common/src/main/java/org/apache/pinot/common/metrics/ControllerMeter.java
index f66279d..79a5261 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/metrics/ControllerMeter.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/metrics/ControllerMeter.java
@@ -52,6 +52,7 @@ public enum ControllerMeter implements AbstractMetrics.Meter {
   CONTROLLER_LEADERSHIP_CHANGE_WITHOUT_CALLBACK("leadershipChangeWithoutCallback", true),
   LLC_STATE_MACHINE_ABORTS("aborts", false),
   LLC_AUTO_CREATED_PARTITIONS("creates", false),
+  LLC_ZOOKEEPER_FETCH_FAILURES("failures", false),
   LLC_ZOOKEEPER_UPDATE_FAILURES("failures", false),
   LLC_KAFKA_DATA_LOSS("dataLoss", false),
   // Introducing a new stream agnostic metric to replace LLC_KAFKA_DATA_LOSS.
@@ -61,8 +62,7 @@ public enum ControllerMeter implements AbstractMetrics.Meter {
   CONTROLLER_PERIODIC_TASK_ERROR("periodicTaskError", false),
   NUMBER_TIMES_SCHEDULE_TASKS_CALLED("tasks", true),
   NUMBER_TASKS_SUBMITTED("tasks", false),
-  NUMBER_SEGMENT_UPLOAD_TIMEOUT_EXCEEDED("SegmentUploadTimeouts", true),
-  PARTITION_ASSIGNMENT_GENERATION_ERROR("partitionAssignmentError", false);
+  NUMBER_SEGMENT_UPLOAD_TIMEOUT_EXCEEDED("SegmentUploadTimeouts", true);
 
   private final String brokerMeterName;
   private final String unit;
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/partition/PartitionAssignment.java b/pinot-common/src/main/java/org/apache/pinot/common/partition/PartitionAssignment.java
deleted file mode 100644
index ae5a95c..0000000
--- a/pinot-common/src/main/java/org/apache/pinot/common/partition/PartitionAssignment.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.pinot.common.partition;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import org.apache.pinot.common.utils.EqualityUtils;
-
-
-/**
- * Class to represent a partition assignment
- */
-public class PartitionAssignment {
-
-  private String _tableName;
-  private Map<String, List<String>> _partitionToInstances;
-
-  public PartitionAssignment(String tableName) {
-    _tableName = tableName;
-    _partitionToInstances = new HashMap<>(1);
-  }
-
-  public PartitionAssignment(String tableName, Map<String, List<String>> partitionToInstances) {
-    _tableName = tableName;
-    _partitionToInstances = partitionToInstances;
-  }
-
-  public String getTableName() {
-    return _tableName;
-  }
-
-  public Map<String, List<String>> getPartitionToInstances() {
-    return _partitionToInstances;
-  }
-
-  public void addPartition(String partitionName, List<String> instances) {
-    _partitionToInstances.put(partitionName, instances);
-  }
-
-  public List<String> getInstancesListForPartition(String partition) {
-    return _partitionToInstances.get(partition);
-  }
-
-  public int getNumPartitions() {
-    return _partitionToInstances.size();
-  }
-
-  /**
-   * Get all instances.
-   *
-   * @return Set of all instances for this table
-   */
-  public List<String> getAllInstances() {
-    Set<String> serverList = new HashSet<>();
-    for (List<String> servers : getPartitionToInstances().values()) {
-      serverList.addAll(servers);
-    }
-    return new ArrayList<>(serverList);
-  }
-
-  @Override
-  public String toString() {
-    return "PartitionAssignment{" + "_tableName='" + _tableName + '\'' + ", _partitionToInstances="
-        + _partitionToInstances + '}';
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (EqualityUtils.isSameReference(this, o)) {
-      return true;
-    }
-
-    if (EqualityUtils.isNullOrNotSameClass(this, o)) {
-      return false;
-    }
-
-    PartitionAssignment that = (PartitionAssignment) o;
-
-    return EqualityUtils.isEqual(_tableName, that._tableName) && EqualityUtils
-        .isEqual(_partitionToInstances, that._partitionToInstances);
-  }
-
-  @Override
-  public int hashCode() {
-    int result = EqualityUtils.hashCodeOf(_tableName);
-    result = EqualityUtils.hashCodeOf(result, _partitionToInstances);
-    return result;
-  }
-}
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/partition/ReplicaGroupBasedStreamPartitionAssignmentStrategy.java b/pinot-common/src/main/java/org/apache/pinot/common/partition/ReplicaGroupBasedStreamPartitionAssignmentStrategy.java
deleted file mode 100644
index d3d5b6a..0000000
--- a/pinot-common/src/main/java/org/apache/pinot/common/partition/ReplicaGroupBasedStreamPartitionAssignmentStrategy.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.pinot.common.partition;
-
-import com.google.common.annotations.VisibleForTesting;
-import java.util.ArrayList;
-import java.util.List;
-import javax.annotation.Nonnull;
-import org.apache.helix.HelixManager;
-import org.apache.pinot.common.exception.InvalidConfigException;
-
-
-/**
- * Replica group based partition assignment strategy for realtime partitions
- */
-public class ReplicaGroupBasedStreamPartitionAssignmentStrategy implements StreamPartitionAssignmentStrategy {
-
-  /**
-   * Fetches the replica group partition assignment znode and assigns partitions across the replica groups.
-   * A vertical slice will be picked form the replica group sets for a partition, based on the formula:
-   * vertical slice = partition % numInstancesPerReplicaGroup
-   */
-  @Override
-  public PartitionAssignment getStreamPartitionAssignment(HelixManager helixManager, @Nonnull String tableNameWithType,
-      @Nonnull List<String> partitions, int numReplicas, List<String> instances)
-      throws InvalidConfigException {
-
-    ReplicaGroupPartitionAssignment replicaGroupPartitionAssignment =
-        getReplicaGroupPartitionAssignment(helixManager, tableNameWithType);
-    if (replicaGroupPartitionAssignment == null) {
-      throw new InvalidConfigException("ReplicaGroupPartitionAssignment is null for table:" + tableNameWithType);
-    }
-    int numReplicaGroups = replicaGroupPartitionAssignment.getNumReplicaGroups();
-    // TODO: we might move to a model of not having the same numInstancesPerReplicaGroup.
-    // We would have to handle numInstancesInReplicaGroup on a replica group by replica group basis, and uniformly assign withing each replica group
-    int numInstancesPerReplicaGroup = replicaGroupPartitionAssignment.getInstancesFromReplicaGroup(0, 0).size();
-
-    PartitionAssignment streamPartitionAssignment = new PartitionAssignment(tableNameWithType);
-
-    List<List<String>> verticalSlices = new ArrayList<>(numInstancesPerReplicaGroup);
-    for (int i = 0; i < numInstancesPerReplicaGroup; i++) {
-      verticalSlices.add(new ArrayList<>(numReplicaGroups));
-    }
-
-    for (int replicaGroupNumber = 0; replicaGroupNumber < numReplicaGroups; replicaGroupNumber++) {
-      List<String> instancesFromReplicaGroup =
-          replicaGroupPartitionAssignment.getInstancesFromReplicaGroup(0, replicaGroupNumber);
-      for (int serverIndex = 0; serverIndex < numInstancesPerReplicaGroup; serverIndex++) {
-        verticalSlices.get(serverIndex).add(instancesFromReplicaGroup.get(serverIndex));
-      }
-    }
-
-    for (String partition : partitions) {
-      int verticalSlice = Integer.parseInt(partition) % numInstancesPerReplicaGroup;
-      streamPartitionAssignment.addPartition(partition, verticalSlices.get(verticalSlice));
-    }
-    return streamPartitionAssignment;
-  }
-
-  @VisibleForTesting
-  protected ReplicaGroupPartitionAssignment getReplicaGroupPartitionAssignment(HelixManager helixManager,
-      String tableNameWithType) {
-    ReplicaGroupPartitionAssignmentGenerator replicaGroupPartitionAssignmentGenerator =
-        new ReplicaGroupPartitionAssignmentGenerator(helixManager.getHelixPropertyStore());
-    return replicaGroupPartitionAssignmentGenerator.getReplicaGroupPartitionAssignment(tableNameWithType);
-  }
-}
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/partition/ReplicaGroupPartitionAssignment.java b/pinot-common/src/main/java/org/apache/pinot/common/partition/ReplicaGroupPartitionAssignment.java
deleted file mode 100644
index b065793..0000000
--- a/pinot-common/src/main/java/org/apache/pinot/common/partition/ReplicaGroupPartitionAssignment.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.pinot.common.partition;
-
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.NoSuchElementException;
-import java.util.Set;
-
-
-/**
- * Class to represent a replica group partition assignment
- *
- */
-public class ReplicaGroupPartitionAssignment extends PartitionAssignment {
-  private static final String SEPARATOR = "_";
-
-  public ReplicaGroupPartitionAssignment(String tableName) {
-    super(tableName);
-  }
-
-  public ReplicaGroupPartitionAssignment(String tableName, Map<String, List<String>> partitionToInstances) {
-    super(tableName, partitionToInstances);
-  }
-
-  /**
-   * Compute total number of partitions in the mapping.
-   *
-   * @return the number of partitions
-   */
-  public int getNumPartitions() {
-    Set<String> partitions = new HashSet<>();
-    for (String key : getPartitionToInstances().keySet()) {
-      String partitionId = key.split(SEPARATOR)[0];
-      partitions.add(partitionId);
-    }
-    return partitions.size();
-  }
-
-  /**
-   * Compute total number of replica groups in the mapping.
-   *
-   * @return the number of replica groups
-   */
-  public int getNumReplicaGroups() {
-    Set<String> replicaGroups = new HashSet<>();
-    for (String key : getPartitionToInstances().keySet()) {
-      String replicaGroupId = key.split(SEPARATOR)[1];
-      replicaGroups.add(replicaGroupId);
-    }
-    return replicaGroups.size();
-  }
-
-  /**
-   * Add an instance to a replica group for a partition.
-   *
-   * @param partition Partition number
-   * @param replicaGroup Replica group number
-   * @param instanceName Name of an instance
-   */
-  public void addInstanceToReplicaGroup(int partition, int replicaGroup, String instanceName) {
-    String key = createMappingKey(partition, replicaGroup);
-    if (!getPartitionToInstances().containsKey(key)) {
-      addPartition(key, new ArrayList<>());
-    }
-    getInstancesListForPartition(key).add(instanceName);
-  }
-
-  /**
-   * Get instances of a replica group for a partition.
-   *
-   * @param partition Partition number
-   * @param replicaGroup Replica group number
-   * @return List of instances belongs to the given partition and replica group
-   */
-  public List<String> getInstancesFromReplicaGroup(int partition, int replicaGroup) {
-    String key = createMappingKey(partition, replicaGroup);
-    if (!getPartitionToInstances().containsKey(key)) {
-      throw new NoSuchElementException();
-    }
-    return getInstancesListForPartition(key);
-  }
-
-  /**
-   * Set instances of a replica group for a partition.
-   *
-   * @param partition Partition number
-   * @param replicaGroup Replica group number
-   * @param instances Instances that belongs to the given partition and replica group
-   */
-  public void setInstancesToReplicaGroup(int partition, int replicaGroup, List<String> instances) {
-    String key = createMappingKey(partition, replicaGroup);
-    addPartition(key, instances);
-  }
-
-  /**
-   * Helper method to create a key for the partition mapping table.
-   *
-   * @param partition Partition number
-   * @param replicaGroup Replica group number
-   * @return Key for the partition mapping table
-   */
-  private String createMappingKey(int partition, int replicaGroup) {
-    return partition + SEPARATOR + replicaGroup;
-  }
-}
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/partition/ReplicaGroupPartitionAssignmentGenerator.java b/pinot-common/src/main/java/org/apache/pinot/common/partition/ReplicaGroupPartitionAssignmentGenerator.java
deleted file mode 100644
index b6f53c6..0000000
--- a/pinot-common/src/main/java/org/apache/pinot/common/partition/ReplicaGroupPartitionAssignmentGenerator.java
+++ /dev/null
@@ -1,128 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.pinot.common.partition;
-
-import com.google.common.base.Preconditions;
-import java.util.Collections;
-import java.util.List;
-import org.apache.helix.AccessOption;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.store.zk.ZkHelixPropertyStore;
-import org.apache.pinot.common.config.ReplicaGroupStrategyConfig;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.metadata.ZKMetadataProvider;
-
-
-/**
- * Class to help to read, write, and generate replica group partition assignment
- *
- */
-public class ReplicaGroupPartitionAssignmentGenerator {
-
-  private ZkHelixPropertyStore<ZNRecord> _propertyStore;
-
-  public ReplicaGroupPartitionAssignmentGenerator(ZkHelixPropertyStore<ZNRecord> propertyStore) {
-    _propertyStore = propertyStore;
-  }
-
-  /**
-   * Read the replica group partition assignment from the property store
-   * @param tableNameWithType a table name
-   * @return Replica group partition assignment
-   */
-  public ReplicaGroupPartitionAssignment getReplicaGroupPartitionAssignment(String tableNameWithType) {
-    String path = ZKMetadataProvider.constructPropertyStorePathForInstancePartitions(tableNameWithType);
-    ZNRecord replicaGroupPartitionAssignment = _propertyStore.get(path, null, AccessOption.PERSISTENT);
-    ReplicaGroupPartitionAssignment partitionAssignment = null;
-    if (replicaGroupPartitionAssignment != null) {
-      partitionAssignment =
-          new ReplicaGroupPartitionAssignment(tableNameWithType, replicaGroupPartitionAssignment.getListFields());
-    }
-    return partitionAssignment;
-  }
-
-  /**
-   * Write the replica group partition assignment to property store
-   *
-   * @param partitionAssignment a replica group partition assignment
-   */
-  public void writeReplicaGroupPartitionAssignment(ReplicaGroupPartitionAssignment partitionAssignment) {
-    String tableNameWithType = partitionAssignment.getTableName();
-    ZNRecord znRecord = new ZNRecord(tableNameWithType);
-    znRecord.setListFields(partitionAssignment.getPartitionToInstances());
-    String path = ZKMetadataProvider.constructPropertyStorePathForInstancePartitions(tableNameWithType);
-    _propertyStore.set(path, znRecord, AccessOption.PERSISTENT);
-  }
-
-  /**
-   * Build the partition mapping table that maps a tuple of (partition number, replica group number) to a list of
-   * servers. Two important configurations are explained below.
-   *
-   * 'numInstancesPerPartition': this number decides the number of servers within a replica group.
-   * 'partitionColumn': this configuration decides whether to use the table or partition level replica groups.
-   *
-   * @param tableNameWithType: Name of table
-   * @param tableConfig: Configuration for table
-   * @return Replica group partition assignment from the given configuration
-   */
-  public ReplicaGroupPartitionAssignment buildReplicaGroupPartitionAssignment(String tableNameWithType,
-      TableConfig tableConfig, int numReplicas, List<String> servers) {
-
-    // Fetch information required to build the mapping table from the table configuration.
-    ReplicaGroupStrategyConfig replicaGroupStrategyConfig =
-        tableConfig.getValidationConfig().getReplicaGroupStrategyConfig();
-    Preconditions.checkNotNull(replicaGroupStrategyConfig, "Replica group strategy config should not be null");
-
-    String partitionColumn = replicaGroupStrategyConfig.getPartitionColumn();
-    int numInstancesPerPartition = replicaGroupStrategyConfig.getNumInstancesPerPartition();
-
-    // If we do not have the partition column configuration, we assume to use the table level replica groups,
-    // which is equivalent to have the same partition number for all segments (i.e. 1 partition).
-    int numColumnPartitions = 1;
-    if (partitionColumn != null) {
-      numColumnPartitions =
-          tableConfig.getIndexingConfig().getSegmentPartitionConfig().getNumPartitions(partitionColumn);
-    }
-
-    int numServers = servers.size();
-
-    // Enforcing disjoint server sets for each replica group.
-    if (numInstancesPerPartition * numReplicas > numServers) {
-      throw new UnsupportedOperationException("Replica group aware segment assignment assumes that servers in "
-          + "each replica group are disjoint. Check the configurations to see if the following inequality holds. "
-          + "'numInstancePerPartition' * 'numReplicas' <= 'totalServerNumbers'");
-    }
-
-    // Creating a mapping table
-    ReplicaGroupPartitionAssignment partitionAssignment = new ReplicaGroupPartitionAssignment(tableNameWithType);
-
-    Collections.sort(servers);
-    for (int partitionId = 0; partitionId < numColumnPartitions; partitionId++) {
-      // If the configuration contains partition column information, we use the segment level replica groups.
-      if (numColumnPartitions != 1) {
-        Collections.shuffle(servers);
-      }
-      for (int i = 0; i < numInstancesPerPartition * numReplicas; i++) {
-        int groupId = i / numInstancesPerPartition;
-        partitionAssignment.addInstanceToReplicaGroup(partitionId, groupId, servers.get(i));
-      }
-    }
-    return partitionAssignment;
-  }
-}
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/partition/StreamPartitionAssignmentGenerator.java b/pinot-common/src/main/java/org/apache/pinot/common/partition/StreamPartitionAssignmentGenerator.java
deleted file mode 100644
index 5598f36..0000000
--- a/pinot-common/src/main/java/org/apache/pinot/common/partition/StreamPartitionAssignmentGenerator.java
+++ /dev/null
@@ -1,139 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.pinot.common.partition;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Lists;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import org.apache.helix.HelixManager;
-import org.apache.helix.model.IdealState;
-import org.apache.pinot.common.config.RealtimeTagConfig;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.exception.InvalidConfigException;
-import org.apache.pinot.common.utils.LLCSegmentName;
-import org.apache.pinot.common.utils.helix.HelixHelper;
-
-
-/**
- * Class to generate stream partitions assignment based on num partitions in ideal state, num tagged instances and num replicas
- */
-public class StreamPartitionAssignmentGenerator {
-
-  private HelixManager _helixManager;
-
-  public StreamPartitionAssignmentGenerator(HelixManager helixManager) {
-    _helixManager = helixManager;
-  }
-
-  /**
-   * Gets stream partition assignment of a table by reading the segment assignment in ideal state
-   */
-  public PartitionAssignment getStreamPartitionAssignmentFromIdealState(TableConfig tableConfig,
-      IdealState idealState) {
-    String tableNameWithType = tableConfig.getTableName();
-
-    // get latest segment in each partition
-    Map<String, LLCSegmentName> partitionIdToLatestSegment = getPartitionToLatestSegments(idealState);
-
-    // extract partition assignment from the latest segments
-    PartitionAssignment partitionAssignment = new PartitionAssignment(tableNameWithType);
-    Map<String, Map<String, String>> mapFields = idealState.getRecord().getMapFields();
-    for (Map.Entry<String, LLCSegmentName> entry : partitionIdToLatestSegment.entrySet()) {
-      String segmentName = entry.getValue().getSegmentName();
-      Map<String, String> instanceStateMap = mapFields.get(segmentName);
-      partitionAssignment.addPartition(entry.getKey(), Lists.newArrayList(instanceStateMap.keySet()));
-    }
-    return partitionAssignment;
-  }
-
-  /**
-   * Generates a map of partition id to latest llc segment
-   */
-  @VisibleForTesting
-  public Map<String, LLCSegmentName> getPartitionToLatestSegments(IdealState idealState) {
-    Map<String, LLCSegmentName> partitionIdToLatestSegment = new HashMap<>();
-    // read all segments
-    Map<String, Map<String, String>> mapFields = idealState.getRecord().getMapFields();
-
-    // get latest segment in each partition
-    for (Map.Entry<String, Map<String, String>> entry : mapFields.entrySet()) {
-      String segmentName = entry.getKey();
-      if (LLCSegmentName.isLowLevelConsumerSegmentName(segmentName)) {
-        LLCSegmentName llcSegmentName = new LLCSegmentName(segmentName);
-        String partitionId = String.valueOf(llcSegmentName.getPartitionId());
-        LLCSegmentName latestSegment = partitionIdToLatestSegment.get(partitionId);
-        if (latestSegment == null || llcSegmentName.getSequenceNumber() > latestSegment.getSequenceNumber()) {
-          partitionIdToLatestSegment.put(partitionId, llcSegmentName);
-        }
-      }
-    }
-    return partitionIdToLatestSegment;
-  }
-
-  public int getNumPartitionsFromIdealState(IdealState idealState) {
-    Set<Integer> partitions = new HashSet<>();
-    Map<String, Map<String, String>> mapFields = idealState.getRecord().getMapFields();
-    for (Map.Entry<String, Map<String, String>> entry : mapFields.entrySet()) {
-      String segmentName = entry.getKey();
-      if (LLCSegmentName.isLowLevelConsumerSegmentName(segmentName)) {
-        LLCSegmentName llcSegmentName = new LLCSegmentName(segmentName);
-        partitions.add(llcSegmentName.getPartitionId());
-      }
-    }
-    return partitions.size();
-  }
-
-  /**
-   * Generates stream partition assignment for given table, using tagged hosts and num partitions
-   */
-  public PartitionAssignment generateStreamPartitionAssignment(TableConfig tableConfig, int numPartitions)
-      throws InvalidConfigException {
-
-    List<String> partitions = new ArrayList<>(numPartitions);
-    for (int i = 0; i < numPartitions; i++) {
-      partitions.add(String.valueOf(i));
-    }
-
-    String tableNameWithType = tableConfig.getTableName();
-    int numReplicas = tableConfig.getValidationConfig().getReplicasPerPartitionNumber();
-    List<String> consumingTaggedInstances = getConsumingTaggedInstances(tableConfig);
-
-    StreamPartitionAssignmentStrategy streamPartitionAssignmentStrategy =
-        StreamPartitionAssignmentStrategyFactory.getStreamPartitionAssignmentStrategy(tableConfig);
-    return streamPartitionAssignmentStrategy
-        .getStreamPartitionAssignment(_helixManager, tableNameWithType, partitions, numReplicas,
-            consumingTaggedInstances);
-  }
-
-  @VisibleForTesting
-  protected List<String> getConsumingTaggedInstances(TableConfig tableConfig) {
-    RealtimeTagConfig realtimeTagConfig = new RealtimeTagConfig(tableConfig);
-    String consumingServerTag = realtimeTagConfig.getConsumingServerTag();
-    List<String> consumingTaggedInstances = HelixHelper.getInstancesWithTag(_helixManager, consumingServerTag);
-    if (consumingTaggedInstances.isEmpty()) {
-      throw new IllegalStateException("No instances found with tag " + consumingServerTag);
-    }
-    return consumingTaggedInstances;
-  }
-}
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/partition/StreamPartitionAssignmentStrategy.java b/pinot-common/src/main/java/org/apache/pinot/common/partition/StreamPartitionAssignmentStrategy.java
deleted file mode 100644
index 8c79261..0000000
--- a/pinot-common/src/main/java/org/apache/pinot/common/partition/StreamPartitionAssignmentStrategy.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.pinot.common.partition;
-
-import java.util.List;
-import javax.annotation.Nonnull;
-import org.apache.helix.HelixManager;
-import org.apache.pinot.common.exception.InvalidConfigException;
-
-
-/**
- * Creates a partition assignment for the partitions of a realtime table
- */
-// TODO: Unify the interfaces for {@link org.apache.pinot.controller.helix.core.sharding.SegmentAssignmentStrategy} and {@link StreamPartitionAssignmentStrategy}
-public interface StreamPartitionAssignmentStrategy {
-
-  /**
-   * Given the list of partitions and replicas, come up with a {@link PartitionAssignment}
-   */
-  // TODO: pass current partition assignment to add smarts which can minimize shuffle
-  PartitionAssignment getStreamPartitionAssignment(HelixManager helixManager, @Nonnull String tableNameWithType,
-      @Nonnull List<String> partitions, int numReplicas, List<String> instances)
-      throws InvalidConfigException;
-}
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/partition/StreamPartitionAssignmentStrategyFactory.java b/pinot-common/src/main/java/org/apache/pinot/common/partition/StreamPartitionAssignmentStrategyFactory.java
deleted file mode 100644
index 063e4c9..0000000
--- a/pinot-common/src/main/java/org/apache/pinot/common/partition/StreamPartitionAssignmentStrategyFactory.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.pinot.common.partition;
-
-import org.apache.pinot.common.config.SegmentsValidationAndRetentionConfig;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.utils.CommonConstants.Helix.DataSource.SegmentAssignmentStrategyType;
-
-
-/**
- * Factory class for constructing the right {@link StreamPartitionAssignmentStrategy} from the table config
- */
-public class StreamPartitionAssignmentStrategyFactory {
-
-  /**
-   * Given a table config, get the {@link StreamPartitionAssignmentStrategy}
-   */
-  static StreamPartitionAssignmentStrategy getStreamPartitionAssignmentStrategy(TableConfig tableConfig) {
-    SegmentsValidationAndRetentionConfig validationConfig = tableConfig.getValidationConfig();
-    if (validationConfig != null) {
-      String segmentAssignmentStrategy = validationConfig.getSegmentAssignmentStrategy();
-      if (segmentAssignmentStrategy != null && SegmentAssignmentStrategyType.ReplicaGroupSegmentAssignmentStrategy
-          .toString().equalsIgnoreCase(segmentAssignmentStrategy)) {
-        return new ReplicaGroupBasedStreamPartitionAssignmentStrategy();
-      }
-    }
-    return new UniformStreamPartitionAssignmentStrategy();
-  }
-}
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/partition/UniformStreamPartitionAssignmentStrategy.java b/pinot-common/src/main/java/org/apache/pinot/common/partition/UniformStreamPartitionAssignmentStrategy.java
deleted file mode 100644
index 5ab10e0..0000000
--- a/pinot-common/src/main/java/org/apache/pinot/common/partition/UniformStreamPartitionAssignmentStrategy.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.pinot.common.partition;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import javax.annotation.Nonnull;
-import org.apache.helix.HelixManager;
-import org.apache.pinot.common.exception.InvalidConfigException;
-import org.apache.pinot.common.utils.EqualityUtils;
-
-
-/**
- * Uniform partition assignment strategy implementation which uniformly sprays partitions across available hosts
- */
-public class UniformStreamPartitionAssignmentStrategy implements StreamPartitionAssignmentStrategy {
-
-  /**
-   * Uniformly sprays the partitions and replicas across given list of instances
-   * Picks starting point based on table hash value. This ensures that we will always pick the same starting point,
-   * and return consistent assignment across calls
-   */
-  @Override
-  public PartitionAssignment getStreamPartitionAssignment(HelixManager helixManager, @Nonnull String tableNameWithType,
-      @Nonnull List<String> partitions, int numReplicas, List<String> allTaggedInstances)
-      throws InvalidConfigException {
-
-    if (allTaggedInstances.size() < numReplicas) {
-      throw new InvalidConfigException(
-          "Not enough consuming instances tagged for UniformStreamPartitionAssignment. Must be at least equal to numReplicas:"
-              + numReplicas);
-    }
-
-    PartitionAssignment partitionAssignment = new PartitionAssignment(tableNameWithType);
-
-    Collections.sort(allTaggedInstances);
-    int numInstances = allTaggedInstances.size();
-    int serverId = Math.abs(EqualityUtils.hashCodeOf(tableNameWithType)) % numInstances;
-    for (String partition : partitions) {
-      List<String> instances = new ArrayList<>(numReplicas);
-      for (int r = 0; r < numReplicas; r++) {
-        instances.add(allTaggedInstances.get(serverId));
-        serverId = (serverId + 1) % numInstances;
-      }
-      partitionAssignment.addPartition(partition, instances);
-    }
-    return partitionAssignment;
-  }
-}
diff --git a/pinot-common/src/test/java/org/apache/pinot/common/config/TableConfigTest.java b/pinot-common/src/test/java/org/apache/pinot/common/config/TableConfigTest.java
index 4819f91..15cf537 100644
--- a/pinot-common/src/test/java/org/apache/pinot/common/config/TableConfigTest.java
+++ b/pinot-common/src/test/java/org/apache/pinot/common/config/TableConfigTest.java
@@ -274,29 +274,6 @@ public class TableConfigTest {
       checkTableConfigWithCompletionConfig(tableConfig, tableConfigToCompare);
     }
     {
-      // With default StreamConsumptionConfig
-      TableConfig tableConfig = tableConfigBuilder.build();
-      assertEquals(tableConfig.getIndexingConfig().getStreamConsumptionConfig().getStreamPartitionAssignmentStrategy(),
-          "UniformStreamPartitionAssignment");
-
-      // with streamConsumptionConfig set
-      tableConfig =
-          tableConfigBuilder.setStreamPartitionAssignmentStrategy("BalancedStreamPartitionAssignment").build();
-      assertEquals(tableConfig.getIndexingConfig().getStreamConsumptionConfig().getStreamPartitionAssignmentStrategy(),
-          "BalancedStreamPartitionAssignment");
-
-      // Serialize then de-serialize
-      TableConfig tableConfigToCompare = TableConfig.fromJsonConfig(tableConfig.toJsonConfig());
-      assertEquals(
-          tableConfigToCompare.getIndexingConfig().getStreamConsumptionConfig().getStreamPartitionAssignmentStrategy(),
-          "BalancedStreamPartitionAssignment");
-
-      tableConfigToCompare = TableConfig.fromZnRecord(tableConfig.toZNRecord());
-      assertEquals(
-          tableConfigToCompare.getIndexingConfig().getStreamConsumptionConfig().getStreamPartitionAssignmentStrategy(),
-          "BalancedStreamPartitionAssignment");
-    }
-    {
       // With star tree config
       StarTreeIndexSpec starTreeIndexSpec = new StarTreeIndexSpec();
       Set<String> dims = new HashSet<>();
diff --git a/pinot-common/src/test/java/org/apache/pinot/common/partition/IdealStateBuilderUtil.java b/pinot-common/src/test/java/org/apache/pinot/common/partition/IdealStateBuilderUtil.java
deleted file mode 100644
index 1f0ac3e..0000000
--- a/pinot-common/src/test/java/org/apache/pinot/common/partition/IdealStateBuilderUtil.java
+++ /dev/null
@@ -1,179 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.pinot.common.partition;
-
-import com.google.common.collect.Lists;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import org.apache.helix.model.IdealState;
-import org.apache.pinot.common.config.TableNameBuilder;
-import org.apache.pinot.common.utils.LLCSegmentName;
-
-import static org.apache.pinot.common.utils.CommonConstants.Helix.StateModel.RealtimeSegmentOnlineOfflineStateModel.CONSUMING;
-
-
-/**
- * Builder for ideal state for testing
- */
-public class IdealStateBuilderUtil {
-
-  private IdealState _idealState;
-  private String _tableName;
-  private String _rawTableName;
-
-  public IdealStateBuilderUtil(String tableNameWithType) {
-    _tableName = tableNameWithType;
-    _rawTableName = TableNameBuilder.extractRawTableName(tableNameWithType);
-    _idealState = new IdealState(tableNameWithType);
-  }
-
-  public IdealStateBuilderUtil(IdealState idealState, String tableNameWithType) {
-    _tableName = tableNameWithType;
-    _rawTableName = TableNameBuilder.extractRawTableName(tableNameWithType);
-    _idealState = idealState;
-  }
-
-  public IdealState build() {
-    return _idealState;
-  }
-
-  public IdealStateBuilderUtil disableIdealState() {
-    _idealState.getRecord().setSimpleField(IdealState.IdealStateProperty.HELIX_ENABLED.name(), "false");
-    return this;
-  }
-
-  public IdealStateBuilderUtil setNumReplicas(int numReplicas) {
-    _idealState.setReplicas(String.valueOf(numReplicas));
-    return this;
-  }
-
-  public String getSegment(int partition, int seqNum) {
-    for (String segmentName : _idealState.getRecord().getMapFields().keySet()) {
-      if (LLCSegmentName.isLowLevelConsumerSegmentName(segmentName)) {
-        LLCSegmentName llcSegmentName = new LLCSegmentName(segmentName);
-        if (llcSegmentName.getPartitionId() == partition && llcSegmentName.getSequenceNumber() == seqNum) {
-          return segmentName;
-        }
-      }
-    }
-    return null;
-  }
-
-  public IdealStateBuilderUtil addSegment(String segmentName, Map<String, String> instanceStateMap) {
-    _idealState.setInstanceStateMap(segmentName, instanceStateMap);
-    return this;
-  }
-
-  public IdealStateBuilderUtil addConsumingSegments(int numPartitions, int seqNum, int numReplicas,
-      List<String> instances) {
-    int serverId = 0;
-    for (int p = 0; p < numPartitions; p++) {
-      LLCSegmentName llcSegmentName = new LLCSegmentName(_rawTableName, p, seqNum, System.currentTimeMillis());
-      Map<String, String> instanceStateMap = new HashMap<>(numReplicas);
-      for (int r = 0; r < numReplicas; r++) {
-        instanceStateMap.put(instances.get(serverId++), CONSUMING);
-        if (serverId == instances.size()) {
-          serverId = 0;
-        }
-      }
-      _idealState.setInstanceStateMap(llcSegmentName.getSegmentName(), instanceStateMap);
-    }
-    return this;
-  }
-
-  public IdealStateBuilderUtil setSegmentState(int partition, int seqNum, String state) {
-    for (String segmentName : _idealState.getRecord().getMapFields().keySet()) {
-      if (LLCSegmentName.isLowLevelConsumerSegmentName(segmentName)) {
-        LLCSegmentName llcSegmentName = new LLCSegmentName(segmentName);
-        if (llcSegmentName.getPartitionId() == partition && llcSegmentName.getSequenceNumber() == seqNum) {
-          Map<String, String> instanceStateMap = _idealState.getInstanceStateMap(segmentName);
-          for (Map.Entry<String, String> entry : instanceStateMap.entrySet()) {
-            instanceStateMap.put(entry.getKey(), state);
-          }
-          break;
-        }
-      }
-    }
-    return this;
-  }
-
-  public IdealStateBuilderUtil setSegmentState(String segmentName, String state) {
-    Map<String, String> instanceStateMap = _idealState.getInstanceStateMap(segmentName);
-    for (Map.Entry<String, String> entry : instanceStateMap.entrySet()) {
-      instanceStateMap.put(entry.getKey(), state);
-    }
-    return this;
-  }
-
-  public IdealStateBuilderUtil addConsumingSegment(int partition, int seqNum, int numReplicas, List<String> instances) {
-    LLCSegmentName llcSegmentName = new LLCSegmentName(_rawTableName, partition, seqNum, System.currentTimeMillis());
-    Map<String, String> instanceStateMap = new HashMap<>(numReplicas);
-    for (int i = 0; i < numReplicas; i++) {
-      instanceStateMap.put(instances.get(i), CONSUMING);
-    }
-    _idealState.setInstanceStateMap(llcSegmentName.getSegmentName(), instanceStateMap);
-    return this;
-  }
-
-  public IdealStateBuilderUtil moveToServers(int partition, int seqNum, List<String> instances) {
-    for (String segmentName : _idealState.getRecord().getMapFields().keySet()) {
-      if (LLCSegmentName.isLowLevelConsumerSegmentName(segmentName)) {
-        LLCSegmentName llcSegmentName = new LLCSegmentName(segmentName);
-        if (llcSegmentName.getPartitionId() == partition && llcSegmentName.getSequenceNumber() == seqNum) {
-          Map<String, String> instanceStateMap = _idealState.getInstanceStateMap(segmentName);
-          Map<String, String> newInstanceStateMap = new HashMap<>(instanceStateMap.size());
-          int serverId = 0;
-          for (Map.Entry<String, String> entry : instanceStateMap.entrySet()) {
-            newInstanceStateMap.put(instances.get(serverId++), entry.getValue());
-          }
-          _idealState.setInstanceStateMap(llcSegmentName.getSegmentName(), newInstanceStateMap);
-          break;
-        }
-      }
-    }
-    return this;
-  }
-
-  public List<String> getInstances(int partition, int seqNum) {
-    List<String> instances = new ArrayList<>();
-    for (String segmentName : _idealState.getRecord().getMapFields().keySet()) {
-      if (LLCSegmentName.isLowLevelConsumerSegmentName(segmentName)) {
-        LLCSegmentName llcSegmentName = new LLCSegmentName(segmentName);
-        if (llcSegmentName.getPartitionId() == partition && llcSegmentName.getSequenceNumber() == seqNum) {
-          Map<String, String> instanceStateMap = _idealState.getInstanceStateMap(segmentName);
-          instances = Lists.newArrayList(instanceStateMap.keySet());
-          break;
-        }
-      }
-    }
-    return instances;
-  }
-
-  public IdealStateBuilderUtil removeSegment(String segmentName) {
-    _idealState.getRecord().getMapFields().remove(segmentName);
-    return this;
-  }
-
-  public IdealStateBuilderUtil clear() {
-    _idealState.getRecord().getMapFields().clear();
-    return this;
-  }
-}
diff --git a/pinot-common/src/test/java/org/apache/pinot/common/partition/ReplicaGroupBasedStreamPartitionAssignmentTest.java b/pinot-common/src/test/java/org/apache/pinot/common/partition/ReplicaGroupBasedStreamPartitionAssignmentTest.java
deleted file mode 100644
index 5ebd3b1..0000000
--- a/pinot-common/src/test/java/org/apache/pinot/common/partition/ReplicaGroupBasedStreamPartitionAssignmentTest.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.pinot.common.partition;
-
-import com.google.common.collect.Lists;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import org.apache.helix.HelixManager;
-import org.apache.pinot.common.exception.InvalidConfigException;
-import org.testng.Assert;
-import org.testng.annotations.Test;
-
-
-/**
- * Tests for {@link ReplicaGroupBasedStreamPartitionAssignmentStrategy}
- */
-public class ReplicaGroupBasedStreamPartitionAssignmentTest {
-
-  @Test
-  public void testReplicaGroupBasedStreamPartitionAssignmentStrategy()
-      throws InvalidConfigException {
-    MockReplicaGroupBasedStreamPartitionAssignmentStrategy mockStreamPartitionAssignmentStrategy =
-        new MockReplicaGroupBasedStreamPartitionAssignmentStrategy();
-
-    // Realtime table with 2 replicas, 6 partitions, 4 numInstancesPerReplicaGroup.
-    // 8 servers distributed across 2 replica groups into 2 sets of 4.
-    String tableNameWithType = "tableName_REALTIME";
-    List<String> partitions = Lists.newArrayList("0", "1", "2", "3", "4", "5");
-    int numReplicas = 2;
-    List<String> allTaggedInstances = Lists
-        .newArrayList("server_1", "server_2", "server_3", "server_4", "server_5", "server_6", "server_7", "server_8");
-
-    ReplicaGroupPartitionAssignment replicaGroupPartitionAssignment =
-        new ReplicaGroupPartitionAssignment(tableNameWithType);
-    replicaGroupPartitionAssignment
-        .setInstancesToReplicaGroup(0, 0, Lists.newArrayList("server_1", "server_2", "server_3", "server_4"));
-    replicaGroupPartitionAssignment
-        .setInstancesToReplicaGroup(0, 1, Lists.newArrayList("server_5", "server_6", "server_7", "server_8"));
-
-    // null replica group partition assignment
-    mockStreamPartitionAssignmentStrategy.setReplicaGroupPartitionAssignment(null);
-    boolean exception = false;
-    try {
-      mockStreamPartitionAssignmentStrategy
-          .getStreamPartitionAssignment(null, tableNameWithType, partitions, numReplicas, allTaggedInstances);
-    } catch (InvalidConfigException e) {
-      exception = true;
-    }
-    Assert.assertTrue(exception);
-
-    // mismatch between numReplicas and numReplicaGroups - follow the replica group assignment
-    mockStreamPartitionAssignmentStrategy.setReplicaGroupPartitionAssignment(replicaGroupPartitionAssignment);
-    PartitionAssignment streamPartitionAssignment = mockStreamPartitionAssignmentStrategy
-        .getStreamPartitionAssignment(null, tableNameWithType, partitions, 5, allTaggedInstances);
-    Assert.assertEquals(streamPartitionAssignment.getInstancesListForPartition("0"),
-        Lists.newArrayList("server_1", "server_5"));
-    Assert.assertEquals(streamPartitionAssignment.getInstancesListForPartition("1"),
-        Lists.newArrayList("server_2", "server_6"));
-    Assert.assertEquals(streamPartitionAssignment.getInstancesListForPartition("2"),
-        Lists.newArrayList("server_3", "server_7"));
-    Assert.assertEquals(streamPartitionAssignment.getInstancesListForPartition("3"),
-        Lists.newArrayList("server_4", "server_8"));
-    Assert.assertEquals(streamPartitionAssignment.getInstancesListForPartition("4"),
-        Lists.newArrayList("server_1", "server_5"));
-    Assert.assertEquals(streamPartitionAssignment.getInstancesListForPartition("5"),
-        Lists.newArrayList("server_2", "server_6"));
-
-    // happy path - correctly generated partition assignment
-    mockStreamPartitionAssignmentStrategy.setReplicaGroupPartitionAssignment(replicaGroupPartitionAssignment);
-    streamPartitionAssignment = mockStreamPartitionAssignmentStrategy
-        .getStreamPartitionAssignment(null, tableNameWithType, partitions, numReplicas, allTaggedInstances);
-    Assert.assertEquals(streamPartitionAssignment.getInstancesListForPartition("0"),
-        Lists.newArrayList("server_1", "server_5"));
-    Assert.assertEquals(streamPartitionAssignment.getInstancesListForPartition("1"),
-        Lists.newArrayList("server_2", "server_6"));
-    Assert.assertEquals(streamPartitionAssignment.getInstancesListForPartition("2"),
-        Lists.newArrayList("server_3", "server_7"));
-    Assert.assertEquals(streamPartitionAssignment.getInstancesListForPartition("3"),
-        Lists.newArrayList("server_4", "server_8"));
-    Assert.assertEquals(streamPartitionAssignment.getInstancesListForPartition("4"),
-        Lists.newArrayList("server_1", "server_5"));
-    Assert.assertEquals(streamPartitionAssignment.getInstancesListForPartition("5"),
-        Lists.newArrayList("server_2", "server_6"));
-
-    // 0 partitions
-    streamPartitionAssignment = mockStreamPartitionAssignmentStrategy
-        .getStreamPartitionAssignment(null, tableNameWithType, Collections.emptyList(), numReplicas,
-            allTaggedInstances);
-    Assert.assertEquals(streamPartitionAssignment.getNumPartitions(), 0);
-
-    // only 1 instance per replica group
-    replicaGroupPartitionAssignment.setInstancesToReplicaGroup(0, 0, Lists.newArrayList("server_1"));
-    replicaGroupPartitionAssignment.setInstancesToReplicaGroup(0, 1, Lists.newArrayList("server_2"));
-    mockStreamPartitionAssignmentStrategy.setReplicaGroupPartitionAssignment(replicaGroupPartitionAssignment);
-    streamPartitionAssignment = mockStreamPartitionAssignmentStrategy
-        .getStreamPartitionAssignment(null, tableNameWithType, partitions, numReplicas, allTaggedInstances);
-    ArrayList<String> verticalSlice = Lists.newArrayList("server_1", "server_2");
-    Assert.assertEquals(streamPartitionAssignment.getInstancesListForPartition("0"), verticalSlice);
-    Assert.assertEquals(streamPartitionAssignment.getInstancesListForPartition("1"), verticalSlice);
-    Assert.assertEquals(streamPartitionAssignment.getInstancesListForPartition("2"), verticalSlice);
-    Assert.assertEquals(streamPartitionAssignment.getInstancesListForPartition("3"), verticalSlice);
-    Assert.assertEquals(streamPartitionAssignment.getInstancesListForPartition("4"), verticalSlice);
-    Assert.assertEquals(streamPartitionAssignment.getInstancesListForPartition("5"), verticalSlice);
-  }
-
-  private class MockReplicaGroupBasedStreamPartitionAssignmentStrategy extends ReplicaGroupBasedStreamPartitionAssignmentStrategy {
-    private ReplicaGroupPartitionAssignment _replicaGroupPartitionAssignment;
-
-    @Override
-    protected ReplicaGroupPartitionAssignment getReplicaGroupPartitionAssignment(HelixManager helixManager,
-        String tableNameWithType) {
-      return _replicaGroupPartitionAssignment;
-    }
-
-    void setReplicaGroupPartitionAssignment(ReplicaGroupPartitionAssignment replicaGroupPartitionAssignment) {
-      _replicaGroupPartitionAssignment = replicaGroupPartitionAssignment;
-    }
-  }
-}
diff --git a/pinot-common/src/test/java/org/apache/pinot/common/partition/StreamPartitionAssignmentGeneratorTest.java b/pinot-common/src/test/java/org/apache/pinot/common/partition/StreamPartitionAssignmentGeneratorTest.java
deleted file mode 100644
index 30272e3..0000000
--- a/pinot-common/src/test/java/org/apache/pinot/common/partition/StreamPartitionAssignmentGeneratorTest.java
+++ /dev/null
@@ -1,364 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.pinot.common.partition;
-
-import com.google.common.collect.Lists;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import org.apache.commons.lang3.RandomStringUtils;
-import org.apache.helix.HelixManager;
-import org.apache.helix.model.IdealState;
-import org.apache.pinot.common.config.SegmentsValidationAndRetentionConfig;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.config.TenantConfig;
-import org.apache.pinot.common.utils.CommonConstants;
-import org.apache.pinot.common.utils.LLCSegmentName;
-import org.testng.Assert;
-import org.testng.annotations.BeforeMethod;
-import org.testng.annotations.Test;
-
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-
-public class StreamPartitionAssignmentGeneratorTest {
-
-  private String aServerTenant = "aTenant";
-  private HelixManager _mockHelixManager;
-  private String[] consumingServerNames;
-
-  private List<String> getConsumingInstanceList(final int nServers) {
-    Assert.assertTrue(nServers <= consumingServerNames.length);
-    String[] instanceArray = Arrays.copyOf(consumingServerNames, nServers);
-    return Lists.newArrayList(instanceArray);
-  }
-
-  @BeforeMethod
-  public void setUp()
-      throws Exception {
-    _mockHelixManager = mock(HelixManager.class);
-
-    final int maxInstances = 20;
-    consumingServerNames = new String[maxInstances];
-    for (int i = 0; i < maxInstances; i++) {
-      consumingServerNames[i] = "ConsumingServer_" + i;
-    }
-  }
-
-  @Test
-  public void testStreamPartitionAssignmentStrategyFactory() {
-    TableConfig tableConfig = mock(TableConfig.class);
-
-    // null validation config
-    StreamPartitionAssignmentStrategy streamPartitionAssignmentStrategy =
-        StreamPartitionAssignmentStrategyFactory.getStreamPartitionAssignmentStrategy(tableConfig);
-    Assert.assertTrue(streamPartitionAssignmentStrategy instanceof UniformStreamPartitionAssignmentStrategy);
-
-    // null segment assignment strategy
-    SegmentsValidationAndRetentionConfig mockValidationConfig = mock(SegmentsValidationAndRetentionConfig.class);
-    when(tableConfig.getValidationConfig()).thenReturn(mockValidationConfig);
-    streamPartitionAssignmentStrategy =
-        StreamPartitionAssignmentStrategyFactory.getStreamPartitionAssignmentStrategy(tableConfig);
-    Assert.assertTrue(streamPartitionAssignmentStrategy instanceof UniformStreamPartitionAssignmentStrategy);
-
-    // incorrect segment assignment strategy
-    when(mockValidationConfig.getSegmentAssignmentStrategy()).thenReturn("someThing Wrong");
-    when(tableConfig.getValidationConfig()).thenReturn(mockValidationConfig);
-    streamPartitionAssignmentStrategy =
-        StreamPartitionAssignmentStrategyFactory.getStreamPartitionAssignmentStrategy(tableConfig);
-    Assert.assertTrue(streamPartitionAssignmentStrategy instanceof UniformStreamPartitionAssignmentStrategy);
-
-    // Unsupported type
-    when(mockValidationConfig.getSegmentAssignmentStrategy()).thenReturn(
-        CommonConstants.Helix.DataSource.SegmentAssignmentStrategyType.BalanceNumSegmentAssignmentStrategy.toString());
-    when(tableConfig.getValidationConfig()).thenReturn(mockValidationConfig);
-    streamPartitionAssignmentStrategy =
-        StreamPartitionAssignmentStrategyFactory.getStreamPartitionAssignmentStrategy(tableConfig);
-    Assert.assertTrue(streamPartitionAssignmentStrategy instanceof UniformStreamPartitionAssignmentStrategy);
-
-    // ReplicaGroup
-    when(mockValidationConfig.getSegmentAssignmentStrategy()).thenReturn(
-        CommonConstants.Helix.DataSource.SegmentAssignmentStrategyType.ReplicaGroupSegmentAssignmentStrategy
-            .toString());
-    when(tableConfig.getValidationConfig()).thenReturn(mockValidationConfig);
-    streamPartitionAssignmentStrategy =
-        StreamPartitionAssignmentStrategyFactory.getStreamPartitionAssignmentStrategy(tableConfig);
-    Assert.assertTrue(streamPartitionAssignmentStrategy instanceof ReplicaGroupBasedStreamPartitionAssignmentStrategy);
-  }
-
-  /**
-   * Given an ideal state, constructs the partition assignment for the table
-   */
-  @Test
-  public void testGetPartitionAssignmentFromIdealState() {
-
-    String tableName = "aTableName_REALTIME";
-    TableConfig tableConfig = mock(TableConfig.class);
-    when(tableConfig.getTableName()).thenReturn(tableName);
-
-    IdealStateBuilderUtil idealStateBuilder = new IdealStateBuilderUtil(tableName);
-    IdealState idealState;
-
-    int numPartitions = 0;
-    int numReplicas = 2;
-
-    // empty ideal state
-    idealState = idealStateBuilder.build();
-    verifyPartitionAssignmentFromIdealState(tableConfig, idealState, numPartitions);
-
-    // 3 partitions 2 replicas, all on 0th seq number (6 segments)
-    numPartitions = 3;
-    List<String> instances = getConsumingInstanceList(4);
-    idealState = idealStateBuilder.addConsumingSegments(numPartitions, 0, numReplicas, instances).build();
-    verifyPartitionAssignmentFromIdealState(tableConfig, idealState, numPartitions);
-
-    // 3 partitions 2 replicas, 2 on 0th seq number 1 on 1st seq number (8 segments)
-    instances = idealStateBuilder.getInstances(0, 0);
-    idealState =
-        idealStateBuilder.setSegmentState(0, 0, "ONLINE").addConsumingSegment(0, 1, numReplicas, instances).build();
-    verifyPartitionAssignmentFromIdealState(tableConfig, idealState, numPartitions);
-
-    // 3 partitions 2 replicas, all on 1st seg num (12 segments)
-    instances = idealStateBuilder.getInstances(1, 0);
-    idealStateBuilder.setSegmentState(1, 0, "ONLINE").addConsumingSegment(1, 1, numReplicas, instances);
-    instances = idealStateBuilder.getInstances(2, 0);
-    idealState =
-        idealStateBuilder.setSegmentState(2, 0, "ONLINE").addConsumingSegment(2, 1, numReplicas, instances).build();
-    verifyPartitionAssignmentFromIdealState(tableConfig, idealState, numPartitions);
-
-    // 3 partitions 2 replicas, seq 0 has moved to COMPLETED servers, seq 1 on CONSUMING instances
-    instances = Lists.newArrayList("s1_completed", "s2_completed");
-    idealState = idealStateBuilder.moveToServers(0, 0, instances).build();
-    verifyPartitionAssignmentFromIdealState(tableConfig, idealState, numPartitions);
-
-    // status of latest segments OFFLINE/ERROR
-    idealState = idealStateBuilder.setSegmentState(1, 1, "OFFLINE").setSegmentState(2, 1, "OFFLINE").build();
-    verifyPartitionAssignmentFromIdealState(tableConfig, idealState, numPartitions);
-
-    // all non llc segments
-    String aNonLLCSegment = "randomName";
-    Map<String, String> instanceStateMap = new HashMap<>(1);
-    instanceStateMap.put("s1", "CONSUMING");
-    instanceStateMap.put("s2", "CONSUMING");
-    idealState = idealStateBuilder.clear().addSegment(aNonLLCSegment, instanceStateMap).build();
-    numPartitions = 0;
-    verifyPartitionAssignmentFromIdealState(tableConfig, idealState, numPartitions);
-
-    // some non llc segments, some llc segments
-    numPartitions = 1;
-    instances = getConsumingInstanceList(6);
-    idealState = idealStateBuilder.addConsumingSegments(numPartitions, 0, numReplicas, instances).build();
-    verifyPartitionAssignmentFromIdealState(tableConfig, idealState, numPartitions);
-  }
-
-  private void verifyPartitionAssignmentFromIdealState(TableConfig tableConfig, IdealState idealState,
-      int numPartitions) {
-    TestStreamPartitionAssignmentGenerator partitionAssignmentGenerator =
-        new TestStreamPartitionAssignmentGenerator(_mockHelixManager);
-    PartitionAssignment partitionAssignmentFromIdealState =
-        partitionAssignmentGenerator.getStreamPartitionAssignmentFromIdealState(tableConfig, idealState);
-    Assert.assertEquals(tableConfig.getTableName(), partitionAssignmentFromIdealState.getTableName());
-    Assert.assertEquals(partitionAssignmentFromIdealState.getNumPartitions(), numPartitions);
-    // check that latest segments are honoring partition assignment
-    Map<String, LLCSegmentName> partitionIdToLatestLLCSegment =
-        partitionAssignmentGenerator.getPartitionToLatestSegments(idealState);
-    for (Map.Entry<String, LLCSegmentName> entry : partitionIdToLatestLLCSegment.entrySet()) {
-      Set<String> idealStateInstances = idealState.getInstanceStateMap(entry.getValue().getSegmentName()).keySet();
-      List<String> partitionAssignmentInstances =
-          partitionAssignmentFromIdealState.getInstancesListForPartition(entry.getKey());
-      Assert.assertEquals(idealStateInstances.size(), partitionAssignmentInstances.size());
-      Assert.assertTrue(idealStateInstances.containsAll(partitionAssignmentInstances));
-    }
-  }
-
-  @Test
-  public void testGeneratePartitionAssignment() {
-    RandomStringUtils.randomAlphabetic(10);
-    for (int i = 0; i < 20; i++) {
-      String tableName = RandomStringUtils.randomAlphabetic(10) + "_REALTIME";
-      testGeneratePartitionAssignmentForTable(tableName);
-    }
-  }
-
-  private void testGeneratePartitionAssignmentForTable(String tableName) {
-    TableConfig tableConfig = mock(TableConfig.class);
-    when(tableConfig.getTableName()).thenReturn(tableName);
-    TenantConfig mockTenantConfig = mock((TenantConfig.class));
-    when(mockTenantConfig.getServer()).thenReturn(aServerTenant);
-    when(tableConfig.getTenantConfig()).thenReturn(mockTenantConfig);
-    SegmentsValidationAndRetentionConfig mockValidationConfig = mock(SegmentsValidationAndRetentionConfig.class);
-    when(mockValidationConfig.getReplicasPerPartitionNumber()).thenReturn(2);
-    when(tableConfig.getValidationConfig()).thenReturn(mockValidationConfig);
-    int numPartitions = 0;
-    List<String> consumingInstanceList = getConsumingInstanceList(0);
-    PartitionAssignment previousPartitionAssignment = new PartitionAssignment(tableName);
-    boolean exceptionExpected;
-    boolean unchanged = false;
-
-    // 0 consuming instances - error not enough instances
-    exceptionExpected = true;
-    previousPartitionAssignment = verifyGeneratePartitionAssignment(tableConfig, numPartitions, consumingInstanceList,
-        previousPartitionAssignment, exceptionExpected, unchanged);
-
-    // 1 consuming instance - error not enough instances
-    consumingInstanceList = getConsumingInstanceList(1);
-    previousPartitionAssignment = verifyGeneratePartitionAssignment(tableConfig, numPartitions, consumingInstanceList,
-        previousPartitionAssignment, exceptionExpected, unchanged);
-
-    // 0 partitions - 3 consuming instances - empty partition assignment
-    consumingInstanceList = getConsumingInstanceList(3);
-    exceptionExpected = false;
-    unchanged = true;
-    previousPartitionAssignment = verifyGeneratePartitionAssignment(tableConfig, numPartitions, consumingInstanceList,
-        previousPartitionAssignment, exceptionExpected, unchanged);
-
-    // 3 partitions - 3 consuming instances
-    numPartitions = 3;
-    unchanged = false;
-    previousPartitionAssignment = verifyGeneratePartitionAssignment(tableConfig, numPartitions, consumingInstanceList,
-        previousPartitionAssignment, exceptionExpected, unchanged);
-
-    // same - shouldn't change
-    unchanged = true;
-    previousPartitionAssignment = verifyGeneratePartitionAssignment(tableConfig, numPartitions, consumingInstanceList,
-        previousPartitionAssignment, exceptionExpected, unchanged);
-
-    // 3 partitions - 12 consuming instances
-    consumingInstanceList = getConsumingInstanceList(12);
-    unchanged = false;
-    previousPartitionAssignment = verifyGeneratePartitionAssignment(tableConfig, numPartitions, consumingInstanceList,
-        previousPartitionAssignment, exceptionExpected, unchanged);
-
-    // same - shouldn't change
-    unchanged = true;
-    previousPartitionAssignment = verifyGeneratePartitionAssignment(tableConfig, numPartitions, consumingInstanceList,
-        previousPartitionAssignment, exceptionExpected, unchanged);
-
-    // 3 partitions - 6 consuming instances
-    consumingInstanceList = getConsumingInstanceList(6);
-    unchanged = false;
-    previousPartitionAssignment = verifyGeneratePartitionAssignment(tableConfig, numPartitions, consumingInstanceList,
-        previousPartitionAssignment, exceptionExpected, unchanged);
-
-    // same - shouldn't change
-    unchanged = true;
-    previousPartitionAssignment = verifyGeneratePartitionAssignment(tableConfig, numPartitions, consumingInstanceList,
-        previousPartitionAssignment, exceptionExpected, unchanged);
-
-    String server0 = consumingInstanceList.get(0);
-    consumingInstanceList.set(0, server0 + "_replaced");
-    unchanged = false;
-    previousPartitionAssignment = verifyGeneratePartitionAssignment(tableConfig, numPartitions, consumingInstanceList,
-        previousPartitionAssignment, exceptionExpected, unchanged);
-
-    // increase in partitions - 4
-    numPartitions = 4;
-    unchanged = false;
-    previousPartitionAssignment = verifyGeneratePartitionAssignment(tableConfig, numPartitions, consumingInstanceList,
-        previousPartitionAssignment, exceptionExpected, unchanged);
-
-    // same - shouldn't change
-    unchanged = true;
-    previousPartitionAssignment = verifyGeneratePartitionAssignment(tableConfig, numPartitions, consumingInstanceList,
-        previousPartitionAssignment, exceptionExpected, unchanged);
-  }
-
-  private PartitionAssignment verifyGeneratePartitionAssignment(TableConfig tableConfig, int numPartitions,
-      List<String> consumingInstanceList, PartitionAssignment previousPartitionAssignment, boolean exceptionExpected,
-      boolean unchanged) {
-    String tableName = tableConfig.getTableName();
-    TestStreamPartitionAssignmentGenerator partitionAssignmentGenerator =
-        new TestStreamPartitionAssignmentGenerator(_mockHelixManager);
-    partitionAssignmentGenerator.setConsumingInstances(consumingInstanceList);
-    PartitionAssignment partitionAssignment;
-    try {
-      partitionAssignment = partitionAssignmentGenerator.generateStreamPartitionAssignment(tableConfig, numPartitions);
-      Assert.assertFalse(exceptionExpected, "Unexpected exception for table " + tableName);
-      verify(tableName, partitionAssignment, numPartitions, consumingInstanceList, unchanged,
-          previousPartitionAssignment);
-    } catch (Exception e) {
-      Assert.assertTrue(exceptionExpected, "Expected exception for table " + tableName);
-      partitionAssignment = previousPartitionAssignment;
-    }
-
-    return partitionAssignment;
-  }
-
-  private void verify(String tableName, PartitionAssignment partitionAssignment, int numPartitions,
-      List<String> consumingInstanceList, boolean unchanged, PartitionAssignment previousPartitionAssignment) {
-    Assert.assertEquals(partitionAssignment.getTableName(), tableName);
-
-    // check num partitions equal
-    Assert.assertEquals(partitionAssignment.getNumPartitions(), numPartitions,
-        "NumPartitions do not match for table " + tableName);
-
-    List<String> instancesUsed = new ArrayList<>();
-    for (int p = 0; p < partitionAssignment.getNumPartitions(); p++) {
-      for (String instance : partitionAssignment.getInstancesListForPartition(String.valueOf(p))) {
-        if (!instancesUsed.contains(instance)) {
-          instancesUsed.add(instance);
-        }
-      }
-    }
-
-    // check all instances belong to the super set
-    Assert.assertTrue(consumingInstanceList.containsAll(instancesUsed), "Instances test failed for table " + tableName);
-
-    // verify strategy is uniform
-    int serverId = 0;
-    for (int p = 0; p < partitionAssignment.getNumPartitions(); p++) {
-      for (String instance : partitionAssignment.getInstancesListForPartition(String.valueOf(p))) {
-        Assert.assertEquals(instancesUsed.get(serverId++), instance,
-            "Uniform strategy test failed for table " + tableName);
-        if (serverId == instancesUsed.size()) {
-          serverId = 0;
-        }
-      }
-    }
-
-    // if nothing changed, should be same as before
-    if (unchanged) {
-      Assert.assertEquals(partitionAssignment, previousPartitionAssignment,
-          "Partition assignment should have been unchanged for table " + tableName);
-    }
-  }
-
-  private class TestStreamPartitionAssignmentGenerator extends StreamPartitionAssignmentGenerator {
-
-    private List<String> _consumingInstances;
-
-    TestStreamPartitionAssignmentGenerator(HelixManager helixManager) {
-      super(helixManager);
-      _consumingInstances = new ArrayList<>();
-    }
-
-    @Override
-    protected List<String> getConsumingTaggedInstances(TableConfig tableConfig) {
-      return _consumingInstances;
-    }
-
-    void setConsumingInstances(List<String> consumingInstances) {
-      _consumingInstances = consumingInstances;
-    }
-  }
-}
diff --git a/pinot-common/src/test/java/org/apache/pinot/common/partition/UniformStreamPartitionAssignmentTest.java b/pinot-common/src/test/java/org/apache/pinot/common/partition/UniformStreamPartitionAssignmentTest.java
deleted file mode 100644
index 64347c8..0000000
--- a/pinot-common/src/test/java/org/apache/pinot/common/partition/UniformStreamPartitionAssignmentTest.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.pinot.common.partition;
-
-import com.google.common.collect.Lists;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import org.apache.pinot.common.exception.InvalidConfigException;
-import org.testng.Assert;
-import org.testng.annotations.Test;
-
-
-/**
- * Tests for {@link UniformStreamPartitionAssignmentStrategy}
- */
-public class UniformStreamPartitionAssignmentTest {
-
-  /**
-   * Tests that we validate necessary variables and honor the uniform sticky assignment
-   * @throws InvalidConfigException
-   */
-  @Test
-  public void testUniformStreamPartitionAssignmentStrategy()
-      throws InvalidConfigException {
-    UniformStreamPartitionAssignmentStrategy uniformStreamPartitionAssignmentStrategy =
-        new UniformStreamPartitionAssignmentStrategy();
-
-    String tableNameWithType = "tableName_REALTIME";
-    List<String> partitions = Lists.newArrayList("0", "1", "2", "3", "4", "5");
-    int numReplicas = 3;
-    List<String> allTaggedInstances = Lists
-        .newArrayList("server_1", "server_2", "server_3", "server_4", "server_5", "server_6", "server_7", "server_8");
-
-    // num replicas more than tagged instances
-    boolean exception = false;
-    try {
-      uniformStreamPartitionAssignmentStrategy
-          .getStreamPartitionAssignment(null, tableNameWithType, partitions, 10, allTaggedInstances);
-    } catch (InvalidConfigException e) {
-      exception = true;
-    }
-    Assert.assertTrue(exception);
-
-    // 0 partitions
-    PartitionAssignment uniformPartitionAssignment = uniformStreamPartitionAssignmentStrategy
-        .getStreamPartitionAssignment(null, tableNameWithType, Collections.emptyList(), numReplicas,
-            allTaggedInstances);
-    Assert.assertEquals(uniformPartitionAssignment.getNumPartitions(), 0);
-
-    // verify sticky uniform assignment
-    uniformPartitionAssignment = uniformStreamPartitionAssignmentStrategy
-        .getStreamPartitionAssignment(null, tableNameWithType, partitions, numReplicas, allTaggedInstances);
-
-    List<String> instancesUsed = new ArrayList<>();
-    for (int p = 0; p < uniformPartitionAssignment.getNumPartitions(); p++) {
-      for (String instance : uniformPartitionAssignment.getInstancesListForPartition(String.valueOf(p))) {
-        if (!instancesUsed.contains(instance)) {
-          instancesUsed.add(instance);
-        }
-      }
-    }
-    Assert.assertTrue(instancesUsed.containsAll(allTaggedInstances));
-
-    int serverIndex = 0;
-    for (int p = 0; p < uniformPartitionAssignment.getNumPartitions(); p++) {
-      List<String> instancesListForPartition =
-          uniformPartitionAssignment.getInstancesListForPartition(String.valueOf(p));
-      for (String instance : instancesListForPartition) {
-        Assert.assertEquals(instance, instancesUsed.get(serverIndex++));
-        if (serverIndex == instancesUsed.size()) {
-          serverIndex = 0;
-        }
-      }
-    }
-  }
-}
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/ControllerStarter.java b/pinot-controller/src/main/java/org/apache/pinot/controller/ControllerStarter.java
index 8a1f512..e92d2f3 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/ControllerStarter.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/ControllerStarter.java
@@ -293,7 +293,7 @@ public class ControllerStarter {
     // Helix resource manager must be started in order to create PinotLLCRealtimeSegmentManager
     LOGGER.info("Starting realtime segment manager");
     _pinotLLCRealtimeSegmentManager =
-        new PinotLLCRealtimeSegmentManager(_helixResourceManager, _config, _controllerMetrics, _leadControllerManager);
+        new PinotLLCRealtimeSegmentManager(_helixResourceManager, _config, _controllerMetrics);
     // TODO: Need to put this inside HelixResourceManager when HelixControllerLeadershipManager is removed.
     _helixResourceManager.registerPinotLLCRealtimeSegmentManager(_pinotLLCRealtimeSegmentManager);
     _segmentCompletionManager =
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java
index 6ea61c1..251a8a8 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java
@@ -1072,7 +1072,7 @@ public class PinotHelixResourceManager {
          * We also need to support the case when a high-level consumer already exists for a table and we are adding
          * the low-level consumers.
          */
-        ensureRealtimeClusterIsSetUp(tableConfig, tableNameWithType, indexingConfig);
+        ensureRealtimeClusterIsSetUp(tableConfig);
 
         // Assign instances
         assignInstances(tableConfig, true);
@@ -1161,27 +1161,31 @@ public class PinotHelixResourceManager {
     }
   }
 
-  private void ensureRealtimeClusterIsSetUp(TableConfig config, String realtimeTableName,
-      IndexingConfig indexingConfig) {
-    StreamConfig streamConfig = new StreamConfig(realtimeTableName, indexingConfig.getStreamConfigs());
-    IdealState idealState = _helixAdmin.getResourceIdealState(_helixClusterName, realtimeTableName);
+  private void ensureRealtimeClusterIsSetUp(TableConfig realtimeTableConfig) {
+    String realtimeTableName = realtimeTableConfig.getTableName();
+    IdealState idealState = getTableIdealState(realtimeTableName);
+    // Skip processing disabled IdealState
+    if (idealState != null && !idealState.isEnabled()) {
+      return;
+    }
+
+    StreamConfig streamConfig = new StreamConfig(realtimeTableConfig);
 
     if (streamConfig.hasHighLevelConsumerType()) {
-      if (streamConfig.hasLowLevelConsumerType()) {
-        // We may be adding on low-level, or creating both.
-        if (idealState == null) {
-          // Need to create both. Create high-level consumer first.
-          createHelixEntriesForHighLevelConsumer(config, realtimeTableName, idealState);
-          idealState = _helixAdmin.getResourceIdealState(_helixClusterName, realtimeTableName);
-          LOGGER.info("Configured new HLC for table {}", realtimeTableName);
-        }
-        // Fall through to create low-level consumers
+      if (idealState == null) {
+        LOGGER.info("Initializing IdealState for HLC table: {}", realtimeTableName);
+        idealState = PinotTableIdealStateBuilder
+            .buildInitialHighLevelRealtimeIdealStateFor(realtimeTableName, realtimeTableConfig, _helixZkManager,
+                _propertyStore, _enableBatchMessageMode);
+        _helixAdmin.addResource(_helixClusterName, realtimeTableName, idealState);
       } else {
-        // Only high-level consumer specified in the config.
-        createHelixEntriesForHighLevelConsumer(config, realtimeTableName, idealState);
-        // Clean up any LLC table if they are present
-        _pinotLLCRealtimeSegmentManager.cleanupLLC(realtimeTableName);
+        // Remove LLC segments if it is not configured
+        if (!streamConfig.hasLowLevelConsumerType()) {
+          _pinotLLCRealtimeSegmentManager.removeLLCSegments(idealState);
+        }
       }
+      // For HLC table, property store entry must exist to trigger watchers to create segments
+      ensurePropertyStoreEntryExistsForHighLevelConsumer(realtimeTableName);
     }
 
     // Either we have only low-level consumer, or both.
@@ -1190,8 +1194,8 @@ public class PinotHelixResourceManager {
       // (unless there are low-level segments already present)
       if (ZKMetadataProvider.getLLCRealtimeSegments(_propertyStore, realtimeTableName).isEmpty()) {
         PinotTableIdealStateBuilder
-            .buildLowLevelRealtimeIdealStateFor(_pinotLLCRealtimeSegmentManager, realtimeTableName, config, idealState,
-                _enableBatchMessageMode);
+            .buildLowLevelRealtimeIdealStateFor(_pinotLLCRealtimeSegmentManager, realtimeTableName, realtimeTableConfig,
+                idealState, _enableBatchMessageMode);
         LOGGER.info("Successfully added Helix entries for low-level consumers for {} ", realtimeTableName);
       } else {
         LOGGER.info("LLC is already set up for table {}, not configuring again", realtimeTableName);
@@ -1199,23 +1203,11 @@ public class PinotHelixResourceManager {
     }
   }
 
-  private void createHelixEntriesForHighLevelConsumer(TableConfig config, String realtimeTableName,
-      IdealState idealState) {
-    if (idealState == null) {
-      idealState = PinotTableIdealStateBuilder
-          .buildInitialHighLevelRealtimeIdealStateFor(realtimeTableName, config, _helixZkManager, _propertyStore,
-              _enableBatchMessageMode);
-      LOGGER.info("Adding helix resource with empty HLC IdealState for {}", realtimeTableName);
-      _helixAdmin.addResource(_helixClusterName, realtimeTableName, idealState);
-    } else {
-      // TODO jfim: We get in this block if we're trying to add a HLC or it already exists. If it doesn't already exist, we need to set instance configs properly (which is done in buildInitialHighLevelRealtimeIdealState, surprisingly enough). For now, do nothing.
-      LOGGER.info("Not reconfiguring HLC for table {}", realtimeTableName);
-    }
-    LOGGER.info("Successfully created empty ideal state for  high level consumer for {} ", realtimeTableName);
-    // Finally, create the propertystore entry that will trigger watchers to create segments
-    String tablePropertyStorePath = ZKMetadataProvider.constructPropertyStorePathForResource(realtimeTableName);
-    if (!_propertyStore.exists(tablePropertyStorePath, AccessOption.PERSISTENT)) {
-      _propertyStore.create(tablePropertyStorePath, new ZNRecord(realtimeTableName), AccessOption.PERSISTENT);
+  private void ensurePropertyStoreEntryExistsForHighLevelConsumer(String realtimeTableName) {
+    String propertyStorePath = ZKMetadataProvider.constructPropertyStorePathForResource(realtimeTableName);
+    if (!_propertyStore.exists(propertyStorePath, AccessOption.PERSISTENT)) {
+      LOGGER.info("Creating property store entry for LLC table: {}", realtimeTableName);
+      _propertyStore.create(propertyStorePath, new ZNRecord(realtimeTableName), AccessOption.PERSISTENT);
     }
   }
 
@@ -1290,7 +1282,7 @@ public class PinotHelixResourceManager {
         IndexingConfig indexingConfig = tableConfig.getIndexingConfig();
         verifyIndexingConfig(tableNameWithType, indexingConfig);
         ZKMetadataProvider.setRealtimeTableConfig(_propertyStore, tableNameWithType, tableConfig.toZNRecord());
-        ensureRealtimeClusterIsSetUp(tableConfig, tableNameWithType, indexingConfig);
+        ensureRealtimeClusterIsSetUp(tableConfig);
 
         // Assign instances
         assignInstances(tableConfig, false);
@@ -1334,12 +1326,6 @@ public class PinotHelixResourceManager {
     }
     tableConfig.setIndexingConfig(newConfigs);
     setExistingTableConfig(tableConfig);
-
-    if (type == TableType.REALTIME) {
-      // Check if HLC table is allowed
-      verifyIndexingConfig(tableNameWithType, newConfigs);
-      ensureRealtimeClusterIsSetUp(tableConfig, tableName, newConfigs);
-    }
   }
 
   private void handleBrokerResource(@Nonnull final String tableName, @Nonnull final List<String> brokersForTenant) {
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotTableIdealStateBuilder.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotTableIdealStateBuilder.java
index 0e2a452..5fb0399 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotTableIdealStateBuilder.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotTableIdealStateBuilder.java
@@ -27,7 +27,6 @@ import org.apache.helix.model.builder.CustomModeISBuilder;
 import org.apache.helix.store.zk.ZkHelixPropertyStore;
 import org.apache.pinot.common.config.RealtimeTagConfig;
 import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.exception.InvalidConfigException;
 import org.apache.pinot.common.metadata.ZKMetadataProvider;
 import org.apache.pinot.common.metadata.instance.InstanceZKMetadata;
 import org.apache.pinot.common.utils.StringUtil;
@@ -113,11 +112,7 @@ public class PinotTableIdealStateBuilder {
     if (idealState == null) {
       idealState = buildEmptyRealtimeIdealStateFor(realtimeTableName, nReplicas, enableBatchMessageMode);
     }
-    try {
-      pinotLLCRealtimeSegmentManager.setupNewTable(realtimeTableConfig, idealState);
-    } catch (InvalidConfigException e) {
-      throw new IllegalStateException("Caught exception when creating table " + realtimeTableName, e);
-    }
+    pinotLLCRealtimeSegmentManager.setUpNewTable(realtimeTableConfig, idealState);
   }
 
   public static int getPartitionCount(StreamConfig streamConfig) {
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/assignment/segment/SegmentAssignmentUtils.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/assignment/segment/SegmentAssignmentUtils.java
index 3cfd567..e950f15 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/assignment/segment/SegmentAssignmentUtils.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/assignment/segment/SegmentAssignmentUtils.java
@@ -20,6 +20,7 @@ package org.apache.pinot.controller.helix.core.assignment.segment;
 
 import com.google.common.base.Preconditions;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.List;
@@ -216,7 +217,7 @@ public class SegmentAssignmentUtils {
   /**
    * Returns the map from instance name to Helix partition state, which can be put into the segment assignment.
    */
-  public static Map<String, String> getInstanceStateMap(List<String> instances, String state) {
+  public static Map<String, String> getInstanceStateMap(Collection<String> instances, String state) {
     Map<String, String> instanceStateMap = new TreeMap<>();
     for (String instanceName : instances) {
       instanceStateMap.put(instanceName, state);
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java
index 526f46c..fb21ee3 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java
@@ -19,69 +19,64 @@
 package org.apache.pinot.controller.helix.core.realtime;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import com.google.common.util.concurrent.Uninterruptibles;
 import java.net.URI;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.TreeMap;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
-import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
-import org.apache.commons.collections.CollectionUtils;
 import org.apache.helix.AccessOption;
 import org.apache.helix.HelixAdmin;
 import org.apache.helix.HelixManager;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.store.zk.ZkHelixPropertyStore;
+import org.apache.pinot.common.assignment.InstancePartitions;
+import org.apache.pinot.common.assignment.InstancePartitionsType;
+import org.apache.pinot.common.assignment.InstancePartitionsUtils;
 import org.apache.pinot.common.config.ColumnPartitionConfig;
 import org.apache.pinot.common.config.SegmentPartitionConfig;
 import org.apache.pinot.common.config.TableConfig;
 import org.apache.pinot.common.config.TableNameBuilder;
-import org.apache.pinot.common.exception.InvalidConfigException;
 import org.apache.pinot.common.metadata.ZKMetadataProvider;
 import org.apache.pinot.common.metadata.segment.ColumnPartitionMetadata;
 import org.apache.pinot.common.metadata.segment.LLCRealtimeSegmentZKMetadata;
 import org.apache.pinot.common.metadata.segment.SegmentPartitionMetadata;
 import org.apache.pinot.common.metrics.ControllerMeter;
 import org.apache.pinot.common.metrics.ControllerMetrics;
-import org.apache.pinot.common.partition.PartitionAssignment;
-import org.apache.pinot.common.partition.StreamPartitionAssignmentGenerator;
 import org.apache.pinot.common.protocols.SegmentCompletionProtocol;
-import org.apache.pinot.common.utils.CommonConstants;
+import org.apache.pinot.common.utils.CommonConstants.Helix.StateModel.RealtimeSegmentOnlineOfflineStateModel;
+import org.apache.pinot.common.utils.CommonConstants.Segment.Realtime.Status;
 import org.apache.pinot.common.utils.LLCSegmentName;
 import org.apache.pinot.common.utils.SegmentName;
 import org.apache.pinot.common.utils.URIUtils;
 import org.apache.pinot.common.utils.helix.HelixHelper;
 import org.apache.pinot.common.utils.retry.RetryPolicies;
 import org.apache.pinot.controller.ControllerConf;
-import org.apache.pinot.controller.LeadControllerManager;
 import org.apache.pinot.controller.api.events.MetadataEventNotifierFactory;
 import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
-import org.apache.pinot.controller.helix.core.PinotHelixSegmentOnlineOfflineStateModelGenerator;
 import org.apache.pinot.controller.helix.core.PinotTableIdealStateBuilder;
+import org.apache.pinot.controller.helix.core.assignment.segment.SegmentAssignment;
+import org.apache.pinot.controller.helix.core.assignment.segment.SegmentAssignmentFactory;
+import org.apache.pinot.controller.helix.core.assignment.segment.SegmentAssignmentUtils;
 import org.apache.pinot.controller.helix.core.realtime.segment.CommittingSegmentDescriptor;
 import org.apache.pinot.controller.helix.core.realtime.segment.FlushThresholdUpdateManager;
 import org.apache.pinot.controller.helix.core.realtime.segment.FlushThresholdUpdater;
 import org.apache.pinot.controller.util.SegmentCompletionUtils;
-import org.apache.pinot.core.realtime.segment.ConsumingSegmentAssignmentStrategy;
-import org.apache.pinot.core.realtime.segment.RealtimeSegmentAssignmentStrategy;
 import org.apache.pinot.core.realtime.stream.OffsetCriteria;
+import org.apache.pinot.core.realtime.stream.PartitionLevelStreamConfig;
 import org.apache.pinot.core.realtime.stream.PartitionOffsetFetcher;
 import org.apache.pinot.core.realtime.stream.StreamConfig;
 import org.apache.pinot.core.realtime.stream.StreamConfigProperties;
-import org.apache.pinot.core.segment.index.ColumnMetadata;
 import org.apache.pinot.core.segment.index.SegmentMetadataImpl;
 import org.apache.pinot.filesystem.PinotFS;
 import org.apache.pinot.filesystem.PinotFSFactory;
@@ -90,12 +85,23 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 
+/**
+ * Segment manager for LLC real-time table.
+ * <p>Segment management APIs:
+ * <ul>
+ *   <li>setUpNewTable(): From any controller</li>
+ *   <li>removeLLCSegments(): From any controller</li>
+ *   <li>commitSegmentFile(): From lead controller only</li>
+ *   <li>commitSegmentMetadata(): From lead controller only</li>
+ *   <li>segmentStoppedConsuming(): From lead controller only</li>
+ *   <li>ensureAllPartitionsConsuming(): From lead controller only</li>
+ * </ul>
+ */
 public class PinotLLCRealtimeSegmentManager {
-  public static final Logger LOGGER = LoggerFactory.getLogger(PinotLLCRealtimeSegmentManager.class);
-  protected static final int STARTING_SEQUENCE_NUMBER = 0; // Initial sequence number for new table segments
-  protected static final long END_OFFSET_FOR_CONSUMING_SEGMENTS = Long.MAX_VALUE;
+  private static final Logger LOGGER = LoggerFactory.getLogger(PinotLLCRealtimeSegmentManager.class);
 
-  private static final String METADATA_TEMP_DIR_SUFFIX = ".metadata.tmp";
+  private static final int STARTING_SEQUENCE_NUMBER = 0; // Initial sequence number for new table segments
+  private static final long END_OFFSET_FOR_CONSUMING_SEGMENTS = Long.MAX_VALUE;
   private static final String METADATA_EVENT_NOTIFIER_PREFIX = "metadata.event.notifier";
 
   // Max time to wait for all LLC segments to complete committing their metadata while stopping the controller.
@@ -105,11 +111,11 @@ public class PinotLLCRealtimeSegmentManager {
   /**
    * After step 1 of segment completion is done,
    * this is the max time until which step 3 is allowed to complete.
-   * See {@link PinotLLCRealtimeSegmentManager#commitSegmentMetadata(String, CommittingSegmentDescriptor)} for explanation of steps 1 2 3
+   * See {@link #commitSegmentMetadataInternal(String, CommittingSegmentDescriptor)} for explanation of steps 1 2 3
    * This includes any backoffs and retries for the steps 2 and 3
    * The segment will be eligible for repairs by the validation manager, if the time  exceeds this value
    */
-  private static int MAX_SEGMENT_COMPLETION_TIME_MILLIS = 300_000; // 5 MINUTES
+  private static final long MAX_SEGMENT_COMPLETION_TIME_MILLIS = 300_000L; // 5 MINUTES
 
   private final HelixAdmin _helixAdmin;
   private final HelixManager _helixManager;
@@ -118,18 +124,17 @@ public class PinotLLCRealtimeSegmentManager {
   private final String _clusterName;
   private final ControllerConf _controllerConf;
   private final ControllerMetrics _controllerMetrics;
+  private final MetadataEventNotifierFactory _metadataEventNotifierFactory;
   private final int _numIdealStateUpdateLocks;
-  private final Lock[] _idealstateUpdateLocks;
+  private final Lock[] _idealStateUpdateLocks;
   private final TableConfigCache _tableConfigCache;
-  private final StreamPartitionAssignmentGenerator _streamPartitionAssignmentGenerator;
   private final FlushThresholdUpdateManager _flushThresholdUpdateManager;
-  private final LeadControllerManager _leadControllerManager;
 
   private volatile boolean _isStopping = false;
   private AtomicInteger _numCompletingSegments = new AtomicInteger(0);
 
   public PinotLLCRealtimeSegmentManager(PinotHelixResourceManager helixResourceManager, ControllerConf controllerConf,
-      ControllerMetrics controllerMetrics, LeadControllerManager leadControllerManager) {
+      ControllerMetrics controllerMetrics) {
     _helixAdmin = helixResourceManager.getHelixAdmin();
     _helixManager = helixResourceManager.getHelixZkManager();
     _propertyStore = helixResourceManager.getPropertyStore();
@@ -137,15 +142,15 @@ public class PinotLLCRealtimeSegmentManager {
     _clusterName = helixResourceManager.getHelixClusterName();
     _controllerConf = controllerConf;
     _controllerMetrics = controllerMetrics;
+    _metadataEventNotifierFactory =
+        MetadataEventNotifierFactory.loadFactory(controllerConf.subset(METADATA_EVENT_NOTIFIER_PREFIX));
     _numIdealStateUpdateLocks = controllerConf.getRealtimeSegmentMetadataCommitNumLocks();
-    _idealstateUpdateLocks = new Lock[_numIdealStateUpdateLocks];
+    _idealStateUpdateLocks = new Lock[_numIdealStateUpdateLocks];
     for (int i = 0; i < _numIdealStateUpdateLocks; i++) {
-      _idealstateUpdateLocks[i] = new ReentrantLock();
+      _idealStateUpdateLocks[i] = new ReentrantLock();
     }
     _tableConfigCache = new TableConfigCache(_propertyStore);
-    _streamPartitionAssignmentGenerator = new StreamPartitionAssignmentGenerator(_helixManager);
     _flushThresholdUpdateManager = new FlushThresholdUpdateManager();
-    _leadControllerManager = leadControllerManager;
   }
 
   public boolean getIsSplitCommitEnabled() {
@@ -181,183 +186,183 @@ public class PinotLLCRealtimeSegmentManager {
     LOGGER.info("Wait completed: Number of completing segments = {}", _numCompletingSegments.get());
   }
 
-  protected boolean isLeader(String tableName) {
-    return _leadControllerManager.isLeaderForTable(tableName);
-  }
-
-  protected boolean isConnected() {
-    return _helixManager.isConnected();
-  }
-
   /**
-   *
-   * @param tableConfig
-   * @param emptyIdealState may contain HLC segments if both HLC and LLC are configured
+   * Sets up the initial segments for a new LLC real-time table.
+   * <p>NOTE: the passed in IdealState may contain HLC segments if both HLC and LLC are configured.
    */
-  public void setupNewTable(TableConfig tableConfig, IdealState emptyIdealState)
-      throws InvalidConfigException {
-    final StreamConfig streamConfig = new StreamConfig(
-        tableConfig.getTableName(),
-        tableConfig.getIndexingConfig().getStreamConfigs());
-    int partitionCount = getPartitionCount(streamConfig);
-    List<String> currentSegments = getExistingSegments(tableConfig.getTableName());
-    // Make sure that there are no low-level segments existing.
-    if (currentSegments != null) {
-      for (String segment : currentSegments) {
-        if (!SegmentName.isHighLevelConsumerSegmentName(segment)) {
-          // For now, we don't support re-creating the low-level realtime segments
-          throw new RuntimeException("Low-level segments already exist for table " + tableConfig.getTableType());
-        }
-      }
+  public void setUpNewTable(TableConfig tableConfig, IdealState idealState) {
+    Preconditions.checkState(!_isStopping, "Segment manager is stopping");
+
+    String realtimeTableName = tableConfig.getTableName();
+    LOGGER.info("Setting up new LLC table: {}", realtimeTableName);
+
+    // Make sure all the existing segments are HLC segments
+    List<String> currentSegments = getAllSegments(realtimeTableName);
+    for (String segmentName : currentSegments) {
+      Preconditions.checkState(SegmentName.isHighLevelConsumerSegmentName(segmentName),
+          "Cannot set up new LLC table: %s with existing non-HLC segment: %s", realtimeTableName, segmentName);
     }
-    _flushThresholdUpdateManager.clearFlushThresholdUpdater(tableConfig);
-    if (!isConnected()) {
-      throw new RuntimeException(
-          "Lost zk connection while setting up new table " + tableConfig.getTableName() + " isConnected="
-              + isConnected());
+
+    _flushThresholdUpdateManager.clearFlushThresholdUpdater(realtimeTableName);
+
+    PartitionLevelStreamConfig partitionLevelStreamConfig = new PartitionLevelStreamConfig(tableConfig);
+    InstancePartitions instancePartitions = getConsumingInstancePartitions(tableConfig);
+    int numPartitions = getNumPartitions(partitionLevelStreamConfig);
+    int numReplicas = getNumReplicas(tableConfig, instancePartitions);
+
+    SegmentAssignment segmentAssignment = SegmentAssignmentFactory.getSegmentAssignment(_helixManager, tableConfig);
+    Map<InstancePartitionsType, InstancePartitions> instancePartitionsMap =
+        Collections.singletonMap(InstancePartitionsType.CONSUMING, instancePartitions);
+
+    long currentTimeMs = getCurrentTimeMs();
+    Map<String, Map<String, String>> instanceStatesMap = idealState.getRecord().getMapFields();
+    for (int partitionId = 0; partitionId < numPartitions; partitionId++) {
+      String segmentName =
+          setupNewPartition(tableConfig, partitionLevelStreamConfig, partitionId, currentTimeMs, instancePartitions,
+              numPartitions, numReplicas);
+      updateInstanceStatesForNewConsumingSegment(instanceStatesMap, null, segmentName, segmentAssignment,
+          instancePartitionsMap);
     }
-    IdealState idealState = setupTable(tableConfig, emptyIdealState, partitionCount);
-    setTableIdealState(tableConfig.getTableName(), idealState);
+
+    setIdealState(realtimeTableName, idealState);
   }
 
-  // Remove all trace of LLC for this table.
-  public void cleanupLLC(final String realtimeTableName) {
-    // If there are any completions in the pipeline we let them commit.
-    Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);
+  /**
+   * Removes all LLC segments from the given IdealState.
+   */
+  public void removeLLCSegments(IdealState idealState) {
+    Preconditions.checkState(!_isStopping, "Segment manager is stopping");
+
+    String realtimeTableName = idealState.getResourceName();
+    LOGGER.info("Removing LLC segments for table: {}", realtimeTableName);
 
-    IdealState idealState = HelixHelper.getTableIdealState(_helixManager, realtimeTableName);
-    final List<String> segmentsToRemove = new ArrayList<String>();
-    Set<String> allSegments = idealState.getPartitionSet();
-    int removeCount = 0;
-    for (String segmentName : allSegments) {
+    List<String> segmentsToRemove = new ArrayList<>();
+    for (String segmentName : idealState.getRecord().getMapFields().keySet()) {
       if (SegmentName.isLowLevelConsumerSegmentName(segmentName)) {
         segmentsToRemove.add(segmentName);
-        removeCount++;
       }
     }
-    LOGGER.info("Attempting to remove {} LLC segments of table {}", removeCount, realtimeTableName);
-
     _helixResourceManager.deleteSegments(realtimeTableName, segmentsToRemove);
   }
 
-  private SegmentPartitionMetadata getPartitionMetadataFromTableConfig(String tableName, int numPartitions,
-      int partitionId) {
-    Map<String, ColumnPartitionMetadata> partitionMetadataMap = new HashMap<>();
-    if (_propertyStore == null) {
-      return null;
-    }
-    TableConfig tableConfig = getRealtimeTableConfig(tableName);
-    SegmentPartitionMetadata partitionMetadata = null;
-    SegmentPartitionConfig partitionConfig = tableConfig.getIndexingConfig().getSegmentPartitionConfig();
-    if (partitionConfig != null && partitionConfig.getColumnPartitionMap() != null
-        && partitionConfig.getColumnPartitionMap().size() > 0) {
-      Map<String, ColumnPartitionConfig> columnPartitionMap = partitionConfig.getColumnPartitionMap();
-      for (Map.Entry<String, ColumnPartitionConfig> entry : columnPartitionMap.entrySet()) {
-        String column = entry.getKey();
-        ColumnPartitionConfig columnPartitionConfig = entry.getValue();
-        partitionMetadataMap.put(column,
-            new ColumnPartitionMetadata(columnPartitionConfig.getFunctionName(), numPartitions,
-                Collections.singleton(partitionId)));
-      }
-      partitionMetadata = new SegmentPartitionMetadata(partitionMetadataMap);
+  @VisibleForTesting
+  TableConfig getTableConfig(String realtimeTableName) {
+    try {
+      return _tableConfigCache.getTableConfig(realtimeTableName);
+    } catch (ExecutionException e) {
+      _controllerMetrics.addMeteredTableValue(realtimeTableName, ControllerMeter.LLC_ZOOKEEPER_FETCH_FAILURES, 1L);
+      throw new IllegalStateException(
+          "Caught exception while loading table config from property store to cache for table: " + realtimeTableName,
+          e);
     }
-    return partitionMetadata;
   }
 
-  /**
-   * Given a segment metadata, build the finalized version of segment partition metadata. This partition metadata will
-   * be written as a part of SegmentZKMetadata.
-   * @param segmentMetadata Segment metadata
-   * @return
-   */
-  private SegmentPartitionMetadata getPartitionMetadataFromSegmentMetadata(SegmentMetadataImpl segmentMetadata) {
-    Map<String, ColumnPartitionMetadata> partitionMetadataMap = new HashMap<>();
-    for (Map.Entry<String, ColumnMetadata> entry : segmentMetadata.getColumnMetadataMap().entrySet()) {
-      String columnName = entry.getKey();
-      ColumnMetadata columnMetadata = entry.getValue();
-      // Check if the column metadata contains the partition information
-      if (columnMetadata.getPartitionFunction() != null) {
-        partitionMetadataMap.put(columnName,
-            new ColumnPartitionMetadata(columnMetadata.getPartitionFunction().toString(),
-                columnMetadata.getNumPartitions(), columnMetadata.getPartitions()));
-      }
+  @VisibleForTesting
+  InstancePartitions getConsumingInstancePartitions(TableConfig tableConfig) {
+    try {
+      return InstancePartitionsUtils
+          .fetchOrComputeInstancePartitions(_helixManager, tableConfig, InstancePartitionsType.CONSUMING);
+    } catch (Exception e) {
+      _controllerMetrics
+          .addMeteredTableValue(tableConfig.getTableName(), ControllerMeter.LLC_ZOOKEEPER_FETCH_FAILURES, 1L);
+      throw e;
     }
-    return new SegmentPartitionMetadata(partitionMetadataMap);
   }
 
-  protected List<String> getExistingSegments(String realtimeTableName) {
-    String propStorePath = ZKMetadataProvider.constructPropertyStorePathForResource(realtimeTableName);
-    return _propertyStore.getChildNames(propStorePath, AccessOption.PERSISTENT);
+  @VisibleForTesting
+  List<String> getAllSegments(String realtimeTableName) {
+    try {
+      return ZKMetadataProvider.getSegments(_propertyStore, realtimeTableName);
+    } catch (Exception e) {
+      _controllerMetrics.addMeteredTableValue(realtimeTableName, ControllerMeter.LLC_ZOOKEEPER_FETCH_FAILURES, 1L);
+      throw e;
+    }
   }
 
-  protected boolean writeSegmentToPropertyStore(String znodePath, ZNRecord znRecord, final String realtimeTableName,
-      int expectedVersion) {
-    boolean success = _propertyStore.set(znodePath, znRecord, expectedVersion, AccessOption.PERSISTENT);
-    if (!success) {
-      LOGGER
-          .error("Failed to write segment to property store at {} for table {}. Expected zookeeper version number: {}",
-              znodePath, realtimeTableName, expectedVersion);
-      return false;
+  @VisibleForTesting
+  List<String> getLLCSegments(String realtimeTableName) {
+    try {
+      return ZKMetadataProvider.getLLCRealtimeSegments(_propertyStore, realtimeTableName);
+    } catch (Exception e) {
+      _controllerMetrics.addMeteredTableValue(realtimeTableName, ControllerMeter.LLC_ZOOKEEPER_FETCH_FAILURES, 1L);
+      throw e;
     }
-    return success;
   }
 
-  protected boolean writeSegmentToPropertyStore(String znodePath, ZNRecord znRecord, final String realtimeTableName) {
-    boolean success = _propertyStore.set(znodePath, znRecord, AccessOption.PERSISTENT);
-    if (!success) {
-      LOGGER.error("Failed to write segment to property store at {} for table {}.", znodePath, realtimeTableName);
-    }
-    return success;
+  private LLCRealtimeSegmentZKMetadata getSegmentZKMetadata(String realtimeTableName, String segmentName) {
+    return getSegmentZKMetadata(realtimeTableName, segmentName, null);
   }
 
-  protected void writeSegmentsToPropertyStore(List<String> paths, List<ZNRecord> records,
-      final String realtimeTableName) {
+  @VisibleForTesting
+  LLCRealtimeSegmentZKMetadata getSegmentZKMetadata(String realtimeTableName, String segmentName, @Nullable Stat stat) {
     try {
-      _propertyStore.setChildren(paths, records, AccessOption.PERSISTENT);
+      ZNRecord znRecord = _propertyStore
+          .get(ZKMetadataProvider.constructPropertyStorePathForSegment(realtimeTableName, segmentName), stat,
+              AccessOption.PERSISTENT);
+      Preconditions
+          .checkState(znRecord != null, "Failed to find segment ZK metadata for segment: %s of table: %s", segmentName,
+              realtimeTableName);
+      return new LLCRealtimeSegmentZKMetadata(znRecord);
     } catch (Exception e) {
-      LOGGER.error("Failed to update idealstate for table {} for paths {}", realtimeTableName, paths, e);
-      _controllerMetrics.addMeteredGlobalValue(ControllerMeter.LLC_ZOOKEEPER_UPDATE_FAILURES, 1);
+      _controllerMetrics.addMeteredTableValue(realtimeTableName, ControllerMeter.LLC_ZOOKEEPER_FETCH_FAILURES, 1L);
       throw e;
     }
   }
 
-  protected IdealState getTableIdealState(String realtimeTableName) {
-    return HelixHelper.getTableIdealState(_helixManager, realtimeTableName);
+  @VisibleForTesting
+  void persistSegmentZKMetadata(String realtimeTableName, LLCRealtimeSegmentZKMetadata segmentZKMetadata,
+      int expectedVersion) {
+    String segmentName = segmentZKMetadata.getSegmentName();
+    LOGGER.info("Persisting segment ZK metadata for segment: {}", segmentName);
+    try {
+      Preconditions.checkState(_propertyStore
+              .set(ZKMetadataProvider.constructPropertyStorePathForSegment(realtimeTableName, segmentName),
+                  segmentZKMetadata.toZNRecord(), expectedVersion, AccessOption.PERSISTENT),
+          "Failed to persist segment ZK metadata for segment: %s of table: %s", segmentName, realtimeTableName);
+    } catch (Exception e) {
+      _controllerMetrics.addMeteredTableValue(realtimeTableName, ControllerMeter.LLC_ZOOKEEPER_UPDATE_FAILURES, 1L);
+      throw e;
+    }
   }
 
-  protected void setTableIdealState(String realtimeTableName, IdealState idealState) {
-    _helixAdmin.setResourceIdealState(_clusterName, realtimeTableName, idealState);
+  @VisibleForTesting
+  IdealState getIdealState(String realtimeTableName) {
+    try {
+      IdealState idealState = HelixHelper.getTableIdealState(_helixManager, realtimeTableName);
+      Preconditions.checkState(idealState != null, "Failed to find IdealState for table: " + realtimeTableName);
+      return idealState;
+    } catch (Exception e) {
+      _controllerMetrics.addMeteredTableValue(realtimeTableName, ControllerMeter.LLC_ZOOKEEPER_FETCH_FAILURES, 1L);
+      throw e;
+    }
   }
 
-  public boolean commitSegmentFile(String tableName, CommittingSegmentDescriptor committingSegmentDescriptor) {
-    if (_isStopping) {
-      LOGGER.info("Returning false since the controller is stopping");
-      return false;
+  @VisibleForTesting
+  void setIdealState(String realtimeTableName, IdealState idealState) {
+    try {
+      _helixAdmin.setResourceIdealState(_clusterName, realtimeTableName, idealState);
+    } catch (Exception e) {
+      _controllerMetrics.addMeteredTableValue(realtimeTableName, ControllerMeter.LLC_ZOOKEEPER_UPDATE_FAILURES, 1L);
+      throw e;
     }
+  }
+
+  public void commitSegmentFile(String realtimeTableName, CommittingSegmentDescriptor committingSegmentDescriptor)
+      throws Exception {
+    Preconditions.checkState(!_isStopping, "Segment manager is stopping");
+
+    String rawTableName = TableNameBuilder.extractRawTableName(realtimeTableName);
     String segmentName = committingSegmentDescriptor.getSegmentName();
+    LOGGER.info("Committing segment file for segment: {}", segmentName);
+
+    // Copy the segment file to the controller
     String segmentLocation = committingSegmentDescriptor.getSegmentLocation();
     URI segmentFileURI = URIUtils.getUri(segmentLocation);
-    URI tableDirURI = URIUtils.getUri(_controllerConf.getDataDir(), tableName);
-    URI uriToMoveTo = URIUtils.getUri(_controllerConf.getDataDir(), tableName, URIUtils.encode(segmentName));
+    URI tableDirURI = URIUtils.getUri(_controllerConf.getDataDir(), rawTableName);
+    URI uriToMoveTo = URIUtils.getUri(_controllerConf.getDataDir(), rawTableName, URIUtils.encode(segmentName));
     PinotFS pinotFS = PinotFSFactory.create(tableDirURI.getScheme());
-
-    if (!isConnected() || !isLeader(tableName)) {
-      // We can potentially log a different value than what we saw ....
-      LOGGER.warn("Lost leadership while committing segment file {}, {} for table {}: isLeader={}, isConnected={}",
-          segmentName, segmentLocation, tableName, isLeader(tableName), isConnected());
-      _controllerMetrics.addMeteredGlobalValue(ControllerMeter.CONTROLLER_NOT_LEADER, 1L);
-      return false;
-    }
-
-    try {
-      if (!pinotFS.move(segmentFileURI, uriToMoveTo, true)) {
-        LOGGER.error("Could not move {} to {}", segmentLocation, segmentName);
-        return false;
-      }
-    } catch (Exception e) {
-      LOGGER.error("Could not move {} to {}", segmentLocation, segmentName, e);
-      return false;
-    }
+    Preconditions.checkState(pinotFS.move(segmentFileURI, uriToMoveTo, true),
+        "Failed to move segment file for segment: %s from: %s to: %s", segmentName, segmentLocation, uriToMoveTo);
 
     // Cleans up tmp segment files under table dir.
     // We only clean up tmp segment files in table level dir, so there's no need to list recursively.
@@ -366,48 +371,44 @@ public class PinotLLCRealtimeSegmentManager {
     try {
       for (String uri : pinotFS.listFiles(tableDirURI, false)) {
         if (uri.contains(SegmentCompletionUtils.getSegmentNamePrefix(segmentName))) {
-          LOGGER.warn("Deleting " + uri);
-          pinotFS.delete(new URI(uri), true);
+          LOGGER.warn("Deleting temporary segment file: {}", uri);
+          Preconditions.checkState(pinotFS.delete(new URI(uri), true), "Failed to delete file: %s", uri);
         }
       }
     } catch (Exception e) {
-      LOGGER.warn("Could not delete tmp segment files for {}", tableDirURI, e);
+      LOGGER.warn("Caught exception while deleting temporary segment files for segment: {}", segmentName, e);
     }
-
-    return true;
   }
 
   /**
    * This method is invoked after the realtime segment is uploaded but before a response is sent to the server.
    * It updates the propertystore segment metadata from IN_PROGRESS to DONE, and also creates new propertystore
    * records for new segments, and puts them in idealstate in CONSUMING state.
-   *
-   * @param rawTableName Raw table name
-   * @param committingSegmentDescriptor
-   * @return boolean
    */
-  public boolean commitSegmentMetadata(String rawTableName, CommittingSegmentDescriptor committingSegmentDescriptor) {
-    if (_isStopping) {
-      LOGGER.info("Returning false since the controller is stopping");
-      return false;
-    }
+  public void commitSegmentMetadata(String realtimeTableName, CommittingSegmentDescriptor committingSegmentDescriptor) {
+    Preconditions.checkState(!_isStopping, "Segment manager is stopping");
+
     try {
       _numCompletingSegments.addAndGet(1);
-      return commitSegmentMetadataInternal(rawTableName, committingSegmentDescriptor);
+      commitSegmentMetadataInternal(realtimeTableName, committingSegmentDescriptor);
     } finally {
       _numCompletingSegments.addAndGet(-1);
     }
   }
 
-  private boolean commitSegmentMetadataInternal(String rawTableName,
+  private void commitSegmentMetadataInternal(String realtimeTableName,
       CommittingSegmentDescriptor committingSegmentDescriptor) {
-
-    final String realtimeTableName = TableNameBuilder.REALTIME.tableNameWithType(rawTableName);
-    TableConfig tableConfig = getRealtimeTableConfig(realtimeTableName);
-    if (tableConfig == null) {
-      LOGGER.warn("Did not find table config for table {}", realtimeTableName);
-      return false;
-    }
+    String committingSegmentName = committingSegmentDescriptor.getSegmentName();
+    LOGGER.info("Committing segment metadata for segment: {}", committingSegmentName);
+
+    TableConfig tableConfig = getTableConfig(realtimeTableName);
+    InstancePartitions instancePartitions = getConsumingInstancePartitions(tableConfig);
+    IdealState idealState = getIdealState(realtimeTableName);
+    Preconditions.checkState(idealState.getInstanceStateMap(committingSegmentName)
+            .containsValue(RealtimeSegmentOnlineOfflineStateModel.CONSUMING),
+        "Failed to find instance in CONSUMING state in IdealState for segment: %s", committingSegmentName);
+    int numPartitions = getNumPartitionsFromIdealState(idealState);
+    int numReplicas = getNumReplicas(tableConfig, instancePartitions);
 
     /*
      * Update zookeeper in 3 steps.
@@ -417,56 +418,22 @@ public class PinotLLCRealtimeSegmentManager {
      * Step 3: Update IDEALSTATES to include new segment in CONSUMING state, and change old segment to ONLINE state.
      */
 
-    final String committingSegmentNameStr = committingSegmentDescriptor.getSegmentName();
-    final long nextOffset = committingSegmentDescriptor.getNextOffset();
-    final LLCSegmentName committingLLCSegmentName = new LLCSegmentName(committingSegmentNameStr);
-    final int partitionId = committingLLCSegmentName.getPartitionId();
-    final int newSeqNum = committingLLCSegmentName.getSequenceNumber() + 1;
-    final long now = System.currentTimeMillis();
-
-    LLCSegmentName newLLCSegmentName =
-        new LLCSegmentName(committingLLCSegmentName.getTableName(), partitionId, newSeqNum, now);
-    String newSegmentNameStr = newLLCSegmentName.getSegmentName();
-
-    IdealState idealState = getTableIdealState(realtimeTableName);
-    Preconditions.checkState(idealState.getInstanceStateMap(committingSegmentNameStr)
-        .containsValue(PinotHelixSegmentOnlineOfflineStateModelGenerator.CONSUMING_STATE));
-    int numPartitions = _streamPartitionAssignmentGenerator.getNumPartitionsFromIdealState(idealState);
-
-    PartitionAssignment partitionAssignment;
-    try {
-      partitionAssignment =
-          _streamPartitionAssignmentGenerator.generateStreamPartitionAssignment(tableConfig, numPartitions);
-    } catch (InvalidConfigException e) {
-      LOGGER
-          .error("Exception when generating partition assignment for table {} and numPartitions {}", realtimeTableName,
-              numPartitions, e);
-      return false;
-    }
-
-    // If an LLC table is dropped (or cleaned up), we will get null here. In that case we should not be creating a new segment
-    if (partitionAssignment == null) {
-      LOGGER.warn("Partition assignment not found for {}", realtimeTableName);
-      throw new RuntimeException("Partition assignment not found. Not committing segment");
-    }
-
     // Step-1
-    boolean success = updateOldSegmentMetadataZNRecord(realtimeTableName, committingLLCSegmentName, nextOffset,
-        committingSegmentDescriptor);
-    if (!success) {
-      return false;
-    }
+    LLCRealtimeSegmentZKMetadata committingSegmentZKMetadata =
+        updateCommittingSegmentZKMetadata(realtimeTableName, committingSegmentDescriptor);
 
     // Step-2
-    success =
-        createNewSegmentMetadataZNRecord(tableConfig, committingLLCSegmentName, newLLCSegmentName, partitionAssignment,
-            committingSegmentDescriptor, false);
-    if (!success) {
-      return false;
-    }
+    long newSegmentCreationTimeMs = getCurrentTimeMs();
+    LLCSegmentName newLLCSegmentName =
+        getNextLLCSegmentName(new LLCSegmentName(committingSegmentName), newSegmentCreationTimeMs);
+    createNewSegmentZKMetadata(tableConfig, new PartitionLevelStreamConfig(tableConfig), newLLCSegmentName,
+        newSegmentCreationTimeMs, committingSegmentDescriptor, committingSegmentZKMetadata, instancePartitions,
+        numPartitions, numReplicas);
 
     // Step-3
-    // TODO Introduce a controller failure here for integration testing
+    SegmentAssignment segmentAssignment = SegmentAssignmentFactory.getSegmentAssignment(_helixManager, tableConfig);
+    Map<InstancePartitionsType, InstancePartitions> instancePartitionsMap =
+        Collections.singletonMap(InstancePartitionsType.CONSUMING, instancePartitions);
 
     // When multiple segments of the same table complete around the same time it is possible that
     // the idealstate update fails due to contention. We serialize the updates to the idealstate
@@ -474,173 +441,117 @@ public class PinotLLCRealtimeSegmentManager {
     // to idealstate from other controllers, but then we have the retry mechanism to get around that.
     // hash code can be negative, so make sure we are getting a positive lock index
     int lockIndex = (realtimeTableName.hashCode() & Integer.MAX_VALUE) % _numIdealStateUpdateLocks;
-    Lock lock = _idealstateUpdateLocks[lockIndex];
+    Lock lock = _idealStateUpdateLocks[lockIndex];
     try {
       lock.lock();
-      updateIdealStateOnSegmentCompletion(realtimeTableName, committingSegmentNameStr, newSegmentNameStr,
-          partitionAssignment);
-      LOGGER.info("Changed {} to ONLINE and created {} in CONSUMING", committingSegmentNameStr, newSegmentNameStr);
-    } catch (Exception e) {
-      LOGGER.error("Caught exception when updating ideal state for {}", committingSegmentNameStr, e);
-      return false;
+      updateIdealStateOnSegmentCompletion(realtimeTableName, committingSegmentName, newLLCSegmentName.getSegmentName(),
+          segmentAssignment, instancePartitionsMap);
     } finally {
       lock.unlock();
     }
 
     // Trigger the metadata event notifier
-    notifyOnSegmentFlush(realtimeTableName);
-
-    return true;
+    _metadataEventNotifierFactory.create().notifyOnSegmentFlush(tableConfig);
   }
 
   /**
-   * Update segment metadata of committing segment
-   * @param realtimeTableName - table name for which segment is being committed
-   * @param committingLLCSegmentName - name of the segment being committed
-   * @param nextOffset - the end offset for this committing segment
-   * @param committingSegmentDescriptor - the metadata of the commit segment.
-   * @return
+   * Updates segment ZK metadata for the committing segment.
    */
-  protected boolean updateOldSegmentMetadataZNRecord(String realtimeTableName, LLCSegmentName committingLLCSegmentName,
-      long nextOffset, CommittingSegmentDescriptor committingSegmentDescriptor) {
+  private LLCRealtimeSegmentZKMetadata updateCommittingSegmentZKMetadata(String realtimeTableName,
+      CommittingSegmentDescriptor committingSegmentDescriptor) {
+    String segmentName = committingSegmentDescriptor.getSegmentName();
+    LOGGER.info("Updating segment ZK metadata for committing segment: {}", segmentName);
 
-    String committingSegmentNameStr = committingLLCSegmentName.getSegmentName();
     Stat stat = new Stat();
-    final LLCRealtimeSegmentZKMetadata committingSegmentMetadata =
-        getRealtimeSegmentZKMetadata(realtimeTableName, committingSegmentNameStr, stat);
-
-    if (committingSegmentMetadata.getStatus() != CommonConstants.Segment.Realtime.Status.IN_PROGRESS) {
-      LOGGER.warn("Status of segment metadata {} has already been changed by other controller for table {}: Status={}",
-          committingSegmentNameStr, realtimeTableName, committingSegmentMetadata.getStatus());
-      return false;
-    }
-    if (committingSegmentDescriptor.getSegmentMetadata() == null) {
-      LOGGER.error("No segment metadata found in descriptor for committing segment {} for table {}",
-          committingLLCSegmentName, realtimeTableName);
-      return false;
-    }
+    LLCRealtimeSegmentZKMetadata committingSegmentZKMetadata =
+        getSegmentZKMetadata(realtimeTableName, segmentName, stat);
+    Preconditions.checkState(committingSegmentZKMetadata.getStatus() == Status.IN_PROGRESS,
+        "Segment status for segment: %s should be IN_PROGRESS, found: %s", segmentName,
+        committingSegmentZKMetadata.getStatus());
     SegmentMetadataImpl segmentMetadata = committingSegmentDescriptor.getSegmentMetadata();
-
-    // TODO: set number of rows to end consumption in new segment metadata, based on memory used and number of rows from old segment
-    committingSegmentMetadata.setEndOffset(nextOffset);
-    committingSegmentMetadata.setStatus(CommonConstants.Segment.Realtime.Status.DONE);
-    String rawTableName = TableNameBuilder.extractRawTableName(realtimeTableName);
-    committingSegmentMetadata.setDownloadUrl(
-        URIUtils.constructDownloadUrl(_controllerConf.generateVipUrl(), rawTableName, committingSegmentNameStr));
-    committingSegmentMetadata.setCrc(Long.valueOf(segmentMetadata.getCrc()));
-    committingSegmentMetadata.setStartTime(segmentMetadata.getTimeInterval().getStartMillis());
-    committingSegmentMetadata.setEndTime(segmentMetadata.getTimeInterval().getEndMillis());
-    committingSegmentMetadata.setTimeUnit(TimeUnit.MILLISECONDS);
-    committingSegmentMetadata.setIndexVersion(segmentMetadata.getVersion());
-    committingSegmentMetadata.setTotalRawDocs(segmentMetadata.getTotalRawDocs());
-    committingSegmentMetadata.setPartitionMetadata(getPartitionMetadataFromSegmentMetadata(segmentMetadata));
-
-    final ZNRecord oldZnRecord = committingSegmentMetadata.toZNRecord();
-    final String oldZnodePath =
-        ZKMetadataProvider.constructPropertyStorePathForSegment(realtimeTableName, committingSegmentNameStr);
-
-    if (!isConnected() || !isLeader(realtimeTableName)) {
-      // We can potentially log a different value than what we saw ....
-      LOGGER.warn("Lost leadership while committing segment metadata for {} for table {}: isLeader={}, isConnected={}",
-          committingSegmentNameStr, realtimeTableName, isLeader(realtimeTableName), isConnected());
-      _controllerMetrics.addMeteredGlobalValue(ControllerMeter.CONTROLLER_NOT_LEADER, 1L);
-      return false;
-    }
-    boolean success = writeSegmentToPropertyStore(oldZnodePath, oldZnRecord, realtimeTableName, stat.getVersion());
-    if (!success) {
-      LOGGER.warn("Fail to write old segment to property store for {} for table {}: isLeader={}, isConnected={}",
-          committingSegmentNameStr, realtimeTableName, isLeader(realtimeTableName), isConnected());
-    }
-    return success;
+    Preconditions.checkState(segmentMetadata != null, "Failed to find segment metadata from descriptor for segment: %s",
+        segmentName);
+
+    committingSegmentZKMetadata.setEndOffset(committingSegmentDescriptor.getNextOffset());
+    committingSegmentZKMetadata.setStatus(Status.DONE);
+    committingSegmentZKMetadata.setDownloadUrl(URIUtils
+        .constructDownloadUrl(_controllerConf.generateVipUrl(), TableNameBuilder.extractRawTableName(realtimeTableName),
+            segmentName));
+    committingSegmentZKMetadata.setCrc(Long.valueOf(segmentMetadata.getCrc()));
+    committingSegmentZKMetadata.setStartTime(segmentMetadata.getTimeInterval().getStartMillis());
+    committingSegmentZKMetadata.setEndTime(segmentMetadata.getTimeInterval().getEndMillis());
+    committingSegmentZKMetadata.setTimeUnit(TimeUnit.MILLISECONDS);
+    committingSegmentZKMetadata.setIndexVersion(segmentMetadata.getVersion());
+    committingSegmentZKMetadata.setTotalRawDocs(segmentMetadata.getTotalRawDocs());
+
+    persistSegmentZKMetadata(realtimeTableName, committingSegmentZKMetadata, stat.getVersion());
+    return committingSegmentZKMetadata;
   }
 
   /**
-   * Creates segment metadata for next sequence number from the segment just committed
-   * @param realtimeTableConfig  table config of the segment for which new metadata is being created
-   * @param committingLLCSegmentName
-   * @param newLLCSegmentName  new segment name
-   * @param partitionAssignment  stream partition assignment for this table
-   * @param committingSegmentDescriptor
-   * @param isNewTableSetup
-   * @return
+   * Creates and persists segment ZK metadata for the new CONSUMING segment.
    */
-  protected boolean createNewSegmentMetadataZNRecord(TableConfig realtimeTableConfig,
-      LLCSegmentName committingLLCSegmentName, LLCSegmentName newLLCSegmentName,
-      PartitionAssignment partitionAssignment, CommittingSegmentDescriptor committingSegmentDescriptor,
-      boolean isNewTableSetup) {
-
-    String realtimeTableName = realtimeTableConfig.getTableName();
-    String rawTableName = TableNameBuilder.extractRawTableName(realtimeTableName);
-
-    LLCRealtimeSegmentZKMetadata committingSegmentZKMetadata = null;
-    if (committingLLCSegmentName != null) {
-      committingSegmentZKMetadata =
-          getRealtimeSegmentZKMetadata(realtimeTableName, committingLLCSegmentName.getSegmentName(), null);
+  private void createNewSegmentZKMetadata(TableConfig tableConfig,
+      PartitionLevelStreamConfig partitionLevelStreamConfig, LLCSegmentName newLLCSegmentName, long creationTimeMs,
+      CommittingSegmentDescriptor committingSegmentDescriptor,
+      @Nullable LLCRealtimeSegmentZKMetadata committingSegmentZKMetadata, InstancePartitions instancePartitions,
+      int numPartitions, int numReplicas) {
+    String realtimeTableName = tableConfig.getTableName();
+    String segmentName = newLLCSegmentName.getSegmentName();
+    long startOffset = committingSegmentDescriptor.getNextOffset();
+    LOGGER
+        .info("Creating segment ZK metadata for new CONSUMING segment: {} with start offset: {} and creation time: {}",
+            segmentName, startOffset, creationTimeMs);
+
+    LLCRealtimeSegmentZKMetadata newSegmentZKMetadata = new LLCRealtimeSegmentZKMetadata();
+    newSegmentZKMetadata.setTableName(realtimeTableName);
+    newSegmentZKMetadata.setSegmentName(segmentName);
+    newSegmentZKMetadata.setCreationTime(creationTimeMs);
+    newSegmentZKMetadata.setStartOffset(startOffset);
+    newSegmentZKMetadata.setEndOffset(END_OFFSET_FOR_CONSUMING_SEGMENTS);
+    newSegmentZKMetadata.setNumReplicas(numReplicas);
+    newSegmentZKMetadata.setStatus(Status.IN_PROGRESS);
+
+    // Add the partition metadata if available
+    SegmentPartitionMetadata partitionMetadata =
+        getPartitionMetadataFromTableConfig(tableConfig, newLLCSegmentName.getPartitionId());
+    if (partitionMetadata != null) {
+      newSegmentZKMetadata.setPartitionMetadata(partitionMetadata);
     }
 
-    int partitionId = newLLCSegmentName.getPartitionId();
-    int numReplicas = partitionAssignment.getInstancesListForPartition(String.valueOf(partitionId)).size();
-    ZNRecord newZnRecord =
-        makeZnRecordForNewSegment(rawTableName, numReplicas, committingSegmentDescriptor.getNextOffset(),
-            newLLCSegmentName, partitionAssignment.getNumPartitions());
-    final LLCRealtimeSegmentZKMetadata newSegmentZKMetadata = new LLCRealtimeSegmentZKMetadata(newZnRecord);
-
+    // Update the flush threshold
     FlushThresholdUpdater flushThresholdUpdater =
-        _flushThresholdUpdateManager.getFlushThresholdUpdater(realtimeTableConfig);
+        _flushThresholdUpdateManager.getFlushThresholdUpdater(partitionLevelStreamConfig);
     flushThresholdUpdater
-        .updateFlushThreshold(newSegmentZKMetadata, committingSegmentZKMetadata, committingSegmentDescriptor,
-            partitionAssignment);
-
-    newZnRecord = newSegmentZKMetadata.toZNRecord();
-
-    final String newSegmentNameStr = newLLCSegmentName.getSegmentName();
-    final String newZnodePath =
-        ZKMetadataProvider.constructPropertyStorePathForSegment(realtimeTableName, newSegmentNameStr);
-
-    if (!isNewTableSetup) {
-      if (!isLeader(realtimeTableName) || !isConnected()) {
-        // We can potentially log a different value than what we saw ....
-        LOGGER.warn(
-            "Lost leadership while committing new segment metadata for {} for table {}: isLeader={}, isConnected={}",
-            newSegmentNameStr, rawTableName, isLeader(realtimeTableName), isConnected());
-        _controllerMetrics.addMeteredGlobalValue(ControllerMeter.CONTROLLER_NOT_LEADER, 1L);
-        return false;
-      }
-    }
+        .updateFlushThreshold(newSegmentZKMetadata, committingSegmentDescriptor, committingSegmentZKMetadata,
+            getMaxNumPartitionsPerInstance(instancePartitions, numPartitions, numReplicas));
 
-    boolean success = writeSegmentToPropertyStore(newZnodePath, newZnRecord, realtimeTableName);
-    if (!success) {
-      LOGGER.warn("Fail to write new segment to property store for {} for table {}: isLeader={}, isConnected={}",
-          newSegmentNameStr, rawTableName, isLeader(realtimeTableName), isConnected());
-    }
-    return success;
+    persistSegmentZKMetadata(realtimeTableName, newSegmentZKMetadata, -1);
   }
 
-  /**
-   * Helper function to return cached table config.
-   *
-   * @param tableName name of the table
-   * @return table configuration that reflects the most recent version
-   */
-  protected TableConfig getRealtimeTableConfig(String tableName) {
-    TableConfig tableConfig;
-    String tableNameWithType = TableNameBuilder.REALTIME.tableNameWithType(tableName);
-    try {
-      tableConfig = _tableConfigCache.getTableConfig(tableNameWithType);
-    } catch (ExecutionException e) {
-      LOGGER.warn("Exception happened while loading the table config ({}) from the property store to the cache.",
-          tableNameWithType, e);
+  @Nullable
+  private SegmentPartitionMetadata getPartitionMetadataFromTableConfig(TableConfig tableConfig, int partitionId) {
+    SegmentPartitionConfig partitionConfig = tableConfig.getIndexingConfig().getSegmentPartitionConfig();
+    if (partitionConfig == null) {
       return null;
     }
-    return tableConfig;
+    Map<String, ColumnPartitionMetadata> partitionMetadataMap = new TreeMap<>();
+    for (Map.Entry<String, ColumnPartitionConfig> entry : partitionConfig.getColumnPartitionMap().entrySet()) {
+      String columnName = entry.getKey();
+      ColumnPartitionConfig columnPartitionConfig = entry.getValue();
+      partitionMetadataMap.put(columnName,
+          new ColumnPartitionMetadata(columnPartitionConfig.getFunctionName(), columnPartitionConfig.getNumPartitions(),
+              Collections.singleton(partitionId)));
+    }
+    return new SegmentPartitionMetadata(partitionMetadataMap);
   }
 
-  public long getCommitTimeoutMS(String tableName) {
+  public long getCommitTimeoutMS(String realtimeTableName) {
     long commitTimeoutMS = SegmentCompletionProtocol.getMaxSegmentCommitTimeMs();
     if (_propertyStore == null) {
       return commitTimeoutMS;
     }
-    TableConfig tableConfig = getRealtimeTableConfig(tableName);
+    TableConfig tableConfig = getTableConfig(realtimeTableName);
     final Map<String, String> streamConfigs = tableConfig.getIndexingConfig().getStreamConfigs();
     if (streamConfigs != null && streamConfigs.containsKey(StreamConfigProperties.SEGMENT_COMMIT_TIMEOUT_SECONDS)) {
       final String commitTimeoutSecondsStr = streamConfigs.get(StreamConfigProperties.SEGMENT_COMMIT_TIMEOUT_SECONDS);
@@ -654,102 +565,23 @@ public class PinotLLCRealtimeSegmentManager {
     return commitTimeoutMS;
   }
 
-  public LLCRealtimeSegmentZKMetadata getRealtimeSegmentZKMetadata(String realtimeTableName, String segmentName,
-      Stat stat) {
-    ZNRecord znRecord = _propertyStore
-        .get(ZKMetadataProvider.constructPropertyStorePathForSegment(realtimeTableName, segmentName), stat,
-            AccessOption.PERSISTENT);
-    if (znRecord == null) {
-      LOGGER.error("Segment metadata not found for table {}, segment {}. (can happen during table drop)",
-          realtimeTableName, segmentName);
-      throw new RuntimeException(
-          "Segment metadata not found for table " + realtimeTableName + " segment " + segmentName);
-    }
-    return new LLCRealtimeSegmentZKMetadata(znRecord);
-  }
-
-  protected long getPartitionOffset(StreamConfig streamConfig, final OffsetCriteria offsetCriteria, int partitionId) {
-    return fetchPartitionOffset(streamConfig, offsetCriteria, partitionId);
+  @VisibleForTesting
+  int getNumPartitions(StreamConfig streamConfig) {
+    return PinotTableIdealStateBuilder.getPartitionCount(streamConfig);
   }
 
-  private long fetchPartitionOffset(StreamConfig streamConfig, final OffsetCriteria offsetCriteria, int partitionId) {
+  @VisibleForTesting
+  long getPartitionOffset(StreamConfig streamConfig, OffsetCriteria offsetCriteria, int partitionId) {
     PartitionOffsetFetcher partitionOffsetFetcher =
         new PartitionOffsetFetcher(offsetCriteria, partitionId, streamConfig);
     try {
       RetryPolicies.fixedDelayRetryPolicy(3, 1000L).attempt(partitionOffsetFetcher);
       return partitionOffsetFetcher.getOffset();
     } catch (Exception e) {
-      Exception fetcherException = partitionOffsetFetcher.getException();
-      LOGGER.error("Could not get offset for topic {} partition {}, criteria {}", streamConfig.getTopicName(),
-          partitionId, offsetCriteria, fetcherException);
-      throw new RuntimeException(fetcherException);
-    }
-  }
-
-  private long getBetterStartOffsetIfNeeded(final String realtimeTableName, final int partition,
-      final LLCSegmentName latestSegment, final long oldestOffsetInStream, final int nextSeqNum) {
-    final LLCRealtimeSegmentZKMetadata oldSegMetadata =
-        getRealtimeSegmentZKMetadata(realtimeTableName, latestSegment.getSegmentName(), null);
-    CommonConstants.Segment.Realtime.Status status = oldSegMetadata.getStatus();
-    long segmentStartOffset = oldestOffsetInStream;
-    // Offset at which the prev segment intended to start consuming
-    final long prevSegStartOffset = oldSegMetadata.getStartOffset();
-    if (status.equals(CommonConstants.Segment.Realtime.Status.IN_PROGRESS)) {
-      if (oldestOffsetInStream <= prevSegStartOffset) {
-        // We still have the same start offset available, re-use it.
-        segmentStartOffset = prevSegStartOffset;
-        LOGGER.info("Choosing previous segment start offset {} for table {} for partition {}, sequence {}",
-            oldestOffsetInStream, realtimeTableName, partition, nextSeqNum);
-      } else {
-        // There is data loss.
-        LOGGER.warn("Data lost from offset {} to {} for table {} partition {} sequence {}", prevSegStartOffset,
-            oldestOffsetInStream, realtimeTableName, partition, nextSeqNum);
-        // Start from the earliest offset in the stream
-        _controllerMetrics.addMeteredTableValue(realtimeTableName, ControllerMeter.LLC_KAFKA_DATA_LOSS, 1);
-        _controllerMetrics.addMeteredTableValue(realtimeTableName, ControllerMeter.LLC_STREAM_DATA_LOSS, 1);
-      }
-    } else {
-      // Status must be DONE, so we have a valid end-offset for the previous segment
-      final long prevSegEndOffset = oldSegMetadata.getEndOffset();  // Will be 0 if the prev segment was not completed.
-      if (oldestOffsetInStream < prevSegEndOffset) {
-        // We don't want to create a segment that overlaps in data with the prev segment. We know that the previous
-        // segment's end offset is available, so use that.
-        segmentStartOffset = prevSegEndOffset;
-        LOGGER.info("Choosing newer offset {} for table {} for partition {}, sequence {}", oldestOffsetInStream,
-            realtimeTableName, partition, nextSeqNum);
-      } else if (oldestOffsetInStream > prevSegEndOffset) {
-        // Stream's oldest offset is greater than the end offset of the prev segment, so there is data loss.
-        LOGGER.warn("Data lost from offset {} to {} for table {} partition {} sequence {}", prevSegEndOffset,
-            oldestOffsetInStream, realtimeTableName, partition, nextSeqNum);
-        _controllerMetrics.addMeteredTableValue(realtimeTableName, ControllerMeter.LLC_KAFKA_DATA_LOSS, 1);
-        _controllerMetrics.addMeteredTableValue(realtimeTableName, ControllerMeter.LLC_STREAM_DATA_LOSS, 1);
-      } else {
-        // The two happen to be equal. A rarity, so log it.
-        LOGGER.info("Earliest offset {} is the same as new segment start offset", oldestOffsetInStream);
-      }
-    }
-    return segmentStartOffset;
-  }
-
-  private ZNRecord makeZnRecordForNewSegment(String realtimeTableName, int numReplicas, long startOffset,
-      LLCSegmentName newSegmentName, int numPartitions) {
-    final LLCRealtimeSegmentZKMetadata newSegMetadata = new LLCRealtimeSegmentZKMetadata();
-    newSegMetadata.setCreationTime(System.currentTimeMillis());
-    newSegMetadata.setStartOffset(startOffset);
-    newSegMetadata.setEndOffset(END_OFFSET_FOR_CONSUMING_SEGMENTS);
-    newSegMetadata.setNumReplicas(numReplicas);
-    newSegMetadata.setTableName(realtimeTableName);
-    newSegMetadata.setSegmentName(newSegmentName.getSegmentName());
-    newSegMetadata.setStatus(CommonConstants.Segment.Realtime.Status.IN_PROGRESS);
-
-    // Add the partition metadata if available.
-    SegmentPartitionMetadata partitionMetadata =
-        getPartitionMetadataFromTableConfig(realtimeTableName, numPartitions, newSegmentName.getPartitionId());
-    if (partitionMetadata != null) {
-      newSegMetadata.setPartitionMetadata(partitionMetadata);
+      throw new IllegalStateException(String
+          .format("Failed to fetch the offset for topic: %s, partition: %s with criteria: %s",
+              streamConfig.getTopicName(), partitionId, offsetCriteria));
     }
-
-    return newSegMetadata.toZNRecord();
   }
 
   /**
@@ -758,112 +590,63 @@ public class PinotLLCRealtimeSegmentManager {
    * When all replicas of this segment are marked offline, the {@link org.apache.pinot.controller.validation.RealtimeSegmentValidationManager},
    * in its next run, will auto-create a new segment with the appropriate offset.
    */
-  public void segmentStoppedConsuming(final LLCSegmentName segmentName, final String instance) {
-    String rawTableName = segmentName.getTableName();
-    String realtimeTableName = TableNameBuilder.REALTIME.tableNameWithType(rawTableName);
-    final String segmentNameStr = segmentName.getSegmentName();
+  public void segmentStoppedConsuming(LLCSegmentName llcSegmentName, String instanceName) {
+    Preconditions.checkState(!_isStopping, "Segment manager is stopping");
+
+    String realtimeTableName = TableNameBuilder.REALTIME.tableNameWithType(llcSegmentName.getTableName());
+    String segmentName = llcSegmentName.getSegmentName();
+    LOGGER.info("Marking CONSUMING segment: {} OFFLINE on instance: {}", segmentName, instanceName);
+
     try {
       HelixHelper.updateIdealState(_helixManager, realtimeTableName, idealState -> {
-        idealState.setPartitionState(segmentNameStr, instance,
-            CommonConstants.Helix.StateModel.SegmentOnlineOfflineStateModel.OFFLINE);
-        Map<String, String> instanceStateMap = idealState.getInstanceStateMap(segmentNameStr);
-        LOGGER.info("Attempting to mark {} offline. Current map:{}", segmentNameStr, instanceStateMap.toString());
+        assert idealState != null;
+        idealState.getInstanceStateMap(segmentName).put(instanceName, RealtimeSegmentOnlineOfflineStateModel.OFFLINE);
         return idealState;
       }, RetryPolicies.exponentialBackoffRetryPolicy(10, 500L, 1.2f));
     } catch (Exception e) {
-      LOGGER.error("Failed to update idealstate for table {} instance {} segment {}", realtimeTableName, instance,
-          segmentNameStr, e);
-      _controllerMetrics.addMeteredGlobalValue(ControllerMeter.LLC_ZOOKEEPER_UPDATE_FAILURES, 1);
+      _controllerMetrics.addMeteredTableValue(realtimeTableName, ControllerMeter.LLC_ZOOKEEPER_UPDATE_FAILURES, 1L);
       throw e;
     }
-    LOGGER.info("Successfully marked {} offline for instance {} since it stopped consuming", segmentNameStr, instance);
-  }
-
-  /**
-   * Helper method to trigger metadata event notifier
-   * @param tableName a table name
-   */
-  private void notifyOnSegmentFlush(String tableName) {
-    final MetadataEventNotifierFactory metadataEventNotifierFactory =
-        MetadataEventNotifierFactory.loadFactory(_controllerConf.subset(METADATA_EVENT_NOTIFIER_PREFIX));
-    final TableConfig tableConfig = getRealtimeTableConfig(tableName);
-    metadataEventNotifierFactory.create().notifyOnSegmentFlush(tableConfig);
-  }
-
-  protected int getPartitionCount(StreamConfig streamConfig) {
-    return PinotTableIdealStateBuilder.getPartitionCount(streamConfig);
-  }
-
-  /**
-   * Returns the LLC realtime segments for the given table.
-   *
-   * @param realtimeTableName Realtime table name
-   * @return List of LLC realtime segment names
-   */
-  @VisibleForTesting
-  protected List<String> getAllSegments(String realtimeTableName) {
-    return ZKMetadataProvider.getLLCRealtimeSegments(_propertyStore, realtimeTableName);
-  }
-
-  /**
-   * Returns the LLC realtime segment ZK metadata for the given table and segment.
-   *
-   * @param realtimeTableName Realtime table name
-   * @param segmentName Segment name (String)
-   * @return LLC realtime segment ZK metadata
-   */
-  @VisibleForTesting
-  protected LLCRealtimeSegmentZKMetadata getSegmentMetadata(String realtimeTableName, String segmentName) {
-    return (LLCRealtimeSegmentZKMetadata) ZKMetadataProvider
-        .getRealtimeSegmentZKMetadata(_propertyStore, realtimeTableName, segmentName);
   }
 
   /**
-   * Returns the latest 2 LLC realtime segment ZK metadata for each partition.
+   * Returns the latest LLC realtime segment ZK metadata for each partition.
    *
    * @param realtimeTableName Realtime table name
-   * @return Map from partition to array of latest LLC realtime segment ZK metadata
+   * @return Map from partition id to the latest LLC realtime segment ZK metadata
    */
-  @VisibleForTesting
-  protected Map<Integer, LLCRealtimeSegmentZKMetadata[]> getLatestMetadata(String realtimeTableName) {
-    List<String> llcRealtimeSegments = getAllSegments(realtimeTableName);
-
-    Map<Integer, LLCSegmentName[]> partitionToLatestSegmentsMap = new HashMap<>();
-    for (String llcRealtimeSegment : llcRealtimeSegments) {
-      LLCSegmentName segmentName = new LLCSegmentName(llcRealtimeSegment);
-      partitionToLatestSegmentsMap.compute(segmentName.getPartitionId(), (partitionId, segmentNames) -> {
-        if (segmentNames == null) {
-          return new LLCSegmentName[]{segmentName, null};
+  private Map<Integer, LLCRealtimeSegmentZKMetadata> getLatestSegmentZKMetadataMap(String realtimeTableName) {
+    List<String> segments = getLLCSegments(realtimeTableName);
+
+    Map<Integer, LLCSegmentName> latestLLCSegmentNameMap = new HashMap<>();
+    for (String segmentName : segments) {
+      LLCSegmentName llcSegmentName = new LLCSegmentName(segmentName);
+      latestLLCSegmentNameMap.compute(llcSegmentName.getPartitionId(), (partitionId, latestLLCSegmentName) -> {
+        if (latestLLCSegmentName == null) {
+          return llcSegmentName;
         } else {
-          if (segmentName.getSequenceNumber() > segmentNames[0].getSequenceNumber()) {
-            segmentNames[1] = segmentNames[0];
-            segmentNames[0] = segmentName;
-          } else if (segmentNames[1] == null || segmentName.getSequenceNumber() > segmentNames[1].getSequenceNumber()) {
-            segmentNames[1] = segmentName;
+          if (llcSegmentName.getSequenceNumber() > latestLLCSegmentName.getSequenceNumber()) {
+            return llcSegmentName;
+          } else {
+            return latestLLCSegmentName;
           }
-          return segmentNames;
         }
       });
     }
 
-    Map<Integer, LLCRealtimeSegmentZKMetadata[]> partitionToLatestMetadataMap = new HashMap<>();
-    for (Map.Entry<Integer, LLCSegmentName[]> entry : partitionToLatestSegmentsMap.entrySet()) {
-      LLCSegmentName[] latestSegments = entry.getValue();
-      LLCRealtimeSegmentZKMetadata latestMetadata =
-          getSegmentMetadata(realtimeTableName, latestSegments[0].getSegmentName());
-      LLCRealtimeSegmentZKMetadata secondLatestMetadata = null;
-      if (latestSegments[1] != null) {
-        secondLatestMetadata = getSegmentMetadata(realtimeTableName, latestSegments[1].getSegmentName());
-      }
-      partitionToLatestMetadataMap
-          .put(entry.getKey(), new LLCRealtimeSegmentZKMetadata[]{latestMetadata, secondLatestMetadata});
+    Map<Integer, LLCRealtimeSegmentZKMetadata> latestSegmentZKMetadataMap = new HashMap<>();
+    for (Map.Entry<Integer, LLCSegmentName> entry : latestLLCSegmentNameMap.entrySet()) {
+      LLCRealtimeSegmentZKMetadata latestSegmentZKMetadata =
+          getSegmentZKMetadata(realtimeTableName, entry.getValue().getSegmentName());
+      latestSegmentZKMetadataMap.put(entry.getKey(), latestSegmentZKMetadata);
     }
 
-    return partitionToLatestMetadataMap;
+    return latestSegmentZKMetadataMap;
   }
 
   /**
-   * Validates llc segments in ideal state and repairs them if necessary
+   * Validates LLC segments in ideal state and repairs them if necessary. This method should only be called from the
+   * leader of the table.
    *
    * During segment commit, we update zookeeper in 3 steps
    * Step 1: Update PROPERTYSTORE to change the old segment metadata status to DONE
@@ -887,87 +670,68 @@ public class PinotLLCRealtimeSegmentManager {
    *
    * TODO: We need to find a place to detect and update a gauge for nonConsumingPartitionsCount for a table, and reset it to 0 at the end of validateLLC
    */
-  public void ensureAllPartitionsConsuming(final TableConfig tableConfig) {
-    final String tableNameWithType = tableConfig.getTableName();
-    final StreamConfig streamConfig = new StreamConfig(tableNameWithType, tableConfig.getIndexingConfig().getStreamConfigs());
-    final int partitionCount = getPartitionCount(streamConfig);
-    HelixHelper.updateIdealState(_helixManager, tableNameWithType, new Function<IdealState, IdealState>() {
-      @Nullable
-      @Override
-      public IdealState apply(@Nullable IdealState idealState) {
-        return ensureAllPartitionsConsuming(tableConfig, idealState, partitionCount);
+  public void ensureAllPartitionsConsuming(TableConfig tableConfig,
+      PartitionLevelStreamConfig partitionLevelStreamConfig) {
+    Preconditions.checkState(!_isStopping, "Segment manager is stopping");
+
+    String realtimeTableName = tableConfig.getTableName();
+    int numPartitions = getNumPartitions(partitionLevelStreamConfig);
+    HelixHelper.updateIdealState(_helixManager, realtimeTableName, idealState -> {
+      assert idealState != null;
+      if (idealState.isEnabled()) {
+        return ensureAllPartitionsConsuming(tableConfig, partitionLevelStreamConfig, idealState, numPartitions);
+      } else {
+        LOGGER.info("Skipping LLC segments validation for disabled table: {}", realtimeTableName);
+        return idealState;
       }
     }, RetryPolicies.exponentialBackoffRetryPolicy(10, 1000L, 1.2f), true);
   }
 
   /**
    * Updates ideal state after completion of a realtime segment
-   * @param tableNameWithType
-   * @param currentSegmentId
-   * @param newSegmentId
-   * @param partitionAssignment
    */
   @VisibleForTesting
-  protected void updateIdealStateOnSegmentCompletion(@Nonnull final String tableNameWithType,
-      @Nonnull final String currentSegmentId, @Nonnull final String newSegmentId,
-      @Nonnull final PartitionAssignment partitionAssignment) {
-
-    HelixHelper.updateIdealState(_helixManager, tableNameWithType, new Function<IdealState, IdealState>() {
-      @Nullable
-      @Override
-      public IdealState apply(@Nullable IdealState idealState) {
-        // When segment completion begins, the zk metadata is updated, followed by ideal state.
-        // We allow only {@link PinotLLCRealtimeSegmentManager::MAX_SEGMENT_COMPLETION_TIME_MILLIS} ms for a segment to complete,
-        // after which the segment is eligible for repairs by the {@link org.apache.pinot.controller.validation.RealtimeSegmentValidationManager}
-        // After updating metadata, if more than {@link PinotLLCRealtimeSegmentManager::MAX_SEGMENT_COMPLETION_TIME_MILLIS} ms elapse and ideal state is still not updated,
-        // the segment could have already been fixed by {@link org.apache.pinot.controller.validation.RealtimeSegmentValidationManager}
-        // Therefore, we do not want to proceed with ideal state update if max segment completion time has exceeded
-        if (isExceededMaxSegmentCompletionTime(tableNameWithType, currentSegmentId, System.currentTimeMillis())) {
-          LOGGER.error("Exceeded max segment completion time. Skipping ideal state update for segment {}",
-              currentSegmentId);
-          throw new HelixHelper.PermanentUpdaterException(
-              "Exceeded max segment completion time for segment " + currentSegmentId);
-        }
-        return updateIdealStateOnSegmentCompletion(idealState, currentSegmentId, newSegmentId, partitionAssignment);
+  void updateIdealStateOnSegmentCompletion(String realtimeTableName, String committingSegmentName,
+      String newSegmentName, SegmentAssignment segmentAssignment,
+      Map<InstancePartitionsType, InstancePartitions> instancePartitionsMap) {
+    HelixHelper.updateIdealState(_helixManager, realtimeTableName, idealState -> {
+      assert idealState != null;
+      // When segment completion begins, the zk metadata is updated, followed by ideal state.
+      // We allow only {@link PinotLLCRealtimeSegmentManager::MAX_SEGMENT_COMPLETION_TIME_MILLIS} ms for a segment to complete,
+      // after which the segment is eligible for repairs by the {@link org.apache.pinot.controller.validation.RealtimeSegmentValidationManager}
+      // After updating metadata, if more than {@link PinotLLCRealtimeSegmentManager::MAX_SEGMENT_COMPLETION_TIME_MILLIS} ms elapse and ideal state is still not updated,
+      // the segment could have already been fixed by {@link org.apache.pinot.controller.validation.RealtimeSegmentValidationManager}
+      // Therefore, we do not want to proceed with ideal state update if max segment completion time has exceeded
+      if (isExceededMaxSegmentCompletionTime(realtimeTableName, committingSegmentName, getCurrentTimeMs())) {
+        LOGGER.error("Exceeded max segment completion time. Skipping ideal state update for segment: {}",
+            committingSegmentName);
+        throw new HelixHelper.PermanentUpdaterException(
+            "Exceeded max segment completion time for segment " + committingSegmentName);
       }
+      updateInstanceStatesForNewConsumingSegment(idealState.getRecord().getMapFields(), committingSegmentName,
+          newSegmentName, segmentAssignment, instancePartitionsMap);
+      return idealState;
     }, RetryPolicies.exponentialBackoffRetryPolicy(10, 1000L, 1.2f));
   }
 
-  /**
-   * Sets up a new table's segments metadata and returns the ideal state setup with initial segments
-   * @param tableConfig
-   * @param idealState
-   * @param partitionCount
-   * @return
-   */
-  private IdealState setupTable(TableConfig tableConfig, IdealState idealState, int partitionCount)
-      throws InvalidConfigException {
-    final String tableNameWithType = tableConfig.getTableName();
-    if (!idealState.isEnabled()) {
-      LOGGER.info("Skipping validation for disabled table {}", tableNameWithType);
-      return idealState;
-    }
-    final StreamConfig streamConfig = new StreamConfig(tableConfig.getTableName(),
-        tableConfig.getIndexingConfig().getStreamConfigs());
-    final long now = getCurrentTimeMs();
-
-    PartitionAssignment partitionAssignment =
-        _streamPartitionAssignmentGenerator.generateStreamPartitionAssignment(tableConfig, partitionCount);
-
-    Set<Integer> newPartitions = new HashSet<>(partitionCount);
-    for (int partition = 0; partition < partitionCount; partition++) {
-      newPartitions.add(partition);
+  @VisibleForTesting
+  void updateInstanceStatesForNewConsumingSegment(Map<String, Map<String, String>> instanceStatesMap,
+      @Nullable String committingSegmentName, String newSegmentName, SegmentAssignment segmentAssignment,
+      Map<InstancePartitionsType, InstancePartitions> instancePartitionsMap) {
+    if (committingSegmentName != null) {
+      // Change committing segment state to ONLINE
+      Set<String> instances = instanceStatesMap.get(committingSegmentName).keySet();
+      instanceStatesMap.put(committingSegmentName,
+          SegmentAssignmentUtils.getInstanceStateMap(instances, RealtimeSegmentOnlineOfflineStateModel.ONLINE));
+      LOGGER.info("Updating segment: {} to ONLINE state", committingSegmentName);
     }
 
-    OffsetCriteria offsetCriteria = streamConfig.getOffsetCriteria();
-    Set<String> consumingSegments =
-        setupNewPartitions(tableConfig, streamConfig, offsetCriteria, partitionAssignment, newPartitions, now);
-
-    RealtimeSegmentAssignmentStrategy segmentAssignmentStrategy = new ConsumingSegmentAssignmentStrategy();
-    Map<String, List<String>> assignments = segmentAssignmentStrategy.assign(consumingSegments, partitionAssignment);
-
-    updateIdealState(idealState, null, consumingSegments, assignments);
-    return idealState;
+    // Assign instances to the new segment and add instances as state CONSUMING
+    List<String> instancesAssigned =
+        segmentAssignment.assignSegment(newSegmentName, instanceStatesMap, instancePartitionsMap);
+    instanceStatesMap.put(newSegmentName, SegmentAssignmentUtils
+        .getInstanceStateMap(instancesAssigned, RealtimeSegmentOnlineOfflineStateModel.CONSUMING));
+    LOGGER.info("Adding new CONSUMING segment: {} to instances: {}", newSegmentName, instancesAssigned);
   }
 
   /*
@@ -989,37 +753,24 @@ public class PinotLLCRealtimeSegmentManager {
    * We check the segment's metadata to see if that is old enough for repair. If it is fairly new, we
    * leave it as it is, to be fixed the next time repair job triggers.
    */
-  @VisibleForTesting
-  protected boolean isTooSoonToCorrect(String tableNameWithType, String segmentId, long now) {
-    Stat stat = new Stat();
-    LLCRealtimeSegmentZKMetadata metadata = getRealtimeSegmentZKMetadata(tableNameWithType, segmentId, stat);
-    long metadataUpdateTime = stat.getMtime();
-    if (now < metadataUpdateTime + MAX_SEGMENT_COMPLETION_TIME_MILLIS) {
-      LOGGER.info("Too soon to correct segment:{} updateTime: {} now:{}", segmentId, metadataUpdateTime, now);
-      return true;
-    }
-    return false;
-  }
 
   /**
-   *
    * Returns true if more than {@link PinotLLCRealtimeSegmentManager::MAX_SEGMENT_COMPLETION_TIME_MILLIS} ms have elapsed since segment metadata update
    */
-  private boolean isExceededMaxSegmentCompletionTime(String tableNameWithType, String segmentId, long now) {
+  @VisibleForTesting
+  boolean isExceededMaxSegmentCompletionTime(String realtimeTableName, String segmentName, long currentTimeMs) {
     Stat stat = new Stat();
-    LLCRealtimeSegmentZKMetadata metadata = getRealtimeSegmentZKMetadata(tableNameWithType, segmentId, stat);
-    long metadataUpdateTime = stat.getMtime();
-    if (now > metadataUpdateTime + MAX_SEGMENT_COMPLETION_TIME_MILLIS) {
-      LOGGER
-          .info("Segment:{}, Now:{}, metadataUpdateTime:{}, Exceeded MAX_SEGMENT_COMPLETION_TIME_MILLIS:{}", segmentId,
-              now, metadataUpdateTime, MAX_SEGMENT_COMPLETION_TIME_MILLIS);
-      return true;
-    }
-    return false;
+    getSegmentZKMetadata(realtimeTableName, segmentName, stat);
+    return currentTimeMs > stat.getMtime() + MAX_SEGMENT_COMPLETION_TIME_MILLIS;
   }
 
   private boolean isAllInstancesInState(Map<String, String> instanceStateMap, String state) {
-    return instanceStateMap.values().stream().allMatch(value -> value.equals(state));
+    for (String value : instanceStateMap.values()) {
+      if (!value.equals(state)) {
+        return false;
+      }
+    }
+    return true;
   }
 
   /*
@@ -1045,47 +796,23 @@ public class PinotLLCRealtimeSegmentManager {
    * TODO: split this method into multiple smaller methods
    */
   @VisibleForTesting
-  protected IdealState ensureAllPartitionsConsuming(final TableConfig tableConfig, IdealState idealState,
-      final int partitionCount) {
-    final String tableNameWithType = tableConfig.getTableName();
-    final StreamConfig streamConfig = new StreamConfig(tableNameWithType, tableConfig.getIndexingConfig().getStreamConfigs());
-    if (!idealState.isEnabled()) {
-      LOGGER.info("Skipping validation for disabled table {}", tableNameWithType);
-      return idealState;
-    }
-    final long now = getCurrentTimeMs();
+  IdealState ensureAllPartitionsConsuming(TableConfig tableConfig,
+      PartitionLevelStreamConfig partitionLevelStreamConfig, IdealState idealState, int numPartitions) {
+    String realtimeTableName = tableConfig.getTableName();
 
-    // Get the metadata for the latest 2 segments of each partition
-    Map<Integer, LLCRealtimeSegmentZKMetadata[]> partitionToLatestMetadata = getLatestMetadata(tableNameWithType);
+    InstancePartitions instancePartitions = getConsumingInstancePartitions(tableConfig);
+    int numReplicas = getNumReplicas(tableConfig, instancePartitions);
 
-    // Find partitions for which there is no metadata at all. These are new partitions that we need to start consuming.
-    Set<Integer> newPartitions = new HashSet<>(partitionCount);
-    for (int partition = 0; partition < partitionCount; partition++) {
-      if (!partitionToLatestMetadata.containsKey(partition)) {
-        LOGGER.info("Found partition {} with no segments", partition);
-        newPartitions.add(partition);
-      }
-    }
+    SegmentAssignment segmentAssignment = SegmentAssignmentFactory.getSegmentAssignment(_helixManager, tableConfig);
+    Map<InstancePartitionsType, InstancePartitions> instancePartitionsMap =
+        Collections.singletonMap(InstancePartitionsType.CONSUMING, instancePartitions);
 
-    PartitionAssignment partitionAssignment;
-    boolean skipNewPartitions = false;
-    try {
-      partitionAssignment =
-          _streamPartitionAssignmentGenerator.generateStreamPartitionAssignment(tableConfig, partitionCount);
-    } catch (InvalidConfigException e) {
-      _controllerMetrics
-          .addMeteredTableValue(tableNameWithType, ControllerMeter.PARTITION_ASSIGNMENT_GENERATION_ERROR, 1L);
-      LOGGER.warn(
-          "Could not generate partition assignment. Fetching partition assignment from ideal state for repair of table {}",
-          tableNameWithType);
-      partitionAssignment =
-          _streamPartitionAssignmentGenerator.getStreamPartitionAssignmentFromIdealState(tableConfig, idealState);
-      skipNewPartitions = true;
-    }
+    Map<String, Map<String, String>> instanceStatesMap = idealState.getRecord().getMapFields();
+    long currentTimeMs = getCurrentTimeMs();
 
-    Set<String> onlineSegments = new HashSet<>(); // collect all segment names which should be updated to ONLINE state
-    Set<String> consumingSegments =
-        new HashSet<>(); // collect all segment names which should be created in CONSUMING state
+    // Get the latest segment ZK metadata for each partition
+    Map<Integer, LLCRealtimeSegmentZKMetadata> latestSegmentZKMetadataMap =
+        getLatestSegmentZKMetadataMap(realtimeTableName);
 
     // Walk over all partitions that we have metadata for, and repair any partitions necessary.
     // Possible things to repair:
@@ -1100,46 +827,35 @@ public class PinotLLCRealtimeSegmentManager {
     //    a. Create a new segment (with the next seq number)
     //       and restart consumption from the same offset (if possible) or a newer offset (if realtime stream does not have the same offset).
     //       In latter case, report data loss.
-    for (Map.Entry<Integer, LLCRealtimeSegmentZKMetadata[]> entry : partitionToLatestMetadata.entrySet()) {
-      int partition = entry.getKey();
-      LLCRealtimeSegmentZKMetadata[] latestMetadataArray = entry.getValue();
-      LLCRealtimeSegmentZKMetadata latestMetadata = latestMetadataArray[0];
-      final String segmentId = latestMetadata.getSegmentName();
-      final LLCSegmentName segmentName = new LLCSegmentName(segmentId);
-
-      Map<String, Map<String, String>> mapFields = idealState.getRecord().getMapFields();
-      if (mapFields.containsKey(segmentId)) {
+    for (Map.Entry<Integer, LLCRealtimeSegmentZKMetadata> entry : latestSegmentZKMetadataMap.entrySet()) {
+      int partitionId = entry.getKey();
+      LLCRealtimeSegmentZKMetadata latestSegmentZKMetadata = entry.getValue();
+      String latestSegmentName = latestSegmentZKMetadata.getSegmentName();
+      LLCSegmentName latestLLCSegmentName = new LLCSegmentName(latestSegmentName);
+
+      Map<String, String> instanceStateMap = instanceStatesMap.get(latestSegmentName);
+      if (instanceStateMap != null) {
         // Latest segment of metadata is in idealstate.
-        Map<String, String> instanceStateMap = idealState.getInstanceStateMap(segmentId);
-        if (instanceStateMap.values().contains(PinotHelixSegmentOnlineOfflineStateModelGenerator.CONSUMING_STATE)) {
-          if (latestMetadata.getStatus().equals(CommonConstants.Segment.Realtime.Status.DONE)) {
+        if (instanceStateMap.values().contains(RealtimeSegmentOnlineOfflineStateModel.CONSUMING)) {
+          if (latestSegmentZKMetadata.getStatus() == Status.DONE) {
 
             // step-1 of commmitSegmentMetadata is done (i.e. marking old segment as DONE)
             // but step-2 is not done (i.e. adding new metadata for the next segment)
             // and ideal state update (i.e. marking old segment as ONLINE and new segment as CONSUMING) is not done either.
-            if (isTooSoonToCorrect(tableNameWithType, segmentId, now)) {
+            if (!isExceededMaxSegmentCompletionTime(realtimeTableName, latestSegmentName, currentTimeMs)) {
               continue;
             }
-            LOGGER.info("{}:Repairing segment for partition {}. "
-                    + "Old segment metadata {} has status DONE, but segments are still in CONSUMING state in ideal STATE",
-                tableNameWithType, partition, segmentId);
-
-            LLCSegmentName newLLCSegmentName = makeNextLLCSegmentName(segmentName, partition, now);
-            LOGGER.info("{}: Creating new segment metadata for {}", tableNameWithType,
-                newLLCSegmentName.getSegmentName());
+            LOGGER.info("Repairing segment: {} which is DONE in segment ZK metadata, but is CONSUMING in IdealState",
+                latestSegmentName);
 
+            LLCSegmentName newLLCSegmentName = getNextLLCSegmentName(latestLLCSegmentName, currentTimeMs);
+            String newSegmentName = newLLCSegmentName.getSegmentName();
             CommittingSegmentDescriptor committingSegmentDescriptor =
-                new CommittingSegmentDescriptor(segmentId, latestMetadata.getEndOffset(), 0);
-            boolean success =
-                createNewSegmentMetadataZNRecord(tableConfig, segmentName, newLLCSegmentName, partitionAssignment,
-                    committingSegmentDescriptor, false);
-
-            // creation of segment metadata could fail due to lost leadership or an unsuccessful write to property store
-            // in such a case, we will exclude the segment from ideal state update and let the next iteration of validation manager fix it
-            if (success) {
-              onlineSegments.add(segmentId);
-              consumingSegments.add(newLLCSegmentName.getSegmentName());
-            }
+                new CommittingSegmentDescriptor(latestSegmentName, latestSegmentZKMetadata.getEndOffset(), 0);
+            createNewSegmentZKMetadata(tableConfig, partitionLevelStreamConfig, newLLCSegmentName, currentTimeMs,
+                committingSegmentDescriptor, latestSegmentZKMetadata, instancePartitions, numPartitions, numReplicas);
+            updateInstanceStatesForNewConsumingSegment(instanceStatesMap, latestSegmentName, newSegmentName,
+                segmentAssignment, instancePartitionsMap);
           }
           // else, the metadata should be IN_PROGRESS, which is the right state for a consuming segment.
         } else { // no replica in CONSUMING state
@@ -1148,34 +864,32 @@ public class PinotLLCRealtimeSegmentManager {
           // 1. all replicas OFFLINE and metadata IN_PROGRESS/DONE - a segment marked itself OFFLINE during consumption for some reason
           // 2. all replicas ONLINE and metadata DONE - Resolved in https://github.com/linkedin/pinot/pull/2890
           // 3. we should never end up with some replicas ONLINE and some OFFLINE.
-          if (isAllInstancesInState(instanceStateMap, PinotHelixSegmentOnlineOfflineStateModelGenerator.OFFLINE_STATE)
-              || !(isTooSoonToCorrect(tableNameWithType, segmentId, now))) {
-
-            // No instances are consuming, so create a new consuming segment.
-            LLCSegmentName newLLCSegmentName = makeNextLLCSegmentName(segmentName, partition, now);
-            LOGGER.info("Creating CONSUMING segment {} for {} partition {}", newLLCSegmentName.getSegmentName(),
-                tableNameWithType, partition);
-
-            // To begin with, set startOffset to the oldest available offset in the stream. Fix it to be the one we want,
-            // depending on what the prev segment had.
-            long startOffset = getPartitionOffset(streamConfig, OffsetCriteria.SMALLEST_OFFSET_CRITERIA, partition);
-            LOGGER.info("Found smallest offset {} for table {} for partition {}", startOffset, tableNameWithType,
-                partition);
-            startOffset = getBetterStartOffsetIfNeeded(tableNameWithType, partition, segmentName, startOffset,
-                newLLCSegmentName.getSequenceNumber());
+          if (isAllInstancesInState(instanceStateMap, RealtimeSegmentOnlineOfflineStateModel.OFFLINE)) {
+            LOGGER.info("Repairing segment: {} which is OFFLINE for all instances in IdealState", latestSegmentName);
+
+            // Create a new segment to re-consume from the previous start offset
+            LLCSegmentName newLLCSegmentName = getNextLLCSegmentName(latestLLCSegmentName, currentTimeMs);
+            long startOffset = latestSegmentZKMetadata.getStartOffset();
+            // Start offset must be higher than the start offset of the stream
+            long partitionStartOffset =
+                getPartitionOffset(partitionLevelStreamConfig, OffsetCriteria.SMALLEST_OFFSET_CRITERIA, partitionId);
+            if (partitionStartOffset > startOffset) {
+              LOGGER.warn("Data lost from offset: {} to: {} for partition: {} of table: {}", startOffset,
+                  partitionStartOffset, partitionId, realtimeTableName);
+              _controllerMetrics.addMeteredTableValue(realtimeTableName, ControllerMeter.LLC_KAFKA_DATA_LOSS, 1L);
+              _controllerMetrics.addMeteredTableValue(realtimeTableName, ControllerMeter.LLC_STREAM_DATA_LOSS, 1L);
+              startOffset = partitionStartOffset;
+            }
 
             CommittingSegmentDescriptor committingSegmentDescriptor =
-                new CommittingSegmentDescriptor(segmentId, startOffset, 0);
-
-            boolean success =
-                createNewSegmentMetadataZNRecord(tableConfig, segmentName, newLLCSegmentName, partitionAssignment,
-                    committingSegmentDescriptor, false);
-
-            // creation of segment metadata could fail due to lost leadership or an unsuccessful write to property store
-            // in such a case, we will exclude the segment from ideal state update and let the next iteration of validation manager fix it
-            if (success) {
-              consumingSegments.add(newLLCSegmentName.getSegmentName());
-            }
+                new CommittingSegmentDescriptor(latestSegmentName, startOffset, 0);
+            createNewSegmentZKMetadata(tableConfig, partitionLevelStreamConfig, newLLCSegmentName, currentTimeMs,
+                committingSegmentDescriptor, latestSegmentZKMetadata, instancePartitions, numPartitions, numReplicas);
+            String newSegmentName = newLLCSegmentName.getSegmentName();
+            updateInstanceStatesForNewConsumingSegment(instanceStatesMap, null, newSegmentName, segmentAssignment,
+                instancePartitionsMap);
+          } else {
+            LOGGER.error("Got unexpected instance state map: {} for segment: {}", instanceStateMap, latestSegmentName);
           }
         }
       } else {
@@ -1184,168 +898,127 @@ public class PinotLLCRealtimeSegmentManager {
         // i.e. after updating old segment metadata (old segment metadata state = DONE)
         // and creating new segment metadata (new segment metadata state = IN_PROGRESS),
         // but before updating ideal state (new segment ideal missing from ideal state)
-        if (isTooSoonToCorrect(tableNameWithType, segmentId, now)) {
-          continue;
-        }
-
-        Preconditions
-            .checkArgument(latestMetadata.getStatus().equals(CommonConstants.Segment.Realtime.Status.IN_PROGRESS));
-        LOGGER.info("{}:Repairing segment for partition {}. Segment {} not found in idealstate", tableNameWithType,
-            partition, segmentId);
-
-        // If there was a prev segment in the same partition, then we need to fix it to be ONLINE.
-        LLCRealtimeSegmentZKMetadata secondLatestMetadata = latestMetadataArray[1];
-        if (secondLatestMetadata == null && skipNewPartitions) {
+        if (!isExceededMaxSegmentCompletionTime(realtimeTableName, latestSegmentName, currentTimeMs)) {
           continue;
         }
-        if (secondLatestMetadata != null) {
-          onlineSegments.add(secondLatestMetadata.getSegmentName());
+        LOGGER.info("Repairing segment: {} which has segment ZK metadata but does not exist in IdealState",
+            latestSegmentName);
+
+        if (latestSegmentZKMetadata.getStatus() == Status.IN_PROGRESS) {
+          // Find the previous CONSUMING segment
+          String previousConsumingSegment = null;
+          for (Map.Entry<String, Map<String, String>> segmentEntry : instanceStatesMap.entrySet()) {
+            LLCSegmentName llcSegmentName = new LLCSegmentName(segmentEntry.getKey());
+            if (llcSegmentName.getPartitionId() == partitionId && segmentEntry.getValue()
+                .containsValue(RealtimeSegmentOnlineOfflineStateModel.CONSUMING)) {
+              previousConsumingSegment = llcSegmentName.getSegmentName();
+              break;
+            }
+          }
+          if (previousConsumingSegment == null) {
+            LOGGER
+                .error("Failed to find previous CONSUMING segment for partition: {} of table: {}, potential data loss",
+                    partitionId, realtimeTableName);
+            _controllerMetrics.addMeteredTableValue(realtimeTableName, ControllerMeter.LLC_KAFKA_DATA_LOSS, 1L);
+            _controllerMetrics.addMeteredTableValue(realtimeTableName, ControllerMeter.LLC_STREAM_DATA_LOSS, 1L);
+          }
+          updateInstanceStatesForNewConsumingSegment(instanceStatesMap, previousConsumingSegment, latestSegmentName,
+              segmentAssignment, instancePartitionsMap);
+        } else {
+          LOGGER.error("Got unexpected status: {} in segment ZK metadata for segment: {}",
+              latestSegmentZKMetadata.getStatus(), latestSegmentName);
         }
-        consumingSegments.add(segmentId);
       }
     }
 
-    if (!skipNewPartitions) {
-      Set<String> newPartitionSegments =
-          setupNewPartitions(tableConfig, streamConfig, OffsetCriteria.SMALLEST_OFFSET_CRITERIA, partitionAssignment,
-              newPartitions, now);
-      consumingSegments.addAll(newPartitionSegments);
-    }
-
-    RealtimeSegmentAssignmentStrategy segmentAssignmentStrategy = new ConsumingSegmentAssignmentStrategy();
-    Map<String, List<String>> assignments;
-    try {
-      assignments = segmentAssignmentStrategy.assign(consumingSegments, partitionAssignment);
-    } catch (InvalidConfigException e) {
-      throw new IllegalStateException(
-          "Caught exception when assigning segments using partition assignment for table " + tableNameWithType);
+    // Set up new partitions if not exist
+    for (int partitionId = 0; partitionId < numPartitions; partitionId++) {
+      if (!latestSegmentZKMetadataMap.containsKey(partitionId)) {
+        String newSegmentName =
+            setupNewPartition(tableConfig, partitionLevelStreamConfig, partitionId, currentTimeMs, instancePartitions,
+                numPartitions, numReplicas);
+        updateInstanceStatesForNewConsumingSegment(instanceStatesMap, null, newSegmentName, segmentAssignment,
+            instancePartitionsMap);
+      }
     }
 
-    updateIdealState(idealState, onlineSegments, consumingSegments, assignments);
     return idealState;
   }
 
-  private LLCSegmentName makeNextLLCSegmentName(LLCSegmentName segmentName, int partition, long now) {
-    final int newSeqNum = segmentName.getSequenceNumber() + 1;
-    LLCSegmentName newLLCSegmentName = new LLCSegmentName(segmentName.getTableName(), partition, newSeqNum, now);
-    return newLLCSegmentName;
-  }
-
-  /**
-   * Updates the ideal state object
-   * Adds the segments in consumingSegments to CONSUMING state using instances from assignments
-   * Sets the segments in onlineSegments to ONLINE state
-   * @param idealState
-   * @param consumingSegments
-   * @param onlineSegments
-   * @param assignments
-   */
-  private void updateIdealState(IdealState idealState, Set<String> onlineSegments, Set<String> consumingSegments,
-      Map<String, List<String>> assignments) {
-    if (onlineSegments != null) {
-      for (String segment : onlineSegments) {
-        Set<String> oldInstances = idealState.getInstanceSet(segment);
-        Preconditions.checkArgument(CollectionUtils.isNotEmpty(oldInstances));
-        for (String instance : oldInstances) {
-          idealState
-              .setPartitionState(segment, instance, PinotHelixSegmentOnlineOfflineStateModelGenerator.ONLINE_STATE);
-        }
-      }
-    }
-
-    if (consumingSegments != null) {
-      for (String segment : consumingSegments) {
-        List<String> newInstances = assignments.get(segment);
-        Map<String, String> instanceStateMap = idealState.getInstanceStateMap(segment);
-        if (instanceStateMap != null) {
-          instanceStateMap.clear();
-        }
-        for (String instance : newInstances) {
-          idealState
-              .setPartitionState(segment, instance, PinotHelixSegmentOnlineOfflineStateModelGenerator.CONSUMING_STATE);
-        }
-      }
-    }
+  private LLCSegmentName getNextLLCSegmentName(LLCSegmentName lastLLCSegmentName, long creationTimeMs) {
+    return new LLCSegmentName(lastLLCSegmentName.getTableName(), lastLLCSegmentName.getPartitionId(),
+        lastLLCSegmentName.getSequenceNumber() + 1, creationTimeMs);
   }
 
   /**
-   * Create metadata for new partitions
-   * @param tableConfig  the table configuration to use for the new partition
-   * @param streamConfig stream configuration associated with the table
-   * @param offsetCriteria the offset to query to start consumption from. Can be different for a
-   *                       new table being setup vs new partitions being added to an existing table
-   * @param partitionAssignment the partition assignment strategy to use
-   * @param newPartitions the new partitions to set up
-   * @param now the current timestamp in milliseconds
-   * @return set of newly created segment names
+   * Sets up a new partition.
+   * <p>Persists the ZK metadata for the first CONSUMING segment, and returns the segment name.
    */
-  private Set<String> setupNewPartitions(TableConfig tableConfig, StreamConfig streamConfig,
-      OffsetCriteria offsetCriteria, PartitionAssignment partitionAssignment, Set<Integer> newPartitions, long now) {
-
-    String tableName = tableConfig.getTableName();
-    Set<String> newSegmentNames = new HashSet<>(newPartitions.size());
-    String rawTableName = TableNameBuilder.extractRawTableName(tableName);
-    int nextSeqNum = STARTING_SEQUENCE_NUMBER;
-
-    for (int partition : newPartitions) {
-      LOGGER.info("Creating CONSUMING segment for {} partition {} with seq {}", tableName, partition, nextSeqNum);
-      long startOffset = getPartitionOffset(streamConfig, offsetCriteria, partition);
-
-      LOGGER.info("Found offset {} for table {} for partition {}", startOffset, tableName, partition);
+  private String setupNewPartition(TableConfig tableConfig, PartitionLevelStreamConfig partitionLevelStreamConfig,
+      int partitionId, long creationTimeMs, InstancePartitions instancePartitions, int numPartitions, int numReplicas) {
+    String realtimeTableName = tableConfig.getTableName();
+    LOGGER.info("Setting up new partition: {} for table: {}", partitionId, realtimeTableName);
 
-      LLCSegmentName newLLCSegmentName = new LLCSegmentName(rawTableName, partition, nextSeqNum, now);
-      CommittingSegmentDescriptor committingSegmentDescriptor = new CommittingSegmentDescriptor(null, startOffset, 0);
-
-      boolean success = createNewSegmentMetadataZNRecord(tableConfig, null, newLLCSegmentName, partitionAssignment,
-          committingSegmentDescriptor, true);
-      // creation of segment metadata could fail due to an unsuccessful write to property store
-      // in such a case, we will exclude the segment from ideal state update and let the validation manager fix it
-      if (success) {
-        newSegmentNames.add(newLLCSegmentName.getSegmentName());
-      }
-    }
-    return newSegmentNames;
+    String rawTableName = TableNameBuilder.extractRawTableName(realtimeTableName);
+    LLCSegmentName newLLCSegmentName =
+        new LLCSegmentName(rawTableName, partitionId, STARTING_SEQUENCE_NUMBER, creationTimeMs);
+    String newSegmentName = newLLCSegmentName.getSegmentName();
+    long startOffset =
+        getPartitionOffset(partitionLevelStreamConfig, partitionLevelStreamConfig.getOffsetCriteria(), partitionId);
+    CommittingSegmentDescriptor committingSegmentDescriptor = new CommittingSegmentDescriptor(null, startOffset, 0);
+    createNewSegmentZKMetadata(tableConfig, partitionLevelStreamConfig, newLLCSegmentName, creationTimeMs,
+        committingSegmentDescriptor, null, instancePartitions, numPartitions, numReplicas);
+
+    return newSegmentName;
   }
 
   @VisibleForTesting
-  protected long getCurrentTimeMs() {
+  long getCurrentTimeMs() {
     return System.currentTimeMillis();
   }
 
-  protected IdealState updateIdealStateOnSegmentCompletion(@Nonnull IdealState idealState,
-      @Nonnull String currentSegmentId, @Nonnull String newSegmentId,
-      @Nonnull PartitionAssignment partitionAssignment) {
-
-    Map<String, List<String>> instanceAssignments;
-
-    RealtimeSegmentAssignmentStrategy strategy = new ConsumingSegmentAssignmentStrategy();
-    try {
-      instanceAssignments = strategy.assign(Lists.newArrayList(newSegmentId), partitionAssignment);
-    } catch (InvalidConfigException e) {
-      _controllerMetrics.addMeteredTableValue(idealState.getResourceName(),
-          ControllerMeter.CONTROLLER_REALTIME_TABLE_SEGMENT_ASSIGNMENT_ERROR, 1L);
-      throw new IllegalStateException("Caught exception when updating ideal state on segment completion", e);
+  private int getNumPartitionsFromIdealState(IdealState idealState) {
+    int numPartitions = 0;
+    for (String segmentName : idealState.getRecord().getMapFields().keySet()) {
+      if (LLCSegmentName.isLowLevelConsumerSegmentName(segmentName)) {
+        numPartitions = Math.max(numPartitions, new LLCSegmentName(segmentName).getPartitionId() + 1);
+      }
     }
+    return numPartitions;
+  }
 
-    List<String> newSegmentInstances = instanceAssignments.get(newSegmentId);
-    Set<String> currentSegmentInstances = idealState.getInstanceSet(currentSegmentId);
-    for (String instance : currentSegmentInstances) {
-      idealState.setPartitionState(currentSegmentId, instance,
-          PinotHelixSegmentOnlineOfflineStateModelGenerator.ONLINE_STATE);
+  private int getNumReplicas(TableConfig tableConfig, InstancePartitions instancePartitions) {
+    if (instancePartitions.getNumReplicaGroups() == 1) {
+      // Non-replica-group based
+      return tableConfig.getValidationConfig().getReplicasPerPartitionNumber();
+    } else {
+      // Replica-group based
+      return instancePartitions.getNumReplicaGroups();
     }
+  }
 
-    // The {@link RealtimeSegmentValidationManager} will fix metadata and ideal state after {@link MAX_SEGMENT_COMPLETION_TIME_MILLIS} of inactivity on the committing segment
-    // If the ideal state update during completion took longer than {@link MAX_SEGMENT_COMPLETION_TIME_MILLIS},the update could already have been done by the fixer thread .
-    // We do not want to overwrite the ideal state. It is possible that the new segment created by the fixer has already progressed to ONLINE.
-    // If we let the below update happen, we will be bringing an ONLINE segment back to CONSUMING, and end up with 2 CONSUMING segments for the partition
-    Map<String, String> stateMap = idealState.getInstanceStateMap(newSegmentId);
-    if (stateMap == null) {
-      for (String instance : newSegmentInstances) {
-        idealState.setPartitionState(newSegmentId, instance,
-            PinotHelixSegmentOnlineOfflineStateModelGenerator.CONSUMING_STATE);
-      }
+  private int getMaxNumPartitionsPerInstance(InstancePartitions instancePartitions, int numPartitions,
+      int numReplicas) {
+    if (instancePartitions.getNumReplicaGroups() == 1) {
+      // Non-replica-group based assignment:
+      // Uniformly spray the partitions and replicas across the instances.
+      // E.g. (6 instances, 3 partitions, 4 replicas)
+      // "0_0": [i0,  i1,  i2,  i3,  i4,  i5  ]
+      //         p0r0 p0r1 p0r2 p1r3 p1r0 p1r1
+      //         p1r2 p1r3 p2r0 p2r1 p2r2 p2r3
+
+      int numInstances = instancePartitions.getInstances(0, 0).size();
+      return (numPartitions * numReplicas + numInstances - 1) / numInstances;
+    } else {
+      // Replica-group based assignment:
+      // Within a replica-group, uniformly spray the partitions across the instances.
+      // E.g. (within a replica-group, 3 instances, 6 partitions)
+      // "0_0": [i0, i1, i2]
+      //         p0  p1  p2
+      //         p3  p4  p5
+
+      int numInstancesPerReplicaGroup = instancePartitions.getInstances(0, 0).size();
+      return (numPartitions + numInstancesPerReplicaGroup - 1) / numInstancesPerReplicaGroup;
     }
-
-    return idealState;
   }
 }
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/SegmentCompletionManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/SegmentCompletionManager.java
index 314cab6..a3acf8d 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/SegmentCompletionManager.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/SegmentCompletionManager.java
@@ -95,7 +95,7 @@ public class SegmentCompletionManager {
     SegmentCompletionProtocol
         .setMaxSegmentCommitTimeMs(TimeUnit.MILLISECONDS.convert(segmentCommitTimeoutSeconds, TimeUnit.SECONDS));
     _fsmLocks = new Lock[NUM_FSM_LOCKS];
-    for (int i = 0; i < NUM_FSM_LOCKS; i ++) {
+    for (int i = 0; i < NUM_FSM_LOCKS; i++) {
       _fsmLocks[i] = new ReentrantLock();
     }
   }
@@ -129,17 +129,17 @@ public class SegmentCompletionManager {
         // TODO if we keep a list of last few committed segments, we don't need to go to zk for this.
         final String realtimeTableName = TableNameBuilder.REALTIME.tableNameWithType(segmentName.getTableName());
         LLCRealtimeSegmentZKMetadata segmentMetadata =
-            _segmentManager.getRealtimeSegmentZKMetadata(realtimeTableName, segmentName.getSegmentName(), null);
+            _segmentManager.getSegmentZKMetadata(realtimeTableName, segmentName.getSegmentName(), null);
         if (segmentMetadata.getStatus().equals(CommonConstants.Segment.Realtime.Status.DONE)) {
           // Best to go through the state machine for this case as well, so that all code regarding state handling is in one place
           // Also good for synchronization, because it is possible that multiple threads take this path, and we don't want
           // multiple instances of the FSM to be created for the same commit sequence at the same time.
           final long endOffset = segmentMetadata.getEndOffset();
-          fsm = SegmentCompletionFSM.fsmInCommit(_segmentManager, this, segmentName, segmentMetadata.getNumReplicas(),
-              endOffset);
+          fsm = SegmentCompletionFSM
+              .fsmInCommit(_segmentManager, this, segmentName, segmentMetadata.getNumReplicas(), endOffset);
         } else if (msgType.equals(SegmentCompletionProtocol.MSG_TYPE_STOPPED_CONSUMING)) {
-          fsm = SegmentCompletionFSM.fsmStoppedConsuming(_segmentManager, this, segmentName,
-              segmentMetadata.getNumReplicas());
+          fsm = SegmentCompletionFSM
+              .fsmStoppedConsuming(_segmentManager, this, segmentName, segmentMetadata.getNumReplicas());
         } else {
           // Segment is in the process of completing, and this is the first one to respond. Create fsm
           fsm = SegmentCompletionFSM.fsmInHolding(_segmentManager, this, segmentName, segmentMetadata.getNumReplicas());
@@ -356,6 +356,7 @@ public class SegmentCompletionManager {
     State _state = State.HOLDING;   // Typically start off in HOLDING state.
     final long _startTimeMs;
     private final LLCSegmentName _segmentName;
+    private final String _rawTableName;
     private final String _realtimeTableName;
     private final int _numReplicas;
     private final Set<String> _excludedServerStateMap;
@@ -399,7 +400,8 @@ public class SegmentCompletionManager {
     private SegmentCompletionFSM(PinotLLCRealtimeSegmentManager segmentManager,
         SegmentCompletionManager segmentCompletionManager, LLCSegmentName segmentName, int numReplicas) {
       _segmentName = segmentName;
-      _realtimeTableName = _segmentName.getTableName();
+      _rawTableName = _segmentName.getTableName();
+      _realtimeTableName = TableNameBuilder.REALTIME.tableNameWithType(_rawTableName);
       _numReplicas = numReplicas;
       _segmentManager = segmentManager;
       _commitStateMap = new HashMap<>(_numReplicas);
@@ -408,9 +410,8 @@ public class SegmentCompletionManager {
       _startTimeMs = _segmentCompletionManager.getCurrentTimeMs();
       _maxTimeToPickWinnerMs = _startTimeMs + MAX_TIME_TO_PICK_WINNER_MS;
       _maxTimeToNotifyWinnerMs = _startTimeMs + MAX_TIME_TO_NOTIFY_WINNER_MS;
-      long initialCommitTimeMs =
-          MAX_TIME_TO_NOTIFY_WINNER_MS + _segmentManager.getCommitTimeoutMS(_realtimeTableName);
-      Long savedCommitTime = _segmentCompletionManager._commitTimeMap.get(_realtimeTableName);
+      long initialCommitTimeMs = MAX_TIME_TO_NOTIFY_WINNER_MS + _segmentManager.getCommitTimeoutMS(_realtimeTableName);
+      Long savedCommitTime = _segmentCompletionManager._commitTimeMap.get(_rawTableName);
       if (savedCommitTime != null && savedCommitTime > initialCommitTimeMs) {
         initialCommitTimeMs = savedCommitTime;
       }
@@ -631,7 +632,8 @@ public class SegmentCompletionManager {
           LOGGER.error("Segment upload failed");
           return abortAndReturnFailed();
         }
-        SegmentCompletionProtocol.Response response = commitSegment(reqParams, isSplitCommit, committingSegmentDescriptor);
+        SegmentCompletionProtocol.Response response =
+            commitSegment(reqParams, isSplitCommit, committingSegmentDescriptor);
         if (!response.equals(SegmentCompletionProtocol.RESP_COMMIT_SUCCESS)) {
           return abortAndReturnFailed();
         } else {
@@ -687,14 +689,14 @@ public class SegmentCompletionManager {
     private SegmentCompletionProtocol.Response abortAndReturnHold(long now, String instanceId, long offset) {
       _state = State.ABORTED;
       _segmentCompletionManager._controllerMetrics
-          .addMeteredTableValue(_realtimeTableName, ControllerMeter.LLC_STATE_MACHINE_ABORTS, 1);
+          .addMeteredTableValue(_rawTableName, ControllerMeter.LLC_STATE_MACHINE_ABORTS, 1);
       return hold(instanceId, offset);
     }
 
     private SegmentCompletionProtocol.Response abortAndReturnFailed() {
       _state = State.ABORTED;
       _segmentCompletionManager._controllerMetrics
-          .addMeteredTableValue(_realtimeTableName, ControllerMeter.LLC_STATE_MACHINE_ABORTS, 1);
+          .addMeteredTableValue(_rawTableName, ControllerMeter.LLC_STATE_MACHINE_ABORTS, 1);
       return SegmentCompletionProtocol.RESP_FAILED;
     }
 
@@ -969,7 +971,13 @@ public class SegmentCompletionManager {
       LOGGER
           .info("Instance {} stopped consuming segment {} at offset {}, state {}, createNew: {}, reason:{}", instanceId,
               _segmentName, offset, _state, createNew, reason);
-      _segmentManager.segmentStoppedConsuming(_segmentName, instanceId);
+      try {
+        _segmentManager.segmentStoppedConsuming(_segmentName, instanceId);
+      } catch (Exception e) {
+        LOGGER.error("Caught exception while processing stopped CONSUMING segment: {} on instance: {}",
+            _segmentName.getSegmentName(), instanceId);
+        return SegmentCompletionProtocol.RESP_FAILED;
+      }
       return SegmentCompletionProtocol.RESP_PROCESSED;
     }
 
@@ -1012,9 +1020,7 @@ public class SegmentCompletionManager {
     }
 
     private SegmentCompletionProtocol.Response commitSegment(SegmentCompletionProtocol.Request.Params reqParams,
-                                                             boolean isSplitCommit,
-                                                             CommittingSegmentDescriptor committingSegmentDescriptor) {
-      boolean success;
+        boolean isSplitCommit, CommittingSegmentDescriptor committingSegmentDescriptor) {
       String instanceId = reqParams.getInstanceId();
       long offset = reqParams.getOffset();
       if (!_state.equals(State.COMMITTER_UPLOADING)) {
@@ -1028,17 +1034,24 @@ public class SegmentCompletionManager {
       // In case of splitCommit, the segment is uploaded to a unique file name indicated by segmentLocation,
       // so we need to move the segment file to its permanent location first before committing the metadata.
       if (isSplitCommit) {
-        if (!_segmentManager.commitSegmentFile(_segmentName.getTableName(), committingSegmentDescriptor)) {
+        try {
+          _segmentManager.commitSegmentFile(_realtimeTableName, committingSegmentDescriptor);
+        } catch (Exception e) {
+          LOGGER.error("Caught exception while committing segment file for segment: {}", _segmentName.getSegmentName());
           return SegmentCompletionProtocol.RESP_FAILED;
         }
       }
-      success = _segmentManager.commitSegmentMetadata(_segmentName.getTableName(), committingSegmentDescriptor);
-      if (success) {
-        _state = State.COMMITTED;
-        LOGGER.info("Committed segment {} at offset {} winner {}", _segmentName.getSegmentName(), offset, instanceId);
-        return SegmentCompletionProtocol.RESP_COMMIT_SUCCESS;
+      try {
+        _segmentManager.commitSegmentMetadata(_realtimeTableName, committingSegmentDescriptor);
+      } catch (Exception e) {
+        LOGGER
+            .error("Caught exception while committing segment metadata for segment: {}", _segmentName.getSegmentName());
+        return SegmentCompletionProtocol.RESP_FAILED;
       }
-      return SegmentCompletionProtocol.RESP_FAILED;
+
+      _state = State.COMMITTED;
+      LOGGER.info("Committed segment {} at offset {} winner {}", _segmentName.getSegmentName(), offset, instanceId);
+      return SegmentCompletionProtocol.RESP_COMMIT_SUCCESS;
     }
 
     private SegmentCompletionProtocol.Response processCommitWhileUploading(String instanceId, long offset, long now) {
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/TableConfigCache.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/TableConfigCache.java
index 05e3398..ddff959 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/TableConfigCache.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/TableConfigCache.java
@@ -18,11 +18,13 @@
  */
 package org.apache.pinot.controller.helix.core.realtime;
 
+import com.google.common.base.Preconditions;
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
+import javax.annotation.Nonnull;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.store.zk.ZkHelixPropertyStore;
 import org.apache.pinot.common.config.TableConfig;
@@ -39,19 +41,19 @@ public class TableConfigCache {
   private static final long DEFAULT_CACHE_TIMEOUT_IN_MINUTE = 60;
 
   private final LoadingCache<String, TableConfig> _tableConfigCache;
-  private final ZkHelixPropertyStore<ZNRecord> _propertyStore;
 
   public TableConfigCache(ZkHelixPropertyStore<ZNRecord> propertyStore) {
     _tableConfigCache = CacheBuilder.newBuilder().maximumSize(DEFAULT_CACHE_SIZE)
         .expireAfterWrite(DEFAULT_CACHE_TIMEOUT_IN_MINUTE, TimeUnit.MINUTES)
         .build(new CacheLoader<String, TableConfig>() {
           @Override
-          public TableConfig load(String tableNameWithType)
-              throws Exception {
-            return ZKMetadataProvider.getTableConfig(_propertyStore, tableNameWithType);
+          public TableConfig load(@Nonnull String tableNameWithType) {
+            TableConfig tableConfig = ZKMetadataProvider.getTableConfig(propertyStore, tableNameWithType);
+            Preconditions
+                .checkState(tableConfig != null, "Failed to find table config for table: %s", tableNameWithType);
+            return tableConfig;
           }
         });
-    _propertyStore = propertyStore;
   }
 
   public TableConfig getTableConfig(String tableNameWithType)
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/segment/DefaultFlushThresholdUpdater.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/segment/DefaultFlushThresholdUpdater.java
index 81f86ae..4612f21 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/segment/DefaultFlushThresholdUpdater.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/segment/DefaultFlushThresholdUpdater.java
@@ -19,13 +19,8 @@
 package org.apache.pinot.controller.helix.core.realtime.segment;
 
 import com.google.common.annotations.VisibleForTesting;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
 import org.apache.pinot.common.metadata.segment.LLCRealtimeSegmentZKMetadata;
-import org.apache.pinot.common.partition.PartitionAssignment;
-import org.apache.pinot.common.utils.LLCSegmentName;
 
 
 /**
@@ -41,35 +36,11 @@ public class DefaultFlushThresholdUpdater implements FlushThresholdUpdater {
   }
 
   @Override
-  public void updateFlushThreshold(@Nonnull LLCRealtimeSegmentZKMetadata newSegmentZKMetadata,
-      LLCRealtimeSegmentZKMetadata committingSegmentZKMetadata, CommittingSegmentDescriptor committingSegmentDescriptor,
-      @Nonnull PartitionAssignment partitionAssignment) {
-
-    // Gather list of instances for this partition
-    String partitionId = new LLCSegmentName(newSegmentZKMetadata.getSegmentName()).getPartitionRange();
-    List<String> instancesListForPartition = partitionAssignment.getInstancesListForPartition(partitionId);
-    Map<String, Integer> partitionCountForInstance = new HashMap<>(instancesListForPartition.size());
-    instancesListForPartition.forEach(instance -> partitionCountForInstance.put(instance, 0));
-
-    // Find partition count for each instance
-    int maxPartitionCountPerInstance = 1;
-    for (Map.Entry<String, List<String>> partitionAndInstanceList : partitionAssignment.getPartitionToInstances()
-        .entrySet()) {
-      List<String> instances = partitionAndInstanceList.getValue();
-      for (String instance : instances) {
-        if (partitionCountForInstance.containsKey(instance)) {
-          int partitionCount = partitionCountForInstance.get(instance) + 1;
-          partitionCountForInstance.put(instance, partitionCount);
-          if (maxPartitionCountPerInstance < partitionCount) {
-            maxPartitionCountPerInstance = partitionCount;
-          }
-        }
-      }
-    }
-
-    // Configure the segment size flush limit based on the maximum number of partitions allocated to a replica
-    int segmentFlushSize = (int) (((float) _tableFlushSize) / maxPartitionCountPerInstance);
-    newSegmentZKMetadata.setSizeThresholdToFlushSegment(segmentFlushSize);
+  public void updateFlushThreshold(LLCRealtimeSegmentZKMetadata newSegmentZKMetadata,
+      CommittingSegmentDescriptor committingSegmentDescriptor,
+      @Nullable LLCRealtimeSegmentZKMetadata committingSegmentZKMetadata, int maxNumPartitionsPerInstance) {
+    // Configure the segment size flush limit based on the maximum number of partitions allocated to an instance
+    newSegmentZKMetadata.setSizeThresholdToFlushSegment(_tableFlushSize / maxNumPartitionsPerInstance);
   }
 
   @VisibleForTesting
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/segment/FlushThresholdUpdateManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/segment/FlushThresholdUpdateManager.java
index df9728c..df1110a 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/segment/FlushThresholdUpdateManager.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/segment/FlushThresholdUpdateManager.java
@@ -20,10 +20,7 @@ package org.apache.pinot.controller.helix.core.realtime.segment;
 
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
-import org.apache.pinot.common.config.TableConfig;
 import org.apache.pinot.core.realtime.stream.PartitionLevelStreamConfig;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 
 /**
@@ -36,29 +33,25 @@ public class FlushThresholdUpdateManager {
   /**
    * Check table config for flush size.
    *
-   * If flush size < 0, create a new DefaultFlushThresholdUpdater with default flush size
    * If flush size > 0, create a new DefaultFlushThresholdUpdater with given flush size.
    * If flush size == 0, create new SegmentSizeBasedFlushThresholdUpdater if not already created. Create only 1 per table, because we want to maintain tuning information for the table in the updater
    */
-  public FlushThresholdUpdater getFlushThresholdUpdater(TableConfig realtimeTableConfig) {
-    final String tableName = realtimeTableConfig.getTableName();
-    PartitionLevelStreamConfig streamConfig =
-        new PartitionLevelStreamConfig(tableName, realtimeTableConfig.getIndexingConfig().getStreamConfigs());
+  public FlushThresholdUpdater getFlushThresholdUpdater(PartitionLevelStreamConfig partitionLevelStreamConfig) {
+    String realtimeTableName = partitionLevelStreamConfig.getTableNameWithType();
+    int flushThresholdRows = partitionLevelStreamConfig.getFlushThresholdRows();
 
-    final int tableFlushSize = streamConfig.getFlushThresholdRows();
-
-    if (tableFlushSize == 0) {
-      final long desiredSegmentSize = streamConfig.getFlushSegmentDesiredSizeBytes();
-      final int flushAutotuneInitialRows = streamConfig.getFlushAutotuneInitialRows();
-      return _flushThresholdUpdaterMap.computeIfAbsent(tableName,
-          k -> new SegmentSizeBasedFlushThresholdUpdater(desiredSegmentSize, flushAutotuneInitialRows));
+    if (flushThresholdRows > 0) {
+      _flushThresholdUpdaterMap.remove(realtimeTableName);
+      return new DefaultFlushThresholdUpdater(flushThresholdRows);
     } else {
-      _flushThresholdUpdaterMap.remove(tableName);
-      return new DefaultFlushThresholdUpdater(tableFlushSize);
+      long flushSegmentDesiredSizeBytes = partitionLevelStreamConfig.getFlushSegmentDesiredSizeBytes();
+      int flushAutotuneInitialRows = partitionLevelStreamConfig.getFlushAutotuneInitialRows();
+      return _flushThresholdUpdaterMap.computeIfAbsent(realtimeTableName,
+          k -> new SegmentSizeBasedFlushThresholdUpdater(flushSegmentDesiredSizeBytes, flushAutotuneInitialRows));
     }
   }
 
-  public void clearFlushThresholdUpdater(TableConfig tableConfig) {
-    _flushThresholdUpdaterMap.remove(tableConfig.getTableName());
+  public void clearFlushThresholdUpdater(String realtimeTableName) {
+    _flushThresholdUpdaterMap.remove(realtimeTableName);
   }
 }
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/segment/FlushThresholdUpdater.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/segment/FlushThresholdUpdater.java
index 742b527..210bb08 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/segment/FlushThresholdUpdater.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/segment/FlushThresholdUpdater.java
@@ -18,25 +18,20 @@
  */
 package org.apache.pinot.controller.helix.core.realtime.segment;
 
-import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
 import org.apache.pinot.common.metadata.segment.LLCRealtimeSegmentZKMetadata;
-import org.apache.pinot.common.partition.PartitionAssignment;
 
 
 /**
- * Interface for the flush threshold updation strategies
- * These implementations are responsible for updating the flush threshold (rows/time) in the given segment metadata
+ * Interface for the flush threshold updating strategies
+ * These implementations are responsible for updating the flush threshold (rows/time) for the given segment ZK metadata
  */
 public interface FlushThresholdUpdater {
 
   /**
-   * Updated the flush threshold of the segment metadata
-   * @param newSegmentZKMetadata - new segment metadata for which the thresholds need to be set
-   * @param committingSegmentZKMetadata - metadata of the committing segment
-   * @param committingSegmentDescriptor
-   * @param partitionAssignment - partition assignment for the table
+   * Updates the flush threshold for the given segment ZK metadata
    */
-  void updateFlushThreshold(@Nonnull LLCRealtimeSegmentZKMetadata newSegmentZKMetadata,
-      LLCRealtimeSegmentZKMetadata committingSegmentZKMetadata, CommittingSegmentDescriptor committingSegmentDescriptor,
-      PartitionAssignment partitionAssignment);
+  void updateFlushThreshold(LLCRealtimeSegmentZKMetadata newSegmentZKMetadata,
+      CommittingSegmentDescriptor committingSegmentDescriptor,
+      @Nullable LLCRealtimeSegmentZKMetadata committingSegmentZKMetadata, int maxNumPartitionsPerInstance);
 }
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/segment/SegmentSizeBasedFlushThresholdUpdater.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/segment/SegmentSizeBasedFlushThresholdUpdater.java
index d3c40c4..4d0dcec 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/segment/SegmentSizeBasedFlushThresholdUpdater.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/segment/SegmentSizeBasedFlushThresholdUpdater.java
@@ -19,9 +19,8 @@
 package org.apache.pinot.controller.helix.core.realtime.segment;
 
 import com.google.common.annotations.VisibleForTesting;
-import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
 import org.apache.pinot.common.metadata.segment.LLCRealtimeSegmentZKMetadata;
-import org.apache.pinot.common.partition.PartitionAssignment;
 import org.apache.pinot.common.utils.LLCSegmentName;
 import org.apache.pinot.common.utils.time.TimeUtils;
 import org.slf4j.Logger;
@@ -36,7 +35,6 @@ import org.slf4j.LoggerFactory;
  * This ensures that we take into account the history of the segment size and number rows
  */
 public class SegmentSizeBasedFlushThresholdUpdater implements FlushThresholdUpdater {
-
   private static final Logger LOGGER = LoggerFactory.getLogger(SegmentSizeBasedFlushThresholdUpdater.class);
 
   private static final double CURRENT_SEGMENT_RATIO_WEIGHT = 0.1;
@@ -89,10 +87,9 @@ public class SegmentSizeBasedFlushThresholdUpdater implements FlushThresholdUpda
 
   // synchronized since this method could be called for multiple partitions of the same table in different threads
   @Override
-  public synchronized void updateFlushThreshold(@Nonnull LLCRealtimeSegmentZKMetadata newSegmentZKMetadata,
-      LLCRealtimeSegmentZKMetadata committingSegmentZKMetadata,
-      @Nonnull CommittingSegmentDescriptor committingSegmentDescriptor, PartitionAssignment partitionAssignment) {
-
+  public synchronized void updateFlushThreshold(LLCRealtimeSegmentZKMetadata newSegmentZKMetadata,
+      CommittingSegmentDescriptor committingSegmentDescriptor,
+      @Nullable LLCRealtimeSegmentZKMetadata committingSegmentZKMetadata, int maxNumPartitionsPerInstance) {
     final String newSegmentName = newSegmentZKMetadata.getSegmentName();
     if (committingSegmentZKMetadata == null) { // first segment of the partition, hence committing segment is null
       if (_latestSegmentRowsToSizeRatio > 0) { // new partition added case
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java
index aa6d556..d051a1b 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java
@@ -36,6 +36,7 @@ import org.apache.pinot.controller.LeadControllerManager;
 import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
 import org.apache.pinot.controller.helix.core.periodictask.ControllerPeriodicTask;
 import org.apache.pinot.controller.helix.core.realtime.PinotLLCRealtimeSegmentManager;
+import org.apache.pinot.core.realtime.stream.PartitionLevelStreamConfig;
 import org.apache.pinot.core.realtime.stream.StreamConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -95,10 +96,9 @@ public class RealtimeSegmentValidationManager extends ControllerPeriodicTask<Rea
         updateRealtimeDocumentCount(tableConfig);
       }
 
-      Map<String, String> streamConfigMap = tableConfig.getIndexingConfig().getStreamConfigs();
-      StreamConfig streamConfig = new StreamConfig(tableNameWithType, streamConfigMap);
-      if (streamConfig.hasLowLevelConsumerType()) {
-        _llcRealtimeSegmentManager.ensureAllPartitionsConsuming(tableConfig);
+      PartitionLevelStreamConfig partitionLevelStreamConfig = new PartitionLevelStreamConfig(tableConfig);
+      if (partitionLevelStreamConfig.hasLowLevelConsumerType()) {
+        _llcRealtimeSegmentManager.ensureAllPartitionsConsuming(tableConfig, partitionLevelStreamConfig);
       }
     }
   }
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java
index 50042e9..451ccbe 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java
@@ -18,1616 +18,918 @@
  */
 package org.apache.pinot.controller.helix.core.realtime;
 
-import com.google.common.collect.Lists;
-import com.google.common.io.Files;
-import com.yammer.metrics.core.MetricsRegistry;
+import com.google.common.base.Preconditions;
 import java.io.File;
 import java.io.IOException;
-import java.lang.reflect.Field;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
-import java.util.Set;
-import javax.annotation.Nonnull;
-import org.apache.commons.configuration.PropertiesConfiguration;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
+import org.apache.commons.configuration.BaseConfiguration;
 import org.apache.commons.io.FileUtils;
-import org.apache.helix.HelixManager;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.manager.zk.ZNRecordSerializer;
 import org.apache.helix.model.IdealState;
-import org.apache.pinot.common.Utils;
-import org.apache.pinot.common.config.IndexingConfig;
-import org.apache.pinot.common.config.SegmentsValidationAndRetentionConfig;
+import org.apache.pinot.common.assignment.InstancePartitions;
+import org.apache.pinot.common.assignment.InstancePartitionsType;
 import org.apache.pinot.common.config.TableConfig;
 import org.apache.pinot.common.config.TableNameBuilder;
-import org.apache.pinot.common.config.TenantConfig;
-import org.apache.pinot.common.exception.InvalidConfigException;
 import org.apache.pinot.common.metadata.segment.LLCRealtimeSegmentZKMetadata;
 import org.apache.pinot.common.metrics.ControllerMetrics;
-import org.apache.pinot.common.partition.IdealStateBuilderUtil;
-import org.apache.pinot.common.partition.PartitionAssignment;
-import org.apache.pinot.common.partition.StreamPartitionAssignmentGenerator;
-import org.apache.pinot.common.protocols.SegmentCompletionProtocol;
-import org.apache.pinot.common.utils.CommonConstants;
+import org.apache.pinot.common.utils.CommonConstants.Helix;
+import org.apache.pinot.common.utils.CommonConstants.Helix.StateModel.RealtimeSegmentOnlineOfflineStateModel;
+import org.apache.pinot.common.utils.CommonConstants.Helix.TableType;
+import org.apache.pinot.common.utils.CommonConstants.Segment.Realtime.Status;
 import org.apache.pinot.common.utils.LLCSegmentName;
 import org.apache.pinot.controller.ControllerConf;
-import org.apache.pinot.controller.LeadControllerManager;
 import org.apache.pinot.controller.api.resources.LLCSegmentCompletionHandlers;
 import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
-import org.apache.pinot.controller.helix.core.PinotTableIdealStateBuilder;
+import org.apache.pinot.controller.helix.core.assignment.segment.SegmentAssignment;
 import org.apache.pinot.controller.helix.core.realtime.segment.CommittingSegmentDescriptor;
 import org.apache.pinot.controller.util.SegmentCompletionUtils;
 import org.apache.pinot.core.indexsegment.generator.SegmentVersion;
 import org.apache.pinot.core.realtime.impl.fakestream.FakeStreamConfigUtils;
 import org.apache.pinot.core.realtime.stream.OffsetCriteria;
+import org.apache.pinot.core.realtime.stream.PartitionLevelStreamConfig;
 import org.apache.pinot.core.realtime.stream.StreamConfig;
 import org.apache.pinot.core.segment.index.SegmentMetadataImpl;
 import org.apache.pinot.filesystem.PinotFSFactory;
 import org.apache.zookeeper.data.Stat;
 import org.joda.time.Interval;
-import org.testng.Assert;
-import org.testng.annotations.AfterTest;
-import org.testng.annotations.BeforeTest;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
-import static org.mockito.Mockito.*;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import static org.testng.Assert.*;
 
 
 public class PinotLLCRealtimeSegmentManagerTest {
-  private static final String clusterName = "testCluster";
-  private static final String DUMMY_HOST = "dummyHost:1234";
-  private static final String DEFAULT_SERVER_TENANT = "freeTenant";
+  private static final File TEMP_DIR = new File(FileUtils.getTempDirectory(), "PinotLLCRealtimeSegmentManagerTest");
   private static final String SCHEME = LLCSegmentCompletionHandlers.getScheme();
-  private String[] serverNames;
-  private static File baseDir;
-  private Random random;
-
-  private PinotHelixResourceManager _mockPinotHelixResourceManager;
-
-  private enum ExternalChange {
-    N_INSTANCES_CHANGED, N_PARTITIONS_INCREASED, N_INSTANCES_CHANGED_AND_PARTITIONS_INCREASED
-  }
-
-  private List<String> getInstanceList(final int nServers) {
-    Assert.assertTrue(nServers <= serverNames.length);
-    String[] instanceArray = Arrays.copyOf(serverNames, nServers);
-    return Arrays.asList(instanceArray);
-  }
-
-  @BeforeTest
+  private static final String RAW_TABLE_NAME = "testTable";
+  private static final String REALTIME_TABLE_NAME = TableNameBuilder.REALTIME.tableNameWithType(RAW_TABLE_NAME);
+
+  private static final long RANDOM_SEED = System.currentTimeMillis();
+  private static final Random RANDOM = new Random(RANDOM_SEED);
+  static final long PARTITION_OFFSET = RANDOM.nextInt(Integer.MAX_VALUE);
+  static final long CURRENT_TIME_MS = System.currentTimeMillis();
+  static final long START_TIME_MS = CURRENT_TIME_MS - TimeUnit.HOURS.toMillis(RANDOM.nextInt(24) + 24);
+  static final long END_TIME_MS = START_TIME_MS + TimeUnit.HOURS.toMillis(RANDOM.nextInt(24) + 1);
+  static final Interval INTERVAL = new Interval(START_TIME_MS, END_TIME_MS);
+  static final String CRC = Long.toString(RANDOM.nextLong());
+  static final String SEGMENT_VERSION =
+      RANDOM.nextBoolean() ? SegmentVersion.v1.toString() : SegmentVersion.v3.toString();
+  static final int NUM_DOCS = RANDOM.nextInt(Integer.MAX_VALUE) + 1;
+
+  @BeforeClass
   public void setUp() {
     // Printing out the random seed to console so that we can use the seed to reproduce failure conditions
-    long seed = new Random().nextLong();
-    System.out.println("Random seed for " + PinotLLCRealtimeSegmentManagerTest.class.getSimpleName() + " is " + seed);
-    random = new Random(seed);
-
-    final int maxInstances = 20;
-    serverNames = new String[maxInstances];
-    for (int i = 0; i < maxInstances; i++) {
-      serverNames[i] = "Server_" + i;
-    }
-    try {
-      baseDir = Files.createTempDir();
-      baseDir.deleteOnExit();
-    } catch (Exception e) {
-
-    }
-    FakePinotLLCRealtimeSegmentManager.IS_CONNECTED = true;
-    FakePinotLLCRealtimeSegmentManager.IS_LEADER = true;
-
-    _mockPinotHelixResourceManager = mock(PinotHelixResourceManager.class);
-    HelixManager mockHelixManager = mock(HelixManager.class);
-    when(_mockPinotHelixResourceManager.getHelixZkManager()).thenReturn(mockHelixManager);
+    System.out.println("Using random seed: " + RANDOM_SEED);
   }
 
-  @AfterTest
-  public void cleanUp()
+  @AfterClass
+  public void tearDown()
       throws IOException {
-    FileUtils.deleteDirectory(baseDir);
+    FileUtils.deleteDirectory(TEMP_DIR);
+  }
+
+  private SegmentMetadataImpl mockSegmentMetadata() {
+    SegmentMetadataImpl segmentMetadata = mock(SegmentMetadataImpl.class);
+    when(segmentMetadata.getTimeInterval()).thenReturn(INTERVAL);
+    when(segmentMetadata.getCrc()).thenReturn(CRC);
+    when(segmentMetadata.getVersion()).thenReturn(SEGMENT_VERSION);
+    when(segmentMetadata.getTotalRawDocs()).thenReturn(NUM_DOCS);
+    return segmentMetadata;
   }
 
   /**
-   * Test cases for new table being created, and initial segments setup that follows. The tables are
-   * set up with stream offset to be the 'largest' and the corresponding ideal state offset is
-   * accordingly verified.
+   * Test cases for new table being created, and initial segments setup that follows.
    */
   @Test
-  public void testSetupNewTable() {
-    String tableName = "validateThisTable_REALTIME";
-    int nReplicas = 2;
-    IdealStateBuilderUtil idealStateBuilder = new IdealStateBuilderUtil(tableName);
-
-    TableConfig tableConfig;
-    IdealState idealState;
-    int nPartitions;
-    boolean invalidConfig;
-    boolean badStream = false;
-    List<String> instances = getInstanceList(1);
-
-    // insufficient instances
-    tableConfig = makeTableConfig(tableName, nReplicas, DUMMY_HOST, DEFAULT_SERVER_TENANT);
-    idealState = idealStateBuilder.build();
-    nPartitions = 4;
-    invalidConfig = true;
-    testSetupNewTable(tableConfig, idealState, nPartitions, nReplicas, instances, invalidConfig, badStream);
-
-    // noop path - 0 partitions
-    badStream = false;
-    tableConfig = makeTableConfig(tableName, nReplicas, DUMMY_HOST, DEFAULT_SERVER_TENANT);
-    idealState = idealStateBuilder.build();
-    nPartitions = 0;
-    invalidConfig = false;
-    instances = getInstanceList(3);
-    testSetupNewTable(tableConfig, idealState, nPartitions, nReplicas, instances, invalidConfig, badStream);
-
-    // noop path - ideal state disabled - this can happen in the code path only if there is already an idealstate with HLC segments in it, and it has been disabled.
-    tableConfig = makeTableConfig(tableName, nReplicas, DUMMY_HOST, DEFAULT_SERVER_TENANT);
-    idealState = idealStateBuilder.disableIdealState().build();
-    nPartitions = 4;
-    invalidConfig = false;
-    testSetupNewTable(tableConfig, idealState, nPartitions, nReplicas, instances, invalidConfig, badStream);
-
-    // clear builder to enable ideal state
-    idealStateBuilder.clear();
-    // happy paths - new table config with nPartitions and sufficient instances
-    tableConfig = makeTableConfig(tableName, nReplicas, DUMMY_HOST, DEFAULT_SERVER_TENANT);
-    idealState = idealStateBuilder.build();
-    nPartitions = 4;
-    invalidConfig = false;
-    testSetupNewTable(tableConfig, idealState, nPartitions, nReplicas, instances, invalidConfig, badStream);
-
-    idealState = idealStateBuilder.build();
-    nPartitions = 8;
-    testSetupNewTable(tableConfig, idealState, nPartitions, nReplicas, instances, invalidConfig, badStream);
-
-    idealState = idealStateBuilder.build();
-    nPartitions = 8;
-    instances = getInstanceList(10);
-    testSetupNewTable(tableConfig, idealState, nPartitions, nReplicas, instances, invalidConfig, badStream);
-
-    idealState = idealStateBuilder.build();
-    nPartitions = 12;
-    testSetupNewTable(tableConfig, idealState, nPartitions, nReplicas, instances, invalidConfig, badStream);
+  public void testSetUpNewTable() {
+    // Insufficient instances - 2 replicas, 1 instance, 4 partitions
+    testSetUpNewTable(2, 1, 4, true);
+
+    // Noop path - 2 replicas, 3 instances, 0 partition
+    testSetUpNewTable(2, 3, 0, false);
+
+    // Happy paths
+    // 2 replicas, 3 instances, 4 partitions
+    testSetUpNewTable(2, 3, 4, false);
+    // 2 replicas, 3 instances, 8 partitions
+    testSetUpNewTable(2, 3, 8, false);
+    // 8 replicas, 10 instances, 4 partitions
+    testSetUpNewTable(8, 10, 4, false);
   }
 
-  private void testSetupNewTable(TableConfig tableConfig, IdealState idealState, int nPartitions, int nReplicas,
-      List<String> instances, boolean invalidConfig, boolean badStream) {
-    FakePinotLLCRealtimeSegmentManager segmentManager =
-        new FakePinotLLCRealtimeSegmentManager(_mockPinotHelixResourceManager, null);
-    segmentManager._partitionAssignmentGenerator.setConsumingInstances(instances);
-    segmentManager.addTableToStore(tableConfig.getTableName(), tableConfig, nPartitions);
+  private void testSetUpNewTable(int numReplicas, int numInstances, int numPartitions, boolean expectException) {
+    FakePinotLLCRealtimeSegmentManager segmentManager = new FakePinotLLCRealtimeSegmentManager();
+    segmentManager._numReplicas = numReplicas;
+    segmentManager.makeTableConfig();
+    segmentManager._numInstances = numInstances;
+    segmentManager.makeConsumingInstancePartitions();
+    segmentManager._numPartitions = numPartitions;
 
     try {
-      segmentManager.setupNewTable(tableConfig, idealState);
-    } catch (InvalidConfigException e) {
-      Assert.assertTrue(invalidConfig);
-      return;
-    } catch (Exception e) { // Bad stream configs, offset fetcher exception
-      Assert.assertTrue(badStream);
+      segmentManager.setUpNewTable();
+      assertFalse(expectException);
+    } catch (IllegalStateException e) {
+      assertTrue(expectException);
       return;
     }
-    Assert.assertFalse(invalidConfig);
-    Assert.assertFalse(badStream);
-    Map<String, LLCSegmentName> partitionToLatestSegments =
-        segmentManager._partitionAssignmentGenerator.getPartitionToLatestSegments(idealState);
-    Map<String, Map<String, String>> mapFields = idealState.getRecord().getMapFields();
-    if (!idealState.isEnabled()) {
-      Assert.assertTrue(partitionToLatestSegments.isEmpty());
-    } else {
-      Assert.assertEquals(mapFields.size(), nPartitions);
-      for (int p = 0; p < nPartitions; p++) {
-        LLCSegmentName llcSegmentName = partitionToLatestSegments.get(String.valueOf(p));
-        String segmentName = llcSegmentName.getSegmentName();
-        Map<String, String> instanceStateMap = mapFields.get(segmentName);
-        Assert.assertEquals(instanceStateMap.size(), nReplicas);
-        for (Map.Entry<String, String> entry : instanceStateMap.entrySet()) {
-          Assert.assertEquals(entry.getValue(), "CONSUMING");
-        }
-        Assert.assertEquals(segmentManager._metadataMap.get(segmentName).getStatus(),
-            CommonConstants.Segment.Realtime.Status.IN_PROGRESS);
-      }
 
-      final List<String> propStorePaths = segmentManager._paths;
-      final List<ZNRecord> propStoreEntries = segmentManager._records;
+    Map<String, Map<String, String>> instanceStatesMap = segmentManager._idealState.getRecord().getMapFields();
+    assertEquals(instanceStatesMap.size(), numPartitions);
+    assertEquals(segmentManager.getAllSegments(REALTIME_TABLE_NAME).size(), numPartitions);
 
-      Assert.assertEquals(propStorePaths.size(), nPartitions);
-      Assert.assertEquals(propStoreEntries.size(), nPartitions);
+    for (int partitionId = 0; partitionId < numPartitions; partitionId++) {
+      LLCSegmentName llcSegmentName = new LLCSegmentName(RAW_TABLE_NAME, partitionId, 0, CURRENT_TIME_MS);
+      String segmentName = llcSegmentName.getSegmentName();
 
-      Map<Integer, ZNRecord> segmentPropStoreMap = new HashMap<>(propStorePaths.size());
-      Map<Integer, String> segmentPathsMap = new HashMap<>(propStorePaths.size());
-      for (String path : propStorePaths) {
-        String segNameStr = path.split("/")[3];
-        int partition = new LLCSegmentName(segNameStr).getPartitionId();
-        segmentPathsMap.put(partition, path);
+      Map<String, String> instanceStateMap = instanceStatesMap.get(segmentName);
+      assertNotNull(instanceStateMap);
+      assertEquals(instanceStateMap.size(), numReplicas);
+      for (String state : instanceStateMap.values()) {
+        assertEquals(state, RealtimeSegmentOnlineOfflineStateModel.CONSUMING);
       }
 
-      for (ZNRecord znRecord : propStoreEntries) {
-        LLCRealtimeSegmentZKMetadata metadata = new LLCRealtimeSegmentZKMetadata(znRecord);
-        segmentPropStoreMap.put(new LLCSegmentName(metadata.getSegmentName()).getPartitionId(), znRecord);
-      }
+      LLCRealtimeSegmentZKMetadata segmentZKMetadata =
+          segmentManager.getSegmentZKMetadata(REALTIME_TABLE_NAME, segmentName, null);
+      assertEquals(segmentZKMetadata.getStatus(), Status.IN_PROGRESS);
+      assertEquals(segmentZKMetadata.getStartOffset(), PARTITION_OFFSET);
+      assertEquals(segmentZKMetadata.getCreationTime(), CURRENT_TIME_MS);
+    }
+  }
 
-      Assert.assertEquals(segmentPathsMap.size(), nPartitions);
-      Assert.assertEquals(segmentPropStoreMap.size(), nPartitions);
-
-      for (int partition = 0; partition < nPartitions; partition++) {
-        final LLCRealtimeSegmentZKMetadata metadata =
-            new LLCRealtimeSegmentZKMetadata(segmentPropStoreMap.get(partition));
-
-        metadata.toString();  // Just for coverage
-        ZNRecord znRecord = metadata.toZNRecord();
-        LLCRealtimeSegmentZKMetadata metadataCopy = new LLCRealtimeSegmentZKMetadata(znRecord);
-        Assert.assertEquals(metadata, metadataCopy);
-        final String path = segmentPathsMap.get(partition);
-        final String segmentName = metadata.getSegmentName();
-        // verify the expected offset is set to the largest as configured in the table config
-        Assert.assertEquals(metadata.getStartOffset(), segmentManager.getLargestStreamOffset());
-        Assert.assertEquals(path, "/SEGMENTS/" + tableConfig.getTableName() + "/" + segmentName);
-        LLCSegmentName llcSegmentName = new LLCSegmentName(segmentName);
-        Assert.assertEquals(llcSegmentName.getPartitionId(), partition);
-        Assert.assertEquals(llcSegmentName.getTableName(),
-            TableNameBuilder.extractRawTableName(tableConfig.getTableName()));
-        Assert.assertEquals(metadata.getNumReplicas(), nReplicas);
-      }
+  private void setUpNewTable(FakePinotLLCRealtimeSegmentManager segmentManager, int numReplicas, int numInstances,
+      int numPartitions) {
+    segmentManager._numReplicas = numReplicas;
+    segmentManager.makeTableConfig();
+    segmentManager._numInstances = numInstances;
+    segmentManager.makeConsumingInstancePartitions();
+    segmentManager._numPartitions = numPartitions;
+    segmentManager.setUpNewTable();
+  }
+
+  @Test
+  public void testCommitSegment() {
+    // Set up a new table with 2 replicas, 5 instances, 4 partition
+    FakePinotLLCRealtimeSegmentManager segmentManager = new FakePinotLLCRealtimeSegmentManager();
+    setUpNewTable(segmentManager, 2, 5, 4);
+    Map<String, Map<String, String>> instanceStatesMap = segmentManager._idealState.getRecord().getMapFields();
+
+    // Commit a segment for partition 0
+    String committingSegment = new LLCSegmentName(RAW_TABLE_NAME, 0, 0, CURRENT_TIME_MS).getSegmentName();
+    CommittingSegmentDescriptor committingSegmentDescriptor =
+        new CommittingSegmentDescriptor(committingSegment, PARTITION_OFFSET + NUM_DOCS, 0L);
+    committingSegmentDescriptor.setSegmentMetadata(mockSegmentMetadata());
+    segmentManager.commitSegmentMetadata(REALTIME_TABLE_NAME, committingSegmentDescriptor);
+
+    // Verify instance states for committed segment and new consuming segment
+    Map<String, String> committedSegmentInstanceStateMap = instanceStatesMap.get(committingSegment);
+    assertNotNull(committedSegmentInstanceStateMap);
+    assertEquals(new HashSet<>(committedSegmentInstanceStateMap.values()),
+        Collections.singleton(RealtimeSegmentOnlineOfflineStateModel.ONLINE));
+
+    String consumingSegment = new LLCSegmentName(RAW_TABLE_NAME, 0, 1, CURRENT_TIME_MS).getSegmentName();
+    Map<String, String> consumingSegmentInstanceStateMap = instanceStatesMap.get(consumingSegment);
+    assertNotNull(consumingSegmentInstanceStateMap);
+    assertEquals(new HashSet<>(consumingSegmentInstanceStateMap.values()),
+        Collections.singleton(RealtimeSegmentOnlineOfflineStateModel.CONSUMING));
+
+    // Verify segment ZK metadata for committed segment and new consuming segment
+    LLCRealtimeSegmentZKMetadata committedSegmentZKMetadata =
+        segmentManager._segmentZKMetadataMap.get(committingSegment);
+    assertEquals(committedSegmentZKMetadata.getStatus(), Status.DONE);
+    assertEquals(committedSegmentZKMetadata.getStartOffset(), PARTITION_OFFSET);
+    assertEquals(committedSegmentZKMetadata.getEndOffset(), PARTITION_OFFSET + NUM_DOCS);
+    assertEquals(committedSegmentZKMetadata.getCreationTime(), CURRENT_TIME_MS);
+    assertEquals(committedSegmentZKMetadata.getTimeInterval(), INTERVAL);
+    assertEquals(committedSegmentZKMetadata.getCrc(), Long.parseLong(CRC));
+    assertEquals(committedSegmentZKMetadata.getIndexVersion(), SEGMENT_VERSION);
+    assertEquals(committedSegmentZKMetadata.getTotalRawDocs(), NUM_DOCS);
+
+    LLCRealtimeSegmentZKMetadata consumingSegmentZKMetadata =
+        segmentManager._segmentZKMetadataMap.get(consumingSegment);
+    assertEquals(consumingSegmentZKMetadata.getStatus(), Status.IN_PROGRESS);
+    assertEquals(consumingSegmentZKMetadata.getStartOffset(), PARTITION_OFFSET + NUM_DOCS);
+    assertEquals(committedSegmentZKMetadata.getCreationTime(), CURRENT_TIME_MS);
+
+    // Turn one instance of the consuming segment OFFLINE and commit the segment
+    consumingSegmentInstanceStateMap.entrySet().iterator().next()
+        .setValue(RealtimeSegmentOnlineOfflineStateModel.OFFLINE);
+    committingSegment = consumingSegment;
+    committingSegmentDescriptor =
+        new CommittingSegmentDescriptor(committingSegment, PARTITION_OFFSET + NUM_DOCS + NUM_DOCS, 0L);
+    committingSegmentDescriptor.setSegmentMetadata(mockSegmentMetadata());
+    segmentManager.commitSegmentMetadata(REALTIME_TABLE_NAME, committingSegmentDescriptor);
+
+    // Verify instance states for committed segment and new consuming segment
+    committedSegmentInstanceStateMap = instanceStatesMap.get(committingSegment);
+    assertNotNull(committedSegmentInstanceStateMap);
+    assertEquals(new HashSet<>(committedSegmentInstanceStateMap.values()),
+        Collections.singleton(RealtimeSegmentOnlineOfflineStateModel.ONLINE));
+
+    consumingSegment = new LLCSegmentName(RAW_TABLE_NAME, 0, 2, CURRENT_TIME_MS).getSegmentName();
+    consumingSegmentInstanceStateMap = instanceStatesMap.get(consumingSegment);
+    assertNotNull(consumingSegmentInstanceStateMap);
+    assertEquals(new HashSet<>(consumingSegmentInstanceStateMap.values()),
+        Collections.singleton(RealtimeSegmentOnlineOfflineStateModel.CONSUMING));
+
+    // Illegal segment commit - commit the segment again
+    try {
+      segmentManager.commitSegmentMetadata(REALTIME_TABLE_NAME, committingSegmentDescriptor);
+      fail();
+    } catch (IllegalStateException e) {
+      // Expected
     }
   }
 
   /**
-   * Test cases for the scenario where stream partitions increase, and the validation manager is attempting to create segments for new partitions
-   * This test assumes that all other factors remain same (no error conditions/inconsistencies in metadata and ideal state)
-   *
-   * The tables are created with "largest" offset and the consuming segments are expected to be set
-   * to consume from the smallest offset.
+   * Test cases for the scenario where stream partitions increase, and the validation manager is attempting to create
+   * segments for new partitions. This test assumes that all other factors remain the same (no error conditions or
+   * inconsistencies in metadata and ideal state).
    */
   @Test
-  public void testValidateLLCPartitionIncrease() {
-
-    FakePinotLLCRealtimeSegmentManager segmentManager =
-        new FakePinotLLCRealtimeSegmentManager(_mockPinotHelixResourceManager, null);
-    String tableName = "validateThisTable_REALTIME";
-    int nReplicas = 2;
-    IdealStateBuilderUtil idealStateBuilder = new IdealStateBuilderUtil(tableName);
-
-    TableConfig tableConfig;
-    IdealState idealState;
-    int nPartitions;
-    boolean skipNewPartitions = false;
-    List<String> instances = getInstanceList(5);
-
-    // empty to 4 partitions
-    tableConfig = makeTableConfig(tableName, nReplicas, DUMMY_HOST, DEFAULT_SERVER_TENANT);
-    idealState = idealStateBuilder.build();
-    nPartitions = 4;
-    validateLLCPartitionsIncrease(segmentManager, idealState, tableConfig, nPartitions, nReplicas, instances,
-        skipNewPartitions);
-
-    // only initial segments present CONSUMING - metadata INPROGRESS - increase numPartitions
-    nPartitions = 6;
-    validateLLCPartitionsIncrease(segmentManager, idealState, tableConfig, nPartitions, nReplicas, instances,
-        skipNewPartitions);
-
-    // 2 partitions advanced a seq number
-    PartitionAssignment partitionAssignment = segmentManager._partitionAssignmentGenerator
-        .getStreamPartitionAssignmentFromIdealState(tableConfig, idealState);
-    for (int p = 0; p < 2; p++) {
-      String segmentName = idealStateBuilder.getSegment(p, 0);
-      advanceASeqForPartition(idealState, segmentManager, partitionAssignment, segmentName, p, 1, 100, tableConfig);
+  public void testSetUpNewPartitions() {
+    // Set up a new table with 2 replicas, 5 instances, 0 partition
+    FakePinotLLCRealtimeSegmentManager segmentManager = new FakePinotLLCRealtimeSegmentManager();
+    setUpNewTable(segmentManager, 2, 5, 0);
+
+    // No-op
+    testSetUpNewPartitions(segmentManager, false);
+
+    // Increase number of partitions from 0 to 2
+    segmentManager._numPartitions = 2;
+    testSetUpNewPartitions(segmentManager, false);
+
+    // Increase number of partitions form 2 to 4
+    segmentManager._numPartitions = 4;
+    testSetUpNewPartitions(segmentManager, false);
+
+    // 2 partitions commit segment
+    for (int partitionId = 0; partitionId < 2; partitionId++) {
+      String segmentName = new LLCSegmentName(RAW_TABLE_NAME, partitionId, 0, CURRENT_TIME_MS).getSegmentName();
+      CommittingSegmentDescriptor committingSegmentDescriptor =
+          new CommittingSegmentDescriptor(segmentName, PARTITION_OFFSET + NUM_DOCS, 0L);
+      committingSegmentDescriptor.setSegmentMetadata(mockSegmentMetadata());
+      segmentManager.commitSegmentMetadata(REALTIME_TABLE_NAME, committingSegmentDescriptor);
     }
-    idealState = idealStateBuilder.build();
-    validateLLCPartitionsIncrease(segmentManager, idealState, tableConfig, nPartitions, nReplicas, instances,
-        skipNewPartitions);
-
-    // increase num partitions
-    nPartitions = 10;
-    validateLLCPartitionsIncrease(segmentManager, idealState, tableConfig, nPartitions, nReplicas, instances,
-        skipNewPartitions);
-
-    // keep num partitions same - noop
-    validateLLCPartitionsIncrease(segmentManager, idealState, tableConfig, nPartitions, nReplicas, instances,
-        skipNewPartitions);
-
-    // keep num partitions same, but bad instances - error
-    instances = getInstanceList(1);
-    validateLLCPartitionsIncrease(segmentManager, idealState, tableConfig, nPartitions, nReplicas, instances,
-        skipNewPartitions);
-
-    // increase num partitions, but bad instances - error
-    nPartitions = 12;
-    skipNewPartitions = true;
-    validateLLCPartitionsIncrease(segmentManager, idealState, tableConfig, nPartitions, nReplicas, instances,
-        skipNewPartitions);
-
-    // increase num partitions, but disabled ideal state - noop
-    idealState = idealStateBuilder.disableIdealState().build();
-    instances = getInstanceList(6);
-    validateLLCPartitionsIncrease(segmentManager, idealState, tableConfig, nPartitions, nReplicas, instances,
-        skipNewPartitions);
-  }
+    testSetUpNewPartitions(segmentManager, false);
 
-  private void advanceASeqForPartition(IdealState idealState, FakePinotLLCRealtimeSegmentManager segmentManager,
-      PartitionAssignment partitionAssignment, String segmentName, int partition, int nextSeqNum, long nextOffset,
-      TableConfig tableConfig) {
-    String tableName = tableConfig.getTableName();
-    String rawTableName = TableNameBuilder.extractRawTableName(tableName);
-    LLCSegmentName llcSegmentName = new LLCSegmentName(segmentName);
-    LLCSegmentName newLlcSegmentName =
-        new LLCSegmentName(rawTableName, partition, nextSeqNum, System.currentTimeMillis());
-    CommittingSegmentDescriptor committingSegmentDescriptor =
-        new CommittingSegmentDescriptor(segmentName, nextOffset, 0);
-    segmentManager.updateOldSegmentMetadataZNRecord(tableName, llcSegmentName, nextOffset, committingSegmentDescriptor);
-    segmentManager.createNewSegmentMetadataZNRecord(tableConfig, llcSegmentName, newLlcSegmentName, partitionAssignment,
-        committingSegmentDescriptor, false);
-    segmentManager.updateIdealStateOnSegmentCompletion(idealState, segmentName, newLlcSegmentName.getSegmentName(),
-        partitionAssignment);
+    // Increase number of partitions form 4 to 6
+    segmentManager._numPartitions = 6;
+    testSetUpNewPartitions(segmentManager, false);
+
+    // No-op
+    testSetUpNewPartitions(segmentManager, false);
+
+    // Reduce number of instances to 1 (illegal because it is less than number of replicas)
+    segmentManager._numInstances = 1;
+    segmentManager.makeConsumingInstancePartitions();
+
+    // No-op
+    testSetUpNewPartitions(segmentManager, false);
+
+    // Increase number of partitions form 6 to 8 (should fail)
+    segmentManager._numPartitions = 8;
+    testSetUpNewPartitions(segmentManager, true);
+
+    // Should fail again
+    testSetUpNewPartitions(segmentManager, true);
+
+    // Increase number of instances back to 5 and allow fixing segments
+    segmentManager._numInstances = 5;
+    segmentManager.makeConsumingInstancePartitions();
+    segmentManager._exceededMaxSegmentCompletionTime = true;
+
+    // Should succeed
+    testSetUpNewPartitions(segmentManager, false);
   }
 
-  private void validateLLCPartitionsIncrease(FakePinotLLCRealtimeSegmentManager segmentManager, IdealState idealState,
-      TableConfig tableConfig, int nPartitions, int nReplicas, List<String> instances, boolean skipNewPartitions) {
-    ZNRecordSerializer znRecordSerializer = new ZNRecordSerializer();
-    IdealState idealStateCopy =
-        new IdealState((ZNRecord) znRecordSerializer.deserialize(znRecordSerializer.serialize(idealState.getRecord())));
-    Map<String, Map<String, String>> oldMapFields = idealStateCopy.getRecord().getMapFields();
-    Map<String, LLCRealtimeSegmentZKMetadata> oldMetadataMap = new HashMap<>(segmentManager._metadataMap.size());
-    for (Map.Entry<String, LLCRealtimeSegmentZKMetadata> entry : segmentManager._metadataMap.entrySet()) {
-      oldMetadataMap.put(entry.getKey(), new LLCRealtimeSegmentZKMetadata(entry.getValue().toZNRecord()));
+  private void testSetUpNewPartitions(FakePinotLLCRealtimeSegmentManager segmentManager, boolean expectException) {
+    Map<String, Map<String, String>> instanceStatesMap = segmentManager._idealState.getRecord().getMapFields();
+    Map<String, Map<String, String>> oldInstanceStatesMap = cloneInstanceStatesMap(instanceStatesMap);
+    Map<String, LLCRealtimeSegmentZKMetadata> segmentZKMetadataMap = segmentManager._segmentZKMetadataMap;
+    Map<String, LLCRealtimeSegmentZKMetadata> oldSegmentZKMetadataMap = cloneSegmentZKMetadataMap(segmentZKMetadataMap);
+
+    try {
+      segmentManager.ensureAllPartitionsConsuming();
+    } catch (IllegalStateException e) {
+      assertTrue(expectException);
+      // Restore the old instance states map
+      segmentManager._idealState.getRecord().setMapFields(oldInstanceStatesMap);
+      return;
     }
-    segmentManager._partitionAssignmentGenerator.setConsumingInstances(instances);
-    IdealState updatedIdealState = segmentManager.ensureAllPartitionsConsuming(tableConfig, idealState, nPartitions);
-    Map<String, Map<String, String>> updatedMapFields = updatedIdealState.getRecord().getMapFields();
-    Map<String, LLCRealtimeSegmentZKMetadata> updatedMetadataMap = segmentManager._metadataMap;
-
-    // Verify - all original metadata and segments unchanged
-    Set<Integer> oldPartitions = new HashSet<>();
-    for (Map.Entry<String, Map<String, String>> entry : oldMapFields.entrySet()) {
+
+    // Check that instance states and ZK metadata remain the same for existing segments
+    int oldNumPartitions = 0;
+    for (Map.Entry<String, Map<String, String>> entry : oldInstanceStatesMap.entrySet()) {
       String segmentName = entry.getKey();
-      Map<String, String> instanceStateMap = entry.getValue();
-      Assert.assertTrue(updatedMapFields.containsKey(segmentName));
-      Map<String, String> updatedInstanceStateMap = updatedMapFields.get(segmentName);
-      Assert.assertEquals(instanceStateMap.size(), updatedInstanceStateMap.size());
-      Assert.assertTrue(instanceStateMap.keySet().containsAll(updatedInstanceStateMap.keySet()));
-      for (Map.Entry<String, String> instanceToState : instanceStateMap.entrySet()) {
-        Assert.assertEquals(instanceToState.getValue(), updatedInstanceStateMap.get(instanceToState.getKey()));
+      assertTrue(instanceStatesMap.containsKey(segmentName));
+      assertEquals(instanceStatesMap.get(segmentName), entry.getValue());
+      assertTrue(oldSegmentZKMetadataMap.containsKey(segmentName));
+      assertTrue(segmentZKMetadataMap.containsKey(segmentName));
+      assertEquals(segmentZKMetadataMap.get(segmentName), oldSegmentZKMetadataMap.get(segmentName));
+      oldNumPartitions = Math.max(oldNumPartitions, new LLCSegmentName(segmentName).getPartitionId() + 1);
+    }
+
+    // Check that for new partitions, each partition should have exactly 1 new segment in CONSUMING state, and metadata
+    // in IN_PROGRESS state
+    Map<Integer, List<String>> partitionIdToSegmentsMap = new HashMap<>();
+    for (Map.Entry<String, Map<String, String>> entry : instanceStatesMap.entrySet()) {
+      String segmentName = entry.getKey();
+      int partitionId = new LLCSegmentName(segmentName).getPartitionId();
+      partitionIdToSegmentsMap.computeIfAbsent(partitionId, k -> new ArrayList<>()).add(segmentName);
+    }
+    for (int partitionId = oldNumPartitions; partitionId < segmentManager._numPartitions; partitionId++) {
+      List<String> segments = partitionIdToSegmentsMap.get(partitionId);
+      assertEquals(segments.size(), 1);
+      String segmentName = segments.get(0);
+      assertFalse(oldInstanceStatesMap.containsKey(segmentName));
+      Map<String, String> instanceStateMap = instanceStatesMap.get(segmentName);
+      assertEquals(instanceStateMap.size(), segmentManager._numReplicas);
+      for (String state : instanceStateMap.values()) {
+        assertEquals(state, RealtimeSegmentOnlineOfflineStateModel.CONSUMING);
       }
-      Assert.assertEquals(oldMetadataMap.get(segmentName), updatedMetadataMap.get(segmentName));
-      LLCSegmentName llcSegmentName = new LLCSegmentName(segmentName);
-      oldPartitions.add(llcSegmentName.getPartitionId());
+      // NOTE: Old segment ZK metadata might exist when previous round failed due to not enough instances
+      assertTrue(segmentZKMetadataMap.containsKey(segmentName));
+      LLCRealtimeSegmentZKMetadata segmentZKMetadata = segmentZKMetadataMap.get(segmentName);
+      assertEquals(segmentZKMetadata.getStatus(), Status.IN_PROGRESS);
+      assertEquals(segmentZKMetadata.getStartOffset(), PARTITION_OFFSET);
+      assertEquals(segmentZKMetadata.getCreationTime(), CURRENT_TIME_MS);
     }
+  }
 
-    List<Integer> allPartitions = new ArrayList<>(nPartitions);
-    for (int p = 0; p < nPartitions; p++) {
-      allPartitions.add(p);
+  private Map<String, Map<String, String>> cloneInstanceStatesMap(Map<String, Map<String, String>> instanceStatesMap) {
+    Map<String, Map<String, String>> clone = new TreeMap<>();
+    for (Map.Entry<String, Map<String, String>> entry : instanceStatesMap.entrySet()) {
+      clone.put(entry.getKey(), new TreeMap<>(entry.getValue()));
     }
-    List<Integer> newPartitions = new ArrayList<>(allPartitions);
-    newPartitions.removeAll(oldPartitions);
+    return clone;
+  }
 
-    Map<Integer, List<String>> partitionToAllSegmentsMap = new HashMap<>(nPartitions);
-    for (Integer p : allPartitions) {
-      partitionToAllSegmentsMap.put(p, new ArrayList<String>());
-    }
-    for (Map.Entry<String, Map<String, String>> entry : updatedMapFields.entrySet()) {
-      String segmentName = entry.getKey();
-      LLCSegmentName llcSegmentName = new LLCSegmentName(segmentName);
-      int partitionId = llcSegmentName.getPartitionId();
-      partitionToAllSegmentsMap.get(partitionId).add(segmentName);
-    }
-
-    // if skipNewPartitions, new partitions should not have any assignment
-
-    // each new partition should have exactly 1 new segment in CONSUMING state, and metadata in IN_PROGRESS state
-    for (Integer partitionId : newPartitions) {
-      List<String> allSegmentsForPartition = partitionToAllSegmentsMap.get(partitionId);
-      if (!skipNewPartitions && idealState.isEnabled()) {
-        Assert.assertEquals(allSegmentsForPartition.size(), 1);
-        for (String segment : allSegmentsForPartition) {
-          Map<String, String> instanceStateMap = updatedMapFields.get(segment);
-          Assert.assertEquals(instanceStateMap.size(), nReplicas);
-          for (Map.Entry<String, String> entry : instanceStateMap.entrySet()) {
-            Assert.assertEquals(entry.getValue(), "CONSUMING");
-          }
-          LLCRealtimeSegmentZKMetadata newPartitionMetadata = updatedMetadataMap.get(segment);
-          // for newly added partitions, we expect offset to be the smallest even though the offset is set to be largest for these tables
-          Assert.assertEquals(newPartitionMetadata.getStartOffset(), segmentManager.getSmallestStreamOffset());
-
-          Assert.assertNotNull(newPartitionMetadata);
-          Assert.assertEquals(newPartitionMetadata.getStatus(), CommonConstants.Segment.Realtime.Status.IN_PROGRESS);
-        }
-      } else {
-        Assert.assertEquals(allSegmentsForPartition.size(), 0);
-      }
+  private Map<String, LLCRealtimeSegmentZKMetadata> cloneSegmentZKMetadataMap(
+      Map<String, LLCRealtimeSegmentZKMetadata> segmentZKMetadataMap) {
+    Map<String, LLCRealtimeSegmentZKMetadata> clone = new HashMap<>();
+    for (Map.Entry<String, LLCRealtimeSegmentZKMetadata> entry : segmentZKMetadataMap.entrySet()) {
+      clone.put(entry.getKey(), new LLCRealtimeSegmentZKMetadata(entry.getValue().toZNRecord()));
     }
+    return clone;
   }
 
   /**
-   * Tests that we can repair all invalid scenarios that can generate during segment completion which leave the ideal state in an incorrect state
+   * Tests that we can repair all invalid scenarios during segment completion.
+   *
    * Segment completion takes place in 3 steps:
-   * 1. Update committing segment metadata to DONE
-   * 2. Create new segment metadata IN_PROGRESS
-   * 3. Update ideal state as committing segment state ONLINE and create new segment with state CONSUMING
+   * 1. Update committing segment ZK metadata to status DONE
+   * 2. Create new segment ZK metadata with status IN_PROGRESS
+   * 3. Update ideal state (change committing segment state to ONLINE and create new segment with state CONSUMING)
    *
-   * If a failure happens before step 1 or after step 3, we do not need to fix it
-   * If a failure happens after step 1 is done and before step 3 completes, we will be left in an incorrect state, and should be able to fix it
+   * If a failure happens before step 1 or after step 3, we do not need to fix it.
+   * If a failure happens after step 1 is done and before step 3 completes, we will be left in an incorrect state, and
+   * should be able to fix it.
    *
    * Scenarios:
-   * 1. Step 3 failed - we will find new segment metadata IN_PROGRESS but no segment in ideal state
-   * Correction: create new segment in ideal state with CONSUMING, update prev segment (if exists) in ideal state to ONLINE
-   *
-   * 2. Step 2 failed - we will find segment metadata DONE but ideal state CONSUMING
-   * Correction: create new segment metadata with state IN_PROGRESS, create new segment in ideal state CONSUMING, update prev segment to ONLINE
-   *
-   * 3. All replicas of a latest segment are in OFFLINE
-   * Correction: create new metadata IN_PROGRESS, new segment in ideal state CONSUMING. If prev metadata was DONE, start from end offset. If prev metadata was IN_PROGRESS, start from prev start offset
+   * 1. Step 3 failed - we will find new segment ZK metadata IN_PROGRESS but no segment in ideal state
+   * Correction: create new CONSUMING segment in ideal state, update previous CONSUMING segment (if exists) in ideal
+   * state to ONLINE
    *
-   * 4. TooSoonToCorrect: Segment completion has 10 minutes to retry and complete between steps 1 and 3.
-   * Correction: Don't correct a segment before the allowed time
+   * 2. Step 2 failed - we will find segment ZK metadata DONE but ideal state CONSUMING
+   * Correction: create new segment ZK metadata with state IN_PROGRESS, create new CONSUMING segment in ideal state,
+   * update previous CONSUMING segment (if exists) in ideal state to ONLINE
    *
-   * 5. Num Partitions increased
-   * Correction: create new metadata and segment in ideal state for new partition
+   * 3. All replicas of the new segment are OFFLINE
+   * Correction: create new segment ZK metadata with state IN_PROGRESS and consume from the previous start offset,
+   * create new CONSUMING segment in ideal state.
    *
-   * 6. Num instances changed
-   * Correction: use updated instances for new segment assignments
-   *
-   * @throws InvalidConfigException
+   * 4. MaxSegmentCompletionTime: Segment completion has 5 minutes to retry and complete between steps 1 and 3.
+   * Correction: Do not correct the segments before the allowed time for segment completion
    */
   @Test
-  public void testValidateLLCRepair()
-      throws InvalidConfigException {
-
-    FakePinotLLCRealtimeSegmentManager segmentManager =
-        new FakePinotLLCRealtimeSegmentManager(_mockPinotHelixResourceManager, null);
-    String tableName = "repairThisTable_REALTIME";
-    String rawTableName = TableNameBuilder.extractRawTableName(tableName);
-    int nReplicas = 2;
-    IdealStateBuilderUtil idealStateBuilder = new IdealStateBuilderUtil(tableName);
-    ZNRecordSerializer znRecordSerializer = new ZNRecordSerializer();
-
-    TableConfig tableConfig;
-    IdealState idealState;
-    int nPartitions;
-    List<String> instances = getInstanceList(5);
-    final int[] numInstancesSet = new int[]{4, 6, 8, 10};
-
-    tableConfig = makeTableConfig(tableName, nReplicas, DUMMY_HOST, DEFAULT_SERVER_TENANT);
-    nPartitions = 4;
-    segmentManager._partitionAssignmentGenerator.setConsumingInstances(instances);
-    PartitionAssignment expectedPartitionAssignment =
-        segmentManager._partitionAssignmentGenerator.generateStreamPartitionAssignment(tableConfig, nPartitions);
-
-    // set up a happy path - segment is present in ideal state, is CONSUMING, metadata says IN_PROGRESS
-    idealState = clearAndSetupHappyPathIdealState(idealStateBuilder, segmentManager, tableConfig, nPartitions);
-
-    // randomly introduce an error condition and assert that we repair it
-    for (int run = 0; run < 200; run++) {
-
-      final boolean step3NotDone = random.nextBoolean();
-
-      final boolean tooSoonToCorrect = random.nextBoolean();
-      segmentManager.tooSoonToCorrect = false;
-      if (tooSoonToCorrect) {
-        segmentManager.tooSoonToCorrect = true;
-      }
-
-      if (step3NotDone) {
-        // failed after completing step 2: hence new segment not present in ideal state
-
-        final boolean prevMetadataNull = random.nextBoolean();
-
-        if (prevMetadataNull) {
-          // very first seq number of this partition i.e. prev metadata null
-          clearAndSetupInitialSegments(idealStateBuilder, segmentManager, tableConfig, nPartitions);
-
-          // generate the error scenario - remove segment from ideal state, but metadata is IN_PROGRESS
-          int randomlySelectedPartition = random.nextInt(nPartitions);
-          Map<String, LLCSegmentName> partitionToLatestSegments =
-              segmentManager._partitionAssignmentGenerator.getPartitionToLatestSegments(idealState);
-          LLCSegmentName llcSegmentName = partitionToLatestSegments.get(String.valueOf(randomlySelectedPartition));
-          idealStateBuilder.removeSegment(llcSegmentName.getSegmentName());
-
-          // get old state
-          Assert.assertNull(idealState.getRecord().getMapFields().get(llcSegmentName.getSegmentName()));
-          nPartitions = expectedPartitionAssignment.getNumPartitions();
-          IdealState idealStateCopy = new IdealState(
-              (ZNRecord) znRecordSerializer.deserialize(znRecordSerializer.serialize(idealState.getRecord())));
-          Map<String, Map<String, String>> oldMapFields = idealStateCopy.getRecord().getMapFields();
-
-          if (tooSoonToCorrect) {
-            segmentManager.ensureAllPartitionsConsuming(tableConfig, idealState, nPartitions);
-            // validate that all entries in oldMapFields are unchanged in new ideal state
-            verifyNoChangeToOldEntries(oldMapFields, idealState);
-            segmentManager.tooSoonToCorrect = false;
-          }
-
-          segmentManager.ensureAllPartitionsConsuming(tableConfig, idealState, nPartitions);
-
-          // verify that new segment gets created in ideal state with CONSUMING
-          Assert.assertNotNull(idealState.getRecord().getMapFields().get(llcSegmentName.getSegmentName()));
-          Assert.assertNotNull(idealState.getRecord().getMapFields().get(llcSegmentName.getSegmentName()).values()
-              .contains("CONSUMING"));
-          Assert.assertNotNull(
-              segmentManager.getRealtimeSegmentZKMetadata(tableName, llcSegmentName.getSegmentName(), null));
-
-          verifyRepairs(tableConfig, idealState, expectedPartitionAssignment, segmentManager, oldMapFields);
-        } else {
-          // not the very first seq number in this partition i.e. prev metadata not null
-
-          // generate the error scenario - segment missing from ideal state, but metadata IN_PROGRESS
-          int randomlySelectedPartition = random.nextInt(nPartitions);
-          Map<String, LLCSegmentName> partitionToLatestSegments =
-              segmentManager._partitionAssignmentGenerator.getPartitionToLatestSegments(idealState);
-          LLCSegmentName latestSegment = partitionToLatestSegments.get(String.valueOf(randomlySelectedPartition));
-          LLCRealtimeSegmentZKMetadata latestMetadata =
-              segmentManager.getRealtimeSegmentZKMetadata(tableName, latestSegment.getSegmentName(), null);
-          LLCSegmentName newLlcSegmentName =
-              new LLCSegmentName(rawTableName, randomlySelectedPartition, latestSegment.getSequenceNumber() + 1,
-                  System.currentTimeMillis());
-          CommittingSegmentDescriptor committingSegmentDescriptor =
-              new CommittingSegmentDescriptor(latestSegment.getSegmentName(), latestMetadata.getStartOffset() + 100, 0);
-          segmentManager
-              .updateOldSegmentMetadataZNRecord(tableName, latestSegment, latestMetadata.getStartOffset() + 100,
-                  committingSegmentDescriptor);
-          segmentManager.createNewSegmentMetadataZNRecord(tableConfig, latestSegment, newLlcSegmentName,
-              expectedPartitionAssignment, committingSegmentDescriptor, false);
-
-          // get old state
-          Assert.assertNull(idealState.getRecord().getMapFields().get(newLlcSegmentName.getSegmentName()));
-          nPartitions = expectedPartitionAssignment.getNumPartitions();
-          IdealState idealStateCopy = new IdealState(
-              (ZNRecord) znRecordSerializer.deserialize(znRecordSerializer.serialize(idealState.getRecord())));
-          Map<String, Map<String, String>> oldMapFields = idealStateCopy.getRecord().getMapFields();
-
-          if (tooSoonToCorrect) {
-            segmentManager.ensureAllPartitionsConsuming(tableConfig, idealState, nPartitions);
-            // validate nothing changed and try again with disabled
-            verifyNoChangeToOldEntries(oldMapFields, idealState);
-            segmentManager.tooSoonToCorrect = false;
-          }
-
-          segmentManager.ensureAllPartitionsConsuming(tableConfig, idealState, nPartitions);
-
-          // verify that new segment gets created in ideal state with CONSUMING and old segment ONLINE
-          Assert.assertNotNull(idealState.getRecord().getMapFields().get(latestSegment.getSegmentName()).values().
-              contains("ONLINE"));
-          Assert.assertNotNull(idealState.getRecord().getMapFields().get(newLlcSegmentName.getSegmentName()));
-          Assert.assertNotNull(idealState.getRecord().getMapFields().get(newLlcSegmentName.getSegmentName()).values().
-              contains("CONSUMING"));
-          Assert.assertNotNull(
-              segmentManager.getRealtimeSegmentZKMetadata(tableName, latestSegment.getSegmentName(), null));
-          Assert.assertNotNull(
-              segmentManager.getRealtimeSegmentZKMetadata(tableName, newLlcSegmentName.getSegmentName(), null));
-
-          verifyRepairs(tableConfig, idealState, expectedPartitionAssignment, segmentManager, oldMapFields);
-        }
-      } else {
-
-        // latest segment metadata is present in ideal state
-
-        // segment is in OFFLINE state
-        final boolean allReplicasInOffline = random.nextBoolean();
-
-        if (allReplicasInOffline) {
-
-          // generate error scenario - all replicas in OFFLINE state
-          int randomlySelectedPartition = random.nextInt(nPartitions);
-          Map<String, LLCSegmentName> partitionToLatestSegments =
-              segmentManager._partitionAssignmentGenerator.getPartitionToLatestSegments(idealState);
-          LLCSegmentName latestSegment = partitionToLatestSegments.get(String.valueOf(randomlySelectedPartition));
-          idealState = idealStateBuilder.setSegmentState(latestSegment.getSegmentName(), "OFFLINE").build();
-
-          // get old state
-          nPartitions = expectedPartitionAssignment.getNumPartitions();
-          IdealState idealStateCopy = new IdealState(
-              (ZNRecord) znRecordSerializer.deserialize(znRecordSerializer.serialize(idealState.getRecord())));
-          Map<String, Map<String, String>> oldMapFields = idealStateCopy.getRecord().getMapFields();
-
-          segmentManager.ensureAllPartitionsConsuming(tableConfig, idealState, nPartitions);
-
-          verifyRepairs(tableConfig, idealState, expectedPartitionAssignment, segmentManager, oldMapFields);
-        } else {
-          final boolean allReplicasInOnline = random.nextBoolean();
-          if (allReplicasInOnline) {
-            // race condition occurred between retentionManager and segment commit.
-            // The metadata was marked to DONE, new metadata created IN_PROGRESS segment state ONLINE and new segment state CONSUMING.
-            // Retention manager saw the new segment metadata before the ideal state was updated, and hence marked segment for deletion
-            // After ideal state update, retention manager deleted the new segment from metadata and ideal state
-
-            // generate error scenario
-            int randomlySelectedPartition = random.nextInt(nPartitions);
-            Map<String, LLCSegmentName> partitionToLatestSegments =
-                segmentManager._partitionAssignmentGenerator.getPartitionToLatestSegments(idealState);
-            LLCSegmentName latestSegment = partitionToLatestSegments.get(String.valueOf(randomlySelectedPartition));
-            LLCRealtimeSegmentZKMetadata latestMetadata =
-                segmentManager.getRealtimeSegmentZKMetadata(tableName, latestSegment.getSegmentName(), null);
-            segmentManager
-                .updateOldSegmentMetadataZNRecord(tableName, latestSegment, latestMetadata.getStartOffset() + 100,
-                    new CommittingSegmentDescriptor(latestSegment.getSegmentName(),
-                        latestMetadata.getStartOffset() + 100, 0));
-            idealState = idealStateBuilder.setSegmentState(latestSegment.getSegmentName(), "ONLINE").build();
-
-            // get old state
-            nPartitions = expectedPartitionAssignment.getNumPartitions();
-            IdealState idealStateCopy = new IdealState(
-                (ZNRecord) znRecordSerializer.deserialize(znRecordSerializer.serialize(idealState.getRecord())));
-            Map<String, Map<String, String>> oldMapFields = idealStateCopy.getRecord().getMapFields();
-
-            if (tooSoonToCorrect) {
-              segmentManager.ensureAllPartitionsConsuming(tableConfig, idealState, nPartitions);
-              // validate nothing changed and try again with disabled
-              verifyNoChangeToOldEntries(oldMapFields, idealState);
-              segmentManager.tooSoonToCorrect = false;
-            }
-            segmentManager.ensureAllPartitionsConsuming(tableConfig, idealState, nPartitions);
-
-            verifyRepairs(tableConfig, idealState, expectedPartitionAssignment, segmentManager, oldMapFields);
-          } else {
-            // at least 1 replica of latest segments in ideal state is in CONSUMING state
-
-            final boolean step2NotDone = random.nextBoolean();
-
-            if (step2NotDone) {
-              // failed after step 1 : hence metadata is DONE but segment is still CONSUMING
-
-              // generate error scenario
-              int randomlySelectedPartition = random.nextInt(nPartitions);
-              Map<String, LLCSegmentName> partitionToLatestSegments =
-                  segmentManager._partitionAssignmentGenerator.getPartitionToLatestSegments(idealState);
-              LLCSegmentName latestSegment = partitionToLatestSegments.get(String.valueOf(randomlySelectedPartition));
-              LLCRealtimeSegmentZKMetadata latestMetadata =
-                  segmentManager.getRealtimeSegmentZKMetadata(tableName, latestSegment.getSegmentName(), null);
-              segmentManager
-                  .updateOldSegmentMetadataZNRecord(tableName, latestSegment, latestMetadata.getStartOffset() + 100,
-                      new CommittingSegmentDescriptor(latestSegment.getSegmentName(),
-                          latestMetadata.getStartOffset() + 100, 0));
-
-              // get old state
-              nPartitions = expectedPartitionAssignment.getNumPartitions();
-              IdealState idealStateCopy = new IdealState(
-                  (ZNRecord) znRecordSerializer.deserialize(znRecordSerializer.serialize(idealState.getRecord())));
-              Map<String, Map<String, String>> oldMapFields = idealStateCopy.getRecord().getMapFields();
-
-              if (tooSoonToCorrect) {
-                segmentManager.ensureAllPartitionsConsuming(tableConfig, idealState, nPartitions);
-                // validate nothing changed and try again with disabled
-                verifyNoChangeToOldEntries(oldMapFields, idealState);
-                segmentManager.tooSoonToCorrect = false;
-              }
-
-              segmentManager.ensureAllPartitionsConsuming(tableConfig, idealState, nPartitions);
-
-              verifyRepairs(tableConfig, idealState, expectedPartitionAssignment, segmentManager, oldMapFields);
-            } else {
-              // happy path
-
-              // get old state
-              nPartitions = expectedPartitionAssignment.getNumPartitions();
-              IdealState idealStateCopy = new IdealState(
-                  (ZNRecord) znRecordSerializer.deserialize(znRecordSerializer.serialize(idealState.getRecord())));
-              Map<String, Map<String, String>> oldMapFields = idealStateCopy.getRecord().getMapFields();
-
-              segmentManager.ensureAllPartitionsConsuming(tableConfig, idealState, nPartitions);
-
-              // verify that nothing changed
-              verifyNoChangeToOldEntries(oldMapFields, idealState);
-
-              verifyRepairs(tableConfig, idealState, expectedPartitionAssignment, segmentManager, oldMapFields);
-            }
-          }
-        }
-      }
+  public void testRepairs() {
+    // Set up a new table with 2 replicas, 5 instances, 4 partitions
+    FakePinotLLCRealtimeSegmentManager segmentManager = new FakePinotLLCRealtimeSegmentManager();
+    setUpNewTable(segmentManager, 2, 5, 4);
+    Map<String, Map<String, String>> instanceStatesMap = segmentManager._idealState.getRecord().getMapFields();
+
+    // Remove the CONSUMING segment from the ideal state for partition 0 (step 3 failed)
+    String consumingSegment = new LLCSegmentName(RAW_TABLE_NAME, 0, 0, CURRENT_TIME_MS).getSegmentName();
+    removeNewConsumingSegment(instanceStatesMap, consumingSegment, null);
+    testRepairs(segmentManager);
+
+    // Remove the CONSUMING segment from the ideal state and segment ZK metadata map for partition 0 (step 2 failed)
+    removeNewConsumingSegment(instanceStatesMap, consumingSegment, null);
+    assertNotNull(segmentManager._segmentZKMetadataMap.remove(consumingSegment));
+    testRepairs(segmentManager);
+
+    // 2 partitions commit segment
+    for (int partitionId = 0; partitionId < 2; partitionId++) {
+      String segmentName = new LLCSegmentName(RAW_TABLE_NAME, partitionId, 0, CURRENT_TIME_MS).getSegmentName();
+      CommittingSegmentDescriptor committingSegmentDescriptor =
+          new CommittingSegmentDescriptor(segmentName, PARTITION_OFFSET + NUM_DOCS, 0L);
+      committingSegmentDescriptor.setSegmentMetadata(mockSegmentMetadata());
+      segmentManager.commitSegmentMetadata(REALTIME_TABLE_NAME, committingSegmentDescriptor);
+    }
+
+    // Remove the CONSUMING segment from the ideal state for partition 0 (step 3 failed)
+    consumingSegment = new LLCSegmentName(RAW_TABLE_NAME, 0, 1, CURRENT_TIME_MS).getSegmentName();
+    String latestCommittedSegment = new LLCSegmentName(RAW_TABLE_NAME, 0, 0, CURRENT_TIME_MS).getSegmentName();
+    removeNewConsumingSegment(instanceStatesMap, consumingSegment, latestCommittedSegment);
+    testRepairs(segmentManager);
+
+    // Remove the CONSUMING segment from the ideal state and segment ZK metadata map for partition 0 (step 2 failed)
+    removeNewConsumingSegment(instanceStatesMap, consumingSegment, latestCommittedSegment);
+    assertNotNull(segmentManager._segmentZKMetadataMap.remove(consumingSegment));
+    testRepairs(segmentManager);
+
+    /*
+      Test all replicas of the new segment are OFFLINE
+     */
+
+    // Set up a new table with 2 replicas, 5 instances, 4 partitions
+    segmentManager = new FakePinotLLCRealtimeSegmentManager();
+    setUpNewTable(segmentManager, 2, 5, 4);
+    instanceStatesMap = segmentManager._idealState.getRecord().getMapFields();
+
+    // Turn all the replicas for the CONSUMING segment to OFFLINE for partition 0
+    consumingSegment = new LLCSegmentName(RAW_TABLE_NAME, 0, 0, CURRENT_TIME_MS).getSegmentName();
+    turnNewConsumingSegmentOffline(instanceStatesMap, consumingSegment);
+    testRepairs(segmentManager);
+
+    // Turn all the replicas for the CONSUMING segment to OFFLINE for partition 0 again
+    consumingSegment = new LLCSegmentName(RAW_TABLE_NAME, 0, 1, CURRENT_TIME_MS).getSegmentName();
+    turnNewConsumingSegmentOffline(instanceStatesMap, consumingSegment);
+    testRepairs(segmentManager);
+
+    // 2 partitions commit segment
+    for (int partitionId = 0; partitionId < 2; partitionId++) {
+      // Sequence number is 2 for partition 0 because segment 0 and 1 are OFFLINE
+      int sequenceNumber = partitionId == 0 ? 2 : 0;
+      String segmentName =
+          new LLCSegmentName(RAW_TABLE_NAME, partitionId, sequenceNumber, CURRENT_TIME_MS).getSegmentName();
+      CommittingSegmentDescriptor committingSegmentDescriptor =
+          new CommittingSegmentDescriptor(segmentName, PARTITION_OFFSET + NUM_DOCS, 0L);
+      committingSegmentDescriptor.setSegmentMetadata(mockSegmentMetadata());
+      segmentManager.commitSegmentMetadata(REALTIME_TABLE_NAME, committingSegmentDescriptor);
+    }
+
+    // Remove the CONSUMING segment from the ideal state for partition 0 (step 3 failed)
+    consumingSegment = new LLCSegmentName(RAW_TABLE_NAME, 0, 3, CURRENT_TIME_MS).getSegmentName();
+    latestCommittedSegment = new LLCSegmentName(RAW_TABLE_NAME, 0, 2, CURRENT_TIME_MS).getSegmentName();
+    removeNewConsumingSegment(instanceStatesMap, consumingSegment, latestCommittedSegment);
+    testRepairs(segmentManager);
+
+    // Remove the CONSUMING segment from the ideal state and segment ZK metadata map for partition 0 (step 2 failed)
+    removeNewConsumingSegment(instanceStatesMap, consumingSegment, latestCommittedSegment);
+    assertNotNull(segmentManager._segmentZKMetadataMap.remove(consumingSegment));
+    testRepairs(segmentManager);
+
+    // Turn all the replicas for the CONSUMING segment to OFFLINE for partition 0
+    consumingSegment = new LLCSegmentName(RAW_TABLE_NAME, 0, 3, CURRENT_TIME_MS).getSegmentName();
+    turnNewConsumingSegmentOffline(instanceStatesMap, consumingSegment);
+    testRepairs(segmentManager);
+
+    // Turn all the replicas for the CONSUMING segment to OFFLINE for partition 0 again
+    consumingSegment = new LLCSegmentName(RAW_TABLE_NAME, 0, 4, CURRENT_TIME_MS).getSegmentName();
+    turnNewConsumingSegmentOffline(instanceStatesMap, consumingSegment);
+    testRepairs(segmentManager);
+  }
 
-      // randomly change instances used, or increase partition count
-      double randomExternalChange = random.nextDouble();
-      if (randomExternalChange <= 0.2) { // introduce an external change for 20% runs
-        ExternalChange externalChange = ExternalChange.values()[random.nextInt(ExternalChange.values().length)];
-        int numInstances;
-        switch (externalChange) {
-          case N_INSTANCES_CHANGED:
-            // randomly change instances and hence set a new expected partition assignment
-            numInstances = numInstancesSet[random.nextInt(numInstancesSet.length)];
-            instances = getInstanceList(numInstances);
-            segmentManager._partitionAssignmentGenerator.setConsumingInstances(instances);
-            expectedPartitionAssignment = segmentManager._partitionAssignmentGenerator
-                .generateStreamPartitionAssignment(tableConfig, nPartitions);
-            break;
-          case N_PARTITIONS_INCREASED:
-            // randomly increase partitions and hence set a new expected partition assignment
-            expectedPartitionAssignment = segmentManager._partitionAssignmentGenerator
-                .generateStreamPartitionAssignment(tableConfig, nPartitions + 1);
-            break;
-          case N_INSTANCES_CHANGED_AND_PARTITIONS_INCREASED:
-            // both changed and hence set a new expected partition assignment
-            numInstances = numInstancesSet[random.nextInt(numInstancesSet.length)];
-            instances = getInstanceList(numInstances);
-            segmentManager._partitionAssignmentGenerator.setConsumingInstances(instances);
-            expectedPartitionAssignment = segmentManager._partitionAssignmentGenerator
-                .generateStreamPartitionAssignment(tableConfig, nPartitions + 1);
-            break;
-        }
+  /**
+   * Removes the new CONSUMING segment and sets the latest committed (ONLINE) segment to CONSUMING if exists in the
+   * ideal state.
+   */
+  private void removeNewConsumingSegment(Map<String, Map<String, String>> instanceStatesMap, String consumingSegment,
+      @Nullable String latestCommittedSegment) {
+    // Consuming segment should have all instances in CONSUMING state
+    Map<String, String> consumingSegmentInstanceStateMap = instanceStatesMap.remove(consumingSegment);
+    assertNotNull(consumingSegmentInstanceStateMap);
+    assertEquals(new HashSet<>(consumingSegmentInstanceStateMap.values()),
+        Collections.singleton(RealtimeSegmentOnlineOfflineStateModel.CONSUMING));
+
+    if (latestCommittedSegment != null) {
+      Map<String, String> latestCommittedSegmentInstanceStateMap = instanceStatesMap.get(latestCommittedSegment);
+      assertNotNull(latestCommittedSegmentInstanceStateMap);
+      for (Map.Entry<String, String> entry : latestCommittedSegmentInstanceStateMap.entrySet()) {
+        // Latest committed segment should have all instances in ONLINE state
+        assertEquals(entry.getValue(), RealtimeSegmentOnlineOfflineStateModel.ONLINE);
+        entry.setValue(RealtimeSegmentOnlineOfflineStateModel.CONSUMING);
       }
     }
   }
 
   /**
-   * Verifies that all entries in old ideal state are unchanged in the new ideal state
-   * There could be new entries in the ideal state due to num partitions increase
-   * @param oldMapFields
-   * @param idealState
+   * Turns all instances for the new CONSUMING segment to OFFLINE in the ideal state.
    */
-  private void verifyNoChangeToOldEntries(Map<String, Map<String, String>> oldMapFields, IdealState idealState) {
-    Map<String, Map<String, String>> newMapFields = idealState.getRecord().getMapFields();
-    for (Map.Entry<String, Map<String, String>> oldMapFieldsEntry : oldMapFields.entrySet()) {
-      String oldSegment = oldMapFieldsEntry.getKey();
-      Map<String, String> oldInstanceStateMap = oldMapFieldsEntry.getValue();
-      Assert.assertTrue(newMapFields.containsKey(oldSegment));
-      Assert.assertTrue(oldInstanceStateMap.equals(newMapFields.get(oldSegment)));
+  private void turnNewConsumingSegmentOffline(Map<String, Map<String, String>> instanceStatesMap,
+      String consumingSegment) {
+    Map<String, String> consumingSegmentInstanceStateMap = instanceStatesMap.get(consumingSegment);
+    assertNotNull(consumingSegmentInstanceStateMap);
+    for (Map.Entry<String, String> entry : consumingSegmentInstanceStateMap.entrySet()) {
+      // Consuming segment should have all instances in CONSUMING state
+      assertEquals(entry.getValue(), RealtimeSegmentOnlineOfflineStateModel.CONSUMING);
+      entry.setValue(RealtimeSegmentOnlineOfflineStateModel.OFFLINE);
     }
   }
 
-  private void verifyRepairs(TableConfig tableConfig, IdealState idealState,
-      PartitionAssignment expectedPartitionAssignment, FakePinotLLCRealtimeSegmentManager segmentManager,
-      Map<String, Map<String, String>> oldMapFields) {
-
-    Map<String, Map<String, String>> mapFields = idealState.getRecord().getMapFields();
-
-    // latest metadata for each partition
-    Map<Integer, LLCRealtimeSegmentZKMetadata[]> latestMetadata =
-        segmentManager.getLatestMetadata(tableConfig.getTableName());
-
-    // latest segment in ideal state for each partition
-    Map<String, LLCSegmentName> partitionToLatestSegments =
-        segmentManager._partitionAssignmentGenerator.getPartitionToLatestSegments(idealState);
-
-    // Assert that we have a latest metadata and segment for each partition
-    List<String> latestSegmentNames = new ArrayList<>();
-    for (int p = 0; p < expectedPartitionAssignment.getNumPartitions(); p++) {
-      LLCRealtimeSegmentZKMetadata[] llcRealtimeSegmentZKMetadata = latestMetadata.get(p);
-      LLCSegmentName latestLlcSegment = partitionToLatestSegments.get(String.valueOf(p));
-      Assert.assertNotNull(llcRealtimeSegmentZKMetadata);
-      Assert.assertNotNull(llcRealtimeSegmentZKMetadata[0]);
-      Assert.assertNotNull(latestLlcSegment);
-      Assert.assertEquals(latestLlcSegment.getSegmentName(), llcRealtimeSegmentZKMetadata[0].getSegmentName());
-      latestSegmentNames.add(latestLlcSegment.getSegmentName());
+  private void testRepairs(FakePinotLLCRealtimeSegmentManager segmentManager) {
+    Map<String, Map<String, String>> oldInstanceStatesMap =
+        cloneInstanceStatesMap(segmentManager._idealState.getRecord().getMapFields());
+    segmentManager._exceededMaxSegmentCompletionTime = false;
+    segmentManager.ensureAllPartitionsConsuming();
+    verifyNoChangeToOldEntries(segmentManager, oldInstanceStatesMap);
+    segmentManager._exceededMaxSegmentCompletionTime = true;
+    segmentManager.ensureAllPartitionsConsuming();
+    verifyRepairs(segmentManager);
+  }
+
+  /**
+   * Verifies that all entries in old ideal state are unchanged in the new ideal state (repair during the segment
+   * completion). There could be new entries in the ideal state if all instances are OFFLINE for the latest segment.
+   */
+  private void verifyNoChangeToOldEntries(FakePinotLLCRealtimeSegmentManager segmentManager,
+      Map<String, Map<String, String>> oldInstanceStatesMap) {
+    Map<String, Map<String, String>> newInstanceStatesMap = segmentManager._idealState.getRecord().getMapFields();
+    for (Map.Entry<String, Map<String, String>> entry : oldInstanceStatesMap.entrySet()) {
+      String segmentName = entry.getKey();
+      assertTrue(newInstanceStatesMap.containsKey(segmentName));
+      assertEquals(newInstanceStatesMap.get(segmentName), entry.getValue());
     }
+  }
+
+  private void verifyRepairs(FakePinotLLCRealtimeSegmentManager segmentManager) {
+    Map<String, Map<String, String>> instanceStatesMap = segmentManager._idealState.getRecord().getMapFields();
 
-    // if it is latest, ideal state should have state CONSUMING. metadata should have status IN_PROGRESS
-    for (Map.Entry<String, Map<String, String>> entry : mapFields.entrySet()) {
+    // Segments are the same for ideal state and ZK metadata
+    assertEquals(instanceStatesMap.keySet(), segmentManager._segmentZKMetadataMap.keySet());
+
+    // Gather the ONLINE/CONSUMING segments for each partition ordered by sequence number
+    List<Map<Integer, String>> partitionIdToSegmentsMap = new ArrayList<>(segmentManager._numPartitions);
+    for (int partitionId = 0; partitionId < segmentManager._numPartitions; partitionId++) {
+      partitionIdToSegmentsMap.add(new TreeMap<>());
+    }
+    for (Map.Entry<String, Map<String, String>> entry : instanceStatesMap.entrySet()) {
       String segmentName = entry.getKey();
       Map<String, String> instanceStateMap = entry.getValue();
-      if (latestSegmentNames.contains(segmentName)) {
-        for (Map.Entry<String, String> instanceState : instanceStateMap.entrySet()) {
-          Assert.assertEquals(instanceState.getValue(), "CONSUMING");
-        }
-        LLCRealtimeSegmentZKMetadata llcRealtimeSegmentZKMetadata = segmentManager._metadataMap.get(segmentName);
-        Assert.assertEquals(llcRealtimeSegmentZKMetadata.getStatus(),
-            CommonConstants.Segment.Realtime.Status.IN_PROGRESS);
-      } else {
-        for (Map.Entry<String, String> instanceState : instanceStateMap.entrySet()) {
-          Assert.assertTrue(instanceState.getValue().equals("ONLINE") || instanceState.getValue().equals("OFFLINE"));
-        }
-      }
-    }
 
-    // newer partitions and new segments will follow new partition assignment.
-    // older segments will follow old partition assignment
-    Set<String> oldSegmentsSet = oldMapFields.keySet();
-    Set<String> newSegmentsSet = mapFields.keySet();
-    Set<String> difference = new HashSet<>(newSegmentsSet);
-    difference.removeAll(oldSegmentsSet);
-    for (String addedSegment : difference) {
-      if (LLCSegmentName.isLowLevelConsumerSegmentName(addedSegment)) {
-        LLCSegmentName segmentName = new LLCSegmentName(addedSegment);
-        Set<String> actualInstances = mapFields.get(addedSegment).keySet();
-        List<String> expectedInstances =
-            expectedPartitionAssignment.getInstancesListForPartition(String.valueOf(segmentName.getPartitionId()));
-        Assert.assertEquals(actualInstances.size(), expectedInstances.size());
-        Assert.assertTrue(actualInstances.containsAll(expectedInstances));
+      // Skip segments with all instances OFFLINE
+      if (instanceStateMap.containsValue(RealtimeSegmentOnlineOfflineStateModel.ONLINE) || instanceStateMap
+          .containsValue(RealtimeSegmentOnlineOfflineStateModel.CONSUMING)) {
+        LLCSegmentName llcSegmentName = new LLCSegmentName(segmentName);
+        int partitionsId = llcSegmentName.getPartitionId();
+        Map<Integer, String> sequenceNumberToSegmentMap = partitionIdToSegmentsMap.get(partitionsId);
+        int sequenceNumber = llcSegmentName.getSequenceNumber();
+        assertFalse(sequenceNumberToSegmentMap.containsKey(sequenceNumber));
+        sequenceNumberToSegmentMap.put(sequenceNumber, segmentName);
       }
     }
-    for (String segment : oldSegmentsSet) {
-      Set<String> expectedInstances = oldMapFields.get(segment).keySet();
-      Set<String> actualInstances = mapFields.get(segment).keySet();
-      Assert.assertEquals(actualInstances.size(), expectedInstances.size());
-      Assert.assertTrue(actualInstances.containsAll(expectedInstances));
-    }
-  }
 
-  private IdealState clearAndSetupHappyPathIdealState(IdealStateBuilderUtil idealStateBuilder,
-      FakePinotLLCRealtimeSegmentManager segmentManager, TableConfig tableConfig, int nPartitions) {
-    IdealState idealState = idealStateBuilder.clear().build();
-    segmentManager._metadataMap.clear();
-
-    segmentManager.ensureAllPartitionsConsuming(tableConfig, idealState, nPartitions);
-    PartitionAssignment partitionAssignment = segmentManager._partitionAssignmentGenerator
-        .getStreamPartitionAssignmentFromIdealState(tableConfig, idealState);
-    for (int p = 0; p < nPartitions; p++) {
-      String segmentName = idealStateBuilder.getSegment(p, 0);
-      advanceASeqForPartition(idealState, segmentManager, partitionAssignment, segmentName, p, 1, 100, tableConfig);
-    }
-    return idealStateBuilder.build();
-  }
+    for (int partitionId = 0; partitionId < segmentManager._numPartitions; partitionId++) {
+      List<String> segments = new ArrayList<>(partitionIdToSegmentsMap.get(partitionId).values());
+      assertFalse(segments.isEmpty());
+      int numSegments = segments.size();
 
-  private IdealState clearAndSetupInitialSegments(IdealStateBuilderUtil idealStateBuilder,
-      FakePinotLLCRealtimeSegmentManager segmentManager, TableConfig tableConfig, int nPartitions) {
-    IdealState idealState = idealStateBuilder.clear().build();
-    segmentManager._metadataMap.clear();
-    segmentManager.ensureAllPartitionsConsuming(tableConfig, idealState, nPartitions);
-    return idealStateBuilder.build();
-  }
+      String latestSegment = segments.get(numSegments - 1);
 
-  @Test
-  public void testPreExistingSegments()
-      throws InvalidConfigException {
-    LLCSegmentName existingSegmentName = new LLCSegmentName("someTable", 1, 31, 12355L);
-    String[] existingSegs = {existingSegmentName.getSegmentName()};
-    FakePinotLLCRealtimeSegmentManager segmentManager =
-        new FakePinotLLCRealtimeSegmentManager(_mockPinotHelixResourceManager, Arrays.asList(existingSegs));
-
-    final String rtTableName = "testPreExistingLLCSegments_REALTIME";
-
-    TableConfig tableConfig = makeTableConfig(rtTableName, 3, DUMMY_HOST, DEFAULT_SERVER_TENANT);
-    segmentManager.addTableToStore(rtTableName, tableConfig, 8);
-    IdealState idealState = PinotTableIdealStateBuilder.buildEmptyRealtimeIdealStateFor(rtTableName, 10, true);
-    try {
-      segmentManager.setupNewTable(tableConfig, idealState);
-      Assert.fail("Did not get expected exception when setting up new table with existing segments in ");
-    } catch (RuntimeException e) {
-      // Expected
-    }
-  }
+      // Latest segment should have CONSUMING instance but no ONLINE instance in ideal state
+      Map<String, String> instanceStateMap = instanceStatesMap.get(latestSegment);
+      assertTrue(instanceStateMap.containsValue(RealtimeSegmentOnlineOfflineStateModel.CONSUMING));
+      assertFalse(instanceStateMap.containsValue(RealtimeSegmentOnlineOfflineStateModel.ONLINE));
 
-  // Make sure that if we are either not leader or we are disconnected, we do not process metadata commit.
-  @Test
-  public void testCommittingSegmentIfDisconnected()
-      throws InvalidConfigException {
-    FakePinotLLCRealtimeSegmentManager segmentManager =
-        new FakePinotLLCRealtimeSegmentManager(_mockPinotHelixResourceManager, null);
-
-    final String tableName = "table_REALTIME";
-    final String rawTableName = TableNameBuilder.extractRawTableName(tableName);
-    final int nInstances = 6;
-    final int nPartitions = 16;
-    final int nReplicas = 3;
-    List<String> instances = getInstanceList(nInstances);
-    SegmentCompletionProtocol.Request.Params reqParams = new SegmentCompletionProtocol.Request.Params();
-    TableConfig tableConfig = makeTableConfig(tableName, nReplicas, DUMMY_HOST, DEFAULT_SERVER_TENANT);
-
-    segmentManager.addTableToStore(tableName, tableConfig, nPartitions);
-
-    IdealState idealState = PinotTableIdealStateBuilder.buildEmptyRealtimeIdealStateFor(tableName, nReplicas, true);
-    segmentManager._partitionAssignmentGenerator.setConsumingInstances(instances);
-    segmentManager.setupNewTable(tableConfig, idealState);
-    // Now commit the first segment of partition 6.
-    final int committingPartition = 6;
-    final long nextOffset = 3425666L;
-    LLCRealtimeSegmentZKMetadata committingSegmentMetadata =
-        new LLCRealtimeSegmentZKMetadata(segmentManager._records.get(committingPartition));
-    segmentManager._paths.clear();
-    segmentManager._records.clear();
-    segmentManager.IS_CONNECTED = false;
-    reqParams.withSegmentName(committingSegmentMetadata.getSegmentName()).withOffset(nextOffset);
-    CommittingSegmentDescriptor committingSegmentDescriptor = CommittingSegmentDescriptor
-        .fromSegmentCompletionReqParamsAndMetadata(reqParams, segmentManager.newMockSegmentMetadata());
-    boolean status = segmentManager.commitSegmentMetadata(rawTableName, committingSegmentDescriptor);
-    Assert.assertFalse(status);
-    Assert.assertEquals(segmentManager._nCallsToUpdateHelix, 0);  // Idealstate not updated
-    Assert.assertEquals(segmentManager._paths.size(), 0);   // propertystore not updated
-    segmentManager.IS_CONNECTED = true;
-    segmentManager.IS_LEADER = false;
-    status = segmentManager.commitSegmentMetadata(rawTableName, committingSegmentDescriptor);
-    Assert.assertFalse(status);
-    Assert.assertEquals(segmentManager._nCallsToUpdateHelix, 0);  // Idealstate not updated
-    Assert.assertEquals(segmentManager._paths.size(), 0);   // propertystore not updated
-    segmentManager.IS_LEADER = true;
-    status = segmentManager.commitSegmentMetadata(rawTableName, committingSegmentDescriptor);
-    Assert.assertTrue(status);
-    Assert.assertEquals(segmentManager._nCallsToUpdateHelix, 1);  // Idealstate updated
-    Assert.assertEquals(segmentManager._paths.size(), 2);   // propertystore updated
-  }
+      // Latest segment ZK metadata should be IN_PROGRESS
+      assertEquals(segmentManager._segmentZKMetadataMap.get(latestSegment).getStatus(), Status.IN_PROGRESS);
 
-  /**
-   * Method which tests segment completion
-   * We do not care for partition increases in this method
-   * Instances increasing will be accounted for
-   */
-  @Test
-  public void testCommittingSegment()
-      throws InvalidConfigException {
-    FakePinotLLCRealtimeSegmentManager segmentManager =
-        new FakePinotLLCRealtimeSegmentManager(_mockPinotHelixResourceManager, null);
-
-    final String rtTableName = "table_REALTIME";
-    final String rawTableName = TableNameBuilder.extractRawTableName(rtTableName);
-    int nInstances = 6;
-    final int nPartitions = 16;
-    final int nReplicas = 2;
-    List<String> instances = getInstanceList(nInstances);
-    SegmentCompletionProtocol.Request.Params reqParams = new SegmentCompletionProtocol.Request.Params();
-    TableConfig tableConfig = makeTableConfig(rtTableName, nReplicas, DUMMY_HOST, DEFAULT_SERVER_TENANT);
-
-    IdealStateBuilderUtil idealStateBuilder = new IdealStateBuilderUtil(rtTableName);
-    IdealState idealState = idealStateBuilder.setNumReplicas(nReplicas).build();
-
-    segmentManager.addTableToStore(rtTableName, tableConfig, nPartitions);
-    segmentManager._partitionAssignmentGenerator.setConsumingInstances(instances);
-    segmentManager.setupNewTable(tableConfig, idealState);
-    PartitionAssignment partitionAssignment =
-        segmentManager._partitionAssignmentGenerator.generateStreamPartitionAssignment(tableConfig, nPartitions);
-
-    // Happy path: committing segment has states {CONSUMING,CONSUMING}
-    int committingPartition = 6;
-    long nextOffset = 2500;
-    LLCRealtimeSegmentZKMetadata committingSegmentMetadata =
-        new LLCRealtimeSegmentZKMetadata(segmentManager._records.get(committingPartition));
-    segmentManager._paths.clear();
-    segmentManager._records.clear();
-    Set<String> prevInstances = idealState.getInstanceSet(committingSegmentMetadata.getSegmentName());
-    reqParams.withSegmentName(committingSegmentMetadata.getSegmentName()).withOffset(nextOffset);
-    CommittingSegmentDescriptor committingSegmentDescriptor = CommittingSegmentDescriptor
-        .fromSegmentCompletionReqParamsAndMetadata(reqParams, segmentManager.newMockSegmentMetadata());
-    boolean status = segmentManager.commitSegmentMetadata(rawTableName, committingSegmentDescriptor);
-    segmentManager.verifyMetadataInteractions();
-    Assert.assertTrue(status);
-    Assert.assertEquals(segmentManager._paths.size(), 2);
-    ZNRecord oldZnRec = segmentManager._records.get(0);
-    ZNRecord newZnRec = segmentManager._records.get(1);
-    testCommitSegmentEntries(segmentManager, committingSegmentMetadata, oldZnRec, newZnRec, prevInstances,
-        partitionAssignment, committingPartition);
-
-    // committing segment has states {OFFLINE, CONSUMING}
-    String latestSegment = newZnRec.getId();
-    Map<String, String> instanceStateMap = idealState.getInstanceStateMap(latestSegment);
-    for (Map.Entry<String, String> entry : instanceStateMap.entrySet()) {
-      entry.setValue("OFFLINE");
-      break;
+      for (int i = 0; i < numSegments - 1; i++) {
+        String segmentName = segments.get(i);
+
+        // Committed segment should have all instances in ONLINE state
+        instanceStateMap = instanceStatesMap.get(segmentName);
+        assertEquals(new HashSet<>(instanceStateMap.values()),
+            Collections.singleton(RealtimeSegmentOnlineOfflineStateModel.ONLINE));
+
+        // Committed segment ZK metadata should be DONE
+        LLCRealtimeSegmentZKMetadata segmentZKMetadata = segmentManager._segmentZKMetadataMap.get(segmentName);
+        assertEquals(segmentZKMetadata.getStatus(), Status.DONE);
+
+        // Verify segment start/end offset
+        assertEquals(segmentZKMetadata.getStartOffset(), PARTITION_OFFSET + i * (long) NUM_DOCS);
+        assertEquals(segmentZKMetadata.getEndOffset(),
+            segmentManager._segmentZKMetadataMap.get(segments.get(i + 1)).getStartOffset());
+      }
     }
-    nextOffset = 5000;
-    committingSegmentMetadata = new LLCRealtimeSegmentZKMetadata(newZnRec);
-    segmentManager._paths.clear();
-    segmentManager._records.clear();
-    prevInstances = idealState.getInstanceSet(committingSegmentMetadata.getSegmentName());
-    reqParams.withSegmentName(committingSegmentMetadata.getSegmentName()).withOffset(nextOffset);
-    committingSegmentDescriptor = CommittingSegmentDescriptor
-        .fromSegmentCompletionReqParamsAndMetadata(reqParams, segmentManager.newMockSegmentMetadata());
-    status = segmentManager.commitSegmentMetadata(rawTableName, committingSegmentDescriptor);
-    segmentManager.verifyMetadataInteractions();
-    Assert.assertTrue(status);
-    Assert.assertEquals(segmentManager._paths.size(), 2);
-    oldZnRec = segmentManager._records.get(0);
-    newZnRec = segmentManager._records.get(1);
-    testCommitSegmentEntries(segmentManager, committingSegmentMetadata, oldZnRec, newZnRec, prevInstances,
-        partitionAssignment, committingPartition);
-
-    // metadata for committing segment is already DONE
-    nextOffset = 7500;
-    segmentManager._metadataMap.get(newZnRec.getId()).setStatus(CommonConstants.Segment.Realtime.Status.DONE);
-
-    committingSegmentMetadata = new LLCRealtimeSegmentZKMetadata(newZnRec);
-    segmentManager._paths.clear();
-    segmentManager._records.clear();
-    reqParams.withSegmentName(committingSegmentMetadata.getSegmentName()).withOffset(nextOffset);
-    // We do not expect the segment metadata to be used. Thus reuse the current metadata.
-    committingSegmentDescriptor = CommittingSegmentDescriptor
-        .fromSegmentCompletionReqParamsAndMetadata(reqParams, segmentManager.getMockSegmentMetadata());
-    status = segmentManager.commitSegmentMetadata(rawTableName, committingSegmentDescriptor);
-    segmentManager.verifyMetadataInteractions();
-    Assert.assertFalse(status);
-
-    // failure after step 2: new metadata inprogress, no segment in ideal state
-    segmentManager._metadataMap.get(newZnRec.getId()).setStatus(CommonConstants.Segment.Realtime.Status.IN_PROGRESS);
-    String deleteSegment = newZnRec.getId();
-    Map<String, String> instanceStateMap1 = idealState.getInstanceStateMap(deleteSegment);
-    idealState = idealStateBuilder.removeSegment(deleteSegment).setSegmentState(oldZnRec.getId(), "CONSUMING").build();
-    nextOffset = 5000;
-    committingSegmentMetadata = new LLCRealtimeSegmentZKMetadata(oldZnRec);
-    segmentManager._paths.clear();
-    segmentManager._records.clear();
-    reqParams.withSegmentName(committingSegmentMetadata.getSegmentName()).withOffset(nextOffset);
-    // We do not expect the segment metadata to be used. Thus reuse the current metadata.
-    committingSegmentDescriptor = CommittingSegmentDescriptor
-        .fromSegmentCompletionReqParamsAndMetadata(reqParams, segmentManager.getMockSegmentMetadata());
-    status = segmentManager.commitSegmentMetadata(rawTableName, committingSegmentDescriptor);
-    segmentManager.verifyMetadataInteractions();
-    Assert.assertFalse(status);
-
-    // instances changed
-    idealStateBuilder.addSegment(deleteSegment, instanceStateMap1);
-    nInstances = 8;
-    instances = getInstanceList(nInstances);
-    segmentManager._partitionAssignmentGenerator.setConsumingInstances(instances);
-    segmentManager._paths.clear();
-    segmentManager._records.clear();
-    committingSegmentMetadata = new LLCRealtimeSegmentZKMetadata(newZnRec);
-    prevInstances = idealState.getInstanceSet(committingSegmentMetadata.getSegmentName());
-    reqParams.withSegmentName(committingSegmentMetadata.getSegmentName()).withOffset(nextOffset);
-    committingSegmentDescriptor = CommittingSegmentDescriptor
-        .fromSegmentCompletionReqParamsAndMetadata(reqParams, segmentManager.newMockSegmentMetadata());
-    status = segmentManager.commitSegmentMetadata(rawTableName, committingSegmentDescriptor);
-    segmentManager.verifyMetadataInteractions();
-    Assert.assertTrue(status);
-    Assert.assertEquals(segmentManager._paths.size(), 2);
-    oldZnRec = segmentManager._records.get(0);
-    newZnRec = segmentManager._records.get(1);
-    testCommitSegmentEntries(segmentManager, committingSegmentMetadata, oldZnRec, newZnRec, prevInstances,
-        partitionAssignment, committingPartition);
   }
 
-  private void testCommitSegmentEntries(FakePinotLLCRealtimeSegmentManager segmentManager,
-      LLCRealtimeSegmentZKMetadata committingSegmentMetadata, ZNRecord oldZnRec, ZNRecord newZnRec,
-      Set<String> prevInstances, PartitionAssignment partitionAssignment, int partition) {
-    // Get the old and new segment metadata and make sure that they are correct.
-
-    LLCRealtimeSegmentZKMetadata oldMetadata = new LLCRealtimeSegmentZKMetadata(oldZnRec);
-    LLCRealtimeSegmentZKMetadata newMetadata = new LLCRealtimeSegmentZKMetadata(newZnRec);
-
-    LLCSegmentName oldSegmentName = new LLCSegmentName(oldMetadata.getSegmentName());
-    LLCSegmentName newSegmentName = new LLCSegmentName(newMetadata.getSegmentName());
-
-    // Assert on propertystore entries
-    Assert.assertEquals(oldSegmentName.getSegmentName(), committingSegmentMetadata.getSegmentName());
-    Assert.assertEquals(newSegmentName.getPartitionId(), oldSegmentName.getPartitionId());
-    Assert.assertEquals(newSegmentName.getSequenceNumber(), oldSegmentName.getSequenceNumber() + 1);
-    Assert.assertEquals(oldMetadata.getStatus(), CommonConstants.Segment.Realtime.Status.DONE);
-    Assert.assertEquals(newMetadata.getStatus(), CommonConstants.Segment.Realtime.Status.IN_PROGRESS);
-    Assert.assertNotNull(oldMetadata.getDownloadUrl());
-    Assert.assertEquals(Long.valueOf(oldMetadata.getCrc()), Long.valueOf(FakePinotLLCRealtimeSegmentManager.CRC));
-    Assert.assertEquals(oldMetadata.getStartTime(), FakePinotLLCRealtimeSegmentManager.INTERVAL.getStartMillis());
-    Assert.assertEquals(oldMetadata.getEndTime(), FakePinotLLCRealtimeSegmentManager.INTERVAL.getEndMillis());
-    Assert.assertEquals(oldMetadata.getTotalRawDocs(), FakePinotLLCRealtimeSegmentManager.NUM_DOCS);
-    Assert.assertEquals(oldMetadata.getIndexVersion(), FakePinotLLCRealtimeSegmentManager.SEGMENT_VERSION);
-
-    // check ideal state
-    IdealState idealState = segmentManager._tableIdealState;
-    Set<String> currInstances = idealState.getInstanceSet(oldSegmentName.getSegmentName());
-    Assert.assertEquals(prevInstances.size(), currInstances.size());
-    Assert.assertTrue(prevInstances.containsAll(currInstances));
-    Set<String> newSegmentInstances = idealState.getInstanceSet(newSegmentName.getSegmentName());
-    List<String> expectedNewInstances = partitionAssignment.getInstancesListForPartition(String.valueOf(partition));
-    Assert.assertEquals(newSegmentInstances.size(), expectedNewInstances.size());
-    Assert.assertTrue(newSegmentInstances.containsAll(expectedNewInstances));
+  @Test(expectedExceptions = IllegalStateException.class)
+  public void testPreExistingSegments() {
+    FakePinotLLCRealtimeSegmentManager segmentManager = new FakePinotLLCRealtimeSegmentManager();
+    segmentManager._numReplicas = 2;
+    segmentManager.makeTableConfig();
+    segmentManager._numInstances = 5;
+    segmentManager.makeConsumingInstancePartitions();
+    segmentManager._numPartitions = 4;
+
+    String existingSegmentName = new LLCSegmentName(RAW_TABLE_NAME, 0, 0, CURRENT_TIME_MS).getSegmentName();
+    segmentManager._segmentZKMetadataMap.put(existingSegmentName, new LLCRealtimeSegmentZKMetadata());
+    segmentManager.setUpNewTable();
   }
 
   @Test
-  public void testCommitSegmentWhenControllerWentThroughGC()
-      throws InvalidConfigException {
-
+  public void testCommitSegmentWhenControllerWentThroughGC() {
+    // Set up a new table with 2 replicas, 5 instances, 4 partitions
     FakePinotLLCRealtimeSegmentManager segmentManager1 =
-        new FakePinotLLCRealtimeSegmentManager(_mockPinotHelixResourceManager, null);
+        new FakePinotLLCRealtimeSegmentManagerII(FakePinotLLCRealtimeSegmentManagerII.Scenario.ZK_VERSION_CHANGED);
+    setUpNewTable(segmentManager1, 2, 5, 4);
     FakePinotLLCRealtimeSegmentManager segmentManager2 =
-        new FakePinotLLCRealtimeSegmentManagerII(_mockPinotHelixResourceManager, null,
-            FakePinotLLCRealtimeSegmentManagerII.SCENARIO_1_ZK_VERSION_NUM_HAS_CHANGE);
-    FakePinotLLCRealtimeSegmentManager segmentManager3 =
-        new FakePinotLLCRealtimeSegmentManagerII(_mockPinotHelixResourceManager, null,
-            FakePinotLLCRealtimeSegmentManagerII.SCENARIO_2_METADATA_STATUS_HAS_CHANGE);
-
-    final String rtTableName = "table_REALTIME";
-    final String rawTableName = TableNameBuilder.extractRawTableName(rtTableName);
-    setupSegmentManager(segmentManager1, rtTableName);
-    setupSegmentManager(segmentManager2, rtTableName);
-    setupSegmentManager(segmentManager3, rtTableName);
-    // Now commit the first segment of partition 6.
-    final int committingPartition = 6;
-    final long nextOffset = 3425666L;
-    SegmentCompletionProtocol.Request.Params reqParams = new SegmentCompletionProtocol.Request.Params();
-    LLCRealtimeSegmentZKMetadata committingSegmentMetadata =
-        new LLCRealtimeSegmentZKMetadata(segmentManager2._records.get(committingPartition));
-
-    reqParams.withSegmentName(committingSegmentMetadata.getSegmentName()).withOffset(nextOffset);
-    CommittingSegmentDescriptor committingSegmentDescriptor = CommittingSegmentDescriptor
-        .fromSegmentCompletionReqParamsAndMetadata(reqParams, segmentManager1.newMockSegmentMetadata());
-    boolean status = segmentManager1.commitSegmentMetadata(rawTableName, committingSegmentDescriptor);
-    Assert.assertTrue(status);  // Committing segment metadata succeeded.
-
-    status = segmentManager2.commitSegmentMetadata(rawTableName, committingSegmentDescriptor);
-    Assert.assertFalse(status); // Committing segment metadata failed.
-
-    status = segmentManager3.commitSegmentMetadata(rawTableName, committingSegmentDescriptor);
-    Assert.assertFalse(status); // Committing segment metadata failed.
-  }
-
-  /**
-   * Tests the scenario where ideal state for new segment, was already updated by some external thread, after step 2 and before step 3 of segment commit.
-   * This can happen if step 3 (ideal state update) took longer than {@link PinotLLCRealtimeSegmentManager::MAX_SEGMENT_COMPLETION_TIME_MILLIS},
-   * making the segment eligible for repairs by {@link org.apache.pinot.controller.validation.RealtimeSegmentValidationManager}
-   * @throws InvalidConfigException
-   */
-  @Test
-  public void testIdealStateAlreadyUpdated()
-      throws InvalidConfigException {
-    FakePinotLLCRealtimeSegmentManager segmentManager =
-        new FakePinotLLCRealtimeSegmentManager(_mockPinotHelixResourceManager, null);
-    String tableNameWithType = "tableName_REALTIME";
-    String rawTableName = "tableName";
-    int nPartitions = 4;
-    int nReplicas = 2;
-    int nInstances = 3;
-    setupSegmentManager(segmentManager, tableNameWithType, nPartitions, nReplicas, nInstances);
-
-    IdealState idealState = segmentManager._tableIdealState;
-    TableConfig tableConfig = segmentManager._tableConfigStore.getTableConfig(tableNameWithType);
-    PartitionAssignment partitionAssignment = segmentManager._partitionAssignmentGenerator
-        .getStreamPartitionAssignmentFromIdealState(tableConfig, idealState);
-    int partitionId = 0;
-    int seq = 0;
-    IdealStateBuilderUtil idealStateBuilderUtil = new IdealStateBuilderUtil(idealState, tableNameWithType);
-    String currentSegmentId = idealStateBuilderUtil.getSegment(partitionId, seq);
-    LLCSegmentName newSegment = new LLCSegmentName(rawTableName, partitionId, ++seq, System.currentTimeMillis());
-    String newSegmentId = newSegment.getSegmentName();
-    ZNRecordSerializer znRecordSerializer = new ZNRecordSerializer();
-    IdealState idealStateCopy =
-        new IdealState((ZNRecord) znRecordSerializer.deserialize(znRecordSerializer.serialize(idealState.getRecord())));
-    segmentManager.updateIdealStateOnSegmentCompletion(idealState, currentSegmentId, newSegmentId, partitionAssignment);
-
-    // check ideal state updates
-    Assert.assertNotEquals(idealState, idealStateCopy);
-
-    // progress ideal state, but send update for old. Should not update
-    currentSegmentId = newSegmentId;
-    List<String> instances = idealStateBuilderUtil.getInstances(partitionId, seq);
-    idealStateBuilderUtil.setSegmentState(partitionId, seq, "ONLINE");
-    idealStateBuilderUtil.addConsumingSegment(partitionId, ++seq, nReplicas, instances);
-    newSegmentId = idealStateBuilderUtil.getSegment(partitionId, seq);
-    idealStateBuilderUtil.setSegmentState(partitionId, seq, "ONLINE");
-    idealStateBuilderUtil.addConsumingSegment(partitionId, ++seq, nReplicas, instances);
-    idealState = idealStateBuilderUtil.build();
-    idealStateCopy =
-        new IdealState((ZNRecord) znRecordSerializer.deserialize(znRecordSerializer.serialize(idealState.getRecord())));
-
-    segmentManager.updateIdealStateOnSegmentCompletion(idealState, currentSegmentId, newSegmentId, partitionAssignment);
-
-    // check no change
-    Assert.assertEquals(idealState, idealStateCopy);
-  }
-
-  private void setupSegmentManager(FakePinotLLCRealtimeSegmentManager segmentManager, String rtTableName,
-      int nPartitions, int nReplicas, int nInstances)
-      throws InvalidConfigException {
-
-    List<String> instances = getInstanceList(nInstances);
-    TableConfig tableConfig = makeTableConfig(rtTableName, nReplicas, DUMMY_HOST, DEFAULT_SERVER_TENANT);
-    IdealState idealState = PinotTableIdealStateBuilder.buildEmptyRealtimeIdealStateFor(rtTableName, nReplicas, true);
+        new FakePinotLLCRealtimeSegmentManagerII(FakePinotLLCRealtimeSegmentManagerII.Scenario.METADATA_STATUS_CHANGED);
+    setUpNewTable(segmentManager2, 2, 5, 4);
 
-    segmentManager.addTableToStore(rtTableName, tableConfig, nPartitions);
-    segmentManager._partitionAssignmentGenerator.setConsumingInstances(instances);
-    segmentManager.setupNewTable(tableConfig, idealState);
-  }
+    // Commit a segment for partition 0
+    String committingSegment = new LLCSegmentName(RAW_TABLE_NAME, 0, 0, CURRENT_TIME_MS).getSegmentName();
+    CommittingSegmentDescriptor committingSegmentDescriptor =
+        new CommittingSegmentDescriptor(committingSegment, PARTITION_OFFSET + NUM_DOCS, 0L);
+    committingSegmentDescriptor.setSegmentMetadata(mockSegmentMetadata());
 
-  private void setupSegmentManager(FakePinotLLCRealtimeSegmentManager segmentManager, String rtTableName)
-      throws InvalidConfigException {
-    final int nInstances = 6;
-    final int nPartitions = 16;
-    final int nReplicas = 3;
-    List<String> instances = getInstanceList(nInstances);
-    TableConfig tableConfig = makeTableConfig(rtTableName, nReplicas, DUMMY_HOST, DEFAULT_SERVER_TENANT);
-    IdealState idealState = PinotTableIdealStateBuilder.buildEmptyRealtimeIdealStateFor(rtTableName, nReplicas, true);
-
-    segmentManager.addTableToStore(rtTableName, tableConfig, nPartitions);
-    segmentManager._partitionAssignmentGenerator.setConsumingInstances(instances);
-    segmentManager.setupNewTable(tableConfig, idealState);
+    try {
+      segmentManager1.commitSegmentMetadata(REALTIME_TABLE_NAME, committingSegmentDescriptor);
+      fail();
+    } catch (IllegalStateException e) {
+      // Expected
+    }
+    try {
+      segmentManager2.commitSegmentMetadata(REALTIME_TABLE_NAME, committingSegmentDescriptor);
+      fail();
+    } catch (IllegalStateException e) {
+      // Expected
+    }
   }
 
   @Test
   public void testCommitSegmentFile()
       throws Exception {
-    PinotFSFactory.init(new PropertiesConfiguration());
-    PinotLLCRealtimeSegmentManager realtimeSegmentManager =
-        new FakePinotLLCRealtimeSegmentManager(_mockPinotHelixResourceManager, Collections.<String>emptyList());
-    String tableName = "fakeTable_REALTIME";
-    String segmentName = "segment";
-    String temporarySegmentLocation = SegmentCompletionUtils.generateSegmentFileName(segmentName);
-
-    File temporaryDirectory = new File(baseDir, tableName);
-    temporaryDirectory.mkdirs();
-    String segmentLocation = SCHEME + temporaryDirectory.toString() + "/" + temporarySegmentLocation;
-
-    FileUtils.write(new File(temporaryDirectory, temporarySegmentLocation), "temporary file contents");
+    PinotFSFactory.init(new BaseConfiguration());
+    File tableDir = new File(TEMP_DIR, RAW_TABLE_NAME);
+    String segmentName = new LLCSegmentName(RAW_TABLE_NAME, 0, 0, CURRENT_TIME_MS).getSegmentName();
+    String segmentFileName = SegmentCompletionUtils.generateSegmentFileName(segmentName);
+    File segmentFile = new File(tableDir, segmentFileName);
+    FileUtils.write(segmentFile, "temporary file contents");
+
+    FakePinotLLCRealtimeSegmentManager segmentManager = new FakePinotLLCRealtimeSegmentManager();
+    String segmentLocation = SCHEME + tableDir + "/" + segmentFileName;
     CommittingSegmentDescriptor committingSegmentDescriptor =
-        new CommittingSegmentDescriptor(segmentName, 100, 0, segmentLocation);
-    Assert.assertTrue(realtimeSegmentManager.commitSegmentFile(tableName, committingSegmentDescriptor));
+        new CommittingSegmentDescriptor(segmentName, PARTITION_OFFSET, 0, segmentLocation);
+    segmentManager.commitSegmentFile(REALTIME_TABLE_NAME, committingSegmentDescriptor);
+    assertFalse(segmentFile.exists());
   }
 
   @Test
   public void testSegmentAlreadyThereAndExtraneousFilesDeleted()
       throws Exception {
-    PinotFSFactory.init(new PropertiesConfiguration());
-    PinotLLCRealtimeSegmentManager realtimeSegmentManager =
-        new FakePinotLLCRealtimeSegmentManager(_mockPinotHelixResourceManager, Collections.<String>emptyList());
-    String tableName = "fakeTable_REALTIME";
-    String segmentName = "segment";
-
-    File temporaryDirectory = new File(baseDir, tableName);
-    temporaryDirectory.mkdirs();
-
-    String segmentFileLocation = SegmentCompletionUtils.generateSegmentFileName(segmentName);
-    String segmentLocation = SCHEME + temporaryDirectory + "/" + segmentFileLocation;
-    String extraSegmentFileLocation = SegmentCompletionUtils.generateSegmentFileName(segmentName);
-    String extraSegmentLocation = SCHEME + temporaryDirectory + "/" + extraSegmentFileLocation;
-    String otherSegmentNotToBeDeleted = "segmentShouldStay";
-    FileUtils.write(new File(temporaryDirectory, segmentFileLocation), "temporary file contents");
-    FileUtils.write(new File(temporaryDirectory, extraSegmentFileLocation), "temporary file contents");
-    FileUtils.write(new File(temporaryDirectory, otherSegmentNotToBeDeleted), "temporary file contents");
-    FileUtils.write(new File(temporaryDirectory, segmentName), "temporary file contents");
-    Assert.assertTrue(realtimeSegmentManager
-        .commitSegmentFile(tableName, new CommittingSegmentDescriptor(segmentName, 100, 0, segmentLocation)));
-    Assert.assertTrue(new File(temporaryDirectory, otherSegmentNotToBeDeleted).exists());
-    Assert.assertFalse(new File(temporaryDirectory, extraSegmentLocation).exists());
-  }
-
-  ////////////////////////////////////////////////////////////////////////////
-  // Fake makers
-  ///////////////////////////////////////////////////////////////////////////
-
-  private TableConfig makeTableConfig(String tableName, int nReplicas, String bootstrapHosts, String serverTenant) {
-    TableConfig mockTableConfig = mock(TableConfig.class);
-    when(mockTableConfig.getTableName()).thenReturn(tableName);
-    SegmentsValidationAndRetentionConfig mockValidationConfig = mock(SegmentsValidationAndRetentionConfig.class);
-    when(mockValidationConfig.getReplicasPerPartition()).thenReturn(Integer.toString(nReplicas));
-    when(mockValidationConfig.getReplicasPerPartitionNumber()).thenReturn(nReplicas);
-    when(mockTableConfig.getValidationConfig()).thenReturn(mockValidationConfig);
-
-    StreamConfig streamConfig = FakeStreamConfigUtils.getDefaultLowLevelStreamConfigs();
-    IndexingConfig mockIndexConfig = mock(IndexingConfig.class);
-    when(mockIndexConfig.getStreamConfigs()).thenReturn(streamConfig.getStreamConfigsMap());
-
-    when(mockTableConfig.getIndexingConfig()).thenReturn(mockIndexConfig);
-    TenantConfig mockTenantConfig = mock(TenantConfig.class);
-    when(mockTenantConfig.getServer()).thenReturn(serverTenant);
-    when(mockTableConfig.getTenantConfig()).thenReturn(mockTenantConfig);
-
-    return mockTableConfig;
+    PinotFSFactory.init(new BaseConfiguration());
+    File tableDir = new File(TEMP_DIR, RAW_TABLE_NAME);
+    String segmentName = new LLCSegmentName(RAW_TABLE_NAME, 0, 0, CURRENT_TIME_MS).getSegmentName();
+    String otherSegmentName = new LLCSegmentName(RAW_TABLE_NAME, 1, 0, CURRENT_TIME_MS).getSegmentName();
+    String segmentFileName = SegmentCompletionUtils.generateSegmentFileName(segmentName);
+    String extraSegmentFileName = SegmentCompletionUtils.generateSegmentFileName(segmentName);
+    String otherSegmentFileName = SegmentCompletionUtils.generateSegmentFileName(otherSegmentName);
+    File segmentFile = new File(tableDir, segmentFileName);
+    File extraSegmentFile = new File(tableDir, extraSegmentFileName);
+    File otherSegmentFile = new File(tableDir, otherSegmentFileName);
+    FileUtils.write(segmentFile, "temporary file contents");
+    FileUtils.write(extraSegmentFile, "temporary file contents");
+    FileUtils.write(otherSegmentFile, "temporary file contents");
+
+    FakePinotLLCRealtimeSegmentManager segmentManager = new FakePinotLLCRealtimeSegmentManager();
+    String segmentLocation = SCHEME + tableDir + "/" + segmentFileName;
+    CommittingSegmentDescriptor committingSegmentDescriptor =
+        new CommittingSegmentDescriptor(segmentName, PARTITION_OFFSET, 0, segmentLocation);
+    segmentManager.commitSegmentFile(REALTIME_TABLE_NAME, committingSegmentDescriptor);
+    assertFalse(segmentFile.exists());
+    assertFalse(extraSegmentFile.exists());
+    assertTrue(otherSegmentFile.exists());
   }
 
-  //////////////////////////////////////////////////////////////////////////////////
-  // Fake classes
-  /////////////////////////////////////////////////////////////////////////////////
-
-  static class FakePinotLLCRealtimeSegmentManager extends PinotLLCRealtimeSegmentManager {
-
-    private static final ControllerConf CONTROLLER_CONF = new ControllerConf();
-    private List<String> _existingLLCSegments = new ArrayList<>(1);
-
-    public List<String> _paths = new ArrayList<>(16);
-    public List<ZNRecord> _records = new ArrayList<>(16);
-    public Map<String, LLCRealtimeSegmentZKMetadata> _metadataMap = new HashMap<>(4);
-    private FakeStreamPartitionAssignmentGenerator _partitionAssignmentGenerator;
-
-    public static final long _largestOffsetToReturn = Integer.MAX_VALUE;
-    public static final long _smallestOffsetToReturn = 0;
-
-    public int _nCallsToUpdateHelix = 0;
-    public int _nCallsToCreateNewSegmentMetadata = 0;
-    public IdealState _tableIdealState;
-    public String _currentTable;
-
-    public static final String CRC = "5680988776500";
-    public static final Interval INTERVAL = new Interval(3000, 4000);
-    public static final String SEGMENT_VERSION = SegmentVersion.v1.toString();
-    public static final int NUM_DOCS = 5099775;
-    public static boolean IS_LEADER = true;
-    public static boolean IS_CONNECTED = true;
-    public boolean tooSoonToCorrect = false;
-
-    public int _version;
-
-    private SegmentMetadataImpl segmentMetadata;
-
-    private TableConfigStore _tableConfigStore;
-
-    protected FakePinotLLCRealtimeSegmentManager(PinotHelixResourceManager pinotHelixResourceManager,
-        List<String> existingLLCSegments) {
-      this(pinotHelixResourceManager, existingLLCSegments, new ControllerMetrics(new MetricsRegistry()));
+  @Test
+  public void testStopSegmentManager()
+      throws Exception {
+    FakePinotLLCRealtimeSegmentManager segmentManager = new FakePinotLLCRealtimeSegmentManager();
+    segmentManager._numReplicas = 2;
+    segmentManager.makeTableConfig();
+    segmentManager._numInstances = 5;
+    segmentManager.makeConsumingInstancePartitions();
+    segmentManager._numPartitions = 4;
+    segmentManager.stop();
+
+    // All operations should fail after stopping the segment manager
+    try {
+      segmentManager.setUpNewTable(segmentManager._tableConfig, new IdealState(REALTIME_TABLE_NAME));
+      fail();
+    } catch (IllegalStateException e) {
+      // Expected
     }
-
-    protected FakePinotLLCRealtimeSegmentManager(PinotHelixResourceManager pinotHelixResourceManager,
-        List<String> existingLLCSegments, ControllerMetrics controllerMetrics) {
-      super(pinotHelixResourceManager, CONTROLLER_CONF, controllerMetrics, mock(LeadControllerManager.class));
-
-      try {
-        TableConfigCache mockCache = mock(TableConfigCache.class);
-        TableConfig mockTableConfig = mock(TableConfig.class);
-        IndexingConfig mockIndexingConfig = mock(IndexingConfig.class);
-        when(mockTableConfig.getIndexingConfig()).thenReturn(mockIndexingConfig);
-        StreamConfig streamConfig = FakeStreamConfigUtils.getDefaultLowLevelStreamConfigs();
-        when(mockIndexingConfig.getStreamConfigs()).thenReturn(streamConfig.getStreamConfigsMap());
-        when(mockCache.getTableConfig(anyString())).thenReturn(mockTableConfig);
-
-        Field tableConfigCacheField = PinotLLCRealtimeSegmentManager.class.getDeclaredField("_tableConfigCache");
-        tableConfigCacheField.setAccessible(true);
-        tableConfigCacheField.set(this, mockCache);
-
-        HelixManager mockHelixManager = mock(HelixManager.class);
-        _partitionAssignmentGenerator = new FakeStreamPartitionAssignmentGenerator(mockHelixManager);
-        Field partitionAssignmentGeneratorField =
-            PinotLLCRealtimeSegmentManager.class.getDeclaredField("_streamPartitionAssignmentGenerator");
-        partitionAssignmentGeneratorField.setAccessible(true);
-        partitionAssignmentGeneratorField.set(this, _partitionAssignmentGenerator);
-      } catch (Exception e) {
-        Utils.rethrowException(e);
-      }
-
-      if (existingLLCSegments != null) {
-        _existingLLCSegments = existingLLCSegments;
-      }
-      CONTROLLER_CONF.setControllerVipHost("vip");
-      CONTROLLER_CONF.setControllerPort("9000");
-      CONTROLLER_CONF.setDataDir(baseDir.toString());
-
-      _version = 0;
-
-      _tableConfigStore = new TableConfigStore();
+    try {
+      segmentManager.removeLLCSegments(new IdealState(REALTIME_TABLE_NAME));
+      fail();
+    } catch (IllegalStateException e) {
+      // Expected
     }
-
-    private SegmentMetadataImpl newMockSegmentMetadata() {
-      segmentMetadata = mock(SegmentMetadataImpl.class);
-      when(segmentMetadata.getCrc()).thenReturn(FakePinotLLCRealtimeSegmentManager.CRC);
-      when(segmentMetadata.getTimeInterval()).thenReturn(FakePinotLLCRealtimeSegmentManager.INTERVAL);
-      when(segmentMetadata.getVersion()).thenReturn(FakePinotLLCRealtimeSegmentManager.SEGMENT_VERSION);
-      when(segmentMetadata.getTotalRawDocs()).thenReturn(FakePinotLLCRealtimeSegmentManager.NUM_DOCS);
-      return segmentMetadata;
+    try {
+      segmentManager.commitSegmentFile(REALTIME_TABLE_NAME, mock(CommittingSegmentDescriptor.class));
+      fail();
+    } catch (IllegalStateException e) {
+      // Expected
     }
-
-    private SegmentMetadataImpl getMockSegmentMetadata() {
-      return segmentMetadata;
+    try {
+      segmentManager.commitSegmentMetadata(REALTIME_TABLE_NAME, mock(CommittingSegmentDescriptor.class));
+      fail();
+    } catch (IllegalStateException e) {
+      // Expected
     }
-
-    void addTableToStore(String tableName, TableConfig tableConfig, int nStreamPartitions) {
-      _tableConfigStore.addTable(tableName, tableConfig, nStreamPartitions);
+    try {
+      segmentManager.segmentStoppedConsuming(new LLCSegmentName(RAW_TABLE_NAME, 0, 0, CURRENT_TIME_MS),
+          Helix.PREFIX_OF_SERVER_INSTANCE + 0);
+      fail();
+    } catch (IllegalStateException e) {
+      // Expected
     }
-
-    void removeTableFromStore(String tableName) {
-      _tableConfigStore.removeTable(tableName);
+    try {
+      segmentManager.ensureAllPartitionsConsuming(segmentManager._tableConfig, segmentManager._streamConfig);
+      fail();
+    } catch (IllegalStateException e) {
+      // Expected
     }
+  }
 
-    @Override
-    protected TableConfig getRealtimeTableConfig(String realtimeTableName) {
-      return _tableConfigStore.getTableConfig(realtimeTableName);
-    }
+  //////////////////////////////////////////////////////////////////////////////////
+  // Fake classes
+  /////////////////////////////////////////////////////////////////////////////////
 
-    @Override
-    protected boolean isTooSoonToCorrect(String tableNameWithType, String segmentId, long now) {
-      return tooSoonToCorrect;
+  private static class FakePinotLLCRealtimeSegmentManager extends PinotLLCRealtimeSegmentManager {
+    static final ControllerConf CONTROLLER_CONF = new ControllerConf();
+
+    static {
+      CONTROLLER_CONF.setDataDir(TEMP_DIR.toString());
     }
 
-    @Override
-    protected boolean writeSegmentToPropertyStore(String nodePath, ZNRecord znRecord, final String realtimeTableName,
-        int expectedVersion) {
-      List<String> paths = new ArrayList<>();
-      List<ZNRecord> records = new ArrayList<>();
-      paths.add(nodePath);
-      records.add(znRecord);
-      if (expectedVersion != _version) {
-        return false;
-      } else {
-        writeSegmentsToPropertyStore(paths, records, realtimeTableName);
-        return true;
-      }
+    int _numReplicas;
+    TableConfig _tableConfig;
+    PartitionLevelStreamConfig _streamConfig;
+    int _numInstances;
+    InstancePartitions _consumingInstancePartitions;
+    Map<String, LLCRealtimeSegmentZKMetadata> _segmentZKMetadataMap = new HashMap<>();
+    Map<String, Integer> _segmentZKMetadataVersionMap = new HashMap<>();
+    IdealState _idealState;
+    int _numPartitions;
+    boolean _exceededMaxSegmentCompletionTime = false;
+
+    FakePinotLLCRealtimeSegmentManager() {
+      super(mock(PinotHelixResourceManager.class), CONTROLLER_CONF, mock(ControllerMetrics.class));
     }
 
-    @Override
-    protected boolean writeSegmentToPropertyStore(String nodePath, ZNRecord znRecord, final String realtimeTableName) {
-      List<String> paths = new ArrayList<>();
-      List<ZNRecord> records = new ArrayList<>();
-      paths.add(nodePath);
-      records.add(znRecord);
-      writeSegmentsToPropertyStore(paths, records, realtimeTableName);
-      return true;
+    void makeTableConfig() {
+      Map<String, String> streamConfigs = FakeStreamConfigUtils.getDefaultLowLevelStreamConfigs().getStreamConfigsMap();
+      _tableConfig =
+          new TableConfig.Builder(TableType.REALTIME).setTableName(RAW_TABLE_NAME).setNumReplicas(_numReplicas)
+              .setLLC(true).setStreamConfigs(streamConfigs).build();
+      _streamConfig = new PartitionLevelStreamConfig(_tableConfig);
     }
 
-    @Override
-    protected void writeSegmentsToPropertyStore(List<String> paths, List<ZNRecord> records, String realtimeTableName) {
-      _paths.addAll(paths);
-      _records.addAll(records);
-      for (int i = 0; i < paths.size(); i++) {
-        String path = paths.get(i);
-        ZNRecord znRecord = records.get(i);
-        String segmentId = getSegmentNameFromPath(path);
-        _existingLLCSegments.add(segmentId);
-        _metadataMap.put(segmentId, new LLCRealtimeSegmentZKMetadata(znRecord));
+    void makeConsumingInstancePartitions() {
+      List<String> instances = new ArrayList<>(_numInstances);
+      for (int i = 0; i < _numInstances; i++) {
+        instances.add(Helix.PREFIX_OF_SERVER_INSTANCE + i);
       }
+      _consumingInstancePartitions =
+          new InstancePartitions(InstancePartitionsType.CONSUMING.getInstancePartitionsName(RAW_TABLE_NAME));
+      _consumingInstancePartitions.setInstances(0, 0, instances);
     }
 
-    private String getSegmentNameFromPath(String path) {
-      return path.substring(path.lastIndexOf("/") + 1);
+    public void setUpNewTable() {
+      setUpNewTable(_tableConfig, new IdealState(REALTIME_TABLE_NAME));
     }
 
-    @Override
-    protected List<String> getAllSegments(String realtimeTableName) {
-      return Lists.newArrayList(_metadataMap.keySet());
+    public void ensureAllPartitionsConsuming() {
+      ensureAllPartitionsConsuming(_tableConfig, _streamConfig, _idealState, _numPartitions);
     }
 
     @Override
-    protected LLCRealtimeSegmentZKMetadata getSegmentMetadata(String realtimeTableName, String segmentName) {
-      return _metadataMap.get(segmentName);
+    TableConfig getTableConfig(String realtimeTableName) {
+      return _tableConfig;
     }
 
     @Override
-    protected List<String> getExistingSegments(String realtimeTableName) {
-      return _existingLLCSegments;
+    InstancePartitions getConsumingInstancePartitions(TableConfig tableConfig) {
+      return _consumingInstancePartitions;
     }
 
     @Override
-    protected void updateIdealStateOnSegmentCompletion(@Nonnull final String tableNameWithType,
-        @Nonnull final String currentSegmentId, @Nonnull final String newSegmentId,
-        @Nonnull final PartitionAssignment partitionAssignment) {
-      _nCallsToUpdateHelix++;
-      super.updateIdealStateOnSegmentCompletion(_tableIdealState, currentSegmentId, newSegmentId, partitionAssignment);
+    List<String> getAllSegments(String realtimeTableName) {
+      return new ArrayList<>(_segmentZKMetadataMap.keySet());
     }
 
     @Override
-    protected boolean createNewSegmentMetadataZNRecord(TableConfig realtimeTableConfig,
-        LLCSegmentName committingSegmentName, LLCSegmentName newLLCSegmentName, PartitionAssignment partitionAssignment,
-        CommittingSegmentDescriptor committingSegmentDescriptor, boolean isNewTableSetup) {
-      _nCallsToCreateNewSegmentMetadata++;
-      return super.createNewSegmentMetadataZNRecord(realtimeTableConfig, committingSegmentName, newLLCSegmentName,
-          partitionAssignment, committingSegmentDescriptor, isNewTableSetup);
+    List<String> getLLCSegments(String realtimeTableName) {
+      return new ArrayList<>(_segmentZKMetadataMap.keySet());
     }
 
     @Override
-    public LLCRealtimeSegmentZKMetadata getRealtimeSegmentZKMetadata(String realtimeTableName, String segmentName,
-        Stat stat) {
-      if (_metadataMap.containsKey(segmentName)) {
-        LLCRealtimeSegmentZKMetadata oldMetadata = _metadataMap.get(segmentName);
-
-        LLCRealtimeSegmentZKMetadata metadata = new LLCRealtimeSegmentZKMetadata();
-        metadata.setSegmentName(oldMetadata.getSegmentName());
-        metadata.setDownloadUrl(oldMetadata.getDownloadUrl());
-        metadata.setNumReplicas(oldMetadata.getNumReplicas());
-        metadata.setEndOffset(oldMetadata.getEndOffset());
-        metadata.setStatus(oldMetadata.getStatus());
-        return metadata;
+    LLCRealtimeSegmentZKMetadata getSegmentZKMetadata(String realtimeTableName, String segmentName,
+        @Nullable Stat stat) {
+      Preconditions.checkState(_segmentZKMetadataMap.containsKey(segmentName));
+      if (stat != null) {
+        stat.setVersion(_segmentZKMetadataVersionMap.get(segmentName));
       }
-      LLCRealtimeSegmentZKMetadata metadata = new LLCRealtimeSegmentZKMetadata();
-      metadata.setSegmentName(segmentName);
-      return metadata;
-    }
-
-    public void verifyMetadataInteractions() {
-      verify(segmentMetadata, times(1)).getCrc();
-      verify(segmentMetadata, times(2)).getTimeInterval();
-      verify(segmentMetadata, times(1)).getVersion();
-      verify(segmentMetadata, times(1)).getTotalRawDocs();
-      verify(segmentMetadata, times(1)).getColumnMetadataMap();
-      verifyNoMoreInteractions(segmentMetadata);
+      return new LLCRealtimeSegmentZKMetadata(_segmentZKMetadataMap.get(segmentName).toZNRecord());
     }
 
     @Override
-    protected long getPartitionOffset(StreamConfig streamConfig, final OffsetCriteria offsetCriteria, int partitionId) {
-      if (offsetCriteria.isLargest()) {
-        return _largestOffsetToReturn;
-      } else {
-        return _smallestOffsetToReturn;
+    void persistSegmentZKMetadata(String realtimeTableName, LLCRealtimeSegmentZKMetadata segmentZKMetadata,
+        int expectedVersion) {
+      String segmentName = segmentZKMetadata.getSegmentName();
+      int version = _segmentZKMetadataVersionMap.getOrDefault(segmentName, -1);
+      if (expectedVersion != -1) {
+        Preconditions.checkState(expectedVersion == version);
       }
+      _segmentZKMetadataMap.put(segmentName, segmentZKMetadata);
+      _segmentZKMetadataVersionMap.put(segmentName, version + 1);
     }
 
-    // package-private
-    long getSmallestStreamOffset() {
-      return _smallestOffsetToReturn;
-    }
-
-    //package-private
-    long getLargestStreamOffset() {
-      return _largestOffsetToReturn;
+    @Override
+    protected IdealState getIdealState(String realtimeTableName) {
+      return _idealState;
     }
 
     @Override
-    protected boolean isLeader(String tableName) {
-      return IS_LEADER;
+    protected void setIdealState(String realtimeTableName, IdealState idealState) {
+      _idealState = idealState;
     }
 
     @Override
-    protected boolean isConnected() {
-      return IS_CONNECTED;
+    void updateIdealStateOnSegmentCompletion(String realtimeTableName, String committingSegmentName,
+        String newSegmentName, SegmentAssignment segmentAssignment,
+        Map<InstancePartitionsType, InstancePartitions> instancePartitionsMap) {
+      updateInstanceStatesForNewConsumingSegment(_idealState.getRecord().getMapFields(), committingSegmentName,
+          newSegmentName, segmentAssignment, instancePartitionsMap);
     }
 
     @Override
-    protected IdealState getTableIdealState(String realtimeTableName) {
-      return _tableIdealState;
+    int getNumPartitions(StreamConfig streamConfig) {
+      return _numPartitions;
     }
 
     @Override
-    protected void setTableIdealState(String realtimeTableName, IdealState idealState) {
-      _tableIdealState = idealState;
+    long getPartitionOffset(StreamConfig streamConfig, OffsetCriteria offsetCriteria, int partitionId) {
+      // The criteria for this test should always be SMALLEST (for default streaming config and new added partitions)
+      assertTrue(offsetCriteria.isSmallest());
+      return PARTITION_OFFSET;
     }
 
     @Override
-    public void setupNewTable(TableConfig tableConfig, IdealState emptyIdealState)
-        throws InvalidConfigException {
-      _currentTable = tableConfig.getTableName();
-      super.setupNewTable(tableConfig, emptyIdealState);
+    boolean isExceededMaxSegmentCompletionTime(String realtimeTableName, String segmentName, long currentTimeMs) {
+      return _exceededMaxSegmentCompletionTime;
     }
 
     @Override
-    protected int getPartitionCount(StreamConfig metadata) {
-      return _tableConfigStore.getNStreamPartitions(_currentTable);
+    long getCurrentTimeMs() {
+      return CURRENT_TIME_MS;
     }
   }
 
-  static class FakePinotLLCRealtimeSegmentManagerII extends FakePinotLLCRealtimeSegmentManager {
-
-    final static int SCENARIO_1_ZK_VERSION_NUM_HAS_CHANGE = 1;
-    final static int SCENARIO_2_METADATA_STATUS_HAS_CHANGE = 2;
+  private static class FakePinotLLCRealtimeSegmentManagerII extends FakePinotLLCRealtimeSegmentManager {
+    enum Scenario {
+      ZK_VERSION_CHANGED, METADATA_STATUS_CHANGED
+    }
 
-    private int _scenario;
+    final Scenario _scenario;
 
-    FakePinotLLCRealtimeSegmentManagerII(PinotHelixResourceManager pinotHelixResourceManager,
-        List<String> existingLLCSegments, int scenario) {
-      super(pinotHelixResourceManager, existingLLCSegments);
+    FakePinotLLCRealtimeSegmentManagerII(Scenario scenario) {
+      super();
       _scenario = scenario;
     }
 
     @Override
-    public LLCRealtimeSegmentZKMetadata getRealtimeSegmentZKMetadata(String realtimeTableName, String segmentName,
-        Stat stat) {
-      LLCRealtimeSegmentZKMetadata metadata = _metadataMap.get(segmentName);
+    LLCRealtimeSegmentZKMetadata getSegmentZKMetadata(String realtimeTableName, String segmentName,
+        @Nullable Stat stat) {
+      LLCRealtimeSegmentZKMetadata segmentZKMetadata = super.getSegmentZKMetadata(realtimeTableName, segmentName, stat);
       switch (_scenario) {
-        case SCENARIO_1_ZK_VERSION_NUM_HAS_CHANGE:
-          // Mock another controller has already updated the segment metadata, which makes the version number self increase.
-          stat.setVersion(_version + 1);
+        case ZK_VERSION_CHANGED:
+          // Mock another controller updated the segment ZK metadata during the process
+          if (stat != null) {
+            persistSegmentZKMetadata(realtimeTableName, segmentZKMetadata, stat.getVersion());
+          }
           break;
-        case SCENARIO_2_METADATA_STATUS_HAS_CHANGE:
-          // Mock another controller has updated the status of the old segment metadata.
-          metadata.setStatus(CommonConstants.Segment.Realtime.Status.DONE);
+        case METADATA_STATUS_CHANGED:
+          // Mock another controller has updated the status of the segment ZK metadata
+          segmentZKMetadata.setStatus(Status.DONE);
           break;
       }
-      return metadata;
-    }
-  }
-
-  private static class FakeStreamPartitionAssignmentGenerator extends StreamPartitionAssignmentGenerator {
-
-    private List<String> _consumingInstances;
-
-    public FakeStreamPartitionAssignmentGenerator(HelixManager helixManager) {
-      super(helixManager);
-      _consumingInstances = new ArrayList<>();
-    }
-
-    @Override
-    protected List<String> getConsumingTaggedInstances(TableConfig tableConfig) {
-      return _consumingInstances;
-    }
-
-    void setConsumingInstances(List<String> consumingInstances) {
-      _consumingInstances = consumingInstances;
-    }
-  }
-
-  static class TableConfigStore {
-    private Map<String, TableConfig> _tableConfigsStore;
-    private Map<String, Integer> _nPartitionsStore;
-
-    TableConfigStore() {
-      _tableConfigsStore = new HashMap<>(1);
-      _nPartitionsStore = new HashMap<>(1);
-    }
-
-    void addTable(String tableName, TableConfig tableConfig, int nStreamPartitions) {
-      _tableConfigsStore.put(tableName, tableConfig);
-      _nPartitionsStore.put(tableName, nStreamPartitions);
-    }
-
-    void removeTable(String tableName) {
-      _tableConfigsStore.remove(tableName);
-      _nPartitionsStore.remove(tableName);
-    }
-
-    TableConfig getTableConfig(String tableName) {
-      return _tableConfigsStore.get(tableName);
-    }
-
-    int getNStreamPartitions(String tableName) {
-      return _nPartitionsStore.get(tableName);
-    }
-
-    List<String> getAllRealtimeTablesWithServerTenant(String serverTenant) {
-      List<String> realtimeTablesWithServerTenant = new ArrayList<>();
-      for (Map.Entry<String, TableConfig> entry : _tableConfigsStore.entrySet()) {
-        if (entry.getValue().getTenantConfig().getServer().equals(serverTenant)) {
-          realtimeTablesWithServerTenant.add(entry.getKey());
-        }
-      }
-      return realtimeTablesWithServerTenant;
+      return segmentZKMetadata;
     }
   }
 }
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/SegmentCompletionTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/SegmentCompletionTest.java
index 7dd2944..f9f838a 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/SegmentCompletionTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/SegmentCompletionTest.java
@@ -18,12 +18,11 @@
  */
 package org.apache.pinot.controller.helix.core.realtime;
 
+import com.google.common.base.Preconditions;
 import com.yammer.metrics.core.MetricsRegistry;
 import java.lang.reflect.Field;
-import java.util.List;
 import java.util.Map;
 import org.apache.helix.HelixManager;
-import org.apache.helix.ZNRecord;
 import org.apache.pinot.common.metadata.segment.LLCRealtimeSegmentZKMetadata;
 import org.apache.pinot.common.metrics.ControllerMetrics;
 import org.apache.pinot.common.protocols.SegmentCompletionProtocol;
@@ -1139,7 +1138,6 @@ public class SegmentCompletionTest {
   }
 
   public static class MockPinotLLCRealtimeSegmentManager extends PinotLLCRealtimeSegmentManager {
-    public static final String clusterName = "someCluster";
     public LLCRealtimeSegmentZKMetadata _segmentMetadata;
     public MockSegmentCompletionManager _segmentCompletionMgr;
     private static final ControllerConf CONTROLLER_CONF = new ControllerConf();
@@ -1153,44 +1151,32 @@ public class SegmentCompletionTest {
 
     protected MockPinotLLCRealtimeSegmentManager(PinotHelixResourceManager pinotHelixResourceManager,
         ControllerMetrics controllerMetrics) {
-      super(pinotHelixResourceManager, CONTROLLER_CONF, controllerMetrics,
-          new LeadControllerManager(pinotHelixResourceManager.getHelixZkManager(), controllerMetrics));
+      super(pinotHelixResourceManager, CONTROLLER_CONF, controllerMetrics);
     }
 
     @Override
-    public LLCRealtimeSegmentZKMetadata getRealtimeSegmentZKMetadata(String realtimeTableName, String segmentName,
-        Stat stat) {
+    public LLCRealtimeSegmentZKMetadata getSegmentZKMetadata(String realtimeTableName, String segmentName, Stat stat) {
       return _segmentMetadata;
     }
 
     @Override
-    public boolean commitSegmentMetadata(String rawTableName, CommittingSegmentDescriptor committingSegmentDescriptor) {
+    public void commitSegmentMetadata(String rawTableName, CommittingSegmentDescriptor committingSegmentDescriptor) {
       _segmentMetadata.setStatus(CommonConstants.Segment.Realtime.Status.DONE);
       _segmentMetadata.setEndOffset(committingSegmentDescriptor.getNextOffset());
       _segmentMetadata.setDownloadUrl(URIUtils.constructDownloadUrl(CONTROLLER_CONF.generateVipUrl(), rawTableName,
           committingSegmentDescriptor.getSegmentName()));
       _segmentMetadata.setEndTime(_segmentCompletionMgr.getCurrentTimeMs());
-      return true;
     }
 
     @Override
-    public boolean commitSegmentFile(String rawTableName, CommittingSegmentDescriptor committingSegmentDescriptor) {
-      if (committingSegmentDescriptor.getSegmentLocation().equals("doNotCommitMe")) {
-        return false;
-      } else {
-        return true;
-      }
+    public void commitSegmentFile(String rawTableName, CommittingSegmentDescriptor committingSegmentDescriptor) {
+      Preconditions.checkState(!committingSegmentDescriptor.getSegmentLocation().equals("doNotCommitMe"));
     }
 
     @Override
-    protected void writeSegmentsToPropertyStore(List<String> paths, List<ZNRecord> records, String realtimeTableName) {
-      _segmentMetadata = new LLCRealtimeSegmentZKMetadata(records.get(0));  // Updated record that we are writing to ZK
-    }
-
-    @Override
-    public void segmentStoppedConsuming(final LLCSegmentName segmentName, final String instance) {
-      _stoppedSegmentName = segmentName;
-      _stoppedInstance = instance;
+    public void segmentStoppedConsuming(final LLCSegmentName llcSegmentName, final String instanceName) {
+      _stoppedSegmentName = llcSegmentName;
+      _stoppedInstance = instanceName;
     }
   }
 
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/segment/FlushThresholdUpdaterTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/segment/FlushThresholdUpdaterTest.java
index b539565..fc3919e 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/segment/FlushThresholdUpdaterTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/segment/FlushThresholdUpdaterTest.java
@@ -18,27 +18,29 @@
  */
 package org.apache.pinot.controller.helix.core.realtime.segment;
 
-import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 import java.util.Random;
-import org.apache.pinot.common.config.TableConfig;
+import org.apache.pinot.common.config.TableNameBuilder;
 import org.apache.pinot.common.metadata.segment.LLCRealtimeSegmentZKMetadata;
-import org.apache.pinot.common.partition.PartitionAssignment;
 import org.apache.pinot.common.utils.CommonConstants;
 import org.apache.pinot.common.utils.LLCSegmentName;
 import org.apache.pinot.core.realtime.impl.fakestream.FakeStreamConfigUtils;
+import org.apache.pinot.core.realtime.stream.PartitionLevelStreamConfig;
 import org.apache.pinot.core.realtime.stream.StreamConfig;
 import org.apache.pinot.core.realtime.stream.StreamConfigProperties;
-import org.testng.Assert;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
+import static org.testng.Assert.*;
+
 
 public class FlushThresholdUpdaterTest {
+  private static final String RAW_TABLE_NAME = "testTable";
+  private static final String REALTIME_TABLE_NAME = TableNameBuilder.REALTIME.tableNameWithType(RAW_TABLE_NAME);
   private static final long DESIRED_SEGMENT_SIZE = StreamConfig.getDefaultDesiredSegmentSizeBytes();
   private static final int DEFAULT_INITIAL_ROWS_THRESHOLD = StreamConfig.getDefaultFlushAutotuneInitialRows();
+
   private Random _random;
   private Map<String, double[][]> datasetGraph;
 
@@ -61,89 +63,6 @@ public class FlushThresholdUpdaterTest {
   }
 
   /**
-   * Tests that we have the right flush threshold set in the segment metadata given the various combinations of servers, partitions and replicas
-   */
-  @Test
-  public void testDefaultUpdateFlushThreshold() {
-
-    PartitionAssignment partitionAssignment = new PartitionAssignment("fakeTable_REALTIME");
-    // 4 partitions assigned to 4 servers, 4 replicas => the segments should have 250k rows each (1M / 4)
-    for (int segmentId = 1; segmentId <= 4; ++segmentId) {
-      List<String> instances = new ArrayList<>();
-
-      for (int replicaId = 1; replicaId <= 4; ++replicaId) {
-        instances.add("Server_1.2.3.4_123" + replicaId);
-      }
-
-      partitionAssignment.addPartition(Integer.toString(segmentId), instances);
-    }
-
-    FlushThresholdUpdater flushThresholdUpdater = new DefaultFlushThresholdUpdater(1000000);
-    // Check that each segment has 250k rows each
-    for (int segmentId = 1; segmentId <= 4; ++segmentId) {
-      LLCRealtimeSegmentZKMetadata metadata = new LLCRealtimeSegmentZKMetadata();
-      metadata.setSegmentName(makeFakeSegmentName(segmentId));
-      flushThresholdUpdater.updateFlushThreshold(metadata, null, null, partitionAssignment);
-      Assert.assertEquals(metadata.getSizeThresholdToFlushSegment(), 250000);
-    }
-
-    // 4 partitions assigned to 4 servers, 2 replicas, 2 partitions/server => the segments should have 500k rows each (1M / 2)
-    partitionAssignment.getPartitionToInstances().clear();
-    for (int segmentId = 1; segmentId <= 4; ++segmentId) {
-      List<String> instances = new ArrayList<>();
-
-      for (int replicaId = 1; replicaId <= 2; ++replicaId) {
-        instances.add("Server_1.2.3.4_123" + ((replicaId + segmentId) % 4));
-      }
-
-      partitionAssignment.addPartition(Integer.toString(segmentId), instances);
-    }
-
-    // Check that each segment has 500k rows each
-    for (int segmentId = 1; segmentId <= 4; ++segmentId) {
-      LLCRealtimeSegmentZKMetadata metadata = new LLCRealtimeSegmentZKMetadata();
-      metadata.setSegmentName(makeFakeSegmentName(segmentId));
-      flushThresholdUpdater.updateFlushThreshold(metadata, null, null, partitionAssignment);
-      Assert.assertEquals(metadata.getSizeThresholdToFlushSegment(), 500000);
-    }
-
-    // 4 partitions assigned to 4 servers, 1 replica, 1 partition/server => the segments should have 1M rows each (1M / 1)
-    partitionAssignment.getPartitionToInstances().clear();
-    for (int segmentId = 1; segmentId <= 4; ++segmentId) {
-      List<String> instances = new ArrayList<>();
-      instances.add("Server_1.2.3.4_123" + segmentId);
-      partitionAssignment.addPartition(Integer.toString(segmentId), instances);
-    }
-
-    // Check that each segment has 1M rows each
-    for (int segmentId = 1; segmentId <= 4; ++segmentId) {
-      LLCRealtimeSegmentZKMetadata metadata = new LLCRealtimeSegmentZKMetadata();
-      metadata.setSegmentName(makeFakeSegmentName(segmentId));
-      flushThresholdUpdater.updateFlushThreshold(metadata, null, null, partitionAssignment);
-      Assert.assertEquals(metadata.getSizeThresholdToFlushSegment(), 1000000);
-    }
-
-    // Assign another partition to all servers => the servers should have 500k rows each (1M / 2)
-    List<String> instances = new ArrayList<>();
-    for (int replicaId = 1; replicaId <= 4; ++replicaId) {
-      instances.add("Server_1.2.3.4_123" + replicaId);
-    }
-    partitionAssignment.addPartition("5", instances);
-
-    // Check that each segment has 500k rows each
-    for (int segmentId = 1; segmentId <= 4; ++segmentId) {
-      LLCRealtimeSegmentZKMetadata metadata = new LLCRealtimeSegmentZKMetadata();
-      metadata.setSegmentName(makeFakeSegmentName(segmentId));
-      flushThresholdUpdater.updateFlushThreshold(metadata, null, null, partitionAssignment);
-      Assert.assertEquals(metadata.getSizeThresholdToFlushSegment(), 500000);
-    }
-  }
-
-  private String makeFakeSegmentName(int id) {
-    return new LLCSegmentName("fakeTable_REALTIME", id, 0, 1234L).getSegmentName();
-  }
-
-  /**
    * Tests the segment size based flush threshold updater. A series of 500 runs is started.
    * We have 3 types of datasets, each having a different segment size to num rows ratio (exponential growth, logarithmic growth, steps)
    * We let 500 segments pass through our algorithm, each time feeding a segment size based on the graph.
@@ -151,10 +70,7 @@ public class FlushThresholdUpdaterTest {
    */
   @Test
   public void testSegmentSizeBasedFlushThreshold() {
-    String tableName = "aRealtimeTable_REALTIME";
-
     for (Map.Entry<String, double[][]> entry : datasetGraph.entrySet()) {
-
       SegmentSizeBasedFlushThresholdUpdater segmentSizeBasedFlushThresholdUpdater =
           new SegmentSizeBasedFlushThresholdUpdater(DESIRED_SEGMENT_SIZE, DEFAULT_INITIAL_ROWS_THRESHOLD);
 
@@ -181,14 +97,13 @@ public class FlushThresholdUpdaterTest {
       LLCRealtimeSegmentZKMetadata committingSegmentMetadata;
       LLCRealtimeSegmentZKMetadata newSegmentMetadata;
 
-      newSegmentMetadata = getNextSegmentMetadata(tableName, startOffset, partitionId, seqNum++);
+      newSegmentMetadata = getNextSegmentMetadata(partitionId, seqNum++, startOffset, System.currentTimeMillis());
       committingSegmentDescriptor = new CommittingSegmentDescriptor(null, startOffset, segmentSizeBytes);
       segmentSizeBasedFlushThresholdUpdater
-          .updateFlushThreshold(newSegmentMetadata, null, committingSegmentDescriptor, null);
-      Assert.assertEquals(newSegmentMetadata.getSizeThresholdToFlushSegment(),
+          .updateFlushThreshold(newSegmentMetadata, committingSegmentDescriptor, null, 0);
+      assertEquals(newSegmentMetadata.getSizeThresholdToFlushSegment(),
           segmentSizeBasedFlushThresholdUpdater.getAutotuneInitialRows());
 
-      System.out.println("NumRowsThreshold, SegmentSize");
       for (int run = 0; run < numRuns; run++) {
         committingSegmentMetadata = new LLCRealtimeSegmentZKMetadata(newSegmentMetadata.toZNRecord());
 
@@ -199,17 +114,17 @@ public class FlushThresholdUpdaterTest {
         startOffset += 1000; // if stopped on time, increment less than 1000
         updateCommittingSegmentMetadata(committingSegmentMetadata, startOffset,
             committingSegmentMetadata.getSizeThresholdToFlushSegment());
-        newSegmentMetadata = getNextSegmentMetadata(tableName, startOffset, partitionId, seqNum++);
+        newSegmentMetadata = getNextSegmentMetadata(partitionId, seqNum++, startOffset, System.currentTimeMillis());
         committingSegmentDescriptor =
             new CommittingSegmentDescriptor(committingSegmentMetadata.getSegmentName(), startOffset, segmentSizeBytes);
         segmentSizeBasedFlushThresholdUpdater
-            .updateFlushThreshold(newSegmentMetadata, committingSegmentMetadata, committingSegmentDescriptor, null);
+            .updateFlushThreshold(newSegmentMetadata, committingSegmentDescriptor, committingSegmentMetadata, 0);
 
         // Assert that segment size is in limits
         if (run > checkRunsAfter) {
-          Assert.assertTrue(segmentSizeBytes < (idealSegmentSize + segmentSizeSwivel),
+          assertTrue(segmentSizeBytes < (idealSegmentSize + segmentSizeSwivel),
               "Segment size check failed for dataset " + entry.getKey());
-          Assert.assertTrue(committingSegmentMetadata.getSizeThresholdToFlushSegment() > numRowsLowerLimit
+          assertTrue(committingSegmentMetadata.getSizeThresholdToFlushSegment() > numRowsLowerLimit
                   && committingSegmentMetadata.getSizeThresholdToFlushSegment() < numRowsUpperLimit,
               "Num rows check failed for dataset " + entry.getKey());
         }
@@ -236,18 +151,17 @@ public class FlushThresholdUpdaterTest {
     return (long) (segmentSize * 1024 * 1024);
   }
 
-  private LLCRealtimeSegmentZKMetadata getNextSegmentMetadata(String realtimeTableName, long startOffset,
-      int partitionId, int seqNum) {
-    LLCSegmentName newSegmentName =
-        new LLCSegmentName(realtimeTableName, partitionId, seqNum, System.currentTimeMillis());
-    LLCRealtimeSegmentZKMetadata newSegMetadata = new LLCRealtimeSegmentZKMetadata();
-    newSegMetadata.setCreationTime(System.currentTimeMillis());
-    newSegMetadata.setStartOffset(startOffset);
-    newSegMetadata.setEndOffset(Long.MAX_VALUE);
-    newSegMetadata.setNumReplicas(3);
-    newSegMetadata.setSegmentName(newSegmentName.getSegmentName());
-    newSegMetadata.setStatus(CommonConstants.Segment.Realtime.Status.IN_PROGRESS);
-    return newSegMetadata;
+  private LLCRealtimeSegmentZKMetadata getNextSegmentMetadata(int partitionId, int sequenceNumber, long startOffset,
+      long creationTimeMs) {
+    LLCRealtimeSegmentZKMetadata segmentZKMetadata = new LLCRealtimeSegmentZKMetadata();
+    segmentZKMetadata.setSegmentName(
+        new LLCSegmentName(RAW_TABLE_NAME, partitionId, sequenceNumber, creationTimeMs).getSegmentName());
+    segmentZKMetadata.setStartOffset(startOffset);
+    segmentZKMetadata.setEndOffset(Long.MAX_VALUE);
+    segmentZKMetadata.setCreationTime(creationTimeMs);
+    segmentZKMetadata.setNumReplicas(3);
+    segmentZKMetadata.setStatus(CommonConstants.Segment.Realtime.Status.IN_PROGRESS);
+    return segmentZKMetadata;
   }
 
   private void updateCommittingSegmentMetadata(LLCRealtimeSegmentZKMetadata committingSegmentMetadata, long endOffset,
@@ -264,85 +178,77 @@ public class FlushThresholdUpdaterTest {
    */
   @Test
   public void testFlushThresholdUpdater() {
-    FlushThresholdUpdateManager manager = new FlushThresholdUpdateManager();
-    TableConfig.Builder tableConfigBuilder = new TableConfig.Builder(CommonConstants.Helix.TableType.REALTIME);
-    tableConfigBuilder.setTableName("tableName_REALTIME");
-    TableConfig realtimeTableConfig;
-
-    FlushThresholdUpdater flushThresholdUpdater;
-    Map<String, String> streamConfigs = FakeStreamConfigUtils.getDefaultLowLevelStreamConfigs().getStreamConfigsMap();
-    tableConfigBuilder.setStreamConfigs(streamConfigs);
+    FlushThresholdUpdateManager flushThresholdUpdateManager = new FlushThresholdUpdateManager();
+    Map<String, String> streamConfigMap = FakeStreamConfigUtils.getDefaultLowLevelStreamConfigs().getStreamConfigsMap();
 
     // flush size set
-    streamConfigs.put(StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_ROWS, "10000");
-    realtimeTableConfig = tableConfigBuilder.build();
-    flushThresholdUpdater = manager.getFlushThresholdUpdater(realtimeTableConfig);
-    Assert.assertEquals(flushThresholdUpdater.getClass(), DefaultFlushThresholdUpdater.class);
-    Assert.assertEquals(((DefaultFlushThresholdUpdater) flushThresholdUpdater).getTableFlushSize(), 10000);
+    streamConfigMap.put(StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_ROWS, "10000");
+    FlushThresholdUpdater flushThresholdUpdater = flushThresholdUpdateManager
+        .getFlushThresholdUpdater(new PartitionLevelStreamConfig(REALTIME_TABLE_NAME, streamConfigMap));
+    assertTrue(flushThresholdUpdater instanceof DefaultFlushThresholdUpdater);
+    assertEquals(((DefaultFlushThresholdUpdater) flushThresholdUpdater).getTableFlushSize(), 10000);
 
     // llc flush size set
-    streamConfigs.remove(StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_ROWS);
-    streamConfigs.put(StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_ROWS + StreamConfigProperties.LLC_SUFFIX, "5000");
-    realtimeTableConfig = tableConfigBuilder.build();
-    flushThresholdUpdater = manager.getFlushThresholdUpdater(realtimeTableConfig);
-    Assert.assertEquals(flushThresholdUpdater.getClass(), DefaultFlushThresholdUpdater.class);
-    Assert.assertEquals(((DefaultFlushThresholdUpdater) flushThresholdUpdater).getTableFlushSize(), 5000);
+    streamConfigMap.remove(StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_ROWS);
+    streamConfigMap
+        .put(StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_ROWS + StreamConfigProperties.LLC_SUFFIX, "5000");
+    flushThresholdUpdater = flushThresholdUpdateManager
+        .getFlushThresholdUpdater(new PartitionLevelStreamConfig(REALTIME_TABLE_NAME, streamConfigMap));
+    assertTrue(flushThresholdUpdater instanceof DefaultFlushThresholdUpdater);
+    assertEquals(((DefaultFlushThresholdUpdater) flushThresholdUpdater).getTableFlushSize(), 5000);
 
     // 0 flush size set
-    streamConfigs.put(StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_ROWS + StreamConfigProperties.LLC_SUFFIX, "0");
-    realtimeTableConfig = tableConfigBuilder.build();
-    flushThresholdUpdater = manager.getFlushThresholdUpdater(realtimeTableConfig);
-    Assert.assertEquals(flushThresholdUpdater.getClass(), SegmentSizeBasedFlushThresholdUpdater.class);
+    streamConfigMap.put(StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_ROWS + StreamConfigProperties.LLC_SUFFIX, "0");
+    flushThresholdUpdater = flushThresholdUpdateManager
+        .getFlushThresholdUpdater(new PartitionLevelStreamConfig(REALTIME_TABLE_NAME, streamConfigMap));
+    assertTrue(flushThresholdUpdater instanceof SegmentSizeBasedFlushThresholdUpdater);
 
     // called again with 0 flush size - same object as above
-    streamConfigs.remove(StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_ROWS + StreamConfigProperties.LLC_SUFFIX);
-    streamConfigs.put(StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_ROWS, "0");
-    realtimeTableConfig = tableConfigBuilder.build();
-    FlushThresholdUpdater flushThresholdUpdaterSame = manager.getFlushThresholdUpdater(realtimeTableConfig);
-    Assert.assertEquals(flushThresholdUpdaterSame.getClass(), SegmentSizeBasedFlushThresholdUpdater.class);
-    Assert.assertEquals(flushThresholdUpdater, flushThresholdUpdaterSame);
-    Assert.assertEquals(((SegmentSizeBasedFlushThresholdUpdater) (flushThresholdUpdater)).getDesiredSegmentSizeBytes(),
-        StreamConfig.getDefaultDesiredSegmentSizeBytes());
+    streamConfigMap.remove(StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_ROWS + StreamConfigProperties.LLC_SUFFIX);
+    streamConfigMap.put(StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_ROWS, "0");
+    assertSame(flushThresholdUpdateManager
+            .getFlushThresholdUpdater(new PartitionLevelStreamConfig(REALTIME_TABLE_NAME, streamConfigMap)),
+        flushThresholdUpdater);
 
     // flush size reset to some number - default received, map cleared of segmentsize based
-    streamConfigs.put(StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_ROWS, "20000");
-    realtimeTableConfig = tableConfigBuilder.build();
-    flushThresholdUpdater = manager.getFlushThresholdUpdater(realtimeTableConfig);
-    Assert.assertEquals(flushThresholdUpdater.getClass(), DefaultFlushThresholdUpdater.class);
-    Assert.assertEquals(((DefaultFlushThresholdUpdater) flushThresholdUpdater).getTableFlushSize(), 20000);
+    streamConfigMap.put(StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_ROWS, "20000");
+    flushThresholdUpdater = flushThresholdUpdateManager
+        .getFlushThresholdUpdater(new PartitionLevelStreamConfig(REALTIME_TABLE_NAME, streamConfigMap));
+    assertTrue(flushThresholdUpdater instanceof DefaultFlushThresholdUpdater);
+    assertEquals(((DefaultFlushThresholdUpdater) flushThresholdUpdater).getTableFlushSize(), 20000);
 
     // optimal segment size set to invalid value. Default remains the same.
-    streamConfigs.put(StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_ROWS, "0");
-    streamConfigs.put(StreamConfigProperties.SEGMENT_FLUSH_DESIRED_SIZE, "Invalid");
-    realtimeTableConfig = tableConfigBuilder.build();
-    flushThresholdUpdater = manager.getFlushThresholdUpdater(realtimeTableConfig);
-    Assert.assertEquals(flushThresholdUpdater.getClass(), SegmentSizeBasedFlushThresholdUpdater.class);
-    Assert.assertEquals(((SegmentSizeBasedFlushThresholdUpdater) (flushThresholdUpdater)).getDesiredSegmentSizeBytes(),
+    streamConfigMap.put(StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_ROWS, "0");
+    streamConfigMap.put(StreamConfigProperties.SEGMENT_FLUSH_DESIRED_SIZE, "Invalid");
+    flushThresholdUpdater = flushThresholdUpdateManager
+        .getFlushThresholdUpdater(new PartitionLevelStreamConfig(REALTIME_TABLE_NAME, streamConfigMap));
+    assertTrue(flushThresholdUpdater instanceof SegmentSizeBasedFlushThresholdUpdater);
+    assertEquals(((SegmentSizeBasedFlushThresholdUpdater) flushThresholdUpdater).getDesiredSegmentSizeBytes(),
         StreamConfig.getDefaultDesiredSegmentSizeBytes());
 
     // Clear the flush threshold updater for this table.
-    streamConfigs.put(StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_ROWS, "20000");
-    realtimeTableConfig = tableConfigBuilder.build();
-    flushThresholdUpdater = manager.getFlushThresholdUpdater(realtimeTableConfig);
+    streamConfigMap.put(StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_ROWS, "20000");
+    flushThresholdUpdateManager
+        .getFlushThresholdUpdater(new PartitionLevelStreamConfig(REALTIME_TABLE_NAME, streamConfigMap));
 
     // optimal segment size set to 500M
     long desiredSegSize = 500 * 1024 * 1024;
-    streamConfigs.put(StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_ROWS, "0");
-    streamConfigs.put(StreamConfigProperties.SEGMENT_FLUSH_DESIRED_SIZE, Long.toString(desiredSegSize));
-    realtimeTableConfig = tableConfigBuilder.build();
-    flushThresholdUpdater = manager.getFlushThresholdUpdater(realtimeTableConfig);
-    Assert.assertEquals(((SegmentSizeBasedFlushThresholdUpdater) (flushThresholdUpdater)).getDesiredSegmentSizeBytes(),
+    streamConfigMap.put(StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_ROWS, "0");
+    streamConfigMap.put(StreamConfigProperties.SEGMENT_FLUSH_DESIRED_SIZE, Long.toString(desiredSegSize));
+    flushThresholdUpdater = flushThresholdUpdateManager
+        .getFlushThresholdUpdater(new PartitionLevelStreamConfig(REALTIME_TABLE_NAME, streamConfigMap));
+    assertTrue(flushThresholdUpdater instanceof SegmentSizeBasedFlushThresholdUpdater);
+    assertEquals(((SegmentSizeBasedFlushThresholdUpdater) flushThresholdUpdater).getDesiredSegmentSizeBytes(),
         desiredSegSize);
-    Assert.assertEquals(((SegmentSizeBasedFlushThresholdUpdater) (flushThresholdUpdater)).getAutotuneInitialRows(),
+    assertEquals(((SegmentSizeBasedFlushThresholdUpdater) flushThresholdUpdater).getAutotuneInitialRows(),
         DEFAULT_INITIAL_ROWS_THRESHOLD);
 
     // initial rows threshold
-    streamConfigs.put(StreamConfigProperties.SEGMENT_FLUSH_AUTOTUNE_INITIAL_ROWS, "500000");
-    realtimeTableConfig = tableConfigBuilder.build();
-    FlushThresholdUpdateManager newManager = new FlushThresholdUpdateManager();
-    flushThresholdUpdater = newManager.getFlushThresholdUpdater(realtimeTableConfig);
-    Assert.assertEquals(((SegmentSizeBasedFlushThresholdUpdater) (flushThresholdUpdater)).getAutotuneInitialRows(),
-        500_000);
+    streamConfigMap.put(StreamConfigProperties.SEGMENT_FLUSH_AUTOTUNE_INITIAL_ROWS, "500000");
+    flushThresholdUpdater = new FlushThresholdUpdateManager()
+        .getFlushThresholdUpdater(new PartitionLevelStreamConfig(REALTIME_TABLE_NAME, streamConfigMap));
+    assertTrue(flushThresholdUpdater instanceof SegmentSizeBasedFlushThresholdUpdater);
+    assertEquals(((SegmentSizeBasedFlushThresholdUpdater) flushThresholdUpdater).getAutotuneInitialRows(), 500_000);
   }
 
   /**
@@ -350,46 +256,35 @@ public class FlushThresholdUpdaterTest {
    */
   @Test
   public void testUpdaterChange() {
-    String tableName = "fakeTable_REALTIME";
     int tableFlushSize = 1_000_000;
     int partitionId = 0;
-    int seqNum = 0;
     long startOffset = 0;
-    long committingSegmentSizeBytes = 0;
-
-    PartitionAssignment partitionAssignment = new PartitionAssignment(tableName);
-    // 4 partitions assigned to 4 servers, 4 replicas => the segments should have 250k rows each (1M / 4)
-    for (int p = 0; p < 4; p++) {
-      List<String> instances = new ArrayList<>();
-
-      for (int replicaId = 0; replicaId < 4; replicaId++) {
-        instances.add("Server_1.2.3.4_123" + replicaId);
-      }
-
-      partitionAssignment.addPartition(Integer.toString(p), instances);
-    }
+    int maxNumPartitionsPerInstance = 4;
 
     // Initially we were using default flush threshold updation - verify that thresholds are as per default strategy
-    LLCRealtimeSegmentZKMetadata metadata0 = getNextSegmentMetadata(tableName, startOffset, partitionId, seqNum++);
+    LLCRealtimeSegmentZKMetadata metadata0 =
+        getNextSegmentMetadata(partitionId, 0, startOffset, System.currentTimeMillis());
 
     FlushThresholdUpdater flushThresholdUpdater = new DefaultFlushThresholdUpdater(tableFlushSize);
-    flushThresholdUpdater.updateFlushThreshold(metadata0, null, null, partitionAssignment);
+    flushThresholdUpdater.updateFlushThreshold(metadata0, null, null, maxNumPartitionsPerInstance);
 
-    Assert.assertEquals(metadata0.getSizeThresholdToFlushSegment(), 250_000);
-    Assert.assertNull(metadata0.getTimeThresholdToFlushSegment());
+    assertEquals(metadata0.getSizeThresholdToFlushSegment(), 250_000);
+    assertNull(metadata0.getTimeThresholdToFlushSegment());
 
     // before committing segment, we switched to size based updation - verify that new thresholds are set as per size based strategy
-    flushThresholdUpdater = new SegmentSizeBasedFlushThresholdUpdater(DESIRED_SEGMENT_SIZE,
-        DEFAULT_INITIAL_ROWS_THRESHOLD);
+    flushThresholdUpdater =
+        new SegmentSizeBasedFlushThresholdUpdater(DESIRED_SEGMENT_SIZE, DEFAULT_INITIAL_ROWS_THRESHOLD);
 
     startOffset += 1000;
     updateCommittingSegmentMetadata(metadata0, startOffset, 250_000);
-    committingSegmentSizeBytes = 180 * 1024 * 1024;
+    long committingSegmentSizeBytes = 180 * 1024 * 1024;
     CommittingSegmentDescriptor committingSegmentDescriptor =
         new CommittingSegmentDescriptor(metadata0.getSegmentName(), startOffset, committingSegmentSizeBytes);
-    LLCRealtimeSegmentZKMetadata metadata1 = getNextSegmentMetadata(tableName, startOffset, partitionId, seqNum++);
-    flushThresholdUpdater.updateFlushThreshold(metadata1, metadata0, committingSegmentDescriptor, partitionAssignment);
-    Assert.assertTrue(
+    LLCRealtimeSegmentZKMetadata metadata1 =
+        getNextSegmentMetadata(partitionId, 1, startOffset, System.currentTimeMillis());
+    flushThresholdUpdater
+        .updateFlushThreshold(metadata1, committingSegmentDescriptor, metadata0, maxNumPartitionsPerInstance);
+    assertTrue(
         metadata1.getSizeThresholdToFlushSegment() != 0 && metadata1.getSizeThresholdToFlushSegment() != 250_000);
 
     // before committing we switched back to default strategy, verify that thresholds are set according to default logic
@@ -400,29 +295,30 @@ public class FlushThresholdUpdaterTest {
     committingSegmentSizeBytes = 190 * 1024 * 1024;
     committingSegmentDescriptor =
         new CommittingSegmentDescriptor(metadata1.getSegmentName(), startOffset, committingSegmentSizeBytes);
-    LLCRealtimeSegmentZKMetadata metadata2 = getNextSegmentMetadata(tableName, startOffset, partitionId, seqNum++);
-    flushThresholdUpdater.updateFlushThreshold(metadata2, metadata1, committingSegmentDescriptor, partitionAssignment);
+    LLCRealtimeSegmentZKMetadata metadata2 =
+        getNextSegmentMetadata(partitionId, 2, startOffset, System.currentTimeMillis());
+    flushThresholdUpdater
+        .updateFlushThreshold(metadata2, committingSegmentDescriptor, metadata1, maxNumPartitionsPerInstance);
 
-    Assert.assertEquals(metadata2.getSizeThresholdToFlushSegment(), 250_000);
-    Assert.assertNull(metadata2.getTimeThresholdToFlushSegment());
+    assertEquals(metadata2.getSizeThresholdToFlushSegment(), 250_000);
+    assertNull(metadata2.getTimeThresholdToFlushSegment());
   }
 
   @Test
   public void testTimeThresholdInSegmentSizeBased() {
-    String tableName = "fakeTable_REALTIME";
     int partitionId = 0;
-    int seqNum = 0;
     long startOffset = 0;
     long committingSegmentSizeBytes;
     CommittingSegmentDescriptor committingSegmentDescriptor;
 
     // initial segment
-    LLCRealtimeSegmentZKMetadata metadata0 = getNextSegmentMetadata(tableName, startOffset, partitionId, seqNum++);
+    LLCRealtimeSegmentZKMetadata metadata0 =
+        getNextSegmentMetadata(partitionId, 0, startOffset, System.currentTimeMillis());
     SegmentSizeBasedFlushThresholdUpdater flushThresholdUpdater =
         new SegmentSizeBasedFlushThresholdUpdater(DESIRED_SEGMENT_SIZE, DEFAULT_INITIAL_ROWS_THRESHOLD);
     committingSegmentDescriptor = new CommittingSegmentDescriptor(metadata0.getSegmentName(), startOffset, 0);
-    flushThresholdUpdater.updateFlushThreshold(metadata0, null, committingSegmentDescriptor, null);
-    Assert.assertEquals(metadata0.getSizeThresholdToFlushSegment(), flushThresholdUpdater.getAutotuneInitialRows());
+    flushThresholdUpdater.updateFlushThreshold(metadata0, committingSegmentDescriptor, null, 0);
+    assertEquals(metadata0.getSizeThresholdToFlushSegment(), flushThresholdUpdater.getAutotuneInitialRows());
 
     // next segment hit time threshold
     startOffset += 1000;
@@ -430,9 +326,10 @@ public class FlushThresholdUpdaterTest {
     committingSegmentSizeBytes = 180 * 1024 * 1024;
     committingSegmentDescriptor =
         new CommittingSegmentDescriptor(metadata0.getSegmentName(), startOffset, committingSegmentSizeBytes);
-    LLCRealtimeSegmentZKMetadata metadata1 = getNextSegmentMetadata(tableName, startOffset, partitionId, seqNum++);
-    flushThresholdUpdater.updateFlushThreshold(metadata1, metadata0, committingSegmentDescriptor, null);
-    Assert.assertEquals(metadata1.getSizeThresholdToFlushSegment(),
+    LLCRealtimeSegmentZKMetadata metadata1 =
+        getNextSegmentMetadata(partitionId, 1, startOffset, System.currentTimeMillis());
+    flushThresholdUpdater.updateFlushThreshold(metadata1, committingSegmentDescriptor, metadata0, 0);
+    assertEquals(metadata1.getSizeThresholdToFlushSegment(),
         (int) (metadata0.getTotalRawDocs() * flushThresholdUpdater.getRowsMultiplierWhenTimeThresholdHit()));
 
     // now we hit rows threshold
@@ -441,57 +338,43 @@ public class FlushThresholdUpdaterTest {
     committingSegmentSizeBytes = 240 * 1024 * 1024;
     committingSegmentDescriptor =
         new CommittingSegmentDescriptor(metadata1.getSegmentName(), startOffset, committingSegmentSizeBytes);
-    LLCRealtimeSegmentZKMetadata metadata2 = getNextSegmentMetadata(tableName, startOffset, partitionId, seqNum++);
-    flushThresholdUpdater.updateFlushThreshold(metadata2, metadata1, committingSegmentDescriptor, null);
-    Assert.assertTrue(metadata2.getSizeThresholdToFlushSegment() != metadata1.getSizeThresholdToFlushSegment());
+    LLCRealtimeSegmentZKMetadata metadata2 =
+        getNextSegmentMetadata(partitionId, 2, startOffset, System.currentTimeMillis());
+    flushThresholdUpdater.updateFlushThreshold(metadata2, committingSegmentDescriptor, metadata1, 0);
+    assertTrue(metadata2.getSizeThresholdToFlushSegment() != metadata1.getSizeThresholdToFlushSegment());
   }
 
   @Test
   public void testMinThreshold() {
-    String tableName = "fakeTable_REALTIME";
-    final int partitionId = 0;
-    int seqNum = 0;
+    int partitionId = 0;
     long startOffset = 0;
-    long committingSegmentSizeBytes;
     CommittingSegmentDescriptor committingSegmentDescriptor;
     long now = System.currentTimeMillis();
     long seg0time = now - 1334_650;
     long seg1time = seg0time + 14_000;
-
-    // initial segment consumes only 15 rows, so next segment has 10k rows min.
-    LLCSegmentName seg0SegmentName = new LLCSegmentName(tableName, partitionId, seqNum, seg0time);
-    LLCRealtimeSegmentZKMetadata metadata0 = getNextSegmentMetadata(tableName, startOffset, partitionId, seqNum++);
-    metadata0.setSegmentName(seg0SegmentName.getSegmentName());
     SegmentSizeBasedFlushThresholdUpdater flushThresholdUpdater =
         new SegmentSizeBasedFlushThresholdUpdater(DESIRED_SEGMENT_SIZE, DEFAULT_INITIAL_ROWS_THRESHOLD);
-    committingSegmentDescriptor =
-        new CommittingSegmentDescriptor(seg0SegmentName.getSegmentName(), startOffset, 10_000);
+
+    // initial segment consumes only 15 rows, so next segment has 10k rows min.
+    LLCRealtimeSegmentZKMetadata metadata0 = getNextSegmentMetadata(partitionId, 0, startOffset, seg0time);
     metadata0.setTotalRawDocs(15);
-    metadata0.setCreationTime(seg0time);
     metadata0.setSizeThresholdToFlushSegment(874_990);
-    LLCSegmentName seg1SegmentName = new LLCSegmentName(tableName, partitionId, seqNum + 1, seg1time);
-    LLCRealtimeSegmentZKMetadata metadata1 = new LLCRealtimeSegmentZKMetadata();
-    metadata1.setSegmentName(seg1SegmentName.getSegmentName());
-    metadata1.setCreationTime(seg1time);
-    flushThresholdUpdater.updateFlushThreshold(metadata1, metadata0, committingSegmentDescriptor, null);
-    Assert.assertEquals(metadata1.getSizeThresholdToFlushSegment(), flushThresholdUpdater.getMinimumNumRowsThreshold());
+    committingSegmentDescriptor = new CommittingSegmentDescriptor(metadata0.getSegmentName(), startOffset, 10_000);
+    LLCRealtimeSegmentZKMetadata metadata1 = getNextSegmentMetadata(partitionId, 1, startOffset, seg1time);
+    flushThresholdUpdater.updateFlushThreshold(metadata1, committingSegmentDescriptor, metadata0, 0);
+    assertEquals(metadata1.getSizeThresholdToFlushSegment(), flushThresholdUpdater.getMinimumNumRowsThreshold());
 
     // seg1 also consumes 20 rows, so seg2 also gets 10k as threshold.
-    LLCSegmentName seg2SegmentName = new LLCSegmentName(tableName, partitionId, seqNum + 2, now);
-    LLCRealtimeSegmentZKMetadata metadata2 = new LLCRealtimeSegmentZKMetadata();
-    metadata2.setSegmentName(seg2SegmentName.getSegmentName());
-    metadata2.setStartTime(now);
+    LLCRealtimeSegmentZKMetadata metadata2 = getNextSegmentMetadata(partitionId, 2, startOffset, now);
     committingSegmentDescriptor =
-        new CommittingSegmentDescriptor(seg1SegmentName.getSegmentName(), startOffset + 1000, 14_000);
+        new CommittingSegmentDescriptor(metadata1.getSegmentName(), startOffset + 1000, 14_000);
     metadata1.setTotalRawDocs(25);
-    flushThresholdUpdater.updateFlushThreshold(metadata2, metadata1, committingSegmentDescriptor, null);
-    Assert.assertEquals(metadata2.getSizeThresholdToFlushSegment(), flushThresholdUpdater.getMinimumNumRowsThreshold());
+    flushThresholdUpdater.updateFlushThreshold(metadata2, committingSegmentDescriptor, metadata1, 0);
+    assertEquals(metadata2.getSizeThresholdToFlushSegment(), flushThresholdUpdater.getMinimumNumRowsThreshold());
   }
 
   @Test
   public void testNonZeroPartitionUpdates() {
-    String tableName = "fakeTable_REALTIME";
-    int seqNum = 0;
     long startOffset = 0;
     CommittingSegmentDescriptor committingSegmentDescriptor;
     long now = System.currentTimeMillis();
@@ -501,39 +384,28 @@ public class FlushThresholdUpdaterTest {
         new SegmentSizeBasedFlushThresholdUpdater(DESIRED_SEGMENT_SIZE, DEFAULT_INITIAL_ROWS_THRESHOLD);
 
     // Initial update is from partition 1
-    LLCSegmentName seg0SegmentName = new LLCSegmentName(tableName, 1, seqNum, seg0time);
-    LLCRealtimeSegmentZKMetadata metadata0 = getNextSegmentMetadata(tableName, startOffset, 1, seqNum++);
-    metadata0.setSegmentName(seg0SegmentName.getSegmentName());
-    committingSegmentDescriptor =
-        new CommittingSegmentDescriptor(seg0SegmentName.getSegmentName(), startOffset, 3_110_000);
+    LLCRealtimeSegmentZKMetadata metadata0 = getNextSegmentMetadata(1, 0, startOffset, seg0time);
     metadata0.setTotalRawDocs(1_234_000);
-    metadata0.setCreationTime(seg0time);
     metadata0.setSizeThresholdToFlushSegment(874_990);
-    LLCSegmentName seg1SegmentName = new LLCSegmentName(tableName, 1, seqNum + 1, seg1time);
-    LLCRealtimeSegmentZKMetadata metadata1 = new LLCRealtimeSegmentZKMetadata();
-    metadata1.setSegmentName(seg1SegmentName.getSegmentName());
-    metadata1.setCreationTime(seg1time);
-    Assert.assertEquals(flushThresholdUpdater.getLatestSegmentRowsToSizeRatio(), 0.0);
-    flushThresholdUpdater.updateFlushThreshold(metadata1, metadata0, committingSegmentDescriptor, null);
-    final double currentRatio = flushThresholdUpdater.getLatestSegmentRowsToSizeRatio();
-    Assert.assertTrue(currentRatio > 0.0);
+    committingSegmentDescriptor = new CommittingSegmentDescriptor(metadata0.getSegmentName(), startOffset, 3_110_000);
+    LLCRealtimeSegmentZKMetadata metadata1 = getNextSegmentMetadata(1, 1, startOffset, seg1time);
+    assertEquals(flushThresholdUpdater.getLatestSegmentRowsToSizeRatio(), 0.0);
+    flushThresholdUpdater.updateFlushThreshold(metadata1, committingSegmentDescriptor, metadata0, 0);
+    double currentRatio = flushThresholdUpdater.getLatestSegmentRowsToSizeRatio();
+    assertTrue(currentRatio > 0.0);
 
     // Next segment update from partition 1 does not change the ratio.
-
-    LLCSegmentName seg2SegmentName = new LLCSegmentName(tableName, 1, seqNum + 2, now);
-    LLCRealtimeSegmentZKMetadata metadata2 = new LLCRealtimeSegmentZKMetadata();
-    metadata2.setSegmentName(seg2SegmentName.getSegmentName());
-    metadata2.setStartTime(now);
+    LLCRealtimeSegmentZKMetadata metadata2 = getNextSegmentMetadata(1, 2, startOffset, now);
     committingSegmentDescriptor =
-        new CommittingSegmentDescriptor(seg1SegmentName.getSegmentName(), startOffset + 1000, 256_000_000);
+        new CommittingSegmentDescriptor(metadata1.getSegmentName(), startOffset + 1000, 256_000_000);
     metadata1.setTotalRawDocs(2_980_880);
-    flushThresholdUpdater.updateFlushThreshold(metadata2, metadata1, committingSegmentDescriptor, null);
-    Assert.assertEquals(flushThresholdUpdater.getLatestSegmentRowsToSizeRatio(), currentRatio);
+    flushThresholdUpdater.updateFlushThreshold(metadata2, committingSegmentDescriptor, metadata1, 0);
+    assertEquals(flushThresholdUpdater.getLatestSegmentRowsToSizeRatio(), currentRatio);
 
     // But if seg1 is from partition 0, the ratio is changed.
-    seg1SegmentName = new LLCSegmentName(tableName, 0, seqNum + 1, seg1time);
-    metadata1.setSegmentName(seg1SegmentName.getSegmentName());
-    flushThresholdUpdater.updateFlushThreshold(metadata2, metadata1, committingSegmentDescriptor, null);
-    Assert.assertTrue(flushThresholdUpdater.getLatestSegmentRowsToSizeRatio() != currentRatio);
+    metadata1 = getNextSegmentMetadata(0, 1, startOffset, seg1time);
+    committingSegmentDescriptor.setSegmentName(metadata1.getSegmentName());
+    flushThresholdUpdater.updateFlushThreshold(metadata2, committingSegmentDescriptor, metadata1, 0);
+    assertNotEquals(flushThresholdUpdater.getLatestSegmentRowsToSizeRatio(), currentRatio);
   }
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/realtime/segment/ConsumingSegmentAssignmentStrategy.java b/pinot-core/src/main/java/org/apache/pinot/core/realtime/segment/ConsumingSegmentAssignmentStrategy.java
deleted file mode 100644
index e101fba..0000000
--- a/pinot-core/src/main/java/org/apache/pinot/core/realtime/segment/ConsumingSegmentAssignmentStrategy.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.pinot.core.realtime.segment;
-
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import org.apache.pinot.common.exception.InvalidConfigException;
-import org.apache.pinot.common.partition.PartitionAssignment;
-import org.apache.pinot.common.utils.LLCSegmentName;
-
-
-/**
- * Assigns the given list of consuming segments onto instances based on given partition assignment
- */
-public class ConsumingSegmentAssignmentStrategy implements RealtimeSegmentAssignmentStrategy {
-
-  /**
-   * Assigns new segments to instances by referring to the partition assignment
-   * @param newSegments segments to assign
-   * @param partitionAssignment partition assignment for the table to which the segments belong
-   * @return map of segment name to instances list
-   */
-  public Map<String, List<String>> assign(Collection<String> newSegments, PartitionAssignment partitionAssignment)
-      throws InvalidConfigException {
-
-    Map<String, List<String>> segmentAssignment = new HashMap<>(newSegments.size());
-
-    for (String segmentName : newSegments) {
-      if (LLCSegmentName.isLowLevelConsumerSegmentName(segmentName)) {
-        LLCSegmentName llcSegmentName = new LLCSegmentName(segmentName);
-        int partitionId = llcSegmentName.getPartitionId();
-        List<String> instancesListForPartition =
-            partitionAssignment.getInstancesListForPartition(String.valueOf(partitionId));
-        if (instancesListForPartition == null) {
-          throw new InvalidConfigException(
-              "No partition assignment " + partitionId + " found for segment " + segmentName);
-        }
-        segmentAssignment.put(segmentName, instancesListForPartition);
-      }
-    }
-    return segmentAssignment;
-  }
-}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/realtime/segment/RealtimeSegmentAssignmentStrategy.java b/pinot-core/src/main/java/org/apache/pinot/core/realtime/segment/RealtimeSegmentAssignmentStrategy.java
deleted file mode 100644
index 0241f01..0000000
--- a/pinot-core/src/main/java/org/apache/pinot/core/realtime/segment/RealtimeSegmentAssignmentStrategy.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.pinot.core.realtime.segment;
-
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import org.apache.pinot.common.exception.InvalidConfigException;
-import org.apache.pinot.common.partition.PartitionAssignment;
-
-
-/**
- * An interface for segment assignment of realtime segments
- */
-public interface RealtimeSegmentAssignmentStrategy {
-
-  /**
-   * Given list of segments and a partition assignment, assigns the segments onto instances
-   * @param newSegments
-   * @param partitionAssignment
-   * @return
-   */
-  Map<String, List<String>> assign(Collection<String> newSegments, PartitionAssignment partitionAssignment)
-      throws InvalidConfigException;
-}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/realtime/stream/PartitionLevelStreamConfig.java b/pinot-core/src/main/java/org/apache/pinot/core/realtime/stream/PartitionLevelStreamConfig.java
index 64c2a9e..fc4728f 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/realtime/stream/PartitionLevelStreamConfig.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/realtime/stream/PartitionLevelStreamConfig.java
@@ -18,67 +18,63 @@
  */
 package org.apache.pinot.core.realtime.stream;
 
+import com.google.common.base.Preconditions;
 import java.util.Map;
+import org.apache.pinot.common.config.TableConfig;
 import org.apache.pinot.common.utils.time.TimeUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 
 /**
- * A {@link StreamConfig} for a partition level stream
+ * A {@link StreamConfig} for LLC partition level stream, which overrides some properties for low-level consumer.
  * This can be removed once we remove HLC implementation from the code
  */
 public class PartitionLevelStreamConfig extends StreamConfig {
+  public PartitionLevelStreamConfig(TableConfig tableConfig) {
+    super(tableConfig);
+  }
 
-  private static final Logger LOGGER = LoggerFactory.getLogger(PartitionLevelStreamConfig.class);
-
-  final private int _flushThresholdRows;
-  final private long _flushThresholdTimeMillis;
-
-  /**
-   * Initializes a partition level stream config using the map of stream configs from the table config
-   * This overrides some properties for low level consumer
-   */
   public PartitionLevelStreamConfig(String tableNameWithType, Map<String, String> streamConfigMap) {
     super(tableNameWithType, streamConfigMap);
+  }
 
-    int flushThresholdRows = super.getFlushThresholdRows();
+  @Override
+  protected int extractFlushThresholdRows(Map<String, String> streamConfigMap) {
     String flushThresholdRowsKey =
         StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_ROWS + StreamConfigProperties.LLC_SUFFIX;
-    String flushThresholdRowsValue = streamConfigMap.get(flushThresholdRowsKey);
-    if (flushThresholdRowsValue != null) {
+    String flushThresholdRowsStr = streamConfigMap.get(flushThresholdRowsKey);
+    if (flushThresholdRowsStr != null) {
       try {
-        flushThresholdRows = Integer.parseInt(flushThresholdRowsValue);
+        int flushThresholdRows = Integer.parseInt(flushThresholdRowsStr);
+        // Flush threshold rows 0 means using segment size based flush threshold
+        Preconditions.checkState(flushThresholdRows >= 0);
+        return flushThresholdRows;
       } catch (Exception e) {
-        LOGGER.warn("Caught exception when parsing low level flush threshold rows {}:{}, defaulting to base value {}",
-            flushThresholdRowsKey, flushThresholdRowsValue, flushThresholdRows, e);
+        int defaultValue = super.extractFlushThresholdRows(streamConfigMap);
+        LOGGER.warn("Invalid config {}: {}, defaulting to: {}", flushThresholdRowsKey, flushThresholdRowsStr,
+            defaultValue);
+        return defaultValue;
       }
+    } else {
+      return super.extractFlushThresholdRows(streamConfigMap);
     }
-    _flushThresholdRows = flushThresholdRows;
+  }
 
-    long flushThresholdTime = super.getFlushThresholdTimeMillis();
+  @Override
+  protected long extractFlushThresholdTimeMillis(Map<String, String> streamConfigMap) {
     String flushThresholdTimeKey =
         StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_TIME + StreamConfigProperties.LLC_SUFFIX;
-    String flushThresholdTimeValue = streamConfigMap.get(flushThresholdTimeKey);
-    if (flushThresholdTimeValue != null) {
+    String flushThresholdTimeStr = streamConfigMap.get(flushThresholdTimeKey);
+    if (flushThresholdTimeStr != null) {
       try {
-        flushThresholdTime = TimeUtils.convertPeriodToMillis(flushThresholdTimeValue);
+        return TimeUtils.convertPeriodToMillis(flushThresholdTimeStr);
       } catch (Exception e) {
-        LOGGER.warn(
-            "Caught exception when converting low level flush threshold period to millis {}:{}, defaulting to base value {}",
-            flushThresholdTimeKey, flushThresholdTimeValue, flushThresholdTime, e);
+        long defaultValue = super.extractFlushThresholdTimeMillis(streamConfigMap);
+        LOGGER.warn("Invalid config {}: {}, defaulting to: {}", flushThresholdTimeKey, flushThresholdTimeStr,
+            defaultValue);
+        return defaultValue;
       }
+    } else {
+      return super.extractFlushThresholdTimeMillis(streamConfigMap);
     }
-    _flushThresholdTimeMillis = flushThresholdTime;
-  }
-
-  @Override
-  public long getFlushThresholdTimeMillis() {
-    return _flushThresholdTimeMillis;
-  }
-
-  @Override
-  public int getFlushThresholdRows() {
-    return _flushThresholdRows;
   }
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/realtime/stream/StreamConfig.java b/pinot-core/src/main/java/org/apache/pinot/core/realtime/stream/StreamConfig.java
index 335fa9b..829e9a9 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/realtime/stream/StreamConfig.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/realtime/stream/StreamConfig.java
@@ -24,6 +24,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
+import org.apache.pinot.common.config.TableConfig;
 import org.apache.pinot.common.utils.DataSize;
 import org.apache.pinot.common.utils.EqualityUtils;
 import org.apache.pinot.common.utils.time.TimeUtils;
@@ -35,7 +36,7 @@ import org.slf4j.LoggerFactory;
  * Provides all the configs related to the stream as configured in the table config
  */
 public class StreamConfig {
-  private static final Logger LOGGER = LoggerFactory.getLogger(StreamConfig.class);
+  protected static final Logger LOGGER = LoggerFactory.getLogger(StreamConfig.class);
 
   /**
    * The type of the stream consumer either HIGHLEVEL or LOWLEVEL. For backward compatibility, adding SIMPLE which is equivalent to LOWLEVEL
@@ -77,11 +78,14 @@ public class StreamConfig {
 
   final private Map<String, String> _streamConfigMap = new HashMap<>();
 
+  public StreamConfig(TableConfig tableConfig) {
+    this(tableConfig.getTableName(), tableConfig.getIndexingConfig().getStreamConfigs());
+  }
+
   /**
    * Initializes a StreamConfig using the map of stream configs from the table config
    */
   public StreamConfig(String tableNameWithType, Map<String, String> streamConfigMap) {
-
     _type = streamConfigMap.get(StreamConfigProperties.STREAM_TYPE);
     Preconditions.checkNotNull(_type, "Stream type cannot be null");
 
@@ -161,36 +165,8 @@ public class StreamConfig {
     }
     _fetchTimeoutMillis = fetchTimeoutMillis;
 
-    int flushThresholdRows = DEFAULT_FLUSH_THRESHOLD_ROWS;
-    String flushThresholdRowsValue = streamConfigMap.get(StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_ROWS);
-    if (flushThresholdRowsValue != null) {
-      try {
-        flushThresholdRows = Integer.parseInt(flushThresholdRowsValue);
-      } catch (Exception e) {
-        LOGGER.warn("Caught exception when parsing flush threshold rows {}:{}, defaulting to {}",
-            StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_ROWS, flushThresholdRowsValue, DEFAULT_FLUSH_THRESHOLD_ROWS,
-            e);
-      }
-    }
-    _flushThresholdRows = flushThresholdRows;
-
-    long flushThresholdTime = DEFAULT_FLUSH_THRESHOLD_TIME;
-    String flushThresholdTimeValue = streamConfigMap.get(StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_TIME);
-    if (flushThresholdTimeValue != null) {
-      try {
-        flushThresholdTime = TimeUtils.convertPeriodToMillis(flushThresholdTimeValue);
-      } catch (Exception e) {
-        try {
-          // For backward compatibility, default is using milliseconds value.
-          flushThresholdTime = Long.parseLong(flushThresholdTimeValue);
-        } catch (Exception e1) {
-          LOGGER.warn("Caught exception when converting flush threshold period to millis {}:{}, defaulting to {}",
-              StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_TIME, flushThresholdTimeValue,
-              DEFAULT_FLUSH_THRESHOLD_TIME, e);
-        }
-      }
-    }
-    _flushThresholdTimeMillis = flushThresholdTime;
+    _flushThresholdRows = extractFlushThresholdRows(streamConfigMap);
+    _flushThresholdTimeMillis = extractFlushThresholdTimeMillis(streamConfigMap);
 
     long flushDesiredSize = -1;
     String flushSegmentDesiredSizeValue = streamConfigMap.get(StreamConfigProperties.SEGMENT_FLUSH_DESIRED_SIZE);
@@ -222,6 +198,44 @@ public class StreamConfig {
     _streamConfigMap.putAll(streamConfigMap);
   }
 
+  protected int extractFlushThresholdRows(Map<String, String> streamConfigMap) {
+    String flushThresholdRowsStr = streamConfigMap.get(StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_ROWS);
+    if (flushThresholdRowsStr != null) {
+      try {
+        int flushThresholdRows = Integer.parseInt(flushThresholdRowsStr);
+        // Flush threshold rows 0 means using segment size based flush threshold
+        Preconditions.checkState(flushThresholdRows >= 0);
+        return flushThresholdRows;
+      } catch (Exception e) {
+        LOGGER.warn("Invalid config {}: {}, defaulting to: {}", StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_ROWS,
+            flushThresholdRowsStr, DEFAULT_FLUSH_THRESHOLD_ROWS);
+        return DEFAULT_FLUSH_THRESHOLD_ROWS;
+      }
+    } else {
+      return DEFAULT_FLUSH_THRESHOLD_ROWS;
+    }
+  }
+
+  protected long extractFlushThresholdTimeMillis(Map<String, String> streamConfigMap) {
+    String flushThresholdTimeStr = streamConfigMap.get(StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_TIME);
+    if (flushThresholdTimeStr != null) {
+      try {
+        return TimeUtils.convertPeriodToMillis(flushThresholdTimeStr);
+      } catch (Exception e) {
+        try {
+          // For backward-compatibility, parse it as milliseconds value
+          return Long.parseLong(flushThresholdTimeStr);
+        } catch (NumberFormatException nfe) {
+          LOGGER.warn("Invalid config {}: {}, defaulting to: {}", StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_TIME,
+              flushThresholdTimeStr, DEFAULT_FLUSH_THRESHOLD_TIME);
+          return DEFAULT_FLUSH_THRESHOLD_TIME;
+        }
+      }
+    } else {
+      return DEFAULT_FLUSH_THRESHOLD_TIME;
+    }
+  }
+
   public String getType() {
     return _type;
   }
@@ -321,9 +335,9 @@ public class StreamConfig {
         + _offsetCriteria + '\'' + ", _connectionTimeoutMillis=" + _connectionTimeoutMillis + ", _fetchTimeoutMillis="
         + _fetchTimeoutMillis + ", _flushThresholdRows=" + _flushThresholdRows + ", _flushThresholdTimeMillis="
         + _flushThresholdTimeMillis + ", _flushSegmentDesiredSizeBytes=" + _flushSegmentDesiredSizeBytes
-        + ", _flushAutotuneInitialRows=" + _flushAutotuneInitialRows + ", _decoderClass='" + _decoderClass
-        + '\'' + ", _decoderProperties=" + _decoderProperties + ", _groupId='" + _groupId
-        + ", _tableNameWithType='" + _tableNameWithType + '}';
+        + ", _flushAutotuneInitialRows=" + _flushAutotuneInitialRows + ", _decoderClass='" + _decoderClass + '\''
+        + ", _decoderProperties=" + _decoderProperties + ", _groupId='" + _groupId + ", _tableNameWithType='"
+        + _tableNameWithType + '}';
   }
 
   @Override
@@ -343,8 +357,8 @@ public class StreamConfig {
         .isEqual(_flushThresholdRows, that._flushThresholdRows) && EqualityUtils
         .isEqual(_flushThresholdTimeMillis, that._flushThresholdTimeMillis) && EqualityUtils
         .isEqual(_flushSegmentDesiredSizeBytes, that._flushSegmentDesiredSizeBytes) && EqualityUtils
-        .isEqual(_flushAutotuneInitialRows, that._flushAutotuneInitialRows) && EqualityUtils
-        .isEqual(_type, that._type) && EqualityUtils.isEqual(_topicName, that._topicName) && EqualityUtils
+        .isEqual(_flushAutotuneInitialRows, that._flushAutotuneInitialRows) && EqualityUtils.isEqual(_type, that._type)
+        && EqualityUtils.isEqual(_topicName, that._topicName) && EqualityUtils
         .isEqual(_consumerTypes, that._consumerTypes) && EqualityUtils
         .isEqual(_consumerFactoryClassName, that._consumerFactoryClassName) && EqualityUtils
         .isEqual(_offsetCriteria, that._offsetCriteria) && EqualityUtils.isEqual(_decoderClass, that._decoderClass)
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/realtime/segment/ConsumingSegmentAssignmentStrategyTest.java b/pinot-core/src/test/java/org/apache/pinot/core/realtime/segment/ConsumingSegmentAssignmentStrategyTest.java
deleted file mode 100644
index 211fa4e..0000000
--- a/pinot-core/src/test/java/org/apache/pinot/core/realtime/segment/ConsumingSegmentAssignmentStrategyTest.java
+++ /dev/null
@@ -1,285 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.pinot.core.realtime.segment;
-
-import com.google.common.collect.Lists;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.Set;
-import org.apache.helix.HelixManager;
-import org.apache.helix.model.IdealState;
-import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.exception.InvalidConfigException;
-import org.apache.pinot.common.partition.IdealStateBuilderUtil;
-import org.apache.pinot.common.partition.PartitionAssignment;
-import org.apache.pinot.common.partition.StreamPartitionAssignmentGenerator;
-import org.apache.pinot.common.utils.LLCSegmentName;
-import org.testng.Assert;
-import org.testng.annotations.BeforeMethod;
-import org.testng.annotations.Test;
-
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-
-/**
- * Test for verification of correct assignment of segments given a partition assignment
- */
-public class ConsumingSegmentAssignmentStrategyTest {
-
-  private String[] consumingServerNames;
-  private static final int MAX_CONSUMING_INSTANCES = 20;
-
-  private List<String> getConsumingInstanceList(final int nServers) {
-    Assert.assertTrue(nServers <= consumingServerNames.length);
-    String[] instanceArray = Arrays.copyOf(consumingServerNames, nServers);
-    return Lists.newArrayList(instanceArray);
-  }
-
-  @BeforeMethod
-  public void setUp()
-      throws Exception {
-
-    consumingServerNames = new String[MAX_CONSUMING_INSTANCES];
-    for (int i = 0; i < MAX_CONSUMING_INSTANCES; i++) {
-      consumingServerNames[i] = "ConsumingServer_" + i;
-    }
-  }
-
-  /**
-   * Verifies that segments in segment assignment matches input list
-   * Verifies that segment assignment is as expected given the partition assignment
-   * @param newSegments
-   * @param partitionAssignment
-   * @param expectException
-   */
-  private void verifyAssignment(List<String> newSegments, PartitionAssignment partitionAssignment,
-      boolean expectException, List<String> expectedSegmentsInSegmentAssignment) {
-
-    ConsumingSegmentAssignmentStrategy strategy = new ConsumingSegmentAssignmentStrategy();
-    try {
-      Map<String, List<String>> segmentAssignment = strategy.assign(newSegments, partitionAssignment);
-      Assert.assertFalse(expectException);
-      Assert.assertEquals(segmentAssignment.keySet().size(), expectedSegmentsInSegmentAssignment.size());
-      Assert.assertTrue(segmentAssignment.keySet().containsAll(expectedSegmentsInSegmentAssignment));
-      for (String segmentName : expectedSegmentsInSegmentAssignment) {
-        LLCSegmentName llcSegmentName = new LLCSegmentName(segmentName);
-        int partitionId = llcSegmentName.getPartitionId();
-        List<String> expectedInstances = partitionAssignment.getInstancesListForPartition(String.valueOf(partitionId));
-        List<String> assignedInstances = segmentAssignment.get(segmentName);
-        Assert.assertEquals(expectedInstances.size(), assignedInstances.size());
-        Assert.assertTrue(expectedInstances.containsAll(assignedInstances));
-      }
-    } catch (InvalidConfigException e) {
-      Assert.assertTrue(expectException);
-    }
-  }
-
-  /**
-   * Tests various scenarios of how segment assignment will be invoked
-   */
-  @Test
-  public void testAssign() {
-    String tableName = "aTableToTest_REALTIME";
-
-    List<String> newSegments;
-    PartitionAssignment partitionAssignment;
-
-    // empty new segments list
-    newSegments = new ArrayList<>();
-    partitionAssignment = new PartitionAssignment(tableName);
-    verifyAssignment(newSegments, partitionAssignment, false, newSegments);
-
-    // non empty new segments list, empty partition assignment
-    LLCSegmentName llcSegmentName0 = new LLCSegmentName(tableName, 0, 0, System.currentTimeMillis());
-    LLCSegmentName llcSegmentName1 = new LLCSegmentName(tableName, 1, 0, System.currentTimeMillis());
-    LLCSegmentName llcSegmentName2 = new LLCSegmentName(tableName, 2, 0, System.currentTimeMillis());
-    newSegments.add(llcSegmentName0.getSegmentName());
-    newSegments.add(llcSegmentName1.getSegmentName());
-    newSegments.add(llcSegmentName2.getSegmentName());
-    verifyAssignment(newSegments, partitionAssignment, true, newSegments);
-
-    // non empty new segments list, non empty partition assignment, partitions match
-    partitionAssignment.addPartition("0", Lists.newArrayList("s1", "s2"));
-    partitionAssignment.addPartition("1", Lists.newArrayList("s3", "s1"));
-    partitionAssignment.addPartition("2", Lists.newArrayList("s2", "s3"));
-    verifyAssignment(newSegments, partitionAssignment, false, newSegments);
-
-    // partition for a segment missing in partition assignment
-    LLCSegmentName llcSegmentName3 = new LLCSegmentName(tableName, 3, 0, System.currentTimeMillis());
-    newSegments.add(llcSegmentName3.getSegmentName());
-    verifyAssignment(newSegments, partitionAssignment, true, newSegments);
-
-    // extra partitions in partition assignment than needed
-    partitionAssignment.addPartition("3", Lists.newArrayList("s1", "s2"));
-    partitionAssignment.addPartition("4", Lists.newArrayList("s3", "s1"));
-    verifyAssignment(newSegments, partitionAssignment, false, newSegments);
-
-    // non llc segment name
-    List<String> goodSegments = Lists.newArrayList(newSegments);
-    newSegments.add("nonLLCSegmentName");
-    verifyAssignment(newSegments, partitionAssignment, false, goodSegments);
-  }
-
-  /**
-   * Tests a segment lifecycle
-   * 1) new CONSUMING segments in a newly created table, residing on consuming servers
-   * 2) consuming segments become ONLINE on the consuming servers and a new sequence of CONSUMING segments is added
-   * 3) ONLINE segments move to completed servers, cCONSUMING segments still on consuming servers
-   * 4) latest CONSUMING segments become OFFLINE
-   * In all these scenarios, we want to check that the final assignment received from the strategy, adheres strictly to what the latest segments say
-   *
-   */
-  @Test
-  public void testSegmentLifecycle()
-      throws Exception {
-    String tableName = "tableName_REALTIME";
-    int numReplicas = 2;
-    List<String> completedInstances = Lists.newArrayList("CompletedServer_0", "CompletedServer_1");
-    long seed = new Random().nextLong();
-    System.out.println("Random seed " + seed);
-    Random rand = new Random(seed);
-    for (int i = 0; i < 20; i++) {
-      int numPartitions = Math.max(2, rand.nextInt(25)); // use at least 2 partitions
-      int numConsumingInstances =
-          Math.max(numReplicas, rand.nextInt(MAX_CONSUMING_INSTANCES)); // use at least numReplicas num servers
-      List<String> consumingInstances = getConsumingInstanceList(numConsumingInstances);
-      testSegmentCompletionScenario(tableName, numPartitions, numReplicas, consumingInstances, completedInstances);
-    }
-  }
-
-  private void testSegmentCompletionScenario(String tableName, int numPartitions, int numReplicas,
-      List<String> consumingInstances, List<String> completedInstances)
-      throws InvalidConfigException {
-
-    IdealState idealState;
-    TableConfig tableConfig = mock(TableConfig.class);
-    when(tableConfig.getTableName()).thenReturn(tableName);
-    IdealStateBuilderUtil idealStateBuilder = new IdealStateBuilderUtil(tableName);
-
-    HelixManager _mockHelixManager = mock(HelixManager.class);
-    TestStreamPartitionAssignmentGenerator partitionAssignmentGenerator =
-        new TestStreamPartitionAssignmentGenerator(_mockHelixManager);
-    partitionAssignmentGenerator.setConsumingInstances(consumingInstances);
-    PartitionAssignment partitionAssignmentFromIdealState;
-    ConsumingSegmentAssignmentStrategy consumingSegmentAssignmentStrategy = new ConsumingSegmentAssignmentStrategy();
-
-    // 1) new table - all partitions have only consuming segments
-
-    // create ideal state, seq 0 in ONLINE on completed servers, seq 1 in CONSUMING on consuming servers
-    idealState = idealStateBuilder.addConsumingSegments(numPartitions, 0, numReplicas, consumingInstances).build();
-
-    // getPartitionsAssignmentFrom
-    partitionAssignmentFromIdealState =
-        partitionAssignmentGenerator.getStreamPartitionAssignmentFromIdealState(tableConfig, idealState);
-
-    // assign
-    List<String> segmentNames = new ArrayList<>(numPartitions);
-    for (int i = 0; i < numPartitions; i++) {
-      LLCSegmentName segmentName = new LLCSegmentName(tableName, i, 1, System.currentTimeMillis());
-      segmentNames.add(segmentName.getSegmentName());
-    }
-    Map<String, List<String>> assignment =
-        consumingSegmentAssignmentStrategy.assign(segmentNames, partitionAssignmentFromIdealState);
-
-    // verify
-    verifyAssignmentIsFromLatest(partitionAssignmentGenerator, idealState, assignment);
-
-    // 2) consuming segments moved to ONLINE, new set of consuming segments generated
-
-    idealState = idealStateBuilder.setSegmentState(0, 0, "ONLINE").setSegmentState(1, 0, "ONLINE")
-        .setSegmentState(2, 0, "ONLINE").addConsumingSegments(numPartitions, 1, numReplicas, consumingInstances)
-        .build();
-
-    partitionAssignmentFromIdealState =
-        partitionAssignmentGenerator.getStreamPartitionAssignmentFromIdealState(tableConfig, idealState);
-
-    segmentNames = new ArrayList<>(numPartitions);
-    for (int i = 0; i < numPartitions; i++) {
-      LLCSegmentName segmentName = new LLCSegmentName(tableName, i, 2, System.currentTimeMillis());
-      segmentNames.add(segmentName.getSegmentName());
-    }
-    assignment = consumingSegmentAssignmentStrategy.assign(segmentNames, partitionAssignmentFromIdealState);
-
-    // verify
-    verifyAssignmentIsFromLatest(partitionAssignmentGenerator, idealState, assignment);
-
-    // 3) ONLINE segments moved to completed servers - latest consuming segments still on consuming
-
-    idealState = idealStateBuilder.moveToServers(0, 0, completedInstances).moveToServers(1, 0, completedInstances)
-        .moveToServers(2, 0, completedInstances).build();
-    partitionAssignmentFromIdealState =
-        partitionAssignmentGenerator.getStreamPartitionAssignmentFromIdealState(tableConfig, idealState);
-
-    assignment = consumingSegmentAssignmentStrategy.assign(segmentNames, partitionAssignmentFromIdealState);
-
-    // verify
-    verifyAssignmentIsFromLatest(partitionAssignmentGenerator, idealState, assignment);
-
-    // 4) latest consuming segments became OFFLINE
-
-    idealState = idealStateBuilder.setSegmentState(0, 1, "OFFLINE").setSegmentState(1, 1, "OFFLINE")
-        .setSegmentState(2, 1, "OFFLINE").build();
-    partitionAssignmentFromIdealState =
-        partitionAssignmentGenerator.getStreamPartitionAssignmentFromIdealState(tableConfig, idealState);
-
-    assignment = consumingSegmentAssignmentStrategy.assign(segmentNames, partitionAssignmentFromIdealState);
-
-    // verify
-    verifyAssignmentIsFromLatest(partitionAssignmentGenerator, idealState, assignment);
-  }
-
-  private void verifyAssignmentIsFromLatest(StreamPartitionAssignmentGenerator streamPartitionAssignmentGenerator,
-      IdealState idealState, Map<String, List<String>> assignment) {
-    Map<String, LLCSegmentName> partitionToLatestSegments =
-        streamPartitionAssignmentGenerator.getPartitionToLatestSegments(idealState);
-    for (Map.Entry<String, List<String>> entry : assignment.entrySet()) {
-      String segmentName = entry.getKey();
-      List<String> assignedInstances = entry.getValue();
-      LLCSegmentName llcSegmentName = new LLCSegmentName(segmentName);
-      int partitionId = llcSegmentName.getPartitionId();
-      LLCSegmentName latestSegment = partitionToLatestSegments.get(String.valueOf(partitionId));
-      Set<String> instancesInIdealState = idealState.getInstanceStateMap(latestSegment.getSegmentName()).keySet();
-      Assert.assertEquals(assignedInstances.size(), instancesInIdealState.size());
-      Assert.assertTrue(assignedInstances.containsAll(instancesInIdealState));
-    }
-  }
-
-  private class TestStreamPartitionAssignmentGenerator extends StreamPartitionAssignmentGenerator {
-
-    private List<String> _consumingInstances;
-
-    public TestStreamPartitionAssignmentGenerator(HelixManager helixManager) {
-      super(helixManager);
-      _consumingInstances = new ArrayList<>();
-    }
-
-    @Override
-    protected List<String> getConsumingTaggedInstances(TableConfig tableConfig) {
-      return _consumingInstances;
-    }
-
-    void setConsumingInstances(List<String> consumingInstances) {
-      _consumingInstances = consumingInstances;
-    }
-  }
-}


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