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 b992571  [CARBONDATA-3649] Hive expression is pushed down to carbon
b992571 is described below

commit b9925710438bd837bdf3142e26213342044ecc5b
Author: xiaohui0318 <245300...@qq.com>
AuthorDate: Thu Jan 2 23:23:20 2020 +0800

    [CARBONDATA-3649] Hive expression is pushed down to carbon
    
    Why is this PR needed?
    
    With more and more scenarios requiring hive to read the carbon format, data 
filtering improvements are needed
    
    What changes were proposed in this PR?
    
    When set hive.optimize.index.filter = true, hive expression can be pushed 
down to carbon to filter the data
    
    Does this PR introduce any user interface change?
    
    No
    
    Is any new testcase added?
    
    Yes
    
    This closes #3557
---
 .../carbondata/hive/Hive2CarbonExpression.java     | 159 +++++++++
 .../carbondata/hive/MapredCarbonInputFormat.java   |  39 +++
 .../carbondata/hive/Hive2CarbonExpressionTest.java | 362 +++++++++++++++++++++
 store/sdk/pom.xml                                  |   6 +
 4 files changed, 566 insertions(+)

diff --git 
a/integration/hive/src/main/java/org/apache/carbondata/hive/Hive2CarbonExpression.java
 
b/integration/hive/src/main/java/org/apache/carbondata/hive/Hive2CarbonExpression.java
new file mode 100644
index 0000000..b545629
--- /dev/null
+++ 
b/integration/hive/src/main/java/org/apache/carbondata/hive/Hive2CarbonExpression.java
@@ -0,0 +1,159 @@
+/*
+ * 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.hive;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.scan.expression.ColumnExpression;
+import org.apache.carbondata.core.scan.expression.Expression;
+import org.apache.carbondata.core.scan.expression.LiteralExpression;
+import 
org.apache.carbondata.core.scan.expression.conditional.EqualToExpression;
+import 
org.apache.carbondata.core.scan.expression.conditional.GreaterThanEqualToExpression;
+import 
org.apache.carbondata.core.scan.expression.conditional.GreaterThanExpression;
+import org.apache.carbondata.core.scan.expression.conditional.InExpression;
+import 
org.apache.carbondata.core.scan.expression.conditional.LessThanEqualToExpression;
+import 
org.apache.carbondata.core.scan.expression.conditional.LessThanExpression;
+import org.apache.carbondata.core.scan.expression.conditional.ListExpression;
+import 
org.apache.carbondata.core.scan.expression.conditional.NotEqualsExpression;
+import org.apache.carbondata.core.scan.expression.logical.AndExpression;
+import org.apache.carbondata.core.scan.expression.logical.OrExpression;
+import org.apache.carbondata.core.util.DataTypeUtil;
+import org.apache.carbondata.hadoop.api.CarbonInputFormat;
+
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeFieldDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFIn;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrGreaterThan;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrLessThan;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPGreaterThan;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPLessThan;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNotEqual;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNotNull;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNull;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr;
+import org.apache.log4j.Logger;
+
+/**
+ * @description: hive expression to carbon expression
+ */
+public class Hive2CarbonExpression {
+  public static final int left = 0;
+  public static final int right = 1;
+  private static final Logger LOG =
+      LogServiceFactory.getLogService(CarbonInputFormat.class.getName());
+
+  public static Expression convertExprHive2Carbon(ExprNodeDesc exprNodeDesc) {
+    if (exprNodeDesc instanceof ExprNodeGenericFuncDesc) {
+      ExprNodeGenericFuncDesc exprNodeGenericFuncDesc = 
(ExprNodeGenericFuncDesc) exprNodeDesc;
+      GenericUDF udf = exprNodeGenericFuncDesc.getGenericUDF();
+      List<ExprNodeDesc> l1 = exprNodeGenericFuncDesc.getChildren();
+      if (udf instanceof GenericUDFIn) {
+        ColumnExpression columnExpression = new 
ColumnExpression(l1.get(left).getCols().get(left),
+            getDateType(l1.get(left).getTypeString()));
+        List<Expression> listExpr = new ArrayList<>();
+        for (int i = right; i < l1.size(); i++) {
+          LiteralExpression literalExpression = new 
LiteralExpression(l1.get(i).getExprString(),
+              getDateType(l1.get(left).getTypeString()));
+          listExpr.add(literalExpression);
+        }
+        ListExpression listExpression = new ListExpression(listExpr);
+        return new InExpression(columnExpression, listExpression);
+      } else if (udf instanceof GenericUDFOPOr) {
+        Expression leftExpression =
+            
convertExprHive2Carbon(exprNodeGenericFuncDesc.getChildren().get(left));
+        Expression rightExpression =
+            
convertExprHive2Carbon(exprNodeGenericFuncDesc.getChildren().get(right));
+        return new OrExpression(leftExpression, rightExpression);
+      } else if (udf instanceof GenericUDFOPAnd) {
+        Expression leftExpression =
+            
convertExprHive2Carbon(exprNodeGenericFuncDesc.getChildren().get(left));
+        Expression rightExpression =
+            
convertExprHive2Carbon(exprNodeGenericFuncDesc.getChildren().get(right));
+        return new AndExpression(leftExpression, rightExpression);
+
+      } else if (udf instanceof GenericUDFOPEqual) {
+        ColumnExpression columnExpression = null;
+        if (l1.get(left) instanceof ExprNodeFieldDesc) {
+          throw new UnsupportedOperationException("Complex types are not 
supported for PPD");
+        } else {
+          columnExpression = new 
ColumnExpression(l1.get(left).getCols().get(left),
+              getDateType(l1.get(left).getTypeString()));
+        }
+        LiteralExpression literalExpression =
+            new LiteralExpression(l1.get(right).getExprString().replace("'", 
""),
+                getDateType(l1.get(right).getTypeString()));
+        return new EqualToExpression(columnExpression, literalExpression);
+      } else if (udf instanceof GenericUDFOPEqualOrGreaterThan) {
+        ColumnExpression columnExpression = new 
ColumnExpression(l1.get(left).getCols().get(left),
+            getDateType(l1.get(left).getTypeString()));
+        LiteralExpression literalExpression = new 
LiteralExpression(l1.get(right).getExprString(),
+            getDateType(l1.get(left).getTypeString()));
+        return new GreaterThanEqualToExpression(columnExpression, 
literalExpression);
+      } else if (udf instanceof GenericUDFOPGreaterThan) {
+        ColumnExpression columnExpression = new 
ColumnExpression(l1.get(left).getCols().get(left),
+            getDateType(l1.get(left).getTypeString()));
+        LiteralExpression literalExpression = new 
LiteralExpression(l1.get(right).getExprString(),
+            getDateType(l1.get(left).getTypeString()));
+        return new GreaterThanExpression(columnExpression, literalExpression);
+      } else if (udf instanceof GenericUDFOPNotEqual) {
+        ColumnExpression columnExpression = new 
ColumnExpression(l1.get(left).getCols().get(left),
+            getDateType(l1.get(left).getTypeString()));
+        LiteralExpression literalExpression = new 
LiteralExpression(l1.get(right).getExprString(),
+            getDateType(l1.get(left).getTypeString()));
+        return new NotEqualsExpression(columnExpression, literalExpression);
+      } else if (udf instanceof GenericUDFOPEqualOrLessThan) {
+        ColumnExpression columnExpression = new 
ColumnExpression(l1.get(left).getCols().get(left),
+            getDateType(l1.get(left).getTypeString()));
+        LiteralExpression literalExpression = new 
LiteralExpression(l1.get(right).getExprString(),
+            getDateType(l1.get(left).getTypeString()));
+        return new LessThanEqualToExpression(columnExpression, 
literalExpression);
+      } else if (udf instanceof GenericUDFOPLessThan) {
+        ColumnExpression columnExpression = new 
ColumnExpression(l1.get(left).getCols().get(left),
+            getDateType(l1.get(left).getTypeString()));
+        LiteralExpression literalExpression = new 
LiteralExpression(l1.get(right).getExprString(),
+            getDateType(l1.get(left).getTypeString()));
+        return new LessThanExpression(columnExpression, literalExpression);
+      } else if (udf instanceof GenericUDFOPNull) {
+        ColumnExpression columnExpression = new 
ColumnExpression(l1.get(left).getCols().get(left),
+            getDateType(l1.get(left).getTypeString()));
+        LiteralExpression literalExpression = new LiteralExpression(null, 
null);
+        return new EqualToExpression(columnExpression, literalExpression, 
true);
+      } else if (udf instanceof GenericUDFOPNotNull) {
+        ColumnExpression columnExpression = new 
ColumnExpression(l1.get(left).getCols().get(left),
+            getDateType(l1.get(left).getTypeString()));
+        LiteralExpression literalExpression = new LiteralExpression(null, 
null);
+        return new NotEqualsExpression(columnExpression, literalExpression, 
true);
+      } else {
+        LOG.error("error:not find type" + udf.toString());
+      }
+    }
+    return null;
+  }
+
+  public static DataType getDateType(String type) {
+    return DataTypeUtil.valueOf(type);
+  }
+}
+
diff --git 
a/integration/hive/src/main/java/org/apache/carbondata/hive/MapredCarbonInputFormat.java
 
b/integration/hive/src/main/java/org/apache/carbondata/hive/MapredCarbonInputFormat.java
index 5faab9c..d5872b8 100644
--- 
a/integration/hive/src/main/java/org/apache/carbondata/hive/MapredCarbonInputFormat.java
+++ 
b/integration/hive/src/main/java/org/apache/carbondata/hive/MapredCarbonInputFormat.java
@@ -23,24 +23,30 @@ import java.util.List;
 import java.util.UUID;
 
 import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.datamap.DataMapFilter;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.exception.InvalidConfigurationException;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.schema.SchemaReader;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
+import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.model.QueryModel;
 import org.apache.carbondata.core.scan.model.QueryModelBuilder;
 import org.apache.carbondata.core.util.DataTypeConverterImpl;
 import org.apache.carbondata.core.util.ObjectSerializationUtil;
 import org.apache.carbondata.hadoop.CarbonInputSplit;
+import org.apache.carbondata.hadoop.api.CarbonInputFormat;
 import org.apache.carbondata.hadoop.api.CarbonTableInputFormat;
 import org.apache.carbondata.hadoop.readsupport.CarbonReadSupport;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.InvalidPathException;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
+import org.apache.hadoop.hive.ql.plan.TableScanDesc;
 import org.apache.hadoop.io.ArrayWritable;
 import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.mapred.InputSplit;
@@ -108,6 +114,11 @@ public class MapredCarbonInputFormat extends 
CarbonTableInputFormat<ArrayWritabl
     jobConf.set(DATABASE_NAME, "_dummyDb_" + UUID.randomUUID().toString());
     jobConf.set(TABLE_NAME, "_dummyTable_" + UUID.randomUUID().toString());
     org.apache.hadoop.mapreduce.JobContext jobContext = 
Job.getInstance(jobConf);
+    try {
+      setFilterPredicates(jobContext.getConfiguration());
+    } catch (Exception e) {
+      e.printStackTrace();
+    }
     CarbonTableInputFormat carbonTableInputFormat = new 
CarbonTableInputFormat();
     List<org.apache.hadoop.mapreduce.InputSplit> splitList =
         carbonTableInputFormat.getSplits(jobContext);
@@ -124,6 +135,29 @@ public class MapredCarbonInputFormat extends 
CarbonTableInputFormat<ArrayWritabl
     return splits;
   }
 
