Repository: carbondata
Updated Branches:
  refs/heads/master afb802e60 -> fae457a35


[CARBONDATA-2356] Added UT Scenarios for LuceneDataMap

This closes #2180


Project: http://git-wip-us.apache.org/repos/asf/carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/carbondata/commit/fae457a3
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/fae457a3
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/fae457a3

Branch: refs/heads/master
Commit: fae457a35669546de2752aa03a23dd9c768d42c7
Parents: afb802e
Author: Indhumathi27 <[email protected]>
Authored: Thu Apr 19 14:58:33 2018 +0530
Committer: Jacky Li <[email protected]>
Committed: Fri Apr 27 18:13:25 2018 +0800

----------------------------------------------------------------------
 .../core/scan/expression/MatchExpression.java   |   8 +-
 .../datamap/lucene/LuceneFineGrainDataMap.java  |  17 +-
 integration/spark-common-cluster-test/pom.xml   |   6 +
 .../cluster/sdv/generated/LuceneTestCase.scala  | 224 ++++++++++++
 .../cluster/sdv/suite/SDVSuites.scala           |   1 +
 .../lucene/LuceneFineGrainDataMapSuite.scala    | 359 ++++++++++++++++++-
 .../carbondata/datamap/TextMatchUDF.scala       |  12 +
 .../scala/org/apache/spark/sql/CarbonEnv.scala  |   3 +-
 .../strategy/CarbonLateDecodeStrategy.scala     |   9 +-
 .../spark/sql/optimizer/CarbonFilters.scala     |   6 +-
 10 files changed, 636 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/fae457a3/core/src/main/java/org/apache/carbondata/core/scan/expression/MatchExpression.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/scan/expression/MatchExpression.java
 
