You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ak...@apache.org on 2022/05/27 06:16:57 UTC

[carbondata] branch master updated: [CARBONDATA-4336] Table Status Versioning

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 57e76eea58 [CARBONDATA-4336] Table Status Versioning
57e76eea58 is described below

commit 57e76eea58b05bed2da3e6e6a073e093a332b2cc
Author: Indhumathi27 <in...@gmail.com>
AuthorDate: Fri Apr 22 18:59:03 2022 +0530

    [CARBONDATA-4336] Table Status Versioning
    
    Why is this PR needed?
    Currently, carbondata will store the records of a transaction (load/insert/IUD/Add/drop segment)
    in a metadata file named `tablestatus’ which will be present in the Metadata directory.
    If the tablestatus file is lost, then the metadata for the transactions cannot be recovered
    directly, as there is no previous version file available for tablestatus. Hence, if we support
    versioning for tablestatus files, then it will be easy to recover the current version tablestatus
    meta from previous version tablestatus files.
    
    Please refer Table Status Versioning & Recovery Tool for more info.
    
    What changes were proposed in this PR?
    -> On each transaction commit, committed the latest load metadata details to a new version file
    -> Updated the latest tablestatus version timestamp in the table properties [CarbonTable cache] and in the hive metastore
    -> Added a table status version tool which can recover the latest transaction details based on old version file
    
    Does this PR introduce any user interface change?
    Yes
    
    Is any new testcase added?
    Yes
    
    This closes #4261
---
 .../core/constants/CarbonCommonConstants.java      |  11 +
 .../carbondata/core/index/IndexInputFormat.java    |   2 +-
 .../carbondata/core/index/IndexStoreManager.java   |   3 +-
 .../apache/carbondata/core/index/IndexUtil.java    |  23 +-
 .../org/apache/carbondata/core/index/Segment.java  |   6 +-
 .../carbondata/core/metadata/SegmentFileStore.java |  47 +--
 .../core/metadata/schema/table/CarbonTable.java    |   4 +
 .../carbondata/core/mutate/CarbonUpdateUtil.java   |  36 +-
 .../TableStatusReadCommittedScope.java             |  13 +-
 .../core/statusmanager/SegmentStatusManager.java   | 192 +++++++----
 .../statusmanager/SegmentUpdateStatusManager.java  |   3 +-
 .../carbondata/core/util/CarbonProperties.java     |   6 +
 .../apache/carbondata/core/util/CarbonUtil.java    |  15 +-
 .../carbondata/core/util/CleanFilesUtil.java       |   2 +-
 .../carbondata/core/util/DeleteLoadFolders.java    |  15 +-
 .../carbondata/core/util/path/CarbonTablePath.java |  11 +
 .../org/apache/carbondata/core/view/MVManager.java |  15 +-
 .../apache/carbondata/core/view/MVProvider.java    |  17 +-
 .../org/apache/carbondata/core/view/MVSchema.java  |  41 ---
 .../core/writer/CarbonIndexFileMergeWriter.java    |  26 +-
 .../blockletindex/TestBlockletIndexFactory.java    |   5 +-
 docs/configuration-parameters.md                   |  55 +--
 docs/faq.md                                        |  17 +
 .../hadoop/api/CarbonOutputCommitter.java          |   4 +-
 .../hadoop/api/CarbonTableInputFormat.java         |  21 +-
 .../index/bloom/BloomCoarseGrainIndexFactory.java  |   3 +-
 .../index/lucene/LuceneIndexFactoryBase.java       |   3 +-
 .../TestCreateIndexWithLoadAndCompaction.scala     |   8 +-
 .../secondaryindex/TestSIWithSecondaryIndex.scala  |   9 +-
 .../apache/spark/util/TestCarbonSegmentUtil.scala  |   7 +-
 .../PrestoInsertIntoTableTestCase.scala            |   3 +-
 .../org/apache/spark/sql/SparkVersionAdapter.scala |   5 +
 .../load/CarbonInternalLoaderUtil.java             |  33 +-
 .../org/apache/carbondata/api/CarbonStore.scala    |  48 ++-
 .../apache/carbondata/index/IndexRebuildRDD.scala  |   7 +-
 .../carbondata/index/secondary/SecondaryIndex.java |   4 +-
 .../index/secondary/SecondaryIndexFactory.java     |  12 +-
 .../indexserver/DistributedRDDUtils.scala          |   2 +-
 .../recovery/tablestatus/TableStatusRecovery.scala | 233 +++++++++++++
 .../spark/rdd/CarbonDataRDDFactory.scala           |   3 +-
 .../carbondata/spark/rdd/CarbonMergerRDD.scala     |   3 +-
 .../spark/rdd/CarbonTableCompactor.scala           |   9 +-
 .../carbondata/spark/rdd/StreamHandoffRDD.scala    |  20 +-
 .../carbondata/spark/util/CarbonScalaUtil.scala    |  45 +++
 .../carbondata/streaming/StreamSinkFactory.scala   |  15 +-
 .../apache/carbondata/trash/DataTrashManager.scala |  61 +++-
 .../org/apache/carbondata/view/MVRefresher.scala   |  56 ++-
 .../org/apache/spark/rdd/CarbonMergeFilesRDD.scala |   2 +-
 .../scala/org/apache/spark/sql/CarbonSource.scala  |   3 +
 .../spark/sql/events/MergeIndexEventListener.scala |   4 +-
 .../sql/execution/command/cache/CacheUtil.scala    |   4 +-
 .../command/management/CarbonAddLoadCommand.scala  |  12 +-
 .../CarbonAlterTableCompactionCommand.scala        |   9 +-
 .../management/CarbonCleanFilesCommand.scala       |   2 +-
 .../management/CarbonDeleteLoadByIdCommand.scala   |   3 +-
 .../CarbonDeleteLoadByLoadDateCommand.scala        |   3 +-
 .../management/CarbonInsertFromStageCommand.scala  |  10 +-
 .../management/CarbonInsertIntoCommand.scala       |  12 +
 .../management/CarbonInsertIntoWithDf.scala        |   4 +
 .../command/management/CarbonLoadDataCommand.scala |   4 +
 .../CarbonShowSegmentsAsSelectCommand.scala        |   2 +-
 .../management/CarbonShowSegmentsCommand.scala     |   2 +-
 .../command/management/CommonLoadUtils.scala       |  17 +-
 .../management/RefreshCarbonTableCommand.scala     |  17 +-
 .../mutation/CarbonProjectForDeleteCommand.scala   |  27 +-
 .../mutation/CarbonProjectForUpdateCommand.scala   |  26 +-
 .../command/mutation/DeleteExecution.scala         |  55 +--
 .../mutation/merge/CarbonMergeDataSetUtil.scala    |   3 +-
 .../command/mutation/merge/MergeUtil.scala         |   8 +-
 .../CarbonAlterTableAddHivePartitionCommand.scala  |   5 +-
 .../CarbonAlterTableDropHivePartitionCommand.scala |  12 +-
 .../table/CarbonCreateTableLikeCommand.scala       |   3 +
 .../datasources/SparkCarbonTableFormat.scala       |   4 +-
 .../execution/strategy/CarbonSourceStrategy.scala  |   4 +-
 .../execution/strategy/MixedFormatHandler.scala    |   4 +-
 .../streaming/CarbonAppendableStreamSink.scala     |   2 +-
 .../spark/sql/hive/CarbonFileMetastore.scala       |   5 +
 .../sql/hive/CarbonHiveIndexMetadataUtil.scala     |  31 ++
 .../org/apache/spark/sql/hive/CarbonRelation.scala |   6 +-
 .../apache/spark/sql/hive/CarbonSessionUtil.scala  |   4 +-
 .../apache/spark/sql/index/CarbonIndexUtil.scala   |  10 +-
 .../sql/parser/CarbonSparkSqlParserUtil.scala      |   6 +
 .../secondaryindex/command/SICreationCommand.scala |   3 +-
 .../sql/secondaryindex/command/SILoadCommand.scala |   6 +-
 .../command/SIRebuildSegmentRunner.scala           |   8 +-
 .../events/CleanFilesPostEventListener.scala       |  24 +-
 .../events/CreateCarbonRelationEventListener.scala |   1 +
 .../events/DeleteSegmentByDateListener.scala       |   4 +-
 .../events/DeleteSegmentByIdListener.scala         |   9 +-
 .../hive/CarbonInternalMetastore.scala             |  14 +
 .../joins/BroadCastSIFilterPushJoin.scala          |  10 +-
 .../spark/sql/secondaryindex/load/Compactor.scala  |   4 +-
 .../secondaryindex/rdd/SecondaryIndexCreator.scala |  28 +-
 .../sql/secondaryindex/util/FileInternalUtil.scala |  10 +-
 .../secondaryindex/util/SecondaryIndexUtil.scala   |  24 +-
 .../org/apache/spark/sql/test/util/QueryTest.scala |   5 +-
 .../scala/org/apache/spark/util/CleanFiles.scala   |   1 +
 .../apache/spark/util/DeleteSegmentByDate.scala    |   4 +-
 .../org/apache/spark/util/DeleteSegmentById.scala  |   3 +-
 .../apache/spark/util/PartitionCacheManager.scala  |   7 +-
 .../org/apache/spark/sql/SparkVersionAdapter.scala |   6 +
 .../testsuite/binary/TestBinaryDataType.scala      |   1 +
 .../testsuite/dataload/TestLoadDataGeneral.scala   |   7 +-
 .../TestLoadDataWithStaleDataInSegmentFolder.scala |   3 +-
 .../recovery/TableStatusRecoveryTest.scala         | 380 +++++++++++++++++++++
 .../allqueries/InsertIntoCarbonTableTestCase.scala |  14 +-
 ...ryWithColumnMetCacheAndCacheLevelProperty.scala |   6 +-
 .../cleanfiles/TestCleanFileCommand.scala          |  19 +-
 .../TestCleanFilesCommandPartitionTable.scala      |  10 +-
 .../createTable/TestCreateExternalTable.scala      |   5 +
 .../createTable/TestCreateTableLike.scala          |   6 +-
 .../CarbonIndexFileMergeTestCase.scala             |   8 +-
 .../CompactionSupportGlobalSortFunctionTest.scala  |   4 +-
 .../CompactionSupportGlobalSortParameterTest.scala |   4 +-
 .../DataCompactionCardinalityBoundryTest.scala     |  11 +-
 .../datacompaction/DataCompactionLockTest.scala    |  10 +-
 .../MajorCompactionIgnoreInMinorTest.scala         |  23 +-
 .../MajorCompactionStopsAfterCompaction.scala      |   4 +-
 .../dataload/TestDataLoadWithFileName.scala        |   4 +-
 .../dataload/TestGlobalSortDataLoad.scala          |   8 +-
 .../dataload/TestRangeColumnDataLoad.scala         |   4 +-
 .../dataretention/DataRetentionTestCase.scala      |   5 +-
 .../testsuite/iud/DeleteCarbonTableTestCase.scala  |  11 +-
 .../testsuite/iud/UpdateCarbonTableTestCase.scala  |   3 +-
 .../longstring/VarcharDataTypesBasicTestCase.scala |   2 +-
 .../testsuite/segment/ShowSegmentTestCase.scala    |   9 +-
 .../StandardPartitionTableCleanTestCase.scala      |   2 +-
 .../StandardPartitionTableLoadingTestCase.scala    |  14 +-
 .../StandardPartitionTableQueryTestCase.scala      |   1 +
 .../rewrite/MVIncrementalLoadingTestcase.scala     |  48 ++-
 .../view/rewrite/TestPartitionWithMV.scala         |   4 +-
 .../spark/carbondata/TableStatusBackupTest.scala   |  10 +-
 .../restructure/AlterTableUpgradeSegmentTest.scala |   8 +-
 .../org/apache/spark/util/CarbonCommandSuite.scala |  14 +-
 .../processing/loading/model/CarbonLoadModel.java  |  25 +-
 .../processing/merger/CarbonDataMergerUtil.java    |  24 +-
 .../processing/util/CarbonLoaderUtil.java          |  28 +-
 .../streaming/segment/StreamSegment.java           |  38 +--
 138 files changed, 1867 insertions(+), 585 deletions(-)

diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
index fc53960a2d..fb45bccba6 100644
--- a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
+++ b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
@@ -2876,4 +2876,15 @@ public final class CarbonCommonConstants {
 
   public static final String CARBON_ENABLE_SCHEMA_ENFORCEMENT_DEFAULT = "true";
 
+  /**
+   * This property will enable creating multi-versioned table status files. The information about
+   * the latest version will be stored in table properties
+   */
+  @CarbonProperty
+  public static final String CARBON_ENABLE_MULTI_VERSION_TABLE_STATUS =
+      "carbon.enable.multi.version.table.status";
+
+  public static final String CARBON_ENABLE_MULTI_VERSION_TABLE_STATUS_DEFAULT = "false";
+
+
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/index/IndexInputFormat.java b/core/src/main/java/org/apache/carbondata/core/index/IndexInputFormat.java
index 4764445839..8763493321 100644
--- a/core/src/main/java/org/apache/carbondata/core/index/IndexInputFormat.java
+++ b/core/src/main/java/org/apache/carbondata/core/index/IndexInputFormat.java
@@ -349,7 +349,7 @@ public class IndexInputFormat extends FileInputFormat<Void, ExtendedBlocklet>
       if (table.isTransactionalTable()) {
         this.readCommittedScope =
             new TableStatusReadCommittedScope(table.getAbsoluteTableIdentifier(),
-                FileFactory.getConfiguration());
+                FileFactory.getConfiguration(), table.getTableStatusVersion());
       } else {
         this.readCommittedScope =
             new LatestFilesReadCommittedScope(table.getTablePath(),
diff --git a/core/src/main/java/org/apache/carbondata/core/index/IndexStoreManager.java b/core/src/main/java/org/apache/carbondata/core/index/IndexStoreManager.java
index 760a3d07b5..d79fedd35e 100644
--- a/core/src/main/java/org/apache/carbondata/core/index/IndexStoreManager.java
+++ b/core/src/main/java/org/apache/carbondata/core/index/IndexStoreManager.java
@@ -493,7 +493,8 @@ public final class IndexStoreManager {
 
     TableSegmentRefresher(CarbonTable table) {
       SegmentStatusManager segmentStatusManager =
-          new SegmentStatusManager(table.getAbsoluteTableIdentifier());
+          new SegmentStatusManager(table.getAbsoluteTableIdentifier(),
+              table.getTableStatusVersion());
       List<Segment> validSegments;
       try {
         validSegments = segmentStatusManager.getValidAndInvalidSegments().getValidSegments();
diff --git a/core/src/main/java/org/apache/carbondata/core/index/IndexUtil.java b/core/src/main/java/org/apache/carbondata/core/index/IndexUtil.java
index 69257f505e..8f64df775c 100644
--- a/core/src/main/java/org/apache/carbondata/core/index/IndexUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/index/IndexUtil.java
@@ -34,9 +34,7 @@ import org.apache.carbondata.core.index.dev.expr.IndexExprWrapper;
 import org.apache.carbondata.core.index.dev.expr.IndexInputSplitWrapper;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.metadata.schema.table.RelationIdentifier;
 import org.apache.carbondata.core.readcommitter.LatestFilesReadCommittedScope;
 import org.apache.carbondata.core.readcommitter.ReadCommittedScope;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
@@ -337,26 +335,9 @@ public class IndexUtil {
   public static SegmentStatusManager.ValidAndInvalidSegmentsInfo getValidAndInvalidSegments(
       CarbonTable carbonTable, Configuration configuration) throws IOException {
     SegmentStatusManager ssm =
-        new SegmentStatusManager(carbonTable.getAbsoluteTableIdentifier(), configuration);
+        new SegmentStatusManager(carbonTable.getAbsoluteTableIdentifier(), configuration,
+            carbonTable.getTableStatusVersion());
     return ssm.getValidAndInvalidSegments(carbonTable.isMV());
   }
 
-  /**
-   * Returns valid segment list for a given RelationIdentifier
-   *
-   * @param relationIdentifier get list of segments for relation identifier
-   * @return list of valid segment id's
-   * @throws IOException
-   */
-  public static List<String> getMainTableValidSegmentList(RelationIdentifier relationIdentifier)
-      throws IOException {
-    List<String> segmentList = new ArrayList<>();
-    List<Segment> validSegments = new SegmentStatusManager(AbsoluteTableIdentifier
-        .from(relationIdentifier.getTablePath(), relationIdentifier.getDatabaseName(),
-            relationIdentifier.getTableName())).getValidAndInvalidSegments().getValidSegments();
-    for (Segment segment : validSegments) {
-      segmentList.add(segment.getSegmentNo());
-    }
-    return segmentList;
-  }
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/index/Segment.java b/core/src/main/java/org/apache/carbondata/core/index/Segment.java
index 4bbe543582..d0c979a951 100644
--- a/core/src/main/java/org/apache/carbondata/core/index/Segment.java
+++ b/core/src/main/java/org/apache/carbondata/core/index/Segment.java
@@ -265,9 +265,11 @@ public class Segment implements Serializable, Writable {
    * @param tablePath
    * @return
    */
-  public static Segment getSegment(String segmentNo, String tablePath) {
+  public static Segment getSegment(String segmentNo, String tablePath,
+      String tblStatusReadVersion) {
     LoadMetadataDetails[] loadMetadataDetails =
-        SegmentStatusManager.readLoadMetadata(CarbonTablePath.getMetadataPath(tablePath));
+        SegmentStatusManager.readLoadMetadata(CarbonTablePath.getMetadataPath(tablePath),
+            tblStatusReadVersion);
     return getSegment(segmentNo, loadMetadataDetails);
   }
 
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java b/core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
index eda819b71f..1759eb41b3 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
@@ -616,9 +616,10 @@ public class SegmentFileStore {
    * @throws IOException
    */
   public static boolean updateTableStatusFile(CarbonTable carbonTable, String segmentId,
-      String segmentFile, String tableId, SegmentFileStore segmentFileStore) throws IOException {
+      String segmentFile, String tableId, SegmentFileStore segmentFileStore,
+      String tblStatusReadVersion) throws IOException {
     return updateTableStatusFile(carbonTable, segmentId, segmentFile, tableId, segmentFileStore,
-        null);
+        null, tblStatusReadVersion);
   }
 
   /**
@@ -629,17 +630,19 @@ public class SegmentFileStore {
    */
   public static boolean updateTableStatusFile(CarbonTable carbonTable, String segmentId,
       String segmentFile, String tableId, SegmentFileStore segmentFileStore,
-      SegmentStatus segmentStatus) throws IOException {
+      SegmentStatus segmentStatus, String tblStatusReadVersion) throws IOException {
     boolean status = false;
     String tablePath = carbonTable.getTablePath();
-    String tableStatusPath = CarbonTablePath.getTableStatusFilePath(tablePath);
+    String tableStatusPath =
+        CarbonTablePath.getTableStatusFilePath(tablePath, tblStatusReadVersion);
     if (!FileFactory.isFileExist(tableStatusPath)) {
       return status;
     }
     String metadataPath = CarbonTablePath.getMetadataPath(tablePath);
     AbsoluteTableIdentifier absoluteTableIdentifier =
         AbsoluteTableIdentifier.from(tablePath, null, null, tableId);
-    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier);
+    SegmentStatusManager segmentStatusManager =
+        new SegmentStatusManager(absoluteTableIdentifier, carbonTable.getTableStatusVersion());
     ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
     int retryCount = CarbonLockUtil
         .getLockProperty(CarbonCommonConstants.NUMBER_OF_TRIES_FOR_CONCURRENT_LOCK,
@@ -651,7 +654,7 @@ public class SegmentFileStore {
       if (carbonLock.lockWithRetries(retryCount, maxTimeout)) {
         LOGGER.info("Acquired lock for table path" + tablePath + " for table status update");
         LoadMetadataDetails[] listOfLoadFolderDetailsArray =
-            SegmentStatusManager.readLoadMetadata(metadataPath);
+            SegmentStatusManager.readLoadMetadata(metadataPath, tblStatusReadVersion);
 
         for (LoadMetadataDetails detail : listOfLoadFolderDetailsArray) {
           // if the segments is in the list of marked for delete then update the status.
@@ -1059,18 +1062,22 @@ public class SegmentFileStore {
    * @param toBeDeleteSegments
    * @throws IOException
    */
-  public static void commitDropPartitions(CarbonTable carbonTable, String uniqueId,
-      List<String> toBeUpdatedSegments, List<String> toBeDeleteSegments,
-      String uuid) throws IOException {
+  public static String commitDropPartitions(CarbonTable carbonTable, String uniqueId,
+      List<String> toBeUpdatedSegments, List<String> toBeDeleteSegments, String uuid)
+      throws IOException {
+    String tblStatusWriteVersion = "";
     if (toBeDeleteSegments.size() > 0 || toBeUpdatedSegments.size() > 0) {
       Set<Segment> segmentSet = new HashSet<>(
-          new SegmentStatusManager(carbonTable.getAbsoluteTableIdentifier())
-              .getValidAndInvalidSegments(carbonTable.isMV()).getValidSegments());
-      CarbonUpdateUtil.updateTableMetadataStatus(segmentSet, carbonTable, uniqueId,
-          true, false,
-          Segment.toSegmentList(toBeDeleteSegments, null),
-          Segment.toSegmentList(toBeUpdatedSegments, null), uuid);
-    }
+          new SegmentStatusManager(carbonTable.getAbsoluteTableIdentifier(),
+              carbonTable.getTableStatusVersion()).getValidAndInvalidSegments(carbonTable.isMV())
+              .getValidSegments());
+      Map<String, String> returnTuple =
+          CarbonUpdateUtil.updateTableMetadataStatus(segmentSet, carbonTable, uniqueId, true, false,
+              Segment.toSegmentList(toBeDeleteSegments, null),
+              Segment.toSegmentList(toBeUpdatedSegments, null), uuid);
+      tblStatusWriteVersion = returnTuple.getOrDefault("tblStatusWriteVersion", "");
+    }
+    return tblStatusWriteVersion;
   }
 
   /**
@@ -1159,7 +1166,8 @@ public class SegmentFileStore {
    */
   public static void cleanSegments(CarbonTable table, List<PartitionSpec> partitionSpecs,
       boolean forceDelete) throws IOException {
-    LoadMetadataDetails[] details = SegmentStatusManager.readLoadMetadata(table.getMetadataPath());
+    LoadMetadataDetails[] details = SegmentStatusManager.readLoadMetadata(table.getMetadataPath(),
+        table.getTableStatusVersion());
     cleanSegments(table, details, partitionSpecs, forceDelete);
   }
 
@@ -1176,8 +1184,9 @@ public class SegmentFileStore {
       Set<String> segments,
       List<PartitionSpec> partitionSpecs,
       boolean forceDelete) throws IOException {
-    LoadMetadataDetails[] details = Arrays
-        .stream(SegmentStatusManager.readLoadMetadata(table.getMetadataPath()))
+    LoadMetadataDetails[] details = Arrays.stream(
+            SegmentStatusManager.readLoadMetadata(table.getMetadataPath(),
+                table.getTableStatusVersion()))
         .filter(detail -> segments.contains(detail.getLoadName()))
         .toArray(LoadMetadataDetails[]::new);
     cleanSegments(table, details, partitionSpecs, forceDelete);
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
index 1e7dfee983..991943e3ed 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
@@ -1285,4 +1285,8 @@ public class CarbonTable implements Serializable, Writable {
     return allIndexes;
   }
 
+  public String getTableStatusVersion() {
+    return this.tableInfo.getFactTable().getTableProperties().getOrDefault("latestversion", "");
+  }
+
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java b/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
index f0f23e7620..e5ed5fe6d5 100644
--- a/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
@@ -252,12 +252,13 @@ public class CarbonUpdateUtil {
    * @param segmentsToBeDeleted
    * @return
    */
-  public static boolean updateTableMetadataStatus(Set<Segment> updatedSegmentsList,
+  public static Map<String, String> updateTableMetadataStatus(Set<Segment> updatedSegmentsList,
       CarbonTable table, String updatedTimeStamp, boolean isTimestampUpdateRequired,
-      boolean isUpdateStatusFileUpdateRequired, List<Segment> segmentsToBeDeleted) {
+      boolean isUpdateStatusFileUpdateRequired, List<Segment> segmentsToBeDeleted,
+      String tblStatusWriteVersion) {
     return updateTableMetadataStatus(updatedSegmentsList, table, updatedTimeStamp,
         isTimestampUpdateRequired, isUpdateStatusFileUpdateRequired,
-        segmentsToBeDeleted, new ArrayList<Segment>(), "");
+        segmentsToBeDeleted, new ArrayList<Segment>(), tblStatusWriteVersion);
   }
 
   /**
@@ -269,17 +270,19 @@ public class CarbonUpdateUtil {
    * @param segmentsToBeDeleted
    * @return
    */
-  public static boolean updateTableMetadataStatus(Set<Segment> updatedSegmentsList,
+  public static Map<String, String> updateTableMetadataStatus(Set<Segment> updatedSegmentsList,
       CarbonTable table, String updatedTimeStamp, boolean isTimestampUpdateRequired,
       boolean isUpdateStatusFileUpdateRequired, List<Segment> segmentsToBeDeleted,
       List<Segment> segmentFilesTobeUpdated, String uuid) {
 
     boolean status = false;
+    Map<String, String> tuple = new HashMap<>();
     String metaDataFilepath = table.getMetadataPath();
     AbsoluteTableIdentifier identifier = table.getAbsoluteTableIdentifier();
     String tableStatusPath =
-        CarbonTablePath.getTableStatusFilePathWithUUID(identifier.getTablePath(), uuid);
-    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(identifier);
+        CarbonTablePath.getTableStatusFilePath(identifier.getTablePath(), uuid);
+    SegmentStatusManager segmentStatusManager =
+        new SegmentStatusManager(identifier, table.getTableStatusVersion());
 
     ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
     boolean lockStatus = false;
@@ -289,8 +292,14 @@ public class CarbonUpdateUtil {
         LOGGER.info("Acquired lock for table" + table.getDatabaseName() + "." + table.getTableName()
              + " for table status update");
 
+        if (uuid.isEmpty() && CarbonProperties.isTableStatusMultiVersionEnabled()) {
+          String tblStatusWriteVersion = String.valueOf(System.currentTimeMillis());
+          tableStatusPath = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath(),
+              tblStatusWriteVersion);
+          tuple.put("tblStatusWriteVersion", tblStatusWriteVersion);
+        }
         LoadMetadataDetails[] listOfLoadFolderDetailsArray =
-            SegmentStatusManager.readLoadMetadata(metaDataFilepath);
+            SegmentStatusManager.readLoadMetadata(metaDataFilepath, table.getTableStatusVersion());
         // to update table status only when required.
         boolean isUpdateRequired = false;
 
@@ -340,7 +349,8 @@ public class CarbonUpdateUtil {
                 .writeLoadDetailsIntoFile(tableStatusPath, listOfLoadFolderDetailsArray);
           }
         } catch (IOException e) {
-          return false;
+          tuple.put("status", "false");
+          return tuple;
         }
 
         status = true;
@@ -361,7 +371,8 @@ public class CarbonUpdateUtil {
         }
       }
     }
-    return status;
+    tuple.put("status", String.valueOf(status));
+    return tuple;
 
   }
 
@@ -701,9 +712,10 @@ public class CarbonUpdateUtil {
    */
   public static long cleanUpDeltaFiles(CarbonTable table, boolean isDryRun) throws IOException {
 
-    SegmentStatusManager ssm = new SegmentStatusManager(table.getAbsoluteTableIdentifier());
-    LoadMetadataDetails[] details =
-            SegmentStatusManager.readLoadMetadata(table.getMetadataPath());
+    SegmentStatusManager ssm =
+        new SegmentStatusManager(table.getAbsoluteTableIdentifier(), table.getTableStatusVersion());
+    LoadMetadataDetails[] details = SegmentStatusManager.readLoadMetadata(table.getMetadataPath(),
+        table.getTableStatusVersion());
     long totalSizeDeleted = 0;
     ArrayList<CarbonFile> filesToBeDeleted = new ArrayList<>();
     SegmentUpdateStatusManager updateStatusManager = new SegmentUpdateStatusManager(table);
diff --git a/core/src/main/java/org/apache/carbondata/core/readcommitter/TableStatusReadCommittedScope.java b/core/src/main/java/org/apache/carbondata/core/readcommitter/TableStatusReadCommittedScope.java
index 68dceebaaa..bf0f6bedfe 100644
--- a/core/src/main/java/org/apache/carbondata/core/readcommitter/TableStatusReadCommittedScope.java
+++ b/core/src/main/java/org/apache/carbondata/core/readcommitter/TableStatusReadCommittedScope.java
@@ -50,20 +50,25 @@ public class TableStatusReadCommittedScope implements ReadCommittedScope {
 
   private AbsoluteTableIdentifier identifier;
 
+  private String tblStatusReadVersion = "";
+
   private transient Configuration configuration;
 
   public TableStatusReadCommittedScope(AbsoluteTableIdentifier identifier,
-      Configuration configuration) throws IOException {
+      Configuration configuration, String tblStatusReadVersion) throws IOException {
     this.identifier = identifier;
     this.configuration = configuration;
+    this.tblStatusReadVersion = tblStatusReadVersion;
     takeCarbonIndexFileSnapShot();
   }
 
   public TableStatusReadCommittedScope(AbsoluteTableIdentifier identifier,
-      LoadMetadataDetails[] loadMetadataDetails, Configuration configuration) {
+      LoadMetadataDetails[] loadMetadataDetails, Configuration configuration,
+      String tblStatusReadVersion) {
     this.identifier = identifier;
     this.configuration = configuration;
     this.loadMetadataDetails = loadMetadataDetails;
+    this.tblStatusReadVersion = tblStatusReadVersion;
   }
 
   @Override
@@ -135,8 +140,8 @@ public class TableStatusReadCommittedScope implements ReadCommittedScope {
   public void takeCarbonIndexFileSnapShot() throws IOException {
     // Only Segment Information is updated.
     // File information will be fetched on the fly according to the fetched segment info.
-    this.loadMetadataDetails = SegmentStatusManager
-        .readTableStatusFile(CarbonTablePath.getTableStatusFilePath(identifier.getTablePath()));
+    this.loadMetadataDetails = SegmentStatusManager.readTableStatusFile(
+        CarbonTablePath.getTableStatusFilePath(identifier.getTablePath(), tblStatusReadVersion));
   }
 
   @Override
diff --git a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
index cf628931a5..10086d410b 100755
--- a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
+++ b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
@@ -23,6 +23,7 @@ import java.io.Closeable;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.EOFException;
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.OutputStreamWriter;
@@ -75,6 +76,8 @@ public class SegmentStatusManager {
 
   private Configuration configuration;
 
+  private final String tblStatusVersion;
+
   private static final int READ_TABLE_STATUS_RETRY_COUNT = CarbonLockUtil
           .getLockProperty(CarbonCommonConstants.NUMBER_OF_TRIES_FOR_CARBON_LOCK,
                   CarbonCommonConstants.NUMBER_OF_TRIES_FOR_CARBON_LOCK_DEFAULT);
@@ -83,14 +86,17 @@ public class SegmentStatusManager {
           .getLockProperty(CarbonCommonConstants.MAX_TIMEOUT_FOR_CARBON_LOCK,
                   CarbonCommonConstants.MAX_TIMEOUT_FOR_CARBON_LOCK_DEFAULT);
 
-  public SegmentStatusManager(AbsoluteTableIdentifier identifier) {
+  public SegmentStatusManager(AbsoluteTableIdentifier identifier, String tblStatusVersion) {
     this.identifier = identifier;
     configuration = FileFactory.getConfiguration();
+    this.tblStatusVersion = tblStatusVersion;
   }
 
-  public SegmentStatusManager(AbsoluteTableIdentifier identifier, Configuration configuration) {
+  public SegmentStatusManager(AbsoluteTableIdentifier identifier, Configuration configuration,
+      String tblStatusVersion) {
     this.identifier = identifier;
     this.configuration = configuration;
+    this.tblStatusVersion = tblStatusVersion;
   }
 
   /**
@@ -105,14 +111,14 @@ public class SegmentStatusManager {
   /**
    * This method will return last modified time of tablestatus file
    */
-  public static long getTableStatusLastModifiedTime(AbsoluteTableIdentifier identifier)
-      throws IOException {
-    String tableStatusPath = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath());
+  public static long getTableStatusLastModifiedTime(AbsoluteTableIdentifier identifier,
+      String tblStatusVersion) throws IOException {
+    String tableStatusPath =
+        CarbonTablePath.getTableStatusFilePath(identifier.getTablePath(), tblStatusVersion);
     if (!FileFactory.isFileExist(tableStatusPath)) {
       return 0L;
     } else {
-      return FileFactory.getCarbonFile(tableStatusPath)
-          .getLastModifiedTime();
+      return FileFactory.getCarbonFile(tableStatusPath).getLastModifiedTime();
     }
   }
 
@@ -143,12 +149,12 @@ public class SegmentStatusManager {
     try {
       if (loadMetadataDetails == null) {
         loadMetadataDetails = readTableStatusFile(
-            CarbonTablePath.getTableStatusFilePath(identifier.getTablePath()));
+            CarbonTablePath.getTableStatusFilePath(identifier.getTablePath(), tblStatusVersion));
       }
 
       if (readCommittedScope == null) {
         readCommittedScope = new TableStatusReadCommittedScope(identifier, loadMetadataDetails,
-            configuration);
+            configuration, tblStatusVersion);
       }
       //just directly iterate Array
       for (LoadMetadataDetails segment : loadMetadataDetails) {
@@ -264,26 +270,30 @@ public class SegmentStatusManager {
    * @return validAndInvalidSegmentsInfo
    */
   public static ValidAndInvalidSegmentsInfo getValidAndInvalidSegmentsInfo(
-      RelationIdentifier relationIdentifier) throws IOException {
+      RelationIdentifier relationIdentifier, String tblStatusVersion) throws IOException {
     return new SegmentStatusManager(AbsoluteTableIdentifier
         .from(relationIdentifier.getTablePath(), relationIdentifier.getDatabaseName(),
-            relationIdentifier.getTableName())).getValidAndInvalidSegments();
+            relationIdentifier.getTableName()), tblStatusVersion).getValidAndInvalidSegments();
   }
 
   /**
-   * Reads the table status file with the specified UUID if non empty.
+   * Reads the table status file with the specified tblStatusVersion if non empty.
    */
-  public static LoadMetadataDetails[] readLoadMetadata(String metaDataFolderPath, String uuid)
-      throws IOException {
+  public static LoadMetadataDetails[] readLoadMetadata(String metaDataFolderPath,
+      String tblStatusVersion) {
     String tableStatusFileName;
-    if (uuid.isEmpty()) {
+    if (tblStatusVersion.isEmpty()) {
       tableStatusFileName = metaDataFolderPath + CarbonCommonConstants.FILE_SEPARATOR
           + CarbonTablePath.TABLE_STATUS_FILE;
     } else {
       tableStatusFileName = metaDataFolderPath + CarbonCommonConstants.FILE_SEPARATOR
-          + CarbonTablePath.TABLE_STATUS_FILE + CarbonCommonConstants.UNDERSCORE + uuid;
+          + CarbonTablePath.TABLE_STATUS_FILE + CarbonCommonConstants.UNDERSCORE + tblStatusVersion;
+    }
+    try {
+      return readTableStatusFile(tableStatusFileName);
+    } catch (IOException e) {
+      return new LoadMetadataDetails[0];
     }
-    return readTableStatusFile(tableStatusFileName);
   }
 
   /**
@@ -318,6 +328,12 @@ public class SegmentStatusManager {
         AtomicFileOperationFactory.getAtomicFileOperations(tableStatusPath);
 
     if (!FileFactory.isFileExist(tableStatusPath)) {
+      if (tableStatusPath.substring(tableStatusPath.lastIndexOf("/") + 1).contains(
+          CarbonTablePath.TABLE_STATUS_FILE + CarbonCommonConstants.UNDERSCORE)) {
+        throw new FileNotFoundException("Table Status Version file {" + tableStatusPath.substring(
+            tableStatusPath.lastIndexOf("/") + 1)
+            + "} not found. Try running TableStatusRecovery tool to recover lost file ");
+      }
       return null;
     }
 
@@ -364,6 +380,9 @@ public class SegmentStatusManager {
         if (retry == 0) {
           // we have retried several times, throw this exception to make the execution failed
           LOG.error("Failed to read table status file:" + tableStatusPath);
+          if (ex.getMessage().contains("Table Status Version file")) {
+            throw new RuntimeException(ex.getMessage(), ex);
+          }
           throw ex;
         }
         try {
@@ -442,8 +461,8 @@ public class SegmentStatusManager {
    * @param tableFolderPath
    * @return
    */
-  public static List<String> updateDeletionStatus(AbsoluteTableIdentifier identifier,
-      List<String> loadIds, String tableFolderPath) throws Exception {
+  public static Map<String, List<String>> updateDeletionStatus(AbsoluteTableIdentifier identifier,
+      List<String> loadIds, String tableFolderPath, String tblStatusReadVersion) throws Exception {
     CarbonTableIdentifier carbonTableIdentifier = identifier.getCarbonTableIdentifier();
     ICarbonLock carbonCleanFilesLock =
         CarbonLockFactory.getCarbonLockObj(identifier, LockUsage.CLEAN_FILES_LOCK);
@@ -453,22 +472,26 @@ public class SegmentStatusManager {
         CarbonLockFactory.getCarbonLockObj(identifier, LockUsage.TABLE_STATUS_LOCK);
     String tableDetails =
         carbonTableIdentifier.getDatabaseName() + "." + carbonTableIdentifier.getTableName();
+    String tblStatusWriteVersion = "";
+    Map<String, List<String>> tuple = new HashMap<>();
     List<String> invalidLoadIds = new ArrayList<String>(0);
+    tuple.put("invalidLoadIds", invalidLoadIds);
     try {
       if (carbonDeleteSegmentLock.lockWithRetries()) {
         LOG.info("Delete segment lock has been successfully acquired");
         if (carbonCleanFilesLock.lockWithRetries()) {
           LOG.info("Clean Files lock has been successfully acquired");
           String dataLoadLocation = CarbonTablePath.getTableStatusFilePath(identifier
-              .getTablePath());
+              .getTablePath(), tblStatusReadVersion);
           LoadMetadataDetails[] listOfLoadFolderDetailsArray = null;
           if (!FileFactory.isFileExist(dataLoadLocation)) {
             // log error.
             LOG.error("Load metadata file is not present.");
-            return loadIds;
+            tuple.put("invalidLoadIds", invalidLoadIds);
+            return tuple;
           }
           // read existing metadata details in load metadata.
-          listOfLoadFolderDetailsArray = readLoadMetadata(tableFolderPath);
+          listOfLoadFolderDetailsArray = readLoadMetadata(tableFolderPath, tblStatusReadVersion);
           if (listOfLoadFolderDetailsArray.length != 0) {
             updateDeletionStatus(identifier, loadIds, listOfLoadFolderDetailsArray, invalidLoadIds);
             if (invalidLoadIds.isEmpty()) {
@@ -477,8 +500,16 @@ public class SegmentStatusManager {
                 LOG.info("Table status lock has been successfully acquired");
                 // To handle concurrency scenarios, always take latest metadata before writing
                 // into status file.
-                LoadMetadataDetails[] latestLoadMetadataDetails = readLoadMetadata(tableFolderPath);
-                writeLoadDetailsIntoFile(dataLoadLocation, updateLatestTableStatusDetails(
+                LoadMetadataDetails[] latestLoadMetadataDetails =
+                    readLoadMetadata(tableFolderPath, tblStatusReadVersion);
+                if (CarbonProperties.isTableStatusMultiVersionEnabled()) {
+                  tblStatusWriteVersion = String.valueOf(System.currentTimeMillis());
+                  List<String> versionList = new ArrayList<>();
+                  versionList.add(tblStatusWriteVersion);
+                  tuple.put("tblStatusWriteVersion", versionList);
+                }
+                writeLoadDetailsIntoFile(CarbonTablePath.getTableStatusFilePath(identifier
+                    .getTablePath(), tblStatusWriteVersion), updateLatestTableStatusDetails(
                     listOfLoadFolderDetailsArray, latestLoadMetadataDetails).stream()
                     .toArray(LoadMetadataDetails[]::new));
               } else {
@@ -490,12 +521,14 @@ public class SegmentStatusManager {
               }
 
             } else {
-              return invalidLoadIds;
+              tuple.put("invalidLoadIds", invalidLoadIds);
+              return tuple;
             }
 
           } else {
             LOG.error("Delete segment by Id is failed. No matching segment id found.");
-            return loadIds;
+            tuple.put("invalidLoadIds", loadIds);
+            return tuple;
           }
         } else {
           String errorMsg = "Delete segment by id is failed for " + tableDetails
@@ -519,7 +552,7 @@ public class SegmentStatusManager {
       CarbonLockUtil.fileUnlock(carbonCleanFilesLock, LockUsage.CLEAN_FILES_LOCK);
     }
 
-    return invalidLoadIds;
+    return tuple;
   }
 
   /**
@@ -529,8 +562,9 @@ public class SegmentStatusManager {
    * @param tableFolderPath
    * @return
    */
-  public static List<String> updateDeletionStatus(AbsoluteTableIdentifier identifier,
-      String loadDate, String tableFolderPath, Long loadStartTime) throws Exception {
+  public static Map<String, List<String>> updateDeletionStatus(AbsoluteTableIdentifier identifier,
+      String loadDate, String tableFolderPath, Long loadStartTime, String readVersion)
+      throws Exception {
     CarbonTableIdentifier carbonTableIdentifier = identifier.getCarbonTableIdentifier();
     ICarbonLock carbonCleanFilesLock =
         CarbonLockFactory.getCarbonLockObj(identifier, LockUsage.CLEAN_FILES_LOCK);
@@ -540,23 +574,33 @@ public class SegmentStatusManager {
         CarbonLockFactory.getCarbonLockObj(identifier, LockUsage.TABLE_STATUS_LOCK);
     String tableDetails =
         carbonTableIdentifier.getDatabaseName() + "." + carbonTableIdentifier.getTableName();
+    String tblStatusWriteVersion = "";
+    Map<String, List<String>> tuple = new HashMap<>();
     List<String> invalidLoadTimestamps = new ArrayList<String>(0);
+    tuple.put("invalidLoadTimestamps", invalidLoadTimestamps);
     try {
       if (carbonDeleteSegmentLock.lockWithRetries()) {
         LOG.info("Delete segment lock has been successfully acquired");
         if (carbonCleanFilesLock.lockWithRetries()) {
           LOG.info("Clean Files lock has been successfully acquired");
+          if (CarbonProperties.isTableStatusMultiVersionEnabled()) {
+            tblStatusWriteVersion = String.valueOf(System.currentTimeMillis());
+            List<String> versionList = new ArrayList<>();
+            versionList.add(tblStatusWriteVersion);
+            tuple.put("tblStatusWriteVersion", versionList);
+          }
           String dataLoadLocation = CarbonTablePath.getTableStatusFilePath(identifier
-              .getTablePath());
+              .getTablePath(), readVersion);
           LoadMetadataDetails[] listOfLoadFolderDetailsArray = null;
 
           if (!FileFactory.isFileExist(dataLoadLocation)) {
             // Table status file is not present, maybe table is empty, ignore this operation
             LOG.warn("Trying to update table metadata file which is not present.");
-            return invalidLoadTimestamps;
+            tuple.put("invalidLoadTimestamps", invalidLoadTimestamps);
+            return tuple;
           }
           // read existing metadata details in load metadata.
-          listOfLoadFolderDetailsArray = readLoadMetadata(tableFolderPath);
+          listOfLoadFolderDetailsArray = readLoadMetadata(tableFolderPath, readVersion);
           if (listOfLoadFolderDetailsArray.length != 0) {
             updateDeletionStatus(identifier, loadDate, listOfLoadFolderDetailsArray,
                 invalidLoadTimestamps, loadStartTime);
@@ -565,8 +609,10 @@ public class SegmentStatusManager {
                 LOG.info("Table status lock has been successfully acquired.");
                 // To handle concurrency scenarios, always take latest metadata before writing
                 // into status file.
-                LoadMetadataDetails[] latestLoadMetadataDetails = readLoadMetadata(tableFolderPath);
-                writeLoadDetailsIntoFile(dataLoadLocation, updateLatestTableStatusDetails(
+                LoadMetadataDetails[] latestLoadMetadataDetails =
+                    readLoadMetadata(tableFolderPath, readVersion);
+                writeLoadDetailsIntoFile(CarbonTablePath.getTableStatusFilePath(identifier
+                    .getTablePath(), tblStatusWriteVersion), updateLatestTableStatusDetails(
                     listOfLoadFolderDetailsArray, latestLoadMetadataDetails).stream()
                     .toArray(LoadMetadataDetails[]::new));
               } else {
@@ -579,13 +625,15 @@ public class SegmentStatusManager {
 
               }
             } else {
-              return invalidLoadTimestamps;
+              tuple.put("invalidLoadTimestamps", invalidLoadTimestamps);
+              return tuple;
             }
 
           } else {
             LOG.error("Delete segment by date is failed. No matching segment found.");
             invalidLoadTimestamps.add(loadDate);
-            return invalidLoadTimestamps;
+            tuple.put("invalidLoadTimestamps", invalidLoadTimestamps);
+            return tuple;
           }
         } else {
           String errorMsg = "Delete segment by id is failed for " + tableDetails
@@ -609,7 +657,7 @@ public class SegmentStatusManager {
       CarbonLockUtil.fileUnlock(carbonCleanFilesLock, LockUsage.CLEAN_FILES_LOCK);
     }
 
-    return invalidLoadTimestamps;
+    return tuple;
   }
 
   /**
@@ -926,7 +974,7 @@ public class SegmentStatusManager {
     boolean loadInProgress = false;
     String metaPath = carbonTable.getMetadataPath();
     LoadMetadataDetails[] listOfLoadFolderDetailsArray =
-              SegmentStatusManager.readLoadMetadata(metaPath);
+              SegmentStatusManager.readLoadMetadata(metaPath, carbonTable.getTableStatusVersion());
     if (listOfLoadFolderDetailsArray.length != 0) {
       for (LoadMetadataDetails loadDetail :listOfLoadFolderDetailsArray) {
         SegmentStatus segmentStatus = loadDetail.getSegmentStatus();
@@ -971,7 +1019,7 @@ public class SegmentStatusManager {
     boolean loadInProgress = false;
     String metaPath = carbonTable.getMetadataPath();
     LoadMetadataDetails[] listOfLoadFolderDetailsArray =
-        SegmentStatusManager.readLoadMetadata(metaPath);
+        SegmentStatusManager.readLoadMetadata(metaPath, carbonTable.getTableStatusVersion());
     if (listOfLoadFolderDetailsArray.length != 0) {
       for (LoadMetadataDetails loadDetail :listOfLoadFolderDetailsArray) {
         SegmentStatus segmentStatus = loadDetail.getSegmentStatus();
@@ -1066,22 +1114,26 @@ public class SegmentStatusManager {
       AbsoluteTableIdentifier absoluteTableIdentifier, LoadMetadataDetails[] details,
       boolean cleanStaleInProgress) {
     // Delete marked loads
-    Set<String> loadsToDelete = DeleteLoadFolders
-        .deleteLoadFoldersFromFileSystem(absoluteTableIdentifier, isForceDeletion, details,
-            carbonTable.getMetadataPath(), cleanStaleInProgress);
+    Set<String> loadsToDelete =
+        DeleteLoadFolders.deleteLoadFoldersFromFileSystem(absoluteTableIdentifier, isForceDeletion,
+            details, carbonTable.getMetadataPath(), cleanStaleInProgress,
+            carbonTable.getTableStatusVersion());
     return new ReturnTuple(details, loadsToDelete);
   }
 
-  public static void deleteLoadsAndUpdateMetadata(CarbonTable carbonTable, boolean isForceDeletion,
-      List<PartitionSpec> partitionSpecs, boolean cleanStaleInprogress,
+  public static String deleteLoadsAndUpdateMetadata(CarbonTable carbonTable,
+      boolean isForceDeletion, List<PartitionSpec> partitionSpecs, boolean cleanStaleInprogress,
       boolean isCleanFilesOperation) throws IOException {
     LoadMetadataDetails[] metadataDetails =
-        SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath());
+        SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath(),
+            carbonTable.getTableStatusVersion());
+    String tblStatusWriteVersion = "";
+    boolean isTblStatusMultiVersionEnabled = CarbonProperties.isTableStatusMultiVersionEnabled();
+    boolean updateCompletionStatus = false;
     // delete the expired segment lock files
     CarbonLockUtil.deleteExpiredSegmentLockFiles(carbonTable);
     if (isLoadDeletionRequired(metadataDetails)) {
       AbsoluteTableIdentifier identifier = carbonTable.getAbsoluteTableIdentifier();
-      boolean updateCompletionStatus = false;
       Set<String> loadsToDelete = new HashSet<>();
       LoadMetadataDetails[] newAddedLoadHistoryList = null;
       ReturnTuple tuple =
@@ -1104,16 +1156,18 @@ public class SegmentStatusManager {
             LOG.info("Table status lock has been successfully acquired.");
             // Again read status and check to verify update required or not.
             LoadMetadataDetails[] details =
-                SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath());
+                SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath(),
+                    carbonTable.getTableStatusVersion());
             ReturnTuple tuple2 =
                 isUpdateRequired(isForceDeletion, carbonTable,
                     identifier, details, cleanStaleInprogress);
             if (tuple2.loadsToDelete.isEmpty()) {
-              return;
+              return tblStatusWriteVersion;
             }
             // read latest table status again.
             LoadMetadataDetails[] latestMetadata =
-                SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath());
+                SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath(),
+                    carbonTable.getTableStatusVersion());
 
             int invisibleSegmentPreserveCnt =
                 CarbonProperties.getInstance().getInvisibleSegmentPreserveCount();
@@ -1128,11 +1182,15 @@ public class SegmentStatusManager {
                   tuple2.details, latestMetadata, invisibleSegmentCnt, maxSegmentId);
               LoadMetadataDetails[] oldLoadHistoryList = readLoadHistoryMetadata(
                   carbonTable.getMetadataPath());
-              LoadMetadataDetails[] newLoadHistoryList = appendLoadHistoryList(
-                  oldLoadHistoryList, tableStatusReturn.arrayOfLoadHistoryDetails);
+              LoadMetadataDetails[] newLoadHistoryList = appendLoadHistoryList(oldLoadHistoryList,
+                  tableStatusReturn.arrayOfLoadHistoryDetails);
+
+              if (isTblStatusMultiVersionEnabled) {
+                tblStatusWriteVersion = String.valueOf(System.currentTimeMillis());
+              }
               writeLoadDetailsIntoFile(
-                  CarbonTablePath.getTableStatusFilePath(carbonTable.getTablePath()),
-                  tableStatusReturn.arrayOfLoadDetails);
+                  CarbonTablePath.getTableStatusFilePath(carbonTable.getTablePath(),
+                      tblStatusWriteVersion), tableStatusReturn.arrayOfLoadDetails);
               writeLoadDetailsIntoFile(
                   CarbonTablePath.getTableStatusHistoryFilePath(carbonTable.getTablePath()),
                   newLoadHistoryList);
@@ -1142,8 +1200,12 @@ public class SegmentStatusManager {
               // update the metadata details from old to new status.
               List<LoadMetadataDetails> latestStatus =
                   updateLoadMetadataFromOldToNew(tuple2.details, latestMetadata);
+              if (isTblStatusMultiVersionEnabled) {
+                tblStatusWriteVersion = String.valueOf(System.currentTimeMillis());
+              }
               writeLoadDetailsIntoFile(
-                  CarbonTablePath.getTableStatusFilePath(identifier.getTablePath()),
+                  CarbonTablePath.getTableStatusFilePath(identifier.getTablePath(),
+                      tblStatusWriteVersion),
                   latestStatus.toArray(new LoadMetadataDetails[0]));
             }
             updateCompletionStatus = true;
@@ -1163,13 +1225,14 @@ public class SegmentStatusManager {
             CarbonLockUtil.fileUnlock(carbonTableStatusLock, LockUsage.TABLE_STATUS_LOCK);
           }
           if (updateCompletionStatus) {
-            DeleteLoadFolders
-                .physicalFactAndMeasureMetadataDeletion(carbonTable, newAddedLoadHistoryList,
-                  isForceDeletion, partitionSpecs, cleanStaleInprogress, loadsToDelete);
+            DeleteLoadFolders.physicalFactAndMeasureMetadataDeletion(carbonTable,
+                newAddedLoadHistoryList, isForceDeletion, partitionSpecs, cleanStaleInprogress,
+                loadsToDelete, tblStatusWriteVersion);
           }
         }
       }
     }
+    return tblStatusWriteVersion;
   }
 
   public static void truncateTable(CarbonTable carbonTable)
@@ -1182,9 +1245,9 @@ public class SegmentStatusManager {
       locked = carbonTableStatusLock.lockWithRetries();
       if (locked) {
         LOG.info("Table status lock has been successfully acquired.");
-        LoadMetadataDetails[] listOfLoadFolderDetailsArray =
-            SegmentStatusManager.readLoadMetadata(
-                CarbonTablePath.getMetadataPath(carbonTable.getTablePath()));
+        LoadMetadataDetails[] listOfLoadFolderDetailsArray = SegmentStatusManager.readLoadMetadata(
+            CarbonTablePath.getMetadataPath(carbonTable.getTablePath()),
+            carbonTable.getTableStatusVersion());
         for (LoadMetadataDetails listOfLoadFolderDetails : listOfLoadFolderDetailsArray) {
           boolean writing;
           switch (listOfLoadFolderDetails.getSegmentStatus()) {
@@ -1203,10 +1266,9 @@ public class SegmentStatusManager {
         for (LoadMetadataDetails listOfLoadFolderDetails : listOfLoadFolderDetailsArray) {
           listOfLoadFolderDetails.setSegmentStatus(SegmentStatus.MARKED_FOR_DELETE);
         }
-        SegmentStatusManager
-            .writeLoadDetailsIntoFile(
-                CarbonTablePath.getTableStatusFilePath(carbonTable.getTablePath()),
-                listOfLoadFolderDetailsArray);
+        SegmentStatusManager.writeLoadDetailsIntoFile(
+            CarbonTablePath.getTableStatusFilePath(carbonTable.getTablePath(),
+                carbonTable.getTableStatusVersion()), listOfLoadFolderDetailsArray);
       } else {
         String dbName = carbonTable.getDatabaseName();
         String tableName = carbonTable.getTableName();
@@ -1323,7 +1385,7 @@ public class SegmentStatusManager {
    */
   public static Map<String, String> mapSegmentToStartTime(CarbonTable carbonTable) {
     LoadMetadataDetails[] loadMetadataDetails = SegmentStatusManager.readLoadMetadata(
-        carbonTable.getMetadataPath());
+        carbonTable.getMetadataPath(), carbonTable.getTableStatusVersion());
     if (loadMetadataDetails != null && loadMetadataDetails.length > 0) {
       Map<String, String> map = new HashMap<>(loadMetadataDetails.length);
       for (LoadMetadataDetails loadMetadataDetail : loadMetadataDetails) {
diff --git a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java
index 8572ae5c45..75859d2a88 100644
--- a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java
+++ b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java
@@ -104,7 +104,8 @@ public class SegmentUpdateStatusManager {
       segmentDetails = new LoadMetadataDetails[0];
     } else {
       segmentDetails = SegmentStatusManager.readLoadMetadata(
-          CarbonTablePath.getMetadataPath(identifier.getTablePath()));
+          CarbonTablePath.getMetadataPath(identifier.getTablePath()),
+          table.getTableStatusVersion());
     }
     this.isPartitionTable = table.isHivePartitionTable();
     if (segmentDetails.length != 0) {
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
index 8d8105a7c1..29bf251be0 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
@@ -1258,6 +1258,12 @@ public final class CarbonProperties {
         CarbonCommonConstants.ENABLE_TABLE_STATUS_BACKUP_DEFAULT).equalsIgnoreCase("true");
   }
 
+  public static boolean isTableStatusMultiVersionEnabled() {
+    return getInstance().getProperty(CarbonCommonConstants.CARBON_ENABLE_MULTI_VERSION_TABLE_STATUS,
+            CarbonCommonConstants.CARBON_ENABLE_MULTI_VERSION_TABLE_STATUS_DEFAULT)
+        .equalsIgnoreCase("true");
+  }
+
   /**
    * Validate the restrictions
    *
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
index 0f71c360d3..19040cf84e 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
@@ -2433,7 +2433,8 @@ public final class CarbonUtil {
         .getProperty(CarbonCommonConstants.ENABLE_CALCULATE_SIZE,
             CarbonCommonConstants.DEFAULT_ENABLE_CALCULATE_SIZE);
     if (isCalculated.equalsIgnoreCase("true")) {
-      SegmentStatusManager segmentStatusManager = new SegmentStatusManager(identifier);
+      SegmentStatusManager segmentStatusManager =
+          new SegmentStatusManager(identifier, carbonTable.getTableStatusVersion());
       ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
       try {
         boolean lockAcquired = true;
@@ -2444,7 +2445,8 @@ public final class CarbonUtil {
           LOGGER.debug("Acquired lock for table for table status update");
           String metadataPath = carbonTable.getMetadataPath();
           LoadMetadataDetails[] loadMetadataDetails =
-              SegmentStatusManager.readLoadMetadata(metadataPath);
+              SegmentStatusManager.readLoadMetadata(metadataPath,
+                  carbonTable.getTableStatusVersion());
 
           for (LoadMetadataDetails loadMetadataDetail : loadMetadataDetails) {
             SegmentStatus loadStatus = loadMetadataDetail.getSegmentStatus();
@@ -2467,14 +2469,14 @@ public final class CarbonUtil {
               totalIndexSize += Long.parseLong(indexSize);
             }
           }
+          String tableStatusPath = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath(),
+              carbonTable.getTableStatusVersion());
           // If it contains old segment, write new load details
           if (needUpdate && updateSize) {
             SegmentStatusManager.writeLoadDetailsIntoFile(
-                CarbonTablePath.getTableStatusFilePath(identifier.getTablePath()),
+                tableStatusPath,
                 loadMetadataDetails);
           }
-          String tableStatusPath =
-              CarbonTablePath.getTableStatusFilePath(identifier.getTablePath());
           if (FileFactory.isFileExist(tableStatusPath)) {
             lastUpdateTime =
                 FileFactory.getCarbonFile(tableStatusPath).getLastModifiedTime();
@@ -3173,7 +3175,8 @@ public final class CarbonUtil {
     } else {
       // get the valid segments
       SegmentStatusManager segmentStatusManager =
-          new SegmentStatusManager(carbonTable.getAbsoluteTableIdentifier());
+          new SegmentStatusManager(carbonTable.getAbsoluteTableIdentifier(),
+              carbonTable.getTableStatusVersion());
       SegmentStatusManager.ValidAndInvalidSegmentsInfo validAndInvalidSegmentsInfo =
           segmentStatusManager.getValidAndInvalidSegments(carbonTable.isMV());
       List<Segment> validSegments = validAndInvalidSegmentsInfo.getValidSegments();
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CleanFilesUtil.java b/core/src/main/java/org/apache/carbondata/core/util/CleanFilesUtil.java
index 44d1d8a922..bfa9b949fc 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CleanFilesUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CleanFilesUtil.java
@@ -171,7 +171,7 @@ public class CleanFilesUtil {
       return;
     }
     LoadMetadataDetails[] details = SegmentStatusManager.readLoadMetadata(carbonTable
-        .getMetadataPath());
+        .getMetadataPath(), carbonTable.getTableStatusVersion());
     if (details == null || details.length == 0) {
       return;
     }
diff --git a/core/src/main/java/org/apache/carbondata/core/util/DeleteLoadFolders.java b/core/src/main/java/org/apache/carbondata/core/util/DeleteLoadFolders.java
index 746fbb6e04..ae2db0a927 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/DeleteLoadFolders.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/DeleteLoadFolders.java
@@ -72,9 +72,11 @@ public final class DeleteLoadFolders {
       boolean isForceDelete,
       List<PartitionSpec> specs,
       boolean cleanStaleInProgress,
-      Set<String> loadsToDelete) {
+      Set<String> loadsToDelete,
+      String tblStatusVersion) {
     LoadMetadataDetails[] currentDetails =
-        SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath());
+        SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath(),
+            tblStatusVersion);
     physicalFactAndMeasureMetadataDeletion(carbonTable,
         currentDetails,
         isForceDelete,
@@ -230,8 +232,9 @@ public final class DeleteLoadFolders {
   }
 
   private static LoadMetadataDetails getCurrentLoadStatusOfSegment(String segmentId,
-      String metadataPath) {
-    LoadMetadataDetails[] currentDetails = SegmentStatusManager.readLoadMetadata(metadataPath);
+      String metadataPath, String version) {
+    LoadMetadataDetails[] currentDetails =
+        SegmentStatusManager.readLoadMetadata(metadataPath, version);
     for (LoadMetadataDetails oneLoad : currentDetails) {
       if (oneLoad.getLoadName().equalsIgnoreCase(segmentId)) {
         return oneLoad;
@@ -242,7 +245,7 @@ public final class DeleteLoadFolders {
 
   public static Set<String> deleteLoadFoldersFromFileSystem(
       AbsoluteTableIdentifier absoluteTableIdentifier, boolean isForceDelete, LoadMetadataDetails[]
-      details, String metadataPath, boolean cleanStaleInProgress) {
+      details, String metadataPath, boolean cleanStaleInProgress, String version) {
     Set<String> loadsToDelete = new HashSet<>();
     if (details != null && details.length != 0) {
       for (LoadMetadataDetails oneLoad : details) {
@@ -251,7 +254,7 @@ public final class DeleteLoadFolders {
           if (oneLoad.getSegmentStatus() == SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS
               || oneLoad.getSegmentStatus() == SegmentStatus.INSERT_IN_PROGRESS) {
             LoadMetadataDetails currentDetails =
-                getCurrentLoadStatusOfSegment(oneLoad.getLoadName(), metadataPath);
+                getCurrentLoadStatusOfSegment(oneLoad.getLoadName(), metadataPath, version);
             if (currentDetails != null && checkIfLoadCanBeDeleted(currentDetails,
                 isForceDelete, cleanStaleInProgress, absoluteTableIdentifier)) {
               oneLoad.setVisibility("false");
diff --git a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java b/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
index 9967723e90..4dc52d8bf0 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
@@ -209,6 +209,17 @@ public class CarbonTablePath {
     return getMetadataPath(tablePath) + CarbonCommonConstants.FILE_SEPARATOR + TABLE_STATUS_FILE;
   }
 
+  /**
+   * Return absolute path of table status file
+   */
+  public static String getTableStatusFilePath(String tablePath, String tblStatusVersion) {
+    String tableStatusPath = getTableStatusFilePath(tablePath);
+    if (tblStatusVersion.isEmpty()) {
+      return tableStatusPath;
+    }
+    return tableStatusPath + CarbonCommonConstants.UNDERSCORE + tblStatusVersion;
+  }
+
   public static String getTableStatusFilePathWithUUID(String tablePath, String uuid) {
     if (!uuid.isEmpty()) {
       return getTableStatusFilePath(tablePath) + CarbonCommonConstants.UNDERSCORE + uuid;
diff --git a/core/src/main/java/org/apache/carbondata/core/view/MVManager.java b/core/src/main/java/org/apache/carbondata/core/view/MVManager.java
index 7253158035..e93048f2b5 100644
--- a/core/src/main/java/org/apache/carbondata/core/view/MVManager.java
+++ b/core/src/main/java/org/apache/carbondata/core/view/MVManager.java
@@ -29,6 +29,7 @@ import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.locks.ICarbonLock;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.metadata.CarbonMetadata;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.RelationIdentifier;
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
@@ -336,10 +337,16 @@ public abstract class MVManager {
         setStatus(schema.identifier, MVStatus.DISABLED);
       }
       RelationIdentifier relationIdentifier = schema.getIdentifier();
+      CarbonTable carbonTable = CarbonMetadata.getInstance()
+          .getCarbonTable(relationIdentifier.getDatabaseName(), relationIdentifier.getTableName());
+      String tblStatusVersion = "";
+      if (null != carbonTable) {
+        tblStatusVersion = carbonTable.getTableStatusVersion();
+      }
       SegmentStatusManager segmentStatusManager = new SegmentStatusManager(AbsoluteTableIdentifier
           .from(relationIdentifier.getTablePath(),
               relationIdentifier.getDatabaseName(),
-              relationIdentifier.getTableName()));
+              relationIdentifier.getTableName()), tblStatusVersion);
       ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
       try {
         if (carbonLock.lockWithRetries()) {
@@ -348,13 +355,13 @@ public abstract class MVManager {
           String metaDataPath =
               CarbonTablePath.getMetadataPath(relationIdentifier.getTablePath());
           LoadMetadataDetails[] loadMetadataDetails =
-              SegmentStatusManager.readLoadMetadata(metaDataPath);
+              SegmentStatusManager.readLoadMetadata(metaDataPath, tblStatusVersion);
           for (LoadMetadataDetails entry : loadMetadataDetails) {
             entry.setSegmentStatus(SegmentStatus.MARKED_FOR_DELETE);
           }
           SegmentStatusManager.writeLoadDetailsIntoFile(
-              CarbonTablePath.getTableStatusFilePath(relationIdentifier.getTablePath()),
-              loadMetadataDetails);
+              CarbonTablePath.getTableStatusFilePath(relationIdentifier.getTablePath(),
+                  tblStatusVersion), loadMetadataDetails);
         } else {
           LOGGER.error("Not able to acquire the lock for Table status update for table "
               + relationIdentifier.getDatabaseName() + "." + relationIdentifier
diff --git a/core/src/main/java/org/apache/carbondata/core/view/MVProvider.java b/core/src/main/java/org/apache/carbondata/core/view/MVProvider.java
index 149ff4f854..9fd07cbf57 100644
--- a/core/src/main/java/org/apache/carbondata/core/view/MVProvider.java
+++ b/core/src/main/java/org/apache/carbondata/core/view/MVProvider.java
@@ -49,6 +49,7 @@ import org.apache.carbondata.core.fileoperations.FileWriteOperation;
 import org.apache.carbondata.core.locks.CarbonLockFactory;
 import org.apache.carbondata.core.locks.ICarbonLock;
 import org.apache.carbondata.core.locks.LockUsage;
+import org.apache.carbondata.core.metadata.CarbonMetadata;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.RelationIdentifier;
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
@@ -352,8 +353,14 @@ public class MVProvider {
     boolean isViewCanBeEnabled = true;
     String viewMetadataPath =
         CarbonTablePath.getMetadataPath(schema.getIdentifier().getTablePath());
+    CarbonTable table = CarbonMetadata.getInstance()
+        .getCarbonTable(schema.identifier.getDatabaseName(), schema.identifier.getTableName());
+    String tblStatusVersion = "";
+    if (null != table) {
+      tblStatusVersion = table.getTableStatusVersion();
+    }
     LoadMetadataDetails[] viewLoadMetadataDetails =
-        SegmentStatusManager.readLoadMetadata(viewMetadataPath);
+        SegmentStatusManager.readLoadMetadata(viewMetadataPath, tblStatusVersion);
     Map<String, List<String>> viewSegmentMap = new HashMap<>();
     for (LoadMetadataDetails loadMetadataDetail : viewLoadMetadataDetails) {
       if (loadMetadataDetail.getSegmentStatus() == SegmentStatus.SUCCESS) {
@@ -372,8 +379,14 @@ public class MVProvider {
     }
     List<RelationIdentifier> relatedTables = schema.getRelatedTables();
     for (RelationIdentifier relatedTable : relatedTables) {
+      CarbonTable carbonTable = CarbonMetadata.getInstance()
+          .getCarbonTable(relatedTable.getDatabaseName(), relatedTable.getTableName());
+      String parentTblVersion = "";
+      if (null != carbonTable) {
+        parentTblVersion = carbonTable.getTableStatusVersion();
+      }
       SegmentStatusManager.ValidAndInvalidSegmentsInfo validAndInvalidSegmentsInfo =
-          SegmentStatusManager.getValidAndInvalidSegmentsInfo(relatedTable);
+          SegmentStatusManager.getValidAndInvalidSegmentsInfo(relatedTable, parentTblVersion);
       List<String> relatedTableSegmentList =
           SegmentStatusManager.getValidSegmentList(validAndInvalidSegmentsInfo);
       if (!relatedTableSegmentList.isEmpty()) {
diff --git a/core/src/main/java/org/apache/carbondata/core/view/MVSchema.java b/core/src/main/java/org/apache/carbondata/core/view/MVSchema.java
index b7564ac5e0..c93356fccb 100644
--- a/core/src/main/java/org/apache/carbondata/core/view/MVSchema.java
+++ b/core/src/main/java/org/apache/carbondata/core/view/MVSchema.java
@@ -29,15 +29,8 @@ import java.util.Objects;
 import java.util.Set;
 
 import org.apache.carbondata.common.Strings;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.metadata.schema.table.RelationIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.Writable;
-import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
-import org.apache.carbondata.core.statusmanager.SegmentStatus;
-import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
-import org.apache.carbondata.core.util.path.CarbonTablePath;
-
-import com.google.gson.Gson;
 
 /**
  * It is the new schema of mv and it has less fields compare to {{@link MVSchema}}
@@ -243,40 +236,6 @@ public class MVSchema implements Serializable, Writable {
     return MVStatus.DISABLED;
   }
 
-  public String getSyncStatus() {
-    LoadMetadataDetails[] loads =
-        SegmentStatusManager.readLoadMetadata(
-            CarbonTablePath.getMetadataPath(this.getIdentifier().getTablePath()));
-    if (loads.length > 0) {
-      for (int i = loads.length - 1; i >= 0; i--) {
-        LoadMetadataDetails load = loads[i];
-        if (load.getSegmentStatus().equals(SegmentStatus.SUCCESS)) {
-          Map<String, List<String>> segmentMaps =
-              new Gson().fromJson(load.getExtraInfo(), Map.class);
-          Map<String, String> syncInfoMap = new HashMap<>();
-          for (Map.Entry<String, List<String>> entry : segmentMaps.entrySet()) {
-            // when in join scenario, one table is loaded and one more is not loaded,
-            // then put value as NA
-            if (entry.getValue().isEmpty()) {
-              syncInfoMap.put(entry.getKey(), "NA");
-            } else {
-              syncInfoMap.put(entry.getKey(), getMaxSegmentID(entry.getValue()));
-            }
-          }
-          String loadEndTime;
-          if (load.getLoadEndTime() == CarbonCommonConstants.SEGMENT_LOAD_TIME_DEFAULT) {
-            loadEndTime = "NA";
-          } else {
-            loadEndTime = new java.sql.Timestamp(load.getLoadEndTime()).toString();
-          }
-          syncInfoMap.put(CarbonCommonConstants.LOAD_SYNC_TIME, loadEndTime);
-          return new Gson().toJson(syncInfoMap);
-        }
-      }
-    }
-    return "NA";
-  }
-
   private static String getMaxSegmentID(List<String> segmentList) {
     double[] segment = new double[segmentList.size()];
     int i = 0;
diff --git a/core/src/main/java/org/apache/carbondata/core/writer/CarbonIndexFileMergeWriter.java b/core/src/main/java/org/apache/carbondata/core/writer/CarbonIndexFileMergeWriter.java
index 2c455d6ef1..2ca0d81e5e 100644
--- a/core/src/main/java/org/apache/carbondata/core/writer/CarbonIndexFileMergeWriter.java
+++ b/core/src/main/java/org/apache/carbondata/core/writer/CarbonIndexFileMergeWriter.java
@@ -35,12 +35,14 @@ import org.apache.carbondata.core.fileoperations.FileWriteOperation;
 import org.apache.carbondata.core.index.Segment;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
 import org.apache.carbondata.core.indexstore.blockletindex.SegmentIndexFileStore;
+import org.apache.carbondata.core.metadata.CarbonMetadata;
 import org.apache.carbondata.core.metadata.SegmentFileStore;
 import org.apache.carbondata.core.metadata.schema.indextable.IndexMetadata;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
 import org.apache.carbondata.core.statusmanager.SegmentStatus;
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
+import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.ObjectSerializationUtil;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.format.MergedBlockIndex;
@@ -80,7 +82,7 @@ public class CarbonIndexFileMergeWriter {
   private String mergeCarbonIndexFilesOfSegment(String segmentId, String tablePath,
       List<String> indexFileNamesTobeAdded, boolean isOldStoreIndexFilesPresent, String uuid,
       String partitionPath) {
-    Segment segment = Segment.getSegment(segmentId, tablePath);
+    Segment segment = Segment.getSegment(segmentId, tablePath, table.getTableStatusVersion());
     String segmentPath = CarbonTablePath.getSegmentPath(tablePath, segmentId);
     try {
       List<CarbonFile> indexFiles = new ArrayList<>();
@@ -239,9 +241,11 @@ public class CarbonIndexFileMergeWriter {
     }
     Map<String, Map<String, byte[]>> indexLocationMap =
         groupIndexesBySegment(fileStore.getCarbonIndexMapWithFullPath());
-    List<PartitionSpec> partitionSpecs = SegmentFileStore
-        .getPartitionSpecs(segmentId, table.getTablePath(), SegmentStatusManager
-            .readLoadMetadata(CarbonTablePath.getMetadataPath(table.getTablePath())));
+    List<PartitionSpec> partitionSpecs =
+        SegmentFileStore.getPartitionSpecs(segmentId, table.getTablePath(),
+            SegmentStatusManager.readLoadMetadata(
+                CarbonTablePath.getMetadataPath(table.getTablePath()),
+                table.getTableStatusVersion()));
     List<String> mergeIndexFiles = new ArrayList<>();
     for (Map.Entry<String, Map<String, byte[]>> entry : indexLocationMap.entrySet()) {
       String mergeIndexFile = writeMergeIndexFile(indexFileNamesTobeAdded,
@@ -284,8 +288,15 @@ public class CarbonIndexFileMergeWriter {
     if (table.isIndexTable()) {
       // To maintain same segment file name mapping between parent and SI table.
       IndexMetadata indexMetadata = table.getIndexMetadata();
-      LoadMetadataDetails[] loadDetails = SegmentStatusManager
-          .readLoadMetadata(CarbonTablePath.getMetadataPath(indexMetadata.getParentTablePath()));
+      String parentTableVersion = "";
+      if (CarbonProperties.isTableStatusMultiVersionEnabled()) {
+        parentTableVersion = CarbonMetadata.getInstance()
+            .getCarbonTable(table.getDatabaseName(), indexMetadata.getParentTableName())
+            .getTableStatusVersion();
+      }
+      LoadMetadataDetails[] loadDetails = SegmentStatusManager.readLoadMetadata(
+          CarbonTablePath.getMetadataPath(indexMetadata.getParentTablePath()),
+          parentTableVersion);
       LoadMetadataDetails loadMetaDetail = Arrays.stream(loadDetails)
           .filter(loadDetail -> loadDetail.getLoadName().equals(segmentId)).findFirst().get();
       newSegmentFileName = loadMetaDetail.getSegmentFile();
@@ -310,7 +321,8 @@ public class CarbonIndexFileMergeWriter {
         throw ex;
       }
       boolean status = SegmentFileStore.updateTableStatusFile(table, segmentId, newSegmentFileName,
-          table.getCarbonTableIdentifier().getTableId(), segmentFileStore);
+          table.getCarbonTableIdentifier().getTableId(), segmentFileStore,
+          table.getTableStatusVersion());
       if (!status) {
         throw new IOException("Table status update with mergeIndex file has failed");
       }
diff --git a/core/src/test/java/org/apache/carbondata/core/indexstore/blockletindex/TestBlockletIndexFactory.java b/core/src/test/java/org/apache/carbondata/core/indexstore/blockletindex/TestBlockletIndexFactory.java
index 70ab48820b..42e259c365 100644
--- a/core/src/test/java/org/apache/carbondata/core/indexstore/blockletindex/TestBlockletIndexFactory.java
+++ b/core/src/test/java/org/apache/carbondata/core/indexstore/blockletindex/TestBlockletIndexFactory.java
@@ -116,8 +116,9 @@ public class TestBlockletIndexFactory {
   @Test public void getValidDistributables() throws IOException {
     BlockletIndexInputSplit blockletIndexInputSplit = new BlockletIndexInputSplit(
         "/opt/store/default/carbon_table/Fact/Part0/Segment_0/0_batchno0-0-1521012756709.carbonindex");
-    Segment segment = new Segment("0", null, new TableStatusReadCommittedScope(carbonTable
-        .getAbsoluteTableIdentifier(), new Configuration(false)));
+    Segment segment = new Segment("0", null,
+        new TableStatusReadCommittedScope(carbonTable.getAbsoluteTableIdentifier(),
+            new Configuration(false), carbonTable.getTableStatusVersion()));
     blockletIndexInputSplit.setSegment(segment);
     BlockletIndexInputSplit indexInputSplit = new BlockletIndexInputSplit(
         "/opt/store/default/carbon_table/Fact/Part0/Segment_0/0_batchno0-0-1521012756701.carbonindex");
diff --git a/docs/configuration-parameters.md b/docs/configuration-parameters.md
index c24518a645..1225190a6d 100644
--- a/docs/configuration-parameters.md
+++ b/docs/configuration-parameters.md
@@ -29,34 +29,35 @@
 ##  System Configuration
 This section provides the details of all the configurations required for the CarbonData System.
 
-| Property | Default Value | Description |
-|----------------------------|-------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- [...]
-| carbon.ddl.base.hdfs.url | (none) | To simplify and shorten the path to be specified in DDL/DML commands, this property is supported. This property is used to configure the HDFS relative path, the path configured in carbon.ddl.base.hdfs.url will be appended to the HDFS path configured in fs.defaultFS of core-site.xml. If this path is configured, then user need not pass the complete path while dataload. For example: If absolute path of the csv file is hdfs://10.18.101.155:54310/data/cnb [...]
-| carbon.badRecords.location | (none) | CarbonData can detect the records not conforming to defined table schema and isolate them as bad records. This property is used to specify where to store such bad records. |
-| carbon.streaming.auto.handoff.enabled | true | CarbonData supports storing of streaming data. To have high throughput for streaming, the data is written in Row format which is highly optimized for write, but performs poorly for query. When this property is true and when the streaming data size reaches ***carbon.streaming.segment.max.size***, CabonData will automatically convert the data to columnar format and optimize it for faster querying. **NOTE:** It is not recommended to keep the  [...]
-| carbon.streaming.segment.max.size | 1024000000 | CarbonData writes streaming data in row format which is optimized for high write throughput. This property defines the maximum size of data to be held is row format, beyond which it will be converted to columnar format in order to support high performance query, provided ***carbon.streaming.auto.handoff.enabled*** is true. **NOTE:** Setting higher value will impact the streaming ingestion. The value has to be configured in bytes. |
-| carbon.segment.lock.files.preserve.hours | 48 | In order to support parallel data loading onto the same table, CarbonData sequences(locks) at the granularity of segments. Operations affecting the segment(like IUD, alter) are blocked from parallel operations. This property value indicates the number of hours the segment lock files will be preserved after dataload. These lock files will be deleted with the clean command after the configured number of hours. |
-| carbon.timestamp.format | yyyy-MM-dd HH:mm:ss | CarbonData can understand data of timestamp type and process it in special manner. It can be so that the format of Timestamp data is different from that understood by CarbonData by default. This configuration allows users to specify the format of Timestamp in their data. |
-| carbon.lock.type | LOCALLOCK | This configuration specifies the type of lock to be acquired during concurrent operations on table. There are following types of lock implementation: - LOCALLOCK: Lock is created on local file system as file. This lock is useful when only one spark driver (thrift server) runs on a machine and no other CarbonData spark application is launched concurrently. - HDFSLOCK: Lock is created on HDFS file system as file. This lock is useful when multiple CarbonData [...]
-| carbon.lock.path | TABLEPATH | This configuration specifies the path where lock files have to be created. Recommended to configure zookeeper lock type or configure HDFS lock path(to this property) in case of S3 file system as locking is not feasible on S3. |
-| enable.offheap.sort | true | Whether carbondata will use offheap or onheap memory. By default, the value is true and carbondata will use the property value from *carbon.unsafe.working.memory.in.mb* or *carbon.unsafe.driver.working.memory.in.mb* as the amount of memory; if it is false, carbondata will use the minimum value between the configured amount of unsafe memory and the 60% of JVM Heap Memory as the amount of memory. |
-| carbon.unsafe.working.memory.in.mb | 512 | CarbonData supports storing data in off-heap memory for certain operations during data loading and query. This helps to avoid the Java GC and thereby improve the overall performance. The Minimum value recommeded is 512MB. Any value below this is reset to default value of 512MB. **NOTE:** The below formulas explain how to arrive at the off-heap size required.<u>Memory Required For Data Loading per executor: </u>(*carbon.number.of.cores.while.lo [...]
-| carbon.unsafe.driver.working.memory.in.mb | (none) | CarbonData supports storing data in unsafe on-heap memory in driver for certain operations like insert into, query for loading index cache. The Minimum value recommended is 512MB. If this configuration is not set, carbondata will use the value of `carbon.unsafe.working.memory.in.mb`. |
-| carbon.update.sync.folder | /tmp/carbondata | CarbonData maintains last modification time entries in modifiedTime.mdt to determine the schema changes and reload only when necessary. This configuration specifies the path where the file needs to be written. |
+| Property | Default Value | Description                                                                                                                                                                                                                                                                                                                                                                                                                                                                     [...]
+|----------------------------|-------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- [...]
+| carbon.ddl.base.hdfs.url | (none) | To simplify and shorten the path to be specified in DDL/DML commands, this property is supported. This property is used to configure the HDFS relative path, the path configured in carbon.ddl.base.hdfs.url will be appended to the HDFS path configured in fs.defaultFS of core-site.xml. If this path is configured, then user need not pass the complete path while dataload. For example: If absolute path of the csv file is hdfs://10.18.101.155:54310/data/cnb [...]
+| carbon.badRecords.location | (none) | CarbonData can detect the records not conforming to defined table schema and isolate them as bad records. This property is used to specify where to store such bad records.                                                                                                                                                                                                                                                                                          [...]
+| carbon.streaming.auto.handoff.enabled | true | CarbonData supports storing of streaming data. To have high throughput for streaming, the data is written in Row format which is highly optimized for write, but performs poorly for query. When this property is true and when the streaming data size reaches ***carbon.streaming.segment.max.size***, CabonData will automatically convert the data to columnar format and optimize it for faster querying. **NOTE:** It is not recommended to keep the  [...]
+| carbon.streaming.segment.max.size | 1024000000 | CarbonData writes streaming data in row format which is optimized for high write throughput. This property defines the maximum size of data to be held is row format, beyond which it will be converted to columnar format in order to support high performance query, provided ***carbon.streaming.auto.handoff.enabled*** is true. **NOTE:** Setting higher value will impact the streaming ingestion. The value has to be configured in bytes.         [...]
+| carbon.segment.lock.files.preserve.hours | 48 | In order to support parallel data loading onto the same table, CarbonData sequences(locks) at the granularity of segments. Operations affecting the segment(like IUD, alter) are blocked from parallel operations. This property value indicates the number of hours the segment lock files will be preserved after dataload. These lock files will be deleted with the clean command after the configured number of hours.                                [...]
+| carbon.timestamp.format | yyyy-MM-dd HH:mm:ss | CarbonData can understand data of timestamp type and process it in special manner. It can be so that the format of Timestamp data is different from that understood by CarbonData by default. This configuration allows users to specify the format of Timestamp in their data.                                                                                                                                                                            [...]
+| carbon.lock.type | LOCALLOCK | This configuration specifies the type of lock to be acquired during concurrent operations on table. There are following types of lock implementation: - LOCALLOCK: Lock is created on local file system as file. This lock is useful when only one spark driver (thrift server) runs on a machine and no other CarbonData spark application is launched concurrently. - HDFSLOCK: Lock is created on HDFS file system as file. This lock is useful when multiple CarbonData [...]
+| carbon.lock.path | TABLEPATH | This configuration specifies the path where lock files have to be created. Recommended to configure zookeeper lock type or configure HDFS lock path(to this property) in case of S3 file system as locking is not feasible on S3.                                                                                                                                                                                                                                           [...]
+| enable.offheap.sort | true | Whether carbondata will use offheap or onheap memory. By default, the value is true and carbondata will use the property value from *carbon.unsafe.working.memory.in.mb* or *carbon.unsafe.driver.working.memory.in.mb* as the amount of memory; if it is false, carbondata will use the minimum value between the configured amount of unsafe memory and the 60% of JVM Heap Memory as the amount of memory.                                                                 [...]
+| carbon.unsafe.working.memory.in.mb | 512 | CarbonData supports storing data in off-heap memory for certain operations during data loading and query. This helps to avoid the Java GC and thereby improve the overall performance. The Minimum value recommeded is 512MB. Any value below this is reset to default value of 512MB. **NOTE:** The below formulas explain how to arrive at the off-heap size required.<u>Memory Required For Data Loading per executor: </u>(*carbon.number.of.cores.while.lo [...]
+| carbon.unsafe.driver.working.memory.in.mb | (none) | CarbonData supports storing data in unsafe on-heap memory in driver for certain operations like insert into, query for loading index cache. The Minimum value recommended is 512MB. If this configuration is not set, carbondata will use the value of `carbon.unsafe.working.memory.in.mb`.                                                                                                                                                          [...]
+| carbon.update.sync.folder | /tmp/carbondata | CarbonData maintains last modification time entries in modifiedTime.mdt to determine the schema changes and reload only when necessary. This configuration specifies the path where the file needs to be written.                                                                                                                                                                                                                                            [...]
 | carbon.invisible.segments.preserve.count | 200 | CarbonData maintains each data load entry in tablestatus file. The entries from this file are not deleted for those segments that are compacted or dropped, but are made invisible. If the number of data loads are very high, the size and number of entries in tablestatus file can become too many causing unnecessary reading of all data. This configuration specifies the number of segment entries to be maintained afte they are compacted or dro [...]
-| carbon.lock.retries | 3 | CarbonData ensures consistency of operations by blocking certain operations from running in parallel. In order to block the operations from running in parallel, lock is obtained on the table. This configuration specifies the maximum number of retries to obtain the lock for any operations other than load. **NOTE:** Data manupulation operations like Compaction,UPDATE,DELETE  or LOADING,UPDATE,DELETE are not allowed to run in parallel. How ever data loading can h [...]
-| carbon.lock.retry.timeout.sec | 5 | Specifies the interval between the retries to obtain the lock for any operation other than load. **NOTE:** Refer to ***carbon.lock.retries*** for understanding why CarbonData uses locks for operations. |
-| carbon.fs.custom.file.provider | None | To support FileTypeInterface for configuring custom CarbonFile implementation to work with custom FileSystem. |
-| carbon.timeseries.first.day.of.week | SUNDAY | This parameter configures which day of the week to be considered as first day of the week. Because first day of the week will be different in different parts of the world. |
-| carbon.enable.tablestatus.backup | false | In cloud object store scenario, overwriting table status file is not an atomic operation since it uses rename API. Thus, it is possible that table status is corrupted if process crashed when overwriting the table status file. To protect from file corruption, user can enable this property. |
-| carbon.trash.retention.days | 7 | This parameter specifies the number of days after which the timestamp based subdirectories are expired in the trash folder. Allowed Min value = 0, Allowed Max Value = 365 days|
-| carbon.clean.file.force.allowed | false | This parameter specifies if the clean files operation with force option is allowed or not.|
-| carbon.cdc.minmax.pruning.enabled | false | This parameter defines whether the min max pruning to be performed on the target table based on the source data. It will be useful when data is not sparse across target table which results in better pruning.|
-| carbon.blocklet.size | 64 MB | Carbondata file consists of blocklets which further consists of column pages. As per the latest V3 format, the default size of a blocklet is 64 MB. It is recommended not to change this value except for some specific use case. |
-| carbon.date.format | yyyy-MM-dd | This property specifies the format which is used for parsing the incoming date field values. |
-| carbon.lock.class | (none) | This specifies the implementation of ICarbonLock interface to be used for acquiring the locks in case of concurrent operations |
-| carbon.data.file.version | V3 | This specifies carbondata file format version. Carbondata file format has evolved with time from V1 to V3 in terms of metadata storage and IO level pruning capabilities. You can find more details [here](https://carbondata.apache.org/file-structure-of-carbondata.html#carbondata-file-format). |
-| spark.carbon.hive.schema.store | false | Carbondata currently supports 2 different types of metastores for storing schemas. This property specifies if Hive metastore is to be used for storing and retrieving table schemas |
+| carbon.lock.retries | 3 | CarbonData ensures consistency of operations by blocking certain operations from running in parallel. In order to block the operations from running in parallel, lock is obtained on the table. This configuration specifies the maximum number of retries to obtain the lock for any operations other than load. **NOTE:** Data manupulation operations like Compaction,UPDATE,DELETE  or LOADING,UPDATE,DELETE are not allowed to run in parallel. How ever data loading can h [...]
+| carbon.lock.retry.timeout.sec | 5 | Specifies the interval between the retries to obtain the lock for any operation other than load. **NOTE:** Refer to ***carbon.lock.retries*** for understanding why CarbonData uses locks for operations.                                                                                                                                                                                                                                                              [...]
+| carbon.fs.custom.file.provider | None | To support FileTypeInterface for configuring custom CarbonFile implementation to work with custom FileSystem.                                                                                                                                                                                                                                                                                                                                                      [...]
+| carbon.timeseries.first.day.of.week | SUNDAY | This parameter configures which day of the week to be considered as first day of the week. Because first day of the week will be different in different parts of the world.                                                                                                                                                                                                                                                                                 [...]
+| carbon.enable.tablestatus.backup | false | In cloud object store scenario, overwriting table status file is not an atomic operation since it uses rename API. Thus, it is possible that table status is corrupted if process crashed when overwriting the table status file. To protect from file corruption, user can enable this property.                                                                                                                                                               [...]
+| carbon.trash.retention.days | 7 | This parameter specifies the number of days after which the timestamp based subdirectories are expired in the trash folder. Allowed Min value = 0, Allowed Max Value = 365 days                                                                                                                                                                                                                                                                                          [...]
+| carbon.clean.file.force.allowed | false | This parameter specifies if the clean files operation with force option is allowed or not.                                                                                                                                                                                                                                                                                                                                                                       [...]
+| carbon.enable.multi.version.table.status | false | This property when enabled, allows creating multi-version table status files, which can be used to recover transaction metadata if the current version tablestatus file is lost. Running clean files with FORCE option will delete old versioned table status files                                                                                                                                                                                     [...]
+| carbon.cdc.minmax.pruning.enabled | false | This parameter defines whether the min max pruning to be performed on the target table based on the source data. It will be useful when data is not sparse across target table which results in better pruning.                                                                                                                                                                                                                                                [...]
+| carbon.blocklet.size | 64 MB | Carbondata file consists of blocklets which further consists of column pages. As per the latest V3 format, the default size of a blocklet is 64 MB. It is recommended not to change this value except for some specific use case.                                                                                                                                                                                                                                           [...]
+| carbon.date.format | yyyy-MM-dd | This property specifies the format which is used for parsing the incoming date field values.                                                                                                                                                                                                                                                                                                                                                                             [...]
+| carbon.lock.class | (none) | This specifies the implementation of ICarbonLock interface to be used for acquiring the locks in case of concurrent operations                                                                                                                                                                                                                                                                                                                                                [...]
+| carbon.data.file.version | V3 | This specifies carbondata file format version. Carbondata file format has evolved with time from V1 to V3 in terms of metadata storage and IO level pruning capabilities. You can find more details [here](https://carbondata.apache.org/file-structure-of-carbondata.html#carbondata-file-format).                                                                                                                                                                        [...]
+| spark.carbon.hive.schema.store | false | Carbondata currently supports 2 different types of metastores for storing schemas. This property specifies if Hive metastore is to be used for storing and retrieving table schemas                                                                                                                                                                                                                                                                               [...]
 
 ## Data Loading Configuration
 
diff --git a/docs/faq.md b/docs/faq.md
index 977fb7efcc..cde3e9e4da 100644
--- a/docs/faq.md
+++ b/docs/faq.md
@@ -30,6 +30,7 @@
 * [How to check LRU cache memory footprint?](#how-to-check-lru-cache-memory-footprint)
 * [How to deal with the trailing task in query?](#How-to-deal-with-the-trailing-task-in-query)
 * [How to manage hybrid file format in carbondata table?](#How-to-manage-hybrid-file-format-in-carbondata-table)
+* [How to recover table status file if lost?](#How-to-recover-table-status-file-if-lost)
 
 # TroubleShooting
 
@@ -465,3 +466,19 @@ Note : Refrain from using "mvn clean package" without specifying the profile.
   **Procedure**
 
   Drop that particular index using Drop Index command so as to clear the stale folders.
+
+## How to recover Table status file if lost
+When `carbon.enable.multi.version.table.status` property if enabled, table will have multiple versions of table status files in the metadata directory. If the current version file is lost, then user can run `TableStatusRecovery` tool to recover the table status file.
+
+**Example:**
+
+```
+TableStatusRecovery.main(args) --> args is of length two: 1. Database Name 2. Table Name
+```
+
+**Note:** 
+
+TableStatus Recovery tool cannot recover table status version files for the below two scenarios
+1. After compaction, if table status file is lost, cannot recover compacted commit transaction, as the lost version file only has merged load details.
+2. After Delete segment by Id/Date, if table status file is lost, cannot recover deleted segment commit transaction, as the lost version file only has the segment status as deleted.
+3. Table status recovery on materialized view table is not supported.
\ No newline at end of file
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonOutputCommitter.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonOutputCommitter.java
index 02c2d4ccd1..8543eafa36 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonOutputCommitter.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonOutputCommitter.java
@@ -236,8 +236,8 @@ public class CarbonOutputCommitter extends FileOutputCommitter {
         (context.getConfiguration().get(CarbonTableOutputFormat.UPDATE_TIMESTAMP) != null);
     if (updateTime != null) {
       CarbonUpdateUtil.updateTableMetadataStatus(Collections.singleton(loadModel.getSegment()),
-          carbonTable, updateTime, true,
-          isUpdateStatusFileUpdateRequired, segmentDeleteList);
+          carbonTable, updateTime, true, isUpdateStatusFileUpdateRequired,
+          segmentDeleteList, carbonTable.getTableStatusVersion());
     }
   }
 
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
index f24a2f1903..fe9dc1e3cf 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
@@ -134,7 +134,8 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
         throw new IOException(e);
       }
     }
-    this.readCommittedScope = getReadCommitted(job, carbonTable.getAbsoluteTableIdentifier());
+    this.readCommittedScope = getReadCommitted(job, carbonTable.getAbsoluteTableIdentifier(),
+        carbonTable.getTableStatusVersion());
     LoadMetadataDetails[] loadMetadataDetails = readCommittedScope.getSegmentList();
     String updateDeltaVersion = job.getConfiguration().get(UPDATE_DELTA_VERSION);
     SegmentUpdateStatusManager updateStatusManager;
@@ -150,7 +151,7 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
     // get all valid segments and set them into the configuration
     SegmentStatusManager segmentStatusManager =
         new SegmentStatusManager(carbonTable.getAbsoluteTableIdentifier(),
-            readCommittedScope.getConfiguration());
+            readCommittedScope.getConfiguration(), carbonTable.getTableStatusVersion());
     SegmentStatusManager.ValidAndInvalidSegmentsInfo segments = segmentStatusManager
         .getValidAndInvalidSegments(carbonTable.isMV(), loadMetadataDetails,
             this.readCommittedScope);
@@ -446,15 +447,16 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
 
     AbsoluteTableIdentifier identifier = table.getAbsoluteTableIdentifier();
 
-    ReadCommittedScope readCommittedScope = getReadCommitted(job, identifier);
+    ReadCommittedScope readCommittedScope =
+        getReadCommitted(job, table.getAbsoluteTableIdentifier(), table.getTableStatusVersion());
     LoadMetadataDetails[] loadMetadataDetails = readCommittedScope.getSegmentList();
 
     SegmentUpdateStatusManager updateStatusManager = new SegmentUpdateStatusManager(
         table, loadMetadataDetails);
     SegmentStatusManager.ValidAndInvalidSegmentsInfo allSegments =
-        new SegmentStatusManager(identifier, readCommittedScope.getConfiguration())
-            .getValidAndInvalidSegments(table.isMV(), loadMetadataDetails,
-                readCommittedScope);
+        new SegmentStatusManager(identifier, readCommittedScope.getConfiguration(),
+            table.getTableStatusVersion()).getValidAndInvalidSegments(table.isMV(),
+            loadMetadataDetails, readCommittedScope);
     Map<String, Long> blockRowCountMapping = new HashMap<>();
     Map<String, Long> segmentAndBlockCountMapping = new HashMap<>();
     Map<String, String> blockToSegmentMapping = new HashMap<>();
@@ -566,12 +568,13 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
     return blockMappingVO;
   }
 
-  public ReadCommittedScope getReadCommitted(JobContext job, AbsoluteTableIdentifier identifier)
-      throws IOException {
+  public ReadCommittedScope getReadCommitted(JobContext job, AbsoluteTableIdentifier identifier,
+      String tblStatusVersion) throws IOException {
     if (readCommittedScope == null) {
       ReadCommittedScope readCommittedScope;
       if (job.getConfiguration().getBoolean(CARBON_TRANSACTIONAL_TABLE, true)) {
-        readCommittedScope = new TableStatusReadCommittedScope(identifier, job.getConfiguration());
+        readCommittedScope =
+            new TableStatusReadCommittedScope(identifier, job.getConfiguration(), tblStatusVersion);
       } else {
         readCommittedScope = getReadCommittedScope(job.getConfiguration());
         if (readCommittedScope == null) {
diff --git a/index/bloom/src/main/java/org/apache/carbondata/index/bloom/BloomCoarseGrainIndexFactory.java b/index/bloom/src/main/java/org/apache/carbondata/index/bloom/BloomCoarseGrainIndexFactory.java
index c80da44ec0..e985ab1e74 100644
--- a/index/bloom/src/main/java/org/apache/carbondata/index/bloom/BloomCoarseGrainIndexFactory.java
+++ b/index/bloom/src/main/java/org/apache/carbondata/index/bloom/BloomCoarseGrainIndexFactory.java
@@ -377,7 +377,8 @@ public class BloomCoarseGrainIndexFactory extends IndexFactory<CoarseGrainIndex>
   @Override
   public void deleteIndexData() {
     SegmentStatusManager ssm =
-        new SegmentStatusManager(getCarbonTable().getAbsoluteTableIdentifier());
+        new SegmentStatusManager(getCarbonTable().getAbsoluteTableIdentifier(),
+            getCarbonTable().getTableStatusVersion());
     try {
       List<Segment> validSegments =
           ssm.getValidAndInvalidSegments(getCarbonTable().isMV()).getValidSegments();
diff --git a/index/lucene/src/main/java/org/apache/carbondata/index/lucene/LuceneIndexFactoryBase.java b/index/lucene/src/main/java/org/apache/carbondata/index/lucene/LuceneIndexFactoryBase.java
index 892ec67a70..15067f5261 100644
--- a/index/lucene/src/main/java/org/apache/carbondata/index/lucene/LuceneIndexFactoryBase.java
+++ b/index/lucene/src/main/java/org/apache/carbondata/index/lucene/LuceneIndexFactoryBase.java
@@ -174,7 +174,8 @@ abstract class LuceneIndexFactoryBase<T extends Index> extends IndexFactory<T> {
    * this method will delete the index folders during drop index
    */
   private void deleteIndex() throws MalformedIndexCommandException {
-    SegmentStatusManager ssm = new SegmentStatusManager(tableIdentifier);
+    SegmentStatusManager ssm =
+        new SegmentStatusManager(tableIdentifier, getCarbonTable().getTableStatusVersion());
     try {
       List<Segment> validSegments =
           ssm.getValidAndInvalidSegments(getCarbonTable().isMV()).getValidSegments();
diff --git a/index/secondary-index/src/test/scala/org/apache/carbondata/spark/testsuite/secondaryindex/TestCreateIndexWithLoadAndCompaction.scala b/index/secondary-index/src/test/scala/org/apache/carbondata/spark/testsuite/secondaryindex/TestCreateIndexWithLoadAndCompaction.scala
index cdd57aff8d..9e57aab82e 100644
--- a/index/secondary-index/src/test/scala/org/apache/carbondata/spark/testsuite/secondaryindex/TestCreateIndexWithLoadAndCompaction.scala
+++ b/index/secondary-index/src/test/scala/org/apache/carbondata/spark/testsuite/secondaryindex/TestCreateIndexWithLoadAndCompaction.scala
@@ -217,7 +217,7 @@ class TestCreateIndexWithLoadAndCompaction extends QueryTest with BeforeAndAfter
         .asInstanceOf[CarbonRelation].carbonTable
       // read load metadata details
       val loadDetails: Array[LoadMetadataDetails] = SegmentStatusManager
-        .readLoadMetadata(CarbonTablePath.getMetadataPath(indexCarbonTable.getTablePath))
+        .readLoadMetadata(CarbonTablePath.getMetadataPath(indexCarbonTable.getTablePath), indexCarbonTable.getTableStatusVersion)
       assert(loadDetails.length == 3)
       // compacted status segment should only be 2
       val compactedStatusSegments = loadDetails
@@ -305,8 +305,10 @@ class TestCreateIndexWithLoadAndCompaction extends QueryTest with BeforeAndAfter
     val mock = mockreadSegmentList()
     sql("CLEAN FILES FOR TABLE table1 options('force'='true')")
     mock.tearDown()
-    val details = SegmentStatusManager.readLoadMetadata(CarbonEnv
-        .getCarbonTable(Some("default"), "idx1")(sqlContext.sparkSession).getMetadataPath)
+    val table = CarbonEnv
+      .getCarbonTable(Some("default"), "idx1")(sqlContext.sparkSession)
+    val details = SegmentStatusManager.readLoadMetadata(table.getMetadataPath,
+      table.getTableStatusVersion)
     assert(SegmentStatusManager.countInvisibleSegments(details, 4) == 1)
     checkSegmentList(4)
     CarbonProperties.getInstance()
diff --git a/index/secondary-index/src/test/scala/org/apache/carbondata/spark/testsuite/secondaryindex/TestSIWithSecondaryIndex.scala b/index/secondary-index/src/test/scala/org/apache/carbondata/spark/testsuite/secondaryindex/TestSIWithSecondaryIndex.scala
index 03feffde9a..821f7485c3 100644
--- a/index/secondary-index/src/test/scala/org/apache/carbondata/spark/testsuite/secondaryindex/TestSIWithSecondaryIndex.scala
+++ b/index/secondary-index/src/test/scala/org/apache/carbondata/spark/testsuite/secondaryindex/TestSIWithSecondaryIndex.scala
@@ -306,7 +306,8 @@ class TestSIWithSecondaryIndex extends QueryTest with BeforeAndAfterAll {
     sql("create index ud_index1 on table uniqdata (workgroupcategoryname) AS 'carbondata'")
     val indexTable = CarbonEnv.getCarbonTable(Some("default"), "ud_index1")(sqlContext.sparkSession)
     val carbontable = CarbonEnv.getCarbonTable(Some("default"), "uniqdata")(sqlContext.sparkSession)
-    val details = SegmentStatusManager.readLoadMetadata(indexTable.getMetadataPath)
+    val details = SegmentStatusManager.readLoadMetadata(indexTable.getMetadataPath,
+      indexTable.getTableStatusVersion)
     val failSegments = List("3", "4")
     sql(s"""set carbon.si.repair.limit = 2""")
     var loadMetadataDetailsList = Array[LoadMetadataDetails]()
@@ -320,10 +321,8 @@ class TestSIWithSecondaryIndex extends QueryTest with BeforeAndAfterAll {
       }
     }
 
-    SegmentStatusManager.writeLoadDetailsIntoFile(
-      indexTable.getMetadataPath + CarbonCommonConstants.FILE_SEPARATOR +
-      CarbonTablePath.TABLE_STATUS_FILE,
-      loadMetadataDetailsList)
+    SegmentStatusManager.writeLoadDetailsIntoFile(CarbonTablePath.getTableStatusFilePath(
+      indexTable.getTablePath, indexTable.getTableStatusVersion), loadMetadataDetailsList)
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_CLEAN_FILES_FORCE_ALLOWED, "true")
     sql(s"CLEAN FILES FOR TABLE ud_index1  OPTIONS('stale_inprogress'='true','force'='true')")
diff --git a/index/secondary-index/src/test/scala/org/apache/spark/util/TestCarbonSegmentUtil.scala b/index/secondary-index/src/test/scala/org/apache/spark/util/TestCarbonSegmentUtil.scala
index 55529b4e3d..f71c611304 100644
--- a/index/secondary-index/src/test/scala/org/apache/spark/util/TestCarbonSegmentUtil.scala
+++ b/index/secondary-index/src/test/scala/org/apache/spark/util/TestCarbonSegmentUtil.scala
@@ -147,7 +147,8 @@ class TestCarbonSegmentUtil extends QueryTest {
     createTable(tableName)
     val carbonTable = CarbonEnv
       .getCarbonTable(Option(databaseName), tableName)(SparkTestQueryExecutor.spark)
-    val loadMetadataDetails = SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath)
+    val loadMetadataDetails = SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath,
+      carbonTable.getTableStatusVersion)
     val expected = SecondaryIndexUtil
       .getMergedLoadName(loadMetadataDetails.toList.asJava)
     assert(expected.equalsIgnoreCase("Segment_0.1"))
@@ -162,7 +163,8 @@ class TestCarbonSegmentUtil extends QueryTest {
     sql(s"INSERT INTO $tableName SELECT 'c1v1', '1', 'c3v1'")
     val carbonTable = CarbonEnv
       .getCarbonTable(Option(databaseName), tableName)(SparkTestQueryExecutor.spark)
-    val loadMetadataDetails = SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath)
+    val loadMetadataDetails = SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath,
+      carbonTable.getTableStatusVersion)
     val exception = intercept[UnsupportedOperationException] {
       SecondaryIndexUtil
         .getMergedLoadName(loadMetadataDetails.toList.asJava)
@@ -179,7 +181,6 @@ class TestCarbonSegmentUtil extends QueryTest {
     createTable(tableName)
     val carbonTable = CarbonEnv
       .getCarbonTable(Option(databaseName), tableName)(SparkTestQueryExecutor.spark)
-    val loadMetadataDetails = SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath)
     val segments: util.List[LoadMetadataDetails] = new util.ArrayList[LoadMetadataDetails]()
     val load1 = new LoadMetadataDetails()
     load1.setLoadName("1")
diff --git a/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoInsertIntoTableTestCase.scala b/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoInsertIntoTableTestCase.scala
index 03c6e1c011..be7695f169 100644
--- a/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoInsertIntoTableTestCase.scala
+++ b/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoInsertIntoTableTestCase.scala
@@ -208,7 +208,8 @@ class PrestoInsertIntoTableTestCase
     val absoluteTableIdentifier: AbsoluteTableIdentifier = getAbsoluteIdentifier("testdb",
       "testtable")
     val carbonTable = SchemaReader.readCarbonTableFromStore(absoluteTableIdentifier)
-    val ssm = new SegmentStatusManager(carbonTable.getAbsoluteTableIdentifier)
+    val ssm = new SegmentStatusManager(carbonTable.getAbsoluteTableIdentifier,
+      carbonTable.getTableStatusVersion)
     ssm.getValidAndInvalidSegments.getValidSegments.asScala.foreach { segment =>
       val loadMetadataDetails = segment.getLoadMetadataDetails
       assert(loadMetadataDetails.getSegmentFile != null)
diff --git a/integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/SparkVersionAdapter.scala b/integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/SparkVersionAdapter.scala
index cfc580c055..e5cb5e0b13 100644
--- a/integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/SparkVersionAdapter.scala
+++ b/integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/SparkVersionAdapter.scala
@@ -440,6 +440,11 @@ trait SparkVersionAdapter {
           val newProperties = tableDesc.properties. +("hasexternalkeyword" -> "false")
           val updatedTableDesc = tableDesc.copy(properties = newProperties)
           CreateTable(updatedTableDesc, mode, query)
+        } else if (tableDesc.storage.properties.contains("latestversion")) {
+          val newProperties = tableDesc.storage
+            .properties.filterNot(_._1.equalsIgnoreCase("latestversion"))
+          val updatedStorage = tableDesc.storage.copy(properties = newProperties)
+          CreateTable(tableDesc.copy(storage = updatedStorage), mode, query)
         } else {
           create
         }
diff --git a/integration/spark/src/main/java/org/apache/spark/sql/secondaryindex/load/CarbonInternalLoaderUtil.java b/integration/spark/src/main/java/org/apache/spark/sql/secondaryindex/load/CarbonInternalLoaderUtil.java
index 5c233ed7f5..77992208e8 100644
--- a/integration/spark/src/main/java/org/apache/spark/sql/secondaryindex/load/CarbonInternalLoaderUtil.java
+++ b/integration/spark/src/main/java/org/apache/spark/sql/secondaryindex/load/CarbonInternalLoaderUtil.java
@@ -38,6 +38,7 @@ import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
 import org.apache.carbondata.core.statusmanager.SegmentStatus;
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
 import org.apache.carbondata.processing.util.CarbonLoaderUtil;
 
 import org.apache.log4j.Logger;
@@ -85,11 +86,14 @@ public class CarbonInternalLoaderUtil {
    */
   public static boolean recordLoadMetadata(List<LoadMetadataDetails> newLoadMetadataDetails,
       List<String> validSegments, CarbonTable carbonTable, List<CarbonTable> indexCarbonTables,
-      String databaseName, String tableName) {
+      CarbonLoadModel loadModel, String tableName, SegmentStatus segmentStatus,
+      boolean isRebuiltSegments) {
+    String databaseName = loadModel.getDatabaseName();
     boolean status = false;
     String metaDataFilepath = carbonTable.getMetadataPath();
     AbsoluteTableIdentifier absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier();
-    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier);
+    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier,
+        loadModel.getLatestTableStatusWriteVersion());
     ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
     try {
       int retryCount = CarbonLockUtil
@@ -107,8 +111,15 @@ public class CarbonInternalLoaderUtil {
           return false;
         }
 
+        loadModel.setLatestTableStatusWriteVersion(String.valueOf(System.currentTimeMillis()));
+        String tblStatusVersion = loadModel.getLatestTableStatusWriteVersion();
+        if (segmentStatus == SegmentStatus.INSERT_IN_PROGRESS
+            || segmentStatus == SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS || isRebuiltSegments) {
+          tblStatusVersion = carbonTable.getTableStatusVersion();
+        }
+
         LoadMetadataDetails[] currentLoadMetadataDetails = SegmentStatusManager.readLoadMetadata(
-            metaDataFilepath);
+            metaDataFilepath, tblStatusVersion);
 
         List<LoadMetadataDetails> updatedLoadMetadataDetails = new ArrayList<>(
             CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
@@ -166,13 +177,14 @@ public class CarbonInternalLoaderUtil {
                     indexTable, newSegmentDetailsListForIndexTable);
 
             SegmentStatusManager.writeLoadDetailsIntoFile(
-                CarbonTablePath.getTableStatusFilePath(indexTable.getTablePath()),
+                CarbonTablePath.getTableStatusFilePath(indexTable.getTablePath(),
+                    loadModel.getLatestTableStatusWriteVersion()),
                 indexTableDetailsList.toArray(new LoadMetadataDetails[0]));
           }
         } else if (carbonTable.isIndexTable()) {
           SegmentStatusManager.writeLoadDetailsIntoFile(
-              metaDataFilepath + CarbonCommonConstants.FILE_SEPARATOR
-                  + CarbonTablePath.TABLE_STATUS_FILE,
+              CarbonTablePath.getTableStatusFilePath(carbonTable.getTablePath(),
+                  loadModel.getLatestTableStatusWriteVersion()),
               updatedLoadMetadataDetails.toArray(new LoadMetadataDetails[0]));
         }
         status = true;
@@ -219,7 +231,7 @@ public class CarbonInternalLoaderUtil {
     boolean isIndexTableSegmentsCompacted = false;
     if (null != currentIndexTable) {
       LoadMetadataDetails[] existingLoadMetaDataDetails = SegmentStatusManager.readLoadMetadata(
-          currentIndexTable.getMetadataPath());
+          currentIndexTable.getMetadataPath(), currentIndexTable.getTableStatusVersion());
       for (LoadMetadataDetails existingLoadMetaDataDetail : existingLoadMetaDataDetails) {
         for (LoadMetadataDetails newLoadMetadataDetail : newLoadMetadataDetails) {
           if (existingLoadMetaDataDetail.getLoadName().equalsIgnoreCase(
@@ -249,7 +261,7 @@ public class CarbonInternalLoaderUtil {
     boolean tableStatusUpdateStatus = false;
     List<String> loadMergeList = new ArrayList<>(Arrays.asList(loadsToMerge));
     SegmentStatusManager segmentStatusManager = new SegmentStatusManager(
-        indexCarbonTable.getAbsoluteTableIdentifier());
+        indexCarbonTable.getAbsoluteTableIdentifier(), indexCarbonTable.getTableStatusVersion());
 
     ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
 
@@ -258,7 +270,7 @@ public class CarbonInternalLoaderUtil {
         LOGGER.info("Acquired lock for the table " + indexCarbonTable.getDatabaseName() + "."
             + indexCarbonTable.getTableName() + " for table status update ");
         LoadMetadataDetails[] loadDetails = SegmentStatusManager.readLoadMetadata(
-            indexCarbonTable.getMetadataPath());
+            indexCarbonTable.getMetadataPath(), indexCarbonTable.getTableStatusVersion());
 
         long modificationOrDeletionTimeStamp = CarbonUpdateUtil.readCurrentTime();
         for (LoadMetadataDetails loadDetail : loadDetails) {
@@ -306,7 +318,8 @@ public class CarbonInternalLoaderUtil {
         List<LoadMetadataDetails> updatedDetailsList = new ArrayList<>(Arrays.asList(loadDetails));
 
         SegmentStatusManager.writeLoadDetailsIntoFile(
-            CarbonTablePath.getTableStatusFilePath(indexCarbonTable.getTablePath()),
+            CarbonTablePath.getTableStatusFilePath(indexCarbonTable.getTablePath(),
+                indexCarbonTable.getTableStatusVersion()),
             updatedDetailsList.toArray(new LoadMetadataDetails[0]));
         tableStatusUpdateStatus = true;
       } else {
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/api/CarbonStore.scala b/integration/spark/src/main/scala/org/apache/carbondata/api/CarbonStore.scala
index 1234a5ee42..9a429f182f 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/api/CarbonStore.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/api/CarbonStore.scala
@@ -27,7 +27,8 @@ import scala.util.control.Breaks.{break, breakable}
 
 import com.google.gson.Gson
 import org.apache.commons.lang3.StringUtils
-import org.apache.spark.sql.CarbonToSparkAdapter
+import org.apache.spark.sql.{CarbonToSparkAdapter, SparkSession}
+import org.apache.spark.sql.hive.CarbonHiveIndexMetadataUtil
 
 import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.common.logging.LogServiceFactory
@@ -50,13 +51,14 @@ object CarbonStore {
   def readSegments(
       tablePath: String,
       showHistory: Boolean,
-      limit: Option[Int]): Array[LoadMetadataDetails] = {
+      limit: Option[Int],
+      tableStatusVersion: String): Array[LoadMetadataDetails] = {
     val metaFolder = CarbonTablePath.getMetadataPath(tablePath)
     var segmentsMetadataDetails = if (showHistory) {
-      SegmentStatusManager.readLoadMetadata(metaFolder) ++
+      SegmentStatusManager.readLoadMetadata(metaFolder, tableStatusVersion) ++
       SegmentStatusManager.readLoadHistoryMetadata(metaFolder)
     } else {
-      SegmentStatusManager.readLoadMetadata(metaFolder)
+      SegmentStatusManager.readLoadMetadata(metaFolder, tableStatusVersion)
     }
     if (!showHistory) {
       segmentsMetadataDetails = segmentsMetadataDetails
@@ -295,16 +297,26 @@ object CarbonStore {
       loadIds: Seq[String],
       dbName: String,
       tableName: String,
-      carbonTable: CarbonTable): Unit = {
+      carbonTable: CarbonTable,
+      session: SparkSession): Unit = {
 
     validateLoadIds(loadIds)
 
     val path = carbonTable.getMetadataPath
 
     try {
-      val invalidLoadIds = SegmentStatusManager.updateDeletionStatus(
-        carbonTable.getAbsoluteTableIdentifier, loadIds.asJava, path).asScala
+      val tuple = SegmentStatusManager.updateDeletionStatus(
+        carbonTable.getAbsoluteTableIdentifier, loadIds.asJava,
+        path, carbonTable.getTableStatusVersion).asScala
+      val invalidLoadIds = tuple("invalidLoadIds").asScala
       if (invalidLoadIds.isEmpty) {
+        val tblStatusWriteVersion = if (tuple.contains("tblStatusWriteVersion")) {
+          tuple("tblStatusWriteVersion").get(0)
+        } else {
+          ""
+        }
+        CarbonHiveIndexMetadataUtil.updateTableStatusVersion(carbonTable,
+          session, tblStatusWriteVersion)
         LOGGER.info(s"Delete segment by Id is successful for $dbName.$tableName.")
       } else {
         sys.error(s"Delete segment by Id is failed. Invalid ID is: ${invalidLoadIds.mkString(",")}")
@@ -321,19 +333,29 @@ object CarbonStore {
       timestamp: String,
       dbName: String,
       tableName: String,
-      carbonTable: CarbonTable): Unit = {
+      carbonTable: CarbonTable,
+      sparkSession: SparkSession): Unit = {
 
     val time = validateTimeFormat(timestamp)
     val path = carbonTable.getMetadataPath
 
     try {
-      val invalidLoadTimestamps =
+      val tuple =
         SegmentStatusManager.updateDeletionStatus(
           carbonTable.getAbsoluteTableIdentifier,
           timestamp,
           path,
-          time).asScala
+          time,
+          carbonTable.getTableStatusVersion).asScala
+      val invalidLoadTimestamps = tuple("invalidLoadTimestamps")
       if (invalidLoadTimestamps.isEmpty) {
+        val tblStatusWriteVersion = if (tuple.contains("tblStatusWriteVersion")) {
+          tuple("tblStatusWriteVersion").get(0)
+        } else {
+          ""
+        }
+        CarbonHiveIndexMetadataUtil.updateTableStatusVersion(carbonTable,
+          sparkSession, tblStatusWriteVersion)
         LOGGER.info(s"Delete segment by date is successful for $dbName.$tableName.")
       } else {
         sys.error("Delete segment by date is failed. No matching segment found.")
@@ -349,11 +371,11 @@ object CarbonStore {
       dbName: String,
       tableName: String,
       storePath: String,
-      segmentId: String): Boolean = {
+      segmentId: String,
+      tableStatusVersion: String): Boolean = {
     val identifier = AbsoluteTableIdentifier.from(storePath, dbName, tableName, tableName)
     val validAndInvalidSegments: SegmentStatusManager.ValidAndInvalidSegmentsInfo = new
-        SegmentStatusManager(
-          identifier).getValidAndInvalidSegments
+        SegmentStatusManager(identifier, tableStatusVersion).getValidAndInvalidSegments
     validAndInvalidSegments.getValidSegments.contains(segmentId)
   }
 
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/index/IndexRebuildRDD.scala b/integration/spark/src/main/scala/org/apache/carbondata/index/IndexRebuildRDD.scala
index d6e62bcb1a..15a9e557cd 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/index/IndexRebuildRDD.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/index/IndexRebuildRDD.scala
@@ -71,7 +71,8 @@ object IndexRebuildRDD {
       schema: IndexSchema
   ): Unit = {
     val tableIdentifier = carbonTable.getAbsoluteTableIdentifier
-    val segmentStatusManager = new SegmentStatusManager(tableIdentifier)
+    val segmentStatusManager = new SegmentStatusManager(tableIdentifier,
+      carbonTable.getTableStatusVersion)
     val validAndInvalidSegments = segmentStatusManager
       .getValidAndInvalidSegments(carbonTable.isMV)
     val validSegments = validAndInvalidSegments.getValidSegments
@@ -428,7 +429,9 @@ class IndexRebuildRDD[K, V](
     // make the partitions based on block path so that all the CarbonInputSplits in a
     // MultiBlockSplit are used for bloom reading. This means 1 task for 1 shard(unique block path).
     val splits = format.getSplits(job)
-    readCommittedScope = format.getReadCommitted(job, null)
+    val carbonTable = CarbonTable.buildFromTableInfo(getTableInfo)
+    readCommittedScope = format.getReadCommitted(
+      job, carbonTable.getAbsoluteTableIdentifier, carbonTable.getTableStatusVersion)
     splits.asScala
       .map(_.asInstanceOf[CarbonInputSplit])
       .groupBy(p => (p.getSegmentId, p.taskId, p.getBlockPath))
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/index/secondary/SecondaryIndex.java b/integration/spark/src/main/scala/org/apache/carbondata/index/secondary/SecondaryIndex.java
index 567ef2120b..1cdc02384d 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/index/secondary/SecondaryIndex.java
+++ b/integration/spark/src/main/scala/org/apache/carbondata/index/secondary/SecondaryIndex.java
@@ -69,9 +69,9 @@ public class SecondaryIndex extends CoarseGrainIndex {
     positionReferenceInfo = model.getPositionReferenceInfo();
   }
 
-  public void validateSegmentList(String indexPath) {
+  public void validateSegmentList(String indexPath, String tableStatusVersion) {
     LoadMetadataDetails[] loadMetadataDetails = SegmentStatusManager
-        .readLoadMetadata(CarbonTablePath.getMetadataPath(indexPath));
+        .readLoadMetadata(CarbonTablePath.getMetadataPath(indexPath), tableStatusVersion);
     Set<String> validSISegments = new HashSet<>();
     for (LoadMetadataDetails loadMetadataDetail : loadMetadataDetails) {
       if (loadMetadataDetail.getSegmentStatus() == SegmentStatus.SUCCESS
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/index/secondary/SecondaryIndexFactory.java b/integration/spark/src/main/scala/org/apache/carbondata/index/secondary/SecondaryIndexFactory.java
index 24623e181a..eeeafa1a60 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/index/secondary/SecondaryIndexFactory.java
+++ b/integration/spark/src/main/scala/org/apache/carbondata/index/secondary/SecondaryIndexFactory.java
@@ -34,11 +34,13 @@ import org.apache.carbondata.core.features.TableOperation;
 import org.apache.carbondata.core.index.IndexFilter;
 import org.apache.carbondata.core.index.IndexInputSplit;
 import org.apache.carbondata.core.index.IndexMeta;
+import org.apache.carbondata.core.index.IndexStoreManager;
 import org.apache.carbondata.core.index.Segment;
 import org.apache.carbondata.core.index.dev.IndexBuilder;
 import org.apache.carbondata.core.index.dev.IndexWriter;
 import org.apache.carbondata.core.index.dev.cgindex.CoarseGrainIndex;
 import org.apache.carbondata.core.index.dev.cgindex.CoarseGrainIndexFactory;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.IndexSchema;
 import org.apache.carbondata.core.scan.filter.intf.ExpressionType;
@@ -102,8 +104,14 @@ public class SecondaryIndexFactory extends CoarseGrainIndexFactory {
         new SecondaryIndexModel(getIndexSchema().getIndexName(), segment.getSegmentNo(),
             allSegmentIds, positionReferenceInfo, segment.getConfiguration()));
     secondaryIndex.setDefaultIndexPrunedBlocklet(segment.getDefaultIndexPrunedBlocklets());
-    secondaryIndex.validateSegmentList(getCarbonTable().getTablePath()
-        .replace(getCarbonTable().getTableName(), getIndexSchema().getIndexName()));
+    String indexTablePath = getCarbonTable().getTablePath()
+        .replace(getCarbonTable().getTableName(), getIndexSchema().getIndexName());
+    AbsoluteTableIdentifier identifier =
+        AbsoluteTableIdentifier.from(indexTablePath, getCarbonTable().getDatabaseName(),
+            getIndexSchema().getIndexName());
+    String tblStatusVersion =
+        IndexStoreManager.getInstance().getCarbonTable(identifier).getTableStatusVersion();
+    secondaryIndex.validateSegmentList(indexTablePath, tblStatusVersion);
     indexes.add(secondaryIndex);
     return indexes;
   }
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/indexserver/DistributedRDDUtils.scala b/integration/spark/src/main/scala/org/apache/carbondata/indexserver/DistributedRDDUtils.scala
index 41db8f1935..3aefe81032 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/indexserver/DistributedRDDUtils.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/indexserver/DistributedRDDUtils.scala
@@ -393,7 +393,7 @@ object DistributedRDDUtils {
       if (indexServerEnabled && prePrimingEnabled) {
         LOGGER.info(s" Loading segments for the table: ${ carbonTable.getTableName } in the cache")
         val readCommittedScope = new TableStatusReadCommittedScope(AbsoluteTableIdentifier.from(
-          carbonTable.getTablePath), conf)
+          carbonTable.getTablePath), conf, carbonTable.getTableStatusVersion)
         val validSegments: Seq[Segment] = segmentId.map {
           segmentToPrime =>
             val loadDetailsForCurrentSegment = readCommittedScope
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/recovery/tablestatus/TableStatusRecovery.scala b/integration/spark/src/main/scala/org/apache/carbondata/recovery/tablestatus/TableStatusRecovery.scala
new file mode 100644
index 0000000000..1681befe07
--- /dev/null
+++ b/integration/spark/src/main/scala/org/apache/carbondata/recovery/tablestatus/TableStatusRecovery.scala
@@ -0,0 +1,233 @@
+/*
+ * 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.carbondata.recovery.tablestatus
+
+import java.util
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.SparkConf
+import org.apache.spark.sql.{CarbonEnv, SparkSession}
+import org.apache.spark.sql.util.SparkSQLUtil
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.filesystem.{CarbonFile, CarbonFileFilter}
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.SegmentFileStore
+import org.apache.carbondata.core.mutate.SegmentUpdateDetails
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatus, SegmentStatusManager, SegmentUpdateStatusManager}
+import org.apache.carbondata.core.util.CarbonUtil
+import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.spark.util.CarbonScalaUtil
+
+object TableStatusRecovery {
+  def main(args: Array[String]): Unit = {
+    // check the argument contains database name and tablename to recover table status file
+    assert(args.length == 2)
+    createCarbonSession()
+    val sparkSession = SparkSQLUtil.getSparkSession
+    val tableName = args(1)
+    val databaseName = args(0)
+    // get carbon table to start table status recovery
+    val carbonTable = try {
+      CarbonEnv.getCarbonTable(Some(databaseName), tableName)(sparkSession)
+    } catch {
+      case ex: Exception =>
+        throw ex
+    }
+
+    if (carbonTable.isMV) {
+      // not supported
+      throw new UnsupportedOperationException("Unsupported operation on Materialized view table")
+    }
+
+    /**
+     * 1. get the current table status version file name associated with carbon table
+     * 2. Check if the current table status version file exists
+     * 3. If does not exists, then read all the old table status version files and find the last
+     *    recent version file and get the load metadata details. For the lost load metadata,
+     *    read the segment files and table status update files to recover the lost
+     *    load metadata entry and add it to previous version load metadata details list.
+     * 4. Write the load metadata details list with version name as [Step:1]
+     * */
+    val tableStatusVersion = carbonTable.getTableStatusVersion
+    val tableStatusPath = CarbonTablePath.getTableStatusFilePath(carbonTable.getTablePath,
+      tableStatusVersion)
+    val tableStatusFile = FileFactory.getCarbonFile(
+      FileFactory.getUpdatedFilePath(tableStatusPath))
+    if (!tableStatusFile.exists()) {
+      // case where the current version table status file is lost, then get the previous table
+      // status version file and update it as the current table status version
+      val tableStatusFiles = CarbonScalaUtil.getTableStatusVersionFiles(carbonTable.getTablePath)
+      // read the segment files in the Metadata directory
+      val segmentFileDir = FileFactory.getCarbonFile(FileFactory.getUpdatedFilePath(
+        CarbonTablePath.getSegmentFilesLocation(carbonTable.getTablePath)))
+      val segmentFiles = segmentFileDir.listFiles()
+        .filter(segmentFile => segmentFile.getName.endsWith(CarbonTablePath.SEGMENT_EXT))
+        .toList
+      if (tableStatusFiles.isEmpty) {
+        if (segmentFiles.isEmpty) {
+          // no metadata found to recover table status file
+          throw new Exception(
+            "Segment Files does not exists to recover load metadata")
+        }
+      }
+      // prepare segment to latest timestamp version map. This is required, in case of drop
+      // partition, where there can be multiple segment files for same segment Id
+      val segToTimeStampMap = new util.HashMap[String, String]()
+      segmentFiles.foreach { segmentFile =>
+        val segFileName = segmentFile.getName
+        val segmentToTimestamp = segFileName.trim.split(CarbonCommonConstants.UNDERSCORE).toList
+        if (!segToTimeStampMap.containsKey(segmentToTimestamp.head)) {
+          segToTimeStampMap.put(segmentToTimestamp.head, segmentToTimestamp.last)
+        } else {
+          val timeStamp = segToTimeStampMap.get(segmentToTimestamp.head)
+          if (timeStamp <= segmentToTimestamp.last) {
+            segToTimeStampMap.put(segmentToTimestamp.head, segmentToTimestamp.last)
+          }
+        }
+      }
+      // iterate the available table status version files and find the most recent table status
+      // version file
+      val latestTableStatusVersionStr = CarbonScalaUtil.getLatestTblStatusVersionBasedOnTimestamp(
+        tableStatusFiles)
+
+      // read the load metadata details with the identified table status version file
+      var loadMetaDetails = SegmentStatusManager.readLoadMetadata(CarbonTablePath.getMetadataPath(
+        carbonTable.getTablePath), latestTableStatusVersionStr).toList
+
+      var updateMetaDetails: Array[SegmentUpdateDetails] = Array.empty
+
+      val tableUpdateStatusFiles = FileFactory.getCarbonFile(CarbonTablePath.getMetadataPath(
+        carbonTable.getTablePath)).listFiles(new CarbonFileFilter {
+        override def accept(file: CarbonFile): Boolean = {
+          file.getName.startsWith(CarbonCommonConstants.TABLEUPDATESTATUS_FILENAME)
+        }
+      })
+
+      // if table has table update status files, iterate and identify the latest table status
+      // update file
+      if (tableUpdateStatusFiles.nonEmpty) {
+        var latestTableUpdateStatusVersion = 0L
+        tableUpdateStatusFiles.foreach { tableStatusFile =>
+          val updateVersionTimeStamp = tableStatusFile.getName
+            .substring(tableStatusFile.getName.indexOf(CarbonCommonConstants.HYPHEN) + 1,
+              tableStatusFile.getName.length).toLong
+          if (latestTableUpdateStatusVersion <= updateVersionTimeStamp) {
+            latestTableUpdateStatusVersion = updateVersionTimeStamp
+          }
+        }
+        updateMetaDetails = SegmentUpdateStatusManager.readLoadMetadata(
+          CarbonCommonConstants.TABLEUPDATESTATUS_FILENAME + CarbonCommonConstants.HYPHEN +
+          latestTableUpdateStatusVersion.toString, carbonTable.getTablePath)
+      }
+
+      // check which segment is missing from lost table status version
+      val missedLoadMetaDetails: util.List[LoadMetadataDetails] =
+        new util.ArrayList[LoadMetadataDetails]()
+      segToTimeStampMap.asScala.foreach { segmentFileEntry =>
+        val segmentFileName = segmentFileEntry._1 + CarbonCommonConstants.UNDERSCORE +
+                              segmentFileEntry._2
+        val segmentId = segmentFileEntry._1
+        val segmentUpdateDetail = updateMetaDetails
+          .filter(_.getSegmentName.equalsIgnoreCase(segmentId))
+        // check if the segment Id from segment file entry exists in load metadata details list.
+        // If does not exist, or if the segment file mapped to the load metadata entry and the
+        // latest segment file timestamp is not same, then prepare new load metadata.
+        if ((!loadMetaDetails.exists(_.getLoadName.equalsIgnoreCase(segmentId))
+             || !loadMetaDetails.filter(_.getLoadName.equalsIgnoreCase(segmentId))
+          .head.getSegmentFile.equalsIgnoreCase(segmentFileName)) &&
+            !segmentId.contains(CarbonCommonConstants.POINT)) {
+          val segFilePath = CarbonTablePath.getSegmentFilePath(
+            carbonTable.getTablePath, segmentFileName)
+          // read segment file and prepare load metadata
+          val segmentFile = SegmentFileStore.readSegmentFile(segFilePath)
+          val loadMetadataDetail = new LoadMetadataDetails()
+          val segmentInfo = segmentFile.getLocationMap.asScala.head._2
+          if (!segmentUpdateDetail.isEmpty) {
+            loadMetadataDetail.setSegmentStatus(segmentUpdateDetail.head.getSegmentStatus)
+            loadMetadataDetail.setModificationOrDeletionTimestamp(segmentUpdateDetail.head
+              .getDeleteDeltaStartTimeAsLong)
+          } else {
+            loadMetadataDetail.setSegmentStatus(getSegmentStatus(segmentInfo.getStatus))
+          }
+          loadMetadataDetail.setLoadName(segmentId)
+          loadMetadataDetail.setSegmentFile(segmentFileName)
+          val dataIndexSize = CarbonUtil.getDataSizeAndIndexSize(carbonTable
+            .getTablePath, new Segment(segmentId, segmentFileName))
+          loadMetadataDetail.setDataSize(dataIndexSize
+            .get(CarbonCommonConstants.CARBON_TOTAL_DATA_SIZE).toString)
+          loadMetadataDetail.setIndexSize(dataIndexSize
+            .get(CarbonCommonConstants.CARBON_TOTAL_INDEX_SIZE).toString)
+          loadMetadataDetail.setLoadEndTime(FileFactory
+            .getCarbonFile(segFilePath)
+            .getLastModifiedTime)
+          missedLoadMetaDetails.add(loadMetadataDetail)
+          if (loadMetaDetails.exists(_.getLoadName.equalsIgnoreCase(segmentId))) {
+            loadMetaDetails = loadMetaDetails.filterNot(_.getLoadName
+              .equalsIgnoreCase(segmentId))
+          }
+        } else if (!segmentUpdateDetail.isEmpty) {
+          // in case of Update/delete, update the already existing load metadata entry with the
+          // latest segment update detail
+          val loadMetadataDetail = loadMetaDetails
+            .find(_.getLoadName.equalsIgnoreCase(segmentId))
+            .head
+          loadMetadataDetail.setSegmentStatus(segmentUpdateDetail.head.getSegmentStatus)
+          loadMetadataDetail.setModificationOrDeletionTimestamp(segmentUpdateDetail.head
+            .getDeleteDeltaStartTimeAsLong)
+          loadMetaDetails = loadMetaDetails.filterNot(_.getLoadName.equalsIgnoreCase(segmentId))
+          missedLoadMetaDetails.add(loadMetadataDetail)
+        }
+      }
+      missedLoadMetaDetails.addAll(loadMetaDetails.asJava)
+      // write new table status file with lost table status version name
+      SegmentStatusManager.writeLoadDetailsIntoFile(CarbonTablePath.getTableStatusFilePath(
+        carbonTable.getTablePath, tableStatusVersion),
+        missedLoadMetaDetails.toArray(new Array[LoadMetadataDetails](missedLoadMetaDetails
+          .size)))
+    }
+  }
+
+  private def getSegmentStatus(segmentStatus: String): SegmentStatus = {
+    if (segmentStatus.equalsIgnoreCase("success")) {
+      SegmentStatus.SUCCESS
+    } else if (segmentStatus.equalsIgnoreCase("Marked for Delete")) {
+      SegmentStatus.MARKED_FOR_DELETE
+    } else if (segmentStatus.equalsIgnoreCase("Failure")) {
+      SegmentStatus.LOAD_FAILURE
+    } else {
+      SegmentStatus.COMPACTED
+    }
+  }
+
+  private def createCarbonSession(): SparkSession = {
+    val spark = SparkSession
+      .builder().config(new SparkConf())
+      .appName("RecoveryTool")
+      .enableHiveSupport()
+      .config("spark.sql.extensions", "org.apache.spark.sql.CarbonExtensions")
+      .getOrCreate()
+    CarbonEnv.getInstance(spark)
+
+    SparkSession.setActiveSession(spark)
+    SparkSession.setDefaultSession(spark)
+    spark
+  }
+}
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index 0449fbae31..7b1d20d203 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@ -742,7 +742,8 @@ object CarbonDataRDDFactory {
         carbonLoadModel.getFactTimeStamp.toString,
         true,
         true,
-        updateModel.get.deletedSegments.asJava)
+        updateModel.get.deletedSegments.asJava,
+        carbonLoadModel.getLatestTableStatusWriteVersion).get("status").toBoolean
     }
     done = done && CarbonLoaderUtil.recordNewLoadMetadata(metadataDetails, carbonLoadModel, false,
       overwriteTable, uuid, false)
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
index 27a13da45a..aecc4a1f01 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
@@ -328,7 +328,8 @@ class CarbonMergerRDD[K, V](
     // Only for range column get the details for the size of segments
     if (null != rangeColumn) {
       loadMetadataDetails = SegmentStatusManager
-        .readLoadMetadata(CarbonTablePath.getMetadataPath(tablePath))
+        .readLoadMetadata(CarbonTablePath.getMetadataPath(tablePath),
+          carbonTable.getTableStatusVersion)
     }
 
     val validSegIds: java.util.List[String] = new util.ArrayList[String]()
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
index d39143aa4f..12824c9223 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
@@ -32,6 +32,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
 import org.apache.spark.sql.execution.command.{AlterTableAddPartitionCommand, AlterTableDropPartitionCommand, CarbonMergerMapping, CompactionCallableModel, CompactionModel}
 import org.apache.spark.sql.execution.command.management.CommonLoadUtils
 import org.apache.spark.sql.execution.datasources.PartitioningUtils
+import org.apache.spark.sql.hive.CarbonHiveIndexMetadataUtil
 import org.apache.spark.sql.util.SparkSQLUtil
 import org.apache.spark.util.{CollectionAccumulator, MergeIndexUtil}
 
@@ -228,8 +229,11 @@ class CarbonTableCompactor(
       maxSegmentColumnSchemaList = null,
       currentPartitions = partitions)
     carbonLoadModel.setTablePath(carbonMergerMapping.hdfsStoreLocation)
+    val tblStatusVersion = carbonLoadModel.getCarbonDataLoadSchema
+      .getCarbonTable.getTableStatusVersion
     carbonLoadModel.setLoadMetadataDetails(
-      SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath).toList.asJava)
+      SegmentStatusManager
+        .readLoadMetadata(carbonTable.getMetadataPath, tblStatusVersion).toList.asJava)
     // trigger event for compaction
     val alterTableCompactionPreEvent: AlterTableCompactionPreEvent =
       AlterTableCompactionPreEvent(sqlContext.sparkSession,
@@ -416,6 +420,9 @@ class CarbonTableCompactor(
                             s" ${ carbonLoadModel.getDatabaseName }." +
                             s"${ carbonLoadModel.getTableName }")
       }
+      CarbonHiveIndexMetadataUtil.updateTableStatusVersion(carbonTable,
+        sc.sparkSession,
+        carbonLoadModel.getLatestTableStatusWriteVersion)
 
       val compactionLoadStatusPostEvent = AlterTableCompactionPostStatusUpdateEvent(sc.sparkSession,
         carbonTable,
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/StreamHandoffRDD.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/StreamHandoffRDD.scala
index bb73d4aef6..035822aa60 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/StreamHandoffRDD.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/StreamHandoffRDD.scala
@@ -25,6 +25,7 @@ import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
 import org.apache.spark.{Partition, SerializableWritable, TaskContext}
 import org.apache.spark.sql.SparkSession
 import org.apache.spark.sql.catalyst.expressions.GenericInternalRow
+import org.apache.spark.sql.hive.CarbonHiveIndexMetadataUtil
 
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.converter.SparkDataTypeConverterImpl
@@ -224,14 +225,16 @@ object StreamHandoffRDD {
                     s" ${ carbonTable.getDatabaseName }.${ carbonTable.getTableName }")
         // handoff streaming segment one by one
         do {
-          val segmentStatusManager = new SegmentStatusManager(identifier)
+          val segmentStatusManager = new SegmentStatusManager(identifier,
+            carbonTable.getTableStatusVersion)
           var loadMetadataDetails: Array[LoadMetadataDetails] = null
           // lock table to read table status file
           val statusLock = segmentStatusManager.getTableStatusLock
           try {
             if (statusLock.lockWithRetries()) {
               loadMetadataDetails = SegmentStatusManager.readLoadMetadata(
-                CarbonTablePath.getMetadataPath(identifier.getTablePath))
+                CarbonTablePath.getMetadataPath(identifier.getTablePath),
+                carbonTable.getTableStatusVersion)
             }
           } finally {
             if (null != statusLock) {
@@ -348,6 +351,10 @@ object StreamHandoffRDD {
 
       val done = updateLoadMetadata(handoffSegmentId, carbonLoadModel)
 
+      CarbonHiveIndexMetadataUtil.updateTableStatusVersion(carbonLoadModel
+        .getCarbonDataLoadSchema
+        .getCarbonTable, sparkSession, carbonLoadModel.getLatestTableStatusWriteVersion)
+
       val loadTablePostStatusUpdateEvent: LoadTablePostStatusUpdateEvent =
         new LoadTablePostStatusUpdateEvent(carbonLoadModel)
       OperationListenerBus.getInstance()
@@ -374,8 +381,10 @@ object StreamHandoffRDD {
     if (!FileFactory.isFileExist(metadataPath)) {
       FileFactory.mkdirs(metadataPath)
     }
-    val tableStatusPath = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath)
-    val segmentStatusManager = new SegmentStatusManager(identifier)
+    val tableStatusPath = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath,
+      loadModel.getLatestTableStatusWriteVersion)
+    val segmentStatusManager = new SegmentStatusManager(identifier,
+      loadModel.getLatestTableStatusWriteVersion)
     val carbonLock = segmentStatusManager.getTableStatusLock
     try {
       if (carbonLock.lockWithRetries()) {
@@ -383,7 +392,8 @@ object StreamHandoffRDD {
           "Acquired lock for table" + loadModel.getDatabaseName() + "." + loadModel.getTableName()
           + " for table status update")
         val listOfLoadFolderDetailsArray =
-          SegmentStatusManager.readLoadMetadata(metaDataFilepath)
+          SegmentStatusManager.readLoadMetadata(metaDataFilepath,
+            loadModel.getLatestTableStatusWriteVersion)
 
         // update new columnar segment to success status
         val newSegment =
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
index d4c5d0fefe..ab06f5d77b 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
@@ -41,12 +41,15 @@ import org.apache.carbondata.common.exceptions.MetadataProcessException
 import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.filesystem.{CarbonFile, CarbonFileFilter}
+import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory
 import org.apache.carbondata.core.keygenerator.directdictionary.timestamp.DateDirectDictionaryGenerator
 import org.apache.carbondata.core.metadata.datatype.{DataTypes => CarbonDataTypes}
 import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, IndexSchema}
 import org.apache.carbondata.core.metadata.schema.table.column.{CarbonColumn, ColumnSchema}
 import org.apache.carbondata.core.util.{ByteUtil, DataTypeUtil}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.processing.exception.DataLoadingException
 import org.apache.carbondata.processing.loading.FailureCauses
 import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException
@@ -780,4 +783,46 @@ object CarbonScalaUtil {
     (response, endTime)
   }
 
+  def getLatestTableStatusVersion(tablePath: String): String = {
+    val tableStatusPath = CarbonTablePath.getTableStatusFilePath(tablePath)
+    if (!FileFactory.isFileExist(tableStatusPath)) {
+      // in case, if table has multi-versioned table status files, then get the latest table
+      // version and add it to table properties
+      val tableStatusFiles = getTableStatusVersionFiles(tablePath)
+      if (tableStatusFiles.isEmpty) {
+        return ""
+      }
+      getLatestTblStatusVersionBasedOnTimestamp(tableStatusFiles)
+    } else {
+      ""
+    }
+  }
+
+  def getTableStatusVersionFiles(tablePath: String): Array[CarbonFile] = {
+    FileFactory.getCarbonFile(CarbonTablePath.getMetadataPath(tablePath))
+      .listFiles(new CarbonFileFilter {
+        override def accept(file: CarbonFile): Boolean = {
+          file.getName.startsWith(CarbonTablePath
+            .TABLE_STATUS_FILE)
+        }
+      }).filterNot(_.getName.contains(CarbonTablePath.TABLE_STATUS_HISTORY_FILE))
+  }
+
+  def getLatestTblStatusVersionBasedOnTimestamp(tableStatusFiles: Array[CarbonFile]): String = {
+    var latestTableStatusVersion = 0L
+    tableStatusFiles.foreach { tableStatusFile =>
+      val versionTimeStamp = tableStatusFile.getName
+        .substring(tableStatusFile.getName.lastIndexOf(CarbonCommonConstants.UNDERSCORE) + 1,
+          tableStatusFile.getName.length).toLong
+      if (latestTableStatusVersion <= versionTimeStamp) {
+        latestTableStatusVersion = versionTimeStamp
+      }
+    }
+    if (latestTableStatusVersion == 0L) {
+      ""
+    } else {
+      latestTableStatusVersion.toString
+    }
+  }
+
 }
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala b/integration/spark/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
index 95ebd78f7c..8ff223bf31 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
@@ -28,6 +28,7 @@ import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
 import org.apache.spark.sql.SparkSession
 import org.apache.spark.sql.execution.command.management.CommonLoadUtils
 import org.apache.spark.sql.execution.streaming.{CarbonAppendableStreamSink, Sink}
+import org.apache.spark.sql.hive.CarbonHiveIndexMetadataUtil
 
 import org.apache.carbondata.common.Maps
 import org.apache.carbondata.common.logging.LogServiceFactory
@@ -106,9 +107,16 @@ object StreamSinkFactory {
       None,
       operationContext)
     // prepare the stream segment
-    val segmentId = getStreamSegmentId(carbonTable)
+    val segmentId = getStreamSegmentId(carbonTable,
+      carbonLoadModel)
     carbonLoadModel.setSegmentId(segmentId)
 
+    if (carbonLoadModel.getLatestTableStatusWriteVersion.nonEmpty) {
+      CarbonHiveIndexMetadataUtil.updateTableStatusVersion(carbonTable,
+        sparkSession,
+        carbonLoadModel.getLatestTableStatusWriteVersion)
+    }
+
     // default is carbon appended stream sink
     val carbonAppendableStreamSink = new CarbonAppendableStreamSink(
       sparkSession,
@@ -150,8 +158,9 @@ object StreamSinkFactory {
    * get current stream segment id
    * @return
    */
-  private def getStreamSegmentId(carbonTable: CarbonTable): String = {
-    val segmentId = StreamSegment.open(carbonTable)
+  private def getStreamSegmentId(carbonTable: CarbonTable,
+      loadModel: CarbonLoadModel): String = {
+    val segmentId = StreamSegment.open(carbonTable, loadModel)
     val segmentDir = CarbonTablePath.getSegmentPath(carbonTable.getTablePath, segmentId)
     val metadataPath = CarbonTablePath.getMetadataPath(carbonTable.getTablePath)
     if (!FileFactory.isFileExist(metadataPath)) {
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/trash/DataTrashManager.scala b/integration/spark/src/main/scala/org/apache/carbondata/trash/DataTrashManager.scala
index d700653688..7b024204e6 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/trash/DataTrashManager.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/trash/DataTrashManager.scala
@@ -20,6 +20,8 @@ package org.apache.carbondata.trash
 import scala.collection.JavaConverters._
 
 import org.apache.commons.lang.StringUtils
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.hive.CarbonHiveIndexMetadataUtil
 
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
@@ -34,6 +36,7 @@ import org.apache.carbondata.core.mutate.CarbonUpdateUtil
 import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager, SegmentUpdateStatusManager}
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil, CleanFilesUtil, DeleteLoadFolders, TrashUtil}
 import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.spark.util.CarbonScalaUtil
 
 object DataTrashManager {
   private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
@@ -53,6 +56,7 @@ object DataTrashManager {
       isForceDelete: Boolean,
       cleanStaleInProgress: Boolean,
       showStatistics: Boolean,
+      sparkSession: SparkSession,
       partitionSpecs: Option[Seq[PartitionSpec]] = None) : Long = {
     // if isForceDelete = true need to throw exception if CARBON_CLEAN_FILES_FORCE_ALLOWED is false
     if (isForceDelete && !CarbonProperties.getInstance().isCleanFilesForceAllowed) {
@@ -90,20 +94,24 @@ object DataTrashManager {
       } else {
         0
       }
+      if (isForceDelete) {
+        cleanUpTableStatusVersionFiles(carbonTable: CarbonTable)
+      }
       // step 3: clean expired segments(MARKED_FOR_DELETE, Compacted, In Progress)
       // Since calculating the the size before and after clean files can be a costly operation
       // have exposed an option where user can change this behaviour.
       if (showStatistics) {
-        val metadataDetails = SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath)
+        val metadataDetails = SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath,
+          carbonTable.getTableStatusVersion)
         val sizeBeforeCleaning = getPreOpSizeSnapshot(carbonTable, metadataDetails)
         checkAndCleanExpiredSegments(carbonTable, isForceDelete,
-          cleanStaleInProgress, partitionSpecs)
+          cleanStaleInProgress, partitionSpecs, sparkSession)
         val sizeAfterCleaning = getPostOpSizeSnapshot(carbonTable, metadataDetails
             .map(a => a.getLoadName).toSet)
         (sizeBeforeCleaning - sizeAfterCleaning + trashFolderSizeStats._1 + deltaFileSize).abs
       } else {
         checkAndCleanExpiredSegments(carbonTable, isForceDelete,
-          cleanStaleInProgress, partitionSpecs)
+          cleanStaleInProgress, partitionSpecs, sparkSession)
         0
       }
     } finally {
@@ -116,6 +124,31 @@ object DataTrashManager {
     }
   }
 
+  def cleanUpTableStatusVersionFiles(carbonTable: CarbonTable): Unit = {
+    // delete old version files except the current version file and the previous version file
+    val tableStatusFiles = CarbonScalaUtil.getTableStatusVersionFiles(carbonTable.getTablePath)
+      .filterNot(_.getName.endsWith(carbonTable.getTableStatusVersion))
+    // delete old version files except the most recent one for recovery
+    var prevTableStatusVersion = 0L
+    var tableStatusVersionFile: CarbonFile = null
+    tableStatusFiles.foreach { tableStatusFile =>
+      if (!tableStatusFile.getName.endsWith(CarbonTablePath.TABLE_STATUS_FILE)) {
+        val versionTimeStamp = tableStatusFile.getName
+          .substring(tableStatusFile.getName.lastIndexOf(CarbonCommonConstants.UNDERSCORE) + 1,
+            tableStatusFile.getName.length).toLong
+        if (prevTableStatusVersion <= versionTimeStamp) {
+          prevTableStatusVersion = versionTimeStamp
+          if (null != tableStatusVersionFile) {
+            tableStatusVersionFile.delete()
+          }
+          tableStatusVersionFile = tableStatusFile
+        } else {
+          tableStatusFile.delete()
+        }
+      }
+    }
+  }
+
   /**
    * Checks the size of the segment files as well as datafiles and index files, this method
    * is used before clean files operation.
@@ -140,7 +173,8 @@ object DataTrashManager {
    * clean files operation.
    */
   def getPostOpSizeSnapshot(carbonTable: CarbonTable, metadataDetails: Set[String]): Long = {
-    val finalMetadataDetails = SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath)
+    val finalMetadataDetails = SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath,
+      carbonTable.getTableStatusVersion)
     var size: Long = 0
     val segmentFileLocation = CarbonTablePath.getSegmentFilesLocation(carbonTable.getTablePath)
     if (FileFactory.isFileExist(segmentFileLocation)) {
@@ -172,6 +206,9 @@ object DataTrashManager {
     } else {
       0
     }
+    if (isForceDelete) {
+      cleanUpTableStatusVersionFiles(carbonTable: CarbonTable)
+    }
     // get size that will be deleted (MFD, COmpacted, Inprogress segments)
     val expiredSegmentsSizeStats = dryRunOnExpiredSegments(carbonTable, isForceDelete,
       cleanStaleInProgress)
@@ -208,10 +245,16 @@ object DataTrashManager {
       carbonTable: CarbonTable,
       isForceDelete: Boolean,
       cleanStaleInProgress: Boolean,
-      partitionSpecsOption: Option[Seq[PartitionSpec]]): Unit = {
+      partitionSpecsOption: Option[Seq[PartitionSpec]],
+      sparkSession: SparkSession): Unit = {
     val partitionSpecs = partitionSpecsOption.map(_.asJava).orNull
-    SegmentStatusManager.deleteLoadsAndUpdateMetadata(carbonTable,
+    val newTblStatusVersion = SegmentStatusManager.deleteLoadsAndUpdateMetadata(carbonTable,
       isForceDelete, partitionSpecs, cleanStaleInProgress, true)
+    if (newTblStatusVersion.nonEmpty) {
+      // if clean files update is complete, then update the table status version to table
+      CarbonHiveIndexMetadataUtil.updateTableStatusVersion(carbonTable,
+        sparkSession, newTblStatusVersion)
+    }
     if (carbonTable.isHivePartitionTable && partitionSpecsOption.isDefined) {
       SegmentFileStore.cleanSegments(carbonTable, partitionSpecs, isForceDelete)
     }
@@ -228,7 +271,8 @@ object DataTrashManager {
       cleanStaleInProgress: Boolean): (Long, Long) = {
     var sizeFreed: Long = 0
     var trashSizeRemaining: Long = 0
-    val loadMetadataDetails = SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath)
+    val loadMetadataDetails = SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath,
+      carbonTable.getTableStatusVersion)
     if (SegmentStatusManager.isLoadDeletionRequired(loadMetadataDetails)) {
       loadMetadataDetails.foreach { oneLoad =>
         if (!oneLoad.getVisibility.equalsIgnoreCase("false")) {
@@ -305,7 +349,8 @@ object DataTrashManager {
       factTimestamp: Long,
       partitionSpecs: Option[Seq[PartitionSpec]]): Unit = {
     val metadataFolderPath = CarbonTablePath.getMetadataPath(carbonTable.getTablePath)
-    val details = SegmentStatusManager.readLoadMetadata(metadataFolderPath)
+    val details = SegmentStatusManager.readLoadMetadata(metadataFolderPath,
+      carbonTable.getTableStatusVersion)
     if (details == null || details.isEmpty) {
       return
     }
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/view/MVRefresher.scala b/integration/spark/src/main/scala/org/apache/carbondata/view/MVRefresher.scala
index 8f1237dbb9..27243aaa31 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/view/MVRefresher.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/view/MVRefresher.scala
@@ -24,8 +24,9 @@ import scala.collection.JavaConverters._
 
 import com.google.gson.Gson
 import org.apache.log4j.Logger
-import org.apache.spark.sql.{CarbonThreadUtil, SparkSession}
+import org.apache.spark.sql.{CarbonEnv, CarbonThreadUtil, SparkSession}
 import org.apache.spark.sql.execution.command.management.CarbonInsertIntoCommand
+import org.apache.spark.sql.hive.CarbonHiveIndexMetadataUtil
 import org.apache.spark.sql.parser.MVQueryParser
 
 import org.apache.carbondata.common.exceptions.sql.NoSuchMVException
@@ -35,6 +36,7 @@ import org.apache.carbondata.core.locks.ICarbonLock
 import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, RelationIdentifier}
 import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatus, SegmentStatusManager}
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager.ValidAndInvalidSegmentsInfo
+import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.core.view.{MVSchema, MVStatus}
 import org.apache.carbondata.processing.util.CarbonLoaderUtil
@@ -66,7 +68,8 @@ object MVRefresher {
       viewSchema.getIdentifier.getTableId)
     val viewIdentifier = viewSchema.getIdentifier
     val viewTableIdentifier = viewTable.getAbsoluteTableIdentifier
-    val segmentStatusManager: SegmentStatusManager = new SegmentStatusManager(viewTableIdentifier)
+    val segmentStatusManager: SegmentStatusManager = new SegmentStatusManager(viewTableIdentifier,
+      viewTable.getTableStatusVersion)
     // Acquire table status lock to handle concurrent data loading
     val lock: ICarbonLock = segmentStatusManager.getTableStatusLock
     val segmentMapping: util.Map[String, util.List[String]] =
@@ -76,7 +79,17 @@ object MVRefresher {
       LOGGER.info("Acquired lock for mv " + viewIdentifier + " for table status update")
       val viewTableMetadataPath: String =
         CarbonTablePath.getMetadataPath(viewIdentifier.getTablePath)
-      val loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTableMetadataPath)
+      val loadMetadataDetails = try {
+        SegmentStatusManager.readLoadMetadata(viewTableMetadataPath,
+          viewTable.getTableStatusVersion)
+      } catch {
+        case ex: RuntimeException =>
+          if (ex.getMessage.contains("Table Status Version file")) {
+            new Array[LoadMetadataDetails](0)
+          } else {
+            throw ex
+          }
+      }
       val loadMetadataDetailList: util.List[LoadMetadataDetails] =
         new util.ArrayList[LoadMetadataDetails](CarbonCommonConstants.DEFAULT_COLLECTION_SIZE)
       // Mark for delete all stale loadMetataDetail
@@ -104,7 +117,8 @@ object MVRefresher {
         }
       }
       if (viewSchema.isRefreshIncremental) {
-        if (!getSpecificSegmentsTobeLoaded(viewSchema, segmentMapping, loadMetadataDetailList)) {
+        if (!getSpecificSegmentsTobeLoaded(
+          viewSchema, segmentMapping, loadMetadataDetailList, session)) {
           return false
         }
       } else {
@@ -112,8 +126,9 @@ object MVRefresher {
           val relatedTableIds =
             viewSchema.getRelatedTables.asScala.filter(_.isCarbonDataTable)
           for (relatedTableId <- relatedTableIds) {
+            val parentTblVersion: String = getTableStatusVersion(relatedTableId, session)
             val validAndInvalidSegmentsInfo =
-              SegmentStatusManager.getValidAndInvalidSegmentsInfo(relatedTableId)
+              SegmentStatusManager.getValidAndInvalidSegmentsInfo(relatedTableId, parentTblVersion)
             val relatedTableSegmentList: util.List[String] = SegmentStatusManager
               .getValidSegmentList(validAndInvalidSegmentsInfo)
             if (relatedTableSegmentList.isEmpty) {
@@ -133,10 +148,17 @@ object MVRefresher {
       loadMetadataDetail.setExtraInfo(segmentMap)
       loadMetadataDetailList.add(loadMetadataDetail)
       newLoadName = segmentId
+      val tblStatusWriteVersion = if (CarbonProperties.isTableStatusMultiVersionEnabled) {
+        System.currentTimeMillis().toString
+      } else {
+        ""
+      }
       SegmentStatusManager.writeLoadDetailsIntoFile(CarbonTablePath.getTableStatusFilePath(
-        viewSchema.getIdentifier.getTablePath),
+        viewSchema.getIdentifier.getTablePath, tblStatusWriteVersion),
         loadMetadataDetailList.toArray(new Array[LoadMetadataDetails](loadMetadataDetailList
           .size)))
+      CarbonHiveIndexMetadataUtil.updateTableStatusVersion(viewTable,
+        session, tblStatusWriteVersion)
     } else {
       LOGGER.error("Not able to acquire the lock for table status update for table " +
                    viewSchema.getIdentifier.getDatabaseName + "." +
@@ -236,15 +258,17 @@ object MVRefresher {
   @throws[IOException]
   private def getSpecificSegmentsTobeLoaded(schema: MVSchema,
       segmentMapping: util.Map[String, util.List[String]],
-      listOfLoadFolderDetails: util.List[LoadMetadataDetails]): Boolean = {
+      listOfLoadFolderDetails: util.List[LoadMetadataDetails],
+      sparkSession: SparkSession): Boolean = {
     val relationIdentifiers: util.List[RelationIdentifier] = schema.getRelatedTables
     // invalidSegmentList holds segment list which needs to be marked for delete
     val invalidSegmentList: util.HashSet[String] = new util.HashSet[String]
     if (listOfLoadFolderDetails.isEmpty) {
       // If segment Map is empty, load all valid segments from main tables to mv
       for (relationIdentifier <- relationIdentifiers.asScala) {
+        val parentTblVersion: String = getTableStatusVersion(relationIdentifier, sparkSession)
         val validAndInvalidSegmentsInfo =
-          SegmentStatusManager.getValidAndInvalidSegmentsInfo(relationIdentifier)
+          SegmentStatusManager.getValidAndInvalidSegmentsInfo(relationIdentifier, parentTblVersion)
         val mainTableSegmentList: util.List[String] = SegmentStatusManager
           .getValidSegmentList(validAndInvalidSegmentsInfo)
         // If mainTableSegmentList is empty, no need to trigger load command
@@ -258,8 +282,9 @@ object MVRefresher {
       for (relationIdentifier <- relationIdentifiers.asScala) {
         val segmentList: util.List[String] = new util.ArrayList[String]
         // Get all segments for parent relationIdentifier
+        val parentTblVersion: String = getTableStatusVersion(relationIdentifier, sparkSession)
         val validAndInvalidSegmentsInfo =
-          SegmentStatusManager.getValidAndInvalidSegmentsInfo(relationIdentifier)
+          SegmentStatusManager.getValidAndInvalidSegmentsInfo(relationIdentifier, parentTblVersion)
         val mainTableSegmentList: util.List[String] = SegmentStatusManager
           .getValidSegmentList(validAndInvalidSegmentsInfo)
         var ifTableStatusUpdateRequired: Boolean = false
@@ -298,8 +323,9 @@ object MVRefresher {
         segmentList.removeAll(mainTableSegmentList)
         mainTableSegmentList.removeAll(originSegmentList)
         if (ifTableStatusUpdateRequired && mainTableSegmentList.isEmpty) {
+          val version = getTableStatusVersion(schema.getIdentifier, sparkSession)
           SegmentStatusManager.writeLoadDetailsIntoFile(CarbonTablePath.getTableStatusFilePath(
-            schema.getIdentifier.getTablePath),
+            schema.getIdentifier.getTablePath, version),
             listOfLoadFolderDetails.toArray(new Array[LoadMetadataDetails](listOfLoadFolderDetails
               .size)))
           return false
@@ -353,6 +379,16 @@ object MVRefresher {
     true
   }
 
+  def getTableStatusVersion(identifier: RelationIdentifier, sparkSession: SparkSession): String = {
+    val carbonTable = CarbonEnv.getCarbonTable(Some(identifier.getDatabaseName),
+      identifier.getTableName)(sparkSession)
+    var parentTblVersion = ""
+    if (null != carbonTable) {
+      parentTblVersion = carbonTable.getTableStatusVersion
+    }
+    parentTblVersion
+  }
+
   /**
    * This method checks if mv table segment has to be reloaded again or not
    */
diff --git a/integration/spark/src/main/scala/org/apache/spark/rdd/CarbonMergeFilesRDD.scala b/integration/spark/src/main/scala/org/apache/spark/rdd/CarbonMergeFilesRDD.scala
index 974035cc74..1cb6f0146b 100644
--- a/integration/spark/src/main/scala/org/apache/spark/rdd/CarbonMergeFilesRDD.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/rdd/CarbonMergeFilesRDD.scala
@@ -198,7 +198,7 @@ object CarbonMergeFilesRDD {
           // when compact segment_index, update table status with new segment file name
           val status = SegmentFileStore.updateTableStatusFile(carbonTable, segmentId,
             newSegmentFileName + CarbonTablePath.SEGMENT_EXT,
-            carbonTable.getCarbonTableIdentifier.getTableId, sfs)
+            carbonTable.getCarbonTableIdentifier.getTableId, sfs, carbonTable.getTableStatusVersion)
           if (!status) {
             throw new IOException("Table status update with mergeIndex file has failed")
           }
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSource.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSource.scala
index 78cc6e4d52..fdf904ccc9 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSource.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSource.scala
@@ -373,6 +373,9 @@ object CarbonSource {
     map.put("tableName", tableInfo.getFactTable.getTableName)
     map.put("isTransactional", tableInfo.isTransactionalTable.toString)
     map.put("isExternal", isExternalTable.toString)
+    if (tableInfo.getFactTable.getTableProperties.containsKey("latestversion")) {
+      map.put("latestversion", tableInfo.getFactTable.getTableProperties.get("latestversion"))
+    }
     map.asScala.toMap
   }
 
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/events/MergeIndexEventListener.scala b/integration/spark/src/main/scala/org/apache/spark/sql/events/MergeIndexEventListener.scala
index c92bce818c..848b84c5ef 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/events/MergeIndexEventListener.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/events/MergeIndexEventListener.scala
@@ -110,8 +110,8 @@ class MergeIndexEventListener extends OperationEventListener with Logging {
           if (lock.lockWithRetries()) {
             LOGGER.info("Acquired the compaction lock for table" +
                         s" ${ carbonMainTable.getDatabaseName }.${ carbonMainTable.getTableName}")
-            val loadFolderDetailsArray = SegmentStatusManager
-              .readLoadMetadata(carbonMainTable.getMetadataPath)
+            val loadFolderDetailsArray = SegmentStatusManager.readLoadMetadata(carbonMainTable
+              .getMetadataPath, carbonMainTable.getTableStatusVersion)
             val segmentFileNameMap: java.util.Map[String, String] = new util.HashMap[String,
               String]()
             var streamingSegment: Set[String] = Set[String]()
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/cache/CacheUtil.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/cache/CacheUtil.scala
index a4b00a0a40..6a2d14134c 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/cache/CacheUtil.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/cache/CacheUtil.scala
@@ -49,8 +49,8 @@ object CacheUtil {
   def getAllIndexFiles(carbonTable: CarbonTable)(sparkSession: SparkSession): List[String] = {
     if (carbonTable.isTransactionalTable) {
       val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
-      val validAndInvalidSegmentsInfo = new SegmentStatusManager(absoluteTableIdentifier)
-        .getValidAndInvalidSegments(carbonTable.isMV)
+      val validAndInvalidSegmentsInfo = new SegmentStatusManager(absoluteTableIdentifier,
+        carbonTable.getTableStatusVersion).getValidAndInvalidSegments(carbonTable.isMV)
       // Fire a job to clear the invalid segments cached in the executors.
       if (CarbonProperties.getInstance().isDistributedPruningEnabled(carbonTable.getDatabaseName,
         carbonTable.getTableName)) {
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAddLoadCommand.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAddLoadCommand.scala
index ed40d9a40d..b2d9fe1969 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAddLoadCommand.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAddLoadCommand.scala
@@ -29,7 +29,7 @@ import org.apache.spark.sql.carbondata.execution.datasources.CarbonSparkDataSour
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.execution.command.{Checker, MetadataCommand}
 import org.apache.spark.sql.execution.strategy.MixedFormatHandler
-import org.apache.spark.sql.hive.CarbonRelation
+import org.apache.spark.sql.hive.{CarbonHiveIndexMetadataUtil, CarbonRelation}
 import org.apache.spark.sql.types.StructType
 
 import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
@@ -99,7 +99,8 @@ case class CarbonAddLoadCommand(
     val inputPath = givenPath
 
     // If a path is already added then we should block the adding of the same path again.
-    val allSegments = SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath)
+    val allSegments = SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath,
+      carbonTable.getTableStatusVersion)
     // If the segment has been already loaded from the same path or the segment is already present
     // in the table and its status is SUCCESS orPARTIALLY_SUCCESS, throw an exception as we should
     // block the adding of the same path again.
@@ -354,11 +355,16 @@ case class CarbonAddLoadCommand(
         segment.getSegmentFileName,
         carbonTable.getCarbonTableIdentifier.getTableId,
         new SegmentFileStore(carbonTable.getTablePath, segment.getSegmentFileName),
-        SegmentStatus.SUCCESS)
+        SegmentStatus.SUCCESS,
+        model.getLatestTableStatusWriteVersion)
     } else {
       false
     }
 
+    CarbonHiveIndexMetadataUtil.updateTableStatusVersion(carbonTable,
+      sparkSession,
+      model.getLatestTableStatusWriteVersion)
+
     val postExecutionEvent = if (success) {
       val loadTablePostStatusUpdateEvent: LoadTablePostStatusUpdateEvent =
         new LoadTablePostStatusUpdateEvent(model)
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
index 634c5f9004..8a97296b17 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
@@ -26,7 +26,7 @@ import org.apache.spark.sql.{AnalysisException, CarbonEnv, Row, SparkSession, SQ
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
 import org.apache.spark.sql.execution.command.{AlterTableModel, AtomicRunnableCommand, CompactionModel}
-import org.apache.spark.sql.hive.CarbonRelation
+import org.apache.spark.sql.hive.{CarbonHiveIndexMetadataUtil, CarbonRelation}
 import org.apache.spark.sql.optimizer.CarbonFilters
 import org.apache.spark.sql.util.CarbonException
 import org.apache.spark.util.AlterTableUtil
@@ -112,7 +112,7 @@ case class CarbonAlterTableCompactionCommand(
       try {
         if (tableStatusLock.lockWithRetries()) {
           val loadMetaDataDetails = SegmentStatusManager.readTableStatusFile(CarbonTablePath
-            .getTableStatusFilePath(table.getTablePath))
+            .getTableStatusFilePath(table.getTablePath, table.getTableStatusVersion))
           loadMetaDataDetails.foreach { loadMetaDataDetail =>
             // "0" check is added to reproduce a scenario similar to 1.1 store where the size
             // would be null. For test case in the new version it would be set to 0.
@@ -123,8 +123,9 @@ case class CarbonAlterTableCompactionCommand(
                   table)
             }
           }
-          SegmentStatusManager.writeLoadDetailsIntoFile(CarbonTablePath
-            .getTableStatusFilePath(table.getTablePath), loadMetaDataDetails)
+          SegmentStatusManager.writeLoadDetailsIntoFile(
+            CarbonTablePath.getTableStatusFilePath(table.getTablePath, table.getTableStatusVersion),
+            loadMetaDataDetails)
         } else {
           throw new ConcurrentOperationException(table.getDatabaseName,
             table.getTableName, "table status update", "upgrade segments")
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonCleanFilesCommand.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonCleanFilesCommand.scala
index 6461604e35..d88cf21241 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonCleanFilesCommand.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonCleanFilesCommand.scala
@@ -98,7 +98,7 @@ case class CarbonCleanFilesCommand(
           carbonTable,
           options.getOrElse("force", "false").toBoolean,
           options.getOrElse("stale_inprogress", "false").toBoolean,
-          showStats,
+          showStats, sparkSession,
           CarbonFilters.getPartitions(Seq.empty[Expression], sparkSession, carbonTable))
       }
       if (showStats) {
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonDeleteLoadByIdCommand.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonDeleteLoadByIdCommand.scala
index 1ba98f3934..3711750608 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonDeleteLoadByIdCommand.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonDeleteLoadByIdCommand.scala
@@ -52,7 +52,8 @@ case class CarbonDeleteLoadByIdCommand(
         loadIds,
         CarbonEnv.getDatabaseName(databaseNameOp)(sparkSession),
         tableName,
-        carbonTable
+        carbonTable,
+        sparkSession
       )
     }
     Seq.empty
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonDeleteLoadByLoadDateCommand.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonDeleteLoadByLoadDateCommand.scala
index 68cd842eaa..e8db1bc2e4 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonDeleteLoadByLoadDateCommand.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonDeleteLoadByLoadDateCommand.scala
@@ -52,7 +52,8 @@ case class CarbonDeleteLoadByLoadDateCommand(
         loadDate,
         CarbonEnv.getDatabaseName(databaseNameOp)(sparkSession),
         tableName,
-        carbonTable)
+        carbonTable,
+        sparkSession)
     }
     Seq.empty
   }
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonInsertFromStageCommand.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonInsertFromStageCommand.scala
index c5f08d9d69..7b7bf3c51b 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonInsertFromStageCommand.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonInsertFromStageCommand.scala
@@ -30,6 +30,7 @@ import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.mapreduce.InputSplit
 import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
 import org.apache.spark.sql.execution.command.{Checker, DataCommand}
+import org.apache.spark.sql.hive.CarbonHiveIndexMetadataUtil
 import org.apache.spark.sql.util.SparkSQLUtil
 
 import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
@@ -233,7 +234,7 @@ case class CarbonInsertFromStageCommand(
     var executorService: ExecutorService = null
     try {
       val segments = SegmentStatusManager.readTableStatusFile(
-        CarbonTablePath.getTableStatusFilePath(table.getTablePath)
+        CarbonTablePath.getTableStatusFilePath(table.getTablePath, table.getTableStatusVersion)
       )
       val matchedSegment = segments.filter(_.getLoadName.equals(segmentId))
       if (matchedSegment.length != 1) {
@@ -266,7 +267,7 @@ case class CarbonInsertFromStageCommand(
             s"segment entry and load again")
           val segmentToWrite = segments.filterNot(_.getLoadName.equals(segmentId))
           SegmentStatusManager.writeLoadDetailsIntoFile(
-            CarbonTablePath.getTableStatusFilePath(table.getTablePath),
+            CarbonTablePath.getTableStatusFilePath(table.getTablePath, table.getTableStatusVersion),
             segmentToWrite)
       }
     } finally {
@@ -330,6 +331,8 @@ case class CarbonInsertFromStageCommand(
           loadModel,
           segmentMetaDataAccumulator)
       }
+      CarbonHiveIndexMetadataUtil.updateTableStatusVersion(table,
+        spark, loadModel.getLatestTableStatusWriteVersion)
       LOGGER.info(s"finish data loading, time taken ${System.currentTimeMillis() - start}ms")
 
       // 4) write segment file and update the segment entry to SUCCESS
@@ -361,7 +364,8 @@ case class CarbonInsertFromStageCommand(
         table, loadModel.getSegmentId, segmentFileName,
         table.getCarbonTableIdentifier.getTableId,
         new SegmentFileStore(table.getTablePath, segmentFileName),
-        SegmentStatus.SUCCESS)
+        SegmentStatus.SUCCESS,
+        loadModel.getLatestTableStatusWriteVersion)
 
       // trigger load post events
       if (status) {
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonInsertIntoCommand.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonInsertIntoCommand.scala
index f59802c9a8..5a24aaaa0a 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonInsertIntoCommand.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonInsertIntoCommand.scala
@@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference,
 import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
 import org.apache.spark.sql.execution.command.{AtomicRunnableCommand, UpdateTableModel}
 import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.hive.CarbonHiveIndexMetadataUtil
 import org.apache.spark.sql.types.{StringType, StructField, StructType}
 import org.apache.spark.util.CausedBy
 
@@ -232,6 +233,11 @@ case class CarbonInsertIntoCommand(databaseNameOp: Option[String],
           isOverwriteTable)
         isUpdateTableStatusRequired = true
       }
+      if (isUpdateTableStatusRequired) {
+        CarbonHiveIndexMetadataUtil.updateTableStatusVersion(carbonLoadModel
+          .getCarbonDataLoadSchema
+          .getCarbonTable, sparkSession, carbonLoadModel.getLatestTableStatusWriteVersion)
+      }
       if (isOverwriteTable) {
         LOGGER.info(s"Overwrite of carbon table with $dbName.$tableName is in progress")
       }
@@ -488,6 +494,12 @@ case class CarbonInsertIntoCommand(databaseNameOp: Option[String],
         updateModel,
         operationContext)
     }
+    if (!table.getTableStatusVersion
+      .equals(loadParams.carbonLoadModel.getLatestTableStatusWriteVersion)) {
+      CarbonHiveIndexMetadataUtil.updateTableStatusVersion(table,
+        loadParams.sparkSession,
+        loadParams.carbonLoadModel.getLatestTableStatusWriteVersion)
+    }
     (rows, loadResult)
   }
 
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonInsertIntoWithDf.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonInsertIntoWithDf.scala
index 8be3555040..2d08697b98 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonInsertIntoWithDf.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonInsertIntoWithDf.scala
@@ -23,6 +23,7 @@ import scala.collection.JavaConverters._
 
 import org.apache.spark.sql.{AnalysisException, DataFrame, Row, SparkSession}
 import org.apache.spark.sql.execution.command.UpdateTableModel
+import org.apache.spark.sql.hive.CarbonHiveIndexMetadataUtil
 import org.apache.spark.util.CausedBy
 
 import org.apache.carbondata.common.logging.LogServiceFactory
@@ -112,6 +113,9 @@ case class CarbonInsertIntoWithDf(databaseNameOp: Option[String],
         CarbonLoaderUtil.readAndUpdateLoadProgressInTableMeta(
           carbonLoadModel,
           isOverwriteTable)
+        CarbonHiveIndexMetadataUtil.updateTableStatusVersion(table,
+          sparkSession,
+          carbonLoadModel.getLatestTableStatusWriteVersion)
         isUpdateTableStatusRequired = true
       }
       if (isOverwriteTable) {
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
index 4ee0e750d3..1497740be1 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
@@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTable
 import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
 import org.apache.spark.sql.execution.command.{AtomicRunnableCommand, DataLoadTableFileMapping}
 import org.apache.spark.sql.execution.datasources.{CatalogFileIndex, HadoopFsRelation, LogicalRelation, SparkCarbonTableFormat}
+import org.apache.spark.sql.hive.CarbonHiveIndexMetadataUtil
 import org.apache.spark.sql.types.{DateType, IntegerType, LongType, StringType, StructType, TimestampType}
 import org.apache.spark.util.{CarbonReflectionUtils, CausedBy, FileUtils}
 
@@ -128,6 +129,9 @@ case class CarbonLoadDataCommand(databaseNameOp: Option[String],
           carbonLoadModel,
           isOverwriteTable)
         isUpdateTableStatusRequired = true
+        CarbonHiveIndexMetadataUtil.updateTableStatusVersion(table,
+          sparkSession,
+          carbonLoadModel.getLatestTableStatusWriteVersion)
       }
       if (isOverwriteTable) {
         LOGGER.info(s"Overwrite of carbon table with $dbName.$tableName is in progress")
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonShowSegmentsAsSelectCommand.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonShowSegmentsAsSelectCommand.scala
index ab0ea56136..6bda29b3f7 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonShowSegmentsAsSelectCommand.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonShowSegmentsAsSelectCommand.scala
@@ -119,7 +119,7 @@ case class CarbonShowSegmentsAsSelectCommand(
       }
     }
 
-    val segments = readSegments(tablePath, showHistory, limit)
+    val segments = readSegments(tablePath, showHistory, limit, carbonTable.getTableStatusVersion)
     makeDfFromRows(sparkSession, tempViewName,
       carbonTable, segments, rows)
   }
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonShowSegmentsCommand.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonShowSegmentsCommand.scala
index 5d6baad9c9..be4d74e81f 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonShowSegmentsCommand.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonShowSegmentsCommand.scala
@@ -67,7 +67,7 @@ case class CarbonShowSegmentsCommand(
       rows = CarbonShowSegmentsCommand.showStages(tableStagePath)
     }
 
-    val segments = readSegments(tablePath, showHistory, limit)
+    val segments = readSegments(tablePath, showHistory, limit, carbonTable.getTableStatusVersion)
     rows ++ showBasic(segments, tablePath, carbonTable.isHivePartitionTable)
   }
 
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CommonLoadUtils.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CommonLoadUtils.scala
index 5cbdb3b949..9a75895d1e 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CommonLoadUtils.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CommonLoadUtils.scala
@@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
 import org.apache.spark.sql.execution.LogicalRDD
 import org.apache.spark.sql.execution.command.UpdateTableModel
 import org.apache.spark.sql.execution.datasources.{CatalogFileIndex, FindDataSourceTable, HadoopFsRelation, LogicalRelation, SparkCarbonTableFormat}
-import org.apache.spark.sql.hive.DistributionUtil
+import org.apache.spark.sql.hive.{CarbonHiveIndexMetadataUtil, DistributionUtil}
 import org.apache.spark.sql.optimizer.CarbonFilters
 import org.apache.spark.sql.types._
 import org.apache.spark.sql.util.SparkSQLUtil
@@ -151,9 +151,9 @@ object CommonLoadUtils {
     val carbonLoadModel = new CarbonLoadModel()
     carbonLoadModel.setFactFilePath(factPath)
     carbonLoadModel.setCarbonTransactionalTable(table.getTableInfo.isTransactionalTable)
-    carbonLoadModel.setAggLoadRequest(
-      internalOptions.getOrElse(CarbonCommonConstants.IS_INTERNAL_LOAD_CALL,
-        CarbonCommonConstants.IS_INTERNAL_LOAD_CALL_DEFAULT).toBoolean)
+    val isInternalLoadCall = internalOptions.getOrElse(CarbonCommonConstants.IS_INTERNAL_LOAD_CALL,
+      CarbonCommonConstants.IS_INTERNAL_LOAD_CALL_DEFAULT).toBoolean
+    carbonLoadModel.setAggLoadRequest(isInternalLoadCall)
     carbonLoadModel.setSegmentId(internalOptions.getOrElse("mergedSegmentName", ""))
     val columnCompressor = table.getTableInfo.getFactTable.getTableProperties.asScala
       .getOrElse(CarbonCommonConstants.COMPRESSOR,
@@ -168,6 +168,11 @@ object CommonLoadUtils {
         javaPartition(k) = v.get
       }
     }
+    // generate new timestamp for tablestatus version
+    if(isInternalLoadCall) {
+      carbonLoadModel.setLatestTableStatusWriteVersion(table.getTableStatusVersion)
+    }
+
     new CarbonLoadModelBuilder(table).build(
       options.asJava,
       optionsFinal,
@@ -706,6 +711,7 @@ object CommonLoadUtils {
     }
     val options = new mutable.HashMap[String, String]()
     options ++= catalogTable.storage.properties
+    options += (("latestversion", loadModel.getLatestTableStatusWriteVersion))
     options += (("overwrite", overWrite.toString))
     if (partition.nonEmpty) {
       val staticPartitionStr = ObjectSerializationUtil.convertObjectToString(
@@ -1065,6 +1071,9 @@ object CommonLoadUtils {
       // Create and ddd the segment to the tablestatus.
       CarbonLoaderUtil.readAndUpdateLoadProgressInTableMeta(loadParams.carbonLoadModel,
         loadParams.isOverwriteTable)
+      CarbonHiveIndexMetadataUtil.updateTableStatusVersion(table,
+        loadParams.sparkSession,
+        loadParams.carbonLoadModel.getLatestTableStatusWriteVersion)
       val convertRelation = convertToLogicalRelation(
         catalogTable,
         loadParams.sizeInBytes,
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/RefreshCarbonTableCommand.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/RefreshCarbonTableCommand.scala
index 160908d617..57c871332f 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/RefreshCarbonTableCommand.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/RefreshCarbonTableCommand.scala
@@ -31,6 +31,7 @@ import org.apache.spark.util.{AlterTableUtil, SparkUtil}
 
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.filesystem.{CarbonFile, CarbonFileFilter}
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.indexstore.PartitionSpec
 import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
@@ -41,6 +42,7 @@ import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
 import org.apache.carbondata.core.statusmanager.{SegmentStatus, SegmentStatusManager}
 import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.events.{withEvents, RefreshTablePostExecutionEvent, RefreshTablePreExecutionEvent}
+import org.apache.carbondata.spark.util.CarbonScalaUtil
 
 /**
  * Command to register carbon table from existing carbon table data
@@ -105,6 +107,14 @@ case class RefreshCarbonTableCommand(
               Seq.empty, true)(sparkSession, sparkSession.sessionState.catalog)
           }
         }
+        val tblStatusVersion = CarbonScalaUtil.getLatestTableStatusVersion(identifier.getTablePath)
+        // in case of multi-version table status file enabled, get the latest version and save to
+        // table properties
+        if (tblStatusVersion.nonEmpty) {
+          tableInfo.getFactTable
+            .getTableProperties
+            .put("latestversion", tblStatusVersion)
+        }
         // remove mv related info from source table properties
         tableInfo.getFactTable
           .getTableProperties.remove(CarbonCommonConstants.RELATED_MV_TABLES_MAP)
@@ -116,7 +126,7 @@ case class RefreshCarbonTableCommand(
         // Register partitions to hive metastore in case of hive partitioning carbon table
         if (tableInfo.getFactTable.getPartitionInfo != null &&
             tableInfo.getFactTable.getPartitionInfo.getPartitionType == PartitionType.NATIVE_HIVE) {
-          registerAllPartitionsToHive(identifier, sparkSession)
+          registerAllPartitionsToHive(identifier, sparkSession, tblStatusVersion)
         }
       }
     }
@@ -216,10 +226,11 @@ case class RefreshCarbonTableCommand(
    */
   private def registerAllPartitionsToHive(
       absIdentifier: AbsoluteTableIdentifier,
-      sparkSession: SparkSession): Unit = {
+      sparkSession: SparkSession,
+      version: String): Unit = {
     val metadataDetails =
       SegmentStatusManager.readLoadMetadata(
-        CarbonTablePath.getMetadataPath(absIdentifier.getTablePath))
+        CarbonTablePath.getMetadataPath(absIdentifier.getTablePath), version)
     // First read all partition information from each segment.
     val allpartitions = metadataDetails.map{ metadata =>
       if (metadata.getSegmentStatus == SegmentStatus.SUCCESS ||
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForDeleteCommand.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForDeleteCommand.scala
index 7d5418641d..5435ed374d 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForDeleteCommand.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForDeleteCommand.scala
@@ -22,6 +22,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.execution.command._
 import org.apache.spark.sql.execution.strategy.MixedFormatHandler
+import org.apache.spark.sql.hive.CarbonHiveIndexMetadataUtil
 import org.apache.spark.sql.types.LongType
 
 import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
@@ -31,6 +32,7 @@ import org.apache.carbondata.core.features.TableOperation
 import org.apache.carbondata.core.locks.{CarbonLockFactory, CarbonLockUtil, LockUsage}
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager
+import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.events.{DeleteFromTablePostEvent, DeleteFromTablePreEvent, OperationContext, OperationListenerBus}
 import org.apache.carbondata.processing.loading.FailureCauses
 import org.apache.carbondata.view.MVManagerInSpark
@@ -60,7 +62,8 @@ private[sql] case class CarbonProjectForDeleteCommand(
     }
 
     // Block the delete operation for non carbon formats
-    if (MixedFormatHandler.otherFormatSegmentsExist(carbonTable.getMetadataPath)) {
+    if (MixedFormatHandler.otherFormatSegmentsExist(carbonTable.getMetadataPath,
+      carbonTable.getTableStatusVersion)) {
       throw new MalformedCarbonCommandException(
         s"Unsupported delete operation on table containing mixed format segments")
     }
@@ -110,14 +113,20 @@ private[sql] case class CarbonProjectForDeleteCommand(
       }
       val executorErrors = ExecutionErrors(FailureCauses.NONE, "")
 
-      val (deletedSegments, deletedRowCount) = DeleteExecution.deleteDeltaExecution(
-        databaseNameOp,
-        tableName,
-        sparkSession,
-        dataRdd,
-        timestamp,
-        isUpdateOperation = false,
-        executorErrors)
+      val (deletedSegments, deletedRowCount, isUpdateRequired, tblStatusWriteVersion) =
+        DeleteExecution.deleteDeltaExecution(
+          databaseNameOp,
+          tableName,
+          sparkSession,
+          dataRdd,
+          timestamp,
+          isUpdateOperation = false,
+          executorErrors)
+
+      if (isUpdateRequired) {
+        CarbonHiveIndexMetadataUtil.updateTableStatusVersion(carbonTable,
+          sparkSession, tblStatusWriteVersion)
+      }
 
       deletedRows = deletedRowCount;
 
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForUpdateCommand.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForUpdateCommand.scala
index 6516d76e3d..773c4a1069 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForUpdateCommand.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForUpdateCommand.scala
@@ -70,7 +70,7 @@ private[sql] case class CarbonProjectForUpdateCommand(
     if (res.isEmpty) {
       return Array(Row(updatedRowCount)).toSeq
     }
-    val carbonTable = CarbonEnv.getCarbonTable(databaseNameOp, tableName)(sparkSession)
+    var carbonTable = CarbonEnv.getCarbonTable(databaseNameOp, tableName)(sparkSession)
     setAuditTable(carbonTable)
     setAuditInfo(Map("plan" -> plan.prettyJson))
     // Do not allow spatial index and its source columns to be updated.
@@ -95,7 +95,8 @@ private[sql] case class CarbonProjectForUpdateCommand(
     }
 
     // Block the update operation for non carbon formats
-    if (MixedFormatHandler.otherFormatSegmentsExist(carbonTable.getMetadataPath)) {
+    if (MixedFormatHandler.otherFormatSegmentsExist(carbonTable.getMetadataPath,
+      carbonTable.getTableStatusVersion)) {
       throw new MalformedCarbonCommandException(
         s"Unsupported update operation on table containing mixed format segments")
     }
@@ -163,14 +164,15 @@ private[sql] case class CarbonProjectForUpdateCommand(
           }
 
           // do delete operation.
-          val (segmentsToBeDeleted, updatedRowCountTemp) = DeleteExecution.deleteDeltaExecution(
-            databaseNameOp,
-            tableName,
-            sparkSession,
-            dataSet.rdd,
-            currentTime + "",
-            isUpdateOperation = true,
-            executionErrors)
+          val (segmentsToBeDeleted, updatedRowCountTemp, isUpdateRequired, tblStatusVersion) =
+            DeleteExecution.deleteDeltaExecution(
+              databaseNameOp,
+              tableName,
+              sparkSession,
+              dataSet.rdd,
+              currentTime + "",
+              isUpdateOperation = true,
+              executionErrors)
 
           if (executionErrors.failureCauses != FailureCauses.NONE) {
             throw new Exception(executionErrors.errorMsg)
@@ -201,7 +203,9 @@ private[sql] case class CarbonProjectForUpdateCommand(
       if (executionErrors.failureCauses != FailureCauses.NONE) {
         throw new Exception(executionErrors.errorMsg)
       }
-
+      if (CarbonProperties.isTableStatusMultiVersionEnabled) {
+        carbonTable = CarbonEnv.getCarbonTable(databaseNameOp, tableName)(sparkSession)
+      }
       // Do IUD Compaction.
       HorizontalCompaction.tryHorizontalCompaction(
         sparkSession, carbonTable)
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/DeleteExecution.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/DeleteExecution.scala
index a006d5fef4..1f42421fdf 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/DeleteExecution.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/DeleteExecution.scala
@@ -64,7 +64,7 @@ object DeleteExecution {
       dataRdd: RDD[Row],
       timestamp: String,
       isUpdateOperation: Boolean,
-      executorErrors: ExecutionErrors): (Seq[Segment], Long) = {
+      executorErrors: ExecutionErrors): (Seq[Segment], Long, Boolean, String) = {
 
     val (res, blockMappingVO) = deleteDeltaExecutionInternal(databaseNameOp,
       tableName, sparkSession, dataRdd, timestamp, isUpdateOperation, executorErrors)
@@ -72,19 +72,19 @@ object DeleteExecution {
     var operatedRowCount = 0L
     // if no loads are present then no need to do anything.
     if (res.flatten.isEmpty) {
-      return (segmentsTobeDeleted, operatedRowCount)
+      return (segmentsTobeDeleted, operatedRowCount, false, "")
     }
     val carbonTable = CarbonEnv.getCarbonTable(databaseNameOp, tableName)(sparkSession)
     // update new status file
-    segmentsTobeDeleted =
+    val (segmentsTobeDeletedNew, isUpdateRequired, tblStatusWriteVersion) =
       checkAndUpdateStatusFiles(executorErrors,
-        res, carbonTable, timestamp,
-        blockMappingVO, isUpdateOperation)
+        res, carbonTable, timestamp, blockMappingVO, isUpdateOperation)
+    segmentsTobeDeleted = segmentsTobeDeletedNew
 
     if (executorErrors.failureCauses == FailureCauses.NONE) {
       operatedRowCount = res.flatten.map(_._2._3).sum
     }
-    (segmentsTobeDeleted, operatedRowCount)
+    (segmentsTobeDeleted, operatedRowCount, isUpdateRequired, tblStatusWriteVersion)
   }
 
   /**
@@ -164,7 +164,8 @@ object DeleteExecution {
     CarbonUpdateUtil
       .createBlockDetailsMap(blockMappingVO, segmentUpdateStatusMngr)
     val metadataDetails = SegmentStatusManager.readTableStatusFile(
-      CarbonTablePath.getTableStatusFilePath(carbonTable.getTablePath))
+      CarbonTablePath.getTableStatusFilePath(carbonTable.getTablePath,
+        carbonTable.getTableStatusVersion))
     val isStandardTable = CarbonUtil.isStandardCarbonTable(carbonTable)
     val rowContRdd =
       sparkSession.sparkContext.parallelize(
@@ -360,7 +361,9 @@ object DeleteExecution {
       carbonTable: CarbonTable,
       timestamp: String,
       blockMappingVO: BlockMappingVO,
-      isUpdateOperation: Boolean): Seq[Segment] = {
+      isUpdateOperation: Boolean): (Seq[Segment], Boolean, String) = {
+    var isUpdateRequired = false
+    var tblStatusWriteVersion = "";
     val blockUpdateDetailsList = new util.ArrayList[SegmentUpdateDetails]()
     val segmentDetails = new util.HashSet[Segment]()
     res.foreach(resultOfSeg => resultOfSeg.foreach(
@@ -388,7 +391,7 @@ object DeleteExecution {
             executorErrors.errorMsg = errorMsg
           }
           LOGGER.error(errorMsg)
-          return Seq.empty[Segment]
+          return (Seq.empty[Segment], isUpdateRequired, tblStatusWriteVersion)
         }
       }))
 
@@ -399,17 +402,21 @@ object DeleteExecution {
 
     // this is delete flow so no need of putting timestamp in the status file.
     if (CarbonUpdateUtil
-          .updateSegmentStatus(blockUpdateDetailsList, carbonTable, timestamp, false, false) &&
-        CarbonUpdateUtil
-          .updateTableMetadataStatus(segmentDetails,
-            carbonTable,
-            timestamp,
-            !isUpdateOperation,
-            !isUpdateOperation,
-            listOfSegmentToBeMarkedDeleted)
-    ) {
-      LOGGER.info(s"Delete data operation is successful for " +
-                  s"${ carbonTable.getDatabaseName }.${ carbonTable.getTableName }")
+      .updateSegmentStatus(blockUpdateDetailsList, carbonTable, timestamp, false, false)) {
+      val tuple = CarbonUpdateUtil
+        .updateTableMetadataStatus(segmentDetails,
+          carbonTable,
+          timestamp,
+          !isUpdateOperation,
+          !isUpdateOperation,
+          listOfSegmentToBeMarkedDeleted,
+          "")
+      if (tuple.get("status").toBoolean) {
+        tblStatusWriteVersion = tuple.getOrDefault("tblStatusWriteVersion", "")
+        isUpdateRequired = true
+        LOGGER.info(s"Delete data operation is successful for " +
+                    s"${ carbonTable.getDatabaseName }.${ carbonTable.getTableName }")
+      }
     } else {
       // In case of failure , clean all related delete delta files
       CarbonUpdateUtil.cleanStaleDeltaFiles(carbonTable, timestamp)
@@ -419,7 +426,7 @@ object DeleteExecution {
       executorErrors.failureCauses = FailureCauses.STATUS_FILE_UPDATION_FAILURE
       executorErrors.errorMsg = errorMessage
     }
-    segmentsTobeDeleted
+    (segmentsTobeDeleted, isUpdateRequired, tblStatusWriteVersion)
   }
 
   // all or none : update status file, only if complete delete operation is successful.
@@ -475,8 +482,10 @@ object DeleteExecution {
         carbonTable.getDatabaseName, carbonTable.getTableName)
       val prePrimingEnabled = CarbonProperties.getInstance().isIndexServerPrePrimingEnabled()
       if (indexServerEnabled && prePrimingEnabled) {
-        val readCommittedScope = new TableStatusReadCommittedScope(AbsoluteTableIdentifier.from(
-          carbonTable.getTablePath), FileFactory.getConfiguration)
+        val readCommittedScope = new TableStatusReadCommittedScope(
+          AbsoluteTableIdentifier.from(carbonTable.getTablePath),
+          FileFactory.getConfiguration,
+          carbonTable.getTableStatusVersion)
         deletedSegments.foreach(_.setReadCommittedScope(readCommittedScope))
         LOGGER.info(s"Loading segments for table: ${ carbonTable.getTableName } in the cache")
         val indexServerLoadEvent: IndexServerLoadEvent =
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/merge/CarbonMergeDataSetUtil.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/merge/CarbonMergeDataSetUtil.scala
index 23a1227c46..96e0b54f2f 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/merge/CarbonMergeDataSetUtil.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/merge/CarbonMergeDataSetUtil.scala
@@ -393,7 +393,8 @@ object CarbonMergeDataSetUtil {
     // blocklets of the target table. If the index server disabled, just call the getSplits of
     // the driver side to cache and get the splits. These CarbonInputSplits basically contain
     // the filePaths and the min max of each columns.
-    val ssm = new SegmentStatusManager(targetCarbonTable.getAbsoluteTableIdentifier)
+    val ssm = new SegmentStatusManager(targetCarbonTable.getAbsoluteTableIdentifier,
+      targetCarbonTable.getTableStatusVersion)
     val validSegments = ssm.getValidAndInvalidSegments.getValidSegments
     val defaultIndex = IndexStoreManager.getInstance.getDefaultIndex(targetCarbonTable)
 
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/merge/MergeUtil.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/merge/MergeUtil.scala
index 3841b38acd..5c63ea31e3 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/merge/MergeUtil.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/merge/MergeUtil.scala
@@ -114,14 +114,16 @@ object MergeUtil {
    */
   def updateStatusIfJustDeleteOperation(targetCarbonTable: CarbonTable,
       factTimestamp: Long): Boolean = {
-    val loadMetaDataDetails = SegmentStatusManager.readTableStatusFile(CarbonTablePath
-      .getTableStatusFilePath(targetCarbonTable.getTablePath))
+    val loadMetaDataDetails = SegmentStatusManager.readTableStatusFile(
+      CarbonTablePath.getTableStatusFilePath(targetCarbonTable.getTablePath,
+        targetCarbonTable.getTableStatusVersion))
     CarbonUpdateUtil.updateTableMetadataStatus(loadMetaDataDetails.map(loadMetadataDetail =>
       new Segment(loadMetadataDetail.getMergedLoadName,
         loadMetadataDetail.getSegmentFile)).toSet.asJava,
       targetCarbonTable,
       factTimestamp.toString,
       true,
-      true, new util.ArrayList[Segment]())
+      true, new util.ArrayList[Segment](),
+      targetCarbonTable.getTableStatusVersion).get("status").toBoolean
   }
 }
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableAddHivePartitionCommand.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableAddHivePartitionCommand.scala
index eaa62d829e..e77c6a9e26 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableAddHivePartitionCommand.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableAddHivePartitionCommand.scala
@@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
 import org.apache.spark.sql.execution.command.{AlterTableAddPartitionCommand, AlterTableDropPartitionCommand, AlterTableModel, AtomicRunnableCommand}
 import org.apache.spark.sql.execution.command.management.CommonLoadUtils
+import org.apache.spark.sql.hive.CarbonHiveIndexMetadataUtil
 import org.apache.spark.sql.optimizer.CarbonFilters
 
 import org.apache.carbondata.common.logging.LogServiceFactory
@@ -38,7 +39,7 @@ import org.apache.carbondata.core.indexstore.PartitionSpec
 import org.apache.carbondata.core.metadata.SegmentFileStore
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.statusmanager.{FileFormat, SegmentStatus}
-import org.apache.carbondata.core.util.{CarbonUtil, ObjectSerializationUtil}
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil, ObjectSerializationUtil}
 import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.events.{withEvents, AlterTableMergeIndexEvent, OperationContext, OperationListenerBus, PostAlterTableHivePartitionCommandEvent, PreAlterTableHivePartitionCommandEvent}
 import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
@@ -139,6 +140,8 @@ case class CarbonAlterTableAddHivePartitionCommand(
       if (hasIndexFiles) {
         // Create new entry in tablestatus file
         CarbonLoaderUtil.readAndUpdateLoadProgressInTableMeta(loadModel, false)
+        CarbonHiveIndexMetadataUtil.updateTableStatusVersion(table,
+          sparkSession, loadModel.getLatestTableStatusWriteVersion)
         // Normally, application will use Carbon SDK to write files into a partition folder, then
         // add the folder to partitioned carbon table.
         // If there are many threads writes to the same partition folder, there will be many
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropHivePartitionCommand.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropHivePartitionCommand.scala
index 2c6de2c5b1..d7d317f4f3 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropHivePartitionCommand.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropHivePartitionCommand.scala
@@ -26,6 +26,7 @@ import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
 import org.apache.spark.sql.execution.command.{AlterTableAddPartitionCommand, AlterTableDropPartitionCommand, AtomicRunnableCommand}
+import org.apache.spark.sql.hive.CarbonHiveIndexMetadataUtil
 import org.apache.spark.sql.parser.CarbonSparkSqlParserUtil
 import org.apache.spark.util.AlterTableUtil
 
@@ -146,8 +147,8 @@ case class CarbonAlterTableDropHivePartitionCommand(
         locksToBeAcquired)(sparkSession)
       // If normal table then set uuid to ""
       val uuid = "";
-      val segments = new SegmentStatusManager(table.getAbsoluteTableIdentifier)
-        .getValidAndInvalidSegments(table.isMV).getValidSegments
+      val segments = new SegmentStatusManager(table.getAbsoluteTableIdentifier,
+        table.getTableStatusVersion).getValidAndInvalidSegments(table.isMV).getValidSegments
       // First drop the partitions from partition mapper files of each segment
       val tuples = new CarbonDropPartitionRDD(sparkSession,
         table.getTablePath,
@@ -164,12 +165,15 @@ case class CarbonAlterTableDropHivePartitionCommand(
           tobeDeletedSegs.add(tobeDeleted.split(",")(0))
         }
       }
+      var tblStatusWriteVersion = ""
       withEvents(operationContext,
         AlterTableDropPartitionPreStatusEvent(table, sparkSession),
         AlterTableDropPartitionPostStatusEvent(table)) {
-        SegmentFileStore.commitDropPartitions(table, uniqueId, tobeUpdatedSegs, tobeDeletedSegs,
-          uuid)
+        tblStatusWriteVersion = SegmentFileStore.commitDropPartitions(table, uniqueId,
+          tobeUpdatedSegs, tobeDeletedSegs, tblStatusWriteVersion)
       }
+      CarbonHiveIndexMetadataUtil.updateTableStatusVersion(table,
+        sparkSession, tblStatusWriteVersion)
       IndexStoreManager.getInstance().clearIndex(table.getAbsoluteTableIdentifier)
       tobeCleanSegs.addAll(tobeUpdatedSegs)
       tobeCleanSegs.addAll(tobeDeletedSegs)
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonCreateTableLikeCommand.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonCreateTableLikeCommand.scala
index 8921601391..7be1215d97 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonCreateTableLikeCommand.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonCreateTableLikeCommand.scala
@@ -63,6 +63,9 @@ case class CarbonCreateTableLikeCommand(
     // remove mv related info from source table tblProperties
     dstTableSchema.getTableProperties.remove(CarbonCommonConstants.RELATED_MV_TABLES_MAP)
 
+    // remove table_status version property of source table
+    dstTableSchema.getTableProperties.remove("latestversion")
+
     val schemaEvol: SchemaEvolution = new SchemaEvolution
     val schEntryList: util.List[SchemaEvolutionEntry] = new util.ArrayList[SchemaEvolutionEntry]
     schemaEvol.setSchemaEvolutionEntryList(schEntryList)
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/datasources/SparkCarbonTableFormat.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/datasources/SparkCarbonTableFormat.scala
index 4a06bcedd7..b0d1809f3b 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/datasources/SparkCarbonTableFormat.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/datasources/SparkCarbonTableFormat.scala
@@ -93,6 +93,7 @@ with Serializable {
         CompressorFactory.getInstance().getCompressor.getName)
     model.setColumnCompressor(columnCompressor)
     model.setMetrics(new DataLoadMetrics())
+    model.setLatestTableStatusWriteVersion(options.getOrElse("latestversion", ""))
 
     val carbonProperty = CarbonProperties.getInstance()
     val optionsFinal = LoadOption.fillOptionWithDefaultValue(options.asJava)
@@ -174,7 +175,8 @@ with Serializable {
       if (!isLoadDetailsContainTheCurrentEntry(
         model.getLoadMetadataDetails.asScala.toArray, loadEntry)) {
         val details =
-          SegmentStatusManager.readLoadMetadata(CarbonTablePath.getMetadataPath(model.getTablePath))
+          SegmentStatusManager.readLoadMetadata(CarbonTablePath.getMetadataPath(model.getTablePath),
+            table.getTableStatusVersion)
         val list = new util.ArrayList[LoadMetadataDetails](details.toList.asJava)
         list.add(loadEntry)
         model.setLoadMetadataDetails(list)
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonSourceStrategy.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonSourceStrategy.scala
index 85044a283b..9564f446e7 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonSourceStrategy.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonSourceStrategy.scala
@@ -200,7 +200,9 @@ private[sql] object CarbonSourceStrategy extends SparkStrategy {
     val filterCondition = unhandledPredicates.reduceLeftOption(expressions.And)
 
     val readCommittedScope =
-      new TableStatusReadCommittedScope(table.identifier, FileFactory.getConfiguration)
+      new TableStatusReadCommittedScope(table.identifier,
+        FileFactory.getConfiguration,
+        table.carbonTable.getTableStatusVersion)
     val extraSegments = MixedFormatHandler.extraSegments(table.identifier, readCommittedScope)
     val extraRDD = MixedFormatHandler.extraRDD(relation, rawProjects, filterPredicates,
       readCommittedScope, table.identifier, extraSegments, vectorReaderEnabled())
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/strategy/MixedFormatHandler.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/strategy/MixedFormatHandler.scala
index 66b7215ff1..330cbfc472 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/strategy/MixedFormatHandler.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/strategy/MixedFormatHandler.scala
@@ -397,8 +397,8 @@ object MixedFormatHandler {
   /**
    * Returns true if any other non-carbon format segment exists
    */
-  def otherFormatSegmentsExist(metadataPath: String): Boolean = {
-    val allSegments = SegmentStatusManager.readLoadMetadata(metadataPath)
+  def otherFormatSegmentsExist(metadataPath: String, tableStatusVersion: String): Boolean = {
+    val allSegments = SegmentStatusManager.readLoadMetadata(metadataPath, tableStatusVersion)
     allSegments.exists(a => a.getFileFormat != null && !a.isCarbonFormat)
   }
 }
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
index a332b634c1..b7e9b9d3da 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
@@ -177,7 +177,7 @@ class CarbonAppendableStreamSink(
    */
   private def checkOrHandOffSegment(): Unit = {
     // get streaming segment, if not exists, create new streaming segment
-    val segmentId = StreamSegment.open(carbonTable)
+    val segmentId = StreamSegment.open(carbonTable, carbonLoadModel)
     if (segmentId.equals(currentSegmentId)) {
       val segmentDir = CarbonTablePath.getSegmentPath(carbonTable.getTablePath, currentSegmentId)
       if (segmentMaxSize <= StreamSegment.size(segmentDir)) {
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
index 3115308248..961380db40 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
@@ -325,6 +325,11 @@ class CarbonFileMetastore extends CarbonMetaStore {
               .getTableProperties
               .put("_external", parameters("isExternal"))
           }
+          if (parameters.contains("latestversion")) {
+            wrapperTableInfo.getFactTable
+              .getTableProperties
+              .put("latestversion", parameters("latestversion"))
+          }
           Some(wrapperTableInfo)
         } else {
           None
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonHiveIndexMetadataUtil.scala b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonHiveIndexMetadataUtil.scala
index b6fb6ee52e..24a05804e3 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonHiveIndexMetadataUtil.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonHiveIndexMetadataUtil.scala
@@ -16,6 +16,8 @@
  */
 package org.apache.spark.sql.hive
 
+import scala.collection.JavaConverters._
+
 import org.apache.hadoop.hive.ql.exec.UDF
 import org.apache.spark.sql.{CarbonEnv, SparkSession}
 import org.apache.spark.sql.catalyst.TableIdentifier
@@ -24,8 +26,10 @@ import org.apache.spark.sql.index.CarbonIndexUtil
 
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.metadata.CarbonMetadata
 import org.apache.carbondata.core.metadata.schema.indextable.IndexTableInfo
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.util.CarbonProperties
 
 
 /**
@@ -59,6 +63,33 @@ object CarbonHiveIndexMetadataUtil {
     }
   }
 
+  def updateTableStatusVersion(carbonTable: CarbonTable,
+      sparkSession: SparkSession,
+      latestVersion: String): Unit = {
+    val isMultiVersionEnabled = CarbonProperties.isTableStatusMultiVersionEnabled
+    if (isMultiVersionEnabled || carbonTable.getTableStatusVersion.nonEmpty) {
+      // save to hive table metadata and update carbon table
+      val sql =
+        s"""ALTER TABLE `${ carbonTable.getDatabaseName }`.`${ carbonTable.getTableName }`
+           | SET SERDEPROPERTIES ('latestversion'='$latestVersion')""".stripMargin
+      CarbonSessionCatalogUtil.getClient(sparkSession).runSqlHive(sql)
+      carbonTable.getTableInfo
+        .getFactTable
+        .getTableProperties
+        .put("latestversion", latestVersion)
+      var propkeys: Seq[String] = Seq.empty
+      propkeys = propkeys.:+("latestversion")
+      CarbonSessionCatalogUtil.alterTableProperties(
+        sparkSession,
+        new TableIdentifier(carbonTable.getTableName, Some(carbonTable.getDatabaseName)),
+        carbonTable.getTableInfo.getFactTable.getTableProperties.asScala.toMap,
+        propkeys)
+      refreshTable(carbonTable.getDatabaseName, carbonTable.getTableName, sparkSession)
+      CarbonMetadata.getInstance.removeTable(carbonTable.getDatabaseName, carbonTable.getTableName)
+      CarbonMetadata.getInstance.loadTableMetadata(carbonTable.getTableInfo)
+    }
+  }
+
   def refreshTable(dbName: String, tableName: String, sparkSession: SparkSession): Unit = {
     val tableWithDb = dbName + "." + tableName
     val tableIdent = sparkSession.sessionState.sqlParser.parseTableIdentifier(tableWithDb)
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonRelation.scala b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonRelation.scala
index 64f6da1e9d..8c4d454053 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonRelation.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonRelation.scala
@@ -183,10 +183,10 @@ case class CarbonRelation(
       }
     } else {
       val tableStatusNewLastUpdatedTime = SegmentStatusManager.getTableStatusLastModifiedTime(
-        carbonTable.getAbsoluteTableIdentifier)
+        carbonTable.getAbsoluteTableIdentifier, carbonTable.getTableStatusVersion)
       if (tableStatusLastUpdateTime != tableStatusNewLastUpdatedTime) {
-        val allSegments = new SegmentStatusManager(carbonTable.getAbsoluteTableIdentifier)
-          .getValidAndInvalidSegments(carbonTable.isMV)
+        val allSegments = new SegmentStatusManager(carbonTable.getAbsoluteTableIdentifier,
+          carbonTable.getTableStatusVersion).getValidAndInvalidSegments(carbonTable.isMV)
         if (allSegments.getValidSegments.isEmpty) {
           sizeInBytesLocalValue = 0L
         } else {
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonSessionUtil.scala b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonSessionUtil.scala
index c9f4c2a33e..9899617a1d 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonSessionUtil.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonSessionUtil.scala
@@ -115,7 +115,9 @@ object CarbonSessionUtil {
       sparkSession: SparkSession,
       carbonTable: CarbonTable): Seq[CatalogTablePartition] = {
     val allPartitions = PartitionCacheManager.get(PartitionCacheKey(carbonTable.getTableId,
-      carbonTable.getTablePath, CarbonUtil.getExpiration_time(carbonTable))).asScala
+      carbonTable.getTablePath,
+      CarbonUtil.getExpiration_time(carbonTable),
+      carbonTable.getTableStatusVersion)).asScala
     ExternalCatalogUtils.prunePartitionsByFilter(
       sparkSession.sessionState.catalog.getTableMetadata(TableIdentifier(carbonTable.getTableName,
         Some(carbonTable.getDatabaseName))),
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/index/CarbonIndexUtil.scala b/integration/spark/src/main/scala/org/apache/spark/sql/index/CarbonIndexUtil.scala
index 5e9e0da7b9..5c55e94b31 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/index/CarbonIndexUtil.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/index/CarbonIndexUtil.scala
@@ -151,7 +151,7 @@ object CarbonIndexUtil {
     .ArrayList[LoadMetadataDetails](
       factLoadMetadataDetails.size)
     val indexTableStatusDetailsArray: Array[LoadMetadataDetails] = SegmentStatusManager
-      .readLoadMetadata(indexTable.getMetadataPath)
+      .readLoadMetadata(indexTable.getMetadataPath, indexTable.getTableStatusVersion)
     if (null !=
         indexTableStatusDetailsArray) {
       for (loadMetadataDetails <- indexTableStatusDetailsArray) {
@@ -282,7 +282,8 @@ object CarbonIndexUtil {
 
     if (isLoadToFailedSISegments && null != failedLoadMetaDataDetils) {
       val metadata = CarbonInternalLoaderUtil
-        .getListOfValidSlices(SegmentStatusManager.readLoadMetadata(indexTable.getMetadataPath))
+        .getListOfValidSlices(SegmentStatusManager.readLoadMetadata(indexTable.getMetadataPath,
+          indexTable.getTableStatusVersion))
       segmentIdToLoadStartTimeMapping = CarbonInternalLoaderUtil
         .getSegmentToLoadStartTimeMapping(carbonLoadModel.getLoadMetadataDetails.asScala.toArray)
         .asScala
@@ -519,7 +520,7 @@ object CarbonIndexUtil {
       if (compactionLock.lockWithRetries()) {
         var mainTableDetails = try {
           SegmentStatusManager.readTableStatusFile(CarbonTablePath.getTableStatusFilePath(
-            carbonTable.getTablePath))
+            carbonTable.getTablePath, carbonTable.getTableStatusVersion))
         } catch {
           case exception: Exception =>
             if (!isLoadOrCompaction) {
@@ -533,7 +534,8 @@ object CarbonIndexUtil {
             loadMetaDataDetails => segments.get.contains(loadMetaDataDetails.getLoadName))
         }
         val siTblLoadMetadataDetails: Array[LoadMetadataDetails] =
-          SegmentStatusManager.readLoadMetadata(indexTable.getMetadataPath)
+          SegmentStatusManager.readLoadMetadata(indexTable.getMetadataPath,
+            indexTable.getTableStatusVersion)
         if (!CarbonInternalLoaderUtil.checkMainTableSegEqualToSISeg(
           mainTableDetails,
           siTblLoadMetadataDetails)) {
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParserUtil.scala b/integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParserUtil.scala
index 02763b8b26..d4b313c2dd 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParserUtil.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParserUtil.scala
@@ -309,6 +309,12 @@ object CarbonSparkSqlParserUtil {
     }
     tableInfo.setTablePath(identifier.getTablePath)
     tableInfo.setTransactionalTable(isTransactionalTable)
+    if (isTransactionalTable && isExternal) {
+      val tblStatusVersion = CarbonScalaUtil.getLatestTableStatusVersion(identifier.getTablePath)
+      if (tblStatusVersion.nonEmpty) {
+        tableInfo.getFactTable.getTableProperties.put("latestversion", tblStatusVersion)
+      }
+    }
     tableInfo
   }
 
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/command/SICreationCommand.scala b/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/command/SICreationCommand.scala
index 2c8dc08444..06e1b51c0b 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/command/SICreationCommand.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/command/SICreationCommand.scala
@@ -256,7 +256,8 @@ private[sql] case class CarbonCreateSecondaryIndexCommand(
       val indexTablePath = CarbonTablePath
         .getMetadataPath(absoluteTableIdentifier.getTablePath)
       val mainTblLoadMetadataDetails: Array[LoadMetadataDetails] =
-        SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath)
+        SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath,
+          carbonTable.getTableStatusVersion)
       var siTblLoadMetadataDetails: Array[LoadMetadataDetails] =
         SegmentStatusManager.readLoadMetadata(indexTablePath)
       if (isRegisterIndex) {
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/command/SILoadCommand.scala b/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/command/SILoadCommand.scala
index 361e9150ee..9bef2ae22a 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/command/SILoadCommand.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/command/SILoadCommand.scala
@@ -143,7 +143,8 @@ private[sql] case class LoadDataForSecondaryIndex(indexModel: IndexModel) extend
 
     def readTableStatusFile(model: CarbonLoadModel): Array[LoadMetadataDetails] = {
       val metadataPath = model.getCarbonDataLoadSchema.getCarbonTable.getMetadataPath
-      val details = SegmentStatusManager.readLoadMetadata(metadataPath)
+      val details = SegmentStatusManager.readLoadMetadata(metadataPath,
+        model.getCarbonDataLoadSchema.getCarbonTable.getTableStatusVersion)
       details
     }
 
@@ -156,7 +157,8 @@ private[sql] case class LoadDataForSecondaryIndex(indexModel: IndexModel) extend
       indexTable: CarbonTable): java.util.List[LoadMetadataDetails] = {
       val loadMetadataDetails: java.util.List[LoadMetadataDetails] = new java.util
         .ArrayList[LoadMetadataDetails]
-      val metadata = SegmentStatusManager.readLoadMetadata(indexTable.getMetadataPath).toSeq
+      val metadata = SegmentStatusManager.readLoadMetadata(indexTable.getMetadataPath,
+        indexTable.getTableStatusVersion).toSeq
         .map(loadMetadataDetail => loadMetadataDetail.getLoadName)
       details.foreach(loadMetadataDetail => {
         if (!metadata.contains(loadMetadataDetail.getLoadName) &&
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/command/SIRebuildSegmentRunner.scala b/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/command/SIRebuildSegmentRunner.scala
index a4fd810c9d..f024af3815 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/command/SIRebuildSegmentRunner.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/command/SIRebuildSegmentRunner.scala
@@ -71,7 +71,8 @@ case class SIRebuildSegmentRunner(
     // check if the list of given segments in the command are valid
     val segmentIds = segments.getOrElse(List.empty)
     if (segmentIds.nonEmpty) {
-      val segmentStatusManager = new SegmentStatusManager(indexTable.getAbsoluteTableIdentifier)
+      val segmentStatusManager = new SegmentStatusManager(indexTable.getAbsoluteTableIdentifier,
+        indexTable.getTableStatusVersion)
       val validSegments = segmentStatusManager.getValidAndInvalidSegments.getValidSegments.asScala
         .map(_.getSegmentNo)
       segmentIds.foreach { segmentId =>
@@ -102,7 +103,8 @@ case class SIRebuildSegmentRunner(
         withEvents(
           LoadTableSIPreExecutionEvent(sparkSession, identifier, null, indexTable),
           LoadTableSIPostExecutionEvent(sparkSession, identifier, null, indexTable)) {
-          SegmentStatusManager.readLoadMetadata(parentTable.getMetadataPath) collect {
+          SegmentStatusManager.readLoadMetadata(parentTable.getMetadataPath,
+            parentTable.getTableStatusVersion) collect {
             case loadDetails if null == segmentList ||
               segmentList.contains(loadDetails.getLoadName) =>
               segmentFileNameMap.put(
@@ -110,7 +112,7 @@ case class SIRebuildSegmentRunner(
           }
 
           val loadMetadataDetails = SegmentStatusManager
-            .readLoadMetadata(indexTable.getMetadataPath)
+            .readLoadMetadata(indexTable.getMetadataPath, indexTable.getTableStatusVersion)
             .filter(loadMetadataDetail =>
               (null == segmentList || segmentList.contains(loadMetadataDetail.getLoadName)) &&
                 (loadMetadataDetail.getSegmentStatus == SegmentStatus.SUCCESS ||
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/events/CleanFilesPostEventListener.scala b/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/events/CleanFilesPostEventListener.scala
index 2c94609087..ef2561b115 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/events/CleanFilesPostEventListener.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/events/CleanFilesPostEventListener.scala
@@ -24,6 +24,7 @@ import org.apache.spark.internal.Logging
 import org.apache.spark.sql.SparkSession
 import org.apache.spark.sql.catalyst.expressions.Expression
 import org.apache.spark.sql.execution.command.management.CarbonCleanFilesCommand
+import org.apache.spark.sql.hive.CarbonHiveIndexMetadataUtil
 import org.apache.spark.sql.index.CarbonIndexUtil
 import org.apache.spark.sql.optimizer.CarbonFilters
 
@@ -34,7 +35,7 @@ import org.apache.carbondata.core.indexstore.PartitionSpec
 import org.apache.carbondata.core.locks.{CarbonLockFactory, ICarbonLock, LockUsage}
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.statusmanager.{SegmentStatus, SegmentStatusManager}
-import org.apache.carbondata.core.util.CarbonUtil
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
 import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.events.{CleanFilesPostEvent, Event, OperationContext, OperationEventListener}
 import org.apache.carbondata.view.MVManagerInSpark
@@ -76,9 +77,14 @@ class CleanFilesPostEventListener extends OperationEventListener with Logging {
         Seq.empty[Expression],
         sparkSession,
         indexTable)
-      SegmentStatusManager.deleteLoadsAndUpdateMetadata(
+      val newTblStatusVersion = SegmentStatusManager.deleteLoadsAndUpdateMetadata(
         indexTable, isForceDelete, partitions.map(_.asJava).orNull, cleanStaleInProgress,
         true)
+      if (newTblStatusVersion.nonEmpty) {
+        // if clean files update is complete, then update the table status version to index table
+        CarbonHiveIndexMetadataUtil.updateTableStatusVersion(indexTable,
+          sparkSession, newTblStatusVersion)
+      }
       cleanUpUnwantedSegmentsOfSIAndUpdateMetadata(indexTable, carbonTable)
     }
   }
@@ -116,10 +122,14 @@ class CleanFilesPostEventListener extends OperationEventListener with Logging {
       indexTableLocked = indexTableStatusLock.lockWithRetries()
       if (mainTableLocked && indexTableLocked) {
         val mainTableMetadataDetails =
-          SegmentStatusManager.readLoadMetadata(mainTable.getMetadataPath).toSet ++
+          SegmentStatusManager
+            .readLoadMetadata(mainTable.getMetadataPath, mainTable.getTableStatusVersion)
+            .toSet ++
           SegmentStatusManager.readLoadHistoryMetadata(mainTable.getMetadataPath).toSet
         val indexTableMetadataDetails =
-          SegmentStatusManager.readLoadMetadata(indexTable.getMetadataPath).toSet
+          SegmentStatusManager
+            .readLoadMetadata(indexTable.getMetadataPath, indexTable.getTableStatusVersion)
+            .toSet
         val segToStatusMap = mainTableMetadataDetails
           .map(detail => detail.getLoadName -> detail.getSegmentStatus).toMap
 
@@ -141,9 +151,9 @@ class CleanFilesPostEventListener extends OperationEventListener with Logging {
           detail.setSegmentStatus(segToStatusMap(detail.getLoadName))
           detail.setVisibility("false")
         }
-        SegmentStatusManager.writeLoadDetailsIntoFile(
-          indexTable.getMetadataPath + CarbonCommonConstants.FILE_SEPARATOR +
-            CarbonTablePath.TABLE_STATUS_FILE, indexTableMetadataDetails.toArray)
+        SegmentStatusManager.writeLoadDetailsIntoFile(CarbonTablePath.getTableStatusFilePath(
+          indexTable.getTablePath, indexTable.getTableStatusVersion),
+          indexTableMetadataDetails.toArray)
       } else {
         LOGGER.error("Unable to get the lock file for main/Index table. Please try again later")
       }
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/events/CreateCarbonRelationEventListener.scala b/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/events/CreateCarbonRelationEventListener.scala
index 97c68ff804..87c1fadbf2 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/events/CreateCarbonRelationEventListener.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/events/CreateCarbonRelationEventListener.scala
@@ -39,6 +39,7 @@ class CreateCarbonRelationEventListener extends OperationEventListener with Logg
         val databaseName = createCarbonRelationPostEvent.carbonTable.getDatabaseName
         val tableName = createCarbonRelationPostEvent.carbonTable.getTableName
         val sparkSession = createCarbonRelationPostEvent.sparkSession
+        CarbonInternalMetastore.refreshTableStatusVersion(carbonTable)(sparkSession)
         CarbonInternalMetastore
           .refreshIndexInfo(databaseName,
             tableName,
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/events/DeleteSegmentByDateListener.scala b/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/events/DeleteSegmentByDateListener.scala
index cdb0ad5c91..4f8bf1a158 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/events/DeleteSegmentByDateListener.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/events/DeleteSegmentByDateListener.scala
@@ -50,8 +50,8 @@ class DeleteSegmentByDateListener extends OperationEventListener with Logging {
           val table = metastore
             .lookupRelation(Some(carbonTable.getDatabaseName), tableName)(sparkSession)
             .asInstanceOf[CarbonRelation].carbonTable
-          CarbonStore
-            .deleteLoadByDate(loadDates, carbonTable.getDatabaseName, table.getTableName, table)
+          CarbonStore.deleteLoadByDate(loadDates, carbonTable.getDatabaseName,
+            table.getTableName, table, sparkSession)
         }
     }
   }
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/events/DeleteSegmentByIdListener.scala b/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/events/DeleteSegmentByIdListener.scala
index 3f76f74731..5253b49b80 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/events/DeleteSegmentByIdListener.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/events/DeleteSegmentByIdListener.scala
@@ -23,12 +23,12 @@ import org.apache.log4j.Logger
 import org.apache.spark.internal.Logging
 import org.apache.spark.sql.CarbonEnv
 import org.apache.spark.sql.hive.CarbonRelation
-import org.apache.spark.sql.index.CarbonIndexUtil
 
 import org.apache.carbondata.api.CarbonStore
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.metadata.index.IndexType
+import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.events.{DeleteSegmentByIdPostEvent, Event, OperationContext, OperationEventListener}
 
@@ -55,12 +55,13 @@ class DeleteSegmentByIdListener extends OperationEventListener with Logging {
             val table = metastore
               .lookupRelation(Some(carbonTable.getDatabaseName), tableName)(sparkSession)
               .asInstanceOf[CarbonRelation].carbonTable
-            val tableStatusFilePath = CarbonTablePath.getTableStatusFilePath(table.getTablePath)
+            val tableStatusFilePath = CarbonTablePath.getTableStatusFilePath(table.getTablePath,
+              table.getTableStatusVersion)
             // this check is added to verify if the table status file for the index table exists
             // or not. Delete on index tables is only to be called if the table status file exists.
             if (FileFactory.isFileExist(tableStatusFilePath)) {
-              CarbonStore
-                .deleteLoadById(loadIds, carbonTable.getDatabaseName, table.getTableName, table)
+              CarbonStore.deleteLoadById(loadIds, carbonTable.getDatabaseName,
+                table.getTableName, table, sparkSession)
             }
           }
         }
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/hive/CarbonInternalMetastore.scala b/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/hive/CarbonInternalMetastore.scala
index 04c6ecf35a..4144963ff0 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/hive/CarbonInternalMetastore.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/hive/CarbonInternalMetastore.scala
@@ -134,6 +134,20 @@ object CarbonInternalMetastore {
     }
   }
 
+  def refreshTableStatusVersion(carbonTable: CarbonTable)(sparkSession: SparkSession): Unit = {
+    if (!carbonTable.isTransactionalTable || carbonTable.isExternalTable) {
+      return
+    }
+    if (null == carbonTable.getTableInfo.getFactTable.getTableProperties.get("latestversion")) {
+      val hiveTable = sparkSession.sessionState
+        .catalog.getTableMetadata(TableIdentifier(carbonTable.getTableName,
+        Some(carbonTable.getDatabaseName)))
+      val version = hiveTable.storage.properties.getOrElse(
+        "latestversion", "")
+      carbonTable.getTableInfo.getFactTable.getTableProperties.put("latestversion", version)
+    }
+  }
+
   def refreshIndexInfo(dbName: String, tableName: String,
       carbonTable: CarbonTable, needLock: Boolean = true)(sparkSession: SparkSession): Unit = {
     // check if secondary index table exists
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/joins/BroadCastSIFilterPushJoin.scala b/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/joins/BroadCastSIFilterPushJoin.scala
index eeda653d58..527720aecc 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/joins/BroadCastSIFilterPushJoin.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/joins/BroadCastSIFilterPushJoin.scala
@@ -89,10 +89,11 @@ case class BroadCastSIFilterPushJoin(
     if (partitions.nonEmpty && secondaryIndexRDD.nonEmpty) {
       secondaryIndexRDD.foreach {
         case value: CarbonScanRDD[InternalRow] =>
+          val tblStatusVersion = value.getTableInfo.getFactTable
+            .getTableProperties.getOrDefault("latestversion", "")
           val siSegments = SegmentStatusManager
             .readLoadMetadata(CarbonTablePath.getMetadataPath(value
-              .getTableInfo
-              .getTablePath))
+              .getTableInfo.getTablePath), tblStatusVersion)
               .filter(loadMetadataDetail =>
                 loadMetadataDetail.getSegmentStatus == SegmentStatus.SUCCESS
                   || loadMetadataDetail.getSegmentStatus == SegmentStatus.MARKED_FOR_UPDATE
@@ -272,7 +273,7 @@ object BroadCastSIFilterPushJoin {
     setQuerySegmentForIndexTable(job.getConfiguration, carbonTable)
     val identifier: AbsoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
     val readCommittedScope: ReadCommittedScope = carbonTableInputFormat.getReadCommitted(job,
-      identifier)
+      carbonTable.getAbsoluteTableIdentifier, carbonTable.getTableStatusVersion)
     val segmentsToAccess: Array[Segment] = carbonTableInputFormat.getSegmentsToAccess(job,
       readCommittedScope)
     val segmentsToAccessSet: util.Set[Segment] = new util.HashSet[Segment]
@@ -280,7 +281,8 @@ object BroadCastSIFilterPushJoin {
       segmentsToAccessSet.add(segId)
     }
     // get all valid segments and set them into the configuration
-    val segmentStatusManager: SegmentStatusManager = new SegmentStatusManager(identifier)
+    val segmentStatusManager: SegmentStatusManager = new SegmentStatusManager(identifier,
+      carbonTable.getTableStatusVersion)
     val segments: SegmentStatusManager.ValidAndInvalidSegmentsInfo = segmentStatusManager
       .getValidAndInvalidSegments(carbonTable.isMV)
     val validSegments: util.List[Segment] = segments.getValidSegments
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/load/Compactor.scala b/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/load/Compactor.scala
index ca7768a5ac..2000b56887 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/load/Compactor.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/load/Compactor.scala
@@ -109,8 +109,8 @@ object Compactor {
           indexCarbonTable.getTablePath,
           indexCarbonTable, mergeIndexProperty = false)
 
-        val loadMetadataDetails = SegmentStatusManager
-          .readLoadMetadata(indexCarbonTable.getMetadataPath)
+        val loadMetadataDetails = SegmentStatusManager.readLoadMetadata(
+          indexCarbonTable.getMetadataPath, indexCarbonTable.getTableStatusVersion)
           .filter(loadMetadataDetail => validSegments.head
             .equalsIgnoreCase(loadMetadataDetail.getLoadName))
 
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/rdd/SecondaryIndexCreator.scala b/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/rdd/SecondaryIndexCreator.scala
index eafbb3ad8d..674525ba4c 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/rdd/SecondaryIndexCreator.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/rdd/SecondaryIndexCreator.scala
@@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.analysis.{UnresolvedAlias, UnresolvedFuncti
 import org.apache.spark.sql.catalyst.expressions.Alias
 import org.apache.spark.sql.catalyst.plans.logical.Project
 import org.apache.spark.sql.execution.command.ExecutionErrors
+import org.apache.spark.sql.hive.CarbonHiveIndexMetadataUtil
 import org.apache.spark.sql.index.CarbonIndexUtil
 import org.apache.spark.sql.secondaryindex.command.SecondaryIndexModel
 import org.apache.spark.sql.secondaryindex.events.{LoadTableSIPostExecutionEvent, LoadTableSIPreExecutionEvent}
@@ -142,7 +143,7 @@ object SecondaryIndexCreator {
       }
       FileInternalUtil
         .updateTableStatus(validSegmentList,
-          secondaryIndexModel.carbonLoadModel.getDatabaseName,
+          secondaryIndexModel.carbonLoadModel,
           secondaryIndexModel.secondaryIndex.indexName,
           segmentStatus,
           secondaryIndexModel.segmentIdToLoadStartTimeMapping,
@@ -151,6 +152,9 @@ object SecondaryIndexCreator {
             String](),
           indexCarbonTable,
           sc.sparkSession)
+      CarbonHiveIndexMetadataUtil.updateTableStatusVersion(indexCarbonTable,
+        sc.sparkSession,
+        secondaryIndexModel.carbonLoadModel.getLatestTableStatusWriteVersion)
       var execInstance = "1"
       // in case of non dynamic executor allocation, number of executors are fixed.
       if (sc.sparkContext.getConf.contains("spark.executor.instances")) {
@@ -347,8 +351,8 @@ object SecondaryIndexCreator {
           indexCarbonTable.getTablePath,
           indexCarbonTable, mergeIndexProperty = false)
 
-        val loadMetadataDetails = SegmentStatusManager
-          .readLoadMetadata(indexCarbonTable.getMetadataPath)
+        val loadMetadataDetails = SegmentStatusManager.readLoadMetadata(
+          indexCarbonTable.getMetadataPath, indexCarbonTable.getTableStatusVersion)
           .filter(loadMetadataDetail => successSISegments.contains(loadMetadataDetail.getLoadName))
 
         val carbonLoadModelForMergeDataFiles = SecondaryIndexUtil
@@ -357,6 +361,8 @@ object SecondaryIndexCreator {
             System.currentTimeMillis(),
             CarbonIndexUtil
               .getCompressorForIndexTable(indexCarbonTable, secondaryIndexModel.carbonTable))
+        carbonLoadModelForMergeDataFiles.setLatestTableStatusWriteVersion(secondaryIndexModel
+          .carbonLoadModel.getLatestTableStatusWriteVersion)
 
         // merge the data files of the loaded segments and take care of
         // merging the index files inside this if needed
@@ -366,14 +372,14 @@ object SecondaryIndexCreator {
             loadMetadataDetails.toList.asJava, carbonLoadModelForMergeDataFiles)(sc)
 
         if (isInsertOverwrite) {
-          val overriddenSegments = SegmentStatusManager
-          .readLoadMetadata(indexCarbonTable.getMetadataPath)
+          val overriddenSegments = SegmentStatusManager.readLoadMetadata(
+            indexCarbonTable.getMetadataPath, indexCarbonTable.getTableStatusVersion)
             .filter(loadMetadata => !successSISegments.contains(loadMetadata.getLoadName))
             .map(_.getLoadName).toList
           FileInternalUtil
             .updateTableStatus(
               overriddenSegments,
-              secondaryIndexModel.carbonLoadModel.getDatabaseName,
+              secondaryIndexModel.carbonLoadModel,
               secondaryIndexModel.secondaryIndex.indexName,
               SegmentStatus.MARKED_FOR_DELETE,
               secondaryIndexModel.segmentIdToLoadStartTimeMapping,
@@ -389,7 +395,7 @@ object SecondaryIndexCreator {
           }
           tableStatusUpdateForSuccess = FileInternalUtil.updateTableStatus(
             successSISegments,
-            secondaryIndexModel.carbonLoadModel.getDatabaseName,
+            secondaryIndexModel.carbonLoadModel,
             secondaryIndexModel.secondaryIndex.indexName,
             SegmentStatus.SUCCESS,
             secondaryIndexModel.segmentIdToLoadStartTimeMapping,
@@ -418,7 +424,7 @@ object SecondaryIndexCreator {
       if (failedSISegments.nonEmpty && !isCompactionCall) {
         tableStatusUpdateForFailure = FileInternalUtil.updateTableStatus(
           failedSISegments,
-          secondaryIndexModel.carbonLoadModel.getDatabaseName,
+          secondaryIndexModel.carbonLoadModel,
           secondaryIndexModel.secondaryIndex.indexName,
           SegmentStatus.MARKED_FOR_DELETE,
           secondaryIndexModel.segmentIdToLoadStartTimeMapping,
@@ -431,6 +437,10 @@ object SecondaryIndexCreator {
         LOGGER.error("Dataload to secondary index creation has failed")
       }
 
+      CarbonHiveIndexMetadataUtil.updateTableStatusVersion(indexCarbonTable,
+        secondaryIndexModel.sqlContext.sparkSession,
+        secondaryIndexModel.carbonLoadModel.getLatestTableStatusWriteVersion)
+
       if (!isCompactionCall) {
         val loadTableSIPostExecutionEvent: LoadTableSIPostExecutionEvent =
           LoadTableSIPostExecutionEvent(sc.sparkSession,
@@ -454,7 +464,7 @@ object SecondaryIndexCreator {
         }
         FileInternalUtil
           .updateTableStatus(validSegmentList,
-            secondaryIndexModel.carbonLoadModel.getDatabaseName,
+            secondaryIndexModel.carbonLoadModel,
             secondaryIndexModel.secondaryIndex.indexName,
             SegmentStatus.MARKED_FOR_DELETE,
             secondaryIndexModel.segmentIdToLoadStartTimeMapping,
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/util/FileInternalUtil.scala b/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/util/FileInternalUtil.scala
index 1a849b6300..08cc9d3008 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/util/FileInternalUtil.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/util/FileInternalUtil.scala
@@ -28,6 +28,7 @@ import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil
 import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatus}
 import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel
 import org.apache.carbondata.processing.util.CarbonLoaderUtil
 
 /**
@@ -37,7 +38,7 @@ object FileInternalUtil {
 
   def updateTableStatus(
     validSegments: List[String],
-    databaseName: String,
+    carbonLoadModel: CarbonLoadModel,
     tableName: String,
     loadStatus: SegmentStatus,
     segmentIdToLoadStartTimeMapping: scala.collection.mutable.Map[String, java.lang.Long],
@@ -74,6 +75,7 @@ object FileInternalUtil {
       CarbonLoaderUtil.addDataIndexSizeIntoMetaEntry(loadMetadataDetail, segmentId, carbonTable)
       loadMetadataDetailsList +:= loadMetadataDetail
     }
+    val isRebuiltSegments = rebuiltSegments.nonEmpty
     val indexTables = CarbonIndexUtil
       .getIndexCarbonTables(carbonTable, sparkSession)
     val status = CarbonInternalLoaderUtil.recordLoadMetadata(
@@ -81,8 +83,10 @@ object FileInternalUtil {
       validSegments.asJava,
       carbonTable,
       indexTables.toList.asJava,
-      databaseName,
-      tableName
+      carbonLoadModel,
+      tableName,
+      loadStatus,
+      isRebuiltSegments
     )
     status
   }
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/util/SecondaryIndexUtil.scala b/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/util/SecondaryIndexUtil.scala
index 63bd726aaa..b58e1d304e 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/util/SecondaryIndexUtil.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/util/SecondaryIndexUtil.scala
@@ -184,8 +184,8 @@ object SecondaryIndexUtil {
       maxSegmentColumnSchemaList = null,
       currentPartitions = partitions)
     carbonLoadModel.setTablePath(carbonMergerMapping.hdfsStoreLocation)
-    carbonLoadModel.setLoadMetadataDetails(
-      SegmentStatusManager.readLoadMetadata(indexCarbonTable.getMetadataPath).toList.asJava)
+    carbonLoadModel.setLoadMetadataDetails(SegmentStatusManager.readLoadMetadata(
+      indexCarbonTable.getMetadataPath, indexCarbonTable.getTableStatusVersion).toList.asJava)
 
     val mergedSegments: util.Set[LoadMetadataDetails] = new util.HashSet[LoadMetadataDetails]()
     var rebuiltSegments: Set[String] = Set[String]()
@@ -291,7 +291,7 @@ object SecondaryIndexUtil {
           if (compactionType == null) {
             FileInternalUtil.updateTableStatus(
               rebuiltSegments.toList,
-              carbonLoadModel.getDatabaseName,
+              carbonLoadModel,
               indexCarbonTable.getTableName,
               SegmentStatus.SUCCESS,
               segmentToLoadStartTimeMap,
@@ -502,20 +502,23 @@ object SecondaryIndexUtil {
    */
   def updateTableStatusForIndexTables(parentCarbonTable: CarbonTable,
       indexTables: java.util.List[CarbonTable]): Unit = {
-    val loadFolderDetailsArrayMainTable =
-      SegmentStatusManager.readLoadMetadata(parentCarbonTable.getMetadataPath)
+    val loadFolderDetailsArrayMainTable = SegmentStatusManager.readLoadMetadata(
+      parentCarbonTable.getMetadataPath, parentCarbonTable.getTableStatusVersion)
     indexTables.asScala.foreach { indexTable =>
       val statusLock =
-        new SegmentStatusManager(indexTable.getAbsoluteTableIdentifier).getTableStatusLock
+        new SegmentStatusManager(indexTable.getAbsoluteTableIdentifier,
+          parentCarbonTable.getTableStatusVersion).getTableStatusLock
       try {
         if (statusLock.lockWithRetries()) {
-          val tableStatusFilePath = CarbonTablePath.getTableStatusFilePath(indexTable.getTablePath)
+          val tableStatusFilePath = CarbonTablePath.getTableStatusFilePath(indexTable.getTablePath,
+            indexTable.getTableStatusVersion)
           if (CarbonUtil.isFileExists(tableStatusFilePath)) {
             val loadFolderDetailsArray = SegmentStatusManager.readLoadMetadata(indexTable
-              .getMetadataPath);
+              .getMetadataPath, indexTable.getTableStatusVersion)
             if (null != loadFolderDetailsArray && loadFolderDetailsArray.nonEmpty) {
               SegmentStatusManager.writeLoadDetailsIntoFile(
-                CarbonTablePath.getTableStatusFilePath(indexTable.getTablePath),
+                CarbonTablePath.getTableStatusFilePath(indexTable.getTablePath,
+                  indexTable.getTableStatusVersion),
                 updateTimeStampForIndexTable(loadFolderDetailsArrayMainTable,
                   loadFolderDetailsArray))
             }
@@ -636,7 +639,8 @@ object SecondaryIndexUtil {
     val carbonDataLoadSchema = new CarbonDataLoadSchema(carbonTable)
     carbonLoadModel.setCarbonDataLoadSchema(carbonDataLoadSchema)
     val compactionSize = CarbonDataMergerUtil.getCompactionSize(compactionType, carbonLoadModel)
-    val segments = SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath)
+    val segments = SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath,
+      carbonTable.getTableStatusVersion)
     (carbonLoadModel, compactionSize, segments)
   }
 
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/test/util/QueryTest.scala b/integration/spark/src/main/scala/org/apache/spark/sql/test/util/QueryTest.scala
index adcfae15c6..771ab3ed7e 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/test/util/QueryTest.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/test/util/QueryTest.scala
@@ -174,10 +174,11 @@ class QueryTest extends PlanTest {
   }
 
   def removeSegmentEntryFromTableStatusFile(carbonTable: CarbonTable, segmentNo: String) : Unit = {
-    val details = SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath)
+    val details = SegmentStatusManager.readLoadMetadata(
+      carbonTable.getMetadataPath, carbonTable.getTableStatusVersion)
       .filter(as => as.getLoadName != segmentNo)
     SegmentStatusManager.writeLoadDetailsIntoFile(CarbonTablePath.getTableStatusFilePath(
-      carbonTable.getTablePath), details)
+      carbonTable.getTablePath, carbonTable.getTableStatusVersion), details)
   }
 
   def printTable(table: String, database: String = "default"): Unit = {
diff --git a/integration/spark/src/main/scala/org/apache/spark/util/CleanFiles.scala b/integration/spark/src/main/scala/org/apache/spark/util/CleanFiles.scala
index 40bd805bc4..fd77127be6 100644
--- a/integration/spark/src/main/scala/org/apache/spark/util/CleanFiles.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/util/CleanFiles.scala
@@ -41,6 +41,7 @@ object CleanFiles {
       isForceDeletion,
       cleanStaleInProgress,
       false,
+      spark,
       CarbonFilters.getPartitions(Seq.empty[Expression], spark, carbonTable))
   }
 
diff --git a/integration/spark/src/main/scala/org/apache/spark/util/DeleteSegmentByDate.scala b/integration/spark/src/main/scala/org/apache/spark/util/DeleteSegmentByDate.scala
index 6149421268..f87a9732c8 100644
--- a/integration/spark/src/main/scala/org/apache/spark/util/DeleteSegmentByDate.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/util/DeleteSegmentByDate.scala
@@ -17,7 +17,6 @@
 package org.apache.spark.util
 
 import org.apache.spark.sql.{CarbonEnv, SparkSession}
-import org.apache.spark.sql.catalyst.TableIdentifier
 
 import org.apache.carbondata.api.CarbonStore
 
@@ -31,7 +30,8 @@ object DeleteSegmentByDate {
       dateValue: String): Unit = {
     TableAPIUtil.validateTableExists(spark, dbName, tableName)
     val carbonTable = CarbonEnv.getCarbonTable(Some(dbName), tableName)(spark)
-    CarbonStore.deleteLoadByDate(dateValue, dbName, tableName, carbonTable)
+    CarbonStore.deleteLoadByDate(dateValue, dbName, tableName,
+      carbonTable, spark)
   }
 
   def main(args: Array[String]): Unit = {
diff --git a/integration/spark/src/main/scala/org/apache/spark/util/DeleteSegmentById.scala b/integration/spark/src/main/scala/org/apache/spark/util/DeleteSegmentById.scala
index 023c7bf265..e0277f9014 100644
--- a/integration/spark/src/main/scala/org/apache/spark/util/DeleteSegmentById.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/util/DeleteSegmentById.scala
@@ -17,7 +17,6 @@
 package org.apache.spark.util
 
 import org.apache.spark.sql.{CarbonEnv, SparkSession}
-import org.apache.spark.sql.catalyst.TableIdentifier
 
 import org.apache.carbondata.api.CarbonStore
 
@@ -35,7 +34,7 @@ object DeleteSegmentById {
       segmentIds: Seq[String]): Unit = {
     TableAPIUtil.validateTableExists(spark, dbName, tableName)
     val carbonTable = CarbonEnv.getCarbonTable(Some(dbName), tableName)(spark)
-    CarbonStore.deleteLoadById(segmentIds, dbName, tableName, carbonTable)
+    CarbonStore.deleteLoadById(segmentIds, dbName, tableName, carbonTable, spark)
   }
 
   def main(args: Array[String]): Unit = {
diff --git a/integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala b/integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
index 2945dfae61..ba00866440 100644
--- a/integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
@@ -49,7 +49,7 @@ object PartitionCacheManager extends Cache[PartitionCacheKey,
     LOGGER.info("Reading partition values from store")
     // read the tableStatus file to get valid and invalid segments
     val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
-      identifier.tablePath, null, null, identifier.tableId))
+      identifier.tablePath, null, null, identifier.tableId), identifier.tblStatusVersion)
       .getValidAndInvalidSegments
     val existingCache = CACHE.get(identifier.tableId)
     val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
@@ -142,7 +142,10 @@ object PartitionCacheManager extends Cache[PartitionCacheKey,
   }
 }
 
-case class PartitionCacheKey(tableId: String, tablePath: String, expirationTime: Long)
+case class PartitionCacheKey(tableId: String,
+    tablePath: String,
+    expirationTime: Long,
+    tblStatusVersion: String)
 
 /**
  * Cacheable instance of the CatalogTablePartitions.
diff --git a/integration/spark/src/main/spark3.1/org/apache/spark/sql/SparkVersionAdapter.scala b/integration/spark/src/main/spark3.1/org/apache/spark/sql/SparkVersionAdapter.scala
index c80394d348..805af13a68 100644
--- a/integration/spark/src/main/spark3.1/org/apache/spark/sql/SparkVersionAdapter.scala
+++ b/integration/spark/src/main/spark3.1/org/apache/spark/sql/SparkVersionAdapter.scala
@@ -483,6 +483,12 @@ trait SparkVersionAdapter {
           CreateTableStatement(create.tableName, create.tableSchema, create.partitioning,
             create.bucketSpec, newProperties, create.provider, create.options,
             location, create.comment, create.serde, create.external, create.ifNotExists)
+        } else if (create.options.contains("latestversion")) {
+          // remove latestversion property in options if present
+          val newOptions = create.options.filterNot(_._1.equalsIgnoreCase("latestversion"))
+          CreateTableStatement(create.tableName, create.tableSchema, create.partitioning,
+            create.bucketSpec, properties, create.provider, newOptions,
+            location, create.comment, create.serde, create.external, create.ifNotExists)
         } else {
           create
         }
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/binary/TestBinaryDataType.scala b/integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/binary/TestBinaryDataType.scala
index edf7306c4d..52b1b0b206 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/binary/TestBinaryDataType.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/binary/TestBinaryDataType.scala
@@ -1173,6 +1173,7 @@ class TestBinaryDataType extends QueryTest with BeforeAndAfterAll {
             sql(s"alter table binarytable CHANGE binaryField binaryField3 STRING ")
         }
         assert(e.getMessage.contains("operation failed for default.binarytable: Alter table data type change operation failed: Given column binaryfield with data type BINARY cannot be modified. Only Int and Decimal data types are allowed for modification"))
+        sql("DROP TABLE IF EXISTS binaryTable")
     }
 
     test("Create table and load data with binary column for hive: test encode with base64") {
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala b/integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
index 2456a23b6f..d47ac5af26 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
@@ -57,7 +57,9 @@ class TestLoadDataGeneral extends QueryTest with BeforeAndAfterEach {
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable(databaseName, tableName)
     val partitionPath =
       CarbonTablePath.getPartitionDir(carbonTable.getAbsoluteTableIdentifier.getTablePath)
-    val segment = Segment.getSegment(segmentId, carbonTable.getAbsoluteTableIdentifier.getTablePath)
+    val segment = Segment.getSegment(segmentId,
+      carbonTable.getAbsoluteTableIdentifier.getTablePath,
+      carbonTable.getTableStatusVersion)
     segment != null
   }
 
@@ -352,7 +354,8 @@ class TestLoadDataGeneral extends QueryTest with BeforeAndAfterEach {
     sql("create table stale(a string) STORED AS carbondata")
     sql("insert into stale values('k')")
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "stale")
-    val tableStatusFile = CarbonTablePath.getTableStatusFilePath(carbonTable.getTablePath)
+    val tableStatusFile = CarbonTablePath.getTableStatusFilePath(carbonTable.getTablePath,
+      carbonTable.getTableStatusVersion)
     FileFactory.getCarbonFile(tableStatusFile).delete()
     sql("insert into stale values('k')")
     // if table lose tablestatus file, the system should keep all data.
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithStaleDataInSegmentFolder.scala b/integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithStaleDataInSegmentFolder.scala
index 656414184d..9ec1c7877c 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithStaleDataInSegmentFolder.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithStaleDataInSegmentFolder.scala
@@ -198,7 +198,8 @@ class TestLoadDataWithStaleDataInSegmentFolder extends QueryTest with BeforeAndA
 
   private def mockStaleDataByRemoveTablestatus(tableName: String): Unit = {
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", tableName)
-    val tableStatusFile = CarbonTablePath.getTableStatusFilePath(carbonTable.getTablePath)
+    val tableStatusFile = CarbonTablePath.getTableStatusFilePath(carbonTable.getTablePath,
+      carbonTable.getTableStatusVersion)
     FileFactory.getCarbonFile(tableStatusFile).delete()
   }
 
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/recovery/TableStatusRecoveryTest.scala b/integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/recovery/TableStatusRecoveryTest.scala
new file mode 100644
index 0000000000..775508186a
--- /dev/null
+++ b/integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/recovery/TableStatusRecoveryTest.scala
@@ -0,0 +1,380 @@
+/*
+ * 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.carbondata.integration.spark.testsuite.recovery
+
+import org.apache.spark.sql.{CarbonEnv, Row}
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.filesystem.{CarbonFile, CarbonFileFilter}
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.recovery.tablestatus.TableStatusRecovery
+import org.apache.carbondata.sdk.file.CarbonWriter
+
+class TableStatusRecoveryTest extends QueryTest with BeforeAndAfterAll {
+
+  override def beforeAll(): Unit = {
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_ENABLE_MULTI_VERSION_TABLE_STATUS, "true")
+    sql("DROP TABLE IF EXISTS table1")
+  }
+
+  override def afterAll(): Unit = {
+    CarbonProperties.getInstance()
+      .removeProperty(CarbonCommonConstants.CARBON_CLEAN_FILES_FORCE_ALLOWED)
+      .removeProperty(CarbonCommonConstants.CARBON_ENABLE_MULTI_VERSION_TABLE_STATUS)
+    sql("DROP TABLE IF EXISTS table1")
+  }
+
+  test("test table status recovery if file is lost after first insert") {
+    sql("DROP TABLE IF EXISTS table1")
+    sql("create table table1 (c1 string,c2 int, c3 int) STORED AS carbondata")
+    sql("insert into table1 values('abc',1, 1)")
+    checkAnswer(sql("select * from table1"), Seq(Row("abc", 1, 1)))
+    val version = deleteTableStatusVersionFile("table1")
+    var err = intercept[RuntimeException] {
+      sql("select count(*) from table1").show()
+    }
+    err = intercept[RuntimeException] {
+      sql("insert into table1 values('abc',1, 1)")
+    }
+    assertException(version, err)
+    val args = "default table1"
+    TableStatusRecovery.main(args.split(" "))
+    checkAnswer(sql("select * from table1"), Seq(Row("abc", 1, 1)))
+  }
+
+  test("test table status recovery for table with global sort") {
+    sql("DROP TABLE IF EXISTS table1")
+    sql("create table table1 (c1 string,c2 int, c3 int) STORED AS carbondata " +
+        "tblproperties('sort_scope'='global_sort', 'sort_columns'='c3')")
+    insertData()
+    checkAnswer(sql("select * from table1"),
+      Seq(Row("abc", 1, 1), Row("abc", 2, 1), Row("abc", 3, 2)))
+    deleteTableStatusVersionFile("table1")
+    val args = "default table1"
+    TableStatusRecovery.main(args.split(" "))
+    sql("alter table table1 compact 'major'")
+    checkAnswer(sql("select * from table1"),
+      Seq(Row("abc", 1, 1), Row("abc", 2, 1), Row("abc", 3, 2)))
+  }
+
+  test("test table status recovery on secondary index table") {
+    def checkResults(): Unit = {
+      checkAnswer(sql("select * from table1 where c1='abc'"),
+        Seq(Row("abc", 1, 1), Row("abc", 2, 1), Row("abc", 3, 2)))
+      checkAnswer(sql("select count(*) from si_index "), Seq(Row(3)))
+    }
+    sql("DROP TABLE IF EXISTS table1")
+    sql("create table table1 (c1 string,c2 int, c3 int) STORED AS carbondata")
+    sql("DROP INDEX IF EXISTS si_index on table1")
+    sql("CREATE INDEX si_index on table table1 (c1) AS 'carbondata' ")
+    insertData()
+    checkResults()
+    val version = deleteTableStatusVersionFile("si_index")
+    var err = intercept[RuntimeException] {
+      sql("SHOW SEGMENTS FOR TABLE si_index").show()
+    }
+    assertException(version, err)
+    err = intercept[RuntimeException] {
+      sql("select * from si_index").show()
+    }
+    assertException(version, err)
+    val args = "default si_index"
+    TableStatusRecovery.main(args.split(" "))
+    checkResults()
+  }
+
+  test("test table status recovery on mv table -- not supported") {
+    def checkResults(): Unit = {
+      checkAnswer(sql("select c2, c3 from table1"),
+        Seq(Row(1, 1), Row(2, 1), Row(3, 2)))
+      checkAnswer(sql("select count(*) from view1"), Seq(Row(3)))
+    }
+    sql("DROP TABLE IF EXISTS table1")
+    sql("create table table1 (c1 string,c2 int, c3 int) STORED AS carbondata")
+    sql("drop MATERIALIZED VIEW if exists view1")
+    sql("CREATE MATERIALIZED VIEW view1 AS SELECT c2, c3 FROM table1")
+    insertData()
+    checkResults()
+    val version = deleteTableStatusVersionFile("view1")
+    var err = intercept[RuntimeException] {
+      sql("select * from view1").show()
+    }
+    assertException(version, err)
+    val args = "default view1"
+    err = intercept[UnsupportedOperationException] {
+      TableStatusRecovery.main(args.split(" "))
+    }
+    assert(err.getMessage.contains("Unsupported operation on Materialized view table"))
+    sql("refresh materialized view view1")
+    checkResults()
+  }
+
+  private def assertException(version: String,
+      err: RuntimeException) = {
+    assert(err.getMessage
+      .contains(s"Table Status Version file {tablestatus_$version} not found. Try running " +
+                "TableStatusRecovery tool to recover lost file "))
+  }
+
+  test("test table status recovery if file is lost after insert and disable versioning") {
+    sql("DROP TABLE IF EXISTS table1")
+    sql("create table table1 (c1 string,c2 int, c3 int) STORED AS carbondata")
+    verifyScenario_Insert()
+    CarbonProperties.getInstance().addProperty(
+      CarbonCommonConstants.CARBON_ENABLE_MULTI_VERSION_TABLE_STATUS, "false")
+    checkAnswer(sql("select * from table1"),
+      Seq(Row("abc", 1, 1), Row("abc", 2, 1), Row("abc", 3, 2)))
+    // will write to tablestatus file
+    sql("insert into table1 values('abcd',11, 12)")
+    checkAnswer(sql("select * from table1"),
+      Seq(Row("abc", 1, 1), Row("abc", 2, 1), Row("abc", 3, 2), Row("abcd", 11, 12)))
+    deleteTableStatusVersionFile("table1")
+    val args = "default table1"
+    TableStatusRecovery.main(args.split(" "))
+    CarbonProperties.getInstance().addProperty(
+      CarbonCommonConstants.CARBON_ENABLE_MULTI_VERSION_TABLE_STATUS, "true")
+    sql("insert into table1 values('abcd',14, 15)")
+    checkAnswer(sql("select * from table1"), Seq(Row("abc", 1, 1), Row("abc", 2, 1),
+      Row("abc", 3, 2), Row("abcd", 11, 12), Row("abcd", 14, 15)))
+  }
+
+  test("verify clean table status old version files") {
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_CLEAN_FILES_FORCE_ALLOWED, "true")
+    sql("DROP TABLE IF EXISTS table1")
+    sql("create table table1 (c1 string,c2 int, c3 int) STORED AS carbondata")
+    insertData()
+    insertData()
+    val carbonTable = CarbonEnv.getCarbonTable(Some("default"), "table1")(sqlContext.sparkSession)
+    var tableStatusFiles = getTableStatusFiles(carbonTable)
+    assert(tableStatusFiles.length == 6)
+    sql("clean files for table table1 options('force'='true')")
+    tableStatusFiles = getTableStatusFiles(carbonTable)
+    assert(tableStatusFiles.length == 2)
+  }
+
+  private def getTableStatusFiles(carbonTable: CarbonTable) = {
+    FileFactory.getCarbonFile(CarbonTablePath.getMetadataPath(carbonTable
+      .getTablePath)).listFiles(new CarbonFileFilter {
+      override def accept(file: CarbonFile): Boolean = {
+        file.getName.startsWith(CarbonTablePath
+          .TABLE_STATUS_FILE)
+      }
+    })
+  }
+
+  test("test table status recovery if file is lost after delete segment") {
+    sql("DROP TABLE IF EXISTS table1")
+    sql("create table table1 (c1 string,c2 int, c3 int) STORED AS carbondata")
+    verifyScenario_Delete_Segment()
+  }
+
+  test("test table status recovery if file is lost after insert") {
+    sql("DROP TABLE IF EXISTS table1")
+    sql("create table table1 (c1 string,c2 int, c3 int) STORED AS carbondata")
+    verifyScenario_Insert()
+  }
+
+  test("test table status recovery if file is lost after update & delete") {
+    sql("DROP TABLE IF EXISTS table1")
+    sql("create table table1 (c1 string,c2 int, c3 int) STORED AS carbondata")
+    verifyScenario_IUD()
+  }
+
+  test("test table status recovery if file is lost after compaction") {
+    sql("DROP TABLE IF EXISTS table1")
+    sql("create table table1 (c1 string,c2 int, c3 int) STORED AS carbondata")
+    verifyScenario_Compaction()
+  }
+
+  test("test table status recovery if file is lost after compaction & clean files") {
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_CLEAN_FILES_FORCE_ALLOWED, "true")
+    sql("DROP TABLE IF EXISTS table1")
+    sql("create table table1 (c1 string,c2 int, c3 int) STORED AS carbondata")
+    verifyScenario_Compaction_CleanFiles()
+  }
+
+  test("test table status recovery if file is lost after insert - partition table") {
+    sql("DROP TABLE IF EXISTS table1")
+    sql("create table table1 (c1 string,c2 int) partitioned by (c3 int)" +
+        "STORED AS carbondata")
+    verifyScenario_Insert()
+  }
+
+  test("test table status recovery if file is lost after update & delete - partition table") {
+    sql("DROP TABLE IF EXISTS table1")
+    sql("create table table1 (c1 string,c2 int) partitioned by (c3 int)" +
+        "STORED AS carbondata")
+    verifyScenario_IUD()
+  }
+
+  test("test table status recovery if file is lost after compaction - partition table") {
+    sql("DROP TABLE IF EXISTS table1")
+    sql("create table table1 (c1 string,c2 int) partitioned by (c3 int)" +
+        "STORED AS carbondata")
+    verifyScenario_Compaction()
+  }
+
+  test("test table status recovery if file is lost after compaction & clean files " +
+       "- partition table") {
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_CLEAN_FILES_FORCE_ALLOWED, "true")
+    sql("DROP TABLE IF EXISTS table1")
+    sql("create table table1 (c1 string,c2 int) partitioned by (c3 int)" +
+        "STORED AS carbondata")
+    verifyScenario_Compaction_CleanFiles()
+  }
+
+  test("test table status recovery if file is lost after add & drop partition") {
+    sql("drop table if exists table1")
+    sql("create table table1 (id int,name String) partitioned by(email string) " +
+        "stored as carbondata")
+    sql("insert into table1 select 1,'blue','abc'")
+    val schemaFile =
+      CarbonTablePath.getSchemaFilePath(
+        CarbonEnv.getCarbonTable(None, "table1")(sqlContext.sparkSession).getTablePath)
+    val sdkWritePath = target + "/" + "def"
+    FileFactory.deleteAllCarbonFilesOfDir(FileFactory.getCarbonFile(sdkWritePath))
+    val writer = CarbonWriter.builder()
+      .outputPath(sdkWritePath)
+      .writtenBy("test")
+      .withSchemaFile(schemaFile)
+      .withCsvInput()
+      .build()
+    writer.write(Seq("2", "red", "def").toArray)
+    writer.write(Seq("3", "black", "def").toArray)
+    writer.close()
+    sql(s"alter table table1 add partition (email='def') location '$sdkWritePath'")
+    val version = deleteTableStatusVersionFile("table1")
+    val err = intercept[RuntimeException] {
+      sql("select count(*) from table1").show()
+    }
+    assertException(version, err)
+    val args = "default table1"
+    TableStatusRecovery.main(args.split(" "))
+    checkAnswer(sql("select name from table1"), Seq(Row("blue"), Row("red"), Row("black")))
+    sql("alter table table1 drop partition(email='def')")
+    checkAnswer(sql("select name from table1"), Seq(Row("blue")))
+    deleteTableStatusVersionFile("table1")
+    TableStatusRecovery.main(args.split(" "))
+    checkAnswer(sql("select name from table1"), Seq(Row("blue")))
+  }
+
+  private def deleteTableStatusVersionFile(tblName: String): String = {
+    val table = CarbonEnv.getCarbonTable(Some("default"), tblName)(sqlContext.sparkSession)
+    val currVersion = table.getTableStatusVersion
+    val status = FileFactory.getCarbonFile(CarbonTablePath.getTableStatusFilePath(
+      table.getTablePath, currVersion)).deleteFile()
+    assert(status.equals(true))
+    currVersion
+  }
+
+  private def verifyScenario_Insert(): Unit = {
+    insertDataAndCheckResult()
+    deleteTableStatusVersionFile("table1")
+    val args = "default table1"
+    TableStatusRecovery.main(args.split(" "))
+    checkAnswer(sql("select * from table1"),
+      Seq(Row("abc", 1, 1), Row("abc", 2, 1), Row("abc", 3, 2)))
+  }
+
+  private def insertDataAndCheckResult(): Unit = {
+    insertData()
+    checkAnswer(sql("select * from table1"),
+      Seq(Row("abc", 1, 1), Row("abc", 2, 1), Row("abc", 3, 2)))
+  }
+
+  private def insertData(): Unit = {
+    sql("insert into table1 values('abc',1, 1)")
+    sql("insert into table1 values('abc', 2, 1)")
+    sql("insert into table1 values('abc', 3, 2)")
+  }
+
+  def verifyScenario_IUD(): Unit = {
+    insertDataAndCheckResult()
+    sql("update table1 set(c2)=(5) where c2=3").show()
+    checkAnswer(sql("select * from table1"),
+      Seq(Row("abc", 1, 1), Row("abc", 2, 1), Row("abc", 5, 2)))
+    sql("update table1 set(c2)=(6) where c2=5").show()
+
+    var table = CarbonEnv.getCarbonTable(Some("default"), "table1")(sqlContext.sparkSession)
+    var currVersion = table.getTableStatusVersion
+    var status = FileFactory.getCarbonFile(CarbonTablePath.getTableStatusFilePath(
+      table.getTablePath, currVersion)).deleteFile()
+    assert(status.equals(true))
+    val args = "default table1"
+    TableStatusRecovery.main(args.split(" "))
+    checkAnswer(sql("select * from table1"),
+      Seq(Row("abc", 1, 1), Row("abc", 2, 1), Row("abc", 6, 2)))
+
+    sql("delete from table1 where c2=6").show()
+    checkAnswer(sql("select * from table1"),
+      Seq(Row("abc", 1, 1), Row("abc", 2, 1)))
+    table = CarbonEnv.getCarbonTable(Some("default"), "table1")(sqlContext.sparkSession)
+    currVersion = table.getTableStatusVersion
+    status = FileFactory.getCarbonFile(CarbonTablePath.getTableStatusFilePath(
+      table.getTablePath, currVersion)).deleteFile()
+    assert(status.equals(true))
+    TableStatusRecovery.main(args.split(" "))
+    checkAnswer(sql("select * from table1"),
+      Seq(Row("abc", 1, 1), Row("abc", 2, 1)))
+  }
+
+  private def verifyScenario_Compaction(): Unit = {
+    insertData()
+    sql("alter table table1 compact 'major'")
+    deleteTableStatusVersionFile("table1")
+    val args = "default table1"
+    TableStatusRecovery.main(args.split(" "))
+    assert(!sql("Show segments for table table1").collect().map(_.get(0)).contains("0.1"))
+    checkAnswer(sql("select * from table1"),
+      Seq(Row("abc", 1, 1), Row("abc", 2, 1), Row("abc", 3, 2)))
+  }
+
+  private def verifyScenario_Compaction_CleanFiles(): Unit = {
+    insertDataAndCheckResult()
+    sql("alter table table1 compact 'major'")
+    sql("clean files for table table1 options('force'='true')")
+    deleteTableStatusVersionFile("table1")
+    val args = "default table1"
+    TableStatusRecovery.main(args.split(" "))
+    assert(sql("Show segments for table table1").collect().map(_.get(0)).contains("0.1"))
+    checkAnswer(sql("select * from table1"),
+      Seq(Row("abc", 1, 1), Row("abc", 2, 1), Row("abc", 3, 2)))
+  }
+
+  private def verifyScenario_Delete_Segment(): Unit = {
+    insertDataAndCheckResult()
+    sql("DELETE FROM TABLE table1 WHERE SEGMENT.ID IN(0)")
+    checkAnswer(sql("select * from table1"),
+      Seq(Row("abc", 2, 1), Row("abc", 3, 2)))
+    deleteTableStatusVersionFile("table1")
+    val args = "default table1"
+    TableStatusRecovery.main(args.split(" "))
+    // cannot recover deleted segment, as the delete info exists only in table status file
+    checkAnswer(sql("select * from table1"),
+      Seq(Row("abc", 1, 1), Row("abc", 2, 1), Row("abc", 3, 2)))
+  }
+}
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/InsertIntoCarbonTableTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/InsertIntoCarbonTableTestCase.scala
index ae43d1156c..f88b070ae3 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/InsertIntoCarbonTableTestCase.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/InsertIntoCarbonTableTestCase.scala
@@ -442,8 +442,10 @@ class InsertIntoCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
   }
 
   private def checkSegment(tableName: String): Boolean = {
-    val storePath_t1 = s"$storeLocation/${tableName.toLowerCase()}"
-    val detailses = SegmentStatusManager.readTableStatusFile(CarbonTablePath.getTableStatusFilePath(storePath_t1))
+    val storePath_t1 = s"$storeLocation/${ tableName.toLowerCase() }"
+    val table = CarbonEnv.getCarbonTable(Some("default"), tableName)(sqlContext.sparkSession)
+    val detailses = SegmentStatusManager.readTableStatusFile(CarbonTablePath.getTableStatusFilePath(
+      storePath_t1, table.getTableStatusVersion))
     detailses.map(_.getSegmentStatus == SegmentStatus.SUCCESS).exists(f => f)
   }
 
@@ -520,14 +522,18 @@ class InsertIntoCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
 
     val dt2 = "dt2"
     sql(s"insert overwrite table $tableName partition(dt='$dt2') select 1, 'a'")
-    val dt1Metas = SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath)
+    var version = CarbonEnv.getCarbonTable(
+      Option(CarbonCommonConstants.DATABASE_DEFAULT_NAME), tableName)(sqlContext.sparkSession).getTableStatusVersion
+    val dt1Metas = SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath, version)
 
     assert(dt1Metas.length == 2)
     val dt1Seg1 = dt1Metas(0)
     val dt2Seg1 = dt1Metas(1)
 
     sql(s"insert overwrite table $tableName partition(dt='$dt2') select 5, 'z'")
-    val dt2Metas = SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath)
+    version = CarbonEnv.getCarbonTable(
+      Option(CarbonCommonConstants.DATABASE_DEFAULT_NAME), tableName)(sqlContext.sparkSession).getTableStatusVersion
+    val dt2Metas = SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath, version)
     assert(dt2Metas.length == 3)
     val dt2Seg30 = dt2Metas(0)
     val dt2Seg31 = dt2Metas(1)
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/TestQueryWithColumnMetCacheAndCacheLevelProperty.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/TestQueryWithColumnMetCacheAndCacheLevelProperty.scala
index e83daa555c..5777daa2ab 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/TestQueryWithColumnMetCacheAndCacheLevelProperty.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/TestQueryWithColumnMetCacheAndCacheLevelProperty.scala
@@ -87,7 +87,9 @@ class TestQueryWithColumnMetCacheAndCacheLevelProperty
       .asInstanceOf[CarbonRelation]
     val carbonTable = relation.carbonTable
     assert(carbonTable.getTableInfo.isSchemaModified == isSchemaModified)
-    val segment: Segment = Segment.getSegment(segmentId, carbonTable.getTablePath)
+    val segment: Segment = Segment.getSegment(segmentId,
+      carbonTable.getTablePath,
+      carbonTable.getTableStatusVersion)
     val defaultIndex: TableIndex = IndexStoreManager.getInstance()
       .getDefaultIndex(carbonTable)
     defaultIndex.getIndexFactory.getIndexes(segment).asScala.toList
@@ -296,7 +298,7 @@ class TestQueryWithColumnMetCacheAndCacheLevelProperty
       new IndexFilter(carbonTable, andExpression).getResolver()
     val exprWrapper = IndexChooser.getDefaultIndex(carbonTable, resolveFilter)
     val segment = new Segment("0", new TableStatusReadCommittedScope(carbonTable
-      .getAbsoluteTableIdentifier, new Configuration(false)))
+      .getAbsoluteTableIdentifier, new Configuration(false), carbonTable.getTableStatusVersion))
     // get the pruned blocklets
     val prunedBlocklets = exprWrapper.prune(List(segment).asJava, null)
     prunedBlocklets.asScala.foreach { blocklet =>
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/cleanfiles/TestCleanFileCommand.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/cleanfiles/TestCleanFileCommand.scala
index 1aba6a6388..1694bb2c7b 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/cleanfiles/TestCleanFileCommand.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/cleanfiles/TestCleanFileCommand.scala
@@ -45,10 +45,10 @@ class TestCleanFileCommand extends QueryTest with BeforeAndAfterAll {
     // do not send the segment folders to trash
     createTable()
     loadData()
-    val path = CarbonEnv.getCarbonTable(Some("default"), "cleantest")(sqlContext.sparkSession)
-      .getTablePath
+    val table = CarbonEnv.getCarbonTable(Some("default"), "cleantest")(sqlContext.sparkSession)
+    val path = table.getTablePath
     val trashFolderPath = CarbonTablePath.getTrashFolderPath(path)
-    editTableStatusFile(path)
+    editTableStatusFile(path, table.getTableStatusVersion)
     assert(!FileFactory.isFileExist(trashFolderPath))
 
     val segmentNumber1 = sql(s"""show segments for table cleantest""").count()
@@ -70,10 +70,10 @@ class TestCleanFileCommand extends QueryTest with BeforeAndAfterAll {
     // do not send the segment folders to trash
     createTable()
     loadData()
-    val path = CarbonEnv.getCarbonTable(Some("default"), "cleantest")(sqlContext.sparkSession)
-      .getTablePath
+    val table = CarbonEnv.getCarbonTable(Some("default"), "cleantest")(sqlContext.sparkSession)
+    val path = table.getTablePath
     val trashFolderPath = CarbonTablePath.getTrashFolderPath(path)
-    editTableStatusFile(path)
+    editTableStatusFile(path, table.getTableStatusVersion)
     assert(!FileFactory.isFileExist(trashFolderPath))
 
     val segmentNumber1 = sql(s"""show segments for table cleantest""").count()
@@ -425,7 +425,8 @@ class TestCleanFileCommand extends QueryTest with BeforeAndAfterAll {
     sql(s"alter table addsegment1 add segment " +
         s"options('path'='${ newPath + 0 }', 'format'='carbon')").collect()
     // testing for in progress segments
-    val tableStatusPath = CarbonTablePath.getTableStatusFilePath(table.getTablePath)
+    val tableStatusPath = CarbonTablePath.getTableStatusFilePath(table.getTablePath,
+      table.getTableStatusVersion)
     val segments = SegmentStatusManager.readTableStatusFile(tableStatusPath)
     segments.foreach(segment => if (segment.getLoadName.equals("9")) {
       segment.setSegmentStatus(SegmentStatus.INSERT_IN_PROGRESS)
@@ -565,9 +566,9 @@ class TestCleanFileCommand extends QueryTest with BeforeAndAfterAll {
     sql("drop table if exists cleantest")
   }
 
-  def editTableStatusFile(carbonTablePath: String) : Unit = {
+  def editTableStatusFile(carbonTablePath: String, version: String) : Unit = {
     // original table status file
-    val f1 = new File(CarbonTablePath.getTableStatusFilePath(carbonTablePath))
+    val f1 = new File(CarbonTablePath.getTableStatusFilePath(carbonTablePath, version))
     val f2 = new File(CarbonTablePath.getMetadataPath(carbonTablePath) + CarbonCommonConstants
         .FILE_SEPARATOR + CarbonCommonConstants.FILE_SEPARATOR + "tmp")
     val w = new PrintWriter(f2)
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/cleanfiles/TestCleanFilesCommandPartitionTable.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/cleanfiles/TestCleanFilesCommandPartitionTable.scala
index 14d6da7250..6750eb570a 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/cleanfiles/TestCleanFilesCommandPartitionTable.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/cleanfiles/TestCleanFilesCommandPartitionTable.scala
@@ -39,10 +39,10 @@ class TestCleanFilesCommandPartitionTable extends QueryTest with BeforeAndAfterA
     // do not send the segment folders to trash
     createParitionTable()
     loadData()
-    val path = CarbonEnv.getCarbonTable(Some("default"), "cleantest")(sqlContext.sparkSession)
-      .getTablePath
+    val table = CarbonEnv.getCarbonTable(Some("default"), "cleantest")(sqlContext.sparkSession)
+    val path = table.getTablePath
     val trashFolderPath = path + CarbonCommonConstants.FILE_SEPARATOR + CarbonTablePath.TRASH_DIR
-    editTableStatusFile(path)
+    editTableStatusFile(path, table.getTableStatusVersion)
     assert(!FileFactory.isFileExist(trashFolderPath))
     val segmentNumber1 = sql(s"""show segments for table cleantest""").count()
     assert(segmentNumber1 == 4)
@@ -442,9 +442,9 @@ class TestCleanFilesCommandPartitionTable extends QueryTest with BeforeAndAfterA
         CarbonCommonConstants.DEFAULT_DELETE_DELTAFILE_COUNT_THRESHOLD_IUD_COMPACTION)
   }
 
-  def editTableStatusFile(carbonTablePath: String) : Unit = {
+  def editTableStatusFile(carbonTablePath: String, version: String) : Unit = {
     // Original Table status file
-    val f1 = new File(CarbonTablePath.getTableStatusFilePath(carbonTablePath))
+    val f1 = new File(CarbonTablePath.getTableStatusFilePath(carbonTablePath, version ))
     // duplicate
     val f2 = new File(CarbonTablePath.getMetadataPath(carbonTablePath) + CarbonCommonConstants
       .FILE_SEPARATOR + CarbonCommonConstants.FILE_SEPARATOR + "tmp")
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateExternalTable.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateExternalTable.scala
index e593ef82a7..fab5ea0ea8 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateExternalTable.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateExternalTable.scala
@@ -35,6 +35,8 @@ class TestCreateExternalTable extends QueryTest with BeforeAndAfterAll {
   var originDataPath: String = _
 
   override def beforeAll(): Unit = {
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_ENABLE_MULTI_VERSION_TABLE_STATUS, "false")
     sql("DROP TABLE IF EXISTS origin")
     sql("drop table IF EXISTS rsext")
     sql("drop table IF EXISTS rstest1")
@@ -46,6 +48,9 @@ class TestCreateExternalTable extends QueryTest with BeforeAndAfterAll {
   }
 
   override def afterAll(): Unit = {
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_ENABLE_MULTI_VERSION_TABLE_STATUS,
+        CarbonCommonConstants.CARBON_ENABLE_MULTI_VERSION_TABLE_STATUS_DEFAULT)
     sql("DROP TABLE IF EXISTS origin")
     sql("drop table IF EXISTS rsext")
     sql("drop table IF EXISTS rstest1")
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableLike.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableLike.scala
index 8d2d9bfb2b..449cd810bf 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableLike.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableLike.scala
@@ -19,6 +19,8 @@ package org.apache.carbondata.spark.testsuite.createTable
 
 import java.util
 
+import scala.collection.JavaConverters._
+
 import org.apache.spark.sql.CarbonEnv
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException
@@ -94,7 +96,9 @@ class TestCreateTableLike extends QueryTest with BeforeAndAfterEach with BeforeA
     assert(checkColumns(fact_src.getListOfColumns, fact_dst.getListOfColumns))
 
     // check table properties same
-    assert(fact_src.getTableProperties.equals(fact_dst.getTableProperties))
+    val srcTblProp = fact_src.getTableProperties.asScala.filterNot(_._1.equals("latestversion"))
+    val dstTblProp = fact_dst.getTableProperties.asScala.filterNot(_._1.equals("latestversion"))
+    assert(srcTblProp.equals(dstTblProp))
 
     // check transaction same
     assert(!(info_src.isTransactionalTable ^ info_dst.isTransactionalTable))
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CarbonIndexFileMergeTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CarbonIndexFileMergeTestCase.scala
index e4be59222c..fead2274f6 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CarbonIndexFileMergeTestCase.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CarbonIndexFileMergeTestCase.scala
@@ -259,15 +259,15 @@ class CarbonIndexFileMergeTestCase
       """.stripMargin)
     sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE fileSize OPTIONS('header'='false')")
     val table = CarbonMetadata.getInstance().getCarbonTable("default", "fileSize")
-    var loadMetadataDetails = SegmentStatusManager
-      .readTableStatusFile(CarbonTablePath.getTableStatusFilePath(table.getTablePath))
+    var loadMetadataDetails = SegmentStatusManager.readTableStatusFile(
+      CarbonTablePath.getTableStatusFilePath(table.getTablePath, table.getTableStatusVersion))
     var segment0 = loadMetadataDetails.filter(x => x.getLoadName.equalsIgnoreCase("0"))
     Assert
       .assertEquals(getIndexOrMergeIndexFileSize(table, "0", CarbonTablePath.INDEX_FILE_EXT),
         segment0.head.getIndexSize.toLong)
     sql("Alter table fileSize compact 'segment_index'")
-    loadMetadataDetails = SegmentStatusManager
-      .readTableStatusFile(CarbonTablePath.getTableStatusFilePath(table.getTablePath))
+    loadMetadataDetails = SegmentStatusManager.readTableStatusFile(
+      CarbonTablePath.getTableStatusFilePath(table.getTablePath, table.getTableStatusVersion))
     segment0 = loadMetadataDetails.filter(x => x.getLoadName.equalsIgnoreCase("0"))
     Assert
       .assertEquals(getIndexOrMergeIndexFileSize(table, "0", CarbonTablePath.MERGE_INDEX_FILE_EXT),
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortFunctionTest.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortFunctionTest.scala
index 2ae15dc9b3..b33761542c 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortFunctionTest.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortFunctionTest.scala
@@ -561,7 +561,9 @@ class CompactionSupportGlobalSortFunctionTest
     if (FileFactory.isFileExist(segmentDir)) {
       new SegmentIndexFileStore().getIndexFilesFromSegment(segmentDir).size()
     } else {
-      val segment = Segment.getSegment(segmentNo, carbonTable.getTablePath)
+      val segment = Segment.getSegment(segmentNo,
+        carbonTable.getTablePath,
+        carbonTable.getTableStatusVersion)
       new SegmentFileStore(carbonTable.getTablePath, segment.getSegmentFileName)
         .getIndexCarbonFiles.size()
     }
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortParameterTest.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortParameterTest.scala
index bf4a07bf48..d7a164acd0 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortParameterTest.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortParameterTest.scala
@@ -595,7 +595,9 @@ class CompactionSupportGlobalSortParameterTest
     if (FileFactory.isFileExist(segmentDir)) {
       new SegmentIndexFileStore().getIndexFilesFromSegment(segmentDir).size()
     } else {
-      val segment = Segment.getSegment(segmentNo, carbonTable.getTablePath)
+      val segment = Segment.getSegment(segmentNo,
+        carbonTable.getTablePath,
+        carbonTable.getTableStatusVersion)
       new SegmentFileStore(carbonTable.getTablePath, segment.getSegmentFileName)
         .getIndexCarbonFiles.size()
     }
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionCardinalityBoundryTest.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionCardinalityBoundryTest.scala
index 47270b60a3..e367b6a90b 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionCardinalityBoundryTest.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionCardinalityBoundryTest.scala
@@ -23,7 +23,7 @@ import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.BeforeAndAfterAll
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata, CarbonTableIdentifier}
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager
 import org.apache.carbondata.core.util.CarbonProperties
 
@@ -69,12 +69,17 @@ class DataCompactionCardinalityBoundryTest extends QueryTest with BeforeAndAfter
   test("check if compaction is completed or not and  verify select query.") {
     var status = true
     var noOfRetries = 0
+    val version = CarbonMetadata
+      .getInstance()
+      .getCarbonTable("default_cardinalityTest")
+      .getTableStatusVersion
     while (status && noOfRetries < 10) {
       val segmentStatusManager: SegmentStatusManager = new SegmentStatusManager(
         AbsoluteTableIdentifier.from(
-          CarbonProperties.getStorePath(),
+          storeLocation + "/cardinalitytest",
           new CarbonTableIdentifier("default", "cardinalityTest", "1")
-        )
+        ),
+        version
       )
       val segments = segmentStatusManager.getValidAndInvalidSegments.getValidSegments.asScala.toList
 
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionLockTest.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionLockTest.scala
index bcea567cef..f9674f9da6 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionLockTest.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionLockTest.scala
@@ -23,7 +23,7 @@ import org.scalatest.BeforeAndAfterAll
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.locks.{CarbonLockFactory, ICarbonLock, LockUsage}
-import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata, CarbonTableIdentifier}
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager
 import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.core.util.path.CarbonTablePath
@@ -35,7 +35,7 @@ class DataCompactionLockTest extends QueryTest with BeforeAndAfterAll {
 
   val absoluteTableIdentifier: AbsoluteTableIdentifier =
       AbsoluteTableIdentifier.from(
-        storeLocation,
+        storeLocation + "/compactionlocktesttable",
         new CarbonTableIdentifier(
           CarbonCommonConstants.DATABASE_DEFAULT_NAME, "compactionlocktesttable", "1")
       )
@@ -91,9 +91,11 @@ class DataCompactionLockTest extends QueryTest with BeforeAndAfterAll {
    * Compaction should fail as lock is being held purposefully
    */
   test("check if compaction is failed or not.") {
-
+    val version = CarbonMetadata.getInstance()
+      .getCarbonTable(absoluteTableIdentifier.getDatabaseName, absoluteTableIdentifier.getTableName)
+      .getTableStatusVersion
     val segmentStatusManager: SegmentStatusManager = new SegmentStatusManager(
-      absoluteTableIdentifier
+      absoluteTableIdentifier, version
     )
     val segments = segmentStatusManager.getValidAndInvalidSegments.getValidSegments.asScala.toList
 
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionIgnoreInMinorTest.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionIgnoreInMinorTest.scala
index 288842bf3a..b3a41512e4 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionIgnoreInMinorTest.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionIgnoreInMinorTest.scala
@@ -82,7 +82,8 @@ class MajorCompactionIgnoreInMinorTest extends QueryTest with BeforeAndAfterAll
     )
 
     val carbonTablePath = carbonTable.getMetadataPath
-    val segs = SegmentStatusManager.readLoadMetadata(carbonTablePath)
+    val segs = SegmentStatusManager.readLoadMetadata(carbonTablePath,
+      carbonTable.getTableStatusVersion)
 
     // status should remain as compacted.
     assertResult(SegmentStatus.COMPACTED)(segs(3).getSegmentStatus)
@@ -102,7 +103,8 @@ class MajorCompactionIgnoreInMinorTest extends QueryTest with BeforeAndAfterAll
       "ignoremajor"
     )
     val carbontablePath = carbonTable.getMetadataPath
-    val segs = SegmentStatusManager.readLoadMetadata(carbontablePath)
+    val segs = SegmentStatusManager.readLoadMetadata(carbontablePath,
+      carbonTable.getTableStatusVersion)
 
     // status should remain as compacted for segment 2.
     assertResult(SegmentStatus.COMPACTED)(segs(3).getSegmentStatus)
@@ -125,7 +127,7 @@ class MajorCompactionIgnoreInMinorTest extends QueryTest with BeforeAndAfterAll
     val absoluteTableIdentifier = carbonTable
       .getAbsoluteTableIdentifier
     val segmentStatusManager: SegmentStatusManager = new SegmentStatusManager(
-      absoluteTableIdentifier)
+      absoluteTableIdentifier, carbonTable.getTableStatusVersion)
 
     // merged segment should not be there
     val segments = segmentStatusManager
@@ -171,7 +173,7 @@ class MajorCompactionIgnoreInMinorTest extends QueryTest with BeforeAndAfterAll
     )
     val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
     val segmentStatusManager: SegmentStatusManager = new SegmentStatusManager(
-      absoluteTableIdentifier)
+      absoluteTableIdentifier, carbonTable.getTableStatusVersion)
 
     // merged segment should not be there
     val segments = segmentStatusManager
@@ -237,10 +239,11 @@ class MajorCompactionIgnoreInMinorTest extends QueryTest with BeforeAndAfterAll
     // do minor compaction
     sql("alter table minor_threshold compact 'minor'")
     // check segment 3 whose size exceed the limit should not be compacted but success
-    val carbonTable = CarbonMetadata.getInstance().getCarbonTable(
+    var carbonTable = CarbonMetadata.getInstance().getCarbonTable(
       CarbonCommonConstants.DATABASE_DEFAULT_NAME, "minor_threshold")
     val carbonTablePath = carbonTable.getMetadataPath
-    val segments = SegmentStatusManager.readLoadMetadata(carbonTablePath);
+    val segments = SegmentStatusManager.readLoadMetadata(carbonTablePath,
+      carbonTable.getTableStatusVersion)
     assertResult(SegmentStatus.SUCCESS)(segments(3).getSegmentStatus)
     assertResult(100030)(sql("select count(*) from minor_threshold").collect().head.get(0))
 
@@ -254,7 +257,10 @@ class MajorCompactionIgnoreInMinorTest extends QueryTest with BeforeAndAfterAll
     // do minor compaction
     sql("alter table minor_threshold compact 'minor'")
     // check segment 3 whose size not exceed the new threshold limit should be compacted now
-    val segments2 = SegmentStatusManager.readLoadMetadata(carbonTablePath);
+    carbonTable = CarbonMetadata.getInstance().getCarbonTable(
+      CarbonCommonConstants.DATABASE_DEFAULT_NAME, "minor_threshold")
+    val segments2 = SegmentStatusManager.readLoadMetadata(
+      carbonTablePath, carbonTable.getTableStatusVersion)
     assertResult(SegmentStatus.COMPACTED)(segments2(3).getSegmentStatus)
     assertResult(400030)(sql("select count(*) from minor_threshold").collect().head.get(0))
 
@@ -313,7 +319,8 @@ class MajorCompactionIgnoreInMinorTest extends QueryTest with BeforeAndAfterAll
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable(
       CarbonCommonConstants.DATABASE_DEFAULT_NAME, "minor_threshold_partition")
     val carbonTablePath2 = carbonTable.getMetadataPath
-    val segments = SegmentStatusManager.readLoadMetadata(carbonTablePath2);
+    val segments = SegmentStatusManager.readLoadMetadata(carbonTablePath2,
+      carbonTable.getTableStatusVersion)
     assertResult(SegmentStatus.SUCCESS)(segments(3).getSegmentStatus)
     assertResult(100030)(sql("select count(*) from " +
       "minor_threshold_partition").collect().head.get(0))
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionStopsAfterCompaction.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionStopsAfterCompaction.scala
index ba18632210..daad76512a 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionStopsAfterCompaction.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionStopsAfterCompaction.scala
@@ -83,7 +83,7 @@ class MajorCompactionStopsAfterCompaction extends QueryTest with BeforeAndAfterA
       val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
 
       val segmentStatusManager: SegmentStatusManager = new SegmentStatusManager(
-        absoluteTableIdentifier)
+        absoluteTableIdentifier, carbonTable.getTableStatusVersion)
 
       val segments = segmentStatusManager.getValidAndInvalidSegments.getValidSegments.asScala.toList
 //      segments.foreach(seg =>
@@ -119,7 +119,7 @@ class MajorCompactionStopsAfterCompaction extends QueryTest with BeforeAndAfterA
     val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
 
     val segmentStatusManager: SegmentStatusManager = new SegmentStatusManager(
-      absoluteTableIdentifier)
+      absoluteTableIdentifier, carbonTable.getTableStatusVersion)
 
     // merged segment should not be there
     val segments = segmentStatusManager
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithFileName.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithFileName.scala
index 91d80dc458..ceea2ee980 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithFileName.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithFileName.scala
@@ -63,7 +63,9 @@ class TestDataLoadWithFileName extends QueryTest with BeforeAndAfterAll {
           }
         })
     } else {
-      val segment = Segment.getSegment("0", carbonTable.getTablePath)
+      val segment = Segment.getSegment("0",
+        carbonTable.getTablePath,
+        carbonTable.getTableStatusVersion)
       val store = new SegmentFileStore(carbonTable.getTablePath, segment.getSegmentFileName)
       store.readIndexFiles(new Configuration(false))
       store.getIndexCarbonFiles.asScala.map(f => new File(f.getAbsolutePath)).toArray
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala
index 344bee26de..8923053c83 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala
@@ -387,7 +387,9 @@ class TestGlobalSortDataLoad extends QueryTest with BeforeAndAfterEach with Befo
     if (FileFactory.isFileExist(segmentDir)) {
       assertResult(Math.max(4, defaultParallelism) + 1)(new File(segmentDir).listFiles().length)
     } else {
-      val segment = Segment.getSegment("0", carbonTable.getTablePath)
+      val segment = Segment.getSegment("0",
+        carbonTable.getTablePath,
+        carbonTable.getTableStatusVersion)
       val store = new SegmentFileStore(carbonTable.getTablePath, segment.getSegmentFileName)
       store.readIndexFiles(new Configuration(false))
       val size = store.getIndexFilesMap.asScala.map(f => f._2.size()).sum
@@ -544,7 +546,9 @@ class TestGlobalSortDataLoad extends QueryTest with BeforeAndAfterEach with Befo
     if (FileFactory.isFileExist(segmentDir)) {
       new SegmentIndexFileStore().getIndexFilesFromSegment(segmentDir).size()
     } else {
-      val segment = Segment.getSegment(segmentNo, carbonTable.getTablePath)
+      val segment = Segment.getSegment(segmentNo,
+        carbonTable.getTablePath,
+        carbonTable.getTableStatusVersion)
       new SegmentFileStore(carbonTable.getTablePath, segment.getSegmentFileName)
         .getIndexCarbonFiles.size()
     }
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestRangeColumnDataLoad.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestRangeColumnDataLoad.scala
index f426afb01d..f9a4ee3bda 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestRangeColumnDataLoad.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestRangeColumnDataLoad.scala
@@ -878,7 +878,9 @@ class TestRangeColumnDataLoad extends QueryTest with BeforeAndAfterEach with Bef
     if (FileFactory.isFileExist(segmentDir)) {
       new SegmentIndexFileStore().getIndexFilesFromSegment(segmentDir).size()
     } else {
-      val segment = Segment.getSegment(segmentNo, carbonTable.getTablePath)
+      val segment = Segment.getSegment(segmentNo,
+        carbonTable.getTablePath,
+        carbonTable.getTableStatusVersion)
       new SegmentFileStore(carbonTable.getTablePath, segment.getSegmentFileName).getIndexCarbonFiles
         .size()
     }
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataretention/DataRetentionTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataretention/DataRetentionTestCase.scala
index 8004b15798..e2e0cafeb4 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataretention/DataRetentionTestCase.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataretention/DataRetentionTestCase.scala
@@ -129,8 +129,11 @@ class DataRetentionTestCase extends QueryTest with BeforeAndAfterAll {
   }
 
   test("RetentionTest_DeleteSegmentsByLoadTime") {
+    val carbonTable2 =
+      CarbonEnv.getCarbonTable(Option(CarbonCommonConstants.DATABASE_DEFAULT_NAME),
+        "dataRetentionTable")(sqlContext.sparkSession)
     val segments: Array[LoadMetadataDetails] =
-      SegmentStatusManager.readLoadMetadata(carbonTablePath)
+      SegmentStatusManager.readLoadMetadata(carbonTablePath, carbonTable2.getTableStatusVersion)
     // check segment length, it should be 3 (loads)
     if (segments.length != 2) {
       assert(false)
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
index 4764083819..122ddc0d3e 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
@@ -228,12 +228,17 @@ class DeleteCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
     val files = FileFactory.getCarbonFile(metaPath)
     val result = CarbonEnv.getInstance(sqlContext.sparkSession).carbonMetaStore.getClass
     if (result.getCanonicalName.contains("CarbonFileMetastore")) {
-      assert(files.listFiles(new CarbonFileFilter {
+      val length = files.listFiles(new CarbonFileFilter {
         override def accept(file: CarbonFile): Boolean = !file.isDirectory
-      }).length == 2)
+      }).length
+      if (CarbonProperties.isTableStatusMultiVersionEnabled) {
+        assert(length == 6)
+      } else {
+        assert(length == 2)
+      }
     }
     else {
-      assert(files.listFiles().length == 2)
+      assert(files.listFiles().length == 6)
     }
     sql("drop table update_status_files")
   }
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala
index 75c7ad78c0..e8cb732521 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala
@@ -271,7 +271,8 @@ class UpdateCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
     sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest22""")
 
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable("iud", "dest22")
-    val tableStatusFile = CarbonTablePath.getTableStatusFilePath(carbonTable.getTablePath)
+    val tableStatusFile = CarbonTablePath.getTableStatusFilePath(carbonTable.getTablePath,
+      carbonTable.getTableStatusVersion)
     FileFactory.getCarbonFile(tableStatusFile).delete()
 
     sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest22""")
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/longstring/VarcharDataTypesBasicTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/longstring/VarcharDataTypesBasicTestCase.scala
index 13ed0da218..ca5a7d02be 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/longstring/VarcharDataTypesBasicTestCase.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/longstring/VarcharDataTypesBasicTestCase.scala
@@ -152,7 +152,7 @@ class VarcharDataTypesBasicTestCase
       CarbonCommonConstants.DATABASE_DEFAULT_NAME, "long_string_table")
     val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
     val segmentStatusManager: SegmentStatusManager =
-      new SegmentStatusManager(absoluteTableIdentifier)
+      new SegmentStatusManager(absoluteTableIdentifier, carbonTable.getTableStatusVersion)
     val segments = segmentStatusManager
       .getValidAndInvalidSegments
       .getValidSegments
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/segment/ShowSegmentTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/segment/ShowSegmentTestCase.scala
index d15d9c60a5..4506b2c947 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/segment/ShowSegmentTestCase.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/segment/ShowSegmentTestCase.scala
@@ -176,19 +176,22 @@ class ShowSegmentTestCase extends QueryTest with BeforeAndAfterAll {
     sql(s"drop table if exists ${tableName}")
     sql(s"create table ${tableName} (name String, age int) STORED AS carbondata "
         + "TBLPROPERTIES('AUTO_LOAD_MERGE'='true','COMPACTION_LEVEL_THRESHOLD'='2,2')")
-    val carbonTable = CarbonEnv.getCarbonTable(Some("default"), tableName)(sqlContext.sparkSession)
     insertTestDataIntoTable(tableName)
     assert(sql(s"show segments on ${ tableName } as select * from ${ tableName }_segments")
              .collect()
              .length == 10)
-    var detail = SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath)
+    var carbonTable = CarbonEnv.getCarbonTable(Some("default"), tableName)(sqlContext.sparkSession)
+    var detail = SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath,
+      carbonTable.getTableStatusVersion)
     var historyDetail = SegmentStatusManager.readLoadHistoryMetadata(carbonTable.getMetadataPath)
     assert(detail.length == 10)
     assert(historyDetail.length == 0)
     sql(s"clean files for table ${tableName} options('force'='true')")
     assert(sql(s"show segments on ${tableName}").collect().length == 2)
     assert(sql(s"show segments on ${tableName} limit 1").collect().length == 1)
-    detail = SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath)
+    carbonTable = CarbonEnv.getCarbonTable(Some("default"), tableName)(sqlContext.sparkSession)
+    detail = SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath,
+      carbonTable.getTableStatusVersion)
     historyDetail = SegmentStatusManager.readLoadHistoryMetadata(carbonTable.getMetadataPath)
     assert(detail.length == 4)
     assert(historyDetail.length == 6)
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableCleanTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableCleanTestCase.scala
index 9342828d12..a3acf44b95 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableCleanTestCase.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableCleanTestCase.scala
@@ -66,7 +66,7 @@ class StandardPartitionTableCleanTestCase extends QueryTest with BeforeAndAfterA
       TableIdentifier(carbonTable.getTableName, Some(carbonTable.getDatabaseName)))
     assert(partitions.get.length == partition)
     val details = SegmentStatusManager.readLoadMetadata(
-      CarbonTablePath.getMetadataPath(carbonTable.getTablePath))
+      CarbonTablePath.getMetadataPath(carbonTable.getTablePath), carbonTable.getTableStatusVersion)
     val segLoad = details.find(_.getLoadName.equals(segmentId)).get
     val seg = new SegmentFileStore(carbonTable.getTablePath, segLoad.getSegmentFile)
     assert(seg.getIndexAndMergeFiles.size == indexes)
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
index 6df78a3563..8dfc225dea 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
@@ -352,7 +352,7 @@ class StandardPartitionTableLoadingTestCase extends QueryTest with BeforeAndAfte
 
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default_mergeindexpartitionthree")
     val details = SegmentStatusManager.readTableStatusFile(
-      CarbonTablePath.getTableStatusFilePath(carbonTable.getTablePath))
+      CarbonTablePath.getTableStatusFilePath(carbonTable.getTablePath, carbonTable.getTableStatusVersion))
     val store = new SegmentFileStore(carbonTable.getTablePath, details(0).getSegmentFile)
     store.readIndexFiles(new Configuration(false))
     store.getIndexFiles
@@ -630,18 +630,18 @@ class StandardPartitionTableLoadingTestCase extends QueryTest with BeforeAndAfte
     sql("select * from partition_cache where b = 1").collect()
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "partition_cache")
     var partitionSpecs: util.List[CatalogTablePartition] = PartitionCacheManager.getIfPresent(
-      PartitionCacheKey(carbonTable.getTableId, carbonTable.getTablePath, 1L))
+      PartitionCacheKey(carbonTable.getTableId, carbonTable.getTablePath, 1L, carbonTable.getTableStatusVersion))
     assert(partitionSpecs.size == 1)
     sql("insert into partition_cache select 'k',2")
     sql("select * from partition_cache where b = 2").collect()
     sql("select * from partition_cache where b = 2").collect()
     partitionSpecs = PartitionCacheManager.getIfPresent(
-      PartitionCacheKey(carbonTable.getTableId, carbonTable.getTablePath, 1L))
+      PartitionCacheKey(carbonTable.getTableId, carbonTable.getTablePath, 1L, carbonTable.getTableStatusVersion))
     assert(partitionSpecs.size == 2)
     sql("delete from table partition_cache where segment.id in (1)")
     sql("select * from partition_cache where b = 2").collect()
     partitionSpecs = PartitionCacheManager.getIfPresent(
-      PartitionCacheKey(carbonTable.getTableId, carbonTable.getTablePath, 1L))
+      PartitionCacheKey(carbonTable.getTableId, carbonTable.getTablePath, 1L, carbonTable.getTableStatusVersion))
     assert(partitionSpecs.size == 1)
     CarbonProperties.getInstance().addProperty("carbon.read.partition.hive.direct", "true")
   }
@@ -655,7 +655,7 @@ class StandardPartitionTableLoadingTestCase extends QueryTest with BeforeAndAfte
     sql("select * from partition_cache where b = 1").collect()
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "partition_cache")
     val partitionSpecs: util.List[CatalogTablePartition] = PartitionCacheManager.getIfPresent(
-      PartitionCacheKey(carbonTable.getTableId, carbonTable.getTablePath, 1L))
+      PartitionCacheKey(carbonTable.getTableId, carbonTable.getTablePath, 1L, carbonTable.getTableStatusVersion))
     assert(partitionSpecs.size == 1)
     assert(partitionSpecs.get(0).spec.size == 2)
     CarbonProperties.getInstance().addProperty("carbon.read.partition.hive.direct", "true")
@@ -673,7 +673,7 @@ class StandardPartitionTableLoadingTestCase extends QueryTest with BeforeAndAfte
     checkAnswer(sql("select count(*) from partition_cache"), Seq(Row(4)))
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "partition_cache")
     val partitionSpecs = PartitionCacheManager.getIfPresent(
-      PartitionCacheKey(carbonTable.getTableId, carbonTable.getTablePath, 1L))
+      PartitionCacheKey(carbonTable.getTableId, carbonTable.getTablePath, 1L, carbonTable.getTableStatusVersion))
     assert(partitionSpecs.size == 2)
     CarbonProperties.getInstance().addProperty("carbon.read.partition.hive.direct", "true")
   }
@@ -686,7 +686,7 @@ class StandardPartitionTableLoadingTestCase extends QueryTest with BeforeAndAfte
     sql("select * from partition_cache where b = 1").collect()
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "partition_cache")
     val partitionSpecs = PartitionCacheManager.getIfPresent(
-      PartitionCacheKey(carbonTable.getTableId, carbonTable.getTablePath, 1L))
+      PartitionCacheKey(carbonTable.getTableId, carbonTable.getTablePath, 1L, carbonTable.getTableStatusVersion))
     assert(partitionSpecs.size == 1)
     CarbonProperties.getInstance().addProperty("carbon.read.partition.hive.direct", "true")
   }
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableQueryTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableQueryTestCase.scala
index 20a0d8c914..216bc44778 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableQueryTestCase.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableQueryTestCase.scala
@@ -340,6 +340,7 @@ test("Creation of partition table should fail if the colname in table schema and
       """.stripMargin)
     sql("insert into partitionTable select 1,'huawei','abc'")
     val location = target + "/" + "def"
+    FileFactory.deleteAllCarbonFilesOfDir(FileFactory.getCarbonFile(location))
     checkAnswer(sql("show partitions partitionTable"), Seq(Row("email=abc")))
     sql(s"""alter table partitionTable add partition (email='def') location '$location'""")
     sql("insert into partitionTable select 1,'huawei','def'")
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/view/rewrite/MVIncrementalLoadingTestcase.scala b/integration/spark/src/test/scala/org/apache/carbondata/view/rewrite/MVIncrementalLoadingTestcase.scala
index 8225d6de56..c1ac180e9b 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/view/rewrite/MVIncrementalLoadingTestcase.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/view/rewrite/MVIncrementalLoadingTestcase.scala
@@ -65,7 +65,8 @@ class MVIncrementalLoadingTestcase extends QueryTest with BeforeAndAfterAll {
       CarbonCommonConstants.DATABASE_DEFAULT_NAME,
       "mv1"
     )
-    var loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTable.getMetadataPath)
+    var loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTable.getMetadataPath,
+      viewTable.getTableStatusVersion)
     var segmentMap = getSegmentMap(loadMetadataDetails(0).getExtraInfo)
     val segmentList = new java.util.ArrayList[String]()
     segmentList.add("0")
@@ -75,7 +76,8 @@ class MVIncrementalLoadingTestcase extends QueryTest with BeforeAndAfterAll {
     loadDataToFactTable("test_table")
     loadDataToFactTable("test_table1")
     sql(s"refresh materialized view mv1")
-    loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTable.getMetadataPath)
+    loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTable.getMetadataPath,
+      viewTable.getTableStatusVersion)
     segmentMap = getSegmentMap(loadMetadataDetails(1).getExtraInfo)
     segmentList.clear()
     segmentList.add("1")
@@ -110,7 +112,8 @@ class MVIncrementalLoadingTestcase extends QueryTest with BeforeAndAfterAll {
     val viewTable = CarbonMetadata.getInstance().getCarbonTable(
       CarbonCommonConstants.DATABASE_DEFAULT_NAME,
       "mv1")
-    val loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTable.getMetadataPath)
+    val loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTable.getMetadataPath,
+      viewTable.getTableStatusVersion)
 
     val segmentMap = getSegmentMap(loadMetadataDetails(0).getExtraInfo)
     val segmentList = new java.util.ArrayList[String]()
@@ -146,11 +149,13 @@ class MVIncrementalLoadingTestcase extends QueryTest with BeforeAndAfterAll {
     val viewTable = CarbonEnv.getCarbonTable(
       Option(CarbonCommonConstants.DATABASE_DEFAULT_NAME),
       "mv1")(sqlContext.sparkSession)
-    var loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTable.getMetadataPath)
+    var loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTable.getMetadataPath,
+      viewTable.getTableStatusVersion)
     assert(loadMetadataDetails(0).getSegmentStatus == SegmentStatus.MARKED_FOR_DELETE)
     checkAnswer(sql("select * from main_table"), sql("select * from testtable"))
     sql(s"refresh materialized view mv1")
-    loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTable.getMetadataPath)
+    loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTable.getMetadataPath,
+      viewTable.getTableStatusVersion)
     val segmentMap = getSegmentMap(loadMetadataDetails(1).getExtraInfo)
     val segmentList = new java.util.ArrayList[String]()
     segmentList.add("0")
@@ -183,7 +188,8 @@ class MVIncrementalLoadingTestcase extends QueryTest with BeforeAndAfterAll {
       CarbonCommonConstants.DATABASE_DEFAULT_NAME,
       "mv1"
     )
-    val loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTable.getMetadataPath)
+    val loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTable.getMetadataPath,
+      viewTable.getTableStatusVersion)
     val segmentMap = getSegmentMap(loadMetadataDetails(3).getExtraInfo)
     val segmentList = new java.util.ArrayList[String]()
     segmentList.add("0")
@@ -221,7 +227,8 @@ class MVIncrementalLoadingTestcase extends QueryTest with BeforeAndAfterAll {
     val viewTable = CarbonMetadata.getInstance().getCarbonTable(
       CarbonCommonConstants.DATABASE_DEFAULT_NAME,
       "mv1")
-    val loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTable.getMetadataPath)
+    val loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTable.getMetadataPath,
+      viewTable.getTableStatusVersion)
     val segmentMap = getSegmentMap(loadMetadataDetails(2).getExtraInfo)
     val segmentList = new java.util.ArrayList[String]()
     segmentList.add("0.1")
@@ -247,11 +254,13 @@ class MVIncrementalLoadingTestcase extends QueryTest with BeforeAndAfterAll {
     val viewTable = CarbonEnv.getCarbonTable(
       Option(CarbonCommonConstants.DATABASE_DEFAULT_NAME),
       "mv1")(sqlContext.sparkSession)
-    var loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTable.getMetadataPath)
+    var loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTable.getMetadataPath,
+      viewTable.getTableStatusVersion)
     assert(loadMetadataDetails(0).getSegmentStatus == SegmentStatus.MARKED_FOR_DELETE)
     checkAnswer(sql(" select a, sum(b) from test_table  group by a"), Seq(Row("d", null)))
     sql(s"refresh materialized view mv1")
-    loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTable.getMetadataPath)
+    loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTable.getMetadataPath,
+      viewTable.getTableStatusVersion)
     val segmentMap = getSegmentMap(loadMetadataDetails(1).getExtraInfo)
     val segmentList = new java.util.ArrayList[String]()
     segmentList.add("2")
@@ -358,7 +367,8 @@ class MVIncrementalLoadingTestcase extends QueryTest with BeforeAndAfterAll {
     val viewTable = CarbonMetadata.getInstance().getCarbonTable(
       CarbonCommonConstants.DATABASE_DEFAULT_NAME,
       "mv1")
-    val loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTable.getMetadataPath)
+    val loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTable.getMetadataPath,
+      viewTable.getTableStatusVersion)
     assert(loadMetadataDetails(0).getSegmentStatus == SegmentStatus.COMPACTED)
     assert(loadMetadataDetails(1).getSegmentStatus == SegmentStatus.COMPACTED)
     var segmentMap = getSegmentMap(loadMetadataDetails(2).getExtraInfo)
@@ -407,7 +417,8 @@ class MVIncrementalLoadingTestcase extends QueryTest with BeforeAndAfterAll {
       CarbonCommonConstants.DATABASE_DEFAULT_NAME,
       "mv1"
     )
-    var loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTable.getMetadataPath)
+    var loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTable.getMetadataPath,
+      viewTable.getTableStatusVersion)
     var segmentMap = getSegmentMap(loadMetadataDetails(0).getExtraInfo)
     val segmentList = new java.util.ArrayList[String]()
     segmentList.add("0")
@@ -416,7 +427,8 @@ class MVIncrementalLoadingTestcase extends QueryTest with BeforeAndAfterAll {
     assert(TestUtil.verifyMVHit(df2.queryExecution.optimizedPlan, "mv1"))
     loadDataToFactTable("test_table")
     loadDataToFactTable("test_table1")
-    loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTable.getMetadataPath)
+    loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTable.getMetadataPath,
+      viewTable.getTableStatusVersion)
     segmentMap = getSegmentMap(loadMetadataDetails(1).getExtraInfo)
     segmentList.clear()
     segmentList.add("1")
@@ -454,7 +466,8 @@ class MVIncrementalLoadingTestcase extends QueryTest with BeforeAndAfterAll {
     val viewTable = CarbonMetadata.getInstance().getCarbonTable(
       CarbonCommonConstants.DATABASE_DEFAULT_NAME,
       "mv1")
-    var loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTable.getMetadataPath)
+    var loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTable.getMetadataPath,
+      viewTable.getTableStatusVersion)
     assert(loadMetadataDetails(0).getSegmentStatus == SegmentStatus.MARKED_FOR_DELETE)
     var segmentMap = getSegmentMap(loadMetadataDetails(1).getExtraInfo)
     val segmentList = new java.util.ArrayList[String]()
@@ -491,7 +504,8 @@ class MVIncrementalLoadingTestcase extends QueryTest with BeforeAndAfterAll {
     val viewTable = CarbonMetadata.getInstance().getCarbonTable(
       CarbonCommonConstants.DATABASE_DEFAULT_NAME,
       "mv1")
-    var loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTable.getMetadataPath)
+    var loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTable.getMetadataPath,
+      viewTable.getTableStatusVersion)
     assert(loadMetadataDetails(0).getSegmentStatus == SegmentStatus.MARKED_FOR_DELETE)
     var segmentMap = getSegmentMap(loadMetadataDetails(1).getExtraInfo)
     val segmentList = new java.util.ArrayList[String]()
@@ -531,7 +545,8 @@ class MVIncrementalLoadingTestcase extends QueryTest with BeforeAndAfterAll {
     val viewTable = CarbonMetadata.getInstance().getCarbonTable(
       CarbonCommonConstants.DATABASE_DEFAULT_NAME,
       "mv1")
-    val loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTable.getMetadataPath)
+    val loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTable.getMetadataPath,
+      viewTable.getTableStatusVersion)
     assert(loadMetadataDetails.length == 1)
     assert(loadMetadataDetails(0).getSegmentStatus == SegmentStatus.MARKED_FOR_DELETE)
   }
@@ -563,7 +578,8 @@ class MVIncrementalLoadingTestcase extends QueryTest with BeforeAndAfterAll {
     val viewTable = CarbonMetadata.getInstance().getCarbonTable(
       CarbonCommonConstants.DATABASE_DEFAULT_NAME,
       "mv1")
-    val loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTable.getMetadataPath)
+    val loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTable.getMetadataPath,
+      viewTable.getTableStatusVersion)
     assert(loadMetadataDetails.length == 1)
     var segmentMap = getSegmentMap(loadMetadataDetails(0).getExtraInfo)
     val segmentList = new java.util.ArrayList[String]()
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/view/rewrite/TestPartitionWithMV.scala b/integration/spark/src/test/scala/org/apache/carbondata/view/rewrite/TestPartitionWithMV.scala
index 552424314c..139acd2373 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/view/rewrite/TestPartitionWithMV.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/view/rewrite/TestPartitionWithMV.scala
@@ -569,7 +569,7 @@ class TestPartitionWithMV extends QueryTest with BeforeAndAfterAll with BeforeAn
   test("test dropping partition which has already been deleted") {
     sql("drop table if exists partitiontable")
     sql("create table partitiontable(id int,name string) partitioned by (email string) " +
-        "STORED AS carbondata tblproperties('sort_scope'='global_sort')")
+        "STORED AS carbondata tblproperties('sort_scope'='global_sort', 'sort_columns'='email')")
     sql("insert into table partitiontable select 1,'huawei','abc'")
     sql("create materialized view ag1 as select count(email),id" +
         " from partitiontable group by id")
@@ -598,7 +598,7 @@ class TestPartitionWithMV extends QueryTest with BeforeAndAfterAll with BeforeAn
   test("test mv table creation with count(*) on Partition table") {
     sql("drop table if exists partitiontable")
     sql("create table partitiontable(id int,name string) partitioned by (email string) " +
-        "STORED AS carbondata tblproperties('sort_scope'='global_sort')")
+        "STORED AS carbondata tblproperties('sort_scope'='global_sort', 'sort_columns'='email')")
     sql("insert into table partitiontable select 1,'huawei','abc'")
     sql("drop materialized view if exists ag1")
     sql("create materialized view ag1 as select count(*),id" +
diff --git a/integration/spark/src/test/scala/org/apache/spark/carbondata/TableStatusBackupTest.scala b/integration/spark/src/test/scala/org/apache/spark/carbondata/TableStatusBackupTest.scala
index f17f2e1771..43557c78ae 100644
--- a/integration/spark/src/test/scala/org/apache/spark/carbondata/TableStatusBackupTest.scala
+++ b/integration/spark/src/test/scala/org/apache/spark/carbondata/TableStatusBackupTest.scala
@@ -32,6 +32,8 @@ import org.apache.carbondata.core.util.path.CarbonTablePath
 
 class TableStatusBackupTest extends QueryTest with BeforeAndAfterAll {
   override protected def beforeAll(): Unit = {
+    CarbonProperties.getInstance().addProperty(
+      CarbonCommonConstants.CARBON_ENABLE_MULTI_VERSION_TABLE_STATUS, "false")
     CarbonProperties.getInstance().addProperty(
       CarbonCommonConstants.ENABLE_TABLE_STATUS_BACKUP, "true")
     sql("drop table if exists source")
@@ -42,12 +44,16 @@ class TableStatusBackupTest extends QueryTest with BeforeAndAfterAll {
     sql("drop table if exists source")
     CarbonProperties.getInstance().addProperty(
       CarbonCommonConstants.ENABLE_TABLE_STATUS_BACKUP, "false")
+    CarbonProperties.getInstance().addProperty(
+      CarbonCommonConstants.CARBON_ENABLE_MULTI_VERSION_TABLE_STATUS,
+      CarbonCommonConstants.CARBON_ENABLE_MULTI_VERSION_TABLE_STATUS_DEFAULT)
   }
 
   test("backup table status file") {
     sql("insert into source values ('A'), ('B')")
-    val tablePath = CarbonEnv.getCarbonTable(None, "source")(sqlContext.sparkSession).getTablePath
-    val tableStatusFilePath = CarbonTablePath.getTableStatusFilePath(tablePath)
+    val table = CarbonEnv.getCarbonTable(None, "source")(sqlContext.sparkSession)
+    val tableStatusFilePath = CarbonTablePath.getTableStatusFilePath(table.getTablePath,
+      table.getTableStatusVersion)
     val oldTableStatus = SegmentStatusManager.readTableStatusFile(tableStatusFilePath)
 
     var mock = new MockUp[SegmentStatusManager]() {
diff --git a/integration/spark/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableUpgradeSegmentTest.scala b/integration/spark/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableUpgradeSegmentTest.scala
index b8fe9f781b..0ffeb103e3 100644
--- a/integration/spark/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableUpgradeSegmentTest.scala
+++ b/integration/spark/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableUpgradeSegmentTest.scala
@@ -36,13 +36,13 @@ class AlterTableUpgradeSegmentTest extends QueryTest with BeforeAndAfterAll {
 
   private def removeDataAndIndexSizeFromTableStatus(table: CarbonTable): Unit = {
     val loadMetaDataDetails = SegmentStatusManager.readTableStatusFile(CarbonTablePath
-      .getTableStatusFilePath(table.getTablePath))
+      .getTableStatusFilePath(table.getTablePath, table.getTableStatusVersion))
     loadMetaDataDetails.foreach { loadMetaDataDetail =>
       loadMetaDataDetail.setIndexSize("0")
       loadMetaDataDetail.setDataSize("0")
     }
     SegmentStatusManager.writeLoadDetailsIntoFile(CarbonTablePath
-      .getTableStatusFilePath(table.getTablePath), loadMetaDataDetails)
+      .getTableStatusFilePath(table.getTablePath, table.getTableStatusVersion), loadMetaDataDetails)
   }
 
   test("test alter table upgrade segment test") {
@@ -50,12 +50,12 @@ class AlterTableUpgradeSegmentTest extends QueryTest with BeforeAndAfterAll {
       CarbonEnv.getCarbonTable(TableIdentifier("altertest"))(sqlContext.sparkSession)
     removeDataAndIndexSizeFromTableStatus(carbonTable)
     val loadMetaDataDetails = SegmentStatusManager.readTableStatusFile(CarbonTablePath
-      .getTableStatusFilePath(carbonTable.getTablePath))
+      .getTableStatusFilePath(carbonTable.getTablePath, carbonTable.getTableStatusVersion))
     loadMetaDataDetails.foreach(detail => assert(detail.getIndexSize.toInt + detail.getDataSize
       .toInt == 0))
     sql("alter table altertest compact 'upgrade_segment'")
     val loadMetaDataDetailsNew = SegmentStatusManager.readTableStatusFile(CarbonTablePath
-      .getTableStatusFilePath(carbonTable.getTablePath))
+      .getTableStatusFilePath(carbonTable.getTablePath, carbonTable.getTableStatusVersion))
     loadMetaDataDetailsNew.foreach{detail =>
       assert(detail.getIndexSize.toInt != 0)
       assert(detail.getDataSize.toInt != 0)}
diff --git a/integration/spark/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala b/integration/spark/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala
index 9d982a134d..bbf678f7b7 100644
--- a/integration/spark/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala
+++ b/integration/spark/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala
@@ -99,17 +99,23 @@ class CarbonCommandSuite extends QueryTest with BeforeAndAfterAll {
 
   private lazy val location = CarbonProperties.getStorePath()
 
-
   test("delete segment by id") {
-    DeleteSegmentById.main(Array(s"${location}", "carbon_table", "0"))
-    assert(!CarbonStore.isSegmentValid("default", "carbon_table", location, "0"))
+    DeleteSegmentById.main(Array(s"${ location }", "carbon_table", "0"))
+    assert(!CarbonStore.isSegmentValid("default", "carbon_table",
+      location + "/carbon_table", "0",
+      CarbonMetadata.getInstance().getCarbonTable("default", "carbon_table").getTableStatusVersion
+    ))
   }
 
   test("delete segment by date") {
     createAndLoadTestTable("carbon_table2", "csv_table")
     val time = new Timestamp(new Date().getTime)
     DeleteSegmentByDate.main(Array(s"${location}", "carbon_table2", time.toString))
-    assert(!CarbonStore.isSegmentValid("default", "carbon_table2", location, "0"))
+    val version = CarbonMetadata.getInstance()
+      .getCarbonTable("default", "carbon_table2")
+      .getTableStatusVersion
+    assert(!CarbonStore.isSegmentValid("default",
+      "carbon_table2", location + "/carbon_table2", "0", version))
     dropTable("carbon_table2")
   }
 
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
index 050b538187..56449cba72 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
@@ -28,6 +28,7 @@ import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
 import org.apache.carbondata.core.statusmanager.SegmentUpdateStatusManager;
+import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.DataLoadMetrics;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 
@@ -230,6 +231,8 @@ public class CarbonLoadModel implements Serializable {
 
   private boolean skipParsers = false;
 
+  private String latestTableStatusWriteVersion = "";
+
   public void setSkipParsers() {
     skipParsers = true;
   }
@@ -485,6 +488,7 @@ public class CarbonLoadModel implements Serializable {
     copyObj.metrics = metrics;
     copyObj.isLoadWithoutConverterStep = isLoadWithoutConverterStep;
     copyObj.isLoadWithoutConverterWithoutReArrangeStep = isLoadWithoutConverterWithoutReArrangeStep;
+    copyObj.latestTableStatusWriteVersion = latestTableStatusWriteVersion;
     return copyObj;
   }
 
@@ -809,8 +813,14 @@ public class CarbonLoadModel implements Serializable {
    * Read segments metadata from table status file and set it to this load model object
    */
   public void readAndSetLoadMetadataDetails() {
+    String tblStatusVersion = carbonDataLoadSchema.getCarbonTable().getTableStatusVersion();
     String metadataPath = CarbonTablePath.getMetadataPath(tablePath);
-    LoadMetadataDetails[] details = SegmentStatusManager.readLoadMetadata(metadataPath);
+    LoadMetadataDetails[] details;
+    if (tblStatusVersion.isEmpty()) {
+      details = SegmentStatusManager.readLoadMetadata(metadataPath);
+    } else {
+      details = SegmentStatusManager.readLoadMetadata(metadataPath, tblStatusVersion);
+    }
     setLoadMetadataDetails(Arrays.asList(details));
   }
 
@@ -905,4 +915,17 @@ public class CarbonLoadModel implements Serializable {
   public void setNonSchemaColumnsPresent(boolean nonSchemaColumnsPresent) {
     this.nonSchemaColumnsPresent = nonSchemaColumnsPresent;
   }
+
+  public String getLatestTableStatusWriteVersion() {
+    if (!CarbonProperties.isTableStatusMultiVersionEnabled()) {
+      return "";
+    }
+    return latestTableStatusWriteVersion;
+  }
+
+  public void setLatestTableStatusWriteVersion(String latestTableStatusWriteVersion) {
+    if (this.latestTableStatusWriteVersion.isEmpty()) {
+      this.latestTableStatusWriteVersion = latestTableStatusWriteVersion;
+    }
+  }
 }
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
index 3fe78e6450..42445e7b75 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
@@ -162,7 +162,8 @@ public final class CarbonDataMergerUtil {
     boolean tableStatusUpdationStatus = false;
     AbsoluteTableIdentifier identifier =
         carbonLoadModel.getCarbonDataLoadSchema().getCarbonTable().getAbsoluteTableIdentifier();
-    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(identifier);
+    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(identifier,
+        carbonLoadModel.getCarbonDataLoadSchema().getCarbonTable().getTableStatusVersion());
 
     ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
 
@@ -177,9 +178,14 @@ public final class CarbonDataMergerUtil {
         LOGGER.info("Acquired lock for the table " + carbonLoadModel.getDatabaseName() + "."
             + carbonLoadModel.getTableName() + " for table status updation ");
 
-        String statusFilePath = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath());
+        carbonLoadModel.setLatestTableStatusWriteVersion(
+            String.valueOf(System.currentTimeMillis()));
 
-        LoadMetadataDetails[] loadDetails = SegmentStatusManager.readLoadMetadata(metaDataFilepath);
+        String statusFilePath = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath(),
+            carbonLoadModel.getLatestTableStatusWriteVersion());
+
+        LoadMetadataDetails[] loadDetails = SegmentStatusManager.readLoadMetadata(metaDataFilepath,
+            carbonLoadModel.getCarbonDataLoadSchema().getCarbonTable().getTableStatusVersion());
 
         long modificationOrDeletionTimeStamp = CarbonUpdateUtil.readCurrentTime();
         for (LoadMetadataDetails loadDetail : loadDetails) {
@@ -840,8 +846,8 @@ public final class CarbonDataMergerUtil {
 
     SegmentStatusManager.ValidAndInvalidSegmentsInfo validAndInvalidSegments = null;
     try {
-      validAndInvalidSegments = new SegmentStatusManager(carbonTable.getAbsoluteTableIdentifier())
-          .getValidAndInvalidSegments(carbonTable.isMV());
+      validAndInvalidSegments = new SegmentStatusManager(carbonTable.getAbsoluteTableIdentifier(),
+          carbonTable.getTableStatusVersion()).getValidAndInvalidSegments(carbonTable.isMV());
     } catch (IOException e) {
       LOGGER.error("Error while getting valid segment list for a table identifier");
       throw new IOException();
@@ -1077,9 +1083,11 @@ public final class CarbonDataMergerUtil {
     String metaDataFilepath = table.getMetadataPath();
     AbsoluteTableIdentifier identifier = table.getAbsoluteTableIdentifier();
 
-    String tableStatusPath = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath());
+    String tableStatusPath = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath(),
+        table.getTableStatusVersion());
 
-    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(identifier);
+    SegmentStatusManager segmentStatusManager =
+        new SegmentStatusManager(identifier, table.getTableStatusVersion());
 
     ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
 
@@ -1093,7 +1101,7 @@ public final class CarbonDataMergerUtil {
                         + " for table status updation");
 
         LoadMetadataDetails[] listOfLoadFolderDetailsArray =
-                SegmentStatusManager.readLoadMetadata(metaDataFilepath);
+            SegmentStatusManager.readLoadMetadata(metaDataFilepath, table.getTableStatusVersion());
 
         for (LoadMetadataDetails loadMetadata : listOfLoadFolderDetailsArray) {
           if (loadMetadata.getLoadName().equalsIgnoreCase("0")) {
diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
index 919742dc7f..6a43452e97 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
@@ -244,8 +244,8 @@ public final class CarbonLoaderUtil {
         FileFactory.mkdirs(metadataPath);
       }
     }
-    String tableStatusPath = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath());
-    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(identifier);
+    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(identifier,
+        loadModel.getCarbonDataLoadSchema().getCarbonTable().getTableStatusVersion());
     ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
     int retryCount = CarbonLockUtil
         .getLockProperty(CarbonCommonConstants.NUMBER_OF_TRIES_FOR_CONCURRENT_LOCK,
@@ -258,9 +258,18 @@ public final class CarbonLoaderUtil {
         LOGGER.info(
             "Acquired lock for table" + loadModel.getDatabaseName() + "." + loadModel.getTableName()
                 + " for table status updation");
+        loadModel.setLatestTableStatusWriteVersion(String.valueOf(System.currentTimeMillis()));
+        String newTblStatusVersion = loadModel.getLatestTableStatusWriteVersion();
+        String tableStatusPath = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath(),
+            loadModel.getLatestTableStatusWriteVersion());
+        if (newMetaEntry.getSegmentStatus() == SegmentStatus.INSERT_IN_PROGRESS
+            || newMetaEntry.getSegmentStatus() == SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS) {
+          newTblStatusVersion =
+              loadModel.getCarbonDataLoadSchema().getCarbonTable().getTableStatusVersion();
+        }
         LoadMetadataDetails[] listOfLoadFolderDetailsArray =
             SegmentStatusManager.readLoadMetadata(
-                CarbonTablePath.getMetadataPath(identifier.getTablePath()));
+                CarbonTablePath.getMetadataPath(identifier.getTablePath()), newTblStatusVersion);
         List<LoadMetadataDetails> listOfLoadFolderDetails =
             new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
         Collections.addAll(listOfLoadFolderDetails, listOfLoadFolderDetailsArray);
@@ -1198,7 +1207,8 @@ public final class CarbonLoaderUtil {
         carbonTable.getDatabaseName(),
         carbonTable.getTablePath(),
         carbonTable.getMetadataPath(),
-        status);
+        status,
+        carbonTable.getTableStatusVersion());
   }
 
   /**
@@ -1206,15 +1216,17 @@ public final class CarbonLoaderUtil {
    */
   public static void updateTableStatusInCaseOfFailure(String loadName,
       AbsoluteTableIdentifier absoluteTableIdentifier, String tableName, String databaseName,
-      String tablePath, String metaDataPath, SegmentStatus status) throws IOException {
-    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier);
+      String tablePath, String metaDataPath, SegmentStatus status, String version)
+      throws IOException {
+    SegmentStatusManager segmentStatusManager =
+        new SegmentStatusManager(absoluteTableIdentifier, version);
     ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
     try {
       if (carbonLock.lockWithRetries()) {
         LOGGER.info("Acquired lock for table" + databaseName + "." + tableName
             + " for table status updation");
         LoadMetadataDetails[] loadMetadataDetails =
-            SegmentStatusManager.readLoadMetadata(metaDataPath);
+            SegmentStatusManager.readLoadMetadata(metaDataPath, version);
         boolean ifTableStatusUpdateRequired = false;
         for (LoadMetadataDetails loadMetadataDetail : loadMetadataDetails) {
           if (loadMetadataDetail.getSegmentStatus() == status && loadName
@@ -1226,7 +1238,7 @@ public final class CarbonLoaderUtil {
         }
         if (ifTableStatusUpdateRequired) {
           SegmentStatusManager
-              .writeLoadDetailsIntoFile(CarbonTablePath.getTableStatusFilePath(tablePath),
+              .writeLoadDetailsIntoFile(CarbonTablePath.getTableStatusFilePath(tablePath, version),
                   loadMetadataDetails);
         }
       } else {
diff --git a/streaming/src/main/java/org/apache/carbondata/streaming/segment/StreamSegment.java b/streaming/src/main/java/org/apache/carbondata/streaming/segment/StreamSegment.java
index 1a2ad0bd8d..a5c1c4f9b5 100644
--- a/streaming/src/main/java/org/apache/carbondata/streaming/segment/StreamSegment.java
+++ b/streaming/src/main/java/org/apache/carbondata/streaming/segment/StreamSegment.java
@@ -70,9 +70,9 @@ public class StreamSegment {
   /**
    * get stream segment or create new stream segment if not exists
    */
-  public static String open(CarbonTable table) throws IOException {
+  public static String open(CarbonTable table, CarbonLoadModel loadModel) throws IOException {
     SegmentStatusManager segmentStatusManager =
-        new SegmentStatusManager(table.getAbsoluteTableIdentifier());
+        new SegmentStatusManager(table.getAbsoluteTableIdentifier(), table.getTableStatusVersion());
     ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
     try {
       if (carbonLock.lockWithRetries()) {
@@ -80,9 +80,8 @@ public class StreamSegment {
             "Acquired lock for table" + table.getDatabaseName() + "." + table.getTableName()
                 + " for stream table get or create segment");
 
-        LoadMetadataDetails[] details =
-            SegmentStatusManager.readLoadMetadata(
-                CarbonTablePath.getMetadataPath(table.getTablePath()));
+        LoadMetadataDetails[] details = SegmentStatusManager.readLoadMetadata(
+            CarbonTablePath.getMetadataPath(table.getTablePath()), table.getTableStatusVersion());
         LoadMetadataDetails streamSegment = null;
         for (LoadMetadataDetails detail : details) {
           if (FileFormat.ROW_V1.equals(detail.getFileFormat())) {
@@ -93,7 +92,8 @@ public class StreamSegment {
           }
         }
         if (null == streamSegment) {
-          return createNewSegment(table, details);
+          loadModel.setLatestTableStatusWriteVersion(String.valueOf(System.currentTimeMillis()));
+          return createNewSegment(table, details, loadModel.getLatestTableStatusWriteVersion());
         } else {
           return streamSegment.getLoadName();
         }
@@ -115,8 +115,8 @@ public class StreamSegment {
     }
   }
 
-  private static String createNewSegment(CarbonTable table, LoadMetadataDetails[] details)
-      throws IOException {
+  private static String createNewSegment(CarbonTable table, LoadMetadataDetails[] details,
+      String latestTblStatusVersion) throws IOException {
     int segmentId = SegmentStatusManager.createNewSegmentId(details);
     LoadMetadataDetails newDetail = new LoadMetadataDetails();
     newDetail.setLoadName(String.valueOf(segmentId));
@@ -130,9 +130,9 @@ public class StreamSegment {
       newDetails[i] = details[i];
     }
     newDetails[i] = newDetail;
-    SegmentStatusManager
-        .writeLoadDetailsIntoFile(CarbonTablePath.getTableStatusFilePath(table.getTablePath()),
-            newDetails);
+    SegmentStatusManager.writeLoadDetailsIntoFile(
+        CarbonTablePath.getTableStatusFilePath(table.getTablePath(), latestTblStatusVersion),
+        newDetails);
     return newDetail.getLoadName();
   }
 
@@ -142,7 +142,7 @@ public class StreamSegment {
   public static String close(CarbonTable table, String segmentId)
       throws IOException {
     SegmentStatusManager segmentStatusManager =
-        new SegmentStatusManager(table.getAbsoluteTableIdentifier());
+        new SegmentStatusManager(table.getAbsoluteTableIdentifier(), table.getTableStatusVersion());
     ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
     try {
       if (carbonLock.lockWithRetries()) {
@@ -150,9 +150,8 @@ public class StreamSegment {
             "Acquired lock for table" + table.getDatabaseName() + "." + table.getTableName()
                 + " for stream table finish segment");
 
-        LoadMetadataDetails[] details =
-            SegmentStatusManager.readLoadMetadata(
-                CarbonTablePath.getMetadataPath(table.getTablePath()));
+        LoadMetadataDetails[] details = SegmentStatusManager.readLoadMetadata(
+            CarbonTablePath.getMetadataPath(table.getTablePath()), table.getTableStatusVersion());
         for (LoadMetadataDetails detail : details) {
           if (segmentId.equals(detail.getLoadName())) {
             detail.setLoadEndTime(System.currentTimeMillis());
@@ -160,7 +159,7 @@ public class StreamSegment {
             break;
           }
         }
-        return createNewSegment(table, details);
+        return createNewSegment(table, details, table.getTableStatusVersion());
       } else {
         LOGGER.error(
             "Not able to acquire the status update lock for streaming table " + table
@@ -189,7 +188,8 @@ public class StreamSegment {
     try {
       if (statusLock.lockWithRetries()) {
         LoadMetadataDetails[] details =
-            SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath());
+            SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath(),
+                carbonTable.getTableStatusVersion());
         boolean updated = false;
         for (LoadMetadataDetails detail : details) {
           if (SegmentStatus.STREAMING == detail.getSegmentStatus()) {
@@ -200,8 +200,8 @@ public class StreamSegment {
         }
         if (updated) {
           SegmentStatusManager.writeLoadDetailsIntoFile(
-              CarbonTablePath.getTableStatusFilePath(carbonTable.getTablePath()),
-              details);
+              CarbonTablePath.getTableStatusFilePath(carbonTable.getTablePath(),
+                  carbonTable.getTableStatusVersion()), details);
         }
       } else {
         String msg = "Failed to acquire table status lock of " + carbonTable.getDatabaseName()