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

[incubator-pinot] branch realtime_segment_assignment created (now 171a0db)

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

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


      at 171a0db  [Resource Assignment] Plug in resource assignment to LLC REALTIME table

This branch includes the following new commits:

     new 171a0db  [Resource Assignment] Plug in resource assignment to LLC REALTIME table

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



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


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

Posted by ja...@apache.org.
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