You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by mc...@apache.org on 2021/03/10 21:40:18 UTC

[incubator-pinot] branch master updated: complete compatibility regression testing (#6650)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 59d02c6  complete compatibility regression testing (#6650)
59d02c6 is described below

commit 59d02c66af3ab828bb3f75c0f832d03db0917f1a
Author: Liang Mingqiang <mi...@linkedin.com>
AuthorDate: Wed Mar 10 13:40:03 2021 -0800

    complete compatibility regression testing (#6650)
    
    * split compCheck.sh as two script so we can build and test separately
    * redirect services' logs to a file for easy of debug
    * wait for zk/controller to be ready
    * fix some minor bugs.
---
 compatibility-verifier/README.md                   |  38 ++++++
 compatibility-verifier/checkoutAndBuild.sh         |  79 +++++++++++
 compatibility-verifier/compCheck.sh                | 150 ++++++++-------------
 .../org/apache/pinot/compat/tests/SegmentOp.java   |   8 +-
 .../pinot/compat/tests/SqlResultComparator.java    |  35 +++--
 .../compat-tests/configs/feature-test-1.json       |   4 +-
 .../compat-tests/data/FeatureTest1-data-00.csv     |  25 ++--
 .../compat-tests/data/FeatureTest1-data-01.csv     |  17 ++-
 .../compat-tests/data/recordReaderConfig.json      |   2 +-
 ...p-publish.yaml => kafka-stream-produce-op.yaml} |  12 +-
 .../compat-tests/kafka-topic-create-op.yaml        |  25 ++++
 ...p-publish.yaml => offline-table-create-op.yaml} |  19 +--
 ...p-publish.yaml => offline-table-delete-op.yaml} |  21 +--
 .../queries/feature-test-1-sql.queries             |  17 ++-
 .../src/test/resources/compat-tests/query-op.yaml  |  28 +---
 .../query-results/feature-test-1-rest-sql.results  |   1 -
 ...-publish.yaml => realtime-table-create-op.yaml} |  17 +--
 ...-publish.yaml => realtime-table-delete-op.yaml} |  21 +--
 ...ream-op-publish.yaml => segment-create-op.yaml} |  24 +---
 .../resources/compat-tests/segment-delete-op.yaml  |  26 ++++
 .../resources/compat-tests/table-segment-op.yaml   |  45 -------
 21 files changed, 302 insertions(+), 312 deletions(-)

diff --git a/compatibility-verifier/README.md b/compatibility-verifier/README.md
new file mode 100644
index 0000000..0f6c9b7
--- /dev/null
+++ b/compatibility-verifier/README.md
@@ -0,0 +1,38 @@
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+-->
+
+# Compatibility Regression Testing Scripts for Apache Pinot
+
+## Usage
+
+### Step 1: checkout source code and build targets for older commit and newer commit
+```shell
+./compatibility-verifier/checkoutAndBuild.sh [olderCommit] [newerCommit] [workingDir]
+```
+***NOTE***: `[workingDir]` is optional, if user does not specify `[workingDir]`, the script will create a temporary working 
+dir and output the path, which can be used in step 2.
+
+### Step 2: run compatibility regression test against the two targets build in step1
+```shell
+./compatibility-verifier/compCheck.sh [workingDir]
+```
+***NOTE***: the script can only be run under the root folder of the project currently. Before run the script, make sure to 
+change to the right directory first.
diff --git a/compatibility-verifier/checkoutAndBuild.sh b/compatibility-verifier/checkoutAndBuild.sh
new file mode 100644
index 0000000..5d71c21
--- /dev/null
+++ b/compatibility-verifier/checkoutAndBuild.sh
@@ -0,0 +1,79 @@
+#!/bin/bash
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+
+# get a temporary directory in case the workingDir is not provided by user
+TMP_DIR=$(mktemp -d 2>/dev/null || mktemp -d -t 'mytmpdir')
+
+# get usage of the script
+function usage() {
+  command=$1
+  echo "Usage: $command olderCommit newerCommit [workingDir]"
+  exit 1
+}
+
+# This function builds Pinot given a specific commit hash and target directory
+function checkoutAndBuild() {
+  commitHash=$1
+  targetDir=$2
+
+  pushd "$targetDir" || exit 1
+  git init
+  git remote add origin https://github.com/apache/incubator-pinot
+  git fetch --depth 1 origin "$commitHash"
+  git checkout FETCH_HEAD
+  mvn install package -DskipTests -Pbin-dist
+  popd || exit 1
+}
+
+# get arguments
+olderCommit=$1
+newerCommit=$2
+
+if [ -n "$3" ]; then
+  workingDir=$3
+  if [ -d "$workingDir" ]; then
+    echo "Directory ${workingDir} already exists. Use a new directory."
+    exit 1
+  fi
+else
+  # use the temp directory in case workingDir is not provided
+  workingDir=$TMP_DIR
+  echo "workingDir: ${workingDir}"
+fi
+
+# create subdirectories for given commits
+oldTargetDir="$workingDir"/oldTargetDir
+newTargetDir="$workingDir"/newTargetDir
+
+if ! mkdir -p "$oldTargetDir"; then
+  echo "Failed to create target directory ${oldTargetDir}"
+  exit 1
+fi
+if ! mkdir -p "$newTargetDir"; then
+  echo "Failed to create target directory ${newTargetDir}"
+  exit 1
+fi
+
+# Building targets
+echo "Building the old version ... "
+checkoutAndBuild "$olderCommit" "$oldTargetDir"
+echo "Building the new version ..."
+checkoutAndBuild "$newerCommit" "$newTargetDir"
diff --git a/compatibility-verifier/compCheck.sh b/compatibility-verifier/compCheck.sh
index 141e6c4..9660ec9 100755
--- a/compatibility-verifier/compCheck.sh
+++ b/compatibility-verifier/compCheck.sh
@@ -34,61 +34,62 @@
 #  and run all the scripts in the directory in alpha order, one script at each
 #  "stage" of upgrade.
 #
-#  We may modify to choose a minimal run in which the same set of operatons are run
+#  We may modify to choose a minimal run in which the same set of operations are run
 #  between any two component upgrades/rollbacks -- this may consist of adding
 #  one more segment to table, adding some more rows to the stream topic, and
 #  running some queries with the new data.
 
-# get a temporary directory in case the workingDir is not provided by user
-TMP_DIR=$(mktemp -d 2>/dev/null || mktemp -d -t 'mytmpdir')
-
-COMPAT_TESTER_PATH="pinot-integration-tests/target/pinot-integration-tests-pkg/bin/pinot-compat-test-runner.sh"
 
 # get usage of the script
 function usage() {
   command=$1
-  echo "Usage: $command olderCommit newerCommit [workingDir]"
+  echo "Usage: $command [workingDir]"
   exit 1
 }
 