+  protected void setFilterPredicates(Configuration configuration) {
+    try {
+      String expr = configuration.get(TableScanDesc.FILTER_EXPR_CONF_STR);
+      if (expr == null) {
+        return;
+      }
+      ExprNodeGenericFuncDesc exprNodeGenericFuncDesc =
+          Utilities.deserializeObject(expr, ExprNodeGenericFuncDesc.class);
+      LOGGER.debug("hive expression:" + 
exprNodeGenericFuncDesc.getGenericUDF());
+      LOGGER.debug("hive expression string:" + 
exprNodeGenericFuncDesc.getExprString());
+      Expression expression = 
Hive2CarbonExpression.convertExprHive2Carbon(exprNodeGenericFuncDesc);
+      if (expression == null) {
+        return;
+      }
+      LOGGER.debug("carbon expression:" + expression.getString());
+      CarbonTable carbonTable = getOrCreateCarbonTable(configuration);
+      DataMapFilter filter = new DataMapFilter(carbonTable, expression);
+      CarbonInputFormat.setFilterPredicates(configuration, filter);
+    } catch (IOException e) {
+      throw new RuntimeException("Error while reading filter expression", e);
+    }
+  }
+
   @Override
   public RecordReader<Void, ArrayWritable> getRecordReader(InputSplit 
inputSplit, JobConf jobConf,
       Reporter reporter) throws IOException {
@@ -131,6 +165,11 @@ public class MapredCarbonInputFormat extends 
CarbonTableInputFormat<ArrayWritabl
     if (inputSplit instanceof CarbonHiveInputSplit) {
       path = ((CarbonHiveInputSplit) inputSplit).getPath().toString();
     }
+    try {
+      setFilterPredicates(jobConf);
+    } catch (Exception e) {
+      e.printStackTrace();
+    }
     QueryModel queryModel = null;
     try {
       jobConf.set(DATABASE_NAME, "_dummyDb_" + UUID.randomUUID().toString());
diff --git 
a/integration/hive/src/test/java/org/apache/carbondata/hive/Hive2CarbonExpressionTest.java
 
b/integration/hive/src/test/java/org/apache/carbondata/hive/Hive2CarbonExpressionTest.java
new file mode 100644
index 0000000..ffefa86
--- /dev/null
+++ 
b/integration/hive/src/test/java/org/apache/carbondata/hive/Hive2CarbonExpressionTest.java
@@ -0,0 +1,362 @@
+/*
+ * 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.hive;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datamap.DataMapFilter;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.hadoop.api.CarbonFileInputFormat;
+import org.apache.carbondata.hadoop.api.CarbonInputFormat;
+import org.apache.carbondata.hadoop.api.CarbonTableInputFormat;
+import org.apache.carbondata.hadoop.testutil.StoreCreator;
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
+
+import com.google.common.collect.Lists;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
+import org.apache.hadoop.hive.ql.plan.TableScanDesc;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFIn;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrGreaterThan;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrLessThan;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPGreaterThan;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPLessThan;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNotEqual;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNotNull;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNull;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static parquet.hadoop.ParquetInputFormat.FILTER_PREDICATE;
+
+/**
+ * @program carbondata
+ * @description: test hive expression to carbondata expression filter
+ * @author: xiaohui
+ * @create: 2020/01/01 15:27
+ */
+
+public class Hive2CarbonExpressionTest {
+  private static StoreCreator creator;
+  private static CarbonLoadModel loadModel;
+  private static CarbonTable table;
+  static {
+    CarbonProperties.getInstance().
+        addProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC, 
"/tmp/carbon/badrecords");
+    CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.CARBON_SYSTEM_FOLDER_LOCATION, 
"/tmp/carbon/");
+    CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.CARBON_WRITTEN_BY_APPNAME, 
"Hive2CarbonExpressionTest");
+    try {
+      creator = new StoreCreator(new File("target/store").getAbsolutePath(),
+          new 
File("../../hadoop/src/test/resources/data.csv").getCanonicalPath());
+      loadModel = creator.createCarbonStore();
+      table=loadModel.getCarbonDataLoadSchema().getCarbonTable();
+      table.setTransactionalTable(false);
+    } catch (Exception e) {
+      Assert.fail("create table failed: " + e.getMessage());
+    }
+  }
+  @Test
+  public void testEqualHiveFilter() throws IOException {
+    ExprNodeDesc column = new ExprNodeColumnDesc(TypeInfoFactory.intTypeInfo, 
"id", null, false);
+    ExprNodeDesc constant = new 
ExprNodeConstantDesc(TypeInfoFactory.intTypeInfo, "1001");
+    List<ExprNodeDesc> children = Lists.newArrayList();
+    children.add(column);
+    children.add(constant);
+    ExprNodeGenericFuncDesc node = new 
ExprNodeGenericFuncDesc(TypeInfoFactory.intTypeInfo,
+        new GenericUDFOPEqual(), children);
+    Configuration configuration=new Configuration();
+    configuration.set("mapreduce.input.carboninputformat.filter.predicate", 
Utilities.serializeExpression(node));
+    CarbonInputFormat.setFilterPredicates(configuration,new 
DataMapFilter(table, Hive2CarbonExpression.convertExprHive2Carbon(node)));
+
+    final Job job = new Job(new JobConf(configuration));
+    final CarbonTableInputFormat format = new CarbonTableInputFormat();
+    format.setTableInfo(job.getConfiguration(), table.getTableInfo());
+    format.setTablePath(job.getConfiguration(), table.getTablePath());
+    format.setTableName(job.getConfiguration(), table.getTableName());
+    format.setDatabaseName(job.getConfiguration(), table.getDatabaseName());
+
+    List<InputSplit> list= format.getSplits(job);
+    Assert.assertTrue(list.size() == 0);
+  }
+
+  @Test
+  public void testNotEqualHiveFilter() throws IOException {
+    ExprNodeDesc column = new ExprNodeColumnDesc(TypeInfoFactory.intTypeInfo, 
"id", null, false);
+    ExprNodeDesc constant = new 
ExprNodeConstantDesc(TypeInfoFactory.intTypeInfo, "500");
+    List<ExprNodeDesc> children = Lists.newArrayList();
+    children.add(column);
+    children.add(constant);
+    ExprNodeGenericFuncDesc node = new 
ExprNodeGenericFuncDesc(TypeInfoFactory.intTypeInfo,
+        new GenericUDFOPNotEqual(), children);
+    Configuration configuration=new Configuration();
+    CarbonInputFormat.setFilterPredicates(configuration,new 
DataMapFilter(table, Hive2CarbonExpression.convertExprHive2Carbon(node)));
+
+    final Job job = new Job(new JobConf(configuration));
+    final CarbonFileInputFormat format = new CarbonFileInputFormat();
+    format.setTableInfo(job.getConfiguration(), table.getTableInfo());
+    format.setTablePath(job.getConfiguration(), table.getTablePath());
+    format.setTableName(job.getConfiguration(), table.getTableName());
+    format.setDatabaseName(job.getConfiguration(), table.getDatabaseName());
+
+    List<InputSplit> list= format.getSplits(job);
+    Assert.assertTrue(list.size() == 1);
+  }
+
+  @Test
+  public void testOrHiveFilter() throws IOException {
+    ExprNodeDesc column1 = new ExprNodeColumnDesc(TypeInfoFactory.intTypeInfo, 
"id", null, false);
+    ExprNodeDesc constant1 = new 
ExprNodeConstantDesc(TypeInfoFactory.intTypeInfo, "500");
+    List<ExprNodeDesc> children1 = Lists.newArrayList();
+    children1.add(column1);
+    children1.add(constant1);
+    ExprNodeGenericFuncDesc node1 = new 
ExprNodeGenericFuncDesc(TypeInfoFactory.intTypeInfo,
+        new GenericUDFOPEqual(), children1);
+    ExprNodeDesc column2 = new ExprNodeColumnDesc(TypeInfoFactory.intTypeInfo, 
"id", null, false);
+    ExprNodeDesc constant2 = new 
ExprNodeConstantDesc(TypeInfoFactory.intTypeInfo, "4999999");
+    List<ExprNodeDesc> children2 = Lists.newArrayList();
+    children2.add(column2);
+    children2.add(constant2);
+    List<ExprNodeDesc> children3 = Lists.newArrayList();
+    ExprNodeGenericFuncDesc node2 = new 
ExprNodeGenericFuncDesc(TypeInfoFactory.intTypeInfo,
+        new GenericUDFOPEqual(), children2);
+    children3.add(node1);
+    children3.add(node2);
+    ExprNodeGenericFuncDesc node3=new 
ExprNodeGenericFuncDesc(TypeInfoFactory.intTypeInfo,
+        new GenericUDFOPOr(),children3);
+    Configuration configuration=new Configuration();
+    CarbonInputFormat.setFilterPredicates(configuration,new 
DataMapFilter(table, Hive2CarbonExpression.convertExprHive2Carbon(node3)));
+    final Job job = new Job(new JobConf(configuration));
+    final CarbonFileInputFormat format = new CarbonFileInputFormat();
+    format.setTableInfo(job.getConfiguration(), table.getTableInfo());
+    format.setTablePath(job.getConfiguration(), table.getTablePath());
+    format.setTableName(job.getConfiguration(), table.getTableName());
+    format.setDatabaseName(job.getConfiguration(), table.getDatabaseName());
+
+    List<InputSplit> list= format.getSplits(job);
+    Assert.assertTrue(list.size() == 1);
+  }
+
+  @Test
+  public void testAndHiveFilter() throws IOException {
+    ExprNodeDesc column1 = new ExprNodeColumnDesc(TypeInfoFactory.intTypeInfo, 
"id", null, false);
+    ExprNodeDesc constant1 = new 
ExprNodeConstantDesc(TypeInfoFactory.intTypeInfo, "500");
+    List<ExprNodeDesc> children1 = Lists.newArrayList();
+    children1.add(column1);
+    children1.add(constant1);
+    ExprNodeGenericFuncDesc node1 = new 
ExprNodeGenericFuncDesc(TypeInfoFactory.intTypeInfo,
+        new GenericUDFOPEqual(), children1);
+    ExprNodeDesc column2 = new ExprNodeColumnDesc(TypeInfoFactory.intTypeInfo, 
"id", null, false);
+    ExprNodeDesc constant2 = new 
ExprNodeConstantDesc(TypeInfoFactory.intTypeInfo, "4999999");
+    List<ExprNodeDesc> children2 = Lists.newArrayList();
+    children2.add(column2);
+    children2.add(constant2);
+    List<ExprNodeDesc> children3 = Lists.newArrayList();
+    ExprNodeGenericFuncDesc node2 = new 
ExprNodeGenericFuncDesc(TypeInfoFactory.intTypeInfo,
+        new GenericUDFOPEqual(), children2);
+    children3.add(node1);
+    children3.add(node2);
+    ExprNodeGenericFuncDesc node3=new 
ExprNodeGenericFuncDesc(TypeInfoFactory.intTypeInfo,
+        new GenericUDFOPAnd(),children3);
+    Configuration configuration=new Configuration();
+    CarbonInputFormat.setFilterPredicates(configuration,new 
DataMapFilter(table, Hive2CarbonExpression.convertExprHive2Carbon(node3)));
+    final Job job = new Job(new JobConf(configuration));
+    final CarbonFileInputFormat format = new CarbonFileInputFormat();
+    format.setTableInfo(job.getConfiguration(), table.getTableInfo());
+    format.setTablePath(job.getConfiguration(), table.getTablePath());
+    format.setTableName(job.getConfiguration(), table.getTableName());
+    format.setDatabaseName(job.getConfiguration(), table.getDatabaseName());
+
+    List<InputSplit> list= format.getSplits(job);
+    Assert.assertTrue(list.size() == 0);
+  }
+
+  @Test
+  public void testNullHiveFilter() throws IOException {
+    ExprNodeDesc column1 = new 
ExprNodeColumnDesc(TypeInfoFactory.booleanTypeInfo, "name", null, false);
+    List<ExprNodeDesc> children1 = Lists.newArrayList();
+    children1.add(column1);
+    ExprNodeGenericFuncDesc node1 = new 
ExprNodeGenericFuncDesc(TypeInfoFactory.booleanTypeInfo,
+        new GenericUDFOPNull(), children1);
+    Configuration configuration=new Configuration();
+    CarbonInputFormat.setFilterPredicates(configuration,new 
DataMapFilter(table, Hive2CarbonExpression.convertExprHive2Carbon(node1)));
+    final Job job = new Job(new JobConf(configuration));
+    final CarbonFileInputFormat format = new CarbonFileInputFormat();
+    format.setTableInfo(job.getConfiguration(), table.getTableInfo());
+    format.setTablePath(job.getConfiguration(), table.getTablePath());
+    format.setTableName(job.getConfiguration(), table.getTableName());
+    format.setDatabaseName(job.getConfiguration(), table.getDatabaseName());
+
+    List<InputSplit> list= format.getSplits(job);
+    Assert.assertTrue(list.size() == 0);
+  }
+
+  @Test
+  public void testNotNullHiveFilter() throws IOException {
+    ExprNodeDesc column1 = new 
ExprNodeColumnDesc(TypeInfoFactory.booleanTypeInfo, "name", null, false);
+    List<ExprNodeDesc> children1 = Lists.newArrayList();
+    children1.add(column1);
+    ExprNodeGenericFuncDesc node1 = new 
ExprNodeGenericFuncDesc(TypeInfoFactory.booleanTypeInfo,
+        new GenericUDFOPNotNull(), children1);
+    Configuration configuration=new Configuration();
+    CarbonInputFormat.setFilterPredicates(configuration,new 
DataMapFilter(table, Hive2CarbonExpression.convertExprHive2Carbon(node1)));
+    final Job job = new Job(new JobConf(configuration));
+    final CarbonFileInputFormat format = new CarbonFileInputFormat();
+    format.setTableInfo(job.getConfiguration(), table.getTableInfo());
+    format.setTablePath(job.getConfiguration(), table.getTablePath());
+    format.setTableName(job.getConfiguration(), table.getTableName());
+    format.setDatabaseName(job.getConfiguration(), table.getDatabaseName());
+
+    List<InputSplit> list= format.getSplits(job);
+    Assert.assertTrue(list.size() == 1);
+  }
+
+  @Test
+  public void testInHiveFilter() throws IOException {
+    ExprNodeDesc column1 = new ExprNodeColumnDesc(TypeInfoFactory.intTypeInfo, 
"id", null, false);
+    List<ExprNodeDesc> children1 = Lists.newArrayList();
+    ExprNodeDesc constant1 = new 
ExprNodeConstantDesc(TypeInfoFactory.intTypeInfo, "500");
+    ExprNodeDesc constant2 = new 
ExprNodeConstantDesc(TypeInfoFactory.intTypeInfo, "600");
+    ExprNodeDesc constant3 = new 
ExprNodeConstantDesc(TypeInfoFactory.intTypeInfo, "700");
+    children1.add(column1);
+    children1.add(constant1);
+    children1.add(constant2);
+    children1.add(constant3);
+    ExprNodeGenericFuncDesc node1 = new 
ExprNodeGenericFuncDesc(TypeInfoFactory.intTypeInfo,
+        new GenericUDFIn(), children1);
+    Configuration configuration=new Configuration();
+    CarbonInputFormat.setFilterPredicates(configuration,new 
DataMapFilter(table, Hive2CarbonExpression.convertExprHive2Carbon(node1)));
+    final Job job = new Job(new JobConf(configuration));
+    final CarbonFileInputFormat format = new CarbonFileInputFormat();
+    format.setTableInfo(job.getConfiguration(), table.getTableInfo());
+    format.setTablePath(job.getConfiguration(), table.getTablePath());
+    format.setTableName(job.getConfiguration(), table.getTableName());
+    format.setDatabaseName(job.getConfiguration(), table.getDatabaseName());
+
+    List<InputSplit> list= format.getSplits(job);
+    Assert.assertTrue(list.size() == 1);
+  }
+
+  @Test
+  public void testEqualOrGreaterThanHiveFilter() throws IOException {
+    ExprNodeDesc column1 = new ExprNodeColumnDesc(TypeInfoFactory.intTypeInfo, 
"id", null, false);
+    List<ExprNodeDesc> children1 = Lists.newArrayList();
+    ExprNodeDesc constant1 = new 
ExprNodeConstantDesc(TypeInfoFactory.intTypeInfo, "0");
+    children1.add(column1);
+    children1.add(constant1);
+    ExprNodeGenericFuncDesc node1 = new 
ExprNodeGenericFuncDesc(TypeInfoFactory.intTypeInfo,
+        new GenericUDFOPEqualOrGreaterThan(), children1);
+    Configuration configuration=new Configuration();
+    CarbonInputFormat.setFilterPredicates(configuration,new 
DataMapFilter(table, Hive2CarbonExpression.convertExprHive2Carbon(node1)));
+    final Job job = new Job(new JobConf(configuration));
+    final CarbonFileInputFormat format = new CarbonFileInputFormat();
+    format.setTableInfo(job.getConfiguration(), table.getTableInfo());
+    format.setTablePath(job.getConfiguration(), table.getTablePath());
+    format.setTableName(job.getConfiguration(), table.getTableName());
+    format.setDatabaseName(job.getConfiguration(), table.getDatabaseName());
+
+    List<InputSplit> list= format.getSplits(job);
+    Assert.assertTrue(list.size() == 1);
+
+  }
+
+  @Test
+  public void testEqualOrLessThanEqualsHiveFilter() throws IOException {
+    ExprNodeDesc column1 = new ExprNodeColumnDesc(TypeInfoFactory.intTypeInfo, 
"id", null, false);
+    List<ExprNodeDesc> children1 = Lists.newArrayList();
+    ExprNodeDesc constant1 = new 
ExprNodeConstantDesc(TypeInfoFactory.intTypeInfo, "1000");
+    children1.add(column1);
+    children1.add(constant1);
+    ExprNodeGenericFuncDesc node1 = new 
ExprNodeGenericFuncDesc(TypeInfoFactory.intTypeInfo,
+        new GenericUDFOPEqualOrLessThan(), children1);
+    Configuration configuration=new Configuration();
+    CarbonInputFormat.setFilterPredicates(configuration,new 
DataMapFilter(table, Hive2CarbonExpression.convertExprHive2Carbon(node1)));
+    final Job job = new Job(new JobConf(configuration));
+    final CarbonFileInputFormat format = new CarbonFileInputFormat();
+    format.setTableInfo(job.getConfiguration(), table.getTableInfo());
+    format.setTablePath(job.getConfiguration(), table.getTablePath());
+    format.setTableName(job.getConfiguration(), table.getTableName());
+    format.setDatabaseName(job.getConfiguration(), table.getDatabaseName());
+
+    List<InputSplit> list= format.getSplits(job);
+    Assert.assertTrue(list.size() == 1);
+
+  }
+
+  @Test
+  public void testLessThanEqualsHiveFilter() throws IOException {
+    ExprNodeDesc column1 = new ExprNodeColumnDesc(TypeInfoFactory.intTypeInfo, 
"id", null, false);
+    List<ExprNodeDesc> children1 = Lists.newArrayList();
+    ExprNodeDesc constant1 = new 
ExprNodeConstantDesc(TypeInfoFactory.intTypeInfo, "0");
+    children1.add(column1);
+    children1.add(constant1);
+    ExprNodeGenericFuncDesc node1 = new 
ExprNodeGenericFuncDesc(TypeInfoFactory.intTypeInfo,
+        new GenericUDFOPLessThan(), children1);
+    Configuration configuration=new Configuration();
+    CarbonInputFormat.setFilterPredicates(configuration,new 
DataMapFilter(table, Hive2CarbonExpression.convertExprHive2Carbon(node1)));
+    final Job job = new Job(new JobConf(configuration));
+    final CarbonFileInputFormat format = new CarbonFileInputFormat();
+    format.setTableInfo(job.getConfiguration(), table.getTableInfo());
+    format.setTablePath(job.getConfiguration(), table.getTablePath());
+    format.setTableName(job.getConfiguration(), table.getTableName());
+    format.setDatabaseName(job.getConfiguration(), table.getDatabaseName());
+
+    List<InputSplit> list= format.getSplits(job);
+    Assert.assertTrue(list.size() == 0);
+
+  }
+
+  @Test
+  public void testGreaterThanHiveFilter() throws IOException {
+    ExprNodeDesc column1 = new ExprNodeColumnDesc(TypeInfoFactory.intTypeInfo, 
"id", null, false);
+    List<ExprNodeDesc> children1 = Lists.newArrayList();
+    ExprNodeDesc constant1 = new 
ExprNodeConstantDesc(TypeInfoFactory.intTypeInfo, "1001");
+    children1.add(column1);
+    children1.add(constant1);
+    ExprNodeGenericFuncDesc node1 = new 
ExprNodeGenericFuncDesc(TypeInfoFactory.intTypeInfo,
+        new GenericUDFOPGreaterThan(), children1);
+    Configuration configuration=new Configuration();
+    CarbonInputFormat.setFilterPredicates(configuration,new 
DataMapFilter(table, Hive2CarbonExpression.convertExprHive2Carbon(node1)));
+    final Job job = new Job(new JobConf(configuration));
+    final CarbonFileInputFormat format = new CarbonFileInputFormat();
+    format.setTableInfo(job.getConfiguration(), table.getTableInfo());
+    format.setTablePath(job.getConfiguration(), table.getTablePath());
+    format.setTableName(job.getConfiguration(), table.getTableName());
+    format.setDatabaseName(job.getConfiguration(), table.getDatabaseName());
+
+    List<InputSplit> list= format.getSplits(job);
+    Assert.assertTrue(list.size() == 0);
+
+  }
+}
diff --git a/store/sdk/pom.xml b/store/sdk/pom.xml
index f28972d..9e43a8b 100644
--- a/store/sdk/pom.xml
+++ b/store/sdk/pom.xml
@@ -24,6 +24,12 @@
       <groupId>org.apache.carbondata</groupId>
       <artifactId>carbondata-hadoop</artifactId>
       <version>${project.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>org.apache.hive</groupId>
+          <artifactId>hive-exec</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
     <dependency>
       <groupId>junit</groupId>

Reply via email to