b/core/src/main/java/org/apache/carbondata/core/scan/expression/MatchExpression.java
index 3ab2885..573100e 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/scan/expression/MatchExpression.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/scan/expression/MatchExpression.java
@@ -29,9 +29,15 @@ import org.apache.carbondata.core.scan.filter.intf.RowIntf;
 @InterfaceAudience.Internal
 public class MatchExpression extends Expression {
   private String queryString;
+  public static String maxDoc;
 
-  public MatchExpression(String queryString) {
+  public MatchExpression(String queryString, String maxDoc) {
     this.queryString = queryString;
+    setMaxDoc(maxDoc);
+  }
+
+  private void setMaxDoc(String maxDoc) {
+    MatchExpression.maxDoc = maxDoc;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fae457a3/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMap.java
----------------------------------------------------------------------
diff --git 
a/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMap.java
 
b/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMap.java
index cf5e2da..a39c7a2 100644
--- 
a/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMap.java
+++ 
b/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMap.java
@@ -35,6 +35,7 @@ import 
org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
 import org.apache.carbondata.core.scan.expression.Expression;
+import org.apache.carbondata.core.scan.expression.MatchExpression;
 import org.apache.carbondata.core.scan.filter.intf.ExpressionType;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
 
@@ -149,6 +150,14 @@ public class LuceneFineGrainDataMap extends 
FineGrainDataMap {
   }
 
   /**
+   * Return Maximum records
+   * @return
+   */
+  private int getMaxDoc() {
+    return Integer.parseInt(MatchExpression.maxDoc);
+  }
+
+  /**
    * Prune the datamap with filter expression. It returns the list of
    * blocklets where these filters can exist.
    */
@@ -161,6 +170,12 @@ public class LuceneFineGrainDataMap extends 
FineGrainDataMap {
 
     // only for test , query all data
     String strQuery = getQueryString(filterExp.getFilterExpression());
+    int maxDocs;
+    try {
+      maxDocs = getMaxDoc();
+    } catch (NumberFormatException e) {
+      maxDocs = Integer.MAX_VALUE;
+    }
 
     if (null == strQuery) {
       return null;
@@ -191,7 +206,7 @@ public class LuceneFineGrainDataMap extends 
FineGrainDataMap {
     // execute index search
     TopDocs result;
     try {
-      result = indexSearcher.search(query, indexReader.maxDoc());
+      result = indexSearcher.search(query, maxDocs);
     } catch (IOException e) {
       String errorMessage =
           String.format("failed to search lucene data, detail is %s", 
e.getMessage());

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fae457a3/integration/spark-common-cluster-test/pom.xml
----------------------------------------------------------------------
diff --git a/integration/spark-common-cluster-test/pom.xml 
b/integration/spark-common-cluster-test/pom.xml
index 028da11..44453b3 100644
--- a/integration/spark-common-cluster-test/pom.xml
+++ b/integration/spark-common-cluster-test/pom.xml
@@ -42,6 +42,12 @@
       <scope>test</scope>
     </dependency>
     <dependency>
+      <groupId>org.apache.carbondata</groupId>
+      <artifactId>carbondata-lucene</artifactId>
+      <version>${project.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>org.apache.spark</groupId>
       <artifactId>spark-hive-thriftserver_${scala.binary.version}</artifactId>
       <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fae457a3/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/LuceneTestCase.scala
----------------------------------------------------------------------
diff --git 
a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/LuceneTestCase.scala
 
b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/LuceneTestCase.scala
new file mode 100644
index 0000000..62835da
--- /dev/null
+++ 
b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/LuceneTestCase.scala
@@ -0,0 +1,224 @@
+
+/*
+ * 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.cluster.sdv.generated
+
+import org.apache.spark.sql.common.util.Include
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+/**
+ * Test Class for LuceneDataMap Testcase to verify all scenarios
+ */
+
+class LuceneTestcase extends QueryTest with BeforeAndAfterAll {
+
+  val csvPath = s"$resourcesPath/source.csv"
+
+  override protected def beforeAll(): Unit = {
+    sql("DROP TABLE IF EXISTS datamap_main")
+  }
+
+  //Create Lucene DataMap With DMProperties(String DataType) on MainTable
+  test("LuceneDataMap_TC001", Include) {
+    sql("DROP TABLE IF EXISTS datamap_main")
+    sql(
+      "CREATE TABLE datamap_main (id Int, date date, country string,name 
String, phonetype " +
+      "string, " +
+      "serialname String,salary int ) STORED BY 
'org.apache.carbondata.format'")
+    sql(
+      s"""
+         | CREATE DATAMAP lucene_datamap ON TABLE datamap_main
+         | USING 'lucene'
+         | DMProperties('TEXT_COLUMNS'='country')
+      """.stripMargin)
+    checkExistence(sql("show datamap on table datamap_main"), true, 
"lucene_datamap")
+    sql("drop datamap if exists lucene_datamap on table datamap_main")
+  }
+
+  //Create Lucene DataMap With DMProperties(Other DataTypes) on MainTable
+  test("LuceneDataMap_TC002", Include) {
+    sql("DROP TABLE IF EXISTS datamap_main")
+    sql(
+      "CREATE TABLE datamap_main (id Int, date date, country string,name 
String, phonetype " +
+      "string, " +
+      "serialname String,salary int ) STORED BY 'org.apache.carbondata.format' 
" +
+      "tblproperties('dictionary_include'='country')")
+    val exception_otherdataType: Exception = intercept[Exception] {
+      sql(
+        s"""
+           | CREATE DATAMAP lucene_datamap ON TABLE datamap_main
+           | USING 'lucene'
+           | DMProperties('TEXT_COLUMNS'='id')
+      """.stripMargin)
+    }
+    assert(exception_otherdataType.getMessage
+      .contains("TEXT_COLUMNS only supports String column. Unsupported column: 
id, " +
+                "DataType: INT"))
+  }
+
+  //Create Lucene DataMap With DMProperties on MainTable and Load Data and 
Query
+  test("LuceneDataMap_TC003", Include) {
+    sql("DROP TABLE IF EXISTS datamap_main")
+    sql(
+      "CREATE TABLE datamap_main (id Int, date string, country string,name 
String, phonetype " +
+      "string, " +
+      "serialname String,salary int ) STORED BY 'org.apache.carbondata.format' 
TBLPROPERTIES" +
+      "('SORT_COLUMNS'='country,name','SORT_SCOPE'='LOCAL_SORT')")
+    sql(
+      s"""
+         | CREATE DATAMAP lucene_datamap ON TABLE datamap_main
+         | USING 'lucene'
+         | DMProperties('TEXT_COLUMNS'='country')
+      """.stripMargin)
+    sql(s"LOAD DATA INPATH '$csvPath' INTO TABLE datamap_main")
+
+    checkAnswer(sql("SELECT * FROM datamap_main WHERE 
TEXT_MATCH('country:china')"),
+      sql("select * from datamap_main where country='china'"))
+    sql("drop datamap if exists lucene_datamap on table datamap_main")
+  }
+
+  //Create Different Lucene DataMap With DMProperties on MainTable and filter 
using 'like','AND'
+  // & 'OR'
+  test("LuceneDataMap_TC004", Include) {
+    sql("DROP TABLE IF EXISTS datamap_main")
+    sql(
+      "CREATE TABLE datamap_main (id Int, date string, country string,name 
String, phonetype " +
+      "string, " +
+      "serialname String,salary int ) STORED BY 'org.apache.carbondata.format' 
")
+    sql(
+      s"""
+         | CREATE DATAMAP lucene_datamap ON TABLE datamap_main
+         | USING 'lucene'
+         | DMProperties('TEXT_COLUMNS'='country,name,serialname')
+      """.stripMargin)
+    sql(s"LOAD DATA LOCAL INPATH '$csvPath' INTO TABLE datamap_main")
+    checkAnswer(sql("SELECT * FROM datamap_main WHERE 
TEXT_MATCH('country:ch*')"),
+      sql("select * from datamap_main where country like 'ch%'"))
+    checkAnswer(sql(
+      "SELECT * FROM datamap_main WHERE TEXT_MATCH('country:ch*') AND 
TEXT_MATCH('name:aa*')"),
+      sql("select * from datamap_main where country like 'ch%' and name like 
'aa%'"))
+    checkAnswer(sql(
+      "SELECT * FROM datamap_main WHERE TEXT_MATCH('country:u* or name:aa*')"),
+      sql("select * from datamap_main where country like 'u%'or name like 
'aa%'"))
+    checkAnswer(sql(
+      "SELECT * FROM datamap_main WHERE TEXT_MATCH('country:u*') OR 
TEXT_MATCH('name:aaa1*') AND " +
+      "TEXT_MATCH('name:aaa2*')"),
+      sql(
+        "select * from datamap_main where country like 'u%' OR name like 
'aaa1%' AND name like " +
+        "'aaa2%'"))
+    sql("drop datamap if exists lucene_datamap on table datamap_main")
+  }
+
+  //Create Different Lucene DataMap With DMProperties on MainTable and check 
Datamap after Delete
+  // Segment
+  test("LuceneDataMap_TC005", Include) {
+    sql("DROP TABLE IF EXISTS datamap_main")
+    sql(
+      "CREATE TABLE datamap_main (id Int, date date, country string,name 
String, phonetype " +
+      "string, " +
+      "serialname String,salary int ) STORED BY 
'org.apache.carbondata.format'")
+    sql(
+      s"""
+         | CREATE DATAMAP lucene_datamap ON TABLE datamap_main
+         | USING 'lucene'
+         | DMProperties('TEXT_COLUMNS'='country,name')
+      """.stripMargin)
+    sql(s"LOAD DATA LOCAL INPATH '$csvPath' INTO TABLE datamap_main")
+    sql(s"LOAD DATA LOCAL INPATH '$csvPath' INTO TABLE datamap_main")
+    checkAnswer(sql("SELECT COUNT(*) FROM datamap_main WHERE 
TEXT_MATCH('country:china')"),
+      sql("select COUNT(*) from datamap_main where country='china'"))
+    sql("delete from table datamap_main where SEGMENT.ID in (0) ")
+    sql("clean files for table datamap_main")
+    checkAnswer(sql("SELECT COUNT(*) FROM datamap_main WHERE 
TEXT_MATCH('country:china')"),
+      sql("select COUNT(*) from datamap_main where country='china'"))
+    sql("drop datamap if exists lucene_datamap on table datamap_main")
+  }
+
+  //Create Different Lucene DataMap With DMProperties on MainTable with 
different 'TBLProperties'
+  // and Load Data with Differnt OPTIONS & Verify
+  test("LuceneDataMap_TC006", Include) {
+    sql("DROP TABLE IF EXISTS datamap_main")
+    sql(
+      "CREATE TABLE datamap_main (id Int, date string, country string,name 
String, phonetype " +
+      "string, " +
+      "serialname String,salary int ) STORED BY 'org.apache.carbondata.format' 
" +
+      "TBLPROPERTIES('SORT_COLUMNS'='country,name','SORT_SCOPE'='LOCAL_SORT')")
+    sql(
+      s"""
+         | CREATE DATAMAP lucene_datamap ON TABLE datamap_main
+         | USING 'lucene'
+         | DMProperties('TEXT_COLUMNS'='country')
+      """.stripMargin)
+    sql(s"LOAD DATA LOCAL INPATH '$csvPath' INTO TABLE datamap_main 
OPTIONS('header'='false'," +
+        
s"'BAD_RECORDS_LOGGER_ENABLE'='FALSE','BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='TRUE')")
+    checkAnswer(sql("SELECT COUNT(*) FROM datamap_main WHERE 
TEXT_MATCH('country:china')"),
+      sql("select COUNT(*) from datamap_main where country='china'"))
+    sql("drop datamap if exists lucene_datamap on table datamap_main")
+  }
+
+  //Create LuceneDataMap With DMProperties on MainTable and Insert data and 
Update and
+  // Verify
+  test("LuceneDataMap_TC007", Include) {
+    sql("DROP TABLE IF EXISTS datamap_main")
+    sql(
+      "CREATE TABLE datamap_main (id Int,country string,name String) STORED BY 
'org.apache" +
+      ".carbondata.format'")
+    sql(
+      s"""
+         | CREATE DATAMAP lucene_datamap ON TABLE datamap_main
+         | USING 'lucene'
+         | DMProperties('TEXT_COLUMNS'='country,name')
+      """.stripMargin)
+    sql("insert into datamap_main select 1,'abc','aa'")
+    sql("insert into datamap_main select 2,'def','ab'")
+    sql("insert into datamap_main select 3,'ghi','ac'")
+    sql("insert into datamap_main select 4,'jkl','ad'")
+    checkAnswer(sql("SELECT * FROM datamap_main WHERE 
TEXT_MATCH('country:def')"),
+      sql("select * from datamap_main where country='def'"))
+    sql("update datamap_main set (country)=('fed') where id=2")
+    checkAnswer(sql("SELECT COUNT(*) FROM datamap_main WHERE 
TEXT_MATCH('country:fed')"),
+      sql("select COUNT(*) from datamap_main where country='fed'"))
+    sql("drop datamap if exists lucene_datamap on table datamap_main")
+  }
+
+  //Check Lucene DataMap when Dictionary_Include is provided for TEXT_COLUMN 
in Main Table
+  test("LuceneDataMap_TC008", Include) {
+    sql("DROP TABLE IF EXISTS datamap_main")
+    sql(
+      "CREATE TABLE datamap_main (id Int, date string, country string,name 
String, phonetype " +
+      "string, " +
+      "serialname String,salary int ) STORED BY 'org.apache.carbondata.format' 
TBLPROPERTIES" +
+      
"('SORT_COLUMNS'='country,name','SORT_SCOPE'='LOCAL_SORT','DICTIONARY_INCLUDE'='country')")
+    val exception_dicitionaryinclude: Exception = intercept[Exception] {
+      sql(
+        s"""
+           | CREATE DATAMAP lucene_datamap ON TABLE datamap_main
+           | USING 'lucene'
+           | DMProperties('TEXT_COLUMNS'='country')
+      """.stripMargin)
+    }
+    assert(exception_dicitionaryinclude.getMessage
+      .contains("TEXT_COLUMNS cannot contain dictionary column country"))
+    sql("drop datamap if exists lucene_datamap on table datamap_main")
+  }
+
+  override protected def afterAll(): Unit = {
+    sql("DROP TABLE IF EXISTS datamap_main")
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fae457a3/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/suite/SDVSuites.scala
----------------------------------------------------------------------
diff --git 
a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/suite/SDVSuites.scala
 
b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/suite/SDVSuites.scala
index 8cccc4c..04b711a 100644
--- 
a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/suite/SDVSuites.scala
+++ 
b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/suite/SDVSuites.scala
@@ -146,6 +146,7 @@ class SDVSuites3 extends Suites with BeforeAndAfterAll {
                     new TableCommentAlterTableTestCase ::
                     new StandardPartitionTestCase ::
                     new PreAggregateTestCase ::
+                    new LuceneTestcase ::
                     new TimeSeriesPreAggregateTestCase :: 
                     new TestPartitionWithGlobalSort ::
                     new SetParameterTestCase :: Nil

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fae457a3/integration/spark-common-test/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapSuite.scala
----------------------------------------------------------------------
diff --git 
a/integration/spark-common-test/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapSuite.scala
 
b/integration/spark-common-test/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapSuite.scala
index ca8ba91..3286f79 100644
--- 
a/integration/spark-common-test/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapSuite.scala
+++ 
b/integration/spark-common-test/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapSuite.scala
@@ -21,11 +21,11 @@ import java.io.{File, PrintWriter}
 
 import scala.util.Random
 
-import org.apache.spark.sql.CarbonEnv
+import org.apache.spark.sql.{CarbonEnv, Row}
 import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.BeforeAndAfterAll
 
-import 
org.apache.carbondata.common.exceptions.sql.MalformedDataMapCommandException
+import 
org.apache.carbondata.common.exceptions.sql.{MalformedCarbonCommandException, 
MalformedDataMapCommandException}
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.core.datamap.DataMapStoreManager
@@ -130,7 +130,6 @@ class LuceneFineGrainDataMapSuite extends QueryTest with 
BeforeAndAfterAll {
       """.stripMargin)
 
     sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE datamap_test 
OPTIONS('header'='false')")
-
     checkAnswer(sql("SELECT * FROM datamap_test WHERE 
TEXT_MATCH('name:n10')"), sql(s"select * from datamap_test where name='n10'"))
     checkAnswer(sql("SELECT * FROM datamap_test WHERE 
TEXT_MATCH('city:c020')"), sql(s"SELECT * FROM datamap_test WHERE city='c020'"))
 
@@ -231,14 +230,368 @@ class LuceneFineGrainDataMapSuite extends QueryTest with 
BeforeAndAfterAll {
     sql("DROP TABLE IF EXISTS datamap_test3")
   }
 
+  test("test lucene fine grain data map for create datamap with Duplicate 
Columns") {
+    sql("DROP TABLE IF EXISTS datamap_test_table")
+    sql(
+      """
+        | CREATE TABLE datamap_test_table(id INT, name STRING, city STRING, 
age INT)
+        | STORED BY 'carbondata'
+        | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='LOCAL_SORT')
+      """.stripMargin)
+    val exception_duplicate_column: Exception = 
intercept[MalformedDataMapCommandException] {
+      sql(
+        s"""
+           | CREATE DATAMAP dm ON TABLE datamap_test_table
+           | USING 'lucene'
+           | DMProperties('TEXT_COLUMNS'='name')
+      """.stripMargin)
+      sql(
+        s"""
+           | CREATE DATAMAP dm1 ON TABLE datamap_test_table
+           | USING 'lucene'
+           | DMProperties('TEXT_COLUMNS'='name')
+      """.stripMargin)
+    }
+    assert(exception_duplicate_column.getMessage
+      .contains("Create lucene datamap dm1 failed, datamap already exists on 
column(s) name"))
+    sql("drop datamap if exists dm on table datamap_test_table")
+  }
+
+  test("test lucene fine grain data map with wildcard matching ") {
+    sql("DROP TABLE IF EXISTS datamap_test_table")
+    sql(
+      """
+        | CREATE TABLE datamap_test_table(id INT, name STRING, city STRING, 
age INT)
+        | STORED BY 'carbondata'
+        | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='LOCAL_SORT')
+      """.stripMargin)
+    sql(
+      s"""
+         | CREATE DATAMAP dm ON TABLE datamap_test_table
+         | USING 'lucene'
+         | DMProperties('TEXT_COLUMNS'='name , city')
+      """.stripMargin)
+    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE datamap_test_table 
OPTIONS('header'='false')")
+    checkAnswer(sql("SELECT * FROM datamap_test_table WHERE 
TEXT_MATCH('name:n99*')"),
+      sql("select * from datamap_test_table where name like 'n99%'"))
+    checkAnswer(sql("SELECT * FROM datamap_test_table WHERE 
TEXT_MATCH('name:n*9')"),
+      sql(s"select * from datamap_test_table where name like 'n%9'"))
+    sql("drop datamap if exists dm on table datamap_test_table")
+  }
+
+  test("test lucene fine grain data map with TEXT_MATCH 'AND' Filter ") {
+    sql("DROP TABLE IF EXISTS datamap_test_table")
+    sql(
+      """
+        | CREATE TABLE datamap_test_table(id INT, name STRING, city STRING, 
age INT)
+        | STORED BY 'carbondata'
+        | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='LOCAL_SORT')
+      """.stripMargin)
+    sql(
+      s"""
+         | CREATE DATAMAP dm ON TABLE datamap_test_table
+         | USING 'lucene'
+         | DMProperties('TEXT_COLUMNS'='name , city')
+      """.stripMargin)
+    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE datamap_test_table 
OPTIONS('header'='false')")
+    checkAnswer(sql(
+      "SELECT * FROM datamap_test_table WHERE TEXT_MATCH('name:n0*') AND 
TEXT_MATCH(' city:c0*')"),
+      sql("select * from datamap_test_table where name like 'n0%' and city 
like 'c0%'"))
+    sql("drop datamap if exists dm on table datamap_test_table")
+  }
+
+  test("test lucene fine grain data map with TEXT_MATCH 'OR' Filter ") {
+    sql("DROP TABLE IF EXISTS datamap_test_table")
+    sql(
+      """
+        | CREATE TABLE datamap_test_table(id INT, name STRING, city STRING, 
age INT)
+        | STORED BY 'carbondata'
+        | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='LOCAL_SORT')
+      """.stripMargin)
+    sql(
+      s"""
+         | CREATE DATAMAP dm ON TABLE datamap_test_table
+         | USING 'lucene'
+         | DMProperties('TEXT_COLUMNS'='name , city')
+      """.stripMargin)
+    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE datamap_test_table 
OPTIONS('header'='false')")
+    checkAnswer(sql(
+      "SELECT * FROM datamap_test_table WHERE TEXT_MATCH('name:n1*') or 
TEXT_MATCH('city:c01*')"),
+      sql("select * from datamap_test_table where name like 'n1%' or city like 
'c01%'"))
+    sql("drop datamap if exists dm on table datamap_test_table")
+  }
+
+  test("test lucene fine grain data map with TEXT_MATCH 'AND' and 'OR' Filter 
") {
+    sql("DROP TABLE IF EXISTS datamap_test_table")
+    sql(
+      """
+        | CREATE TABLE datamap_test_table(id INT, name STRING, city STRING, 
age INT)
+        | STORED BY 'carbondata'
+        | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='LOCAL_SORT')
+      """.stripMargin)
+    sql(
+      s"""
+         | CREATE DATAMAP dm ON TABLE datamap_test_table
+         | USING 'lucene'
+         | DMProperties('TEXT_COLUMNS'='name , city')
+      """.stripMargin)
+    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE datamap_test_table 
OPTIONS('header'='false')")
+    checkAnswer(sql(
+      "SELECT * FROM datamap_test_table WHERE TEXT_MATCH('name:n1*') OR 
TEXT_MATCH ('city:c01*') " +
+      "AND TEXT_MATCH('city:C02*')"),
+      sql(
+        "select * from datamap_test_table where name like 'n1%' OR city like 
'c01%' and city like" +
+        " 'c02%'"))
+    sql("drop datamap if exists dm on table datamap_test_table")
+  }
+
+  test("test lucene fine grain data map with compaction-Major ") {
+    sql("DROP TABLE IF EXISTS datamap_test_table")
+    sql(
+      """
+        | CREATE TABLE datamap_test_table(id INT, name STRING, city STRING, 
age INT)
+        | STORED BY 'carbondata'
+        | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='LOCAL_SORT')
+      """.stripMargin)
+    sql(
+      s"""
+         | CREATE DATAMAP dm ON TABLE datamap_test_table
+         | USING 'lucene'
+         | DMProperties('TEXT_COLUMNS'='name , city')
+      """.stripMargin)
+    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE datamap_test_table 
OPTIONS('header'='false')")
+    checkAnswer(sql("SELECT * FROM datamap_test_table WHERE 
TEXT_MATCH('name:n10')"),
+      sql("select * from datamap_test_table where name='n10'"))
+    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE datamap_test_table 
OPTIONS('header'='false')")
+    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE datamap_test_table 
OPTIONS('header'='false')")
+    sql("alter table datamap_test_table compact 'major'")
+    checkAnswer(sql("SELECT COUNT(*) FROM datamap_test_table WHERE 
TEXT_MATCH('name:n10')"),
+      sql("select COUNT(*) from datamap_test_table where name='n10'"))
+    sql("drop datamap if exists dm on table datamap_test_table")
+  }
+
+  test("test lucene fine grain data map with compaction-Minor ") {
+    sql("DROP TABLE IF EXISTS datamap_test_table")
+    sql(
+      """
+        | CREATE TABLE datamap_test_table(id INT, name STRING, city STRING, 
age INT)
+        | STORED BY 'carbondata'
+        | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='LOCAL_SORT')
+      """.stripMargin)
+    sql(
+      s"""
+         | CREATE DATAMAP dm ON TABLE datamap_test_table
+         | USING 'lucene'
+         | DMProperties('TEXT_COLUMNS'='name , city')
+      """.stripMargin)
+    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE datamap_test_table 
OPTIONS('header'='false')")
+    checkAnswer(sql("SELECT * FROM datamap_test_table WHERE 
TEXT_MATCH('name:n10')"),
+      sql("select * from datamap_test_table where name='n10'"))
+    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE datamap_test_table 
OPTIONS('header'='false')")
+    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE datamap_test_table 
OPTIONS('header'='false')")
+    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE datamap_test_table 
OPTIONS('header'='false')")
+    sql("alter table datamap_test_table compact 'minor'")
+    checkAnswer(sql("SELECT COUNT(*) FROM datamap_test_table WHERE 
TEXT_MATCH('name:n10')"),
+      sql("select count(*) from datamap_test_table where name='n10'"))
+    sql("drop datamap if exists dm on table datamap_test_table")
+  }
+
+  test("test lucene fine grain data map with GLOBAL_SORT_SCOPE ") {
+    sql("DROP TABLE IF EXISTS datamap_test_table")
+    sql(
+      """
+        | CREATE TABLE datamap_test_table(id INT, name STRING, city STRING, 
age INT)
+        | STORED BY 'carbondata'
+        | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='GLOBAL_SORT')
+      """.stripMargin)
+    sql(
+      s"""
+         | CREATE DATAMAP dm ON TABLE datamap_test_table
+         | USING 'lucene'
+         | DMProperties('TEXT_COLUMNS'='name , city')
+      """.stripMargin)
+    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE datamap_test_table 
OPTIONS('header'='false','GLOBAL_SORT_PARTITIONS'='2')")
+    checkAnswer(sql("SELECT * FROM datamap_test_table WHERE 
TEXT_MATCH('name:n10')"),
+      sql("select * from datamap_test_table where name='n10'"))
+    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE datamap_test_table 
OPTIONS('header'='false','GLOBAL_SORT_PARTITIONS'='2')")
+    checkAnswer(sql("SELECT * FROM datamap_test_table WHERE 
TEXT_MATCH('name:n10')"),
+      sql("select * from datamap_test_table where name='n10'"))
+    sql("drop datamap if exists dm on table datamap_test_table")
+  }
+
+  test("test lucene fine grain data map with ALTER ADD and DROP Table COLUMN 
on Lucene DataMap") {
+    sql("DROP TABLE IF EXISTS datamap_test_table")
+    sql(
+      """
+        | CREATE TABLE datamap_test_table(id INT, name STRING, city STRING, 
age INT)
+        | STORED BY 'carbondata'
+        | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='GLOBAL_SORT')
+      """.stripMargin)
+    sql(
+      s"""
+         | CREATE DATAMAP dm2 ON TABLE datamap_test_table
+         | USING 'lucene'
+         | DMProperties('TEXT_COLUMNS'='name , city')
+      """.stripMargin)
+    val exception_add_column: Exception = 
intercept[MalformedCarbonCommandException] {
+      sql("alter table dm2 add columns(city1 string)")
+    }
+    assert(exception_add_column.getMessage
+      .contains("Unsupported alter operation on hive table"))
+    val exception_drop_column: Exception = 
intercept[MalformedCarbonCommandException] {
+      sql("alter table dm2 drop columns(name)")
+    }
+    assert(exception_drop_column.getMessage
+      .contains("Unsupported alter operation on hive table"))
+    sql("drop datamap if exists dm2 on table datamap_test_table")
+  }
+  test("test Clean Files and check Lucene DataMap") {
+    sql("DROP TABLE IF EXISTS datamap_test_table")
+    sql(
+      """
+        | CREATE TABLE datamap_test_table(id INT, name STRING, city STRING, 
age INT)
+        | STORED BY 'carbondata'
+        | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='GLOBAL_SORT')
+      """.stripMargin)
+    sql(
+      s"""
+         | CREATE DATAMAP dm2 ON TABLE datamap_test_table
+         | USING 'lucene'
+         | DMProperties('TEXT_COLUMNS'='name , city')
+      """.stripMargin)
+    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE datamap_test_table 
OPTIONS('header'='false')")
+    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE datamap_test_table 
OPTIONS('header'='false')")
+    checkAnswer(sql("SELECT count(*) FROM datamap_test_table WHERE 
TEXT_MATCH('name:n99*')"),
+      sql("select count(*) from datamap_test_table where name like 'n99%'"))
+    sql("delete from table datamap_test_table where SEGMENT.ID in (0) ")
+    checkAnswer(sql("SELECT count(*) FROM datamap_test_table WHERE 
TEXT_MATCH('name:n99*')"),
+      sql("select count(*) from datamap_test_table where name like 'n99%'"))
+    sql("clean files for table datamap_test_table")
+    sql("drop datamap if exists dm2 on table datamap_test_table")
+  }
+
+  test("test lucene fine grain data map with TEXT_MATCH 'NOT' Filter ") {
+    sql("DROP TABLE IF EXISTS datamap_test_table")
+    sql(
+      """
+        | CREATE TABLE datamap_test_table(id INT, name STRING, city STRING, 
age INT)
+        | STORED BY 'carbondata'
+        | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='LOCAL_SORT')
+      """.stripMargin)
+    sql(
+      s"""
+         | CREATE DATAMAP dm ON TABLE datamap_test_table
+         | USING 'lucene'
+         | DMProperties('TEXT_COLUMNS'='name , city')
+          """.stripMargin)
+    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE datamap_test_table 
OPTIONS('header'='false')")
+    //check NOT filter with TEXTMATCH term-search
+    checkAnswer(sql("SELECT * FROM datamap_test_table WHERE 
TEXT_MATCH('name:n0 NOT n1')"),
+      sql("select *from datamap_test_table where name='n0' AND not name='n1'"))
+    //check NOT filter with TEXTMATCH wildcard-search
+    checkAnswer(sql("SELECT * FROM datamap_test_table WHERE 
TEXT_MATCH('name:n1* NOT n2*')"),
+      sql("select *from datamap_test_table where name like'n1%' AND not name 
like 'n2%'"))
+    //check NOT filter with TEXTMATCH wildcard-search using AND on different 
columns
+    checkAnswer(sql(
+      "select *from datamap_test_table where TEXT_MATCH('name:n1*')AND 
TEXT_MATCH('city:c01* NOT " +
+      "c02*')"),
+      sql("select *from datamap_test_table where name like'n1%' AND not 
city='c02%'"))
+    sql("drop datamap if exists dm on table datamap_test_table")
+  }
+
+  test("test lucene fine grain data map for show datamaps with Preaggregate 
and Lucene") {
+    sql("DROP TABLE IF EXISTS datamap_main")
+    sql("create table datamap_main (a string, b string, c string) stored by 
'carbondata'")
+    sql(
+      s"""
+         | CREATE DATAMAP dm_lucene ON TABLE datamap_main
+         | USING 'lucene'
+         | DMProperties('TEXT_COLUMNS'='c')
+      """.stripMargin)
+    sql(
+      "create datamap dm_pre on table datamap_main USING 'preaggregate' as 
select a,sum(b) " +
+      "from datamap_main group by a")
+    checkExistence(sql("show datamap on table datamap_main"), true, "dm_pre")
+    checkExistence(sql("show datamap on table datamap_main"), true, 
"dm_lucene")
+    sql("drop datamap if exists dm_pre on table datamap_main")
+    sql("drop datamap if exists dm_lucene on table datamap_main")
+  }
+
+  test("test lucene fine grain data map with CTAS") {
+    sql("DROP TABLE IF EXISTS source_table")
+    sql("DROP TABLE IF EXISTS target_table")
+    sql(
+      """
+        | CREATE TABLE source_table(id INT, name STRING, city STRING, age INT)
+        | STORED BY 'carbondata'
+        | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='LOCAL_SORT')
+      """.stripMargin)
+    sql(
+      s"""
+         | CREATE DATAMAP dm ON TABLE source_table
+         | USING 'lucene'
+         | DMProperties('TEXT_COLUMNS'='name,city')
+          """.stripMargin)
+    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE source_table 
OPTIONS('header'='false')")
+    sql(
+      """
+        | CREATE TABLE target_table
+        | STORED BY 'carbondata'
+        | AS
+        | Select * from source_table where TEXT_MATCH('name:n1*')
+      """.stripMargin)
+    checkAnswer(sql("SELECT count(*) FROM target_table"),
+      sql("select count(*) from source_table where name like 'n1%'"))
+    sql("DROP TABLE IF EXISTS source_table")
+    sql("DROP TABLE IF EXISTS target_table")
+  }
+
+  test("test lucene fine grain data map with text-match limit") {
+    sql(
+      s"""
+         | CREATE DATAMAP dm ON TABLE datamap_test
+         | USING 'lucene'
+         | DMProperties('TEXT_COLUMNS'='name , city')
+      """.stripMargin)
+
+    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE datamap_test 
OPTIONS('header'='false')")
+    checkAnswer(sql("select count(*) from datamap_test where 
TEXT_MATCH_WITH_LIMIT('name:n10*',10)"),Seq(Row(10)))
+    checkAnswer(sql("select count(*) from datamap_test where 
TEXT_MATCH_WITH_LIMIT('name:n10*',50)"),Seq(Row(50)))
+    sql("drop datamap dm on table datamap_test")
+  }
+
+  test("test lucene fine grain data map with InsertOverwrite") {
+    sql(
+      s"""
+         | CREATE DATAMAP dm ON TABLE datamap_test
+         | USING 'lucene'
+         | DMProperties('TEXT_COLUMNS'='name , city')
+      """.stripMargin)
+
+    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE datamap_test 
OPTIONS('header'='false')")
+    sql(
+      """
+        | CREATE TABLE table1(id INT, name STRING, city STRING, age INT)
+        | STORED BY 'carbondata'
+        | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='LOCAL_SORT')
+      """.stripMargin)
+    sql("INSERT OVERWRITE TABLE table1 select *from datamap_test where 
TEXT_MATCH('name:n*')")
+    checkAnswer(sql("select count(*) from table1"),Seq(Row(10000)))
+    sql("drop datamap dm on table datamap_test")
+  }
+
   override protected def afterAll(): Unit = {
     LuceneFineGrainDataMapSuite.deleteFile(file2)
     sql("DROP TABLE IF EXISTS normal_test")
     sql("DROP TABLE IF EXISTS datamap_test")
+    sql("DROP TABLE IF EXISTS source_table")
+    sql("DROP TABLE IF EXISTS target_table")
+    sql("DROP TABLE IF EXISTS datamap_test_table")
     sql("DROP TABLE IF EXISTS datamap_test1")
     sql("DROP TABLE IF EXISTS datamap_test2")
     sql("DROP TABLE IF EXISTS datamap_test3")
     sql("DROP TABLE IF EXISTS datamap_test4")
+    sql("DROP TABLE IF EXISTS datamap_main")
     sql("use default")
     sql("drop database if exists lucene cascade")
     CarbonProperties.getInstance()

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fae457a3/integration/spark2/src/main/scala/org/apache/carbondata/datamap/TextMatchUDF.scala
----------------------------------------------------------------------
diff --git 
a/integration/spark2/src/main/scala/org/apache/carbondata/datamap/TextMatchUDF.scala
 
b/integration/spark2/src/main/scala/org/apache/carbondata/datamap/TextMatchUDF.scala
index 093e479..703f27a 100644
--- 
a/integration/spark2/src/main/scala/org/apache/carbondata/datamap/TextMatchUDF.scala
+++ 
b/integration/spark2/src/main/scala/org/apache/carbondata/datamap/TextMatchUDF.scala
@@ -29,6 +29,18 @@ class TextMatchUDF extends ((String) => Boolean) with 
Serializable {
 }
 
 @InterfaceAudience.Internal
+class TextMatchMaxDocUDF extends ((String, Int) => Boolean) with Serializable {
+  override def apply(v1: String, v2: Int): Boolean = {
+    v1.length > 0
+  }
+}
+
[email protected]
 case class TextMatch(queryString: String) extends Filter {
   override def references: Array[String] = null
 }
+
[email protected]
+case class TextMatchLimit(queryString: String, maxDoc: String) extends Filter {
+  override def references: Array[String] = null
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fae457a3/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
----------------------------------------------------------------------
diff --git 
a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala 
b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
index ecf2088..977322b 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
@@ -36,7 +36,7 @@ import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.util._
-import org.apache.carbondata.datamap.TextMatchUDF
+import org.apache.carbondata.datamap.{TextMatchMaxDocUDF, TextMatchUDF}
 import org.apache.carbondata.events._
 import 
org.apache.carbondata.processing.loading.events.LoadEvents.{LoadMetadataEvent, 
LoadTablePostStatusUpdateEvent, LoadTablePreExecutionEvent, 
LoadTablePreStatusUpdateEvent}
 import org.apache.carbondata.spark.rdd.SparkReadSupport
@@ -81,6 +81,7 @@ class CarbonEnv {
     // register for lucene datamap
     // TODO: move it to proper place, it should be registered by datamap 
implementation
     sparkSession.udf.register("text_match", new TextMatchUDF)
+    sparkSession.udf.register("text_match_with_limit", new TextMatchMaxDocUDF)
 
     // added for handling timeseries function like hour, minute, day , month , 
year
     sparkSession.udf.register("timeseries", new TimeSeriesFunction)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fae457a3/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
----------------------------------------------------------------------
diff --git 
a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
 
b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
index 59dd2e9..bca0946 100644
--- 
a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
+++ 
b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
@@ -42,7 +42,7 @@ import 
org.apache.carbondata.core.metadata.schema.BucketingInfo
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.statusmanager.SegmentUpdateStatusManager
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.datamap.{TextMatch, TextMatchUDF}
+import org.apache.carbondata.datamap.{TextMatch, TextMatchLimit, 
TextMatchMaxDocUDF, TextMatchUDF}
 import org.apache.carbondata.spark.CarbonAliasDecoderRelation
 import org.apache.carbondata.spark.rdd.CarbonScanRDD
 import org.apache.carbondata.spark.util.CarbonScalaUtil
@@ -536,6 +536,13 @@ private[sql] class CarbonLateDecodeStrategy extends 
SparkStrategy {
         }
         Some(TextMatch(u.children.head.toString()))
 
+      case u: ScalaUDF if u.function.isInstanceOf[TextMatchMaxDocUDF] =>
+        if (u.children.size > 2) {
+          throw new MalformedCarbonCommandException(
+            "TEXT_MATCH UDF syntax: TEXT_MATCH_LIMIT('luceneQuerySyntax')")
+        }
+        Some(TextMatchLimit(u.children.head.toString(), 
u.children.last.toString()))
+
       case or@Or(left, right) =>
 
         val leftFilter = translateFilter(left, true)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fae457a3/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
----------------------------------------------------------------------
diff --git 
a/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
 
b/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
index b267ca2..1e46998 100644
--- 
a/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
+++ 
b/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
@@ -44,7 +44,7 @@ import 
org.apache.carbondata.core.scan.expression.logical.{AndExpression, FalseE
 import org.apache.carbondata.core.scan.filter.intf.ExpressionType
 import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.core.util.ThreadLocalSessionInfo
-import org.apache.carbondata.datamap.TextMatch
+import org.apache.carbondata.datamap.{TextMatch, TextMatchLimit}
 import org.apache.carbondata.spark.CarbonAliasDecoderRelation
 import org.apache.carbondata.spark.util.CarbonScalaUtil
 
@@ -141,7 +141,9 @@ object CarbonFilters {
         case FalseExpr() =>
           Some(new FalseExpression(null))
         case TextMatch(queryString) =>
-          Some(new MatchExpression(queryString))
+          Some(new MatchExpression(queryString, null))
+        case TextMatchLimit(queryString, maxDoc) =>
+          Some(new MatchExpression(queryString, maxDoc))
         case _ => None
       }
     }

Reply via email to