You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by ne...@apache.org on 2022/03/28 17:19:08 UTC

[pinot] branch master updated: FixedSegmentSelector for tier configs (#8389)

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

nehapawar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/pinot.git


The following commit(s) were added to refs/heads/master by this push:
     new 0d6b5da  FixedSegmentSelector for tier configs (#8389)
0d6b5da is described below

commit 0d6b5da7c6872229501faf6cfc27481f35f52584
Author: Neha Pawar <ne...@gmail.com>
AuthorDate: Mon Mar 28 10:18:51 2022 -0700

    FixedSegmentSelector for tier configs (#8389)
    
    * FixedSegmentSelector for tier configs
    
    * Review comments
    
    * fix test
    
    * jdk 8 breakage
    
    * Review comments:
---
 .../common/tier/FixedTierSegmentSelector.java      | 80 ++++++++++++++++++++
 .../org/apache/pinot/common/tier/TierFactory.java  |  8 ++
 .../pinot/common/utils/config/TierConfigUtils.java | 25 ++++---
 .../pinot/common/tier/TierConfigUtilsTest.java     | 63 +++++++++++++---
 .../pinot/common/tier/TierSegmentSelectorTest.java | 54 ++++++++++++++
 .../common/utils/config/TableConfigSerDeTest.java  | 18 +++--
 ...NonReplicaGroupTieredSegmentAssignmentTest.java |  6 +-
 ...NonReplicaGroupTieredSegmentAssignmentTest.java |  6 +-
 .../TableRebalancerClusterStatelessTest.java       | 24 ++++--
 .../segment/local/utils/TableConfigUtils.java      |  2 +-
 .../segment/local/utils/TableConfigUtilsTest.java  | 85 +++++++++++++---------
 .../apache/pinot/spi/config/table/TierConfig.java  | 12 +++
 12 files changed, 310 insertions(+), 73 deletions(-)

diff --git a/pinot-common/src/main/java/org/apache/pinot/common/tier/FixedTierSegmentSelector.java b/pinot-common/src/main/java/org/apache/pinot/common/tier/FixedTierSegmentSelector.java
new file mode 100644
index 0000000..c9db747
--- /dev/null
+++ b/pinot-common/src/main/java/org/apache/pinot/common/tier/FixedTierSegmentSelector.java
@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.common.tier;
+
+import com.google.common.base.Preconditions;
+import java.util.Set;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.helix.HelixManager;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.common.metadata.segment.SegmentZKMetadata;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.utils.CommonConstants.Segment.Realtime;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
+
+
+/**
+ * A {@link TierSegmentSelector} strategy which selects segments for a tier based on a fixed list
+ */
+public class FixedTierSegmentSelector implements TierSegmentSelector {
+  private final Set<String> _segmentsToSelect;
+  private final HelixManager _helixManager;
+
+  public FixedTierSegmentSelector(HelixManager helixManager, Set<String> segmentsToSelect) {
+    _segmentsToSelect = segmentsToSelect;
+    _helixManager = helixManager;
+  }
+
+  @Override
+  public String getType() {
+    return TierFactory.FIXED_SEGMENT_SELECTOR_TYPE;
+  }
+
+  /**
+   * Checks if a segment is eligible for the tier based on fixed list
+   * @param tableNameWithType Name of the table
+   * @param segmentName Name of the segment
+   * @return true if eligible
+   */
+  @Override
+  public boolean selectSegment(String tableNameWithType, String segmentName) {
+    if (CollectionUtils.isNotEmpty(_segmentsToSelect) && _segmentsToSelect.contains(segmentName)) {
+      if (TableType.OFFLINE == TableNameBuilder.getTableTypeFromTableName(tableNameWithType)) {
+        return true;
+      }
+
+      SegmentZKMetadata segmentZKMetadata =
+          ZKMetadataProvider.getSegmentZKMetadata(_helixManager.getHelixPropertyStore(), tableNameWithType,
+              segmentName);
+      Preconditions.checkNotNull(segmentZKMetadata, "Could not find zk metadata for segment: {} of table: {}",
+          segmentName, tableNameWithType);
+      return !segmentZKMetadata.getStatus().equals(Realtime.Status.IN_PROGRESS);
+    }
+    return false;
+  }
+
+  public Set<String> getSegmentsToSelect() {
+    return _segmentsToSelect;
+  }
+
+  @Override
+  public String toString() {
+    return "FixedTierSegmentSelector{" + "_segmentsToSelect=" + _segmentsToSelect + '}';
+  }
+}
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/tier/TierFactory.java b/pinot-common/src/main/java/org/apache/pinot/common/tier/TierFactory.java
index f33a784..b50b055 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/tier/TierFactory.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/tier/TierFactory.java
@@ -18,6 +18,9 @@
  */
 package org.apache.pinot.common.tier;
 
+import com.google.common.collect.Sets;
+import java.util.Collections;
+import org.apache.commons.collections.CollectionUtils;
 import org.apache.helix.HelixManager;
 import org.apache.pinot.spi.config.table.TierConfig;
 
@@ -28,6 +31,7 @@ import org.apache.pinot.spi.config.table.TierConfig;
 public final class TierFactory {
 
   public static final String TIME_SEGMENT_SELECTOR_TYPE = "time";
+  public static final String FIXED_SEGMENT_SELECTOR_TYPE = "fixed";
   public static final String PINOT_SERVER_STORAGE_TYPE = "pinot_server";
 
   private TierFactory() {
@@ -43,6 +47,10 @@ public final class TierFactory {
     String segmentSelectorType = tierConfig.getSegmentSelectorType();
     if (segmentSelectorType.equalsIgnoreCase(TierFactory.TIME_SEGMENT_SELECTOR_TYPE)) {
       segmentSelector = new TimeBasedTierSegmentSelector(helixManager, tierConfig.getSegmentAge());
+    } else if (segmentSelectorType.equalsIgnoreCase(TierFactory.FIXED_SEGMENT_SELECTOR_TYPE)) {
+      segmentSelector = new FixedTierSegmentSelector(helixManager,
+          CollectionUtils.isEmpty(tierConfig.getSegmentList()) ? Collections.emptySet()
+              : Sets.newHashSet(tierConfig.getSegmentList()));
     } else {
       throw new IllegalStateException("Unsupported segmentSelectorType: " + segmentSelectorType);
     }
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/config/TierConfigUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/config/TierConfigUtils.java
index 818807c..e8a864e 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/utils/config/TierConfigUtils.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/config/TierConfigUtils.java
@@ -18,16 +18,15 @@
  */
 package org.apache.pinot.common.utils.config;
 
-import com.google.common.base.Preconditions;
 import java.util.ArrayList;
 import java.util.Comparator;
 import java.util.List;
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.helix.HelixManager;
+import org.apache.pinot.common.tier.FixedTierSegmentSelector;
 import org.apache.pinot.common.tier.Tier;
 import org.apache.pinot.common.tier.TierFactory;
 import org.apache.pinot.common.tier.TierSegmentSelector;
-import org.apache.pinot.common.tier.TierStorage;
 import org.apache.pinot.common.tier.TimeBasedTierSegmentSelector;
 import org.apache.pinot.spi.config.table.TableConfig;
 import org.apache.pinot.spi.config.table.TierConfig;
@@ -76,20 +75,24 @@ public final class TierConfigUtils {
   }
 
   /**
-   * Comparator for sorting the {@link Tier}.
-   * As of now, we have only 1 type of {@link TierSegmentSelector} and 1 type of {@link TierStorage}.
-   * Tier with an older age bucket in {@link TimeBasedTierSegmentSelector} should appear before a younger age bucket,
-   * in sort order
-   * TODO: As we add more types, this logic needs to be upgraded
+   * Comparator for sorting the {@link Tier}. In the sort order
+   * 1) {@link FixedTierSegmentSelector} are always before others
+   * 2) For {@link TimeBasedTierSegmentSelector}, tiers with an older age bucket appear before a younger age bucket,
    */
   public static Comparator<Tier> getTierComparator() {
     return (o1, o2) -> {
       TierSegmentSelector s1 = o1.getSegmentSelector();
       TierSegmentSelector s2 = o2.getSegmentSelector();
-      Preconditions.checkState(TierFactory.TIME_SEGMENT_SELECTOR_TYPE.equalsIgnoreCase(s1.getType()),
-          "Unsupported segmentSelectorType class %s", s1.getClass());
-      Preconditions.checkState(TierFactory.TIME_SEGMENT_SELECTOR_TYPE.equalsIgnoreCase(s2.getType()),
-          "Unsupported segmentSelectorType class %s", s2.getClass());
+      if (TierFactory.FIXED_SEGMENT_SELECTOR_TYPE.equalsIgnoreCase(s1.getType())
+          && TierFactory.FIXED_SEGMENT_SELECTOR_TYPE.equalsIgnoreCase(s2.getType())) {
+        return 0;
+      }
+      if (TierFactory.FIXED_SEGMENT_SELECTOR_TYPE.equalsIgnoreCase(s1.getType())) {
+        return -1;
+      }
+      if (TierFactory.FIXED_SEGMENT_SELECTOR_TYPE.equalsIgnoreCase(s2.getType())) {
+        return 1;
+      }
       Long period1 = ((TimeBasedTierSegmentSelector) s1).getSegmentAgeMillis();
       Long period2 = ((TimeBasedTierSegmentSelector) s2).getSegmentAgeMillis();
       return period2.compareTo(period1);
diff --git a/pinot-common/src/test/java/org/apache/pinot/common/tier/TierConfigUtilsTest.java b/pinot-common/src/test/java/org/apache/pinot/common/tier/TierConfigUtilsTest.java
index 1afd525..3205ee0 100644
--- a/pinot-common/src/test/java/org/apache/pinot/common/tier/TierConfigUtilsTest.java
+++ b/pinot-common/src/test/java/org/apache/pinot/common/tier/TierConfigUtilsTest.java
@@ -19,6 +19,7 @@
 package org.apache.pinot.common.tier;
 
 import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
 import java.util.Collections;
 import java.util.Comparator;
 import org.apache.pinot.common.utils.config.TierConfigUtils;
@@ -49,7 +50,18 @@ public class TierConfigUtilsTest {
     Assert.assertFalse(TierConfigUtils.shouldRelocateToTiers(tableConfig));
 
     tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName("myTable").setTierConfigList(
-        Lists.newArrayList(new TierConfig("myTier", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "10d",
+        Lists.newArrayList(new TierConfig("myTier", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "10d", null,
+            TierFactory.PINOT_SERVER_STORAGE_TYPE, "tag_OFFLINE", null, null))).build();
+    Assert.assertTrue(TierConfigUtils.shouldRelocateToTiers(tableConfig));
+
+    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName("myTable").setTierConfigList(
+        Lists.newArrayList(
+            new TierConfig("myTier", TierFactory.FIXED_SEGMENT_SELECTOR_TYPE, "10d", Lists.newArrayList("seg0", "seg1"),
+                TierFactory.PINOT_SERVER_STORAGE_TYPE, "tag_OFFLINE", null, null))).build();
+    Assert.assertTrue(TierConfigUtils.shouldRelocateToTiers(tableConfig));
+
+    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName("myTable").setTierConfigList(
+        Lists.newArrayList(new TierConfig("myTier", TierFactory.FIXED_SEGMENT_SELECTOR_TYPE, null, null,
             TierFactory.PINOT_SERVER_STORAGE_TYPE, "tag_OFFLINE", null, null))).build();
     Assert.assertTrue(TierConfigUtils.shouldRelocateToTiers(tableConfig));
 
@@ -65,9 +77,15 @@ public class TierConfigUtilsTest {
     Assert.assertFalse(TierConfigUtils.shouldRelocateToTiers(tableConfig));
 
     tableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName("myTable").setTierConfigList(
-        Lists.newArrayList(new TierConfig("myTier", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "10d",
+        Lists.newArrayList(new TierConfig("myTier", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "10d", null,
             TierFactory.PINOT_SERVER_STORAGE_TYPE, "tag_OFFLINE", null, null))).build();
     Assert.assertTrue(TierConfigUtils.shouldRelocateToTiers(tableConfig));
+
+    tableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName("myTable").setTierConfigList(
+        Lists.newArrayList(
+            new TierConfig("myTier", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, null, Lists.newArrayList("seg0", "seg1"),
+                TierFactory.PINOT_SERVER_STORAGE_TYPE, "tag_OFFLINE", null, null))).build();
+    Assert.assertTrue(TierConfigUtils.shouldRelocateToTiers(tableConfig));
   }
 
   /**
@@ -75,9 +93,8 @@ public class TierConfigUtilsTest {
    */
   @Test
   public void testGetTier() {
-    TierConfig tierConfig =
-        new TierConfig("tier1", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "30d", TierFactory.PINOT_SERVER_STORAGE_TYPE,
-            "tier1_tag_OFFLINE", null, null);
+    TierConfig tierConfig = new TierConfig("tier1", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "30d", null,
+        TierFactory.PINOT_SERVER_STORAGE_TYPE, "tier1_tag_OFFLINE", null, null);
     Tier tier = TierFactory.getTier(tierConfig, null);
     Assert.assertEquals(tier.getName(), "tier1");
     Assert.assertTrue(tier.getSegmentSelector() instanceof TimeBasedTierSegmentSelector);
@@ -88,9 +105,28 @@ public class TierConfigUtilsTest {
     Assert.assertEquals(tier.getStorage().getType(), TierFactory.PINOT_SERVER_STORAGE_TYPE);
     Assert.assertEquals(((PinotServerTierStorage) tier.getStorage()).getServerTag(), "tier1_tag_OFFLINE");
 
+    tierConfig = new TierConfig("tier1", TierFactory.FIXED_SEGMENT_SELECTOR_TYPE, null,
+        Lists.newArrayList("segment1", "segment2", "segment3"), TierFactory.PINOT_SERVER_STORAGE_TYPE,
+        "tier1_tag_OFFLINE", null, null);
+    tier = TierFactory.getTier(tierConfig, null);
+    Assert.assertEquals(tier.getName(), "tier1");
+    Assert.assertTrue(tier.getSegmentSelector() instanceof FixedTierSegmentSelector);
+    Assert.assertEquals(tier.getSegmentSelector().getType(), TierFactory.FIXED_SEGMENT_SELECTOR_TYPE);
+    Assert.assertEquals(((FixedTierSegmentSelector) tier.getSegmentSelector()).getSegmentsToSelect(),
+        Sets.newHashSet("segment1", "segment2", "segment3"));
+
+    tierConfig = new TierConfig("tier1", TierFactory.FIXED_SEGMENT_SELECTOR_TYPE, null,
+        null, TierFactory.PINOT_SERVER_STORAGE_TYPE,
+        "tier1_tag_OFFLINE", null, null);
+    tier = TierFactory.getTier(tierConfig, null);
+    Assert.assertEquals(tier.getName(), "tier1");
+    Assert.assertTrue(tier.getSegmentSelector() instanceof FixedTierSegmentSelector);
+    Assert.assertEquals(tier.getSegmentSelector().getType(), TierFactory.FIXED_SEGMENT_SELECTOR_TYPE);
+    Assert.assertTrue(((FixedTierSegmentSelector) tier.getSegmentSelector()).getSegmentsToSelect().isEmpty());
+
     tierConfig =
-        new TierConfig("tier1", "unknown", "30d", TierFactory.PINOT_SERVER_STORAGE_TYPE, "tier1_tag_OFFLINE", null,
-            null);
+        new TierConfig("tier1", "unknown", "30d", null, TierFactory.PINOT_SERVER_STORAGE_TYPE, "tier1_tag_OFFLINE",
+            null, null);
     try {
       TierFactory.getTier(tierConfig, null);
       Assert.fail("Should have failed due to unsupported segmentSelectorType");
@@ -99,8 +135,8 @@ public class TierConfigUtilsTest {
     }
 
     tierConfig =
-        new TierConfig("tier1", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "30d", "unknown", "tier1_tag_OFFLINE", null,
-            null);
+        new TierConfig("tier1", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "30d", null, "unknown", "tier1_tag_OFFLINE",
+            null, null);
     try {
       TierFactory.getTier(tierConfig, null);
       Assert.fail("Should've failed due to unsupported storageType");
@@ -132,6 +168,11 @@ public class TierConfigUtilsTest {
         new Tier("tier5", new TimeBasedTierSegmentSelector(null, "1d"),
             new PinotServerTierStorage("tag_OFFLINE", null, null));
 
+    Tier tier6 = new Tier("tier6", new FixedTierSegmentSelector(null, Sets.newHashSet("seg0")),
+        new PinotServerTierStorage("tag_OFFLINE", null, null));
+    Tier tier7 = new Tier("tier6", new FixedTierSegmentSelector(null, Sets.newHashSet("seg1")),
+        new PinotServerTierStorage("tag_OFFLINE", null, null));
+
     Assert.assertEquals(tierComparator.compare(tier1, tier2), 1);
     Assert.assertEquals(tierComparator.compare(tier1, tier3), -1);
     Assert.assertEquals(tierComparator.compare(tier1, tier4), -1);
@@ -140,5 +181,9 @@ public class TierConfigUtilsTest {
     Assert.assertEquals(tierComparator.compare(tier3, tier4), -1);
     Assert.assertEquals(tierComparator.compare(tier1, tier1), 0);
     Assert.assertEquals(tierComparator.compare(tier3, tier5), 0);
+
+    Assert.assertEquals(tierComparator.compare(tier6, tier7), 0);
+    Assert.assertEquals(tierComparator.compare(tier6, tier5), -1);
+    Assert.assertEquals(tierComparator.compare(tier4, tier7), 1);
   }
 }
diff --git a/pinot-common/src/test/java/org/apache/pinot/common/tier/TierSegmentSelectorTest.java b/pinot-common/src/test/java/org/apache/pinot/common/tier/TierSegmentSelectorTest.java
index ae6a8de..cee73f4 100644
--- a/pinot-common/src/test/java/org/apache/pinot/common/tier/TierSegmentSelectorTest.java
+++ b/pinot-common/src/test/java/org/apache/pinot/common/tier/TierSegmentSelectorTest.java
@@ -18,6 +18,8 @@
  */
 package org.apache.pinot.common.tier;
 
+import com.google.common.collect.Sets;
+import java.util.Collections;
 import java.util.concurrent.TimeUnit;
 import org.apache.helix.HelixManager;
 import org.apache.helix.ZNRecord;
@@ -111,4 +113,56 @@ public class TierSegmentSelectorTest {
     segmentSelector = new TimeBasedTierSegmentSelector(helixManager, "120h");
     Assert.assertFalse(segmentSelector.selectSegment(tableNameWithType, segmentName));
   }
+
+  @Test
+  public void testFixedSegmentSelector() {
+
+    // offline segment
+    String segmentName = "segment_0";
+    String tableNameWithType = "myTable_OFFLINE";
+
+    FixedTierSegmentSelector segmentSelector = new FixedTierSegmentSelector(null, Collections.emptySet());
+    Assert.assertFalse(segmentSelector.selectSegment(tableNameWithType, segmentName));
+
+    segmentSelector = new FixedTierSegmentSelector(null, Sets.newHashSet("segment_1", "segment_2"));
+    Assert.assertFalse(segmentSelector.selectSegment(tableNameWithType, segmentName));
+
+    segmentSelector = new FixedTierSegmentSelector(null, Sets.newHashSet("SEGMENT_0", "segment_1", "segment_2"));
+    Assert.assertFalse(segmentSelector.selectSegment(tableNameWithType, segmentName));
+
+    segmentSelector = new FixedTierSegmentSelector(null, Sets.newHashSet("segment_0", "segment_1", "segment_2"));
+    Assert.assertTrue(segmentSelector.selectSegment(tableNameWithType, segmentName));
+
+    segmentSelector = new FixedTierSegmentSelector(null, Sets.newHashSet("segment %", "segment_2"));
+    Assert.assertFalse(segmentSelector.selectSegment(tableNameWithType, segmentName));
+    Assert.assertTrue(segmentSelector.selectSegment(tableNameWithType, "segment %"));
+
+    // realtime segment
+    segmentName = "myTable__4__1__" + 123456789;
+    tableNameWithType = "myTable_REALTIME";
+    SegmentZKMetadata realtimeSegmentZKMetadata = new SegmentZKMetadata(segmentName);
+    realtimeSegmentZKMetadata.setStartTime(System.currentTimeMillis() - 10000);
+    realtimeSegmentZKMetadata.setEndTime(System.currentTimeMillis() - 5000);
+    realtimeSegmentZKMetadata.setTimeUnit(TimeUnit.MILLISECONDS);
+    realtimeSegmentZKMetadata.setStatus(CommonConstants.Segment.Realtime.Status.DONE);
+    realtimeSegmentZKMetadata.setNumReplicas(1);
+    ZNRecord segmentZKMetadataZNRecord = realtimeSegmentZKMetadata.toZNRecord();
+
+    HelixManager helixManager = mock(HelixManager.class);
+    ZkHelixPropertyStore<ZNRecord> propertyStore = mock(ZkHelixPropertyStore.class);
+    when(propertyStore
+        .get(eq(ZKMetadataProvider.constructPropertyStorePathForSegment(tableNameWithType, segmentName)), any(),
+            anyInt())).thenReturn(segmentZKMetadataZNRecord);
+    when(helixManager.getHelixPropertyStore()).thenReturn(propertyStore);
+    segmentSelector = new FixedTierSegmentSelector(helixManager, Sets.newHashSet(segmentName, "foo", "bar"));
+    Assert.assertTrue(segmentSelector.selectSegment(tableNameWithType, segmentName));
+
+    realtimeSegmentZKMetadata.setStatus(CommonConstants.Segment.Realtime.Status.IN_PROGRESS);
+    segmentZKMetadataZNRecord = realtimeSegmentZKMetadata.toZNRecord();
+    when(propertyStore
+        .get(eq(ZKMetadataProvider.constructPropertyStorePathForSegment(tableNameWithType, segmentName)), any(),
+            anyInt())).thenReturn(segmentZKMetadataZNRecord);
+    when(helixManager.getHelixPropertyStore()).thenReturn(propertyStore);
+    Assert.assertFalse(segmentSelector.selectSegment(tableNameWithType, segmentName));
+  }
 }
diff --git a/pinot-common/src/test/java/org/apache/pinot/common/utils/config/TableConfigSerDeTest.java b/pinot-common/src/test/java/org/apache/pinot/common/utils/config/TableConfigSerDeTest.java
index 8d82b02..aa57465 100644
--- a/pinot-common/src/test/java/org/apache/pinot/common/utils/config/TableConfigSerDeTest.java
+++ b/pinot-common/src/test/java/org/apache/pinot/common/utils/config/TableConfigSerDeTest.java
@@ -300,10 +300,12 @@ public class TableConfigSerDeTest {
     {
       // With tier config
       List<TierConfig> tierConfigList = Lists.newArrayList(
-          new TierConfig("tierA", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "10d", TierFactory.PINOT_SERVER_STORAGE_TYPE,
-              "tierA_tag_OFFLINE", null, null),
-          new TierConfig("tierB", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "30d", TierFactory.PINOT_SERVER_STORAGE_TYPE,
-              "tierB_tag_OFFLINE", null, null));
+          new TierConfig("tierA", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "10d", null,
+              TierFactory.PINOT_SERVER_STORAGE_TYPE, "tierA_tag_OFFLINE", null, null),
+          new TierConfig("tierB", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "30d", null,
+              TierFactory.PINOT_SERVER_STORAGE_TYPE, "tierB_tag_OFFLINE", null, null),
+          new TierConfig("tier0", TierFactory.FIXED_SEGMENT_SELECTOR_TYPE, null, Lists.newArrayList("seg0"),
+              TierFactory.PINOT_SERVER_STORAGE_TYPE, "tierB_tag_OFFLINE", null, null));
       TableConfig tableConfig = tableConfigBuilder.setTierConfigList(tierConfigList).build();
 
       checkTierConfigList(tableConfig);
@@ -461,7 +463,7 @@ public class TableConfigSerDeTest {
   private void checkTierConfigList(TableConfig tableConfig) {
     List<TierConfig> tierConfigsList = tableConfig.getTierConfigsList();
     assertNotNull(tierConfigsList);
-    assertEquals(tierConfigsList.size(), 2);
+    assertEquals(tierConfigsList.size(), 3);
     assertEquals(tierConfigsList.get(0).getName(), "tierA");
     assertEquals(tierConfigsList.get(0).getSegmentSelectorType(), TierFactory.TIME_SEGMENT_SELECTOR_TYPE);
     assertEquals(tierConfigsList.get(0).getSegmentAge(), "10d");
@@ -472,6 +474,12 @@ public class TableConfigSerDeTest {
     assertEquals(tierConfigsList.get(1).getSegmentAge(), "30d");
     assertEquals(tierConfigsList.get(1).getStorageType(), TierFactory.PINOT_SERVER_STORAGE_TYPE);
     assertEquals(tierConfigsList.get(1).getServerTag(), "tierB_tag_OFFLINE");
+    assertEquals(tierConfigsList.get(2).getName(), "tier0");
+    assertEquals(tierConfigsList.get(2).getSegmentSelectorType(), TierFactory.FIXED_SEGMENT_SELECTOR_TYPE);
+    assertNull(tierConfigsList.get(2).getSegmentAge());
+    assertEquals(tierConfigsList.get(2).getSegmentList(), Lists.newArrayList("seg0"));
+    assertEquals(tierConfigsList.get(2).getStorageType(), TierFactory.PINOT_SERVER_STORAGE_TYPE);
+    assertEquals(tierConfigsList.get(2).getServerTag(), "tierB_tag_OFFLINE");
   }
 
   private void checkInstanceAssignmentConfig(TableConfig tableConfig) {
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/assignment/segment/OfflineNonReplicaGroupTieredSegmentAssignmentTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/assignment/segment/OfflineNonReplicaGroupTieredSegmentAssignmentTest.java
index ba2c1c7..8454de4 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/assignment/segment/OfflineNonReplicaGroupTieredSegmentAssignmentTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/assignment/segment/OfflineNonReplicaGroupTieredSegmentAssignmentTest.java
@@ -102,11 +102,11 @@ public class OfflineNonReplicaGroupTieredSegmentAssignmentTest {
   @BeforeClass
   public void setUp() {
     List<TierConfig> tierConfigList = Lists.newArrayList(
-        new TierConfig(TIER_A_NAME, TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "50d",
+        new TierConfig(TIER_A_NAME, TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "50d", null,
             TierFactory.PINOT_SERVER_STORAGE_TYPE, TAG_A_NAME, null, null),
-        new TierConfig(TIER_B_NAME, TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "70d",
+        new TierConfig(TIER_B_NAME, TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "70d", null,
             TierFactory.PINOT_SERVER_STORAGE_TYPE, TAG_B_NAME, null, null),
-        new TierConfig(TIER_C_NAME, TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "120d",
+        new TierConfig(TIER_C_NAME, TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "120d", null,
             TierFactory.PINOT_SERVER_STORAGE_TYPE, TAG_C_NAME, null, null));
     TableConfig tableConfig =
         new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).setNumReplicas(NUM_REPLICAS)
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/assignment/segment/RealtimeNonReplicaGroupTieredSegmentAssignmentTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/assignment/segment/RealtimeNonReplicaGroupTieredSegmentAssignmentTest.java
index 4863b96..ba33245 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/assignment/segment/RealtimeNonReplicaGroupTieredSegmentAssignmentTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/assignment/segment/RealtimeNonReplicaGroupTieredSegmentAssignmentTest.java
@@ -110,11 +110,11 @@ public class RealtimeNonReplicaGroupTieredSegmentAssignmentTest {
     }
 
     List<TierConfig> tierConfigList = Lists.newArrayList(
-        new TierConfig(TIER_A_NAME, TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "10d",
+        new TierConfig(TIER_A_NAME, TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "10d", null,
             TierFactory.PINOT_SERVER_STORAGE_TYPE, TAG_A_NAME, null, null),
-        new TierConfig(TIER_B_NAME, TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "20d",
+        new TierConfig(TIER_B_NAME, TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "20d", null,
             TierFactory.PINOT_SERVER_STORAGE_TYPE, TAG_B_NAME, null, null),
-        new TierConfig(TIER_C_NAME, TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "30d",
+        new TierConfig(TIER_C_NAME, TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "30d", null,
             TierFactory.PINOT_SERVER_STORAGE_TYPE, TAG_C_NAME, null, null));
     TableConfig tableConfig =
         new TableConfigBuilder(TableType.REALTIME).setTableName(RAW_TABLE_NAME).setNumReplicas(NUM_REPLICAS)
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/rebalance/TableRebalancerClusterStatelessTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/rebalance/TableRebalancerClusterStatelessTest.java
index f2372da..f26cdea 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/rebalance/TableRebalancerClusterStatelessTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/rebalance/TableRebalancerClusterStatelessTest.java
@@ -19,6 +19,7 @@
 package org.apache.pinot.controller.helix.core.rebalance;
 
 import com.google.common.collect.Lists;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.Map;
@@ -65,6 +66,7 @@ public class TableRebalancerClusterStatelessTest extends ControllerTest {
   private static final String NO_TIER_NAME = "noTier";
   private static final String TIER_A_NAME = "tierA";
   private static final String TIER_B_NAME = "tierB";
+  private static final String TIER_FIXED_NAME = "tierFixed";
 
   @BeforeClass
   public void setUp()
@@ -318,7 +320,7 @@ public class TableRebalancerClusterStatelessTest extends ControllerTest {
 
   /**
    * Tests rebalance with tier configs
-   * Add 10 segments, with segment metadat end time 3 days apart starting from now to 30 days ago
+   * Add 10 segments, with segment metadata end time 3 days apart starting from now to 30 days ago
    * 1. run rebalance - should see no change
    * 2. add nodes for tiers and run rebalance - should see no change
    * 3. add tier config and run rebalance - should see changed assignment
@@ -374,11 +376,15 @@ public class TableRebalancerClusterStatelessTest extends ControllerTest {
     assertEquals(rebalanceResult.getSegmentAssignment(), oldSegmentAssignment);
 
     // add tier config
+    ArrayList<String> fixedTierSegments =
+        Lists.newArrayList(SEGMENT_NAME_PREFIX + 6, SEGMENT_NAME_PREFIX + 3, SEGMENT_NAME_PREFIX + 1);
     tableConfig.setTierConfigsList(Lists.newArrayList(
-        new TierConfig(TIER_A_NAME, TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "7d", TierFactory.PINOT_SERVER_STORAGE_TYPE,
-            TIER_A_NAME + "_OFFLINE", null, null),
-        new TierConfig(TIER_B_NAME, TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "15d",
-            TierFactory.PINOT_SERVER_STORAGE_TYPE, TIER_B_NAME + "_OFFLINE", null, null)));
+        new TierConfig(TIER_A_NAME, TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "7d", null,
+            TierFactory.PINOT_SERVER_STORAGE_TYPE, TIER_A_NAME + "_OFFLINE", null, null),
+        new TierConfig(TIER_B_NAME, TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "15d", null,
+            TierFactory.PINOT_SERVER_STORAGE_TYPE, TIER_B_NAME + "_OFFLINE", null, null),
+        new TierConfig(TIER_FIXED_NAME, TierFactory.FIXED_SEGMENT_SELECTOR_TYPE, null, fixedTierSegments,
+            TierFactory.PINOT_SERVER_STORAGE_TYPE, NO_TIER_NAME + "_OFFLINE", null, null)));
     _helixResourceManager.updateTableConfig(tableConfig);
 
     // rebalance should change assignment
@@ -388,10 +394,14 @@ public class TableRebalancerClusterStatelessTest extends ControllerTest {
     // check that segments have moved to tiers
     Map<String, Map<String, String>> tierSegmentAssignment = rebalanceResult.getSegmentAssignment();
     for (Map.Entry<String, Map<String, String>> entry : tierSegmentAssignment.entrySet()) {
-      int segId = Integer.parseInt(entry.getKey().split("_")[1]);
+      String segment = entry.getKey();
+      int segId = Integer.parseInt(segment.split("_")[1]);
       Map<String, String> instanceStateMap = entry.getValue();
       String expectedPrefix;
-      if (segId > 4) {
+      if (fixedTierSegments.contains(segment)) {
+        expectedPrefix = NO_TIER_NAME + "_" + SERVER_INSTANCE_ID_PREFIX;
+      } else
+        if (segId > 4) {
         expectedPrefix = TIER_B_NAME + "_" + SERVER_INSTANCE_ID_PREFIX;
       } else if (segId > 2) {
         expectedPrefix = TIER_A_NAME + "_" + SERVER_INSTANCE_ID_PREFIX;
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java
index bec8392..f3fa1d4 100644
--- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java
@@ -521,7 +521,7 @@ public final class TableConfigUtils {
                 segmentSelectorType, tierName);
         Preconditions.checkState(TimeUtils.isPeriodValid(segmentAge),
             "segmentAge: %s must be a valid period string (eg. 30d, 24h) in tier: %s", segmentAge, tierName);
-      } else {
+      } else if (!segmentSelectorType.equalsIgnoreCase(TierFactory.FIXED_SEGMENT_SELECTOR_TYPE)) {
         throw new IllegalStateException(
             "Unsupported segmentSelectorType: " + segmentSelectorType + " in tier: " + tierName);
       }
diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/utils/TableConfigUtilsTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/utils/TableConfigUtilsTest.java
index f78f9b2..51d1907 100644
--- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/utils/TableConfigUtilsTest.java
+++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/utils/TableConfigUtilsTest.java
@@ -528,32 +528,32 @@ public class TableConfigUtilsTest {
     TableConfigUtils.validate(tableConfig, schema);
 
     // 1 tier configs
-    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME).setTierConfigList(Lists
-        .newArrayList(new TierConfig("tier1", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "30d",
+    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME).setTierConfigList(
+        Lists.newArrayList(new TierConfig("tier1", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "30d", null,
             TierFactory.PINOT_SERVER_STORAGE_TYPE, "tier1_tag_OFFLINE", null, null))).build();
     TableConfigUtils.validate(tableConfig, schema);
 
     // 2 tier configs, case insensitive check
-    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME).setTierConfigList(Lists
-        .newArrayList(new TierConfig("tier1", TierFactory.TIME_SEGMENT_SELECTOR_TYPE.toLowerCase(), "30d",
+    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME).setTierConfigList(
+        Lists.newArrayList(new TierConfig("tier1", TierFactory.TIME_SEGMENT_SELECTOR_TYPE.toLowerCase(), "30d", null,
                 TierFactory.PINOT_SERVER_STORAGE_TYPE, "tier1_tag_OFFLINE", null, null),
-            new TierConfig("tier2", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "40d",
+            new TierConfig("tier2", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "40d", null,
                 TierFactory.PINOT_SERVER_STORAGE_TYPE.toLowerCase(), "tier2_tag_OFFLINE", null, null))).build();
     TableConfigUtils.validate(tableConfig, schema);
 
     //realtime table
     tableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName(TABLE_NAME).setTimeColumnName(TIME_COLUMN)
-        .setTierConfigList(Lists.newArrayList(new TierConfig("tier1", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "30d",
+        .setTierConfigList(Lists.newArrayList(
+            new TierConfig("tier1", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "30d", null,
                 TierFactory.PINOT_SERVER_STORAGE_TYPE.toLowerCase(), "tier1_tag_OFFLINE", null, null),
-            new TierConfig("tier2", TierFactory.TIME_SEGMENT_SELECTOR_TYPE.toLowerCase(), "40d",
+            new TierConfig("tier2", TierFactory.TIME_SEGMENT_SELECTOR_TYPE.toLowerCase(), "40d", null,
                 TierFactory.PINOT_SERVER_STORAGE_TYPE, "tier2_tag_OFFLINE", null, null))).build();
     TableConfigUtils.validate(tableConfig, schema);
 
     // tier name empty
-    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME).setTierConfigList(Lists
-        .newArrayList(
-            new TierConfig("", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "30d", TierFactory.PINOT_SERVER_STORAGE_TYPE,
-                "tier1_tag_OFFLINE", null, null))).build();
+    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME).setTierConfigList(
+        Lists.newArrayList(new TierConfig("", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "30d", null,
+            TierFactory.PINOT_SERVER_STORAGE_TYPE, "tier1_tag_OFFLINE", null, null))).build();
     try {
       TableConfigUtils.validate(tableConfig, schema);
       Assert.fail("Should have failed due to empty tier name");
@@ -562,10 +562,10 @@ public class TableConfigUtilsTest {
     }
 
     // tier name repeats
-    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME).setTierConfigList(Lists
-        .newArrayList(new TierConfig("sameTierName", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "30d",
+    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME).setTierConfigList(
+        Lists.newArrayList(new TierConfig("sameTierName", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "30d", null,
                 TierFactory.PINOT_SERVER_STORAGE_TYPE, "tier1_tag_OFFLINE", null, null),
-            new TierConfig("sameTierName", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "100d",
+            new TierConfig("sameTierName", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "100d", null,
                 TierFactory.PINOT_SERVER_STORAGE_TYPE, "tier2_tag_OFFLINE", null, null))).build();
     try {
       TableConfigUtils.validate(tableConfig, schema);
@@ -575,10 +575,10 @@ public class TableConfigUtilsTest {
     }
 
     // segmentSelectorType invalid
-    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME).setTierConfigList(Lists
-        .newArrayList(new TierConfig("tier1", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "30d",
+    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME).setTierConfigList(
+        Lists.newArrayList(new TierConfig("tier1", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "30d", null,
                 TierFactory.PINOT_SERVER_STORAGE_TYPE, "tier1_tag_OFFLINE", null, null),
-            new TierConfig("tier2", "unsupportedSegmentSelector", "40d", TierFactory.PINOT_SERVER_STORAGE_TYPE,
+            new TierConfig("tier2", "unsupportedSegmentSelector", "40d", null, TierFactory.PINOT_SERVER_STORAGE_TYPE,
                 "tier2_tag_OFFLINE", null, null))).build();
     try {
       TableConfigUtils.validate(tableConfig, schema);
@@ -588,10 +588,10 @@ public class TableConfigUtilsTest {
     }
 
     // segmentAge not provided for TIME segmentSelectorType
-    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME).setTierConfigList(Lists
-        .newArrayList(
-            new TierConfig("tier1", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, null, TierFactory.PINOT_SERVER_STORAGE_TYPE,
-                "tier1_tag_OFFLINE", null, null), new TierConfig("tier2", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "40d",
+    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME).setTierConfigList(
+        Lists.newArrayList(new TierConfig("tier1", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, null, null,
+                TierFactory.PINOT_SERVER_STORAGE_TYPE, "tier1_tag_OFFLINE", null, null),
+            new TierConfig("tier2", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "40d", null,
                 TierFactory.PINOT_SERVER_STORAGE_TYPE, "tier2_tag_OFFLINE", null, null))).build();
     try {
       TableConfigUtils.validate(tableConfig, schema);
@@ -601,10 +601,10 @@ public class TableConfigUtilsTest {
     }
 
     // segmentAge invalid
-    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME).setTierConfigList(Lists
-        .newArrayList(new TierConfig("tier1", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "30d",
+    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME).setTierConfigList(
+        Lists.newArrayList(new TierConfig("tier1", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "30d", null,
                 TierFactory.PINOT_SERVER_STORAGE_TYPE, "tier1_tag_OFFLINE", null, null),
-            new TierConfig("tier2", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "3600",
+            new TierConfig("tier2", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "3600", null,
                 TierFactory.PINOT_SERVER_STORAGE_TYPE, "tier2_tag_OFFLINE", null, null))).build();
 
     try {
@@ -614,11 +614,28 @@ public class TableConfigUtilsTest {
       // expected
     }
 
+    // fixedSegmentSelector
+    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME).setTierConfigList(
+        Lists.newArrayList(new TierConfig("tier1", TierFactory.FIXED_SEGMENT_SELECTOR_TYPE, null, null,
+            TierFactory.PINOT_SERVER_STORAGE_TYPE, "tier1_tag_OFFLINE", null, null))).build();
+    TableConfigUtils.validate(tableConfig, schema);
+    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME).setTierConfigList(
+        Lists.newArrayList(new TierConfig("tier1", TierFactory.FIXED_SEGMENT_SELECTOR_TYPE, "30d", Lists.newArrayList(),
+            TierFactory.PINOT_SERVER_STORAGE_TYPE, "tier1_tag_OFFLINE", null, null))).build();
+    TableConfigUtils.validate(tableConfig, schema);
+    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME).setTierConfigList(
+        Lists.newArrayList(
+            new TierConfig("tier1", TierFactory.FIXED_SEGMENT_SELECTOR_TYPE, null, Lists.newArrayList("seg0", "seg1"),
+                TierFactory.PINOT_SERVER_STORAGE_TYPE, "tier1_tag_OFFLINE", null, null))).build();
+    TableConfigUtils.validate(tableConfig, schema);
+
     // storageType invalid
-    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME).setTierConfigList(Lists
-        .newArrayList(new TierConfig("tier1", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "30d", "unsupportedStorageType",
-            "tier1_tag_OFFLINE", null, null), new TierConfig("tier2", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "40d",
-            TierFactory.PINOT_SERVER_STORAGE_TYPE, "tier2_tag_OFFLINE", null, null))).build();
+    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME).setTierConfigList(
+        Lists.newArrayList(
+            new TierConfig("tier1", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "30d", null, "unsupportedStorageType",
+                "tier1_tag_OFFLINE", null, null),
+            new TierConfig("tier2", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "40d", null,
+                TierFactory.PINOT_SERVER_STORAGE_TYPE, "tier2_tag_OFFLINE", null, null))).build();
 
     try {
       TableConfigUtils.validate(tableConfig, schema);
@@ -628,10 +645,10 @@ public class TableConfigUtilsTest {
     }
 
     // serverTag not provided for PINOT_SERVER storageType
-    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME).setTierConfigList(Lists
-        .newArrayList(new TierConfig("tier1", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "30d",
+    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME).setTierConfigList(
+        Lists.newArrayList(new TierConfig("tier1", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "30d", null,
                 TierFactory.PINOT_SERVER_STORAGE_TYPE, "tier1_tag_OFFLINE", null, null),
-            new TierConfig("tier2", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "40d",
+            new TierConfig("tier2", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "40d", null,
                 TierFactory.PINOT_SERVER_STORAGE_TYPE, null, null, null))).build();
     try {
       TableConfigUtils.validate(tableConfig, schema);
@@ -641,10 +658,10 @@ public class TableConfigUtilsTest {
     }
 
     // serverTag invalid
-    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME).setTierConfigList(Lists
-        .newArrayList(new TierConfig("tier1", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "30d",
+    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME).setTierConfigList(
+        Lists.newArrayList(new TierConfig("tier1", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "30d", null,
                 TierFactory.PINOT_SERVER_STORAGE_TYPE, "tier1_tag", null, null),
-            new TierConfig("tier2", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "40d",
+            new TierConfig("tier2", TierFactory.TIME_SEGMENT_SELECTOR_TYPE, "40d", null,
                 TierFactory.PINOT_SERVER_STORAGE_TYPE, "tier2_tag_OFFLINE", null, null))).build();
     try {
       TableConfigUtils.validate(tableConfig, schema);
diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/TierConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/TierConfig.java
index 755116b..a3a2d6a 100644
--- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/TierConfig.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/TierConfig.java
@@ -21,6 +21,7 @@ package org.apache.pinot.spi.config.table;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonPropertyDescription;
 import com.google.common.base.Preconditions;
+import java.util.List;
 import java.util.Map;
 import javax.annotation.Nullable;
 import org.apache.pinot.spi.config.BaseJsonConfig;
@@ -39,6 +40,9 @@ public class TierConfig extends BaseJsonConfig {
   @JsonPropertyDescription("For 'TIME' segment selector, the period after which to select segments for this tier")
   private final String _segmentAge;
 
+  @JsonPropertyDescription("For 'FIXED' segment selector, the list of segments to select for this tier")
+  private final List<String> _segmentList;
+
   @JsonPropertyDescription("The type of storage")
   private final String _storageType;
 
@@ -58,6 +62,7 @@ public class TierConfig extends BaseJsonConfig {
   public TierConfig(@JsonProperty(value = "name", required = true) String name,
       @JsonProperty(value = "segmentSelectorType", required = true) String segmentSelectorType,
       @JsonProperty("segmentAge") @Nullable String segmentAge,
+      @JsonProperty("segmentList") @Nullable List<String> segmentList,
       @JsonProperty(value = "storageType", required = true) String storageType,
       @JsonProperty("serverTag") @Nullable String serverTag,
       @JsonProperty("tierBackend") @Nullable String tierBackend,
@@ -69,6 +74,7 @@ public class TierConfig extends BaseJsonConfig {
     _name = name;
     _segmentSelectorType = segmentSelectorType;
     _segmentAge = segmentAge;
+    _segmentList = segmentList;
     _storageType = storageType;
     _serverTag = serverTag;
     _tierBackend = tierBackend;
@@ -83,10 +89,16 @@ public class TierConfig extends BaseJsonConfig {
     return _segmentSelectorType;
   }
 
+  @Nullable
   public String getSegmentAge() {
     return _segmentAge;
   }
 
+  @Nullable
+  public List<String> getSegmentList() {
+    return _segmentList;
+  }
+
   public String getStorageType() {
     return _storageType;
   }

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