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

jackylk 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 586aac5  [CARBONDATA-3677] Fixed performance issue for drop table
586aac5 is described below

commit 586aac59b7a428d54dfb071cbd8f316c76ada366
Author: kunal642 <kunalkapoor...@gmail.com>
AuthorDate: Mon Feb 3 20:52:13 2020 +0530

    [CARBONDATA-3677] Fixed performance issue for drop table
    
    Why is this PR needed?
    Drop table is very slow because of unnecessary jobs being fired to clear 
cache from executors.
    
    What changes were proposed in this PR?
    1. Clear the datamaps for embedded mode when the reader is closed to avoid 
firing a 2nd job.
    2. While drop only fire the job to clear from executors if either the table 
has a CG datamap or index server is enabled.
    
    Does this PR introduce any user interface change?
    No
    
    Is any new testcase added?
    No
    
    This closes #3601
---
 .../core/datamap/DataMapStoreManager.java          | 26 +++++++++++++--
 .../core/datamap/DistributableDataMapFormat.java   |  5 +++
 .../carbondata/hadoop/CarbonRecordReader.java      |  2 +-
 .../StandardPartitionTableCleanTestCase.scala      |  6 ++--
 .../carbondata/indexserver/DataMapJobs.scala       |  3 --
 .../indexserver/DistributedPruneRDD.scala          |  1 -
 .../command/management/CarbonLoadDataCommand.scala | 18 +++-------
 .../execution/command/CarbonHiveCommands.scala     | 13 --------
 .../carbondata/sdk/file/CarbonReaderBuilder.java   |  4 +--
 .../carbondata/sdk/file/CarbonReaderTest.java      | 38 +++++++++++-----------
 10 files changed, 58 insertions(+), 58 deletions(-)

diff --git 
a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
 