-# cleanup the temporary directory when exiting the script
-function cleanup() {
-  if [ -n "$TMP_DIR" ] && [ -d "$TMP_DIR" ] && [ "$workingDir" = "$TMP_DIR" ] ; then
-    echo "The temporary directory $TMP_DIR needs to be cleaned up."
-  fi
+function waitForZkReady() {
+  # TODO: check ZK to be ready instead of sleep
+  sleep 60
+  echo "zookeeper is ready"
 }
 
-# This function builds Pinot given a specific commit hash and target directory
-function checkoutAndBuild() {
-  commitHash=$1
-  targetDir=$2
-
-  pushd "$targetDir" || exit 1
-  git init
-  git remote add origin https://github.com/apache/incubator-pinot
-  git fetch --depth 1 origin "$commitHash"
-  git checkout FETCH_HEAD
-  mvn install package -DskipTests -Pbin-dist
-  popd || exit 1
+function waitForControllerReady() {
+  # TODO: check Controller to be ready instead of sleep
+  sleep 60
+  echo "controller is ready"
 }
 
+function waitForKafkaReady() {
+  # TODO: check kafka to be ready instead of sleep
+  sleep 10
+  echo "kafka is ready"
+}
+
+function waitForClusterReady() {
+  # TODO: check cluster to be ready instead of sleep
+  sleep 2
+  echo "Cluster ready."
+}
 # Given a component and directory, start that version of the specific component
 function startService() {
   serviceName=$1
   dirName=$2
-  # Upon start, save the pid of the process for a component into a file in /tmp/{component}.pid, which is then used to stop it
+  # Upon start, save the pid of the process for a component into a file in /working_dir/{component}.pid, which is then used to stop it
   pushd "$dirName"/pinot-tools/target/pinot-tools-pkg/bin  || exit 1
   if [ "$serviceName" = "zookeeper" ]; then
-    sh -c 'echo $$ > $0/zookeeper.pid; exec ./pinot-admin.sh StartZookeeper' "${dirName}" &
+    sh -c 'rm -rf ${0}/zkdir'
+    sh -c 'echo $$ > $0/zookeeper.pid; exec ./pinot-admin.sh StartZookeeper -dataDir ${0}/zkdir > ${0}/zookeeper.log 2>&1' "${dirName}" &
   elif [ "$serviceName" = "controller" ]; then
-    sh -c 'echo $$ > $0/controller.pid; exec ./pinot-admin.sh StartController' "${dirName}" &
+    sh -c 'echo $$ > $0/controller.pid; exec ./pinot-admin.sh StartController > ${0}/controller.log 2>&1' "${dirName}" &
   elif [ "$serviceName" = "broker" ]; then
-    sh -c 'echo $$ > $0/broker.pid; exec ./pinot-admin.sh StartBroker' "${dirName}" &
+    sh -c 'echo $$ > $0/broker.pid; exec ./pinot-admin.sh StartBroker > ${0}/broker.log 2>&1' "${dirName}" &
   elif [ "$serviceName" = "server" ]; then
-    sh -c 'echo $$ > $0/server.pid; exec ./pinot-admin.sh StartServer' "${dirName}" &
+    sh -c 'echo $$ > $0/server.pid; exec ./pinot-admin.sh StartServer > ${0}/server.log 2>&1' "${dirName}" &
   elif [ "$serviceName" = "kafka" ]; then
-    sh -c 'echo $$ > $0/kafka.pid; exec ./pinot-admin.sh StartKafka -zkAddress localhost:2181/kafka' "${dirName}" &
+    sh -c 'echo $$ > $0/kafka.pid; exec ./pinot-admin.sh StartKafka -zkAddress localhost:2181/kafka > ${0}/kafka.log 2>&1' "${dirName}" &
   fi
+
+  echo "${serviceName} started"
   popd || exit 1
 }
 
@@ -105,17 +106,24 @@ function stopService() {
   else
     echo "Pid file ${dirName}/${serviceName}.pid  not found. Failed to stop component ${serviceName}"
   fi
+  echo "${serviceName} stopped"
 }
 
 # Starts a Pinot cluster given a specific target directory
 function startServices() {
   dirName=$1
   startService zookeeper "$dirName"
+  # Controller depends on zookeeper, if not wait zookeeper to be ready, controller will crash.
+  waitForZkReady
   startService controller "$dirName"
+  # Broker depends on controller, if not wait controller to be ready, broker will crash.
+  waitForControllerReady
   startService broker "$dirName"
   startService server "$dirName"
   startService kafka "$dirName"
+  waitForKafkaReady
   echo "Cluster started."
+  waitForClusterReady
 }
 
 # Stops the currently running Pinot cluster
@@ -142,73 +150,18 @@ function setupCompatTester() {
 # Main
 #
 
-# cleanp the temporary directory when the bash script exits
-trap cleanup EXIT
-
-setupCompatTester
-
-###############################################################################
-# XXX BEGIN Temporary
-# While the individual components are under development, it is useful to start
-# zookeeper, controler, broker, server and kafka outside of this command and
-# debug as needed.
-#
-# Start the components as follows (or in debugger, if debugging)
-#
-#   rm -rf /tmp/zkdir && ${PINOT_ADMIN_CMD} StartZookeeper -dataDir /tmp/zkdir
-#   ${PINOT_ADMIN_CMD} StartController
-#   ${PINOT_ADMIN_CMD} StartBroker
-#   ${PINOT_ADMIN_CMD} StartServer
-#   ${PINOT_ADMIN_CMD} StartKafka -zkAddress localhost:2181
-#
-# To compile the compat tester command alone, do the following:
-#   cd incubator-pinot
-#   mvn clean install -DskipTests
-#   mvn -pl pinot-integration-tests  package -DskipTests
-#
-if [ $# -ne 2 ]; then echo "Usage: $0 <yaml-file-name> <generation-number> (Be sure to start all components)"; exit 1; fi
-${COMPAT_TESTER} $1 $2; if [ $? -ne 0 ]; then echo "Command failed"; exit 1; fi
-exit 0
-# XXX END Temporary
-##############################################################################
-
-if [ $# -lt 2 ] || [ $# -gt 3 ] ; then
+if [ $# -ne 1 ] ; then
   usage compCheck
 fi
 
-# get arguments
-olderCommit=$1
-newerCommit=$2
-
-if [ -n "$3" ]; then
-  workingDir=$3
-  if [ -d "$workingDir" ]; then
-    echo "Directory ${workingDir} already exists. Use a new directory."
-    exit 1
-  fi
-else
-  # use the temp directory in case workingDir is not provided
-  workingDir=$TMP_DIR
-fi
+COMPAT_TESTER_PATH="pinot-integration-tests/target/pinot-integration-tests-pkg/bin/pinot-compat-test-runner.sh"
 
 # create subdirectories for given commits
+workingDir=$1
 oldTargetDir="$workingDir"/oldTargetDir
 newTargetDir="$workingDir"/newTargetDir
 
-if ! mkdir -p "$oldTargetDir"; then
-  echo "Failed to create target directory ${oldTargetDir}"
-  exit 1
-fi
-if ! mkdir -p "$newTargetDir"; then
-  echo "Failed to create target directory ${newTargetDir}"
-  exit 1
-fi
-
-# Building targets
-echo "Building the old version ... "
-checkoutAndBuild "$olderCommit" "$oldTargetDir"
-echo "Building the new version ..."
-checkoutAndBuild "$newerCommit" "$newTargetDir"
+setupCompatTester
 
 # check that the default ports are open
 if [ "$(lsof -t -i:8097 -s TCP:LISTEN)" ] || [ "$(lsof -t -i:8098 -sTCP:LISTEN)" ] || [ "$(lsof -t -i:8099 -sTCP:LISTEN)" ] ||
@@ -217,29 +170,32 @@ if [ "$(lsof -t -i:8097 -s TCP:LISTEN)" ] || [ "$(lsof -t -i:8098 -sTCP:LISTEN)"
   exit 1
 fi
 
+
 # Setup initial cluster with olderCommit and do rolling upgrade
 startServices "$oldTargetDir"
-#$COMPAT_TESTER pre-controller-upgrade.yaml; if [ $? -ne 0 ]; then exit 1; fi
+#$COMPAT_TESTER pre-controller-upgrade.yaml 1; if [ $? -ne 0 ]; then exit 1; fi
 stopService controller "$oldTargetDir"
 startService controller "$newTargetDir"
-#$COMPAT_TESTER pre-broker-upgrade.yaml; if [ $? -ne 0 ]; then exit 1; fi
+waitForControllerReady
+#$COMPAT_TESTER pre-broker-upgrade.yaml 2; if [ $? -ne 0 ]; then exit 1; fi
 stopService broker "$oldTargetDir"
 startService broker "$newTargetDir"
-#$COMPAT_TESTER pre-server-upgrade.yaml; if [ $? -ne 0 ]; then exit 1; fi
+#$COMPAT_TESTER pre-server-upgrade.yaml 3; if [ $? -ne 0 ]; then exit 1; fi
 stopService server "$oldTargetDir"
 startService server "$newTargetDir"
-#$COMPAT_TESTER post-server-upgrade.yaml; if [ $? -ne 0 ]; then exit 1; fi
+#$COMPAT_TESTER post-server-upgrade.yaml 4; if [ $? -ne 0 ]; then exit 1; fi
 
-# Upgrade complated, now do a rollback
+# Upgrade completed, now do a rollback
 stopService server "$newTargetDir"
 startService server "$oldTargetDir"
-#$COMPAT_TESTER post-server-rollback.yaml; if [ $? -ne 0 ]; then exit 1; fi
+#$COMPAT_TESTER post-server-rollback.yaml 5; if [ $? -ne 0 ]; then exit 1; fi
 stopService broker "$newTargetDir"
 startService broker "$oldTargetDir"
-#$COMPAT_TESTER post-broker-rollback.yaml; if [ $? -ne 0 ]; then exit 1; fi
+#$COMPAT_TESTER post-broker-rollback.yaml 6; if [ $? -ne 0 ]; then exit 1; fi
 stopService controller "$newTargetDir"
 startService controller "$oldTargetDir"
-#$COMPAT_TESTER post-controller-rollback.yaml; if [ $? -ne 0 ]; then exit 1; fi
+waitForControllerReady
+#$COMPAT_TESTER post-controller-rollback.yaml 7; if [ $? -ne 0 ]; then exit 1; fi
 stopServices "$oldTargetDir"
 
-exit 0
+exit 0
\ No newline at end of file
diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/compat/tests/SegmentOp.java b/pinot-integration-tests/src/test/java/org/apache/pinot/compat/tests/SegmentOp.java
index b0e26e7..f7ff5aa 100644
--- a/pinot-integration-tests/src/test/java/org/apache/pinot/compat/tests/SegmentOp.java
+++ b/pinot-integration-tests/src/test/java/org/apache/pinot/compat/tests/SegmentOp.java
@@ -184,9 +184,9 @@ public class SegmentOp extends BaseOp {
       throws Exception {
     TableConfig tableConfig = JsonUtils.fileToObject(new File(_tableConfigFileName), TableConfig.class);
     _tableName = tableConfig.getTableName();
-    // if user does not specify segmentName, use tableName + generationNumber
+    // if user does not specify segmentName, use tableName_generationNumber
     if (_segmentName == null || _segmentName.isEmpty()) {
-      _segmentName = _tableName + _generationNumber;
+      _segmentName = _tableName + "_" + _generationNumber;
     }
 
     Schema schema = JsonUtils.fileToObject(new File(_schemaFileName), Schema.class);
@@ -285,9 +285,9 @@ public class SegmentOp extends BaseOp {
     try {
       TableConfig tableConfig = JsonUtils.fileToObject(new File(_tableConfigFileName), TableConfig.class);
       _tableName = tableConfig.getTableName();
-      // if user does not specify segmentName, use tableName + generationNumber
+      // if user does not specify segmentName, use tableName_generationNumber
       if (_segmentName == null || _segmentName.isEmpty()) {
-        _segmentName = _tableName + _generationNumber;
+        _segmentName = _tableName + "_" + _generationNumber;
       }
 
       ControllerTest.sendDeleteRequest(ControllerRequestURLBuilder.baseUrl(ClusterDescriptor.CONTROLLER_URL)
diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/compat/tests/SqlResultComparator.java b/pinot-integration-tests/src/test/java/org/apache/pinot/compat/tests/SqlResultComparator.java
index c117452..29b614b 100644
--- a/pinot-integration-tests/src/test/java/org/apache/pinot/compat/tests/SqlResultComparator.java
+++ b/pinot-integration-tests/src/test/java/org/apache/pinot/compat/tests/SqlResultComparator.java
@@ -43,8 +43,9 @@ public class SqlResultComparator {
 
   private static final String FIELD_RESULT_TABLE = "resultTable";
   private static final String FIELD_DATA_SCHEMA = "dataSchema";
-  private static final String FIELD_ROWS = "rows";
+  private static final String FIELD_COLUMN_NAMES = "columnNames";
   private static final String FIELD_COLUMN_DATA_TYPES = "columnDataTypes";
+  private static final String FIELD_ROWS = "rows";
   private static final String FIELD_IS_SUPERSET = "isSuperset";
   private static final String FIELD_NUM_DOCS_SCANNED = "numDocsScanned";
   private static final String FIELD_EXCEPTIONS = "exceptions";
@@ -126,11 +127,12 @@ public class SqlResultComparator {
   }
 
   public static boolean areMetadataEqual(JsonNode actual, JsonNode expected) {
-    return areNumServersQueriedEqual(actual, expected) && areNumServersRespondedEqual(actual, expected)
-        && areNumSegmentsQueriedEqual(actual, expected) && areNumSegmentsProcessedEqual(actual, expected)
-        && areNumSegmentsMatchedEqual(actual, expected) && areNumConsumingSegmentsQueriedEqual(actual, expected)
-        && areNumDocsScannedEqual(actual, expected) && areNumEntriesScannedInFilterEqual(actual, expected)
-        && areNumEntriesScannedPostFilterEqual(actual, expected) && areNumGroupsLimitReachedEqual(actual, expected);
+    /*
+     * Since we add more and more several segments with different generations during compatibility test,
+     * metadata such as "numSegmentsQueried", "numDocsScanned" will be different, we only compare
+     * "numServersQueried" and "numServersResponded" here.
+     * */
+    return areNumServersQueriedEqual(actual, expected) && areNumServersRespondedEqual(actual, expected);
   }
 
   private static boolean areNumGroupsLimitReachedEqual(JsonNode actual, JsonNode expected) {
@@ -269,13 +271,20 @@ public class SqlResultComparator {
   }
 
   private static boolean areDataSchemaEqual(JsonNode actual, JsonNode expected) {
-    JsonNode actualDataSchema = actual.get(FIELD_RESULT_TABLE).get(FIELD_DATA_SCHEMA);
-    JsonNode expecteDataSchema = expected.get(FIELD_RESULT_TABLE).get(FIELD_DATA_SCHEMA);
-
-    String actualDataSchemaStr = actualDataSchema.toString();
-    String expecteDataSchemaStr = expecteDataSchema.toString();
-    if (!actualDataSchemaStr.equals(expecteDataSchemaStr)) {
-      LOGGER.error("The dataSchema don't match! Actual: {}, Expected: {}", actualDataSchema, expecteDataSchema);
+    /*
+    * Field "dataSchema" is an array, which contains "columnNames" and "columnDataTypes". However there is no orders
+    * between "columnNames" and "columnDataTypes", so we extract and append them when compare instead of compare
+    * "dataSchema" directly.
+    * */
+    JsonNode actualColumnNames = actual.get(FIELD_RESULT_TABLE).get(FIELD_DATA_SCHEMA).get(FIELD_COLUMN_NAMES);
+    JsonNode expectedColumnNames = expected.get(FIELD_RESULT_TABLE).get(FIELD_DATA_SCHEMA).get(FIELD_COLUMN_NAMES);
+    JsonNode actualColumnDataTypes = actual.get(FIELD_RESULT_TABLE).get(FIELD_DATA_SCHEMA).get(FIELD_COLUMN_DATA_TYPES);
+    JsonNode expectedColumnDataTypes = expected.get(FIELD_RESULT_TABLE).get(FIELD_DATA_SCHEMA).get(FIELD_COLUMN_DATA_TYPES);
+
+    String actualDataSchemaStr = actualColumnNames.toString() + actualColumnDataTypes.toString();
+    String expectedDataSchemaStr = expectedColumnNames.toString() + expectedColumnDataTypes.toString();
+    if (!actualDataSchemaStr.equals(expectedDataSchemaStr)) {
+      LOGGER.error("The dataSchema don't match! Actual: {}, Expected: {}", actualDataSchemaStr, expectedDataSchemaStr);
       return false;
     }
     return true;
diff --git a/pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1.json b/pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1.json
index 3e1acf4..a160228 100644
--- a/pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1.json
+++ b/pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1.json
@@ -21,7 +21,9 @@
     "retentionTimeValue": "",
     "segmentAssignmentStrategy": "BalanceNumSegmentAssignmentStrategy",
     "segmentPushFrequency": "daily",
-    "segmentPushType": "REFRESH"
+    "segmentPushType": "REFRESH",
+    "timeColumnName": "HoursSinceEpoch",
+    "timeType": "HOURS"
   },
   "tableIndexConfig": {
     "aggregateMetrics": false,
diff --git a/pinot-integration-tests/src/test/resources/compat-tests/data/FeatureTest1-data-00.csv b/pinot-integration-tests/src/test/resources/compat-tests/data/FeatureTest1-data-00.csv
index 410ea8b..bb537d9 100644
--- a/pinot-integration-tests/src/test/resources/compat-tests/data/FeatureTest1-data-00.csv
+++ b/pinot-integration-tests/src/test/resources/compat-tests/data/FeatureTest1-data-00.csv
@@ -1,13 +1,12 @@
-
-# generationNumber stringDimSV1 stringDimSV2 longDimSV1 longDimSV2 stringDimMV1 stringDimMV2 intDimMV1 intDimMV2 textDim1 mapDim1__KEYS mapDim1__VALUES mapDim2json intMetric1 longMetric1 floatMetric1 doubleMetric1
-__GENERATION_NUMBER__,"s1-0",s2-0,1,2,m1-0-0;m1-0-1,m2-0-0;m2-0-1,3;4,6;7,Java C++ Python,01a0bc,k1;k2;k3;k4;k5,1;1;2;2;2,"{""k1"":1,""k2"":1,""k3"":2,""k4"":2,""k5"":2}",10,11,12.1,13.1
-__GENERATION_NUMBER__,"s1-0",s2-0,1,2,m1-0-0;m1-0-1,m2-0-0;m2-0-1,3;4,6;7,Java C++ Python,4877625602,k1;k2;k3;k4;k5,3;3;3;3;3,"{""k1"":3,""k2"":3,""k3"":3,""k4"":3,""k5"":3}",10,11,12.1,13.1, # Dupliate of row 0 1
-__GENERATION_NUMBER__,s1-2,s2-2,11,21,m1-2-0;m1-2-1,m2-2-0;m2-2-1,32;42,62;72,Java C++ golang,13225573e3f5,k1;k2;k3;k4;k5,4;5;6;7;7,"{""k1"":4,""k2"":5,""k3"":6,""k4"":7,""k5"":7}",10,21,22.1,23.10
-__GENERATION_NUMBER__,s1-2,s2-2,11,21,m1-3-0;m1-3-1,m2-3-0;m2-3-1,32;42,62;72,Java C++ golang,deadbeef,k1;k2;k3;k4;k5,7;7;7;7;7,"{""k1"":7,""k2"":7,""k3"":7,""k4"":7,""k5"":7}",10,21,22.1,23.10, # All sv cols same as prev
-__GENERATION_NUMBER__,s1-4,s2-4,41,22,m1-2-0;m1-2-1,m2-2-0;m2-2-1,42;52,72;82,Java C++ golang,deed0507,k1;k2;k3;k4;k5,7;7;8;8;8,"{""k1"":7,""k2"":7,""k3"":8,""k4"":8,""k5"":8}",14,24,24.1,24.10, # All mv cols same as row 2
-__GENERATION_NUMBER__,s1-5,,,32,m1-5-0,m2-2-0,,92;22,golang shell bash,,k1;k2;k3;k4;k5,7;7;7;7;7,"{""k1"":7,""k2"":7,""k3"":7,""k4"":7,""k5"":7}",,24,,24.10, # Default values for some columns
-__GENERATION_NUMBER__,s1-6,s2-6,7611,7621,m1-5-0;m1-5-1;m1-5-2,m2-2-0,392;462,6662;782,C++ golang python,deed0507,k1;k2;k3;k4;k5,7;8;9;10;20,"{""k1"":7,""k2"":8,""k3"":9,""k4"":10,""k5"":20}",101,251,262.1,263.10, # 3 values in MV
-__GENERATION_NUMBER__,s1-6,s2-6,7611,7621,m1-5-0;m1-5-1;m1-5-2,m2-2-0,392;462,6662;782,C++ golang python,deed0507,k1;k2;k3;k4;k5,7;8;9;10;20,"{""k1"":7,""k2"":8,""k3"":9,""k4"":10,""k5"":20}",2147483647,251,262.1,263.10, # MAX_INT in int metric
-__GENERATION_NUMBER__,s1-6,s2-6,7611,7621,m1-5-0;m1-5-1;m1-5-2,m2-2-0,392;462,6662;782,C++ golang python,deed0507,k1;k2;k3;k4;k5,7;8;9;10;20,"{""k1"":7,""k2"":8,""k3"":9,""k4"":10,""k5"":20}",2147483647,251,262.1,263.10, # MAX_INT in int metric
-__GENERATION_NUMBER__,s1-7,s2-7,6766,6777,m1-6-0;m1-6-1;m1-6-2;m1-6-3,m2-6-0;m2-6-1,392;462,6662;782,golang Java,d54d0507,k1;k2;k3;k4;k5,31;31;32;32;32,"{""k1"":31,""k2"":31,""k3"":32,""k4"":32,""k5"":32}",87,251,262.10,263.10
-
+# HoursSinceEpoch generationNumber stringDimSV1 stringDimSV2 longDimSV1 longDimSV2 stringDimMV1 stringDimMV2 intDimMV1 intDimMV2 textDim1 mapDim1__KEYS mapDim1__VALUES mapDim2json intMetric1 longMetric1 floatMetric1 doubleMetric1
+# Add some common rows from first segment, and some new rows as well
+123456,__GENERATION_NUMBER__,"s1-0",s2-0,1,2,m1-0-0;m1-0-1,m2-0-0;m2-0-1,3;4,6;7,Java C++ Python,01a0bc,k1;k2;k3;k4;k5,1;1;2;2;2,"{""k1"":1,""k2"":1,""k3"":2,""k4"":2,""k5"":2}",10,11,12.1,13.1
+123456,__GENERATION_NUMBER__,"s1-0",s2-0,1,2,m1-0-0;m1-0-1,m2-0-0;m2-0-1,3;4,6;7,Java C++ Python,4877625602,k1;k2;k3;k4;k5,3;3;3;3;3,"{""k1"":3,""k2"":3,""k3"":3,""k4"":3,""k5"":3}",10,11,12.1,13.1, # Dupliate of row 0 1
+123456,__GENERATION_NUMBER__,s1-2,s2-2,11,21,m1-2-0;m1-2-1,m2-2-0;m2-2-1,32;42,62;72,Java C++ golang,13225573e3f5,k1;k2;k3;k4;k5,4;5;6;7;7,"{""k1"":4,""k2"":5,""k3"":6,""k4"":7,""k5"":7}",10,21,22.1,23.10
+123456,__GENERATION_NUMBER__,s1-2,s2-2,11,21,m1-3-0;m1-3-1,m2-3-0;m2-3-1,32;42,62;72,Java C++ golang,deadbeef,k1;k2;k3;k4;k5,7;7;7;7;7,"{""k1"":7,""k2"":7,""k3"":7,""k4"":7,""k5"":7}",10,21,22.1,23.10, # All sv cols same as prev
+123456,__GENERATION_NUMBER__,s1-4,s2-4,41,22,m1-2-0;m1-2-1,m2-2-0;m2-2-1,42;52,72;82,Java C++ golang,deed0507,k1;k2;k3;k4;k5,7;7;8;8;8,"{""k1"":7,""k2"":7,""k3"":8,""k4"":8,""k5"":8}",14,24,24.1,24.10, # All mv cols same as row 2
+123456,__GENERATION_NUMBER__,s1-5,,,32,m1-5-0,m2-2-0,,92;22,golang shell bash,,k1;k2;k3;k4;k5,7;7;7;7;7,"{""k1"":7,""k2"":7,""k3"":7,""k4"":7,""k5"":7}",,24,,24.10, # Default values for some columns
+123456,__GENERATION_NUMBER__,s1-6,s2-6,7611,7621,m1-5-0;m1-5-1;m1-5-2,m2-2-0,392;462,6662;782,C++ golang python,deed0507,k1;k2;k3;k4;k5,7;8;9;10;20,"{""k1"":7,""k2"":8,""k3"":9,""k4"":10,""k5"":20}",101,251,262.1,263.10, # 3 values in MV
+123456,__GENERATION_NUMBER__,s1-6,s2-6,7611,7621,m1-5-0;m1-5-1;m1-5-2,m2-2-0,392;462,6662;782,C++ golang python,deed0507,k1;k2;k3;k4;k5,7;8;9;10;20,"{""k1"":7,""k2"":8,""k3"":9,""k4"":10,""k5"":20}",2147483647,251,262.1,263.10, # MAX_INT in int metric
+123456,__GENERATION_NUMBER__,s1-6,s2-6,7611,7621,m1-5-0;m1-5-1;m1-5-2,m2-2-0,392;462,6662;782,C++ golang python,deed0507,k1;k2;k3;k4;k5,7;8;9;10;20,"{""k1"":7,""k2"":8,""k3"":9,""k4"":10,""k5"":20}",2147483647,251,262.1,263.10, # MAX_INT in int metric
+123456,__GENERATION_NUMBER__,s1-7,s2-7,6766,6777,m1-6-0;m1-6-1;m1-6-2;m1-6-3,m2-6-0;m2-6-1,392;462,6662;782,golang Java,d54d0507,k1;k2;k3;k4;k5,31;31;32;32;32,"{""k1"":31,""k2"":31,""k3"":32,""k4"":32,""k5"":32}",87,251,262.10,263.10
diff --git a/pinot-integration-tests/src/test/resources/compat-tests/data/FeatureTest1-data-01.csv b/pinot-integration-tests/src/test/resources/compat-tests/data/FeatureTest1-data-01.csv
index 84966ea..0636a6e 100644
--- a/pinot-integration-tests/src/test/resources/compat-tests/data/FeatureTest1-data-01.csv
+++ b/pinot-integration-tests/src/test/resources/compat-tests/data/FeatureTest1-data-01.csv
@@ -1,10 +1,9 @@
-
-# generationNumber stringDimSV1 stringDimSV2 longDimSV1 longDimSV2 stringDimMV1 stringDimMV2 intDimMV1 intDimMV2 textDim1 mapDim1__KEYS mapDim1__VALUES intMetric1 longMetric1 floatMetric1 doubleMetric1
+# HoursSinceEpoch generationNumber stringDimSV1 stringDimSV2 longDimSV1 longDimSV2 stringDimMV1 stringDimMV2 intDimMV1 intDimMV2 textDim1 mapDim1__KEYS mapDim1__VALUES intMetric1 longMetric1 floatMetric1 doubleMetric1
 # Add some common rows from first segment, and some new rows as well
-__GENERATION_NUMBER__,"s1-0",s2-0,1,2,m1-0-0;m1-0-1,m2-0-0;m2-0-1,3;4,6;7,Java C++ Python,01a0bc,k1;k2;k3;k4;k5,1;1;2;2;2,"{""k1"":1,""k2"":1,""k3"":2,""k4"":2,""k5"":2}",10,11,12.1,13.1
-__GENERATION_NUMBER__,s1-2,s2-2,11,21,m1-2-0;m1-2-1,m2-2-0;m2-2-1,32;42,62;72,Java C++ golang,13225573e3f5,k1;k2;k3;k4;k5,4;5;6;7;7,"{""k1"":4,""k2"":5,""k3"":6,""k4"":7,""k5"":7}",10,21,22.1,23.10
-__GENERATION_NUMBER__,s1-5,,,32,m1-5-0,m2-2-0,,92;22,golang shell bash,,k1;k2;k3;k4;k5,7;7;7;7;7,"{""k1"":7,""k2"":7,""k3"":7,""k4"":7,""k5"":7}",,24,,24.10, # Default values for some columns
-__GENERATION_NUMBER__,s1-6,s2-6,7611,7621,m1-5-0;m1-5-1;m1-5-2,m2-2-0,392;462,6662;782,C++ golang python,deed0507,k1;k2;k3;k4;k5,7;8;9;10;20,"{""k1"":7,""k2"":8,""k3"":9,""k4"":10,""k5"":20}",101,251,262.1,263.10, # 3 values in MV
-__GENERATION_NUMBER__,s1-6,s2-6,7611,7621,m1-5-0;m1-5-1;m1-5-2,m2-2-0,392;462,6662;782,C++ golang python,deed0507,k1;k2;k3;k4;k5,7;8;9;10;20,"{""k1"":7,""k2"":8,""k3"":9,""k4"":10,""k5"":20}",2147483647,251,262.1,263.10, # MAX_INT in int metric
-__GENERATION_NUMBER__,foo1,foo2,234,983,foo;bar;baz,one,392;462,6662;782,golang python MS-Word,097865,k1;k2;k3;k4;k5,131;131;132;132;132,"{""k1"":131,""k2"":131,""k3"":132,""k4"":132,""k5"":132}",65,251,262.1,263.10
-__GENERATION_NUMBER__,foo4,foo4,234,983,foo;bar,one;two,32;62,662;72,python MS-Word UNIX,6755884e,k1;k2;k3;k4;k5,231;231;232;232;232,"{""k1"":231,""k2"":231,""k3"":232,""k4"":232,""k5"":232}",165,21,22.01,23.90
+123456,__GENERATION_NUMBER__,"s1-0",s2-0,1,2,m1-0-0;m1-0-1,m2-0-0;m2-0-1,3;4,6;7,Java C++ Python,01a0bc,k1;k2;k3;k4;k5,1;1;2;2;2,"{""k1"":1,""k2"":1,""k3"":2,""k4"":2,""k5"":2}",10,11,12.1,13.1
+123456,__GENERATION_NUMBER__,s1-2,s2-2,11,21,m1-2-0;m1-2-1,m2-2-0;m2-2-1,32;42,62;72,Java C++ golang,13225573e3f5,k1;k2;k3;k4;k5,4;5;6;7;7,"{""k1"":4,""k2"":5,""k3"":6,""k4"":7,""k5"":7}",10,21,22.1,23.10
+123456,__GENERATION_NUMBER__,s1-5,,,32,m1-5-0,m2-2-0,,92;22,golang shell bash,,k1;k2;k3;k4;k5,7;7;7;7;7,"{""k1"":7,""k2"":7,""k3"":7,""k4"":7,""k5"":7}",,24,,24.10, # Default values for some columns
+123456,__GENERATION_NUMBER__,s1-6,s2-6,7611,7621,m1-5-0;m1-5-1;m1-5-2,m2-2-0,392;462,6662;782,C++ golang python,deed0507,k1;k2;k3;k4;k5,7;8;9;10;20,"{""k1"":7,""k2"":8,""k3"":9,""k4"":10,""k5"":20}",101,251,262.1,263.10, # 3 values in MV
+123456,__GENERATION_NUMBER__,s1-6,s2-6,7611,7621,m1-5-0;m1-5-1;m1-5-2,m2-2-0,392;462,6662;782,C++ golang python,deed0507,k1;k2;k3;k4;k5,7;8;9;10;20,"{""k1"":7,""k2"":8,""k3"":9,""k4"":10,""k5"":20}",2147483647,251,262.1,263.10, # MAX_INT in int metric
+123456,__GENERATION_NUMBER__,foo1,foo2,234,983,foo;bar;baz,one,392;462,6662;782,golang python MS-Word,097865,k1;k2;k3;k4;k5,131;131;132;132;132,"{""k1"":131,""k2"":131,""k3"":132,""k4"":132,""k5"":132}",65,251,262.1,263.10
+123456,__GENERATION_NUMBER__,foo4,foo4,234,983,foo;bar,one;two,32;62,662;72,python MS-Word UNIX,6755884e,k1;k2;k3;k4;k5,231;231;232;232;232,"{""k1"":231,""k2"":231,""k3"":232,""k4"":232,""k5"":232}",165,21,22.01,23.90
diff --git a/pinot-integration-tests/src/test/resources/compat-tests/data/recordReaderConfig.json b/pinot-integration-tests/src/test/resources/compat-tests/data/recordReaderConfig.json
index 07ea5e1..ebaa04e 100644
--- a/pinot-integration-tests/src/test/resources/compat-tests/data/recordReaderConfig.json
+++ b/pinot-integration-tests/src/test/resources/compat-tests/data/recordReaderConfig.json
@@ -1,5 +1,5 @@
 {
     "commentMarker" : "#",
     "header" :
-            "generationNumber,stringDimSV1,stringDimSV2,longDimSV1,longDimSV2,stringDimMV1,stringDimMV2,intDimMV1,intDimMV2,textDim1,bytesDimSV1,mapDim1__KEYS,mapDim1__VALUES,mapDim2json,intMetric1,longMetric1,floatMetric1,doubleMetric1"
+            "HoursSinceEpoch,generationNumber,stringDimSV1,stringDimSV2,longDimSV1,longDimSV2,stringDimMV1,stringDimMV2,intDimMV1,intDimMV2,textDim1,bytesDimSV1,mapDim1__KEYS,mapDim1__VALUES,mapDim2json,intMetric1,longMetric1,floatMetric1,doubleMetric1"
 }
diff --git a/pinot-integration-tests/src/test/resources/compat-tests/stream-op-publish.yaml b/pinot-integration-tests/src/test/resources/compat-tests/kafka-stream-produce-op.yaml
similarity index 70%
copy from pinot-integration-tests/src/test/resources/compat-tests/stream-op-publish.yaml
copy to pinot-integration-tests/src/test/resources/compat-tests/kafka-stream-produce-op.yaml
index 14119a6..1475284 100644
--- a/pinot-integration-tests/src/test/resources/compat-tests/stream-op-publish.yaml
+++ b/pinot-integration-tests/src/test/resources/compat-tests/kafka-stream-produce-op.yaml
@@ -17,19 +17,9 @@
 # under the License.
 #
 
-# Operations to be done.
-description: tst file for stream op PRODUCE
+description: Yaml file for publishing rows to Kafka topic PinotRealtimeFeatureTest1Event
 operations:
   - type: streamOp
-    description: create Kafka topic
-    op: CREATE
-    streamConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1-realtime-stream-config.json
-  - type: tableOp
-    description: Create table feature-test-1-realtime.json
-    op: CREATE
-    schemaFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/FeatureTest1-schema.json
-    tableConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1-realtime.json
-  - type: streamOp
     description: publish rows to PinotRealtimeFeatureTest1Event
     op: PRODUCE
     streamConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1-realtime-stream-config.json
diff --git a/pinot-integration-tests/src/test/resources/compat-tests/kafka-topic-create-op.yaml b/pinot-integration-tests/src/test/resources/compat-tests/kafka-topic-create-op.yaml
new file mode 100644
index 0000000..f2c6bea
--- /dev/null
+++ b/pinot-integration-tests/src/test/resources/compat-tests/kafka-topic-create-op.yaml
@@ -0,0 +1,25 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+description: Yaml file for creating Kafka topic PinotRealtimeFeatureTest1Event
+operations:
+  - type: streamOp
+    description: create Kafka topic PinotRealtimeFeatureTest1Event
+    op: CREATE
+    streamConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1-realtime-stream-config.json
\ No newline at end of file
diff --git a/pinot-integration-tests/src/test/resources/compat-tests/stream-op-publish.yaml b/pinot-integration-tests/src/test/resources/compat-tests/offline-table-create-op.yaml
similarity index 50%
copy from pinot-integration-tests/src/test/resources/compat-tests/stream-op-publish.yaml
copy to pinot-integration-tests/src/test/resources/compat-tests/offline-table-create-op.yaml
index 14119a6..391927c 100644
--- a/pinot-integration-tests/src/test/resources/compat-tests/stream-op-publish.yaml
+++ b/pinot-integration-tests/src/test/resources/compat-tests/offline-table-create-op.yaml
@@ -17,23 +17,10 @@
 # under the License.
 #
 
-# Operations to be done.
-description: tst file for stream op PRODUCE
+description: Yaml file for creating a offline table
 operations:
-  - type: streamOp
-    description: create Kafka topic
-    op: CREATE
-    streamConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1-realtime-stream-config.json
   - type: tableOp
-    description: Create table feature-test-1-realtime.json
+    description: Create table feature-test-1.json
     op: CREATE
     schemaFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/FeatureTest1-schema.json
-    tableConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1-realtime.json
-  - type: streamOp
-    description: publish rows to PinotRealtimeFeatureTest1Event
-    op: PRODUCE
-    streamConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1-realtime-stream-config.json
-    numRows: 1
-    inputDataFileName: pinot-integration-tests/src/test/resources/compat-tests/data/FeatureTest1-data-00.csv
-    recordReaderConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/data/recordReaderConfig.json
-    tableConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1-realtime.json
+    tableConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1.json
\ No newline at end of file
diff --git a/pinot-integration-tests/src/test/resources/compat-tests/stream-op-publish.yaml b/pinot-integration-tests/src/test/resources/compat-tests/offline-table-delete-op.yaml
similarity index 50%
copy from pinot-integration-tests/src/test/resources/compat-tests/stream-op-publish.yaml
copy to pinot-integration-tests/src/test/resources/compat-tests/offline-table-delete-op.yaml
index 14119a6..e9dddf9 100644
--- a/pinot-integration-tests/src/test/resources/compat-tests/stream-op-publish.yaml
+++ b/pinot-integration-tests/src/test/resources/compat-tests/offline-table-delete-op.yaml
@@ -17,23 +17,10 @@
 # under the License.
 #
 
-# Operations to be done.
-description: tst file for stream op PRODUCE
+description: Yaml file for deleting a offline table
 operations:
-  - type: streamOp
-    description: create Kafka topic
-    op: CREATE
-    streamConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1-realtime-stream-config.json
   - type: tableOp
-    description: Create table feature-test-1-realtime.json
-    op: CREATE
+    description: Delete table feature-test-1.json
+    op: DELETE
     schemaFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/FeatureTest1-schema.json
-    tableConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1-realtime.json
-  - type: streamOp
-    description: publish rows to PinotRealtimeFeatureTest1Event
-    op: PRODUCE
-    streamConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1-realtime-stream-config.json
-    numRows: 1
-    inputDataFileName: pinot-integration-tests/src/test/resources/compat-tests/data/FeatureTest1-data-00.csv
-    recordReaderConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/data/recordReaderConfig.json
-    tableConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1-realtime.json
+    tableConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1.json
\ No newline at end of file
diff --git a/pinot-integration-tests/src/test/resources/compat-tests/queries/feature-test-1-sql.queries b/pinot-integration-tests/src/test/resources/compat-tests/queries/feature-test-1-sql.queries
index 16c5dd3..81edad9 100644
--- a/pinot-integration-tests/src/test/resources/compat-tests/queries/feature-test-1-sql.queries
+++ b/pinot-integration-tests/src/test/resources/compat-tests/queries/feature-test-1-sql.queries
@@ -18,9 +18,9 @@ SELECT textDim1, longDimSV2, doubleMetric1 FROM FeatureTest1 WHERE generationNum
 SELECT textDim1, longDimSV2, doubleMetric1 FROM FeatureTest1 WHERE generationNumber = __GENERATION_NUMBER__ AND bytesDimSV1 = 'deed0507' LIMIT 2
 # This is a query exhausting all records: there are 4 qualified records in total, the query limit the number of returned records as 10
 SELECT textDim1, longDimSV2, doubleMetric1 FROM FeatureTest1 WHERE generationNumber = __GENERATION_NUMBER__ AND bytesDimSV1 = 'deed0507' LIMIT 10
-SELECT stringDimSV1, longDimSV1, intDimMV1, intDimMV2, stringDimMV2 FROM FeatureTest1 WHERE generationNumber = __GENERATION_NUMBER__ AND stringDimSV1 != 's1-6' AND longDimSV1 BETWEEN 10 AND 1000 OR (intDimMV1 < 42 AND stringDimMV2 IN ('m2-0-0', 'm2-2-0') AND intDimMV2 NOT IN (6,72)) ORDER BY longDimSV1 LIMIT 5
-SELECT stringDimSV1, longDimSV1, intDimMV1, intDimMV2, stringDimMV2 FROM FeatureTest1 WHERE generationNumber = __GENERATION_NUMBER__ AND stringDimSV1 != 's1-6' AND longDimSV1 BETWEEN 10 AND 1000 OR (intDimMV1 < 42 AND stringDimMV2 IN ('m2-0-0', 'm2-2-0') AND intDimMV2 NOT IN (6,72)) ORDER BY longDimSV1 DESC
-SELECT stringDimSV1, longDimSV1, intDimMV1, intDimMV2, stringDimMV2 FROM FeatureTest1 WHERE generationNumber = __GENERATION_NUMBER__ AND stringDimSV1 != 's1-6' AND longDimSV1 BETWEEN 10 AND 1000 OR (intDimMV1 < 42 AND stringDimMV2 IN ('m2-0-0', 'm2-2-0') AND intDimMV2 NOT IN (6,72)) ORDER BY longDimSV1 DESC, stringDimSV1 LIMIT 3
+SELECT stringDimSV1, longDimSV1, intDimMV1, intDimMV2, stringDimMV2 FROM FeatureTest1 WHERE generationNumber = __GENERATION_NUMBER__ AND (stringDimSV1 != 's1-6' AND longDimSV1 BETWEEN 10 AND 1000 OR (intDimMV1 < 42 AND stringDimMV2 IN ('m2-0-0', 'm2-2-0') AND intDimMV2 NOT IN (6,72))) ORDER BY longDimSV1 LIMIT 5
+SELECT stringDimSV1, longDimSV1, intDimMV1, intDimMV2, stringDimMV2 FROM FeatureTest1 WHERE generationNumber = __GENERATION_NUMBER__ AND (stringDimSV1 != 's1-6' AND longDimSV1 BETWEEN 10 AND 1000 OR (intDimMV1 < 42 AND stringDimMV2 IN ('m2-0-0', 'm2-2-0') AND intDimMV2 NOT IN (6,72))) ORDER BY longDimSV1 DESC
+SELECT stringDimSV1, longDimSV1, intDimMV1, intDimMV2, stringDimMV2 FROM FeatureTest1 WHERE generationNumber = __GENERATION_NUMBER__ AND (stringDimSV1 != 's1-6' AND longDimSV1 BETWEEN 10 AND 1000 OR (intDimMV1 < 42 AND stringDimMV2 IN ('m2-0-0', 'm2-2-0') AND intDimMV2 NOT IN (6,72))) ORDER BY longDimSV1 DESC, stringDimSV1 LIMIT 3
 
 # Selection & Grouping on Aggregation
 SELECT longDimSV1, intDimMV1, count(*) FROM FeatureTest1 WHERE generationNumber = __GENERATION_NUMBER__ GROUP BY longDimSV1, intDimMV1 ORDER BY longDimSV1 LIMIT 5
@@ -30,11 +30,11 @@ SELECT longDimSV1, intDimMV1, percentile(longDimSV1, 80), percentileMV(intDimMV1
 SELECT longDimSV1, intDimMV1, distinctCount(longDimSV1), distinctCountMV(intDimMV1), distinctCountHLL(longDimSV1), distinctCountHLLMV(intDimMV1) FROM FeatureTest1 WHERE generationNumber = __GENERATION_NUMBER__ GROUP BY longDimSV1, intDimMV1 ORDER BY longDimSV1 LIMIT 5
 
 # Selection & Filtering & Grouping on Aggregation
-SELECT longDimSV1, intDimMV1, count(*) FROM FeatureTest1 WHERE generationNumber = __GENERATION_NUMBER__ AND stringDimSV1 != 's1-6' AND longDimSV1 BETWEEN 10 AND 1000 OR (intDimMV1 < 42 AND stringDimMV2 IN ('m2-0-0', 'm2-2-0') AND intDimMV2 NOT IN (6,72)) GROUP BY longDimSV1, intDimMV1 ORDER BY longDimSV1, intDimMV1 LIMIT 5
-SELECT longDimSV1, intDimMV1, sum(intMetric1), sumMV(intDimMV1), min(intMetric1), minMV(intDimMV2), max(longDimSV1), maxMV(intDimMV1) FROM FeatureTest1 WHERE generationNumber = __GENERATION_NUMBER__ AND stringDimSV1 != 's1-6' AND longDimSV1 BETWEEN 10 AND 1000 OR (intDimMV1 < 42 AND stringDimMV2 IN ('m2-0-0', 'm2-2-0') AND intDimMV2 NOT IN (6,72)) GROUP BY longDimSV1, intDimMV1 ORDER BY longDimSV1, intDimMV1 LIMIT 5
-SELECT longDimSV1, intDimMV1, count(longDimSV1), countMV(intDimMV1), avg(floatMetric1), avgMV(intDimMV2), minMaxRange(doubleMetric1), minMaxRangeMV(intDimMV2) FROM FeatureTest1 WHERE generationNumber = __GENERATION_NUMBER__ AND stringDimSV1 != 's1-6' AND longDimSV1 BETWEEN 10 AND 1000 OR (intDimMV1 < 42 AND stringDimMV2 IN ('m2-0-0', 'm2-2-0') AND intDimMV2 NOT IN (6,72)) GROUP BY longDimSV1, intDimMV1 ORDER BY longDimSV1, intDimMV1 LIMIT 5
-SELECT longDimSV1, intDimMV1, percentile(longDimSV1, 80), percentileMV(intDimMV1, 90), percentileEst(longDimSV1, 80), percentileEstMV(intDimMV1, 90), percentileTDigest(longDimSV1, 80), percentileTDigestMV(intDimMV1, 90) FROM FeatureTest1 WHERE generationNumber = __GENERATION_NUMBER__ AND stringDimSV1 != 's1-6' AND longDimSV1 BETWEEN 10 AND 1000 OR (intDimMV1 < 42 AND stringDimMV2 IN ('m2-0-0', 'm2-2-0') AND intDimMV2 NOT IN (6,72)) GROUP BY longDimSV1, intDimMV1 ORDER BY longDimSV1, intD [...]
-SELECT longDimSV1, intDimMV1, distinctCount(longDimSV1), distinctCountMV(intDimMV1), distinctCountHLL(longDimSV1), distinctCountHLLMV(intDimMV1) FROM FeatureTest1 WHERE generationNumber = __GENERATION_NUMBER__ AND stringDimSV1 != 's1-6' AND longDimSV1 BETWEEN 10 AND 1000 OR (intDimMV1 < 42 AND stringDimMV2 IN ('m2-0-0', 'm2-2-0') AND intDimMV2 NOT IN (6,72)) GROUP BY longDimSV1, intDimMV1 ORDER BY longDimSV1, intDimMV1 LIMIT 5
+SELECT longDimSV1, intDimMV1, count(*) FROM FeatureTest1 WHERE generationNumber = __GENERATION_NUMBER__ AND (stringDimSV1 != 's1-6' AND longDimSV1 BETWEEN 10 AND 1000 OR (intDimMV1 < 42 AND stringDimMV2 IN ('m2-0-0', 'm2-2-0') AND intDimMV2 NOT IN (6,72))) GROUP BY longDimSV1, intDimMV1 ORDER BY longDimSV1, intDimMV1 LIMIT 5
+SELECT longDimSV1, intDimMV1, sum(intMetric1), sumMV(intDimMV1), min(intMetric1), minMV(intDimMV2), max(longDimSV1), maxMV(intDimMV1) FROM FeatureTest1 WHERE generationNumber = __GENERATION_NUMBER__ AND (stringDimSV1 != 's1-6' AND longDimSV1 BETWEEN 10 AND 1000 OR (intDimMV1 < 42 AND stringDimMV2 IN ('m2-0-0', 'm2-2-0') AND intDimMV2 NOT IN (6,72))) GROUP BY longDimSV1, intDimMV1 ORDER BY longDimSV1, intDimMV1 LIMIT 5
+SELECT longDimSV1, intDimMV1, count(longDimSV1), countMV(intDimMV1), avg(floatMetric1), avgMV(intDimMV2), minMaxRange(doubleMetric1), minMaxRangeMV(intDimMV2) FROM FeatureTest1 WHERE generationNumber = __GENERATION_NUMBER__ AND (stringDimSV1 != 's1-6' AND longDimSV1 BETWEEN 10 AND 1000 OR (intDimMV1 < 42 AND stringDimMV2 IN ('m2-0-0', 'm2-2-0') AND intDimMV2 NOT IN (6,72))) GROUP BY longDimSV1, intDimMV1 ORDER BY longDimSV1, intDimMV1 LIMIT 5
+SELECT longDimSV1, intDimMV1, percentile(longDimSV1, 80), percentileMV(intDimMV1, 90), percentileEst(longDimSV1, 80), percentileEstMV(intDimMV1, 90), percentileTDigest(longDimSV1, 80), percentileTDigestMV(intDimMV1, 90) FROM FeatureTest1 WHERE generationNumber = __GENERATION_NUMBER__ AND (stringDimSV1 != 's1-6' AND longDimSV1 BETWEEN 10 AND 1000 OR (intDimMV1 < 42 AND stringDimMV2 IN ('m2-0-0', 'm2-2-0') AND intDimMV2 NOT IN (6,72))) GROUP BY longDimSV1, intDimMV1 ORDER BY longDimSV1, in [...]
+SELECT longDimSV1, intDimMV1, distinctCount(longDimSV1), distinctCountMV(intDimMV1), distinctCountHLL(longDimSV1), distinctCountHLLMV(intDimMV1) FROM FeatureTest1 WHERE generationNumber = __GENERATION_NUMBER__ AND (stringDimSV1 != 's1-6' AND longDimSV1 BETWEEN 10 AND 1000 OR (intDimMV1 < 42 AND stringDimMV2 IN ('m2-0-0', 'm2-2-0') AND intDimMV2 NOT IN (6,72))) GROUP BY longDimSV1, intDimMV1 ORDER BY longDimSV1, intDimMV1 LIMIT 5
 
 # Transformation Functions
 SELECT add(longDimSV1, sub(longDimSV2, 3)), mod(intMetric1, 10), div(doubleMetric1, mult(floatMetric1, 5)) FROM FeatureTest1 WHERE generationNumber = __GENERATION_NUMBER__ ORDER BY add(longDimSV1, sub(longDimSV2, 3)) DESC, mod(intMetric1, 10)
@@ -86,4 +86,3 @@ SELECT count(*) FROM FeatureTest1 WHERE generationNumber = __GENERATION_NUMBER__
 SELECT count(*) FROM FeatureTest1 WHERE generationNumber = __GENERATION_NUMBER__ AND regexp_like(stringDimMV2, 'm2.*0') GROUP BY stringDimMV2 LIMIT 3
 SELECT stringDimSV1, longDimSV1, intDimMV1 FROM FeatureTest1 WHERE generationNumber = __GENERATION_NUMBER__ ORDER BY stringDimSV1 DESC, longDimSV1 LIMIT 3
 SELECT stringDimSV1, longDimSV1, intDimMV1 FROM FeatureTest1 WHERE generationNumber = __GENERATION_NUMBER__ ORDER BY stringDimSV1 DESC, longDimSV1 DESC LIMIT 3
-SELECT * FROM FeatureTest1 WHERE generationNumber = __GENERATION_NUMBER__ ORDER BY longDimSV1 DESC, longMetric1 LIMIT 3
diff --git a/pinot-integration-tests/src/test/resources/compat-tests/query-op.yaml b/pinot-integration-tests/src/test/resources/compat-tests/query-op.yaml
index 03b6acf..c244f17 100644
--- a/pinot-integration-tests/src/test/resources/compat-tests/query-op.yaml
+++ b/pinot-integration-tests/src/test/resources/compat-tests/query-op.yaml
@@ -17,33 +17,9 @@
 # under the License.
 #
 
-# Operations to be done.
-description: Debug yaml file for query ops
+description: Yaml file for query ops
 operations:
-  - type: tableOp
-    description: Create table feature-test-1.json
-    op: CREATE
-    schemaFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/FeatureTest1-schema.json
-    tableConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1.json
-  - type: segmentOp
-    description: Build and upload segment using FeatureTest1-data-00.csv
-    op: UPLOAD
-    inputDataFileName: pinot-integration-tests/src/test/resources/compat-tests/data/FeatureTest1-data-00.csv
-    schemaFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/FeatureTest1-schema.json
-    tableConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1.json
-    recordReaderConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/data/recordReaderConfig.json
-    segmentName: FeatureTest1_Segment
   - type: queryOp
     description: Run query on FeatureTest1 using SQL
     queryFileName: pinot-integration-tests/src/test/resources/compat-tests/queries/feature-test-1-sql.queries
-    expectedResultsFileName: pinot-integration-tests/src/test/resources/compat-tests/query-results/feature-test-1-rest-sql.results
-  - type: segmentOp
-    description: Delete segment FeatureTest1_Segment
-    op: DELETE
-    tableConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1.json
-    segmentName: FeatureTest1_Segment
-  - type: tableOp
-    description: Delete table feature-test-1.json
-    op: DELETE
-    schemaFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/FeatureTest1-schema.json
-    tableConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1.json
\ No newline at end of file
+    expectedResultsFileName: pinot-integration-tests/src/test/resources/compat-tests/query-results/feature-test-1-rest-sql.results
\ No newline at end of file
diff --git a/pinot-integration-tests/src/test/resources/compat-tests/query-results/feature-test-1-rest-sql.results b/pinot-integration-tests/src/test/resources/compat-tests/query-results/feature-test-1-rest-sql.results
index ecc446f..48e4ea0 100644
--- a/pinot-integration-tests/src/test/resources/compat-tests/query-results/feature-test-1-rest-sql.results
+++ b/pinot-integration-tests/src/test/resources/compat-tests/query-results/feature-test-1-rest-sql.results
@@ -82,4 +82,3 @@
 {"resultTable":{"dataSchema":{"columnDataTypes":["LONG"],"columnNames":["count(*)"]},"rows":[[6],[2],[2]]},"exceptions":[],"numServersQueried":1,"numServersResponded":1,"numSegmentsQueried":1,"numSegmentsProcessed":1,"numSegmentsMatched":1,"numConsumingSegmentsQueried":0,"numDocsScanned":10,"numEntriesScannedInFilter":16,"numEntriesScannedPostFilter":10,"numGroupsLimitReached":false,"totalDocs":10,"timeUsedMs":4,"segmentStatistics":[],"traceInfo":{},"minConsumingFreshnessTimeMs":0}
 {"resultTable":{"dataSchema":{"columnDataTypes":["STRING","LONG","INT_ARRAY"],"columnNames":["stringDimSV1","longDimSV1","intDimMV1"]},"rows":[["s1-7",6766,[392,462]],["s1-6",7611,[392,462]],["s1-6",7611,[392,462]]]},"exceptions":[],"numServersQueried":1,"numServersResponded":1,"numSegmentsQueried":1,"numSegmentsProcessed":1,"numSegmentsMatched":1,"numConsumingSegmentsQueried":0,"numDocsScanned":10,"numEntriesScannedInFilter":0,"numEntriesScannedPostFilter":33,"numGroupsLimitReached":fal [...]
 {"resultTable":{"dataSchema":{"columnDataTypes":["STRING","LONG","INT_ARRAY"],"columnNames":["stringDimSV1","longDimSV1","intDimMV1"]},"rows":[["s1-7",6766,[392,462]],["s1-6",7611,[392,462]],["s1-6",7611,[392,462]]]},"exceptions":[],"numServersQueried":1,"numServersResponded":1,"numSegmentsQueried":1,"numSegmentsProcessed":1,"numSegmentsMatched":1,"numConsumingSegmentsQueried":0,"numDocsScanned":10,"numEntriesScannedInFilter":0,"numEntriesScannedPostFilter":33,"numGroupsLimitReached":fal [...]
-{"resultTable":{"dataSchema":{"columnDataTypes":["INT","BYTES","DOUBLE","FLOAT","INT","INT_ARRAY","INT_ARRAY","INT","LONG","LONG","LONG","STRING_ARRAY","INT_ARRAY","STRING","STRING_ARRAY","STRING_ARRAY","STRING","STRING","STRING"],"columnNames":["HoursSinceEpoch","bytesDimSV1","doubleMetric1","floatMetric1","generationNumber","intDimMV1","intDimMV2","intMetric1","longDimSV1","longDimSV2","longMetric1","mapDim1__KEYS","mapDim1__VALUES","mapDim2json","stringDimMV1","stringDimMV2","stringDi [...]
diff --git a/pinot-integration-tests/src/test/resources/compat-tests/stream-op-publish.yaml b/pinot-integration-tests/src/test/resources/compat-tests/realtime-table-create-op.yaml
similarity index 53%
copy from pinot-integration-tests/src/test/resources/compat-tests/stream-op-publish.yaml
copy to pinot-integration-tests/src/test/resources/compat-tests/realtime-table-create-op.yaml
index 14119a6..c71c3b3 100644
--- a/pinot-integration-tests/src/test/resources/compat-tests/stream-op-publish.yaml
+++ b/pinot-integration-tests/src/test/resources/compat-tests/realtime-table-create-op.yaml
@@ -17,23 +17,10 @@
 # under the License.
 #
 
-# Operations to be done.
-description: tst file for stream op PRODUCE
+description: Yaml file for creating a realtime table
 operations:
-  - type: streamOp
-    description: create Kafka topic
-    op: CREATE
-    streamConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1-realtime-stream-config.json
   - type: tableOp
     description: Create table feature-test-1-realtime.json
     op: CREATE
     schemaFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/FeatureTest1-schema.json
-    tableConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1-realtime.json
-  - type: streamOp
-    description: publish rows to PinotRealtimeFeatureTest1Event
-    op: PRODUCE
-    streamConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1-realtime-stream-config.json
-    numRows: 1
-    inputDataFileName: pinot-integration-tests/src/test/resources/compat-tests/data/FeatureTest1-data-00.csv
-    recordReaderConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/data/recordReaderConfig.json
-    tableConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1-realtime.json
+    tableConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1-realtime.json
\ No newline at end of file
diff --git a/pinot-integration-tests/src/test/resources/compat-tests/stream-op-publish.yaml b/pinot-integration-tests/src/test/resources/compat-tests/realtime-table-delete-op.yaml
similarity index 50%
copy from pinot-integration-tests/src/test/resources/compat-tests/stream-op-publish.yaml
copy to pinot-integration-tests/src/test/resources/compat-tests/realtime-table-delete-op.yaml
index 14119a6..8daf65d 100644
--- a/pinot-integration-tests/src/test/resources/compat-tests/stream-op-publish.yaml
+++ b/pinot-integration-tests/src/test/resources/compat-tests/realtime-table-delete-op.yaml
@@ -17,23 +17,10 @@
 # under the License.
 #
 
-# Operations to be done.
-description: tst file for stream op PRODUCE
+description: Yaml file for deleting a realtime table
 operations:
-  - type: streamOp
-    description: create Kafka topic
-    op: CREATE
-    streamConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1-realtime-stream-config.json
   - type: tableOp
-    description: Create table feature-test-1-realtime.json
-    op: CREATE
+    description: Delete table feature-test-1-realtime.json
+    op: DELETE
     schemaFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/FeatureTest1-schema.json
-    tableConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1-realtime.json
-  - type: streamOp
-    description: publish rows to PinotRealtimeFeatureTest1Event
-    op: PRODUCE
-    streamConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1-realtime-stream-config.json
-    numRows: 1
-    inputDataFileName: pinot-integration-tests/src/test/resources/compat-tests/data/FeatureTest1-data-00.csv
-    recordReaderConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/data/recordReaderConfig.json
-    tableConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1-realtime.json
+    tableConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1-realtime.json
\ No newline at end of file
diff --git a/pinot-integration-tests/src/test/resources/compat-tests/stream-op-publish.yaml b/pinot-integration-tests/src/test/resources/compat-tests/segment-create-op.yaml
similarity index 59%
rename from pinot-integration-tests/src/test/resources/compat-tests/stream-op-publish.yaml
rename to pinot-integration-tests/src/test/resources/compat-tests/segment-create-op.yaml
index 14119a6..f4843b9 100644
--- a/pinot-integration-tests/src/test/resources/compat-tests/stream-op-publish.yaml
+++ b/pinot-integration-tests/src/test/resources/compat-tests/segment-create-op.yaml
@@ -18,22 +18,12 @@
 #
 
 # Operations to be done.
-description: tst file for stream op PRODUCE
+description: Yaml file for uploading a segment
 operations:
-  - type: streamOp
-    description: create Kafka topic
-    op: CREATE
-    streamConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1-realtime-stream-config.json
-  - type: tableOp
-    description: Create table feature-test-1-realtime.json
-    op: CREATE
-    schemaFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/FeatureTest1-schema.json
-    tableConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1-realtime.json
-  - type: streamOp
-    description: publish rows to PinotRealtimeFeatureTest1Event
-    op: PRODUCE
-    streamConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1-realtime-stream-config.json
-    numRows: 1
+  - type: segmentOp
+    description: Build and upload segment using FeatureTest1-data-00.csv
+    op: UPLOAD
     inputDataFileName: pinot-integration-tests/src/test/resources/compat-tests/data/FeatureTest1-data-00.csv
-    recordReaderConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/data/recordReaderConfig.json
-    tableConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1-realtime.json
+    schemaFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/FeatureTest1-schema.json
+    tableConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1.json
+    recordReaderConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/data/recordReaderConfig.json
\ No newline at end of file
diff --git a/pinot-integration-tests/src/test/resources/compat-tests/segment-delete-op.yaml b/pinot-integration-tests/src/test/resources/compat-tests/segment-delete-op.yaml
new file mode 100644
index 0000000..f734d72
--- /dev/null
+++ b/pinot-integration-tests/src/test/resources/compat-tests/segment-delete-op.yaml
@@ -0,0 +1,26 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+description: Yaml file for deleting a segment
+operations:
+  - type: segmentOp
+    description: Delete segment FeatureTest1_Segment
+    op: DELETE
+    tableConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1.json
+    segmentName: FeatureTest1_Segment
\ No newline at end of file
diff --git a/pinot-integration-tests/src/test/resources/compat-tests/table-segment-op.yaml b/pinot-integration-tests/src/test/resources/compat-tests/table-segment-op.yaml
deleted file mode 100644
index 58f7eae..0000000
--- a/pinot-integration-tests/src/test/resources/compat-tests/table-segment-op.yaml
+++ /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.
-#
-
-# Operations to be done.
-description: Debug yaml file for table and segment ops
-operations:
-    - type: tableOp
-      description: Create table feature-test-1.json
-      op: CREATE
-      schemaFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/FeatureTest1-schema.json
-      tableConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1.json
-    - type: segmentOp
-      description: Build and upload segment using FeatureTest1-data-00.csv
-      op: UPLOAD
-      inputDataFileName: pinot-integration-tests/src/test/resources/compat-tests/data/FeatureTest1-data-00.csv
-      schemaFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/FeatureTest1-schema.json
-      tableConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1.json
-      recordReaderConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/data/recordReaderConfig.json
-      segmentName: FeatureTest1_Segment
-    - type: segmentOp
-      description: Delete segment FeatureTest1_Segment
-      op: DELETE
-      tableConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1.json
-      segmentName: FeatureTest1_Segment
-    - type: tableOp
-      description: Delete table feature-test-1.json
-      op: DELETE
-      schemaFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/FeatureTest1-schema.json
-      tableConfigFileName: pinot-integration-tests/src/test/resources/compat-tests/configs/feature-test-1.json


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