b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
index 6a4a5de..cf3c897 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
@@ -513,7 +513,18 @@ public final class DataMapStoreManager {
    * @param identifier Table identifier
    */
   public void clearDataMaps(AbsoluteTableIdentifier identifier) {
-    clearDataMaps(identifier, true);
+    CarbonTable carbonTable = getCarbonTable(identifier);
+    boolean launchJob = false;
+    try {
+      // launchJob will be true if either the table has a CGDatamap or index 
server is enabled for
+      // the specified table.
+      launchJob = hasCGDataMap(carbonTable) ||
+          
CarbonProperties.getInstance().isDistributedPruningEnabled(identifier.getDatabaseName(),
+              identifier.getTableName());
+    } catch (IOException e) {
+      LOGGER.warn("Unable to launch job to clear datamaps.", e);
+    }
+    clearDataMapCache(identifier, launchJob);
   }
 
   /**
@@ -521,9 +532,9 @@ public final class DataMapStoreManager {
    *
    * @param identifier Table identifier
    */
-  public void clearDataMaps(AbsoluteTableIdentifier identifier, boolean 
launchJob) {
+  public void clearDataMapCache(AbsoluteTableIdentifier identifier, boolean 
clearInAllWorkers) {
     String tableId = identifier.getCarbonTableIdentifier().getTableId();
-    if (launchJob) {
+    if (clearInAllWorkers) {
       // carbon table need to lookup only if launch job is set.
       CarbonTable carbonTable = getCarbonTable(identifier);
       if (null != carbonTable) {
@@ -791,4 +802,13 @@ public final class DataMapStoreManager {
     }
   }
 
+  private boolean hasCGDataMap(CarbonTable carbonTable) throws IOException {
+    for (TableDataMap tableDataMap : getAllVisibleDataMap(carbonTable)) {
+      if 
(tableDataMap.getDataMapFactory().getDataMapLevel().equals(DataMapLevel.CG)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
 }
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datamap/DistributableDataMapFormat.java
 
b/core/src/main/java/org/apache/carbondata/core/datamap/DistributableDataMapFormat.java
index 615d881..7eff356 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/datamap/DistributableDataMapFormat.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/datamap/DistributableDataMapFormat.java
@@ -196,6 +196,11 @@ public class DistributableDataMapFormat extends 
FileInputFormat<Void, ExtendedBl
 
       @Override
       public void close() {
+        // Clear the datamaps from executor
+        if (isFallbackJob) {
+          DataMapStoreManager.getInstance()
+              .clearDataMapCache(table.getAbsoluteTableIdentifier(), false);
+        }
       }
     };
   }
diff --git 
a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonRecordReader.java 
b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonRecordReader.java
index 6dd6f46..c8c1834 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonRecordReader.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonRecordReader.java
@@ -161,7 +161,7 @@ public class CarbonRecordReader<T> extends 
AbstractRecordReader<T> {
     logStatistics(rowCount, queryModel.getStatisticsRecorder());
     if (!skipClearDataMapAtClose) {
       // Clear the datamap cache
-      DataMapStoreManager.getInstance().clearDataMaps(
+      DataMapStoreManager.getInstance().clearDataMapCache(
           queryModel.getTable().getAbsoluteTableIdentifier(), false);
     }
     // close read support
diff --git 
a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableCleanTestCase.scala
 
b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableCleanTestCase.scala
index 706d332..b4fd4c5 100644
--- 
a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableCleanTestCase.scala
+++ 
b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableCleanTestCase.scala
@@ -16,7 +16,7 @@
  */
 package org.apache.carbondata.spark.testsuite.standardpartition
 
-import org.apache.spark.sql.Row
+import org.apache.spark.sql.{CarbonEnv, Row}
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.optimizer.CarbonFilters
 import org.apache.spark.sql.test.util.QueryTest
@@ -51,7 +51,9 @@ class StandardPartitionTableCleanTestCase extends QueryTest 
with BeforeAndAfterA
   }
 
   def validateDataFiles(tableUniqueName: String, segmentId: String, partition: 
Int, indexes: Int): Unit = {
-    val carbonTable = 
CarbonMetadata.getInstance().getCarbonTable(tableUniqueName)
+    val tableAndDbName = tableUniqueName.split("_")
+    val carbonTable = CarbonEnv.getCarbonTable(Some(tableAndDbName(0)), 
tableAndDbName(1))(
+      sqlContext.sparkSession)
     val partitions = CarbonFilters.getPartitions(
       Seq.empty,
       sqlContext.sparkSession,
diff --git 
a/integration/spark2/src/main/scala/org/apache/carbondata/indexserver/DataMapJobs.scala
 
b/integration/spark2/src/main/scala/org/apache/carbondata/indexserver/DataMapJobs.scala
index 4986cba..733d0fa 100644
--- 
a/integration/spark2/src/main/scala/org/apache/carbondata/indexserver/DataMapJobs.scala
+++ 
b/integration/spark2/src/main/scala/org/apache/carbondata/indexserver/DataMapJobs.scala
@@ -123,9 +123,6 @@ class EmbeddedDataMapJob extends AbstractDataMapJob {
     dataMapFormat.setFallbackJob()
     val splits = 
IndexServer.getSplits(dataMapFormat).getExtendedBlockets(dataMapFormat
       .getCarbonTable.getTablePath, dataMapFormat.getQueryId, 
dataMapFormat.isCountStarJob)
-    // Fire a job to clear the cache from executors as Embedded mode does not 
maintain the cache.
-    IndexServer.invalidateSegmentCache(dataMapFormat.getCarbonTable, 
dataMapFormat
-      .getValidSegmentIds.asScala.toArray, isFallBack = true)
     spark.sparkContext.setLocalProperty("spark.job.description", 
originalJobDesc)
     splits
   }
diff --git 
a/integration/spark2/src/main/scala/org/apache/carbondata/indexserver/DistributedPruneRDD.scala
 
b/integration/spark2/src/main/scala/org/apache/carbondata/indexserver/DistributedPruneRDD.scala
index 76d33b4..3284d46 100644
--- 
a/integration/spark2/src/main/scala/org/apache/carbondata/indexserver/DistributedPruneRDD.scala
+++ 
b/integration/spark2/src/main/scala/org/apache/carbondata/indexserver/DistributedPruneRDD.scala
@@ -142,7 +142,6 @@ private[indexserver] class DistributedPruneRDD(@transient 
private val ss: SparkS
         while (reader.nextKeyValue()) {
           blocklets.add(reader.getCurrentValue)
         }
-        reader.close()
         blocklets.asScala
       }
     }
diff --git 
a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
 
b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
index 7697719..482a5e1 100644
--- 
a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
+++ 
b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
@@ -102,23 +102,13 @@ case class CarbonLoadDataCommand(
   var finalPartition : Map[String, Option[String]] = Map.empty
 
   override def processMetadata(sparkSession: SparkSession): Seq[Row] = {
-    val LOGGER = 
LogServiceFactory.getLogService(this.getClass.getCanonicalName)
     val dbName = CarbonEnv.getDatabaseName(databaseNameOp)(sparkSession)
     setAuditTable(dbName, tableName)
     table = if (tableInfoOp.isDefined) {
-        CarbonTable.buildFromTableInfo(tableInfoOp.get)
-      } else {
-        val relation = CarbonEnv.getInstance(sparkSession).carbonMetaStore
-          .lookupRelation(Option(dbName), 
tableName)(sparkSession).asInstanceOf[CarbonRelation]
-        if (relation == null) {
-          throw new NoSuchTableException(dbName, tableName)
-        }
-        if (null == relation.carbonTable) {
-          LOGGER.error(s"Data loading failed. table not found: 
$dbName.$tableName")
-          throw new NoSuchTableException(dbName, tableName)
-        }
-        relation.carbonTable
-      }
+      CarbonTable.buildFromTableInfo(tableInfoOp.get)
+    } else {
+      CarbonEnv.getCarbonTable(Option(dbName), tableName)(sparkSession)
+    }
     if (table.isHivePartitionTable) {
       logicalPartitionRelation =
         new FindDataSourceTable(sparkSession).apply(
diff --git 
a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala
 
b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala
index 3f9df54..6622513 100644
--- 
a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala
+++ 
b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala
@@ -98,20 +98,7 @@ object CarbonSetCommand {
       }
     } else if (key.startsWith(CarbonCommonConstants.CARBON_DATAMAP_VISIBLE)) {
       if (key.split("\\.").length == 6) {
-        val keyArray = key.split("\\.")
-        val dbName = keyArray(keyArray.length - 3)
-        val tableName = keyArray(keyArray.length - 2)
-        val table = CarbonEnv.getCarbonTable(Some(dbName), 
tableName)(SparkSQLUtil.getSparkSession)
-        val isValid = DataMapStoreManager.getInstance
-          .isDataMapExist(table.getTableId, keyArray(keyArray.length - 1))
-        if (!isValid) throw new InvalidConfigurationException(String.format(
-          "Invalid configuration of %s, datamap does not exist",
-          key))
         sessionParams.addProperty(key.toLowerCase, value)
-      } else {
-        throw new MalformedCarbonCommandException("property should be in " +
-          "\" 
carbon.datamap.visible.<database_name>.<table_name>.<datamap_name>" +
-          " = <true/false> \" format")
       }
     } else if 
(key.startsWith(CarbonCommonConstants.CARBON_LOAD_DATAMAPS_PARALLEL)) {
       if (key.split("\\.").length == 6) {
diff --git 
a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReaderBuilder.java
 
b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReaderBuilder.java
index 536da6f..ddc4e26 100644
--- 
a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReaderBuilder.java
+++ 
b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReaderBuilder.java
@@ -374,7 +374,7 @@ public class CarbonReaderBuilder {
       }
     } catch (Exception ex) {
       // Clear the datamap cache as it can get added in getSplits() method
-      DataMapStoreManager.getInstance().clearDataMaps(
+      DataMapStoreManager.getInstance().clearDataMapCache(
           
format.getOrCreateCarbonTable((job.getConfiguration())).getAbsoluteTableIdentifier(),
           false);
       throw ex;
@@ -433,7 +433,7 @@ public class CarbonReaderBuilder {
     } finally {
       if (format != null) {
         // Clear the datamap cache as it is added in getSplits() method
-        DataMapStoreManager.getInstance().clearDataMaps(
+        DataMapStoreManager.getInstance().clearDataMapCache(
             
format.getOrCreateCarbonTable((job.getConfiguration())).getAbsoluteTableIdentifier(),
             false);
       }
diff --git 
a/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonReaderTest.java 
b/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonReaderTest.java
index 92f819b..bd735da 100644
--- 
a/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonReaderTest.java
+++ 
b/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonReaderTest.java
@@ -67,7 +67,7 @@ public class CarbonReaderTest extends TestCase {
     String path = "./testWriteFiles";
     FileUtils.deleteDirectory(new File(path));
     DataMapStoreManager.getInstance()
-        .clearDataMaps(AbsoluteTableIdentifier.from(path), false);
+        .clearDataMapCache(AbsoluteTableIdentifier.from(path), false);
     Field[] fields = new Field[2];
     fields[0] = new Field("name", DataTypes.STRING);
     fields[1] = new Field("age", DataTypes.INT);
@@ -163,7 +163,7 @@ public class CarbonReaderTest extends TestCase {
   @Test public void testReadWithZeroBatchSize() throws Exception {
     String path = "./testWriteFiles";
     FileUtils.deleteDirectory(new File(path));
-    
DataMapStoreManager.getInstance().clearDataMaps(AbsoluteTableIdentifier.from(path),
 false);
+    
DataMapStoreManager.getInstance().clearDataMapCache(AbsoluteTableIdentifier.from(path),
 false);
     Field[] fields = new Field[2];
     fields[0] = new Field("name", DataTypes.STRING);
     fields[1] = new Field("age", DataTypes.INT);
@@ -188,7 +188,7 @@ public class CarbonReaderTest extends TestCase {
   public void testReadBatchWithZeroBatchSize() throws Exception {
     String path = "./testWriteFiles";
     FileUtils.deleteDirectory(new File(path));
-    
DataMapStoreManager.getInstance().clearDataMaps(AbsoluteTableIdentifier.from(path),
 false);
+    
DataMapStoreManager.getInstance().clearDataMapCache(AbsoluteTableIdentifier.from(path),
 false);
     Field[] fields = new Field[2];
     fields[0] = new Field("name", DataTypes.STRING);
     fields[1] = new Field("age", DataTypes.INT);
@@ -212,7 +212,7 @@ public class CarbonReaderTest extends TestCase {
     String path = "./testWriteFiles";
     FileUtils.deleteDirectory(new File(path));
     DataMapStoreManager.getInstance()
-        .clearDataMaps(AbsoluteTableIdentifier.from(path), false);
+        .clearDataMapCache(AbsoluteTableIdentifier.from(path), false);
     String path1 = path + "/0testdir";
     String path2 = path + "/testdir";
 
@@ -259,7 +259,7 @@ public class CarbonReaderTest extends TestCase {
     String path = "./testWriteFiles";
     FileUtils.deleteDirectory(new File(path));
     DataMapStoreManager.getInstance()
-        .clearDataMaps(AbsoluteTableIdentifier.from(path), false);
+        .clearDataMapCache(AbsoluteTableIdentifier.from(path), false);
     Field[] fields = new Field[2];
     fields[0] = new Field("name", DataTypes.STRING);
     fields[1] = new Field("age", DataTypes.INT);
@@ -296,7 +296,7 @@ public class CarbonReaderTest extends TestCase {
     String path = "./testWriteFiles";
     FileUtils.deleteDirectory(new File(path));
     DataMapStoreManager.getInstance()
-        .clearDataMaps(AbsoluteTableIdentifier.from(path), false);
+        .clearDataMapCache(AbsoluteTableIdentifier.from(path), false);
     Field[] fields = new Field[3];
     fields[0] = new Field("name", DataTypes.STRING);
     fields[1] = new Field("age", DataTypes.INT);
@@ -339,7 +339,7 @@ public class CarbonReaderTest extends TestCase {
     String path = "./testWriteFiles";
     FileUtils.deleteDirectory(new File(path));
     DataMapStoreManager.getInstance()
-        .clearDataMaps(AbsoluteTableIdentifier.from(path), false);
+        .clearDataMapCache(AbsoluteTableIdentifier.from(path), false);
     Field[] fields = new Field[3];
     fields[0] = new Field("name", DataTypes.STRING);
     fields[1] = new Field("age", DataTypes.INT);
@@ -382,7 +382,7 @@ public class CarbonReaderTest extends TestCase {
     String path = "./testWriteFiles";
     FileUtils.deleteDirectory(new File(path));
     DataMapStoreManager.getInstance()
-        .clearDataMaps(AbsoluteTableIdentifier.from(path), false);
+        .clearDataMapCache(AbsoluteTableIdentifier.from(path), false);
     Field[] fields = new Field[3];
     fields[0] = new Field("name", DataTypes.STRING);
     fields[1] = new Field("age", DataTypes.INT);
@@ -425,7 +425,7 @@ public class CarbonReaderTest extends TestCase {
     String path = "./testWriteFiles";
     FileUtils.deleteDirectory(new File(path));
     DataMapStoreManager.getInstance()
-        .clearDataMaps(AbsoluteTableIdentifier.from(path), false);
+        .clearDataMapCache(AbsoluteTableIdentifier.from(path), false);
     Field[] fields = new Field[3];
     fields[0] = new Field("name", DataTypes.STRING);
     fields[1] = new Field("age", DataTypes.INT);
@@ -468,7 +468,7 @@ public class CarbonReaderTest extends TestCase {
     String path = "./testWriteFiles";
     FileUtils.deleteDirectory(new File(path));
     DataMapStoreManager.getInstance()
-        .clearDataMaps(AbsoluteTableIdentifier.from(path), false);
+        .clearDataMapCache(AbsoluteTableIdentifier.from(path), false);
     Field[] fields = new Field[3];
     fields[0] = new Field("name", DataTypes.STRING);
     fields[1] = new Field("age", DataTypes.INT);
@@ -518,7 +518,7 @@ public class CarbonReaderTest extends TestCase {
 
     TestUtil.writeFilesAndVerify(200, new Schema(fields), path);
     DataMapStoreManager.getInstance()
-        .clearDataMaps(AbsoluteTableIdentifier.from(path), false);
+        .clearDataMapCache(AbsoluteTableIdentifier.from(path), false);
     ColumnExpression columnExpression = new ColumnExpression("doubleField", 
DataTypes.DOUBLE);
     LessThanExpression lessThanExpression = new 
LessThanExpression(columnExpression,
         new LiteralExpression("13.5", DataTypes.DOUBLE));
@@ -554,7 +554,7 @@ public class CarbonReaderTest extends TestCase {
     String path = "./testWriteFiles";
     FileUtils.deleteDirectory(new File(path));
     DataMapStoreManager.getInstance()
-        .clearDataMaps(AbsoluteTableIdentifier.from(path), false);
+        .clearDataMapCache(AbsoluteTableIdentifier.from(path), false);
     Field[] fields = new Field[3];
     fields[0] = new Field("name", DataTypes.STRING);
     fields[1] = new Field("age", DataTypes.INT);
@@ -599,9 +599,9 @@ public class CarbonReaderTest extends TestCase {
     FileUtils.deleteDirectory(new File(path1));
     FileUtils.deleteDirectory(new File(path2));
     DataMapStoreManager.getInstance()
-        .clearDataMaps(AbsoluteTableIdentifier.from(path1), false);
+        .clearDataMapCache(AbsoluteTableIdentifier.from(path1), false);
     DataMapStoreManager.getInstance()
-        .clearDataMaps(AbsoluteTableIdentifier.from(path2), false);
+        .clearDataMapCache(AbsoluteTableIdentifier.from(path2), false);
     Field[] fields = new Field[] { new Field("c1", "string"),
          new Field("c2", "int") };
     Schema schema = new Schema(fields);
@@ -663,7 +663,7 @@ public class CarbonReaderTest extends TestCase {
     String path = "./testWriteFiles";
     FileUtils.deleteDirectory(new File(path));
     DataMapStoreManager.getInstance()
-        .clearDataMaps(AbsoluteTableIdentifier.from(path), false);
+        .clearDataMapCache(AbsoluteTableIdentifier.from(path), false);
     Field[] fields = new Field[2];
     fields[0] = new Field("name", DataTypes.STRING);
     fields[1] = new Field("age", DataTypes.INT);
@@ -700,7 +700,7 @@ public class CarbonReaderTest extends TestCase {
     String path = "./testWriteFiles";
     FileUtils.deleteDirectory(new File(path));
     DataMapStoreManager.getInstance()
-        .clearDataMaps(AbsoluteTableIdentifier.from(path), false);
+        .clearDataMapCache(AbsoluteTableIdentifier.from(path), false);
     Field[] fields = new Field[2];
     fields[0] = new Field("name", DataTypes.STRING);
     fields[1] = new Field("age", DataTypes.INT);
@@ -736,7 +736,7 @@ public class CarbonReaderTest extends TestCase {
     String path = "./testWriteFiles";
     FileUtils.deleteDirectory(new File(path));
     DataMapStoreManager.getInstance()
-        .clearDataMaps(AbsoluteTableIdentifier.from(path), false);
+        .clearDataMapCache(AbsoluteTableIdentifier.from(path), false);
     Field[] fields = new Field[2];
     fields[0] = new Field("name", DataTypes.STRING);
     fields[1] = new Field("age", DataTypes.INT);
@@ -777,7 +777,7 @@ public class CarbonReaderTest extends TestCase {
     String path = "./testWriteFiles";
     FileUtils.deleteDirectory(new File(path));
     DataMapStoreManager.getInstance()
-        .clearDataMaps(AbsoluteTableIdentifier.from(path), false);
+        .clearDataMapCache(AbsoluteTableIdentifier.from(path), false);
     Field[] fields = new Field[2];
     fields[0] = new Field("name", DataTypes.STRING);
     fields[1] = new Field("age", DataTypes.INT);
@@ -807,7 +807,7 @@ public class CarbonReaderTest extends TestCase {
     String path = "./testWriteFiles";
     FileUtils.deleteDirectory(new File(path));
     DataMapStoreManager.getInstance()
-        .clearDataMaps(AbsoluteTableIdentifier.from(path), false);
+        .clearDataMapCache(AbsoluteTableIdentifier.from(path), false);
     Field[] fields = new Field[2];
     fields[0] = new Field("name", DataTypes.STRING);
     fields[1] = new Field("age", DataTypes.INT);

Reply via email to