You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ra...@apache.org on 2018/09/05 12:40:09 UTC

[1/8] carbondata git commit: [CARBONDATA-2532][Integration] Carbon to support spark 2.3.1 version (Refactored streaming module and column vector to support spark 2.3.1)

Repository: carbondata
Updated Branches:
  refs/heads/master cf74c88cc -> b611a861a


http://git-wip-us.apache.org/repos/asf/carbondata/blob/bcef656d/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamRecordReader.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamRecordReader.java b/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamRecordReader.java
deleted file mode 100644
index 02e0ddf..0000000
--- a/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamRecordReader.java
+++ /dev/null
@@ -1,772 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.streaming;
-
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.nio.ByteBuffer;
-import java.util.BitSet;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.carbondata.core.cache.Cache;
-import org.apache.carbondata.core.cache.CacheProvider;
-import org.apache.carbondata.core.cache.CacheType;
-import org.apache.carbondata.core.cache.dictionary.Dictionary;
-import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
-import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
-import org.apache.carbondata.core.metadata.blocklet.index.BlockletMinMaxIndex;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
-import org.apache.carbondata.core.metadata.encoder.Encoding;
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
-import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.core.reader.CarbonHeaderReader;
-import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
-import org.apache.carbondata.core.scan.filter.FilterUtil;
-import org.apache.carbondata.core.scan.filter.GenericQueryType;
-import org.apache.carbondata.core.scan.filter.executer.FilterExecuter;
-import org.apache.carbondata.core.scan.filter.intf.RowImpl;
-import org.apache.carbondata.core.scan.filter.intf.RowIntf;
-import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
-import org.apache.carbondata.core.scan.model.QueryModel;
-import org.apache.carbondata.core.util.CarbonMetadataUtil;
-import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.DataTypeUtil;
-import org.apache.carbondata.format.BlockletHeader;
-import org.apache.carbondata.format.FileHeader;
-import org.apache.carbondata.hadoop.CarbonInputSplit;
-import org.apache.carbondata.hadoop.CarbonMultiBlockSplit;
-import org.apache.carbondata.hadoop.InputMetricsStats;
-import org.apache.carbondata.hadoop.api.CarbonTableInputFormat;
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-import org.apache.spark.memory.MemoryMode;
-import org.apache.spark.sql.catalyst.InternalRow;
-import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
-import org.apache.spark.sql.execution.vectorized.ColumnVector;
-import org.apache.spark.sql.execution.vectorized.ColumnarBatch;
-import org.apache.spark.sql.types.CalendarIntervalType;
-import org.apache.spark.sql.types.Decimal;
-import org.apache.spark.sql.types.DecimalType;
-import org.apache.spark.sql.types.StructField;
-import org.apache.spark.sql.types.StructType;
-import org.apache.spark.unsafe.types.CalendarInterval;
-import org.apache.spark.unsafe.types.UTF8String;
-
-/**
- * Stream record reader
- */
-public class CarbonStreamRecordReader extends RecordReader<Void, Object> {
-  // vector reader
-  private boolean isVectorReader;
-
-  // metadata
-  private CarbonTable carbonTable;
-  private CarbonColumn[] storageColumns;
-  private boolean[] isRequired;
-  private DataType[] measureDataTypes;
-  private int dimensionCount;
-  private int measureCount;
-
-  // input
-  private FileSplit fileSplit;
-  private Configuration hadoopConf;
-  private StreamBlockletReader input;
-  private boolean isFirstRow = true;
-  private QueryModel model;
-
-  // decode data
-  private BitSet allNonNull;
-  private boolean[] isNoDictColumn;
-  private DirectDictionaryGenerator[] directDictionaryGenerators;
-  private CacheProvider cacheProvider;
-  private Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache;
-  private GenericQueryType[] queryTypes;
-
-  // vectorized reader
-  private StructType outputSchema;
-  private ColumnarBatch columnarBatch;
-  private boolean isFinished = false;
-
-  // filter
-  private FilterExecuter filter;
-  private boolean[] isFilterRequired;
-  private Object[] filterValues;
-  private RowIntf filterRow;
-  private int[] filterMap;
-
-  // output
-  private CarbonColumn[] projection;
-  private boolean[] isProjectionRequired;
-  private int[] projectionMap;
-  private Object[] outputValues;
-  private InternalRow outputRow;
-
-  // empty project, null filter
-  private boolean skipScanData;
-
-  // return raw row for handoff
-  private boolean useRawRow = false;
-
-  // InputMetricsStats
-  private InputMetricsStats inputMetricsStats;
-
-  @Override public void initialize(InputSplit split, TaskAttemptContext context)
-      throws IOException, InterruptedException {
-    // input
-    if (split instanceof CarbonInputSplit) {
-      fileSplit = (CarbonInputSplit) split;
-    } else if (split instanceof CarbonMultiBlockSplit) {
-      fileSplit = ((CarbonMultiBlockSplit) split).getAllSplits().get(0);
-    } else {
-      fileSplit = (FileSplit) split;
-    }
-
-    // metadata
-    hadoopConf = context.getConfiguration();
-    if (model == null) {
-      CarbonTableInputFormat format = new CarbonTableInputFormat<Object>();
-      model = format.createQueryModel(split, context);
-    }
-    carbonTable = model.getTable();
-    List<CarbonDimension> dimensions =
-        carbonTable.getDimensionByTableName(carbonTable.getTableName());
-    dimensionCount = dimensions.size();
-    List<CarbonMeasure> measures =
-        carbonTable.getMeasureByTableName(carbonTable.getTableName());
-    measureCount = measures.size();
-    List<CarbonColumn> carbonColumnList =
-        carbonTable.getStreamStorageOrderColumn(carbonTable.getTableName());
-    storageColumns = carbonColumnList.toArray(new CarbonColumn[carbonColumnList.size()]);
-    isNoDictColumn = CarbonDataProcessorUtil.getNoDictionaryMapping(storageColumns);
-    directDictionaryGenerators = new DirectDictionaryGenerator[storageColumns.length];
-    for (int i = 0; i < storageColumns.length; i++) {
-      if (storageColumns[i].hasEncoding(Encoding.DIRECT_DICTIONARY)) {
-        directDictionaryGenerators[i] = DirectDictionaryKeyGeneratorFactory
-            .getDirectDictionaryGenerator(storageColumns[i].getDataType());
-      }
-    }
-    measureDataTypes = new DataType[measureCount];
-    for (int i = 0; i < measureCount; i++) {
-      measureDataTypes[i] = storageColumns[dimensionCount + i].getDataType();
-    }
-
-    // decode data
-    allNonNull = new BitSet(storageColumns.length);
-    projection = model.getProjectionColumns();
-
-    isRequired = new boolean[storageColumns.length];
-    boolean[] isFiltlerDimensions = model.getIsFilterDimensions();
-    boolean[] isFiltlerMeasures = model.getIsFilterMeasures();
-    isFilterRequired = new boolean[storageColumns.length];
-    filterMap = new int[storageColumns.length];
-    for (int i = 0; i < storageColumns.length; i++) {
-      if (storageColumns[i].isDimension()) {
-        if (isFiltlerDimensions[storageColumns[i].getOrdinal()]) {
-          isRequired[i] = true;
-          isFilterRequired[i] = true;
-          filterMap[i] = storageColumns[i].getOrdinal();
-        }
-      } else {
-        if (isFiltlerMeasures[storageColumns[i].getOrdinal()]) {
-          isRequired[i] = true;
-          isFilterRequired[i] = true;
-          filterMap[i] = carbonTable.getDimensionOrdinalMax() + storageColumns[i].getOrdinal();
-        }
-      }
-    }
-
-    isProjectionRequired = new boolean[storageColumns.length];
-    projectionMap = new int[storageColumns.length];
-    for (int j = 0; j < projection.length; j++) {
-      for (int i = 0; i < storageColumns.length; i++) {
-        if (storageColumns[i].getColName().equals(projection[j].getColName())) {
-          isRequired[i] = true;
-          isProjectionRequired[i] = true;
-          projectionMap[i] = j;
-          break;
-        }
-      }
-    }
-
-    // initialize filter
-    if (null != model.getFilterExpressionResolverTree()) {
-      initializeFilter();
-    } else if (projection.length == 0) {
-      skipScanData = true;
-    }
-
-  }
-
-  private void initializeFilter() {
-
-    List<ColumnSchema> wrapperColumnSchemaList = CarbonUtil
-        .getColumnSchemaList(carbonTable.getDimensionByTableName(carbonTable.getTableName()),
-            carbonTable.getMeasureByTableName(carbonTable.getTableName()));
-    int[] dimLensWithComplex = new int[wrapperColumnSchemaList.size()];
-    for (int i = 0; i < dimLensWithComplex.length; i++) {
-      dimLensWithComplex[i] = Integer.MAX_VALUE;
-    }
-
-    int[] dictionaryColumnCardinality =
-        CarbonUtil.getFormattedCardinality(dimLensWithComplex, wrapperColumnSchemaList);
-    SegmentProperties segmentProperties =
-        new SegmentProperties(wrapperColumnSchemaList, dictionaryColumnCardinality);
-    Map<Integer, GenericQueryType> complexDimensionInfoMap = new HashMap<>();
-
-    FilterResolverIntf resolverIntf = model.getFilterExpressionResolverTree();
-    filter = FilterUtil.getFilterExecuterTree(resolverIntf, segmentProperties,
-        complexDimensionInfoMap);
-    // for row filter, we need update column index
-    FilterUtil.updateIndexOfColumnExpression(resolverIntf.getFilterExpression(),
-        carbonTable.getDimensionOrdinalMax());
-
-  }
-
-  public void setQueryModel(QueryModel model) {
-    this.model = model;
-  }
-
-  private byte[] getSyncMarker(String filePath) throws IOException {
-    CarbonHeaderReader headerReader = new CarbonHeaderReader(filePath);
-    FileHeader header = headerReader.readHeader();
-    return header.getSync_marker();
-  }
-
-  public void setUseRawRow(boolean useRawRow) {
-    this.useRawRow = useRawRow;
-  }
-
-  private void initializeAtFirstRow() throws IOException {
-    filterValues = new Object[carbonTable.getDimensionOrdinalMax() + measureCount];
-    filterRow = new RowImpl();
-    filterRow.setValues(filterValues);
-
-    outputValues = new Object[projection.length];
-    outputRow = new GenericInternalRow(outputValues);
-
-    Path file = fileSplit.getPath();
-
-    byte[] syncMarker = getSyncMarker(file.toString());
-
-    FileSystem fs = file.getFileSystem(hadoopConf);
-
-    int bufferSize = Integer.parseInt(hadoopConf.get(CarbonStreamInputFormat.READ_BUFFER_SIZE,
-        CarbonStreamInputFormat.READ_BUFFER_SIZE_DEFAULT));
-
-    FSDataInputStream fileIn = fs.open(file, bufferSize);
-    fileIn.seek(fileSplit.getStart());
-    input = new StreamBlockletReader(syncMarker, fileIn, fileSplit.getLength(),
-        fileSplit.getStart() == 0);
-
-    cacheProvider = CacheProvider.getInstance();
-    cache = cacheProvider.createCache(CacheType.FORWARD_DICTIONARY);
-    queryTypes = CarbonStreamInputFormat.getComplexDimensions(carbonTable, storageColumns, cache);
-
-    outputSchema = new StructType((StructField[])
-        DataTypeUtil.getDataTypeConverter().convertCarbonSchemaToSparkSchema(projection));
-  }
-
-  @Override public boolean nextKeyValue() throws IOException, InterruptedException {
-    if (isFirstRow) {
-      isFirstRow = false;
-      initializeAtFirstRow();
-    }
-    if (isFinished) {
-      return false;
-    }
-
-    if (isVectorReader) {
-      return nextColumnarBatch();
-    }
-
-    return nextRow();
-  }
-
-  /**
-   * for vector reader, check next columnar batch
-   */
-  private boolean nextColumnarBatch() throws IOException {
-    boolean hasNext;
-    boolean scanMore = false;
-    do {
-      // move to the next blocklet
-      hasNext = input.nextBlocklet();
-      if (hasNext) {
-        // read blocklet header
-        BlockletHeader header = input.readBlockletHeader();
-        if (isScanRequired(header)) {
-          scanMore = !scanBlockletAndFillVector(header);
-        } else {
-          input.skipBlockletData(true);
-          scanMore = true;
-        }
-      } else {
-        isFinished = true;
-        scanMore = false;
-      }
-    } while (scanMore);
-    return hasNext;
-  }
-
-  /**
-   * check next Row
-   */
-  private boolean nextRow() throws IOException {
-    // read row one by one
-    try {
-      boolean hasNext;
-      boolean scanMore = false;
-      do {
-        hasNext = input.hasNext();
-        if (hasNext) {
-          if (skipScanData) {
-            input.nextRow();
-            scanMore = false;
-          } else {
-            if (useRawRow) {
-              // read raw row for streaming handoff which does not require decode raw row
-              readRawRowFromStream();
-            } else {
-              readRowFromStream();
-            }
-            if (null != filter) {
-              scanMore = !filter.applyFilter(filterRow, carbonTable.getDimensionOrdinalMax());
-            } else {
-              scanMore = false;
-            }
-          }
-        } else {
-          if (input.nextBlocklet()) {
-            BlockletHeader header = input.readBlockletHeader();
-            if (isScanRequired(header)) {
-              if (skipScanData) {
-                input.skipBlockletData(false);
-              } else {
-                input.readBlockletData(header);
-              }
-            } else {
-              input.skipBlockletData(true);
-            }
-            scanMore = true;
-          } else {
-            isFinished = true;
-            scanMore = false;
-          }
-        }
-      } while (scanMore);
-      return hasNext;
-    } catch (FilterUnsupportedException e) {
-      throw new IOException("Failed to filter row in detail reader", e);
-    }
-  }
-
-  @Override public Void getCurrentKey() throws IOException, InterruptedException {
-    return null;
-  }
-
-  @Override public Object getCurrentValue() throws IOException, InterruptedException {
-    if (isVectorReader) {
-      int value = columnarBatch.numValidRows();
-      if (inputMetricsStats != null) {
-        inputMetricsStats.incrementRecordRead((long) value);
-      }
-
-      return columnarBatch;
-    }
-
-    if (inputMetricsStats != null) {
-      inputMetricsStats.incrementRecordRead(1L);
-    }
-
-    return outputRow;
-  }
-
-  private boolean isScanRequired(BlockletHeader header) {
-    if (filter != null && header.getBlocklet_index() != null) {
-      BlockletMinMaxIndex minMaxIndex = CarbonMetadataUtil.convertExternalMinMaxIndex(
-          header.getBlocklet_index().getMin_max_index());
-      if (minMaxIndex != null) {
-        BitSet bitSet =
-            filter.isScanRequired(minMaxIndex.getMaxValues(), minMaxIndex.getMinValues());
-        if (bitSet.isEmpty()) {
-          return false;
-        } else {
-          return true;
-        }
-      }
-    }
-    return true;
-  }
-
-  private boolean scanBlockletAndFillVector(BlockletHeader header) throws IOException {
-    // if filter is null and output projection is empty, use the row number of blocklet header
-    if (skipScanData) {
-      int rowNums = header.getBlocklet_info().getNum_rows();
-      columnarBatch = ColumnarBatch.allocate(outputSchema, MemoryMode.OFF_HEAP, rowNums);
-      columnarBatch.setNumRows(rowNums);
-      input.skipBlockletData(true);
-      return rowNums > 0;
-    }
-
-    input.readBlockletData(header);
-    columnarBatch = ColumnarBatch.allocate(outputSchema, MemoryMode.OFF_HEAP, input.getRowNums());
-    int rowNum = 0;
-    if (null == filter) {
-      while (input.hasNext()) {
-        readRowFromStream();
-        putRowToColumnBatch(rowNum++);
-      }
-    } else {
-      try {
-        while (input.hasNext()) {
-          readRowFromStream();
-          if (filter.applyFilter(filterRow, carbonTable.getDimensionOrdinalMax())) {
-            putRowToColumnBatch(rowNum++);
-          }
-        }
-      } catch (FilterUnsupportedException e) {
-        throw new IOException("Failed to filter row in vector reader", e);
-      }
-    }
-    columnarBatch.setNumRows(rowNum);
-    return rowNum > 0;
-  }
-
-  private void readRowFromStream() {
-    input.nextRow();
-    short nullLen = input.readShort();
-    BitSet nullBitSet = allNonNull;
-    if (nullLen > 0) {
-      nullBitSet = BitSet.valueOf(input.readBytes(nullLen));
-    }
-    int colCount = 0;
-    // primitive type dimension
-    for (; colCount < isNoDictColumn.length; colCount++) {
-      if (nullBitSet.get(colCount)) {
-        if (isFilterRequired[colCount]) {
-          filterValues[filterMap[colCount]] = CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY;
-        }
-        if (isProjectionRequired[colCount]) {
-          outputValues[projectionMap[colCount]] = null;
-        }
-      } else {
-        if (isNoDictColumn[colCount]) {
-          int v = input.readShort();
-          if (isRequired[colCount]) {
-            byte[] b = input.readBytes(v);
-            if (isFilterRequired[colCount]) {
-              filterValues[filterMap[colCount]] = b;
-            }
-            if (isProjectionRequired[colCount]) {
-              outputValues[projectionMap[colCount]] =
-                  DataTypeUtil.getDataBasedOnDataTypeForNoDictionaryColumn(b,
-                      storageColumns[colCount].getDataType());
-            }
-          } else {
-            input.skipBytes(v);
-          }
-        } else if (null != directDictionaryGenerators[colCount]) {
-          if (isRequired[colCount]) {
-            if (isFilterRequired[colCount]) {
-              filterValues[filterMap[colCount]] = input.copy(4);
-            }
-            if (isProjectionRequired[colCount]) {
-              outputValues[projectionMap[colCount]] =
-                  directDictionaryGenerators[colCount].getValueFromSurrogate(input.readInt());
-            } else {
-              input.skipBytes(4);
-            }
-          } else {
-            input.skipBytes(4);
-          }
-        } else {
-          if (isRequired[colCount]) {
-            if (isFilterRequired[colCount]) {
-              filterValues[filterMap[colCount]] = input.copy(4);
-            }
-            if (isProjectionRequired[colCount]) {
-              outputValues[projectionMap[colCount]] = input.readInt();
-            } else {
-              input.skipBytes(4);
-            }
-          } else {
-            input.skipBytes(4);
-          }
-        }
-      }
-    }
-    // complex type dimension
-    for (; colCount < dimensionCount; colCount++) {
-      if (nullBitSet.get(colCount)) {
-        if (isFilterRequired[colCount]) {
-          filterValues[filterMap[colCount]] = null;
-        }
-        if (isProjectionRequired[colCount]) {
-          outputValues[projectionMap[colCount]] = null;
-        }
-      } else {
-        short v = input.readShort();
-        if (isRequired[colCount]) {
-          byte[] b = input.readBytes(v);
-          if (isFilterRequired[colCount]) {
-            filterValues[filterMap[colCount]] = b;
-          }
-          if (isProjectionRequired[colCount]) {
-            outputValues[projectionMap[colCount]] = queryTypes[colCount]
-                .getDataBasedOnDataType(ByteBuffer.wrap(b));
-          }
-        } else {
-          input.skipBytes(v);
-        }
-      }
-    }
-    // measure
-    DataType dataType;
-    for (int msrCount = 0; msrCount < measureCount; msrCount++, colCount++) {
-      if (nullBitSet.get(colCount)) {
-        if (isFilterRequired[colCount]) {
-          filterValues[filterMap[colCount]] = null;
-        }
-        if (isProjectionRequired[colCount]) {
-          outputValues[projectionMap[colCount]] = null;
-        }
-      } else {
-        dataType = measureDataTypes[msrCount];
-        if (dataType == DataTypes.BOOLEAN) {
-          if (isRequired[colCount]) {
-            boolean v = input.readBoolean();
-            if (isFilterRequired[colCount]) {
-              filterValues[filterMap[colCount]] = v;
-            }
-            if (isProjectionRequired[colCount]) {
-              outputValues[projectionMap[colCount]] = v;
-            }
-          } else {
-            input.skipBytes(1);
-          }
-        } else if (dataType == DataTypes.SHORT) {
-          if (isRequired[colCount]) {
-            short v = input.readShort();
-            if (isFilterRequired[colCount]) {
-              filterValues[filterMap[colCount]] = v;
-            }
-            if (isProjectionRequired[colCount]) {
-              outputValues[projectionMap[colCount]] = v;
-            }
-          } else {
-            input.skipBytes(2);
-          }
-        } else if (dataType == DataTypes.INT) {
-          if (isRequired[colCount]) {
-            int v = input.readInt();
-            if (isFilterRequired[colCount]) {
-              filterValues[filterMap[colCount]] = v;
-            }
-            if (isProjectionRequired[colCount]) {
-              outputValues[projectionMap[colCount]] = v;
-            }
-          } else {
-            input.skipBytes(4);
-          }
-        } else if (dataType == DataTypes.LONG) {
-          if (isRequired[colCount]) {
-            long v = input.readLong();
-            if (isFilterRequired[colCount]) {
-              filterValues[filterMap[colCount]] = v;
-            }
-            if (isProjectionRequired[colCount]) {
-              outputValues[projectionMap[colCount]] = v;
-            }
-          } else {
-            input.skipBytes(8);
-          }
-        } else if (dataType == DataTypes.DOUBLE) {
-          if (isRequired[colCount]) {
-            double v = input.readDouble();
-            if (isFilterRequired[colCount]) {
-              filterValues[filterMap[colCount]] = v;
-            }
-            if (isProjectionRequired[colCount]) {
-              outputValues[projectionMap[colCount]] = v;
-            }
-          } else {
-            input.skipBytes(8);
-          }
-        } else if (DataTypes.isDecimal(dataType)) {
-          int len = input.readShort();
-          if (isRequired[colCount]) {
-            BigDecimal v = DataTypeUtil.byteToBigDecimal(input.readBytes(len));
-            if (isFilterRequired[colCount]) {
-              filterValues[filterMap[colCount]] = v;
-            }
-            if (isProjectionRequired[colCount]) {
-              outputValues[projectionMap[colCount]] =
-                  DataTypeUtil.getDataTypeConverter().convertFromBigDecimalToDecimal(v);
-            }
-          } else {
-            input.skipBytes(len);
-          }
-        }
-      }
-    }
-  }
-
-  private void readRawRowFromStream() {
-    input.nextRow();
-    short nullLen = input.readShort();
-    BitSet nullBitSet = allNonNull;
-    if (nullLen > 0) {
-      nullBitSet = BitSet.valueOf(input.readBytes(nullLen));
-    }
-    int colCount = 0;
-    // primitive type dimension
-    for (; colCount < isNoDictColumn.length; colCount++) {
-      if (nullBitSet.get(colCount)) {
-        outputValues[colCount] = CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY;
-      } else {
-        if (isNoDictColumn[colCount]) {
-          int v = input.readShort();
-          outputValues[colCount] = input.readBytes(v);
-        } else {
-          outputValues[colCount] = input.readInt();
-        }
-      }
-    }
-    // complex type dimension
-    for (; colCount < dimensionCount; colCount++) {
-      if (nullBitSet.get(colCount)) {
-        outputValues[colCount] = null;
-      } else {
-        short v = input.readShort();
-        outputValues[colCount] = input.readBytes(v);
-      }
-    }
-    // measure
-    DataType dataType;
-    for (int msrCount = 0; msrCount < measureCount; msrCount++, colCount++) {
-      if (nullBitSet.get(colCount)) {
-        outputValues[colCount] = null;
-      } else {
-        dataType = measureDataTypes[msrCount];
-        if (dataType == DataTypes.BOOLEAN) {
-          outputValues[colCount] = input.readBoolean();
-        } else if (dataType == DataTypes.SHORT) {
-          outputValues[colCount] = input.readShort();
-        } else if (dataType == DataTypes.INT) {
-          outputValues[colCount] = input.readInt();
-        } else if (dataType == DataTypes.LONG) {
-          outputValues[colCount] = input.readLong();
-        } else if (dataType == DataTypes.DOUBLE) {
-          outputValues[colCount] = input.readDouble();
-        } else if (DataTypes.isDecimal(dataType)) {
-          int len = input.readShort();
-          outputValues[colCount] = DataTypeUtil.byteToBigDecimal(input.readBytes(len));
-        }
-      }
-    }
-  }
-
-  private void putRowToColumnBatch(int rowId) {
-    for (int i = 0; i < projection.length; i++) {
-      Object value = outputValues[i];
-      ColumnVector col = columnarBatch.column(i);
-      org.apache.spark.sql.types.DataType t = col.dataType();
-      if (null == value) {
-        col.putNull(rowId);
-      } else {
-        if (t == org.apache.spark.sql.types.DataTypes.BooleanType) {
-          col.putBoolean(rowId, (boolean)value);
-        } else if (t == org.apache.spark.sql.types.DataTypes.ByteType) {
-          col.putByte(rowId, (byte) value);
-        } else if (t == org.apache.spark.sql.types.DataTypes.ShortType) {
-          col.putShort(rowId, (short) value);
-        } else if (t == org.apache.spark.sql.types.DataTypes.IntegerType) {
-          col.putInt(rowId, (int) value);
-        } else if (t == org.apache.spark.sql.types.DataTypes.LongType) {
-          col.putLong(rowId, (long) value);
-        } else if (t == org.apache.spark.sql.types.DataTypes.FloatType) {
-          col.putFloat(rowId, (float) value);
-        } else if (t == org.apache.spark.sql.types.DataTypes.DoubleType) {
-          col.putDouble(rowId, (double) value);
-        } else if (t == org.apache.spark.sql.types.DataTypes.StringType) {
-          UTF8String v = (UTF8String) value;
-          col.putByteArray(rowId, v.getBytes());
-        } else if (t instanceof org.apache.spark.sql.types.DecimalType) {
-          DecimalType dt = (DecimalType)t;
-          Decimal d = Decimal.fromDecimal(value);
-          if (dt.precision() <= Decimal.MAX_INT_DIGITS()) {
-            col.putInt(rowId, (int)d.toUnscaledLong());
-          } else if (dt.precision() <= Decimal.MAX_LONG_DIGITS()) {
-            col.putLong(rowId, d.toUnscaledLong());
-          } else {
-            final BigInteger integer = d.toJavaBigDecimal().unscaledValue();
-            byte[] bytes = integer.toByteArray();
-            col.putByteArray(rowId, bytes, 0, bytes.length);
-          }
-        } else if (t instanceof CalendarIntervalType) {
-          CalendarInterval c = (CalendarInterval) value;
-          col.getChildColumn(0).putInt(rowId, c.months);
-          col.getChildColumn(1).putLong(rowId, c.microseconds);
-        } else if (t instanceof org.apache.spark.sql.types.DateType) {
-          col.putInt(rowId, (int) value);
-        } else if (t instanceof org.apache.spark.sql.types.TimestampType) {
-          col.putLong(rowId, (long) value);
-        }
-      }
-    }
-  }
-
-  @Override public float getProgress() throws IOException, InterruptedException {
-    return 0;
-  }
-
-  public void setVectorReader(boolean isVectorReader) {
-    this.isVectorReader = isVectorReader;
-  }
-
-  public void setInputMetricsStats(InputMetricsStats inputMetricsStats) {
-    this.inputMetricsStats = inputMetricsStats;
-  }
-
-  @Override public void close() throws IOException {
-    if (null != input) {
-      input.close();
-    }
-    if (null != columnarBatch) {
-      columnarBatch.close();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bcef656d/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamUtils.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamUtils.java b/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamUtils.java
new file mode 100644
index 0000000..a7940f9
--- /dev/null
+++ b/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamUtils.java
@@ -0,0 +1,40 @@
+/*
+ * 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.streaming;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+
+/**
+ * Util class which does utility function for stream module
+ */
+public class CarbonStreamUtils {
+
+  public static Constructor getConstructorWithReflection(String className,
+                                                           Class<?>... parameterTypes)
+            throws ClassNotFoundException, NoSuchMethodException {
+    Class loadedClass = Class.forName(className);
+    return loadedClass.getConstructor(parameterTypes);
+
+  }
+
+  public static Object getInstanceWithReflection(Constructor cons, Object... initargs) throws
+          IllegalAccessException,
+          InvocationTargetException, InstantiationException {
+    return cons.newInstance(initargs);
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bcef656d/streaming/src/main/java/org/apache/carbondata/streaming/StreamBlockletReader.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/carbondata/streaming/StreamBlockletReader.java b/streaming/src/main/java/org/apache/carbondata/streaming/StreamBlockletReader.java
index 43fe6ed..5c7ad5e 100644
--- a/streaming/src/main/java/org/apache/carbondata/streaming/StreamBlockletReader.java
+++ b/streaming/src/main/java/org/apache/carbondata/streaming/StreamBlockletReader.java
@@ -46,7 +46,8 @@ public class StreamBlockletReader {
   private int rowIndex = 0;
   private boolean isHeaderPresent;
 
-  StreamBlockletReader(byte[] syncMarker, InputStream in, long limit, boolean isHeaderPresent) {
+  public StreamBlockletReader(byte[] syncMarker, InputStream in, long limit,
+      boolean isHeaderPresent) {
     this.syncMarker = syncMarker;
     syncLen = syncMarker.length;
     syncBuffer = new byte[syncLen];
@@ -96,7 +97,7 @@ public class StreamBlockletReader {
     return false;
   }
 
-  BlockletHeader readBlockletHeader() throws IOException {
+  public BlockletHeader readBlockletHeader() throws IOException {
     int len = readIntFromStream();
     byte[] b = new byte[len];
     if (!readBytesFromStream(b, 0, len)) {
@@ -108,7 +109,7 @@ public class StreamBlockletReader {
     return header;
   }
 
-  void readBlockletData(BlockletHeader header) throws IOException {
+  public void readBlockletData(BlockletHeader header) throws IOException {
     ensureCapacity(header.getBlocklet_length());
     offset = 0;
     int len = readIntFromStream();
@@ -119,7 +120,7 @@ public class StreamBlockletReader {
     compressor.rawUncompress(b, buffer);
   }
 
-  void skipBlockletData(boolean reset) throws IOException {
+  public void skipBlockletData(boolean reset) throws IOException {
     int len = readIntFromStream();
     skip(len);
     pos += len;
@@ -140,7 +141,7 @@ public class StreamBlockletReader {
   /**
    * find the next blocklet
    */
-  boolean nextBlocklet() throws IOException {
+  public boolean nextBlocklet() throws IOException {
     if (pos >= limitStart) {
       return false;
     }
@@ -158,15 +159,15 @@ public class StreamBlockletReader {
     return pos < limitEnd;
   }
 
-  boolean hasNext() throws IOException {
+  public boolean hasNext() throws IOException {
     return rowIndex < rowNums;
   }
 
-  void nextRow() {
+  public void nextRow() {
     rowIndex++;
   }
 
-  int readIntFromStream() throws IOException {
+  public int readIntFromStream() throws IOException {
     int ch1 = in.read();
     int ch2 = in.read();
     int ch3 = in.read();
@@ -182,7 +183,7 @@ public class StreamBlockletReader {
    * @return <code>true</code> if reading data successfully, or
    * <code>false</code> if there is no more data because the end of the stream has been reached.
    */
-  boolean readBytesFromStream(byte[] b, int offset, int len) throws IOException {
+  public boolean readBytesFromStream(byte[] b, int offset, int len) throws IOException {
     int readLen = in.read(b, offset, len);
     if (readLen < 0) {
       return false;
@@ -195,24 +196,24 @@ public class StreamBlockletReader {
     }
   }
 
-  boolean readBoolean() {
+  public boolean readBoolean() {
     return (buffer[offset++]) != 0;
   }
 
-  short readShort() {
+  public short readShort() {
     short v =  (short) ((buffer[offset + 1] & 255) +
         ((buffer[offset]) << 8));
     offset += 2;
     return v;
   }
 
-  byte[] copy(int len) {
+  public byte[] copy(int len) {
     byte[] b = new byte[len];
     System.arraycopy(buffer, offset, b, 0, len);
     return b;
   }
 
-  int readInt() {
+  public int readInt() {
     int v = ((buffer[offset + 3] & 255) +
         ((buffer[offset + 2] & 255) << 8) +
         ((buffer[offset + 1] & 255) << 16) +
@@ -221,7 +222,7 @@ public class StreamBlockletReader {
     return v;
   }
 
-  long readLong() {
+  public long readLong() {
     long v = ((long)(buffer[offset + 7] & 255)) +
         ((long) (buffer[offset + 6] & 255) << 8) +
         ((long) (buffer[offset + 5] & 255) << 16) +
@@ -234,26 +235,26 @@ public class StreamBlockletReader {
     return v;
   }
 
-  double readDouble() {
+  public double readDouble() {
     return Double.longBitsToDouble(readLong());
   }
 
-  byte[] readBytes(int len) {
+  public byte[] readBytes(int len) {
     byte[] b = new byte[len];
     System.arraycopy(buffer, offset, b, 0, len);
     offset += len;
     return b;
   }
 
-  void skipBytes(int len) {
+  public void skipBytes(int len) {
     offset += len;
   }
 
-  int getRowNums() {
+  public int getRowNums() {
     return rowNums;
   }
 
-  void close() {
+  public void close() {
     CarbonUtil.closeStreams(in);
   }
 }


[3/8] carbondata git commit: [CARBONDATA-2532][Integration] Carbon to support spark 2.3.1 version(Make API changes in carbon to be compatible with spark 2.3)

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index b14e328..d55c21a 100644
--- a/pom.xml
+++ b/pom.xml
@@ -569,6 +569,66 @@
                 <sourceDirectory>${basedir}/hadoop/src/main/java</sourceDirectory>
                 <sourceDirectory>${basedir}/integration/spark2/src/main/scala</sourceDirectory>
                 <sourceDirectory>${basedir}/integration/spark2/src/main/spark2.2</sourceDirectory>
+                <sourceDirectory>${basedir}/integration/spark2/src/main/commonTo2.2And2.3</sourceDirectory>
+                <sourceDirectory>${basedir}/integration/spark2/src/main/java</sourceDirectory>
+                <sourceDirectory>${basedir}/integration/spark-common/src/main/scala</sourceDirectory>
+                <sourceDirectory>${basedir}/integration/spark-common/src/main/java</sourceDirectory>
+                <sourceDirectory>${basedir}/integration/spark-common-test/src/main/scala</sourceDirectory>
+                <sourceDirectory>${basedir}/integration/spark-common-test/src/main/java</sourceDirectory>
+                <sourceDirectory>${basedir}/integration/hive/src/main/scala</sourceDirectory>
+                <sourceDirectory>${basedir}/integration/hive/src/main/java</sourceDirectory>
+                <sourceDirectory>${basedir}/integration/presto/src/main/scala</sourceDirectory>
+                <sourceDirectory>${basedir}/integration/presto/src/main/java</sourceDirectory>
+                <sourceDirectory>${basedir}/streaming/src/main/java</sourceDirectory>
+                <sourceDirectory>${basedir}/streaming/src/main/scala</sourceDirectory>
+                <sourceDirectory>${basedir}/store/sdk/src/main/java</sourceDirectory>
+                <sourceDirectory>${basedir}/store/search/src/main/scala</sourceDirectory>
+                <sourceDirectory>${basedir}/store/search/src/main/java</sourceDirectory>
+                <sourceDirectory>${basedir}/datamap/bloom/src/main/java</sourceDirectory>
+                <sourceDirectory>${basedir}/datamap/lucene/src/main/java</sourceDirectory>
+              </sourceDirectories>
+            </configuration>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+    <profile>
+      <id>spark-2.3</id>
+      <properties>
+        <spark.version>2.3.1</spark.version>
+        <scala.binary.version>2.11</scala.binary.version>
+        <scala.version>2.11.8</scala.version>
+      </properties>
+      <modules>
+        <module>integration/spark2</module>
+        <module>integration/hive</module>
+        <module>integration/presto</module>
+        <module>streaming</module>
+        <module>examples/spark2</module>
+        <module>datamap/lucene</module>
+        <module>datamap/bloom</module>
+      </modules>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.eluder.coveralls</groupId>
+            <artifactId>coveralls-maven-plugin</artifactId>
+            <version>4.3.0</version>
+            <configuration>
+              <repoToken>opPwqWW41vYppv6KISea3u1TJvE1ugJ5Y</repoToken>
+              <sourceEncoding>UTF-8</sourceEncoding>
+              <jacocoReports>
+                <jacocoReport>${basedir}/target/carbondata-coverage-report/carbondata-coverage-report.xml
+                </jacocoReport>
+              </jacocoReports>
+              <sourceDirectories>
+                <sourceDirectory>${basedir}/common/src/main/java</sourceDirectory>
+                <sourceDirectory>${basedir}/core/src/main/java</sourceDirectory>
+                <sourceDirectory>${basedir}/processing/src/main/java</sourceDirectory>
+                <sourceDirectory>${basedir}/hadoop/src/main/java</sourceDirectory>
+                <sourceDirectory>${basedir}/integration/spark2/src/main/scala</sourceDirectory>
+                <sourceDirectory>${basedir}/integration/spark2/src/main/commonTo2.2And2.3</sourceDirectory>
+                <sourceDirectory>${basedir}/integration/spark2/src/main/spark2.3</sourceDirectory>
                 <sourceDirectory>${basedir}/integration/spark2/src/main/java</sourceDirectory>
                 <sourceDirectory>${basedir}/integration/spark-common/src/main/scala</sourceDirectory>
                 <sourceDirectory>${basedir}/integration/spark-common/src/main/java</sourceDirectory>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/store/search/src/main/scala/org/apache/spark/rpc/Master.scala
----------------------------------------------------------------------
diff --git a/store/search/src/main/scala/org/apache/spark/rpc/Master.scala b/store/search/src/main/scala/org/apache/spark/rpc/Master.scala
index b7630fb..97951ea 100644
--- a/store/search/src/main/scala/org/apache/spark/rpc/Master.scala
+++ b/store/search/src/main/scala/org/apache/spark/rpc/Master.scala
@@ -81,7 +81,7 @@ class Master(sparkConf: SparkConf) {
           do {
             try {
               LOG.info(s"starting registry-service on $hostAddress:$port")
-              val config = RpcEnvConfig(
+              val config = RpcUtil.getRpcEnvConfig(
                 sparkConf, "registry-service", hostAddress, "", port,
                 new SecurityManager(sparkConf), clientMode = false)
               rpcEnv = new NettyRpcEnvFactory().create(config)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/store/search/src/main/scala/org/apache/spark/rpc/RpcUtil.scala
----------------------------------------------------------------------
diff --git a/store/search/src/main/scala/org/apache/spark/rpc/RpcUtil.scala b/store/search/src/main/scala/org/apache/spark/rpc/RpcUtil.scala
new file mode 100644
index 0000000..f15bb8f
--- /dev/null
+++ b/store/search/src/main/scala/org/apache/spark/rpc/RpcUtil.scala
@@ -0,0 +1,56 @@
+/*
+ * 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.spark.rpc
+
+import org.apache.spark.{SecurityManager, SPARK_VERSION, SparkConf}
+import org.apache.spark.util.Utils
+
+object RpcUtil {
+
+  def getRpcEnvConfig(conf: SparkConf,
+      name: String,
+      bindAddress: String,
+      advertiseAddress: String,
+      port: Int,
+      securityManager: SecurityManager,
+      clientMode: Boolean): RpcEnvConfig = {
+    val className = "org.apache.spark.rpc.RpcEnvConfig"
+    if (SPARK_VERSION.startsWith("2.1") || SPARK_VERSION.startsWith("2.2")) {
+      createObject(className, conf, name, bindAddress,
+        advertiseAddress, port.asInstanceOf[Object],
+        securityManager, clientMode.asInstanceOf[Object])._1.asInstanceOf[RpcEnvConfig]
+    } else if (SPARK_VERSION.startsWith("2.3")) {
+      // numUsableCores if it is 0 then spark will consider the available CPUs on the host.
+      val numUsableCores: Int = 0
+      createObject(className, conf, name, bindAddress,
+        advertiseAddress, port.asInstanceOf[Object],
+        securityManager, numUsableCores.asInstanceOf[Object],
+        clientMode.asInstanceOf[Object])._1.asInstanceOf[RpcEnvConfig]
+    } else {
+      throw new UnsupportedOperationException("Spark version not supported")
+    }
+  }
+
+  def createObject(className: String, conArgs: Object*): (Any, Class[_]) = {
+    val clazz = Utils.classForName(className)
+    val ctor = clazz.getConstructors.head
+    ctor.setAccessible(true)
+    (ctor.newInstance(conArgs: _*), clazz)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/store/search/src/main/scala/org/apache/spark/rpc/Worker.scala
----------------------------------------------------------------------
diff --git a/store/search/src/main/scala/org/apache/spark/rpc/Worker.scala b/store/search/src/main/scala/org/apache/spark/rpc/Worker.scala
index 0f2138a..08baeeb 100644
--- a/store/search/src/main/scala/org/apache/spark/rpc/Worker.scala
+++ b/store/search/src/main/scala/org/apache/spark/rpc/Worker.scala
@@ -60,7 +60,7 @@ object Worker {
         do {
           try {
             LOG.info(s"starting search-service on $hostAddress:$port")
-            val config = RpcEnvConfig(
+            val config = RpcUtil.getRpcEnvConfig(
               conf, s"worker-$hostAddress", hostAddress, "", port,
               new SecurityManager(conf), clientMode = false)
             rpcEnv = new NettyRpcEnvFactory().create(config)
@@ -89,7 +89,7 @@ object Worker {
   private def registerToMaster(masterHostAddress: String, masterPort: Int): String = {
     LOG.info(s"trying to register to master $masterHostAddress:$masterPort")
     val conf = new SparkConf()
-    val config = RpcEnvConfig(conf, "registry-client", masterHostAddress, "", masterPort,
+    val config = RpcUtil.getRpcEnvConfig(conf, "registry-client", masterHostAddress, "", masterPort,
       new SecurityManager(conf), clientMode = true)
     val rpcEnv: RpcEnv = new NettyRpcEnvFactory().create(config)
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamInputFormat.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamInputFormat.java b/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamInputFormat.java
index c0e270c..835b115 100644
--- a/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamInputFormat.java
+++ b/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamInputFormat.java
@@ -64,7 +64,7 @@ public class CarbonStreamInputFormat extends FileInputFormat<Void, Object> {
     this.inputMetricsStats = inputMetricsStats;
   }
 
-  public void setVectorReader(boolean vectorReader) {
+  public void setIsVectorReader(boolean vectorReader) {
     isVectorReader = vectorReader;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/streaming/src/test/java/org/apache/carbondata/streaming/CarbonStreamInputFormatTest.java
----------------------------------------------------------------------
diff --git a/streaming/src/test/java/org/apache/carbondata/streaming/CarbonStreamInputFormatTest.java b/streaming/src/test/java/org/apache/carbondata/streaming/CarbonStreamInputFormatTest.java
deleted file mode 100644
index a224446..0000000
--- a/streaming/src/test/java/org/apache/carbondata/streaming/CarbonStreamInputFormatTest.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.streaming;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.List;
-import java.util.UUID;
-
-import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
-import org.apache.carbondata.core.statusmanager.FileFormat;
-import org.apache.carbondata.hadoop.CarbonInputSplit;
-import org.apache.carbondata.hadoop.CarbonMultiBlockSplit;
-import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil;
-
-import junit.framework.TestCase;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.JobID;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.hadoop.mapreduce.TaskID;
-import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class CarbonStreamInputFormatTest extends TestCase {
-
-  private TaskAttemptID taskAttemptId;
-  private TaskAttemptContext taskAttemptContext;
-  private Configuration hadoopConf;
-  private AbsoluteTableIdentifier identifier;
-  private String tablePath;
-
-
-  @Override protected void setUp() throws Exception {
-    tablePath = new File("target/stream_input").getCanonicalPath();
-    String dbName = "default";
-    String tableName = "stream_table_input";
-    identifier = AbsoluteTableIdentifier.from(
-        tablePath,
-        new CarbonTableIdentifier(dbName, tableName, UUID.randomUUID().toString()));
-
-    JobID jobId = CarbonInputFormatUtil.getJobId(new Date(), 0);
-    TaskID taskId = new TaskID(jobId, TaskType.MAP, 0);
-    taskAttemptId = new TaskAttemptID(taskId, 0);
-
-    hadoopConf = new Configuration();
-    taskAttemptContext = new TaskAttemptContextImpl(hadoopConf, taskAttemptId);
-  }
-
-  private InputSplit buildInputSplit() throws IOException {
-    CarbonInputSplit carbonInputSplit = new CarbonInputSplit();
-    List<CarbonInputSplit> splitList = new ArrayList<>();
-    splitList.add(carbonInputSplit);
-    return new CarbonMultiBlockSplit(splitList, new String[] { "localhost" },
-        FileFormat.ROW_V1);
-  }
-
-  @Test public void testCreateRecordReader() {
-    try {
-      InputSplit inputSplit = buildInputSplit();
-      CarbonStreamInputFormat inputFormat = new CarbonStreamInputFormat();
-      RecordReader recordReader = inputFormat.createRecordReader(inputSplit, taskAttemptContext);
-      Assert.assertNotNull("Failed to create record reader", recordReader);
-    } catch (Exception e) {
-      e.printStackTrace();
-      Assert.assertTrue(e.getMessage(), false);
-    }
-  }
-
-  @Override protected void tearDown() throws Exception {
-    super.tearDown();
-    if (tablePath != null) {
-      FileFactory.deleteAllFilesOfDir(new File(tablePath));
-    }
-  }
-}


[5/8] carbondata git commit: [CARBONDATA-2532][Integration] Carbon to support spark 2.3.1 version(Make API changes in carbon to be compatible with spark 2.3)

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/scala/org/apache/carbondata/stream/CarbonStreamRecordReader.java
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/stream/CarbonStreamRecordReader.java b/integration/spark2/src/main/scala/org/apache/carbondata/stream/CarbonStreamRecordReader.java
index 124413d..747b064 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/stream/CarbonStreamRecordReader.java
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/stream/CarbonStreamRecordReader.java
@@ -18,17 +18,14 @@
 package org.apache.carbondata.stream;
 
 import java.io.IOException;
-import java.lang.reflect.Constructor;
-import java.lang.reflect.Method;
 import java.math.BigDecimal;
+import java.math.BigInteger;
 import java.nio.ByteBuffer;
 import java.util.BitSet;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.cache.Cache;
 import org.apache.carbondata.core.cache.CacheProvider;
 import org.apache.carbondata.core.cache.CacheType;
@@ -78,12 +75,11 @@ import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-
 import org.apache.spark.memory.MemoryMode;
+import org.apache.spark.sql.CarbonVectorProxy;
 import org.apache.spark.sql.catalyst.InternalRow;
 import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
-import org.apache.spark.sql.types.StructField;
-import org.apache.spark.sql.types.StructType;
+import org.apache.spark.sql.types.*;
 
 /**
  * Stream record reader
@@ -117,7 +113,7 @@ public class CarbonStreamRecordReader extends RecordReader<Void, Object> {
 
   // vectorized reader
   private StructType outputSchema;
-  private Object vectorProxy;
+  private CarbonVectorProxy vectorProxy;
   private boolean isFinished = false;
 
   // filter
@@ -143,9 +139,6 @@ public class CarbonStreamRecordReader extends RecordReader<Void, Object> {
   // InputMetricsStats
   private InputMetricsStats inputMetricsStats;
 
-  private static final LogService LOGGER =
-          LogServiceFactory.getLogService(CarbonStreamRecordReader.class.getName());
-
   public CarbonStreamRecordReader(boolean isVectorReader, InputMetricsStats inputMetricsStats,
       QueryModel mdl, boolean useRawRow) {
     this.isVectorReader = isVectorReader;
@@ -400,21 +393,15 @@ public class CarbonStreamRecordReader extends RecordReader<Void, Object> {
     return null;
   }
 
-  @Override public Object getCurrentValue() throws IOException, InterruptedException {
-    if (isVectorReader) {
-      Method method = null;
-      try {
-        method = vectorProxy.getClass().getMethod("numRows");
-        int value = (int) method.invoke(vectorProxy);
-        if (inputMetricsStats != null) {
-          inputMetricsStats.incrementRecordRead((long) value);
+    @Override public Object getCurrentValue() throws IOException, InterruptedException {
+        if (isVectorReader) {
+            int value = vectorProxy.numRows();
+            if (inputMetricsStats != null) {
+                inputMetricsStats.incrementRecordRead((long) value);
+            }
+
+            return vectorProxy.getColumnarBatch();
         }
-        method = vectorProxy.getClass().getMethod("getColumnarBatch");
-        return method.invoke(vectorProxy);
-      } catch (Exception e) {
-        throw new IOException(e);
-      }
-    }
 
     if (inputMetricsStats != null) {
       inputMetricsStats.incrementRecordRead(1L);
@@ -440,50 +427,39 @@ public class CarbonStreamRecordReader extends RecordReader<Void, Object> {
     return true;
   }
 
-  private boolean scanBlockletAndFillVector(BlockletHeader header) throws IOException {
-    Constructor cons = null;
-    // if filter is null and output projection is empty, use the row number of blocklet header
-    int rowNum = 0;
-    String methodName = "setNumRows";
-    try {
-      String vectorReaderClassName = "org.apache.spark.sql.CarbonVectorProxy";
-      cons = CarbonStreamUtils.getConstructorWithReflection(vectorReaderClassName, MemoryMode.class,
-              StructType.class, int.class);
-      if (skipScanData) {
-
-        int rowNums = header.getBlocklet_info().getNum_rows();
-        vectorProxy = cons.newInstance(MemoryMode.OFF_HEAP, outputSchema, rowNums);
-        Method setNumRowsMethod = vectorProxy.getClass().getMethod(methodName, int.class);
-        setNumRowsMethod.invoke(vectorProxy, rowNums);
-        input.skipBlockletData(true);
-        return rowNums > 0;
-      }
-      input.readBlockletData(header);
-      vectorProxy = cons.newInstance(MemoryMode.OFF_HEAP,outputSchema, input.getRowNums());
-      if (null == filter) {
-        while (input.hasNext()) {
-          readRowFromStream();
-          putRowToColumnBatch(rowNum++);
+    private boolean scanBlockletAndFillVector(BlockletHeader header) throws IOException {
+        // if filter is null and output projection is empty, use the row number of blocklet header
+        if (skipScanData) {
+            int rowNums = header.getBlocklet_info().getNum_rows();
+            vectorProxy= new CarbonVectorProxy(MemoryMode.OFF_HEAP,outputSchema,rowNums);
+            vectorProxy.setNumRows(rowNums);
+            input.skipBlockletData(true);
+            return rowNums > 0;
         }
-      } else {
-        try {
-          while (input.hasNext()) {
-            readRowFromStream();
-            if (filter.applyFilter(filterRow, carbonTable.getDimensionOrdinalMax())) {
-              putRowToColumnBatch(rowNum++);
+
+        input.readBlockletData(header);
+        vectorProxy= new CarbonVectorProxy(MemoryMode.OFF_HEAP,outputSchema,input.getRowNums());
+        int rowNum = 0;
+        if (null == filter) {
+            while (input.hasNext()) {
+                readRowFromStream();
+                putRowToColumnBatch(rowNum++);
+            }
+        } else {
+            try {
+                while (input.hasNext()) {
+                    readRowFromStream();
+                    if (filter.applyFilter(filterRow, carbonTable.getDimensionOrdinalMax())) {
+                        putRowToColumnBatch(rowNum++);
+                    }
+                }
+            } catch (FilterUnsupportedException e) {
+                throw new IOException("Failed to filter row in vector reader", e);
             }
-          }
-        } catch (FilterUnsupportedException e) {
-          throw new IOException("Failed to filter row in vector reader", e);
         }
-      }
-      Method setNumRowsMethod = vectorProxy.getClass().getMethod(methodName, int.class);
-      setNumRowsMethod.invoke(vectorProxy, rowNum);
-    } catch (Exception e) {
-      throw new IOException("Failed to fill row in  vector reader", e);
+        vectorProxy.setNumRows(rowNum);
+        return rowNum > 0;
     }
-    return rowNum > 0;
-  }
 
   private void readRowFromStream() {
     input.nextRow();
@@ -719,43 +695,24 @@ public class CarbonStreamRecordReader extends RecordReader<Void, Object> {
     }
   }
 
-  private void putRowToColumnBatch(int rowId) {
-    Class<?>[] paramTypes = {int.class, Object.class, int.class};
-    Method putRowToColumnBatch = null;
-    try {
-      putRowToColumnBatch = vectorProxy.getClass().getMethod("putRowToColumnBatch", paramTypes);
+    private void putRowToColumnBatch(int rowId) {
+        for (int i = 0; i < projection.length; i++) {
+            Object value = outputValues[i];
+            vectorProxy.putRowToColumnBatch(rowId,value,i);
 
-    } catch (Exception e) {
-      LOGGER.error(
-              "Unable to put the row in the vector" + "rowid: " + rowId + e);
-    }
-    for (int i = 0; i < projection.length; i++) {
-      Object value = outputValues[i];
-      try {
-        putRowToColumnBatch.invoke(vectorProxy, rowId, value, i);
-      } catch (Exception e) {
-        LOGGER.error(
-                "Unable to put the row in the vector" + "rowid: " + rowId + e);
-      }
+        }
     }
-  }
 
-  @Override public float getProgress() throws IOException, InterruptedException {
-    return 0;
-  }
-
-  @Override public void close() throws IOException {
-    if (null != input) {
-      input.close();
+    @Override public float getProgress() throws IOException, InterruptedException {
+        return 0;
     }
-    if (null != vectorProxy) {
-      try {
-        Method closeMethod = vectorProxy.getClass().getMethod("close");
-        closeMethod.invoke(vectorProxy);
-      } catch (Exception e) {
-        LOGGER.error(
-                "Unable to close the stream vector reader" + e);
-      }
+
+    @Override public void close() throws IOException {
+        if (null != input) {
+            input.close();
+        }
+        if (null != vectorProxy) {
+            vectorProxy.close();
+        }
     }
-  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/scala/org/apache/spark/CarbonInputMetrics.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/CarbonInputMetrics.scala b/integration/spark2/src/main/scala/org/apache/spark/CarbonInputMetrics.scala
index 6f038eb..41fc013 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/CarbonInputMetrics.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/CarbonInputMetrics.scala
@@ -20,7 +20,6 @@ import java.lang.Long
 
 import org.apache.spark.deploy.SparkHadoopUtil
 import org.apache.spark.executor.InputMetrics
-import org.apache.spark.sql.execution.vectorized.ColumnarBatch
 
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.util.TaskMetricsMap

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
index d433470..450ead1 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
@@ -21,16 +21,11 @@ import scala.collection.mutable
 
 import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.expressions.aggregate.Count
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Count}
 import org.apache.spark.sql.catalyst.plans.logical.{UnaryNode, _}
 import org.apache.spark.sql.execution.datasources.LogicalRelation
-import org.apache.spark.sql.hive.HiveSessionCatalog
 import org.apache.spark.sql.optimizer.CarbonDecoderRelation
-import org.apache.spark.sql.types.{StringType, TimestampType}
 
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.spark.CarbonAliasDecoderRelation
 
 case class CarbonDictionaryCatalystDecoder(
@@ -125,22 +120,40 @@ case class InsertIntoCarbonTable (table: CarbonDatasourceHadoopRelation,
 object CountStarPlan {
   type ReturnType = (mutable.MutableList[Attribute], LogicalPlan)
 
-  /**
-   * It fill count star query attribute.
-   */
-  private def fillCountStarAttribute(
-      expr: Expression,
-      outputColumns: mutable.MutableList[Attribute]) {
-    expr match {
-      case par@Alias(_, _) =>
-        val head = par.children.head.children.head
-        head match {
-          case count: Count if count.children.head.isInstanceOf[Literal] =>
-            outputColumns += par.toAttribute
-          case _ =>
-        }
-    }
-  }
+ /**
+  * It fill count star query attribute.
+  * 2.2.1 plan
+  * Aggregate [count(1) AS count(1)#30L]
+  * +- Project
+  *
+  *2.3.0 plan
+  * Aggregate [cast(count(1) as string) AS count(1)#29]
+  * +- Project
+  */
+ private def fillCountStarAttribute(
+     expr: Expression,
+     outputColumns: mutable.MutableList[Attribute]) {
+   expr match {
+     case par@Alias(cast: Cast, _) =>
+       if (cast.child.isInstanceOf[AggregateExpression]) {
+         val head = cast.child.children.head
+         head match {
+           case count: Count if count.children.head.isInstanceOf[Literal] =>
+             outputColumns += par.toAttribute
+           case _ =>
+         }
+       }
+     case par@Alias(child, _) =>
+       if (child.isInstanceOf[AggregateExpression]) {
+         val head = child.children.head
+         head match {
+           case count: Count if count.children.head.isInstanceOf[Literal] =>
+             outputColumns += par.toAttribute
+           case _ =>
+         }
+       }
+   }
+ }
 
   def unapply(plan: LogicalPlan): Option[ReturnType] = {
     plan match {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCountStar.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCountStar.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCountStar.scala
index 9b78db0..ac8eb64 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCountStar.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCountStar.scala
@@ -30,6 +30,8 @@ import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.execution.LeafExecNode
 import org.apache.spark.sql.optimizer.CarbonFilters
+import org.apache.spark.sql.types.StringType
+import org.apache.spark.unsafe.types.UTF8String
 
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
@@ -64,7 +66,13 @@ case class CarbonCountStar(
             carbonTable.getTableName,
             Some(carbonTable.getDatabaseName))).map(_.asJava).orNull),
       carbonTable)
-    val value = new GenericInternalRow(Seq(Long.box(rowCount)).toArray.asInstanceOf[Array[Any]])
+    val valueRaw =
+      attributesRaw.head.dataType match {
+        case StringType => Seq(UTF8String.fromString(Long.box(rowCount).toString)).toArray
+          .asInstanceOf[Array[Any]]
+        case _ => Seq(Long.box(rowCount)).toArray.asInstanceOf[Array[Any]]
+      }
+    val value = new GenericInternalRow(valueRaw)
     val unsafeProjection = UnsafeProjection.create(output.map(_.dataType).toArray)
     val row = if (outUnsafeRows) unsafeProjection(value) else value
     sparkContext.parallelize(Seq(row))

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala
index 96a8162..d6117de 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala
@@ -356,12 +356,7 @@ object CarbonSession {
         // Register a successfully instantiated context to the singleton. This should be at the
         // end of the class definition so that the singleton is updated only if there is no
         // exception in the construction of the instance.
-        sparkContext.addSparkListener(new SparkListener {
-          override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = {
-            SparkSession.setDefaultSession(null)
-            SparkSession.sqlListener.set(null)
-          }
-        })
+        CarbonToSparkAdapater.addSparkListener(sparkContext)
         session.streams.addListener(new CarbonStreamingQueryListener(session))
       }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/scala/org/apache/spark/sql/CustomDeterministicExpression.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CustomDeterministicExpression.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CustomDeterministicExpression.scala
index 6312746..1ff9bc3 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CustomDeterministicExpression.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CustomDeterministicExpression.scala
@@ -34,8 +34,6 @@ case class CustomDeterministicExpression(nonDt: Expression ) extends Expression
 
   override def children: Seq[Expression] = Seq()
 
-  override def deterministic: Boolean = true
-
   def childexp : Expression = nonDt
 
   override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = ev.copy("")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
----------------------------------------------------------------------
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 516f9af..52801b1 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
@@ -884,17 +884,21 @@ case class CarbonLoadDataCommand(
       // datatype is always int
       val column = table.getColumnByName(table.getTableName, attr.name)
       if (column.hasEncoding(Encoding.DICTIONARY)) {
-        AttributeReference(
-          attr.name,
+        CarbonToSparkAdapater.createAttributeReference(attr.name,
           IntegerType,
           attr.nullable,
-          attr.metadata)(attr.exprId, attr.qualifier, attr.isGenerated)
+          attr.metadata,
+          attr.exprId,
+          attr.qualifier,
+          attr)
       } else if (attr.dataType == TimestampType || attr.dataType == DateType) {
-        AttributeReference(
-          attr.name,
+        CarbonToSparkAdapater.createAttributeReference(attr.name,
           LongType,
           attr.nullable,
-          attr.metadata)(attr.exprId, attr.qualifier, attr.isGenerated)
+          attr.metadata,
+          attr.exprId,
+          attr.qualifier,
+          attr)
       } else {
         attr
       }
@@ -1095,7 +1099,8 @@ case class CarbonLoadDataCommand(
 
     CarbonReflectionUtils.getLogicalRelation(hdfsRelation,
       hdfsRelation.schema.toAttributes,
-      Some(catalogTable))
+      Some(catalogTable),
+      false)
   }
 
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/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 1fcba3e..8f128fe 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
@@ -35,6 +35,7 @@ import org.apache.spark.sql.sources.{BaseRelation, Filter}
 import org.apache.spark.sql.types._
 import org.apache.spark.sql.CarbonExpressions.{MatchCast => Cast}
 import org.apache.spark.sql.carbondata.execution.datasources.CarbonSparkDataSourceUtil
+import org.apache.spark.util.CarbonReflectionUtils
 
 import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.core.constants.CarbonCommonConstants
@@ -48,6 +49,7 @@ import org.apache.carbondata.spark.CarbonAliasDecoderRelation
 import org.apache.carbondata.spark.rdd.CarbonScanRDD
 import org.apache.carbondata.spark.util.CarbonScalaUtil
 
+
 /**
  * Carbon specific optimization for late decode (convert dictionary key to value as late as
  * possible), which can improve the aggregation performance and reduce memory usage
@@ -55,17 +57,33 @@ import org.apache.carbondata.spark.util.CarbonScalaUtil
 private[sql] class CarbonLateDecodeStrategy extends SparkStrategy {
   val PUSHED_FILTERS = "PushedFilters"
 
+  /*
+  Spark 2.3.1 plan there can be case of multiple projections like below
+  Project [substring(name, 1, 2)#124, name#123, tupleId#117, cast(rand(-6778822102499951904)#125
+  as string) AS rand(-6778822102499951904)#137]
+   +- Project [substring(name#123, 1, 2) AS substring(name, 1, 2)#124, name#123, UDF:getTupleId()
+    AS tupleId#117,
+       customdeterministicexpression(rand(-6778822102499951904)) AS rand(-6778822102499951904)#125]
+   +- Relation[imei#118,age#119,task#120L,num#121,level#122,name#123]
+   CarbonDatasourceHadoopRelation []
+ */
   def apply(plan: LogicalPlan): Seq[SparkPlan] = {
     plan match {
       case PhysicalOperation(projects, filters, l: LogicalRelation)
         if l.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
         val relation = l.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
-        pruneFilterProject(
-          l,
-          projects,
-          filters,
-          (a, f, needDecoder, p) => toCatalystRDD(l, a, relation.buildScan(
-            a.map(_.name).toArray, filters, projects, f, p), needDecoder)) :: Nil
+        // In Spark 2.3.1 there is case of multiple projections like below
+        // if 1 projection is failed then need to continue to other
+        try {
+          pruneFilterProject(
+            l,
+            projects,
+            filters,
+            (a, f, needDecoder, p) => toCatalystRDD(l, a, relation.buildScan(
+              a.map(_.name).toArray, filters, projects, f, p), needDecoder)) :: Nil
+        } catch {
+          case e: CarbonPhysicalPlanException => Nil
+        }
       case CarbonDictionaryCatalystDecoder(relations, profile, aliasMap, _, child) =>
         if ((profile.isInstanceOf[IncludeProfile] && profile.isEmpty) ||
             !CarbonDictionaryDecoder.
@@ -157,17 +175,20 @@ private[sql] class CarbonLateDecodeStrategy extends SparkStrategy {
     if (names.nonEmpty) {
       val partitionSet = AttributeSet(names
         .map(p => relation.output.find(_.name.equalsIgnoreCase(p)).get))
-      val partitionKeyFilters =
-        ExpressionSet(ExpressionSet(filterPredicates).filter(_.references.subsetOf(partitionSet)))
+      val partitionKeyFilters = CarbonToSparkAdapater
+        .getPartitionKeyFilter(partitionSet, filterPredicates)
       // Update the name with lower case as it is case sensitive while getting partition info.
       val updatedPartitionFilters = partitionKeyFilters.map { exp =>
         exp.transform {
           case attr: AttributeReference =>
-            AttributeReference(
+            CarbonToSparkAdapater.createAttributeReference(
               attr.name.toLowerCase,
               attr.dataType,
               attr.nullable,
-              attr.metadata)(attr.exprId, attr.qualifier, attr.isGenerated)
+              attr.metadata,
+              attr.exprId,
+              attr.qualifier,
+              attr)
         }
       }
       partitions =
@@ -224,7 +245,7 @@ private[sql] class CarbonLateDecodeStrategy extends SparkStrategy {
       }
     }
 
-    val (unhandledPredicates, pushedFilters) =
+    val (unhandledPredicates, pushedFilters, handledFilters ) =
       selectFilters(relation.relation, candidatePredicates)
 
     // A set of column attributes that are only referenced by pushed down filters.  We can eliminate
@@ -232,10 +253,13 @@ private[sql] class CarbonLateDecodeStrategy extends SparkStrategy {
     val handledSet = {
       val handledPredicates = filterPredicates.filterNot(unhandledPredicates.contains)
       val unhandledSet = AttributeSet(unhandledPredicates.flatMap(_.references))
-      AttributeSet(handledPredicates.flatMap(_.references)) --
-      (projectSet ++ unhandledSet).map(relation.attributeMap)
+      try {
+        AttributeSet(handledPredicates.flatMap(_.references)) --
+        (projectSet ++ unhandledSet).map(relation.attributeMap)
+      } catch {
+        case e => throw new CarbonPhysicalPlanException
+      }
     }
-
     // Combines all Catalyst filter `Expression`s that are either not convertible to data source
     // `Filter`s or cannot be handled by `relation`.
     val filterCondition = unhandledPredicates.reduceLeftOption(expressions.And)
@@ -309,6 +333,7 @@ private[sql] class CarbonLateDecodeStrategy extends SparkStrategy {
         scanBuilder,
         candidatePredicates,
         pushedFilters,
+        handledFilters,
         metadata,
         needDecoder,
         updateRequestedColumns.asInstanceOf[Seq[Attribute]])
@@ -345,6 +370,7 @@ private[sql] class CarbonLateDecodeStrategy extends SparkStrategy {
         scanBuilder,
         candidatePredicates,
         pushedFilters,
+        handledFilters,
         metadata,
         needDecoder,
         updateRequestedColumns.asInstanceOf[Seq[Attribute]])
@@ -359,9 +385,10 @@ private[sql] class CarbonLateDecodeStrategy extends SparkStrategy {
       output: Seq[Attribute],
       partitions: Seq[PartitionSpec],
       scanBuilder: (Seq[Attribute], Seq[Expression], Seq[Filter],
-        ArrayBuffer[AttributeReference], Seq[PartitionSpec]) => RDD[InternalRow],
+        ArrayBuffer[AttributeReference], Seq[PartitionSpec])
+        => RDD[InternalRow],
       candidatePredicates: Seq[Expression],
-      pushedFilters: Seq[Filter],
+      pushedFilters: Seq[Filter], handledFilters: Seq[Filter],
       metadata: Map[String, String],
       needDecoder: ArrayBuffer[AttributeReference],
       updateRequestedColumns: Seq[Attribute]): DataSourceScanExec = {
@@ -380,19 +407,16 @@ private[sql] class CarbonLateDecodeStrategy extends SparkStrategy {
         metadata,
         relation.catalogTable.map(_.identifier), relation)
     } else {
-      RowDataSourceScanExec(output,
-        scanBuilder(updateRequestedColumns,
-          candidatePredicates,
-          pushedFilters,
-          needDecoder,
-          partitions),
-        relation.relation,
-        getPartitioning(table.carbonTable, updateRequestedColumns),
-        metadata,
-        relation.catalogTable.map(_.identifier))
+      val partition = getPartitioning(table.carbonTable, updateRequestedColumns)
+      val rdd = scanBuilder(updateRequestedColumns, candidatePredicates,
+        pushedFilters, needDecoder, partitions)
+      CarbonReflectionUtils.getRowDataSourceScanExecObj(relation, output,
+        pushedFilters, handledFilters,
+        rdd, partition, metadata)
     }
   }
 
+
   def updateRequestedColumnsFunc(requestedColumns: Seq[Expression],
       relation: CarbonDatasourceHadoopRelation,
       needDecoder: ArrayBuffer[AttributeReference]): Seq[Expression] = {
@@ -471,7 +495,7 @@ private[sql] class CarbonLateDecodeStrategy extends SparkStrategy {
 
   protected[sql] def selectFilters(
       relation: BaseRelation,
-      predicates: Seq[Expression]): (Seq[Expression], Seq[Filter]) = {
+      predicates: Seq[Expression]): (Seq[Expression], Seq[Filter], Seq[Filter]) = {
 
     // In case of ComplexType dataTypes no filters should be pushed down. IsNotNull is being
     // explicitly added by spark and pushed. That also has to be handled and pushed back to
@@ -536,7 +560,7 @@ private[sql] class CarbonLateDecodeStrategy extends SparkStrategy {
     // a filter to every row or not.
     val (_, translatedFilters) = translated.unzip
 
-    (unrecognizedPredicates ++ unhandledPredicates, translatedFilters)
+    (unrecognizedPredicates ++ unhandledPredicates, translatedFilters, handledFilters)
   }
 
   /**
@@ -701,3 +725,5 @@ private[sql] class CarbonLateDecodeStrategy extends SparkStrategy {
     }
   }
 }
+
+class CarbonPhysicalPlanException extends Exception

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
index 0c9490d..4499b19 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
@@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.strategy
 import org.apache.spark.sql._
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, UnresolvedRelation}
+import org.apache.spark.sql.catalyst.catalog.CatalogTable
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.execution.{SparkPlan, SparkStrategy}
 import org.apache.spark.sql.execution.command._
@@ -38,9 +39,17 @@ import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.util.{CarbonProperties, ThreadLocalSessionInfo}
 
-/**
- * Carbon strategies for ddl commands
- */
+  /**
+   * Carbon strategies for ddl commands
+   * CreateDataSourceTableAsSelectCommand class has extra argument in
+   * 2.3, so need to add wrapper to match the case
+   */
+object MatchCreateDataSourceTable {
+  def unapply(plan: LogicalPlan): Option[(CatalogTable, SaveMode, LogicalPlan)] = plan match {
+    case t: CreateDataSourceTableAsSelectCommand => Some(t.table, t.mode, t.query)
+    case _ => None
+  }
+}
 
 class DDLStrategy(sparkSession: SparkSession) extends SparkStrategy {
   val LOGGER: LogService =
@@ -231,7 +240,7 @@ class DDLStrategy(sparkSession: SparkSession) extends SparkStrategy {
         val cmd =
           CreateDataSourceTableCommand(updatedCatalog, ignoreIfExists = mode == SaveMode.Ignore)
         ExecutedCommandExec(cmd) :: Nil
-      case cmd@CreateDataSourceTableAsSelectCommand(tableDesc, mode, query)
+      case MatchCreateDataSourceTable(tableDesc, mode, query)
         if tableDesc.provider.get != DDLUtils.HIVE_PROVIDER
            && (tableDesc.provider.get.equals("org.apache.spark.sql.CarbonSource")
                || tableDesc.provider.get.equalsIgnoreCase("carbondata")) =>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
index 97c37df..62e2d85 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
@@ -32,7 +32,7 @@ import org.apache.spark.sql.execution.command.mutation.CarbonProjectForDeleteCom
 import org.apache.spark.sql.execution.datasources.{CatalogFileIndex, FileFormat, HadoopFsRelation, LogicalRelation, SparkCarbonTableFormat}
 import org.apache.spark.sql.types.StructType
 import org.apache.spark.sql.util.CarbonException
-import org.apache.spark.util.CarbonReflectionUtils
+import org.apache.spark.util.{CarbonReflectionUtils, SparkUtil}
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datamap.DataMapStoreManager
@@ -74,9 +74,9 @@ case class CarbonIUDAnalysisRule(sparkSession: SparkSession) extends Rule[Logica
             "Update operation is not supported for table which has index datamaps")
         }
       }
-      val tableRelation = if (SPARK_VERSION.startsWith("2.1")) {
+      val tableRelation = if (SparkUtil.isSparkVersionEqualTo("2.1")) {
         relation
-      } else if (SPARK_VERSION.startsWith("2.2")) {
+      } else if (SparkUtil.isSparkVersionXandAbove("2.2")) {
         alias match {
           case Some(_) =>
             CarbonReflectionUtils.getSubqueryAlias(
@@ -206,9 +206,9 @@ case class CarbonIUDAnalysisRule(sparkSession: SparkSession) extends Rule[Logica
           }
         }
         // include tuple id in subquery
-        if (SPARK_VERSION.startsWith("2.1")) {
+        if (SparkUtil.isSparkVersionEqualTo("2.1")) {
           Project(projList, relation)
-        } else if (SPARK_VERSION.startsWith("2.2")) {
+        } else if (SparkUtil.isSparkVersionXandAbove("2.2")) {
           alias match {
             case Some(_) =>
               val subqueryAlias = CarbonReflectionUtils.getSubqueryAlias(
@@ -277,14 +277,13 @@ case class CarbonPreInsertionCasts(sparkSession: SparkSession) extends Rule[Logi
           case attr => attr
         }
       }
-      val version = SPARK_VERSION
       val newChild: LogicalPlan = if (newChildOutput == child.output) {
-        if (version.startsWith("2.1")) {
+        if (SparkUtil.isSparkVersionEqualTo("2.1")) {
           CarbonReflectionUtils.getField("child", p).asInstanceOf[LogicalPlan]
-        } else if (version.startsWith("2.2")) {
+        } else if (SparkUtil.isSparkVersionEqualTo("2.2")) {
           CarbonReflectionUtils.getField("query", p).asInstanceOf[LogicalPlan]
         } else {
-          throw new UnsupportedOperationException(s"Spark version $version is not supported")
+          throw new UnsupportedOperationException(s"Spark version $SPARK_VERSION is not supported")
         }
       } else {
         Project(newChildOutput, child)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
index 70e61bc..1840c5d 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
@@ -23,7 +23,6 @@ import java.net.URI
 import scala.collection.mutable.ArrayBuffer
 
 import org.apache.hadoop.fs.permission.{FsAction, FsPermission}
-import org.apache.spark.SPARK_VERSION
 import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, CarbonEnv, SparkSession}
 import org.apache.spark.sql.CarbonExpressions.{CarbonSubqueryAlias => SubqueryAlias}
 import org.apache.spark.sql.catalyst.TableIdentifier
@@ -31,7 +30,8 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
 import org.apache.spark.sql.catalyst.catalog.CatalogTable
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.execution.datasources.LogicalRelation
-import org.apache.spark.util.CarbonReflectionUtils
+import org.apache.spark.sql.sources.BaseRelation
+import org.apache.spark.util.{CarbonReflectionUtils, SparkUtil}
 
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.cache.dictionary.ManageDictionaryAndBTree
@@ -43,9 +43,8 @@ import org.apache.carbondata.core.fileoperations.FileWriteOperation
 import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata, CarbonTableIdentifier}
 import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
 import org.apache.carbondata.core.metadata.schema
-import org.apache.carbondata.core.metadata.schema.table
+import org.apache.carbondata.core.metadata.schema.{table, SchemaReader}
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.metadata.schema.SchemaReader
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
 import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.core.writer.ThriftWriter
@@ -72,6 +71,13 @@ case class DictionaryMap(dictionaryMap: Map[String, Boolean]) {
   }
 }
 
+object MatchLogicalRelation {
+  def unapply(logicalPlan: LogicalPlan): Option[(BaseRelation, Any, Any)] = logicalPlan match {
+    case l: LogicalRelation => Some(l.relation, l.output, l.catalogTable)
+    case _ => None
+  }
+}
+
 class CarbonFileMetastore extends CarbonMetaStore {
 
   @transient
@@ -143,13 +149,13 @@ class CarbonFileMetastore extends CarbonMetaStore {
       sparkSession.catalog.currentDatabase)
     val relation = sparkSession.sessionState.catalog.lookupRelation(tableIdentifier) match {
       case SubqueryAlias(_,
-      LogicalRelation(carbonDatasourceHadoopRelation: CarbonDatasourceHadoopRelation, _, _)) =>
+      MatchLogicalRelation(carbonDatasourceHadoopRelation: CarbonDatasourceHadoopRelation, _, _)) =>
         carbonDatasourceHadoopRelation.carbonRelation
-      case LogicalRelation(
+      case MatchLogicalRelation(
       carbonDatasourceHadoopRelation: CarbonDatasourceHadoopRelation, _, _) =>
         carbonDatasourceHadoopRelation.carbonRelation
       case SubqueryAlias(_, c)
-        if SPARK_VERSION.startsWith("2.2") &&
+        if (SparkUtil.isSparkVersionXandAbove("2.2")) &&
            (c.getClass.getName.equals("org.apache.spark.sql.catalyst.catalog.CatalogRelation") ||
             c.getClass.getName.equals("org.apache.spark.sql.catalyst.catalog.HiveTableRelation") ||
             c.getClass.getName.equals(
@@ -598,13 +604,13 @@ class CarbonFileMetastore extends CarbonMetaStore {
     val relation: LogicalPlan = sparkSession.sessionState.catalog.lookupRelation(tableIdentifier)
     relation match {
       case SubqueryAlias(_,
-      LogicalRelation(carbonDataSourceHadoopRelation: CarbonDatasourceHadoopRelation, _, _)) =>
+      MatchLogicalRelation(carbonDataSourceHadoopRelation: CarbonDatasourceHadoopRelation, _, _)) =>
         carbonDataSourceHadoopRelation
-      case LogicalRelation(
+      case MatchLogicalRelation(
       carbonDataSourceHadoopRelation: CarbonDatasourceHadoopRelation, _, _) =>
         carbonDataSourceHadoopRelation
       case SubqueryAlias(_, c)
-        if SPARK_VERSION.startsWith("2.2") &&
+        if (SparkUtil.isSparkVersionXandAbove("2.2")) &&
            (c.getClass.getName.equals("org.apache.spark.sql.catalyst.catalog.CatalogRelation") ||
             c.getClass.getName
               .equals("org.apache.spark.sql.catalyst.catalog.HiveTableRelation") ||

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonPreAggregateRules.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonPreAggregateRules.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonPreAggregateRules.scala
index c59246d..76ff41a 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonPreAggregateRules.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonPreAggregateRules.scala
@@ -273,11 +273,14 @@ case class CarbonPreAggregateQueryRules(sparkSession: SparkSession) extends Rule
         case attr: AttributeReference =>
           updatedExpression.find { p => p._1.sameRef(attr) } match {
             case Some((_, childAttr)) =>
-              AttributeReference(
+              CarbonToSparkAdapater.createAttributeReference(
                 childAttr.name,
                 childAttr.dataType,
                 childAttr.nullable,
-                childAttr.metadata)(childAttr.exprId, attr.qualifier, attr.isGenerated)
+                childAttr.metadata,
+                childAttr.exprId,
+                attr.qualifier,
+                attr)
             case None =>
               attr
           }
@@ -296,11 +299,14 @@ case class CarbonPreAggregateQueryRules(sparkSession: SparkSession) extends Rule
         case attr: AttributeReference =>
           updatedExpression.find { p => p._1.sameRef(attr) } match {
             case Some((_, childAttr)) =>
-              AttributeReference(
+              CarbonToSparkAdapater.createAttributeReference(
                 childAttr.name,
                 childAttr.dataType,
                 childAttr.nullable,
-                childAttr.metadata)(childAttr.exprId, attr.qualifier, attr.isGenerated)
+                childAttr.metadata,
+                childAttr.exprId,
+                attr.qualifier,
+                attr)
             case None =>
               attr
           }
@@ -777,24 +783,36 @@ case class CarbonPreAggregateQueryRules(sparkSession: SparkSession) extends Rule
         val factAlias = factPlanExpForStreaming(name)
         // create attribute reference object for each expression
         val attrs = factAlias.map { factAlias =>
-          AttributeReference(
+          CarbonToSparkAdapater.createAttributeReference(
             name,
             alias.dataType,
-            alias.nullable) (factAlias.exprId, alias.qualifier, alias.isGenerated)
+            alias.nullable,
+            Metadata.empty,
+            factAlias.exprId,
+            alias.qualifier,
+            alias)
         }
         // add aggregate function in Aggregate node added for handling streaming
         // to aggregate results from fact and aggregate table
         val updatedAggExp = getAggregateExpressionForAggregation(aggExp, attrs)
         // same reference id will be used as it can be used by above nodes in the plan like
         // sort, project, join
-        Alias(
+        CarbonToSparkAdapater.createAliasRef(
           updatedAggExp.head,
-          name)(alias.exprId, alias.qualifier, Option(alias.metadata), alias.isGenerated)
+          name,
+          alias.exprId,
+          alias.qualifier,
+          Option(alias.metadata),
+          Some(alias))
       case alias@Alias(expression, name) =>
-        AttributeReference(
+        CarbonToSparkAdapater.createAttributeReference(
           name,
           alias.dataType,
-          alias.nullable) (alias.exprId, alias.qualifier, alias.isGenerated)
+          alias.nullable,
+          Metadata.empty,
+          alias.exprId,
+          alias.qualifier,
+          alias)
     }
     updatedExp
   }
@@ -897,9 +915,10 @@ case class CarbonPreAggregateQueryRules(sparkSession: SparkSession) extends Rule
           case attr: AttributeReference =>
             newAggExp += attr
           case exp: Expression =>
-            newAggExp += Alias(
+            newAggExp += CarbonToSparkAdapater.createAliasRef(
               exp,
-              "dummy_" + counter)(NamedExpression.newExprId, None, None, false)
+              "dummy_" + counter,
+              NamedExpression.newExprId)
             counter = counter + 1
         }
       }
@@ -923,12 +942,12 @@ case class CarbonPreAggregateQueryRules(sparkSession: SparkSession) extends Rule
         // get the new aggregate expression
         val newAggExp = getAggFunctionForFactStreaming(aggExp)
         val updatedExp = newAggExp.map { exp =>
-          Alias(exp,
-              name)(
-              NamedExpression.newExprId,
-              alias.qualifier,
+          CarbonToSparkAdapater.createAliasRef(exp,
+            name,
+            NamedExpression.newExprId,
+            alias.qualifier,
             Some(alias.metadata),
-              alias.isGenerated)
+            Some(alias))
         }
         // adding to map which will be used while Adding an Aggregate node for handling streaming
         // table plan change
@@ -936,10 +955,13 @@ case class CarbonPreAggregateQueryRules(sparkSession: SparkSession) extends Rule
         updatedExp
       case alias@Alias(exp: Expression, name) =>
         val newAlias = Seq(alias)
-        val attr = AttributeReference(name,
-            alias.dataType,
-            alias.nullable,
-            alias.metadata) (alias.exprId, alias.qualifier, alias.isGenerated)
+        val attr = CarbonToSparkAdapater.createAttributeReference(name,
+          alias.dataType,
+          alias.nullable,
+          alias.metadata,
+          alias.exprId,
+          alias.qualifier,
+          alias)
         factPlanGrpExpForStreaming.put(
           AggExpToColumnMappingModel(
             removeQualifiers(PreAggregateUtil.normalizeExprId(exp, plan.allAttributes))),
@@ -1093,11 +1115,14 @@ case class CarbonPreAggregateQueryRules(sparkSession: SparkSession) extends Rule
   private def removeQualifiers(expression: Expression) : Expression = {
     expression.transform {
       case attr: AttributeReference =>
-        AttributeReference(
+        CarbonToSparkAdapater.createAttributeReference(
           attr.name,
           attr.dataType,
           attr.nullable,
-          attr.metadata)(attr.exprId, None, attr.isGenerated)
+          attr.metadata,
+          attr.exprId,
+          None,
+          attr)
     }
   }
 
@@ -1363,10 +1388,13 @@ case class CarbonPreAggregateQueryRules(sparkSession: SparkSession) extends Rule
           attr,
           attributes)
         val newExpressionId = NamedExpression.newExprId
-        val childTableAttr = AttributeReference(attr.name,
+        val childTableAttr = CarbonToSparkAdapater.createAttributeReference(attr.name,
           childAttr.dataType,
           childAttr.nullable,
-          childAttr.metadata)(newExpressionId, childAttr.qualifier, attr.isGenerated)
+          childAttr.metadata,
+          newExpressionId,
+          childAttr.qualifier,
+          attr)
         updatedExpression.put(attr, childTableAttr)
         // returning the alias to show proper column name in output
         Seq(Alias(childAttr,
@@ -1378,12 +1406,20 @@ case class CarbonPreAggregateQueryRules(sparkSession: SparkSession) extends Rule
           attr,
           attributes)
         val newExpressionId = NamedExpression.newExprId
-        val parentTableAttr = AttributeReference(name,
+        val parentTableAttr = CarbonToSparkAdapater.createAttributeReference(name,
           alias.dataType,
-          alias.nullable) (alias.exprId, alias.qualifier, alias.isGenerated)
-        val childTableAttr = AttributeReference(name,
+          alias.nullable,
+          Metadata.empty,
+          alias.exprId,
+          alias.qualifier,
+          alias)
+        val childTableAttr = CarbonToSparkAdapater.createAttributeReference(name,
           alias.dataType,
-          alias.nullable) (newExpressionId, alias.qualifier, alias.isGenerated)
+          alias.nullable,
+          Metadata.empty,
+          newExpressionId,
+          alias.qualifier,
+          alias)
         updatedExpression.put(parentTableAttr, childTableAttr)
         // returning alias with child attribute reference
         Seq(Alias(childAttr,
@@ -1409,13 +1445,21 @@ case class CarbonPreAggregateQueryRules(sparkSession: SparkSession) extends Rule
           val newExpressionId = NamedExpression.newExprId
           // create a parent attribute reference which will be replced on node which may be referred
           // by node like sort join
-          val parentTableAttr = AttributeReference(name,
+          val parentTableAttr = CarbonToSparkAdapater.createAttributeReference(name,
             alias.dataType,
-            alias.nullable)(alias.exprId, alias.qualifier, alias.isGenerated)
+            alias.nullable,
+            Metadata.empty,
+            alias.exprId,
+            alias.qualifier,
+            alias)
           // creating a child attribute reference which will be replced
-          val childTableAttr = AttributeReference(name,
+          val childTableAttr = CarbonToSparkAdapater.createAttributeReference(name,
             alias.dataType,
-            alias.nullable)(newExpressionId, alias.qualifier, alias.isGenerated)
+            alias.nullable,
+            Metadata.empty,
+            newExpressionId,
+            alias.qualifier,
+            alias)
           // adding to map, will be used during other node updation like sort, join, project
           updatedExpression.put(parentTableAttr, childTableAttr)
           // returning alias with child attribute reference
@@ -1426,12 +1470,12 @@ case class CarbonPreAggregateQueryRules(sparkSession: SparkSession) extends Rule
           // for streaming table
           // create alias for aggregate table
           val aggExpForStreaming = aggExp.map{ exp =>
-            Alias(exp,
-              name)(
+            CarbonToSparkAdapater.createAliasRef(exp,
+              name,
               NamedExpression.newExprId,
               alias.qualifier,
               Some(alias.metadata),
-              alias.isGenerated).asInstanceOf[NamedExpression]
+              Some(alias)).asInstanceOf[NamedExpression]
           }
           aggExpForStreaming
         }
@@ -1460,12 +1504,20 @@ case class CarbonPreAggregateQueryRules(sparkSession: SparkSession) extends Rule
             }
           }
         val newExpressionId = NamedExpression.newExprId
-        val parentTableAttr = AttributeReference(name,
+        val parentTableAttr = CarbonToSparkAdapater.createAttributeReference(name,
           alias.dataType,
-          alias.nullable) (alias.exprId, alias.qualifier, alias.isGenerated)
-        val childTableAttr = AttributeReference(name,
+          alias.nullable,
+          Metadata.empty,
+          alias.exprId,
+          alias.qualifier,
+          alias)
+        val childTableAttr = CarbonToSparkAdapater.createAttributeReference(name,
           alias.dataType,
-          alias.nullable) (newExpressionId, alias.qualifier, alias.isGenerated)
+          alias.nullable,
+          Metadata.empty,
+          newExpressionId,
+          alias.qualifier,
+          alias)
         updatedExpression.put(parentTableAttr, childTableAttr)
         Seq(Alias(updatedExp, name)(newExpressionId,
           alias.qualifier).asInstanceOf[NamedExpression])
@@ -1787,20 +1839,23 @@ case class CarbonPreAggregateDataLoadingRules(sparkSession: SparkSession)
               // named expression list otherwise update the list and add it to set
               if (!validExpressionsMap.contains(AggExpToColumnMappingModel(sumExp))) {
                 namedExpressionList +=
-                Alias(expressions.head, name + "_ sum")(NamedExpression.newExprId,
+                CarbonToSparkAdapater.createAliasRef(expressions.head,
+                  name + "_ sum",
+                  NamedExpression.newExprId,
                   alias.qualifier,
                   Some(alias.metadata),
-                  alias.isGenerated)
+                  Some(alias))
                 validExpressionsMap += AggExpToColumnMappingModel(sumExp)
               }
               // check with same expression already count is present then do not add to
               // named expression list otherwise update the list and add it to set
               if (!validExpressionsMap.contains(AggExpToColumnMappingModel(countExp))) {
                 namedExpressionList +=
-                Alias(expressions.last, name + "_ count")(NamedExpression.newExprId,
-                  alias.qualifier,
-                  Some(alias.metadata),
-                  alias.isGenerated)
+                CarbonToSparkAdapater.createAliasRef(expressions.last, name + "_ count",
+                    NamedExpression.newExprId,
+                    alias.qualifier,
+                    Some(alias.metadata),
+                    Some(alias))
                 validExpressionsMap += AggExpToColumnMappingModel(countExp)
               }
             } else {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/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 450902a..b96b6a7 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
@@ -32,6 +32,7 @@ import org.apache.spark.sql.CarbonExpressions.{MatchCast => Cast}
 import org.apache.spark.sql.carbondata.execution.datasources.CarbonSparkDataSourceUtil
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.hive.CarbonSessionCatalog
+import org.apache.spark.util.{CarbonReflectionUtils, SparkUtil}
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datamap.Segment
@@ -49,6 +50,7 @@ import org.apache.carbondata.datamap.{TextMatch, TextMatchLimit}
 import org.apache.carbondata.spark.CarbonAliasDecoderRelation
 
 
+
 /**
  * All filter conversions are done here.
  */
@@ -297,6 +299,27 @@ object CarbonFilters {
     filters.flatMap(translate(_, false)).toArray
   }
 
+  /**
+   * This API checks whether StringTrim object is compatible with
+   * carbon,carbon only deals with the space any other symbol should
+   * be ignored.So condition is SPARK version < 2.3.
+   * If it is 2.3 then trimStr field should be empty
+   *
+   * @param stringTrim
+   * @return
+   */
+  def isStringTrimCompatibleWithCarbon(stringTrim: StringTrim): Boolean = {
+    var isCompatible = true
+    if (SparkUtil.isSparkVersionXandAbove("2.3")) {
+      val trimStr = CarbonReflectionUtils.getField("trimStr", stringTrim)
+        .asInstanceOf[Option[Expression]]
+      if (trimStr.isDefined) {
+        isCompatible = false
+      }
+    }
+    isCompatible
+  }
+
   def transformExpression(expr: Expression): CarbonExpression = {
     expr match {
       case Or(left, right)
@@ -385,7 +408,8 @@ object CarbonFilters {
           new CarbonLiteralExpression(maxValueLimit,
             CarbonSparkDataSourceUtil.convertSparkToCarbonDataType(dataType)))
         new AndExpression(l, r)
-      case StringTrim(child) => transformExpression(child)
+      case strTrim: StringTrim if isStringTrimCompatibleWithCarbon(strTrim) =>
+        transformExpression(strTrim)
       case s: ScalaUDF =>
         new MatchExpression(s.children.head.toString())
       case _ =>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonLateDecodeRule.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonLateDecodeRule.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonLateDecodeRule.scala
index b81d0a1..48c6377 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonLateDecodeRule.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonLateDecodeRule.scala
@@ -786,8 +786,12 @@ class CarbonLateDecodeRule extends Rule[LogicalPlan] with PredicateHelper {
           p.transformAllExpressions {
             case a@Alias(exp, _)
               if !exp.deterministic && !exp.isInstanceOf[CustomDeterministicExpression] =>
-              Alias(CustomDeterministicExpression(exp), a.name)(a.exprId, a.qualifier,
-                a.explicitMetadata, a.isGenerated)
+              CarbonToSparkAdapater.createAliasRef(CustomDeterministicExpression(exp),
+                a.name,
+                a.exprId,
+                a.qualifier,
+                a.explicitMetadata,
+                Some(a))
             case exp: NamedExpression
               if !exp.deterministic && !exp.isInstanceOf[CustomDeterministicExpression] =>
               CustomDeterministicExpression(exp)
@@ -800,8 +804,12 @@ class CarbonLateDecodeRule extends Rule[LogicalPlan] with PredicateHelper {
           f.transformAllExpressions {
             case a@Alias(exp, _)
               if !exp.deterministic && !exp.isInstanceOf[CustomDeterministicExpression] =>
-              Alias(CustomDeterministicExpression(exp), a.name)(a.exprId, a.qualifier,
-                a.explicitMetadata, a.isGenerated)
+              CarbonToSparkAdapater.createAliasRef(CustomDeterministicExpression(exp),
+                a.name,
+                a.exprId,
+                a.qualifier,
+                a.explicitMetadata,
+                Some(a))
             case exp: NamedExpression
               if !exp.deterministic && !exp.isInstanceOf[CustomDeterministicExpression] =>
               CustomDeterministicExpression(exp)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
index 8eb47fc..1622724 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.parser
 import scala.collection.mutable
 import scala.language.implicitConversions
 
-import org.apache.spark.sql.{CarbonEnv, DeleteRecords, UpdateTable}
+import org.apache.spark.sql.{CarbonToSparkAdapater, DeleteRecords, UpdateTable}
 import org.apache.spark.sql.catalyst.{CarbonDDLSqlParser, TableIdentifier}
 import org.apache.spark.sql.catalyst.CarbonTableIdentifierImplicit._
 import org.apache.spark.sql.catalyst.plans.logical._
@@ -479,7 +479,7 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
         logicalPlan match {
           case _: CarbonCreateTableCommand =>
             ExplainCommand(logicalPlan, extended = isExtended.isDefined)
-          case _ => ExplainCommand(OneRowRelation)
+          case _ => CarbonToSparkAdapater.getExplainCommandObj
         }
     }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/spark2.1/org/apache/spark/sql/CarbonToSparkAdapater.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/spark2.1/org/apache/spark/sql/CarbonToSparkAdapater.scala b/integration/spark2/src/main/spark2.1/org/apache/spark/sql/CarbonToSparkAdapater.scala
new file mode 100644
index 0000000..d5fe6a4
--- /dev/null
+++ b/integration/spark2/src/main/spark2.1/org/apache/spark/sql/CarbonToSparkAdapater.scala
@@ -0,0 +1,75 @@
+/*
+ * 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.spark.sql
+
+import org.apache.spark.SparkContext
+import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
+import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, AttributeSet, ExprId, Expression, ExpressionSet, NamedExpression, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext
+import org.apache.spark.sql.catalyst.plans.logical.OneRowRelation
+import org.apache.spark.sql.execution.command.ExplainCommand
+import org.apache.spark.sql.types.{DataType, Metadata}
+
+object CarbonToSparkAdapater {
+
+  def addSparkListener(sparkContext: SparkContext) = {
+    sparkContext.addSparkListener(new SparkListener {
+      override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = {
+        SparkSession.setDefaultSession(null)
+        SparkSession.sqlListener.set(null)
+      }
+    })
+  }
+
+  def createAttributeReference(name: String, dataType: DataType, nullable: Boolean,
+                               metadata: Metadata,exprId: ExprId, qualifier: Option[String],
+                               attrRef : NamedExpression): AttributeReference = {
+    AttributeReference(
+      name,
+      dataType,
+      nullable,
+      metadata)(exprId, qualifier,attrRef.isGenerated)
+  }
+
+  def createAliasRef(child: Expression,
+                     name: String,
+                     exprId: ExprId = NamedExpression.newExprId,
+                     qualifier: Option[String] = None,
+                     explicitMetadata: Option[Metadata] = None,
+                     namedExpr: Option[NamedExpression] = None): Alias = {
+    val isGenerated:Boolean = if (namedExpr.isDefined) {
+      namedExpr.get.isGenerated
+    } else {
+      false
+    }
+    Alias(child, name)(exprId, qualifier, explicitMetadata,isGenerated)
+  }
+
+  def getExplainCommandObj() : ExplainCommand = {
+    ExplainCommand(OneRowRelation)
+  }
+
+  def getPartitionKeyFilter(
+      partitionSet: AttributeSet,
+      filterPredicates: Seq[Expression]): ExpressionSet = {
+    ExpressionSet(
+      ExpressionSet(filterPredicates)
+        .filter(_.references.subsetOf(partitionSet)))
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/spark2.1/org/apache/spark/sql/CarbonVectorProxy.java
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/spark2.1/org/apache/spark/sql/CarbonVectorProxy.java b/integration/spark2/src/main/spark2.1/org/apache/spark/sql/CarbonVectorProxy.java
deleted file mode 100644
index 7fa01e9..0000000
--- a/integration/spark2/src/main/spark2.1/org/apache/spark/sql/CarbonVectorProxy.java
+++ /dev/null
@@ -1,226 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.spark.sql;
-
-import java.math.BigInteger;
-
-import org.apache.spark.memory.MemoryMode;
-import org.apache.spark.sql.catalyst.InternalRow;
-import org.apache.spark.sql.execution.vectorized.ColumnarBatch;
-import org.apache.spark.sql.types.CalendarIntervalType;
-import org.apache.spark.sql.types.DataType;
-import org.apache.spark.sql.types.Decimal;
-import org.apache.spark.sql.types.DecimalType;
-import org.apache.spark.sql.types.StructField;
-import org.apache.spark.sql.types.StructType;
-import org.apache.spark.unsafe.types.CalendarInterval;
-import org.apache.spark.unsafe.types.UTF8String;
-
-/**
- * Adapter class which handles the columnar vector reading of the carbondata
- * based on the spark ColumnVector and ColumnarBatch API. This proxy class
- * handles the complexity of spark 2.1 version related api changes since
- * spark ColumnVector and ColumnarBatch interfaces are still evolving.
- */
-public class CarbonVectorProxy {
-
-    private ColumnarBatch columnarBatch;
-
-    /**
-     * Adapter class which handles the columnar vector reading of the carbondata
-     * based on the spark ColumnVector and ColumnarBatch API. This proxy class
-     * handles the complexity of spark 2.3 version related api changes since
-     * spark ColumnVector and ColumnarBatch interfaces are still evolving.
-     *
-     * @param memMode       which represent the type onheap or offheap vector.
-     * @param rowNum        rows number for vector reading
-     * @param structFileds, metadata related to current schema of table.
-     */
-    public CarbonVectorProxy(MemoryMode memMode, int rowNum, StructField[] structFileds) {
-        columnarBatch = ColumnarBatch.allocate(new StructType(structFileds), memMode, rowNum);
-    }
-
-    public CarbonVectorProxy(MemoryMode memMode, StructType outputSchema, int rowNum) {
-        columnarBatch = ColumnarBatch.allocate(outputSchema, memMode, rowNum);
-    }
-
-    /**
-     * Sets the number of rows in this batch.
-     */
-    public void setNumRows(int numRows) {
-        columnarBatch.setNumRows(numRows);
-    }
-
-    /**
-     * Returns the number of rows for read, including filtered rows.
-     */
-    public int numRows() {
-        return columnarBatch.capacity();
-    }
-
-    /**
-     * Called to close all the columns in this batch. It is not valid to access the data after
-     * calling this. This must be called at the end to clean up memory allocations.
-     */
-    public void close() {
-        columnarBatch.close();
-    }
-
-    /**
-     * Returns the row in this batch at `rowId`. Returned row is reused across calls.
-     */
-    public InternalRow getRow(int rowId) {
-        return columnarBatch.getRow(rowId);
-    }
-
-    /**
-     * Returns the row in this batch at `rowId`. Returned row is reused across calls.
-     */
-    public Object getColumnarBatch() {
-        return columnarBatch;
-    }
-
-    public void resetDictionaryIds(int ordinal) {
-        columnarBatch.column(ordinal).getDictionaryIds().reset();
-    }
-
-    /**
-     * Resets this column for writing. The currently stored values are no longer accessible.
-     */
-    public void reset() {
-        columnarBatch.reset();
-    }
-
-    public void putRowToColumnBatch(int rowId, Object value, int offset) {
-        org.apache.spark.sql.types.DataType t = dataType(offset);
-        if (null == value) {
-            putNull(rowId, offset);
-        } else {
-            if (t == org.apache.spark.sql.types.DataTypes.BooleanType) {
-                putBoolean(rowId, (boolean) value, offset);
-            } else if (t == org.apache.spark.sql.types.DataTypes.ByteType) {
-                putByte(rowId, (byte) value, offset);
-            } else if (t == org.apache.spark.sql.types.DataTypes.ShortType) {
-                putShort(rowId, (short) value, offset);
-            } else if (t == org.apache.spark.sql.types.DataTypes.IntegerType) {
-                putInt(rowId, (int) value, offset);
-            } else if (t == org.apache.spark.sql.types.DataTypes.LongType) {
-                putLong(rowId, (long) value, offset);
-            } else if (t == org.apache.spark.sql.types.DataTypes.FloatType) {
-                putFloat(rowId, (float) value, offset);
-            } else if (t == org.apache.spark.sql.types.DataTypes.DoubleType) {
-                putDouble(rowId, (double) value, offset);
-            } else if (t == org.apache.spark.sql.types.DataTypes.StringType) {
-                UTF8String v = (UTF8String) value;
-                putByteArray(rowId, v.getBytes(), offset);
-            } else if (t instanceof org.apache.spark.sql.types.DecimalType) {
-                DecimalType dt = (DecimalType) t;
-                Decimal d = Decimal.fromDecimal(value);
-                if (dt.precision() <= Decimal.MAX_INT_DIGITS()) {
-                    putInt(rowId, (int) d.toUnscaledLong(), offset);
-                } else if (dt.precision() <= Decimal.MAX_LONG_DIGITS()) {
-                    putLong(rowId, d.toUnscaledLong(), offset);
-                } else {
-                    final BigInteger integer = d.toJavaBigDecimal().unscaledValue();
-                    byte[] bytes = integer.toByteArray();
-                    putByteArray(rowId, bytes, 0, bytes.length, offset);
-                }
-            } else if (t instanceof CalendarIntervalType) {
-                CalendarInterval c = (CalendarInterval) value;
-                columnarBatch.column(offset).getChildColumn(0).putInt(rowId, c.months);
-                columnarBatch.column(offset).getChildColumn(1).putLong(rowId, c.microseconds);
-            } else if (t instanceof org.apache.spark.sql.types.DateType) {
-                putInt(rowId, (int) value, offset);
-            } else if (t instanceof org.apache.spark.sql.types.TimestampType) {
-                putLong(rowId, (long) value, offset);
-            }
-        }
-    }
-
-    public void putBoolean(int rowId, boolean value, int ordinal) {
-        columnarBatch.column(ordinal).putBoolean(rowId, (boolean) value);
-    }
-
-    public void putByte(int rowId, byte value, int ordinal) {
-        columnarBatch.column(ordinal).putByte(rowId, (byte) value);
-    }
-
-    public void putShort(int rowId, short value, int ordinal) {
-        columnarBatch.column(ordinal).putShort(rowId, (short) value);
-    }
-
-    public void putInt(int rowId, int value, int ordinal) {
-        columnarBatch.column(ordinal).putInt(rowId, (int) value);
-    }
-
-    public void putFloat(int rowId, float value, int ordinal) {
-        columnarBatch.column(ordinal).putFloat(rowId, (float) value);
-    }
-
-    public void putLong(int rowId, long value, int ordinal) {
-        columnarBatch.column(ordinal).putLong(rowId, (long) value);
-    }
-
-    public void putDouble(int rowId, double value, int ordinal) {
-        columnarBatch.column(ordinal).putDouble(rowId, (double) value);
-    }
-
-    public void putByteArray(int rowId, byte[] value, int ordinal) {
-        columnarBatch.column(ordinal).putByteArray(rowId, (byte[]) value);
-    }
-
-    public void putInts(int rowId, int count, int value, int ordinal) {
-        columnarBatch.column(ordinal).putInts(rowId, count, value);
-    }
-
-    public void putShorts(int rowId, int count, short value, int ordinal) {
-        columnarBatch.column(ordinal).putShorts(rowId, count, value);
-    }
-
-    public void putLongs(int rowId, int count, long value, int ordinal) {
-        columnarBatch.column(ordinal).putLongs(rowId, count, value);
-    }
-
-    public void putDecimal(int rowId, Decimal value, int precision, int ordinal) {
-        columnarBatch.column(ordinal).putDecimal(rowId, value, precision);
-
-    }
-
-    public void putDoubles(int rowId, int count, double value, int ordinal) {
-        columnarBatch.column(ordinal).putDoubles(rowId, count, value);
-    }
-
-    public void putByteArray(int rowId, byte[] value, int offset, int length, int ordinal) {
-        columnarBatch.column(ordinal).putByteArray(rowId, (byte[]) value, offset, length);
-    }
-
-    public void putNull(int rowId, int ordinal) {
-        columnarBatch.column(ordinal).putNull(rowId);
-    }
-
-    public void putNulls(int rowId, int count, int ordinal) {
-        columnarBatch.column(ordinal).putNulls(rowId, count);
-    }
-
-    public boolean isNullAt(int rowId, int ordinal) {
-        return columnarBatch.column(ordinal).isNullAt(rowId);
-    }
-
-    public DataType dataType(int ordinal) {
-        return columnarBatch.column(ordinal).dataType();
-    }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/spark2.1/org/apache/spark/sql/hive/CreateCarbonSourceTableAsSelectCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/spark2.1/org/apache/spark/sql/hive/CreateCarbonSourceTableAsSelectCommand.scala b/integration/spark2/src/main/spark2.1/org/apache/spark/sql/hive/CreateCarbonSourceTableAsSelectCommand.scala
index 989b1d5..dd690e4 100644
--- a/integration/spark2/src/main/spark2.1/org/apache/spark/sql/hive/CreateCarbonSourceTableAsSelectCommand.scala
+++ b/integration/spark2/src/main/spark2.1/org/apache/spark/sql/hive/CreateCarbonSourceTableAsSelectCommand.scala
@@ -34,11 +34,9 @@ import org.apache.spark.sql.types.StructType
 
 /**
  * Create table 'using carbondata' and insert the query result into it.
- *
  * @param table the Catalog Table
  * @param mode  SaveMode:Ignore,OverWrite,ErrorIfExists,Append
  * @param query the query whose result will be insert into the new relation
- *
  */
 
 case class CreateCarbonSourceTableAsSelectCommand(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/spark2.2/org/apache/spark/sql/CarbonToSparkAdapater.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/spark2.2/org/apache/spark/sql/CarbonToSparkAdapater.scala b/integration/spark2/src/main/spark2.2/org/apache/spark/sql/CarbonToSparkAdapater.scala
new file mode 100644
index 0000000..7a68e3e
--- /dev/null
+++ b/integration/spark2/src/main/spark2.2/org/apache/spark/sql/CarbonToSparkAdapater.scala
@@ -0,0 +1,82 @@
+/*
+ * 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.spark.sql
+
+import org.apache.spark.SparkContext
+import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
+import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, AttributeSet, ExprId, Expression, ExpressionSet, NamedExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext
+import org.apache.spark.sql.catalyst.optimizer.OptimizeCodegen
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.command.ExplainCommand
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{DataType, Metadata}
+
+object CarbonToSparkAdapater {
+
+  def addSparkListener(sparkContext: SparkContext) = {
+    sparkContext.addSparkListener(new SparkListener {
+      override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = {
+        SparkSession.setDefaultSession(null)
+        SparkSession.sqlListener.set(null)
+      }
+    })
+  }
+
+  def createAttributeReference(name: String, dataType: DataType, nullable: Boolean,
+                               metadata: Metadata,exprId: ExprId, qualifier: Option[String],
+                               attrRef : NamedExpression): AttributeReference = {
+    AttributeReference(
+      name,
+      dataType,
+      nullable,
+      metadata)(exprId, qualifier,attrRef.isGenerated)
+  }
+
+  def createAliasRef(child: Expression,
+                     name: String,
+                     exprId: ExprId = NamedExpression.newExprId,
+                     qualifier: Option[String] = None,
+                     explicitMetadata: Option[Metadata] = None,
+                     namedExpr: Option[NamedExpression] = None): Alias = {
+    val isGenerated:Boolean = if (namedExpr.isDefined) {
+      namedExpr.get.isGenerated
+    } else {
+      false
+    }
+    Alias(child, name)(exprId, qualifier, explicitMetadata,isGenerated)
+  }
+
+  def getExplainCommandObj() : ExplainCommand = {
+    ExplainCommand(OneRowRelation)
+  }
+
+  def getPartitionKeyFilter(
+      partitionSet: AttributeSet,
+      filterPredicates: Seq[Expression]): ExpressionSet = {
+    ExpressionSet(
+      ExpressionSet(filterPredicates)
+        .filter(_.references.subsetOf(partitionSet)))
+  }
+
+  def getOptimizeCodegenRule(conf :SQLConf): Seq[Rule[LogicalPlan]] = {
+    Seq(OptimizeCodegen(conf))
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/spark2.2/org/apache/spark/sql/CarbonVectorProxy.java
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/spark2.2/org/apache/spark/sql/CarbonVectorProxy.java b/integration/spark2/src/main/spark2.2/org/apache/spark/sql/CarbonVectorProxy.java
deleted file mode 100644
index 944b32e..0000000
--- a/integration/spark2/src/main/spark2.2/org/apache/spark/sql/CarbonVectorProxy.java
+++ /dev/null
@@ -1,229 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.spark.sql;
-
-import java.math.BigInteger;
-
-import org.apache.spark.memory.MemoryMode;
-import org.apache.spark.sql.catalyst.InternalRow;
-import org.apache.spark.sql.execution.vectorized.ColumnarBatch;
-import org.apache.spark.sql.types.CalendarIntervalType;
-import org.apache.spark.sql.types.DataType;
-import org.apache.spark.sql.types.Decimal;
-import org.apache.spark.sql.types.DecimalType;
-import org.apache.spark.sql.types.StructField;
-import org.apache.spark.sql.types.StructType;
-import org.apache.spark.unsafe.types.CalendarInterval;
-import org.apache.spark.unsafe.types.UTF8String;
-
-/**
- * Adapter class which handles the columnar vector reading of the carbondata
- * based on the spark ColumnVector and ColumnarBatch API. This proxy class
- * handles the complexity of spark 2.2 version related api changes since
- * spark ColumnVector and ColumnarBatch interfaces are still evolving.
- */
-public class CarbonVectorProxy {
-
-    private ColumnarBatch columnarBatch;
-
-    /**
-     * Adapter class which handles the columnar vector reading of the carbondata
-     * based on the spark ColumnVector and ColumnarBatch API. This proxy class
-     * handles the complexity of spark 2.3 version related api changes since
-     * spark ColumnVector and ColumnarBatch interfaces are still evolving.
-     *
-     * @param memMode       which represent the type onheap or offheap vector.
-     * @param rowNum        rows number for vector reading
-     * @param structFileds, metadata related to current schema of table.
-     */
-    public CarbonVectorProxy(MemoryMode memMode, int rowNum, StructField[] structFileds) {
-        columnarBatch = ColumnarBatch.allocate(new StructType(structFileds), memMode, rowNum);
-    }
-
-    public CarbonVectorProxy(MemoryMode memMode, StructType outputSchema, int rowNum) {
-        columnarBatch = ColumnarBatch.allocate(outputSchema, memMode, rowNum);
-    }
-
-    /**
-     * Sets the number of rows in this batch.
-     */
-    public void setNumRows(int numRows) {
-        columnarBatch.setNumRows(numRows);
-    }
-
-    /**
-     * Returns the number of rows for read, including filtered rows.
-     */
-    public int numRows() {
-        return columnarBatch.capacity();
-    }
-
-    /**
-     * Called to close all the columns in this batch. It is not valid to access the data after
-     * calling this. This must be called at the end to clean up memory allocations.
-     */
-    public void close() {
-        columnarBatch.close();
-    }
-
-    /**
-     * Returns the row in this batch at `rowId`. Returned row is reused across calls.
-     */
-    public InternalRow getRow(int rowId) {
-        return columnarBatch.getRow(rowId);
-    }
-
-    /**
-     * Returns the row in this batch at `rowId`. Returned row is reused across calls.
-     */
-    public Object getColumnarBatch() {
-        return columnarBatch;
-    }
-
-    public Object reserveDictionaryIds(int capacity , int dummyOrdinal) {
-        return columnarBatch.column(ordinal).reserveDictionaryIds(capacity);
-    }
-
-    public void resetDictionaryIds(int ordinal) {
-        columnarBatch.column(ordinal).getDictionaryIds().reset();
-    }
-
-    /**
-     * Resets this column for writing. The currently stored values are no longer accessible.
-     */
-    public void reset() {
-        columnarBatch.reset();
-    }
-
-    public void putRowToColumnBatch(int rowId, Object value, int offset) {
-        org.apache.spark.sql.types.DataType t = dataType(offset);
-        if (null == value) {
-            putNull(rowId, offset);
-        } else {
-            if (t == org.apache.spark.sql.types.DataTypes.BooleanType) {
-                putBoolean(rowId, (boolean) value, offset);
-            } else if (t == org.apache.spark.sql.types.DataTypes.ByteType) {
-                putByte(rowId, (byte) value, offset);
-            } else if (t == org.apache.spark.sql.types.DataTypes.ShortType) {
-                putShort(rowId, (short) value, offset);
-            } else if (t == org.apache.spark.sql.types.DataTypes.IntegerType) {
-                putInt(rowId, (int) value, offset);
-            } else if (t == org.apache.spark.sql.types.DataTypes.LongType) {
-                putLong(rowId, (long) value, offset);
-            } else if (t == org.apache.spark.sql.types.DataTypes.FloatType) {
-                putFloat(rowId, (float) value, offset);
-            } else if (t == org.apache.spark.sql.types.DataTypes.DoubleType) {
-                putDouble(rowId, (double) value, offset);
-            } else if (t == org.apache.spark.sql.types.DataTypes.StringType) {
-                UTF8String v = (UTF8String) value;
-                putByteArray(rowId, v.getBytes(), offset);
-            } else if (t instanceof org.apache.spark.sql.types.DecimalType) {
-                DecimalType dt = (DecimalType) t;
-                Decimal d = Decimal.fromDecimal(value);
-                if (dt.precision() <= Decimal.MAX_INT_DIGITS()) {
-                    putInt(rowId, (int) d.toUnscaledLong(), offset);
-                } else if (dt.precision() <= Decimal.MAX_LONG_DIGITS()) {
-                    putLong(rowId, d.toUnscaledLong(), offset);
-                } else {
-                    final BigInteger integer = d.toJavaBigDecimal().unscaledValue();
-                    byte[] bytes = integer.toByteArray();
-                    putByteArray(rowId, bytes, 0, bytes.length, offset);
-                }
-            } else if (t instanceof CalendarIntervalType) {
-                CalendarInterval c = (CalendarInterval) value;
-                columnarBatch.column(offset).getChildColumn(0).putInt(rowId, c.months);
-                columnarBatch.column(offset).getChildColumn(1).putLong(rowId, c.microseconds);
-            } else if (t instanceof org.apache.spark.sql.types.DateType) {
-                putInt(rowId, (int) value, offset);
-            } else if (t instanceof org.apache.spark.sql.types.TimestampType) {
-                putLong(rowId, (long) value, offset);
-            }
-        }
-    }
-
-    public void putBoolean(int rowId, boolean value, int ordinal) {
-        columnarBatch.column(ordinal).putBoolean(rowId, (boolean) value);
-    }
-
-    public void putByte(int rowId, byte value, int ordinal) {
-        columnarBatch.column(ordinal).putByte(rowId, (byte) value);
-    }
-
-    public void putShort(int rowId, short value, int ordinal) {
-        columnarBatch.column(ordinal).putShort(rowId, (short) value);
-    }
-
-    public void putInt(int rowId, int value, int ordinal) {
-        columnarBatch.column(ordinal).putInt(rowId, (int) value);
-    }
-
-    public void putFloat(int rowId, float value, int ordinal) {
-        columnarBatch.column(ordinal).putFloat(rowId, (float) value);
-    }
-
-    public void putLong(int rowId, long value, int ordinal) {
-        columnarBatch.column(ordinal).putLong(rowId, (long) value);
-    }
-
-    public void putDouble(int rowId, double value, int ordinal) {
-        columnarBatch.column(ordinal).putDouble(rowId, (double) value);
-    }
-
-    public void putByteArray(int rowId, byte[] value, int ordinal) {
-        columnarBatch.column(ordinal).putByteArray(rowId, (byte[]) value);
-    }
-
-    public void putInts(int rowId, int count, int value, int ordinal) {
-        columnarBatch.column(ordinal).putInts(rowId, count, value);
-    }
-
-    public void putShorts(int rowId, int count, short value, int ordinal) {
-        columnarBatch.column(ordinal).putShorts(rowId, count, value);
-    }
-
-    public void putLongs(int rowId, int count, long value, int ordinal) {
-        columnarBatch.column(ordinal).putLongs(rowId, count, value);
-    }
-
-    public void putDecimal(int rowId, Decimal value, int precision, int ordinal) {
-        columnarBatch.column(ordinal).putDecimal(rowId, value, precision);
-    }
-
-    public void putDoubles(int rowId, int count, double value, int ordinal) {
-        columnarBatch.column(ordinal).putDoubles(rowId, count, value);
-    }
-
-    public void putByteArray(int rowId, byte[] value, int offset, int length, int ordinal) {
-        columnarBatch.column(ordinal).putByteArray(rowId, (byte[]) value, offset, length);
-    }
-
-    public void putNull(int rowId, int ordinal) {
-        columnarBatch.column(ordinal).putNull(rowId);
-    }
-
-    public void putNulls(int rowId, int count, int ordinal) {
-        columnarBatch.column(ordinal).putNulls(rowId, count);
-    }
-
-    public boolean isNullAt(int rowId, int ordinal) {
-        return columnarBatch.column(ordinal).isNullAt(rowId);
-    }
-
-    public DataType dataType(int ordinal) {
-        return columnarBatch.column(ordinal).dataType();
-    }
-}


[6/8] carbondata git commit: [CARBONDATA-2532][Integration] Carbon to support spark 2.3.1 version(Make API changes in carbon to be compatible with spark 2.3)

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark-datasource/src/main/scala/org/apache/spark/util/SparkUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-datasource/src/main/scala/org/apache/spark/util/SparkUtil.scala b/integration/spark-datasource/src/main/scala/org/apache/spark/util/SparkUtil.scala
new file mode 100644
index 0000000..0b3d35b
--- /dev/null
+++ b/integration/spark-datasource/src/main/scala/org/apache/spark/util/SparkUtil.scala
@@ -0,0 +1,60 @@
+/*
+ * 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.spark.util
+
+import org.apache.spark.{SPARK_VERSION, TaskContext}
+
+/*
+ * this object use to handle file splits
+ */
+object SparkUtil {
+
+  def setTaskContext(context: TaskContext): Unit = {
+    val localThreadContext = TaskContext.get()
+    if (localThreadContext == null) {
+      TaskContext.setTaskContext(context)
+    }
+  }
+
+  /**
+   * Utility method to compare the Spark Versions.
+   * This API ignores the sub-version and compares with only major version
+   * Version passed should be of format x.y  e.g 2.2 ,2.3 , SPARK_VERSION
+   * will be of format x.y.z e.g 2.3.0,2.2.1
+   */
+  def isSparkVersionXandAbove(xVersion: String, isEqualComparision: Boolean = false): Boolean = {
+    val tmpArray = SPARK_VERSION.split("\\.")
+    // convert to float
+    val sparkVersion = if (tmpArray.length >= 2) {
+      (tmpArray(0) + "." + tmpArray(1)).toFloat
+    } else {
+      (tmpArray(0) + ".0").toFloat
+    }
+    // compare the versions
+    if (isEqualComparision) {
+      sparkVersion == xVersion.toFloat
+    } else {
+      sparkVersion >= xVersion.toFloat
+    }
+  }
+
+  def isSparkVersionEqualTo(xVersion: String): Boolean = {
+    isSparkVersionXandAbove(xVersion, true)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark-datasource/src/main/spark2.1andspark2.2/org/apache/spark/sql/CarbonVectorProxy.java
----------------------------------------------------------------------
diff --git a/integration/spark-datasource/src/main/spark2.1andspark2.2/org/apache/spark/sql/CarbonVectorProxy.java b/integration/spark-datasource/src/main/spark2.1andspark2.2/org/apache/spark/sql/CarbonVectorProxy.java
new file mode 100644
index 0000000..f39bc93
--- /dev/null
+++ b/integration/spark-datasource/src/main/spark2.1andspark2.2/org/apache/spark/sql/CarbonVectorProxy.java
@@ -0,0 +1,272 @@
+/*
+ * 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.spark.sql;
+
+import java.math.BigInteger;
+
+import org.apache.parquet.column.Dictionary;
+import org.apache.spark.memory.MemoryMode;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.execution.vectorized.ColumnarBatch;
+import org.apache.spark.sql.execution.vectorized.ColumnVector;
+import org.apache.spark.sql.types.CalendarIntervalType;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.Decimal;
+import org.apache.spark.sql.types.DecimalType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.CalendarInterval;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/**
+ * Adapter class which handles the columnar vector reading of the carbondata
+ * based on the spark ColumnVector and ColumnarBatch API. This proxy class
+ * handles the complexity of spark 2.1 version related api changes since
+ * spark ColumnVector and ColumnarBatch interfaces are still evolving.
+ */
+public class CarbonVectorProxy {
+
+    private ColumnarBatch columnarBatch;
+
+    /**
+     * Adapter class which handles the columnar vector reading of the carbondata
+     * based on the spark ColumnVector and ColumnarBatch API. This proxy class
+     * handles the complexity of spark 2.3 version related api changes since
+     * spark ColumnVector and ColumnarBatch interfaces are still evolving.
+     *
+     * @param memMode       which represent the type onheap or offheap vector.
+     * @param rowNum        rows number for vector reading
+     * @param structFileds, metadata related to current schema of table.
+     */
+    public CarbonVectorProxy(MemoryMode memMode, int rowNum, StructField[] structFileds) {
+        columnarBatch = ColumnarBatch.allocate(new StructType(structFileds), memMode, rowNum);
+    }
+
+    public CarbonVectorProxy(MemoryMode memMode, StructType outputSchema, int rowNum) {
+        columnarBatch = ColumnarBatch.allocate(outputSchema, memMode, rowNum);
+    }
+
+    public ColumnVector getColumnVector(int ordinal) {
+        return columnarBatch.column(ordinal);
+    }
+
+    /**
+     * Sets the number of rows in this batch.
+     */
+    public void setNumRows(int numRows) {
+        columnarBatch.setNumRows(numRows);
+    }
+
+    public Object reserveDictionaryIds(int capacity , int ordinal) {
+        return columnarBatch.column(ordinal).reserveDictionaryIds(capacity);
+    }
+
+    /**
+     * Returns the number of rows for read, including filtered rows.
+     */
+    public int numRows() {
+        return columnarBatch.capacity();
+    }
+
+    public void setDictionary(Object dictionary, int ordinal) {
+        if (dictionary instanceof Dictionary) {
+            columnarBatch.column(ordinal).setDictionary((Dictionary) dictionary);
+        } else {
+            columnarBatch.column(ordinal).setDictionary(null);
+        }
+    }
+
+    public void putNull(int rowId, int ordinal) {
+        columnarBatch.column(ordinal).putNull(rowId);
+    }
+
+    public void putNulls(int rowId, int count, int ordinal) {
+        columnarBatch.column(ordinal).putNulls(rowId, count);
+    }
+
+    /**
+     * Called to close all the columns in this batch. It is not valid to access the data after
+     * calling this. This must be called at the end to clean up memory allocations.
+     */
+    public void close() {
+        columnarBatch.close();
+    }
+
+    /**
+     * Returns the row in this batch at `rowId`. Returned row is reused across calls.
+     */
+    public InternalRow getRow(int rowId) {
+        return columnarBatch.getRow(rowId);
+    }
+
+    /**
+     * Returns the row in this batch at `rowId`. Returned row is reused across calls.
+     */
+    public Object getColumnarBatch() {
+        return columnarBatch;
+    }
+
+    public void resetDictionaryIds(int ordinal) {
+        columnarBatch.column(ordinal).getDictionaryIds().reset();
+    }
+
+    /**
+     * This API will return a columnvector from a batch of column vector rows
+     * based on the ordinal
+     *
+     * @param ordinal
+     * @return
+     */
+    public ColumnVector column(int ordinal) {
+        return columnarBatch.column(ordinal);
+    }
+
+    public boolean hasDictionary(int ordinal) {
+        return columnarBatch.column(ordinal).hasDictionary();
+    }
+
+    /**
+     * Resets this column for writing. The currently stored values are no longer accessible.
+     */
+    public void reset() {
+        columnarBatch.reset();
+    }
+
+    public void putRowToColumnBatch(int rowId, Object value, int offset) {
+        org.apache.spark.sql.types.DataType t = dataType(offset);
+        if (null == value) {
+            putNull(rowId, offset);
+        } else {
+            if (t == org.apache.spark.sql.types.DataTypes.BooleanType) {
+                putBoolean(rowId, (boolean) value, offset);
+            } else if (t == org.apache.spark.sql.types.DataTypes.ByteType) {
+                putByte(rowId, (byte) value, offset);
+            } else if (t == org.apache.spark.sql.types.DataTypes.ShortType) {
+                putShort(rowId, (short) value, offset);
+            } else if (t == org.apache.spark.sql.types.DataTypes.IntegerType) {
+                putInt(rowId, (int) value, offset);
+            } else if (t == org.apache.spark.sql.types.DataTypes.LongType) {
+                putLong(rowId, (long) value, offset);
+            } else if (t == org.apache.spark.sql.types.DataTypes.FloatType) {
+                putFloat(rowId, (float) value, offset);
+            } else if (t == org.apache.spark.sql.types.DataTypes.DoubleType) {
+                putDouble(rowId, (double) value, offset);
+            } else if (t == org.apache.spark.sql.types.DataTypes.StringType) {
+                UTF8String v = (UTF8String) value;
+                putByteArray(rowId, v.getBytes(), offset);
+            } else if (t instanceof org.apache.spark.sql.types.DecimalType) {
+                DecimalType dt = (DecimalType) t;
+                Decimal d = Decimal.fromDecimal(value);
+                if (dt.precision() <= Decimal.MAX_INT_DIGITS()) {
+                    putInt(rowId, (int) d.toUnscaledLong(), offset);
+                } else if (dt.precision() <= Decimal.MAX_LONG_DIGITS()) {
+                    putLong(rowId, d.toUnscaledLong(), offset);
+                } else {
+                    final BigInteger integer = d.toJavaBigDecimal().unscaledValue();
+                    byte[] bytes = integer.toByteArray();
+                    putByteArray(rowId, bytes, 0, bytes.length, offset);
+                }
+            } else if (t instanceof CalendarIntervalType) {
+                CalendarInterval c = (CalendarInterval) value;
+                columnarBatch.column(offset).getChildColumn(0).putInt(rowId, c.months);
+                columnarBatch.column(offset).getChildColumn(1).putLong(rowId, c.microseconds);
+            } else if (t instanceof org.apache.spark.sql.types.DateType) {
+                putInt(rowId, (int) value, offset);
+            } else if (t instanceof org.apache.spark.sql.types.TimestampType) {
+                putLong(rowId, (long) value, offset);
+            }
+        }
+    }
+
+    public void putBoolean(int rowId, boolean value, int ordinal) {
+        columnarBatch.column(ordinal).putBoolean(rowId, (boolean) value);
+    }
+
+    public void putByte(int rowId, byte value, int ordinal) {
+        columnarBatch.column(ordinal).putByte(rowId, (byte) value);
+    }
+
+    public void putShort(int rowId, short value, int ordinal) {
+        columnarBatch.column(ordinal).putShort(rowId, (short) value);
+    }
+
+    public void putInt(int rowId, int value, int ordinal) {
+        columnarBatch.column(ordinal).putInt(rowId, (int) value);
+    }
+
+    public void putFloat(int rowId, float value, int ordinal) {
+        columnarBatch.column(ordinal).putFloat(rowId, (float) value);
+    }
+
+    public void putLong(int rowId, long value, int ordinal) {
+        columnarBatch.column(ordinal).putLong(rowId, (long) value);
+    }
+
+    public void putDouble(int rowId, double value, int ordinal) {
+        columnarBatch.column(ordinal).putDouble(rowId, (double) value);
+    }
+
+    public void putByteArray(int rowId, byte[] value, int ordinal) {
+        columnarBatch.column(ordinal).putByteArray(rowId, (byte[]) value);
+    }
+
+    public void putInts(int rowId, int count, int value, int ordinal) {
+        columnarBatch.column(ordinal).putInts(rowId, count, value);
+    }
+
+    public void putShorts(int rowId, int count, short value, int ordinal) {
+        columnarBatch.column(ordinal).putShorts(rowId, count, value);
+    }
+
+    public void putLongs(int rowId, int count, long value, int ordinal) {
+        columnarBatch.column(ordinal).putLongs(rowId, count, value);
+    }
+
+    public void putDecimal(int rowId, Decimal value, int precision, int ordinal) {
+        columnarBatch.column(ordinal).putDecimal(rowId, value, precision);
+
+    }
+
+    public void putDoubles(int rowId, int count, double value, int ordinal) {
+        columnarBatch.column(ordinal).putDoubles(rowId, count, value);
+    }
+
+    public void putByteArray(int rowId, byte[] value, int offset, int length, int ordinal) {
+        columnarBatch.column(ordinal).putByteArray(rowId, (byte[]) value, offset, length);
+    }
+
+    public boolean isNullAt(int rowId, int ordinal) {
+        return columnarBatch
+                .column(ordinal).isNullAt(rowId);
+    }
+
+    public DataType dataType(int ordinal) {
+        return columnarBatch.column(ordinal).dataType();
+    }
+
+    public void putNotNull(int rowId, int ordinal) {
+        columnarBatch.column(ordinal).putNotNull(rowId);
+    }
+
+    public void putNotNulls(int rowId, int count, int ordinal) {
+        columnarBatch.column(ordinal).putNotNulls(rowId, count);
+    }
+
+    public void putDictionaryInt(int rowId, int value, int ordinal) {
+        columnarBatch.column(ordinal).getDictionaryIds().putInt(rowId, (int) value);
+    }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark-datasource/src/main/spark2.3plus/org/apache/spark/sql/CarbonVectorProxy.java
----------------------------------------------------------------------
diff --git a/integration/spark-datasource/src/main/spark2.3plus/org/apache/spark/sql/CarbonVectorProxy.java b/integration/spark-datasource/src/main/spark2.3plus/org/apache/spark/sql/CarbonVectorProxy.java
new file mode 100644
index 0000000..0f23294
--- /dev/null
+++ b/integration/spark-datasource/src/main/spark2.3plus/org/apache/spark/sql/CarbonVectorProxy.java
@@ -0,0 +1,276 @@
+/*
+ * 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.spark.sql;
+
+import java.math.BigInteger;
+
+import org.apache.spark.memory.MemoryMode;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.execution.vectorized.Dictionary;
+import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
+import org.apache.spark.sql.types.*;
+import org.apache.spark.sql.vectorized.ColumnarBatch;
+import org.apache.spark.unsafe.types.CalendarInterval;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/**
+ * Adapter class which handles the columnar vector reading of the carbondata
+ * based on the spark ColumnVector and ColumnarBatch API. This proxy class
+ * handles the complexity of spark 2.3 version related api changes since
+ * spark ColumnVector and ColumnarBatch interfaces are still evolving.
+ */
+public class CarbonVectorProxy {
+
+    private ColumnarBatch columnarBatch;
+    private WritableColumnVector[] columnVectors;
+
+    /**
+     * Adapter class which handles the columnar vector reading of the carbondata
+     * based on the spark ColumnVector and ColumnarBatch API. This proxy class
+     * handles the complexity of spark 2.3 version related api changes since
+     * spark ColumnVector and ColumnarBatch interfaces are still evolving.
+     *
+     * @param memMode       which represent the type onheap or offheap vector.
+     * @param rowNum        rows number for vector reading
+     * @param structFileds, metadata related to current schema of table.
+     */
+    public CarbonVectorProxy(MemoryMode memMode, int rowNum, StructField[] structFileds) {
+        columnVectors = ColumnVectorFactory
+                .getColumnVector(memMode, new StructType(structFileds), rowNum);
+        columnarBatch = new ColumnarBatch(columnVectors);
+        columnarBatch.setNumRows(rowNum);
+    }
+
+    public CarbonVectorProxy(MemoryMode memMode, StructType outputSchema, int rowNum) {
+        columnVectors = ColumnVectorFactory
+                .getColumnVector(memMode, outputSchema, rowNum);
+        columnarBatch = new ColumnarBatch(columnVectors);
+        columnarBatch.setNumRows(rowNum);
+    }
+
+    /**
+     * Returns the number of rows for read, including filtered rows.
+     */
+    public int numRows() {
+        return columnarBatch.numRows();
+    }
+
+    public Object reserveDictionaryIds(int capacity, int ordinal) {
+        return columnVectors[ordinal].reserveDictionaryIds(capacity);
+    }
+
+    /**
+     * This API will return a columnvector from a batch of column vector rows
+     * based on the ordinal
+     *
+     * @param ordinal
+     * @return
+     */
+    public WritableColumnVector column(int ordinal) {
+        return (WritableColumnVector) columnarBatch.column(ordinal);
+    }
+
+    public WritableColumnVector getColumnVector(int ordinal) {
+        return columnVectors[ordinal];
+    }
+
+    /**
+     * Resets this column for writing. The currently stored values are no longer accessible.
+     */
+    public void reset() {
+        for (WritableColumnVector col : columnVectors) {
+            col.reset();
+        }
+    }
+
+    public void resetDictionaryIds(int ordinal) {
+        columnVectors[ordinal].getDictionaryIds().reset();
+    }
+
+    /**
+     * Returns the row in this batch at `rowId`. Returned row is reused across calls.
+     */
+    public InternalRow getRow(int rowId) {
+        return columnarBatch.getRow(rowId);
+    }
+
+
+    /**
+     * Returns the row in this batch at `rowId`. Returned row is reused across calls.
+     */
+    public Object getColumnarBatch() {
+        return columnarBatch;
+    }
+
+    /**
+     * Called to close all the columns in this batch. It is not valid to access the data after
+     * calling this. This must be called at the end to clean up memory allocations.
+     */
+    public void close() {
+        columnarBatch.close();
+    }
+
+    /**
+     * Sets the number of rows in this batch.
+     */
+    public void setNumRows(int numRows) {
+        columnarBatch.setNumRows(numRows);
+    }
+
+    public void putRowToColumnBatch(int rowId, Object value, int offset) {
+        org.apache.spark.sql.types.DataType t = dataType(offset);
+        if (null == value) {
+            putNull(rowId, offset);
+        } else {
+            if (t == org.apache.spark.sql.types.DataTypes.BooleanType) {
+                putBoolean(rowId, (boolean) value, offset);
+            } else if (t == org.apache.spark.sql.types.DataTypes.ByteType) {
+                putByte(rowId, (byte) value, offset);
+            } else if (t == org.apache.spark.sql.types.DataTypes.ShortType) {
+                putShort(rowId, (short) value, offset);
+            } else if (t == org.apache.spark.sql.types.DataTypes.IntegerType) {
+                putInt(rowId, (int) value, offset);
+            } else if (t == org.apache.spark.sql.types.DataTypes.LongType) {
+                putLong(rowId, (long) value, offset);
+            } else if (t == org.apache.spark.sql.types.DataTypes.FloatType) {
+                putFloat(rowId, (float) value, offset);
+            } else if (t == org.apache.spark.sql.types.DataTypes.DoubleType) {
+                putDouble(rowId, (double) value, offset);
+            } else if (t == org.apache.spark.sql.types.DataTypes.StringType) {
+                UTF8String v = (UTF8String) value;
+                putByteArray(rowId, v.getBytes(), offset);
+            } else if (t instanceof DecimalType) {
+                DecimalType dt = (DecimalType) t;
+                Decimal d = Decimal.fromDecimal(value);
+                if (dt.precision() <= Decimal.MAX_INT_DIGITS()) {
+                    putInt(rowId, (int) d.toUnscaledLong(), offset);
+                } else if (dt.precision() <= Decimal.MAX_LONG_DIGITS()) {
+                    putLong(rowId, d.toUnscaledLong(), offset);
+                } else {
+                    final BigInteger integer = d.toJavaBigDecimal().unscaledValue();
+                    byte[] bytes = integer.toByteArray();
+                    putByteArray(rowId, bytes, 0, bytes.length, offset);
+                }
+            } else if (t instanceof CalendarIntervalType) {
+                CalendarInterval c = (CalendarInterval) value;
+                columnVectors[offset].getChild(0).putInt(rowId, c.months);
+                columnVectors[offset].getChild(1).putLong(rowId, c.microseconds);
+            } else if (t instanceof org.apache.spark.sql.types.DateType) {
+                putInt(rowId, (int) value, offset);
+            } else if (t instanceof org.apache.spark.sql.types.TimestampType) {
+                putLong(rowId, (long) value, offset);
+            }
+        }
+    }
+
+    public void putBoolean(int rowId, boolean value, int ordinal) {
+        columnVectors[ordinal].putBoolean(rowId, (boolean) value);
+    }
+
+    public void putByte(int rowId, byte value, int ordinal) {
+        columnVectors[ordinal].putByte(rowId, (byte) value);
+    }
+
+    public void putShort(int rowId, short value, int ordinal) {
+        columnVectors[ordinal].putShort(rowId, (short) value);
+    }
+
+    public void putInt(int rowId, int value, int ordinal) {
+        columnVectors[ordinal].putInt(rowId, (int) value);
+    }
+
+    public void putDictionaryInt(int rowId, int value, int ordinal) {
+        columnVectors[ordinal].getDictionaryIds().putInt(rowId, (int) value);
+    }
+
+    public void putFloat(int rowId, float value, int ordinal) {
+        columnVectors[ordinal].putFloat(rowId, (float) value);
+    }
+
+    public void putLong(int rowId, long value, int ordinal) {
+        columnVectors[ordinal].putLong(rowId, (long) value);
+    }
+
+    public void putDouble(int rowId, double value, int ordinal) {
+        columnVectors[ordinal].putDouble(rowId, (double) value);
+    }
+
+    public void putByteArray(int rowId, byte[] value, int ordinal) {
+        columnVectors[ordinal].putByteArray(rowId, (byte[]) value);
+    }
+
+    public void putInts(int rowId, int count, int value, int ordinal) {
+        columnVectors[ordinal].putInts(rowId, count, value);
+    }
+
+    public void putShorts(int rowId, int count, short value, int ordinal) {
+        columnVectors[ordinal].putShorts(rowId, count, value);
+    }
+
+    public void putLongs(int rowId, int count, long value, int ordinal) {
+        columnVectors[ordinal].putLongs(rowId, count, value);
+    }
+
+    public void putDecimal(int rowId, Decimal value, int precision, int ordinal) {
+        columnVectors[ordinal].putDecimal(rowId, value, precision);
+
+    }
+
+    public void putDoubles(int rowId, int count, double value, int ordinal) {
+        columnVectors[ordinal].putDoubles(rowId, count, value);
+    }
+
+    public void putByteArray(int rowId, byte[] value, int offset, int length, int ordinal) {
+        columnVectors[ordinal].putByteArray(rowId, (byte[]) value, offset, length);
+    }
+
+    public void putNull(int rowId, int ordinal) {
+        columnVectors[ordinal].putNull(rowId);
+    }
+
+    public void putNulls(int rowId, int count, int ordinal) {
+        columnVectors[ordinal].putNulls(rowId, count);
+    }
+
+    public void putNotNull(int rowId, int ordinal) {
+        columnVectors[ordinal].putNotNull(rowId);
+    }
+
+    public void putNotNulls(int rowId, int count, int ordinal) {
+        columnVectors[ordinal].putNotNulls(rowId, count);
+    }
+
+    public boolean isNullAt(int rowId, int ordinal) {
+        return columnVectors[ordinal].isNullAt(rowId);
+    }
+
+    public boolean hasDictionary(int ordinal) {
+        return columnVectors[ordinal].hasDictionary();
+    }
+
+    public void setDictionary(Object dictionary, int ordinal) {
+        if (dictionary instanceof Dictionary) {
+            columnVectors[ordinal].setDictionary((Dictionary) dictionary);
+        } else {
+            columnVectors[ordinal].setDictionary(null);
+        }
+    }
+
+    public DataType dataType(int ordinal) {
+        return columnVectors[ordinal].dataType();
+    }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark-datasource/src/main/spark2.3plus/org/apache/spark/sql/ColumnVectorFactory.java
----------------------------------------------------------------------
diff --git a/integration/spark-datasource/src/main/spark2.3plus/org/apache/spark/sql/ColumnVectorFactory.java b/integration/spark-datasource/src/main/spark2.3plus/org/apache/spark/sql/ColumnVectorFactory.java
new file mode 100644
index 0000000..6fe5ede
--- /dev/null
+++ b/integration/spark-datasource/src/main/spark2.3plus/org/apache/spark/sql/ColumnVectorFactory.java
@@ -0,0 +1,45 @@
+/*
+ * 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.spark.sql;
+
+import org.apache.spark.memory.MemoryMode;
+import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
+import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
+import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.sql.vectorized.ColumnarBatch;
+
+public class ColumnVectorFactory {
+
+
+    public static WritableColumnVector[] getColumnVector(MemoryMode memMode, StructType outputSchema, int rowNums) {
+
+
+        WritableColumnVector[] writableColumnVectors = null;
+        switch (memMode) {
+            case ON_HEAP:
+                writableColumnVectors = OnHeapColumnVector
+                        .allocateColumns(rowNums, outputSchema);
+                break;
+            case OFF_HEAP:
+                writableColumnVectors = OffHeapColumnVector
+                        .allocateColumns(rowNums, outputSchema);
+                break;
+        }
+        return writableColumnVectors;
+    }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/TestCreateTableUsingSparkCarbonFileFormat.scala
----------------------------------------------------------------------
diff --git a/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/TestCreateTableUsingSparkCarbonFileFormat.scala b/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/TestCreateTableUsingSparkCarbonFileFormat.scala
index 12b5cbc..a0c4a0b 100644
--- a/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/TestCreateTableUsingSparkCarbonFileFormat.scala
+++ b/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/TestCreateTableUsingSparkCarbonFileFormat.scala
@@ -22,6 +22,7 @@ import java.io.File
 import org.apache.commons.io.FileUtils
 import org.scalatest.{BeforeAndAfterAll, FunSuite}
 import org.apache.spark.sql.carbondata.datasource.TestUtil._
+import org.apache.spark.util.SparkUtil
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile
@@ -110,10 +111,10 @@ class TestCreateTableUsingSparkCarbonFileFormat extends FunSuite with BeforeAndA
     spark.sql("DROP TABLE IF EXISTS sdkOutputTable")
 
     //data source file format
-    if (spark.sparkContext.version.startsWith("2.1")) {
+    if (SparkUtil.isSparkVersionEqualTo("2.1")) {
       //data source file format
       spark.sql(s"""CREATE TABLE sdkOutputTable USING carbon OPTIONS (PATH '$filePath') """)
-    } else if (spark.sparkContext.version.startsWith("2.2")) {
+    } else if (SparkUtil.isSparkVersionXandAbove("2.2")) {
       //data source file format
       spark.sql(
         s"""CREATE TABLE sdkOutputTable USING carbon LOCATION
@@ -158,10 +159,10 @@ class TestCreateTableUsingSparkCarbonFileFormat extends FunSuite with BeforeAndA
     spark.sql("DROP TABLE IF EXISTS sdkOutputTable")
 
     //data source file format
-    if (spark.sparkContext.version.startsWith("2.1")) {
+    if (SparkUtil.isSparkVersionEqualTo("2.1")) {
       //data source file format
       spark.sql(s"""CREATE TABLE sdkOutputTable USING carbon OPTIONS (PATH '$filePath') """)
-    } else if (spark.sparkContext.version.startsWith("2.2")) {
+    } else if (SparkUtil.isSparkVersionXandAbove("2.2")) {
       //data source file format
       spark.sql(
         s"""CREATE TABLE sdkOutputTable USING carbon LOCATION
@@ -180,7 +181,6 @@ class TestCreateTableUsingSparkCarbonFileFormat extends FunSuite with BeforeAndA
     cleanTestData()
   }
 
-
   // TODO: Make the sparkCarbonFileFormat to work without index file
   test("Read sdk writer output file without Carbondata file should fail") {
     buildTestData(false)
@@ -190,10 +190,10 @@ class TestCreateTableUsingSparkCarbonFileFormat extends FunSuite with BeforeAndA
 
     val exception = intercept[Exception] {
       //    data source file format
-      if (spark.sparkContext.version.startsWith("2.1")) {
+      if (SparkUtil.isSparkVersionEqualTo("2.1")) {
         //data source file format
         spark.sql(s"""CREATE TABLE sdkOutputTable USING carbon OPTIONS (PATH '$filePath') """)
-      } else if (spark.sparkContext.version.startsWith("2.2")) {
+      } else if (SparkUtil.isSparkVersionXandAbove("2.2")) {
         //data source file format
         spark.sql(
           s"""CREATE TABLE sdkOutputTable USING carbon LOCATION
@@ -220,10 +220,10 @@ class TestCreateTableUsingSparkCarbonFileFormat extends FunSuite with BeforeAndA
 
     val exception = intercept[Exception] {
       //data source file format
-      if (spark.sparkContext.version.startsWith("2.1")) {
+      if (SparkUtil.isSparkVersionEqualTo("2.1")) {
         //data source file format
         spark.sql(s"""CREATE TABLE sdkOutputTable USING carbon OPTIONS (PATH '$filePath') """)
-      } else if (spark.sparkContext.version.startsWith("2.2")) {
+      } else if (SparkUtil.isSparkVersionXandAbove("2.2")) {
         //data source file format
         spark.sql(
           s"""CREATE TABLE sdkOutputTable USING carbon LOCATION
@@ -250,10 +250,10 @@ class TestCreateTableUsingSparkCarbonFileFormat extends FunSuite with BeforeAndA
     //data source file format
     spark.sql("DROP TABLE IF EXISTS sdkOutputTable")
 
-    if (spark.sparkContext.version.startsWith("2.1")) {
+    if (SparkUtil.isSparkVersionEqualTo("2.1")) {
       //data source file format
       spark.sql(s"""CREATE TABLE sdkOutputTable USING carbon OPTIONS (PATH '$filePath') """)
-    } else if (spark.sparkContext.version.startsWith("2.2")) {
+    } else if (SparkUtil.isSparkVersionXandAbove("2.2")) {
       //data source file format
       spark.sql(
         s"""CREATE TABLE sdkOutputTable USING carbon LOCATION
@@ -299,10 +299,10 @@ class TestCreateTableUsingSparkCarbonFileFormat extends FunSuite with BeforeAndA
     assert(new File(filePath).exists())
     spark.sql("DROP TABLE IF EXISTS sdkOutputTable")
 
-    if (spark.sparkContext.version.startsWith("2.1")) {
+    if (SparkUtil.isSparkVersionEqualTo("2.1")) {
       //data source file format
       spark.sql(s"""CREATE TABLE sdkOutputTable USING carbon OPTIONS (PATH '$filePath') """)
-    } else if (spark.sparkContext.version.startsWith("2.2")) {
+    } else if (SparkUtil.isSparkVersionXandAbove("2.2")) {
       //data source file format
       spark.sql(
         s"""CREATE TABLE sdkOutputTable USING carbon LOCATION

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/pom.xml
----------------------------------------------------------------------
diff --git a/integration/spark2/pom.xml b/integration/spark2/pom.xml
index 24af8ec..5af4fbe 100644
--- a/integration/spark2/pom.xml
+++ b/integration/spark2/pom.xml
@@ -54,11 +54,26 @@
       <groupId>org.apache.carbondata</groupId>
       <artifactId>carbondata-lucene</artifactId>
       <version>${project.version}</version>
+      <exclusions>
+        <!-- need to Exclude net.jpountz jar from this project.
+         Spark has changed this jar to org.lz4:lz4-java
+         net.jpountz and org.lz4 has same class Name -->
+        <exclusion>
+          <groupId>net.jpountz.lz4</groupId>
+          <artifactId>lz4</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.apache.carbondata</groupId>
       <artifactId>carbondata-bloom</artifactId>
       <version>${project.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>net.jpountz.lz4</groupId>
+          <artifactId>lz4</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.apache.spark</groupId>
@@ -75,6 +90,25 @@
       <scope>${spark.deps.scope}</scope>
     </dependency>
     <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-core_${scala.binary.version}</artifactId>
+      <version>${spark.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-sql_${scala.binary.version}</artifactId>
+      <version>${spark.version}</version>
+      <exclusions>
+        <!-- from transitive dependency com.univocity:univocity-parsers:2.5.9
+        is added from the org.apache.spark:spark-sql_2.11,so need to remove
+        this version.Carbon uses 2.2.1 version  -->
+        <exclusion>
+          <groupId>com.univocity</groupId>
+          <artifactId>univocity-parsers</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>
@@ -233,6 +267,8 @@
             <configuration>
               <excludes>
                 <exclude>src/main/spark2.2</exclude>
+                <exclude>src/main/spark2.3</exclude>
+                <exclude>src/main/commonTo2.2And2.3</exclude>
               </excludes>
             </configuration>
           </plugin>
@@ -276,6 +312,7 @@
             <configuration>
               <excludes>
                 <exclude>src/main/spark2.1</exclude>
+                <exclude>src/main/spark2.3</exclude>
               </excludes>
             </configuration>
           </plugin>
@@ -293,6 +330,49 @@
                 <configuration>
                   <sources>
                     <source>src/main/spark2.2</source>
+                    <source>src/main/commonTo2.2And2.3</source>
+                  </sources>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+    <profile>
+      <id>spark-2.3</id>
+      <properties>
+        <spark.version>2.3.1</spark.version>
+        <scala.binary.version>2.11</scala.binary.version>
+        <scala.version>2.11.8</scala.version>
+      </properties>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-compiler-plugin</artifactId>
+            <configuration>
+              <excludes>
+                <exclude>src/main/spark2.1</exclude>
+                <exclude>src/main/spark2.2</exclude>
+              </excludes>
+            </configuration>
+          </plugin>
+          <plugin>
+            <groupId>org.codehaus.mojo</groupId>
+            <artifactId>build-helper-maven-plugin</artifactId>
+            <version>3.0.0</version>
+            <executions>
+              <execution>
+                <id>add-source</id>
+                <phase>generate-sources</phase>
+                <goals>
+                  <goal>add-source</goal>
+                </goals>
+                <configuration>
+                  <sources>
+                    <source>src/main/spark2.3</source>
+                    <source>src/main/commonTo2.2And2.3</source>
                   </sources>
                 </configuration>
               </execution>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonAnalyzer.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonAnalyzer.scala b/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonAnalyzer.scala
new file mode 100644
index 0000000..dfb89fd
--- /dev/null
+++ b/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonAnalyzer.scala
@@ -0,0 +1,51 @@
+/*
+ * 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.spark.sql.hive
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.analysis.Analyzer
+import org.apache.spark.sql.catalyst.catalog.SessionCatalog
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.util.CarbonReflectionUtils
+
+class CarbonAnalyzer(catalog: SessionCatalog,
+    conf: SQLConf,
+    sparkSession: SparkSession,
+    analyzer: Analyzer) extends Analyzer(catalog, conf) {
+
+  val mvPlan = try {
+    CarbonReflectionUtils.createObject(
+      "org.apache.carbondata.mv.datamap.MVAnalyzerRule",
+      sparkSession)._1.asInstanceOf[Rule[LogicalPlan]]
+  } catch {
+    case e: Exception =>
+      null
+  }
+
+  override def execute(plan: LogicalPlan): LogicalPlan = {
+    var logicalPlan = analyzer.execute(plan)
+    logicalPlan = CarbonPreAggregateDataLoadingRules(sparkSession).apply(logicalPlan)
+    logicalPlan = CarbonPreAggregateQueryRules(sparkSession).apply(logicalPlan)
+    if (mvPlan != null) {
+      mvPlan.apply(logicalPlan)
+    } else {
+      logicalPlan
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonInMemorySessionState.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonInMemorySessionState.scala b/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonInMemorySessionState.scala
new file mode 100644
index 0000000..ba6aae5
--- /dev/null
+++ b/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonInMemorySessionState.scala
@@ -0,0 +1,278 @@
+/*
+ * 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.spark.sql.hive
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.spark.sql.carbondata.execution.datasources.CarbonSparkDataSourceUtil
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry}
+import org.apache.spark.sql.catalyst.catalog._
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.optimizer.Optimizer
+import org.apache.spark.sql.catalyst.parser.ParserInterface
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.datasources._
+import org.apache.spark.sql.execution.strategy.{CarbonLateDecodeStrategy, DDLStrategy, StreamingTableStrategy}
+import org.apache.spark.sql.internal.{SQLConf, SessionResourceLoader, SessionState, SessionStateBuilder}
+import org.apache.spark.sql.optimizer.{CarbonIUDRule, CarbonLateDecodeRule, CarbonUDFTransformRule}
+import org.apache.spark.sql.parser.CarbonSparkSqlParser
+import org.apache.spark.sql.types.{StructField, StructType}
+import org.apache.spark.sql.{CarbonEnv, SparkSession}
+
+import org.apache.carbondata.core.util.CarbonUtil
+import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.format.TableInfo
+import org.apache.carbondata.spark.util.CarbonScalaUtil
+
+/**
+ * This class will have carbon catalog and refresh the relation from cache if the carbontable in
+ * carbon catalog is not same as cached carbon relation's carbon table
+ *
+ * @param externalCatalog
+ * @param globalTempViewManager
+ * @param sparkSession
+ * @param functionResourceLoader
+ * @param functionRegistry
+ * @param conf
+ * @param hadoopConf
+ */
+class InMemorySessionCatalog(
+    externalCatalog: ExternalCatalog,
+    globalTempViewManager: GlobalTempViewManager,
+    functionRegistry: FunctionRegistry,
+    sparkSession: SparkSession,
+    conf: SQLConf,
+    hadoopConf: Configuration,
+    parser: ParserInterface,
+    functionResourceLoader: FunctionResourceLoader)
+  extends SessionCatalog(
+    externalCatalog,
+    globalTempViewManager,
+    functionRegistry,
+    conf,
+    hadoopConf,
+    parser,
+    functionResourceLoader
+  ) with CarbonSessionCatalog {
+
+  override def alterTableRename(oldTableIdentifier: TableIdentifier,
+      newTableIdentifier: TableIdentifier,
+      newTablePath: String): Unit = {
+    sparkSession.sessionState.catalog.renameTable(oldTableIdentifier, newTableIdentifier)
+  }
+
+  override def alterTable(tableIdentifier: TableIdentifier,
+      schemaParts: String,
+      cols: Option[Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]])
+  : Unit = {
+    // NOt Required in case of In-memory catalog
+  }
+
+  override def alterAddColumns(tableIdentifier: TableIdentifier,
+      schemaParts: String,
+      newColumns: Option[Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]])
+  : Unit = {
+    val catalogTable = sparkSession.sessionState.catalog.getTableMetadata(tableIdentifier)
+    val structType = catalogTable.schema
+    var newStructType = structType
+    newColumns.get.foreach {cols =>
+      newStructType = structType
+        .add(cols.getColumnName,
+          CarbonSparkDataSourceUtil.convertCarbonToSparkDataType(cols.getDataType))
+    }
+    alterSchema(newStructType, catalogTable, tableIdentifier)
+  }
+
+  override def alterDropColumns(tableIdentifier: TableIdentifier,
+      schemaParts: String,
+      dropCols: Option[Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]])
+  : Unit = {
+    val catalogTable = sparkSession.sessionState.catalog.getTableMetadata(tableIdentifier)
+    val fields = catalogTable.schema.fields.filterNot { field =>
+      dropCols.get.exists { col =>
+        col.getColumnName.equalsIgnoreCase(field.name)
+      }
+    }
+    alterSchema(new StructType(fields), catalogTable, tableIdentifier)
+  }
+
+  override def alterColumnChangeDataType(tableIdentifier: TableIdentifier,
+      schemaParts: String,
+      columns: Option[Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]])
+  : Unit = {
+    val catalogTable = sparkSession.sessionState.catalog.getTableMetadata(tableIdentifier)
+    val a = catalogTable.schema.fields.flatMap { field =>
+      columns.get.map { col =>
+        if (col.getColumnName.equalsIgnoreCase(field.name)) {
+          StructField(col.getColumnName,
+            CarbonSparkDataSourceUtil.convertCarbonToSparkDataType(col.getDataType))
+        } else {
+          field
+        }
+      }
+    }
+    alterSchema(new StructType(a), catalogTable, tableIdentifier)
+  }
+
+  private def alterSchema(structType: StructType,
+      catalogTable: CatalogTable,
+      tableIdentifier: TableIdentifier): Unit = {
+    val copy = catalogTable.copy(schema = structType)
+    sparkSession.sessionState.catalog.alterTable(copy)
+    sparkSession.sessionState.catalog.refreshTable(tableIdentifier)
+  }
+
+  lazy val carbonEnv = {
+    val env = new CarbonEnv
+    env.init(sparkSession)
+    env
+  }
+
+  def getCarbonEnv() : CarbonEnv = {
+    carbonEnv
+  }
+
+  // Initialize all listeners to the Operation bus.
+  CarbonEnv.initListeners()
+
+  def getThriftTableInfo(tablePath: String): TableInfo = {
+    val tableMetadataFile = CarbonTablePath.getSchemaFilePath(tablePath)
+    CarbonUtil.readSchemaFile(tableMetadataFile)
+  }
+
+  override def lookupRelation(name: TableIdentifier): LogicalPlan = {
+    val rtnRelation = super.lookupRelation(name)
+    val isRelationRefreshed =
+      CarbonSessionUtil.refreshRelation(rtnRelation, name)(sparkSession)
+    if (isRelationRefreshed) {
+      super.lookupRelation(name)
+    } else {
+      rtnRelation
+    }
+  }
+
+  /**
+   * returns hive client from HiveExternalCatalog
+   *
+   * @return
+   */
+  def getClient(): org.apache.spark.sql.hive.client.HiveClient = {
+    null
+  }
+
+  override def createPartitions(
+      tableName: TableIdentifier,
+      parts: Seq[CatalogTablePartition],
+      ignoreIfExists: Boolean): Unit = {
+    try {
+      val table = CarbonEnv.getCarbonTable(tableName)(sparkSession)
+      val updatedParts = CarbonScalaUtil.updatePartitions(parts, table)
+      super.createPartitions(tableName, updatedParts, ignoreIfExists)
+    } catch {
+      case e: Exception =>
+        super.createPartitions(tableName, parts, ignoreIfExists)
+    }
+  }
+
+  /**
+   * This is alternate way of getting partition information. It first fetches all partitions from
+   * hive and then apply filter instead of querying hive along with filters.
+   * @param partitionFilters
+   * @param sparkSession
+   * @param identifier
+   * @return
+   */
+  override def getPartitionsAlternate(partitionFilters: Seq[Expression],
+      sparkSession: SparkSession,
+      identifier: TableIdentifier) = {
+    CarbonSessionUtil.prunePartitionsByFilter(partitionFilters, sparkSession, identifier)
+  }
+
+  /**
+   * Update the storageformat with new location information
+   */
+  override def updateStorageLocation(
+      path: Path,
+      storage: CatalogStorageFormat,
+      newTableName: String,
+      dbName: String): CatalogStorageFormat = {
+    storage.copy(locationUri = Some(path.toUri))
+  }
+}
+
+class CarbonInMemorySessionStateBuilder (sparkSession: SparkSession,
+    parentState: Option[SessionState] = None)
+  extends SessionStateBuilder(sparkSession, parentState) {
+
+  override lazy val sqlParser: ParserInterface = new CarbonSparkSqlParser(conf, sparkSession)
+
+  experimentalMethods.extraStrategies =
+    Seq(new StreamingTableStrategy(sparkSession),
+      new CarbonLateDecodeStrategy,
+      new DDLStrategy(sparkSession)
+    )
+  experimentalMethods.extraOptimizations = Seq(new CarbonIUDRule,
+    new CarbonUDFTransformRule,
+    new CarbonLateDecodeRule)
+
+  /**
+   * Internal catalog for managing table and database states.
+   */
+  override protected lazy val catalog: InMemorySessionCatalog = {
+    val catalog = new InMemorySessionCatalog(
+      externalCatalog,
+      session.sharedState.globalTempViewManager,
+      functionRegistry,
+      sparkSession,
+      conf,
+      SessionState.newHadoopConf(session.sparkContext.hadoopConfiguration, conf),
+      sqlParser,
+      resourceLoader)
+    parentState.foreach(_.catalog.copyStateTo(catalog))
+    catalog
+  }
+
+  private def externalCatalog: ExternalCatalog =
+    session.sharedState.externalCatalog.asInstanceOf[ExternalCatalog]
+
+  override protected lazy val resourceLoader: SessionResourceLoader = {
+    new SessionResourceLoader(session)
+  }
+
+  override lazy val optimizer: Optimizer = new CarbonOptimizer(catalog, conf, experimentalMethods)
+
+  override protected def analyzer: Analyzer = new CarbonAnalyzer(catalog, conf, sparkSession,
+    new Analyzer(catalog, conf) {
+      override val extendedResolutionRules: Seq[Rule[LogicalPlan]] =
+        new FindDataSourceTable(session) +:
+        new ResolveSQLOnFile(session) +:
+        new CarbonIUDAnalysisRule(sparkSession) +:
+        new CarbonPreInsertionCasts(sparkSession) +: customResolutionRules
+      override val extendedCheckRules: Seq[LogicalPlan => Unit] =
+        PreWriteCheck :: HiveOnlyCheck :: Nil
+      override val postHocResolutionRules: Seq[Rule[LogicalPlan]] =
+        PreprocessTableCreation(session) +:
+        PreprocessTableInsertion(conf) +:
+        DataSourceAnalysis(conf) +:
+        customPostHocResolutionRules
+    }
+  )
+  override protected def newBuilder: NewBuilder = new CarbonInMemorySessionStateBuilder(_, _)
+}
+

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonOptimizerUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonOptimizerUtil.scala b/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonOptimizerUtil.scala
new file mode 100644
index 0000000..72d3ae2
--- /dev/null
+++ b/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonOptimizerUtil.scala
@@ -0,0 +1,44 @@
+package org.apache.spark.sql.hive
+
+import org.apache.spark.sql.CarbonDatasourceHadoopRelation
+import org.apache.spark.sql.catalyst.expressions.{Exists, In, ListQuery, ScalarSubquery}
+import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan}
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+
+object CarbonOptimizerUtil {
+  def transformForScalarSubQuery(plan: LogicalPlan): LogicalPlan = {
+    // In case scalar subquery add flag in relation to skip the decoder plan in optimizer rule, And
+    // optimize whole plan at once.
+    val transFormedPlan = plan.transform {
+      case filter: Filter =>
+        filter.transformExpressions {
+          case s: ScalarSubquery =>
+            val tPlan = s.plan.transform {
+              case lr: LogicalRelation
+                if lr.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
+                lr.relation.asInstanceOf[CarbonDatasourceHadoopRelation].isSubquery += true
+                lr
+            }
+            ScalarSubquery(tPlan, s.children, s.exprId)
+          case e: Exists =>
+            val tPlan = e.plan.transform {
+              case lr: LogicalRelation
+                if lr.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
+                lr.relation.asInstanceOf[CarbonDatasourceHadoopRelation].isSubquery += true
+                lr
+            }
+            Exists(tPlan, e.children.map(_.canonicalized), e.exprId)
+
+          case In(value, Seq(l:ListQuery)) =>
+            val tPlan = l.plan.transform {
+              case lr: LogicalRelation
+                if lr.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
+                lr.relation.asInstanceOf[CarbonDatasourceHadoopRelation].isSubquery += true
+                lr
+            }
+            In(value, Seq(ListQuery(tPlan, l.children, l.exprId)))
+        }
+    }
+    transFormedPlan
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonSessionState.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonSessionState.scala b/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonSessionState.scala
new file mode 100644
index 0000000..f3168d7
--- /dev/null
+++ b/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonSessionState.scala
@@ -0,0 +1,269 @@
+/*
+ * 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.spark.sql.hive
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry}
+import org.apache.spark.sql.catalyst.catalog._
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.optimizer.Optimizer
+import org.apache.spark.sql.catalyst.parser.ParserInterface
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.datasources.{FindDataSourceTable, PreWriteCheck, ResolveSQLOnFile, _}
+import org.apache.spark.sql.execution.strategy.{CarbonLateDecodeStrategy, DDLStrategy, StreamingTableStrategy}
+import org.apache.spark.sql.hive.client.HiveClient
+import org.apache.spark.sql.internal.{SQLConf, SessionState}
+import org.apache.spark.sql.optimizer.{CarbonIUDRule, CarbonLateDecodeRule, CarbonUDFTransformRule}
+import org.apache.spark.sql.parser.CarbonSparkSqlParser
+
+import org.apache.carbondata.spark.util.CarbonScalaUtil
+
+/**
+ * This class will have carbon catalog and refresh the relation from cache if the carbontable in
+ * carbon catalog is not same as cached carbon relation's carbon table
+ *
+ * @param externalCatalog
+ * @param globalTempViewManager
+ * @param sparkSession
+ * @param functionResourceLoader
+ * @param functionRegistry
+ * @param conf
+ * @param hadoopConf
+ */
+class CarbonHiveSessionCatalog(
+    externalCatalog: HiveExternalCatalog,
+    globalTempViewManager: GlobalTempViewManager,
+    functionRegistry: FunctionRegistry,
+    sparkSession: SparkSession,
+    conf: SQLConf,
+    hadoopConf: Configuration,
+    parser: ParserInterface,
+    functionResourceLoader: FunctionResourceLoader)
+  extends HiveSessionCatalog (
+    externalCatalog,
+    globalTempViewManager,
+    new HiveMetastoreCatalog(sparkSession),
+    functionRegistry,
+    conf,
+    hadoopConf,
+    parser,
+    functionResourceLoader
+  ) with CarbonSessionCatalog {
+
+  private lazy val carbonEnv = {
+    val env = new CarbonEnv
+    env.init(sparkSession)
+    env
+  }
+  /**
+   * return's the carbonEnv instance
+   * @return
+   */
+  override def getCarbonEnv() : CarbonEnv = {
+    carbonEnv
+  }
+
+  // Initialize all listeners to the Operation bus.
+  CarbonEnv.initListeners()
+
+  override def lookupRelation(name: TableIdentifier): LogicalPlan = {
+    val rtnRelation = super.lookupRelation(name)
+    val isRelationRefreshed =
+      CarbonSessionUtil.refreshRelation(rtnRelation, name)(sparkSession)
+    if (isRelationRefreshed) {
+      super.lookupRelation(name)
+    } else {
+      rtnRelation
+    }
+  }
+
+  /**
+   * returns hive client from HiveExternalCatalog
+   *
+   * @return
+   */
+  override def getClient(): org.apache.spark.sql.hive.client.HiveClient = {
+    sparkSession.asInstanceOf[CarbonSession].sharedState.externalCatalog
+      .asInstanceOf[HiveExternalCatalog].client
+  }
+
+  def alterTableRename(oldTableIdentifier: TableIdentifier,
+      newTableIdentifier: TableIdentifier,
+      newTablePath: String): Unit = {
+    getClient().runSqlHive(
+      s"ALTER TABLE ${ oldTableIdentifier.database.get }.${ oldTableIdentifier.table } " +
+      s"RENAME TO ${ oldTableIdentifier.database.get }.${ newTableIdentifier.table }")
+    getClient().runSqlHive(
+      s"ALTER TABLE ${ oldTableIdentifier.database.get }.${ newTableIdentifier.table} " +
+      s"SET SERDEPROPERTIES" +
+      s"('tableName'='${ newTableIdentifier.table }', " +
+      s"'dbName'='${ oldTableIdentifier.database.get }', 'tablePath'='${ newTablePath }')")
+  }
+
+  override def alterTable(tableIdentifier: TableIdentifier,
+      schemaParts: String,
+      cols: Option[Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]])
+  : Unit = {
+    getClient()
+      .runSqlHive(s"ALTER TABLE ${tableIdentifier.database.get}.${ tableIdentifier.table } " +
+                  s"SET TBLPROPERTIES(${ schemaParts })")
+  }
+
+  override def alterAddColumns(tableIdentifier: TableIdentifier,
+      schemaParts: String,
+      cols: Option[Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]])
+  : Unit = {
+    alterTable(tableIdentifier, schemaParts, cols)
+  }
+
+  override def alterDropColumns(tableIdentifier: TableIdentifier,
+      schemaParts: String,
+      cols: Option[Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]])
+  : Unit = {
+    alterTable(tableIdentifier, schemaParts, cols)
+  }
+
+  override def alterColumnChangeDataType(tableIdentifier: TableIdentifier,
+      schemaParts: String,
+      cols: Option[Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]])
+  : Unit = {
+    alterTable(tableIdentifier, schemaParts, cols)
+  }
+
+  override def createPartitions(
+      tableName: TableIdentifier,
+      parts: Seq[CatalogTablePartition],
+      ignoreIfExists: Boolean): Unit = {
+    try {
+      val table = CarbonEnv.getCarbonTable(tableName)(sparkSession)
+      val updatedParts = CarbonScalaUtil.updatePartitions(parts, table)
+      super.createPartitions(tableName, updatedParts, ignoreIfExists)
+    } catch {
+      case e: Exception =>
+        super.createPartitions(tableName, parts, ignoreIfExists)
+    }
+  }
+
+  /**
+   * This is alternate way of getting partition information. It first fetches all partitions from
+   * hive and then apply filter instead of querying hive along with filters.
+   * @param partitionFilters
+   * @param sparkSession
+   * @param identifier
+   * @return
+   */
+  override def getPartitionsAlternate(partitionFilters: Seq[Expression],
+      sparkSession: SparkSession,
+      identifier: TableIdentifier) = {
+    CarbonSessionUtil.prunePartitionsByFilter(partitionFilters, sparkSession, identifier)
+  }
+
+  /**
+   * Update the storageformat with new location information
+   */
+  override def updateStorageLocation(
+      path: Path,
+      storage: CatalogStorageFormat,
+      newTableName: String,
+      dbName: String): CatalogStorageFormat = {
+    storage.copy(locationUri = Some(path.toUri))
+  }
+}
+
+/**
+ * Session state implementation to override sql parser and adding strategies
+ *
+ * @param sparkSession
+ */
+class CarbonSessionStateBuilder(sparkSession: SparkSession,
+    parentState: Option[SessionState] = None)
+  extends HiveSessionStateBuilder(sparkSession, parentState) {
+
+  override lazy val sqlParser: ParserInterface = new CarbonSparkSqlParser(conf, sparkSession)
+
+  experimentalMethods.extraStrategies =
+    Seq(new StreamingTableStrategy(sparkSession),
+        new CarbonLateDecodeStrategy,
+        new DDLStrategy(sparkSession)
+    )
+  experimentalMethods.extraOptimizations = Seq(new CarbonIUDRule,
+    new CarbonUDFTransformRule,
+    new CarbonLateDecodeRule)
+
+  /**
+   * Internal catalog for managing table and database states.
+   */
+  /**
+   * Create a [[CarbonSessionStateBuilder]].
+   */
+  override protected lazy val catalog: CarbonHiveSessionCatalog = {
+    val catalog = new CarbonHiveSessionCatalog(
+      externalCatalog,
+      session.sharedState.globalTempViewManager,
+      functionRegistry,
+      sparkSession,
+      conf,
+      SessionState.newHadoopConf(session.sparkContext.hadoopConfiguration, conf),
+      sqlParser,
+      resourceLoader)
+    parentState.foreach(_.catalog.copyStateTo(catalog))
+    catalog
+  }
+
+  private def externalCatalog: HiveExternalCatalog =
+    session.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog]
+
+  /**
+   * Create a Hive aware resource loader.
+   */
+  override protected lazy val resourceLoader: HiveSessionResourceLoader = {
+    val client: HiveClient = externalCatalog.client.newSession()
+    new HiveSessionResourceLoader(session, client)
+  }
+
+  override lazy val optimizer: Optimizer = new CarbonOptimizer(catalog, conf, experimentalMethods)
+
+  override protected def analyzer: Analyzer = new CarbonAnalyzer(catalog, conf, sparkSession,
+    new Analyzer(catalog, conf) {
+
+      override val extendedResolutionRules: Seq[Rule[LogicalPlan]] =
+        new ResolveHiveSerdeTable(session) +:
+        new FindDataSourceTable(session) +:
+        new ResolveSQLOnFile(session) +:
+        new CarbonIUDAnalysisRule(sparkSession) +:
+        new CarbonPreInsertionCasts(sparkSession) +: customResolutionRules
+
+      override val extendedCheckRules: Seq[LogicalPlan => Unit] =
+      PreWriteCheck :: HiveOnlyCheck :: Nil
+
+      override val postHocResolutionRules: Seq[Rule[LogicalPlan]] =
+        new DetermineTableStats(session) +:
+        RelationConversions(conf, catalog) +:
+        PreprocessTableCreation(session) +:
+        PreprocessTableInsertion(conf) +:
+        DataSourceAnalysis(conf) +:
+        HiveAnalysis +:
+        customPostHocResolutionRules
+    }
+  )
+
+  override protected def newBuilder: NewBuilder = new CarbonSessionStateBuilder(_, _)
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonSessionUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonSessionUtil.scala b/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonSessionUtil.scala
new file mode 100644
index 0000000..1a22e99
--- /dev/null
+++ b/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonSessionUtil.scala
@@ -0,0 +1,96 @@
+/*
+* 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.spark.sql.hive
+
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTablePartition, ExternalCatalogUtils}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, CarbonEnv, SparkSession}
+import org.apache.spark.util.CarbonReflectionUtils
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+
+/**
+ * This class refresh the relation from cache if the carbontable in
+ * carbon catalog is not same as cached carbon relation's carbon table.
+ */
+object CarbonSessionUtil {
+
+  val LOGGER = LogServiceFactory.getLogService("CarbonSessionUtil")
+
+  /**
+   * The method refreshes the cache entry
+   *
+   * @param rtnRelation [[LogicalPlan]] represents the given table or view.
+   * @param name        tableName
+   * @param sparkSession
+   * @return
+   */
+  def refreshRelation(rtnRelation: LogicalPlan, name: TableIdentifier)
+    (sparkSession: SparkSession): Boolean = {
+    var isRelationRefreshed = false
+    rtnRelation match {
+      case SubqueryAlias(_,
+      MatchLogicalRelation(_: CarbonDatasourceHadoopRelation, _, _)
+      ) =>
+        isRelationRefreshed = CarbonEnv.refreshRelationFromCache(name)(sparkSession)
+      case MatchLogicalRelation(_: CarbonDatasourceHadoopRelation, _, _) =>
+        isRelationRefreshed = CarbonEnv.refreshRelationFromCache(name)(sparkSession)
+      case SubqueryAlias(_, relation) if
+      relation.getClass.getName.equals("org.apache.spark.sql.catalyst.catalog.CatalogRelation") ||
+      relation.getClass.getName
+        .equals("org.apache.spark.sql.catalyst.catalog.HiveTableRelation") ||
+      relation.getClass.getName.equals(
+        "org.apache.spark.sql.catalyst.catalog.UnresolvedCatalogRelation"
+      ) =>
+        val catalogTable =
+          CarbonReflectionUtils.getFieldOfCatalogTable(
+            "tableMeta",
+            relation
+          ).asInstanceOf[CatalogTable]
+        isRelationRefreshed =
+          CarbonEnv.refreshRelationFromCache(catalogTable.identifier)(sparkSession)
+      case _ =>
+    }
+    isRelationRefreshed
+  }
+
+  /**
+   * This is alternate way of getting partition information. It first fetches all partitions from
+   * hive and then apply filter instead of querying hive along with filters.
+   *
+   * @param partitionFilters
+   * @param sparkSession
+   * @param identifier
+   * @return
+   */
+  def prunePartitionsByFilter(partitionFilters: Seq[Expression],
+      sparkSession: SparkSession,
+      identifier: TableIdentifier): Seq[CatalogTablePartition] = {
+    val allPartitions = sparkSession.sessionState.catalog.listPartitions(identifier)
+    ExternalCatalogUtils.prunePartitionsByFilter(
+      sparkSession.sessionState.catalog.getTableMetadata(identifier),
+      allPartitions,
+      partitionFilters,
+      sparkSession.sessionState.conf.sessionLocalTimeZone
+    )
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonSqlConf.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonSqlConf.scala b/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonSqlConf.scala
new file mode 100644
index 0000000..2128ffd
--- /dev/null
+++ b/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonSqlConf.scala
@@ -0,0 +1,148 @@
+/*
+ * 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.spark.sql.hive
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.internal.SQLConf.buildConf
+
+import org.apache.carbondata.core.constants.{CarbonCommonConstants, CarbonLoadOptionConstants}
+import org.apache.carbondata.core.util.CarbonProperties
+
+/**
+ * To initialize dynamic values default param
+ */
+class CarbonSQLConf(sparkSession: SparkSession) {
+
+  val carbonProperties = CarbonProperties.getInstance()
+
+  /**
+   * To initialize dynamic param defaults along with usage docs
+   */
+  def addDefaultCarbonParams(): Unit = {
+    val ENABLE_UNSAFE_SORT =
+      buildConf(CarbonCommonConstants.ENABLE_UNSAFE_SORT)
+        .doc("To enable/ disable unsafe sort.")
+        .booleanConf
+        .createWithDefault(carbonProperties.getProperty(CarbonCommonConstants.ENABLE_UNSAFE_SORT,
+          CarbonCommonConstants.ENABLE_UNSAFE_SORT_DEFAULT).toBoolean)
+    val CARBON_CUSTOM_BLOCK_DISTRIBUTION =
+      buildConf(CarbonCommonConstants.CARBON_CUSTOM_BLOCK_DISTRIBUTION)
+        .doc("To set carbon task distribution.")
+        .stringConf
+        .createWithDefault(carbonProperties
+          .getProperty(CarbonCommonConstants.CARBON_TASK_DISTRIBUTION,
+            CarbonCommonConstants.CARBON_TASK_DISTRIBUTION_DEFAULT))
+    val BAD_RECORDS_LOGGER_ENABLE =
+      buildConf(CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORDS_LOGGER_ENABLE)
+        .doc("To enable/ disable carbon bad record logger.")
+        .booleanConf
+        .createWithDefault(CarbonLoadOptionConstants
+          .CARBON_OPTIONS_BAD_RECORDS_LOGGER_ENABLE_DEFAULT.toBoolean)
+    val BAD_RECORDS_ACTION =
+      buildConf(CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORDS_ACTION)
+        .doc("To configure the bad records action.")
+        .stringConf
+        .createWithDefault(carbonProperties
+          .getProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION,
+            CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION_DEFAULT))
+    val IS_EMPTY_DATA_BAD_RECORD =
+      buildConf(CarbonLoadOptionConstants.CARBON_OPTIONS_IS_EMPTY_DATA_BAD_RECORD)
+        .doc("Property to decide weather empty data to be considered bad/ good record.")
+        .booleanConf
+        .createWithDefault(CarbonLoadOptionConstants.CARBON_OPTIONS_IS_EMPTY_DATA_BAD_RECORD_DEFAULT
+          .toBoolean)
+    val SORT_SCOPE =
+      buildConf(CarbonLoadOptionConstants.CARBON_OPTIONS_SORT_SCOPE)
+        .doc("Property to specify sort scope.")
+        .stringConf
+        .createWithDefault(carbonProperties.getProperty(CarbonCommonConstants.LOAD_SORT_SCOPE,
+          CarbonCommonConstants.LOAD_SORT_SCOPE_DEFAULT))
+    val BATCH_SORT_SIZE_INMB =
+      buildConf(CarbonLoadOptionConstants.CARBON_OPTIONS_BATCH_SORT_SIZE_INMB)
+        .doc("Property to specify batch sort size in MB.")
+        .stringConf
+        .createWithDefault(carbonProperties
+          .getProperty(CarbonCommonConstants.LOAD_BATCH_SORT_SIZE_INMB,
+            CarbonCommonConstants.LOAD_BATCH_SORT_SIZE_INMB_DEFAULT))
+    val SINGLE_PASS =
+      buildConf(CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS)
+        .doc("Property to enable/disable single_pass.")
+        .booleanConf
+        .createWithDefault(CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS_DEFAULT.toBoolean)
+    val BAD_RECORD_PATH =
+      buildConf(CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORD_PATH)
+        .doc("Property to configure the bad record location.")
+        .stringConf
+        .createWithDefault(carbonProperties.getProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC,
+          CarbonCommonConstants.CARBON_BADRECORDS_LOC_DEFAULT_VAL))
+    val GLOBAL_SORT_PARTITIONS =
+      buildConf(CarbonLoadOptionConstants.CARBON_OPTIONS_GLOBAL_SORT_PARTITIONS)
+        .doc("Property to configure the global sort partitions.")
+        .stringConf
+        .createWithDefault(carbonProperties
+          .getProperty(CarbonCommonConstants.LOAD_GLOBAL_SORT_PARTITIONS,
+            CarbonCommonConstants.LOAD_GLOBAL_SORT_PARTITIONS_DEFAULT))
+    val DATEFORMAT =
+      buildConf(CarbonLoadOptionConstants.CARBON_OPTIONS_DATEFORMAT)
+        .doc("Property to configure data format for date type columns.")
+        .stringConf
+        .createWithDefault(CarbonLoadOptionConstants.CARBON_OPTIONS_DATEFORMAT_DEFAULT)
+    val CARBON_INPUT_SEGMENTS = buildConf(
+      "carbon.input.segments.<database_name>.<table_name>")
+      .doc("Property to configure the list of segments to query.").stringConf
+      .createWithDefault(carbonProperties
+        .getProperty("carbon.input.segments.<database_name>.<table_name>", "*"))
+  }
+  /**
+   * to set the dynamic properties default values
+   */
+  def addDefaultCarbonSessionParams(): Unit = {
+    sparkSession.conf.set(CarbonCommonConstants.ENABLE_UNSAFE_SORT,
+      carbonProperties.getProperty(CarbonCommonConstants.ENABLE_UNSAFE_SORT,
+        CarbonCommonConstants.ENABLE_UNSAFE_SORT_DEFAULT).toBoolean)
+    sparkSession.conf.set(CarbonCommonConstants.CARBON_CUSTOM_BLOCK_DISTRIBUTION,
+      carbonProperties
+        .getProperty(CarbonCommonConstants.CARBON_TASK_DISTRIBUTION,
+          CarbonCommonConstants.CARBON_TASK_DISTRIBUTION_DEFAULT))
+    sparkSession.conf.set(CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORDS_LOGGER_ENABLE,
+      CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORDS_LOGGER_ENABLE_DEFAULT.toBoolean)
+    sparkSession.conf.set(CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORDS_ACTION,
+      carbonProperties.getProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION,
+        CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION_DEFAULT))
+    sparkSession.conf.set(CarbonLoadOptionConstants.CARBON_OPTIONS_IS_EMPTY_DATA_BAD_RECORD,
+      CarbonLoadOptionConstants.CARBON_OPTIONS_IS_EMPTY_DATA_BAD_RECORD_DEFAULT.toBoolean)
+    sparkSession.conf.set(CarbonLoadOptionConstants.CARBON_OPTIONS_SORT_SCOPE,
+      carbonProperties.getProperty(CarbonCommonConstants.LOAD_SORT_SCOPE,
+        CarbonCommonConstants.LOAD_SORT_SCOPE_DEFAULT))
+    sparkSession.conf.set(CarbonLoadOptionConstants.CARBON_OPTIONS_BATCH_SORT_SIZE_INMB,
+      carbonProperties.getProperty(CarbonCommonConstants.LOAD_BATCH_SORT_SIZE_INMB,
+        CarbonCommonConstants.LOAD_BATCH_SORT_SIZE_INMB_DEFAULT))
+    sparkSession.conf.set(CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS,
+      CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS_DEFAULT.toBoolean)
+    sparkSession.conf.set(CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORD_PATH,
+      carbonProperties.getProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC,
+        CarbonCommonConstants.CARBON_BADRECORDS_LOC_DEFAULT_VAL))
+    sparkSession.conf.set(CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORD_PATH,
+      carbonProperties.getProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC,
+        CarbonCommonConstants.CARBON_BADRECORDS_LOC_DEFAULT_VAL))
+    sparkSession.conf.set(CarbonLoadOptionConstants.CARBON_OPTIONS_GLOBAL_SORT_PARTITIONS,
+      carbonProperties.getProperty(CarbonCommonConstants.LOAD_GLOBAL_SORT_PARTITIONS,
+        CarbonCommonConstants.LOAD_GLOBAL_SORT_PARTITIONS_DEFAULT))
+    sparkSession.conf.set(CarbonLoadOptionConstants.CARBON_OPTIONS_DATEFORMAT,
+      CarbonLoadOptionConstants.CARBON_OPTIONS_DATEFORMAT_DEFAULT)
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CreateCarbonSourceTableAsSelectCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CreateCarbonSourceTableAsSelectCommand.scala b/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CreateCarbonSourceTableAsSelectCommand.scala
new file mode 100644
index 0000000..74c0f97
--- /dev/null
+++ b/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CreateCarbonSourceTableAsSelectCommand.scala
@@ -0,0 +1,130 @@
+
+/*
+ * 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.spark.sql.hive
+
+import java.net.URI
+
+import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType, CatalogUtils}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.command.{AlterTableRecoverPartitionsCommand, RunnableCommand}
+import org.apache.spark.sql.execution.datasources.{DataSource, HadoopFsRelation}
+import org.apache.spark.sql.sources.BaseRelation
+import org.apache.spark.sql.{AnalysisException, Dataset, Row, SaveMode, SparkSession}
+import org.apache.spark.util.CarbonReflectionUtils
+
+/**
+ * Create table 'using carbondata' and insert the query result into it.
+ *
+ * @param table the Catalog Table
+ * @param mode  SaveMode:Ignore,OverWrite,ErrorIfExists,Append
+ * @param query the query whose result will be insert into the new relation
+ *
+ */
+
+case class CreateCarbonSourceTableAsSelectCommand(
+    table: CatalogTable,
+    mode: SaveMode,
+    query: LogicalPlan)
+  extends RunnableCommand {
+
+  override protected def innerChildren: Seq[LogicalPlan] = Seq(query)
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    assert(table.tableType != CatalogTableType.VIEW)
+    assert(table.provider.isDefined)
+
+    val sessionState = sparkSession.sessionState
+    val db = table.identifier.database.getOrElse(sessionState.catalog.getCurrentDatabase)
+    val tableIdentWithDB = table.identifier.copy(database = Some(db))
+    val tableName = tableIdentWithDB.unquotedString
+
+    if (sessionState.catalog.tableExists(tableIdentWithDB)) {
+      assert(mode != SaveMode.Overwrite,
+        s"Expect the table $tableName has been dropped when the save mode is Overwrite")
+
+      if (mode == SaveMode.ErrorIfExists) {
+        throw new AnalysisException(s"Table $tableName already exists. You need to drop it first.")
+      }
+      if (mode == SaveMode.Ignore) {
+        // Since the table already exists and the save mode is Ignore, we will just return.
+        return Seq.empty
+      }
+
+      saveDataIntoTable(
+        sparkSession, table, table.storage.locationUri, query, SaveMode.Append, tableExists = true)
+    } else {
+      assert(table.schema.isEmpty)
+
+      val tableLocation = if (table.tableType == CatalogTableType.MANAGED) {
+        Some(sessionState.catalog.defaultTablePath(table.identifier))
+      } else {
+        table.storage.locationUri
+      }
+      val result = saveDataIntoTable(
+        sparkSession, table, tableLocation, query, SaveMode.Overwrite, tableExists = false)
+
+      result match {
+        case fs: HadoopFsRelation if table.partitionColumnNames.nonEmpty &&
+                                     sparkSession.sqlContext.conf.manageFilesourcePartitions =>
+          // Need to recover partitions into the metastore so our saved data is visible.
+          sessionState.executePlan(AlterTableRecoverPartitionsCommand(table.identifier)).toRdd
+        case _ =>
+      }
+    }
+
+    Seq.empty[Row]
+  }
+
+  private def saveDataIntoTable(
+      session: SparkSession,
+      table: CatalogTable,
+      tableLocation: Option[URI],
+      data: LogicalPlan,
+      mode: SaveMode,
+      tableExists: Boolean): BaseRelation = {
+    // Create the relation based on the input logical plan: `data`.
+    val pathOption = tableLocation.map("path" -> CatalogUtils.URIToString(_))
+    val dataSource = DataSource(
+      session,
+      className = table.provider.get,
+      partitionColumns = table.partitionColumnNames,
+      bucketSpec = table.bucketSpec,
+      options = table.storage.properties ++ pathOption,
+      catalogTable = if (tableExists) {
+        Some(table)
+      } else {
+        None
+      })
+
+    try {
+      val physicalPlan = session.sessionState.executePlan(data).executedPlan
+      CarbonReflectionUtils.invokewriteAndReadMethod(dataSource,
+        Dataset.ofRows(session, query),
+        data,
+        session,
+        mode,
+        query,
+        physicalPlan)
+    } catch {
+      case ex: AnalysisException =>
+        logError(s"Failed to write to table ${ table.identifier.unquotedString }", ex)
+        throw ex
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/SqlAstBuilderHelper.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/SqlAstBuilderHelper.scala b/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/SqlAstBuilderHelper.scala
new file mode 100644
index 0000000..8f1477c
--- /dev/null
+++ b/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/SqlAstBuilderHelper.scala
@@ -0,0 +1,110 @@
+/*
+ * 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.spark.sql.hive
+
+import org.apache.spark.sql.catalyst.parser.ParserUtils.{string, withOrigin}
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{AddTableColumnsContext,ChangeColumnContext, CreateTableContext, ShowTablesContext}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.SparkSqlAstBuilder
+import org.apache.spark.sql.execution.command.{AlterTableAddColumnsModel,AlterTableDataTypeChangeModel}
+import org.apache.spark.sql.execution.command.schema.{CarbonAlterTableAddColumnCommand,CarbonAlterTableDataTypeChangeCommand}
+import org.apache.spark.sql.execution.command.table.{CarbonExplainCommand, CarbonShowTablesCommand}
+import org.apache.spark.sql.parser.CarbonSpark2SqlParser
+import org.apache.spark.sql.types.DecimalType
+
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+
+trait SqlAstBuilderHelper extends SparkSqlAstBuilder {
+
+
+  override def visitChangeColumn(ctx: ChangeColumnContext): LogicalPlan = {
+
+    val newColumn = visitColType(ctx.colType)
+    if (!ctx.identifier.getText.equalsIgnoreCase(newColumn.name)) {
+      throw new MalformedCarbonCommandException(
+        "Column names provided are different. Both the column names should be same")
+    }
+
+    val (typeString, values): (String, Option[List[(Int, Int)]]) = newColumn.dataType match {
+      case d: DecimalType => ("decimal", Some(List((d.precision, d.scale))))
+      case _ => (newColumn.dataType.typeName.toLowerCase, None)
+    }
+
+    val alterTableChangeDataTypeModel =
+      AlterTableDataTypeChangeModel(new CarbonSpark2SqlParser().parseDataType(typeString, values),
+        new CarbonSpark2SqlParser()
+          .convertDbNameToLowerCase(Option(ctx.tableIdentifier().db).map(_.getText)),
+        ctx.tableIdentifier().table.getText.toLowerCase,
+        ctx.identifier.getText.toLowerCase,
+        newColumn.name.toLowerCase)
+
+    CarbonAlterTableDataTypeChangeCommand(alterTableChangeDataTypeModel)
+  }
+
+
+  def visitAddTableColumns(parser: CarbonSpark2SqlParser,
+      ctx: AddTableColumnsContext): LogicalPlan = {
+    val cols = Option(ctx.columns).toSeq.flatMap(visitColTypeList)
+    val fields = parser.getFields(cols)
+    val tblProperties = scala.collection.mutable.Map.empty[String, String]
+    val tableModel = new CarbonSpark2SqlParser().prepareTableModel(false,
+      new CarbonSpark2SqlParser().convertDbNameToLowerCase(Option(ctx.tableIdentifier().db)
+        .map(_.getText)),
+      ctx.tableIdentifier.table.getText.toLowerCase,
+      fields,
+      Seq.empty,
+      tblProperties,
+      None,
+      true)
+
+    val alterTableAddColumnsModel = AlterTableAddColumnsModel(
+      Option(ctx.tableIdentifier().db).map(_.getText),
+      ctx.tableIdentifier.table.getText,
+      tblProperties.toMap,
+      tableModel.dimCols,
+      tableModel.msrCols,
+      tableModel.highcardinalitydims.getOrElse(Seq.empty))
+
+    CarbonAlterTableAddColumnCommand(alterTableAddColumnsModel)
+  }
+
+  override def visitCreateTable(ctx: CreateTableContext): LogicalPlan = {
+    super.visitCreateTable(ctx)
+  }
+
+  override def visitShowTables(ctx: ShowTablesContext): LogicalPlan = {
+    withOrigin(ctx) {
+      if (CarbonProperties.getInstance()
+        .getProperty(CarbonCommonConstants.CARBON_SHOW_DATAMAPS,
+          CarbonCommonConstants.CARBON_SHOW_DATAMAPS_DEFAULT).toBoolean) {
+        super.visitShowTables(ctx)
+      } else {
+        CarbonShowTablesCommand(
+          Option(ctx.db).map(_.getText),
+          Option(ctx.pattern).map(string))
+      }
+    }
+  }
+
+  override def visitExplain(ctx: SqlBaseParser.ExplainContext): LogicalPlan = {
+    CarbonExplainCommand(super.visitExplain(ctx))
+  }
+}


[4/8] carbondata git commit: [CARBONDATA-2532][Integration] Carbon to support spark 2.3.1 version(Make API changes in carbon to be compatible with spark 2.3)

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonAnalyzer.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonAnalyzer.scala b/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonAnalyzer.scala
deleted file mode 100644
index dfb89fd..0000000
--- a/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonAnalyzer.scala
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.spark.sql.hive
-
-import org.apache.spark.sql.SparkSession
-import org.apache.spark.sql.catalyst.analysis.Analyzer
-import org.apache.spark.sql.catalyst.catalog.SessionCatalog
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.util.CarbonReflectionUtils
-
-class CarbonAnalyzer(catalog: SessionCatalog,
-    conf: SQLConf,
-    sparkSession: SparkSession,
-    analyzer: Analyzer) extends Analyzer(catalog, conf) {
-
-  val mvPlan = try {
-    CarbonReflectionUtils.createObject(
-      "org.apache.carbondata.mv.datamap.MVAnalyzerRule",
-      sparkSession)._1.asInstanceOf[Rule[LogicalPlan]]
-  } catch {
-    case e: Exception =>
-      null
-  }
-
-  override def execute(plan: LogicalPlan): LogicalPlan = {
-    var logicalPlan = analyzer.execute(plan)
-    logicalPlan = CarbonPreAggregateDataLoadingRules(sparkSession).apply(logicalPlan)
-    logicalPlan = CarbonPreAggregateQueryRules(sparkSession).apply(logicalPlan)
-    if (mvPlan != null) {
-      mvPlan.apply(logicalPlan)
-    } else {
-      logicalPlan
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonInMemorySessionState.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonInMemorySessionState.scala b/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonInMemorySessionState.scala
deleted file mode 100644
index ba6aae5..0000000
--- a/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonInMemorySessionState.scala
+++ /dev/null
@@ -1,278 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.spark.sql.hive
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.Path
-import org.apache.spark.sql.carbondata.execution.datasources.CarbonSparkDataSourceUtil
-import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry}
-import org.apache.spark.sql.catalyst.catalog._
-import org.apache.spark.sql.catalyst.expressions.Expression
-import org.apache.spark.sql.catalyst.optimizer.Optimizer
-import org.apache.spark.sql.catalyst.parser.ParserInterface
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.execution.datasources._
-import org.apache.spark.sql.execution.strategy.{CarbonLateDecodeStrategy, DDLStrategy, StreamingTableStrategy}
-import org.apache.spark.sql.internal.{SQLConf, SessionResourceLoader, SessionState, SessionStateBuilder}
-import org.apache.spark.sql.optimizer.{CarbonIUDRule, CarbonLateDecodeRule, CarbonUDFTransformRule}
-import org.apache.spark.sql.parser.CarbonSparkSqlParser
-import org.apache.spark.sql.types.{StructField, StructType}
-import org.apache.spark.sql.{CarbonEnv, SparkSession}
-
-import org.apache.carbondata.core.util.CarbonUtil
-import org.apache.carbondata.core.util.path.CarbonTablePath
-import org.apache.carbondata.format.TableInfo
-import org.apache.carbondata.spark.util.CarbonScalaUtil
-
-/**
- * This class will have carbon catalog and refresh the relation from cache if the carbontable in
- * carbon catalog is not same as cached carbon relation's carbon table
- *
- * @param externalCatalog
- * @param globalTempViewManager
- * @param sparkSession
- * @param functionResourceLoader
- * @param functionRegistry
- * @param conf
- * @param hadoopConf
- */
-class InMemorySessionCatalog(
-    externalCatalog: ExternalCatalog,
-    globalTempViewManager: GlobalTempViewManager,
-    functionRegistry: FunctionRegistry,
-    sparkSession: SparkSession,
-    conf: SQLConf,
-    hadoopConf: Configuration,
-    parser: ParserInterface,
-    functionResourceLoader: FunctionResourceLoader)
-  extends SessionCatalog(
-    externalCatalog,
-    globalTempViewManager,
-    functionRegistry,
-    conf,
-    hadoopConf,
-    parser,
-    functionResourceLoader
-  ) with CarbonSessionCatalog {
-
-  override def alterTableRename(oldTableIdentifier: TableIdentifier,
-      newTableIdentifier: TableIdentifier,
-      newTablePath: String): Unit = {
-    sparkSession.sessionState.catalog.renameTable(oldTableIdentifier, newTableIdentifier)
-  }
-
-  override def alterTable(tableIdentifier: TableIdentifier,
-      schemaParts: String,
-      cols: Option[Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]])
-  : Unit = {
-    // NOt Required in case of In-memory catalog
-  }
-
-  override def alterAddColumns(tableIdentifier: TableIdentifier,
-      schemaParts: String,
-      newColumns: Option[Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]])
-  : Unit = {
-    val catalogTable = sparkSession.sessionState.catalog.getTableMetadata(tableIdentifier)
-    val structType = catalogTable.schema
-    var newStructType = structType
-    newColumns.get.foreach {cols =>
-      newStructType = structType
-        .add(cols.getColumnName,
-          CarbonSparkDataSourceUtil.convertCarbonToSparkDataType(cols.getDataType))
-    }
-    alterSchema(newStructType, catalogTable, tableIdentifier)
-  }
-
-  override def alterDropColumns(tableIdentifier: TableIdentifier,
-      schemaParts: String,
-      dropCols: Option[Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]])
-  : Unit = {
-    val catalogTable = sparkSession.sessionState.catalog.getTableMetadata(tableIdentifier)
-    val fields = catalogTable.schema.fields.filterNot { field =>
-      dropCols.get.exists { col =>
-        col.getColumnName.equalsIgnoreCase(field.name)
-      }
-    }
-    alterSchema(new StructType(fields), catalogTable, tableIdentifier)
-  }
-
-  override def alterColumnChangeDataType(tableIdentifier: TableIdentifier,
-      schemaParts: String,
-      columns: Option[Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]])
-  : Unit = {
-    val catalogTable = sparkSession.sessionState.catalog.getTableMetadata(tableIdentifier)
-    val a = catalogTable.schema.fields.flatMap { field =>
-      columns.get.map { col =>
-        if (col.getColumnName.equalsIgnoreCase(field.name)) {
-          StructField(col.getColumnName,
-            CarbonSparkDataSourceUtil.convertCarbonToSparkDataType(col.getDataType))
-        } else {
-          field
-        }
-      }
-    }
-    alterSchema(new StructType(a), catalogTable, tableIdentifier)
-  }
-
-  private def alterSchema(structType: StructType,
-      catalogTable: CatalogTable,
-      tableIdentifier: TableIdentifier): Unit = {
-    val copy = catalogTable.copy(schema = structType)
-    sparkSession.sessionState.catalog.alterTable(copy)
-    sparkSession.sessionState.catalog.refreshTable(tableIdentifier)
-  }
-
-  lazy val carbonEnv = {
-    val env = new CarbonEnv
-    env.init(sparkSession)
-    env
-  }
-
-  def getCarbonEnv() : CarbonEnv = {
-    carbonEnv
-  }
-
-  // Initialize all listeners to the Operation bus.
-  CarbonEnv.initListeners()
-
-  def getThriftTableInfo(tablePath: String): TableInfo = {
-    val tableMetadataFile = CarbonTablePath.getSchemaFilePath(tablePath)
-    CarbonUtil.readSchemaFile(tableMetadataFile)
-  }
-
-  override def lookupRelation(name: TableIdentifier): LogicalPlan = {
-    val rtnRelation = super.lookupRelation(name)
-    val isRelationRefreshed =
-      CarbonSessionUtil.refreshRelation(rtnRelation, name)(sparkSession)
-    if (isRelationRefreshed) {
-      super.lookupRelation(name)
-    } else {
-      rtnRelation
-    }
-  }
-
-  /**
-   * returns hive client from HiveExternalCatalog
-   *
-   * @return
-   */
-  def getClient(): org.apache.spark.sql.hive.client.HiveClient = {
-    null
-  }
-
-  override def createPartitions(
-      tableName: TableIdentifier,
-      parts: Seq[CatalogTablePartition],
-      ignoreIfExists: Boolean): Unit = {
-    try {
-      val table = CarbonEnv.getCarbonTable(tableName)(sparkSession)
-      val updatedParts = CarbonScalaUtil.updatePartitions(parts, table)
-      super.createPartitions(tableName, updatedParts, ignoreIfExists)
-    } catch {
-      case e: Exception =>
-        super.createPartitions(tableName, parts, ignoreIfExists)
-    }
-  }
-
-  /**
-   * This is alternate way of getting partition information. It first fetches all partitions from
-   * hive and then apply filter instead of querying hive along with filters.
-   * @param partitionFilters
-   * @param sparkSession
-   * @param identifier
-   * @return
-   */
-  override def getPartitionsAlternate(partitionFilters: Seq[Expression],
-      sparkSession: SparkSession,
-      identifier: TableIdentifier) = {
-    CarbonSessionUtil.prunePartitionsByFilter(partitionFilters, sparkSession, identifier)
-  }
-
-  /**
-   * Update the storageformat with new location information
-   */
-  override def updateStorageLocation(
-      path: Path,
-      storage: CatalogStorageFormat,
-      newTableName: String,
-      dbName: String): CatalogStorageFormat = {
-    storage.copy(locationUri = Some(path.toUri))
-  }
-}
-
-class CarbonInMemorySessionStateBuilder (sparkSession: SparkSession,
-    parentState: Option[SessionState] = None)
-  extends SessionStateBuilder(sparkSession, parentState) {
-
-  override lazy val sqlParser: ParserInterface = new CarbonSparkSqlParser(conf, sparkSession)
-
-  experimentalMethods.extraStrategies =
-    Seq(new StreamingTableStrategy(sparkSession),
-      new CarbonLateDecodeStrategy,
-      new DDLStrategy(sparkSession)
-    )
-  experimentalMethods.extraOptimizations = Seq(new CarbonIUDRule,
-    new CarbonUDFTransformRule,
-    new CarbonLateDecodeRule)
-
-  /**
-   * Internal catalog for managing table and database states.
-   */
-  override protected lazy val catalog: InMemorySessionCatalog = {
-    val catalog = new InMemorySessionCatalog(
-      externalCatalog,
-      session.sharedState.globalTempViewManager,
-      functionRegistry,
-      sparkSession,
-      conf,
-      SessionState.newHadoopConf(session.sparkContext.hadoopConfiguration, conf),
-      sqlParser,
-      resourceLoader)
-    parentState.foreach(_.catalog.copyStateTo(catalog))
-    catalog
-  }
-
-  private def externalCatalog: ExternalCatalog =
-    session.sharedState.externalCatalog.asInstanceOf[ExternalCatalog]
-
-  override protected lazy val resourceLoader: SessionResourceLoader = {
-    new SessionResourceLoader(session)
-  }
-
-  override lazy val optimizer: Optimizer = new CarbonOptimizer(catalog, conf, experimentalMethods)
-
-  override protected def analyzer: Analyzer = new CarbonAnalyzer(catalog, conf, sparkSession,
-    new Analyzer(catalog, conf) {
-      override val extendedResolutionRules: Seq[Rule[LogicalPlan]] =
-        new FindDataSourceTable(session) +:
-        new ResolveSQLOnFile(session) +:
-        new CarbonIUDAnalysisRule(sparkSession) +:
-        new CarbonPreInsertionCasts(sparkSession) +: customResolutionRules
-      override val extendedCheckRules: Seq[LogicalPlan => Unit] =
-        PreWriteCheck :: HiveOnlyCheck :: Nil
-      override val postHocResolutionRules: Seq[Rule[LogicalPlan]] =
-        PreprocessTableCreation(session) +:
-        PreprocessTableInsertion(conf) +:
-        DataSourceAnalysis(conf) +:
-        customPostHocResolutionRules
-    }
-  )
-  override protected def newBuilder: NewBuilder = new CarbonInMemorySessionStateBuilder(_, _)
-}
-

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonOptimizer.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonOptimizer.scala b/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonOptimizer.scala
index a046763..2f9ad3e 100644
--- a/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonOptimizer.scala
+++ b/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonOptimizer.scala
@@ -17,12 +17,10 @@
 
 package org.apache.spark.sql.hive
 
-import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, ExperimentalMethods}
+import org.apache.spark.sql.ExperimentalMethods
 import org.apache.spark.sql.catalyst.catalog.SessionCatalog
-import org.apache.spark.sql.catalyst.expressions.{Exists, In, ListQuery, ScalarSubquery}
-import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.execution.SparkOptimizer
-import org.apache.spark.sql.execution.datasources.LogicalRelation
 import org.apache.spark.sql.internal.SQLConf
 
 
@@ -37,41 +35,3 @@ class CarbonOptimizer(
     super.execute(transFormedPlan)
   }
 }
-
-object CarbonOptimizerUtil {
-  def transformForScalarSubQuery(plan: LogicalPlan): LogicalPlan = {
-    // In case scalar subquery add flag in relation to skip the decoder plan in optimizer rule, And
-    // optimize whole plan at once.
-    val transFormedPlan = plan.transform {
-      case filter: Filter =>
-        filter.transformExpressions {
-          case s: ScalarSubquery =>
-            val tPlan = s.plan.transform {
-              case lr: LogicalRelation
-                if lr.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
-                lr.relation.asInstanceOf[CarbonDatasourceHadoopRelation].isSubquery += true
-                lr
-            }
-            ScalarSubquery(tPlan, s.children, s.exprId)
-          case e: Exists =>
-            val tPlan = e.plan.transform {
-              case lr: LogicalRelation
-                if lr.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
-                lr.relation.asInstanceOf[CarbonDatasourceHadoopRelation].isSubquery += true
-                lr
-            }
-            Exists(tPlan, e.children.map(_.canonicalized), e.exprId)
-
-          case In(value, Seq(l@ListQuery(sub, _, exprId))) =>
-            val tPlan = sub.transform {
-              case lr: LogicalRelation
-                if lr.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
-                lr.relation.asInstanceOf[CarbonDatasourceHadoopRelation].isSubquery += true
-                lr
-            }
-            In(value, Seq(ListQuery(tPlan, l.children, exprId)))
-        }
-    }
-    transFormedPlan
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonSessionState.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonSessionState.scala b/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonSessionState.scala
deleted file mode 100644
index a7a255e..0000000
--- a/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonSessionState.scala
+++ /dev/null
@@ -1,277 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.spark.sql.hive
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.Path
-import org.apache.spark.sql._
-import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry}
-import org.apache.spark.sql.catalyst.catalog._
-import org.apache.spark.sql.catalyst.expressions.Expression
-import org.apache.spark.sql.catalyst.optimizer.Optimizer
-import org.apache.spark.sql.catalyst.parser.ParserInterface
-import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan}
-import org.apache.spark.sql.catalyst.parser.{ParserInterface, SqlBaseParser}
-import org.apache.spark.sql.catalyst.parser.ParserUtils.{string, withOrigin}
-import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{AddTableColumnsContext, ChangeColumnContext, CreateHiveTableContext, CreateTableContext, ShowTablesContext}
-import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, SubqueryAlias}
-import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.execution.datasources.{FindDataSourceTable, PreWriteCheck, ResolveSQLOnFile, _}
-import org.apache.spark.sql.execution.command._
-import org.apache.spark.sql.execution.command.schema.{CarbonAlterTableAddColumnCommand, CarbonAlterTableDataTypeChangeCommand}
-import org.apache.spark.sql.execution.command.table.{CarbonExplainCommand, CarbonShowTablesCommand}
-import org.apache.spark.sql.execution.datasources.{FindDataSourceTable, LogicalRelation, PreWriteCheck, ResolveSQLOnFile, _}
-import org.apache.spark.sql.execution.strategy.{CarbonLateDecodeStrategy, DDLStrategy, StreamingTableStrategy}
-import org.apache.spark.sql.hive.client.HiveClient
-import org.apache.spark.sql.internal.{SQLConf, SessionState}
-import org.apache.spark.sql.optimizer.{CarbonIUDRule, CarbonLateDecodeRule, CarbonUDFTransformRule}
-import org.apache.spark.sql.parser.CarbonSparkSqlParser
-
-import org.apache.carbondata.spark.util.CarbonScalaUtil
-
-/**
- * This class will have carbon catalog and refresh the relation from cache if the carbontable in
- * carbon catalog is not same as cached carbon relation's carbon table
- *
- * @param externalCatalog
- * @param globalTempViewManager
- * @param sparkSession
- * @param functionResourceLoader
- * @param functionRegistry
- * @param conf
- * @param hadoopConf
- */
-class CarbonHiveSessionCatalog(
-    externalCatalog: HiveExternalCatalog,
-    globalTempViewManager: GlobalTempViewManager,
-    functionRegistry: FunctionRegistry,
-    sparkSession: SparkSession,
-    conf: SQLConf,
-    hadoopConf: Configuration,
-    parser: ParserInterface,
-    functionResourceLoader: FunctionResourceLoader)
-  extends HiveSessionCatalog (
-    externalCatalog,
-    globalTempViewManager,
-    new HiveMetastoreCatalog(sparkSession),
-    functionRegistry,
-    conf,
-    hadoopConf,
-    parser,
-    functionResourceLoader
-  ) with CarbonSessionCatalog {
-
-  private lazy val carbonEnv = {
-    val env = new CarbonEnv
-    env.init(sparkSession)
-    env
-  }
-  /**
-   * return's the carbonEnv instance
-   * @return
-   */
-  override def getCarbonEnv() : CarbonEnv = {
-    carbonEnv
-  }
-
-  // Initialize all listeners to the Operation bus.
-  CarbonEnv.initListeners()
-
-  override def lookupRelation(name: TableIdentifier): LogicalPlan = {
-    val rtnRelation = super.lookupRelation(name)
-    val isRelationRefreshed =
-      CarbonSessionUtil.refreshRelation(rtnRelation, name)(sparkSession)
-    if (isRelationRefreshed) {
-      super.lookupRelation(name)
-    } else {
-      rtnRelation
-    }
-  }
-
-  /**
-   * returns hive client from HiveExternalCatalog
-   *
-   * @return
-   */
-  override def getClient(): org.apache.spark.sql.hive.client.HiveClient = {
-    sparkSession.asInstanceOf[CarbonSession].sharedState.externalCatalog
-      .asInstanceOf[HiveExternalCatalog].client
-  }
-
-  def alterTableRename(oldTableIdentifier: TableIdentifier,
-      newTableIdentifier: TableIdentifier,
-      newTablePath: String): Unit = {
-    getClient().runSqlHive(
-      s"ALTER TABLE ${ oldTableIdentifier.database.get }.${ oldTableIdentifier.table } " +
-      s"RENAME TO ${ oldTableIdentifier.database.get }.${ newTableIdentifier.table }")
-    getClient().runSqlHive(
-      s"ALTER TABLE ${ oldTableIdentifier.database.get }.${ newTableIdentifier.table} " +
-      s"SET SERDEPROPERTIES" +
-      s"('tableName'='${ newTableIdentifier.table }', " +
-      s"'dbName'='${ oldTableIdentifier.database.get }', 'tablePath'='${ newTablePath }')")
-  }
-
-  override def alterTable(tableIdentifier: TableIdentifier,
-      schemaParts: String,
-      cols: Option[Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]])
-  : Unit = {
-    getClient()
-      .runSqlHive(s"ALTER TABLE ${tableIdentifier.database.get}.${ tableIdentifier.table } " +
-                  s"SET TBLPROPERTIES(${ schemaParts })")
-  }
-
-  override def alterAddColumns(tableIdentifier: TableIdentifier,
-      schemaParts: String,
-      cols: Option[Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]])
-  : Unit = {
-    alterTable(tableIdentifier, schemaParts, cols)
-  }
-
-  override def alterDropColumns(tableIdentifier: TableIdentifier,
-      schemaParts: String,
-      cols: Option[Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]])
-  : Unit = {
-    alterTable(tableIdentifier, schemaParts, cols)
-  }
-
-  override def alterColumnChangeDataType(tableIdentifier: TableIdentifier,
-      schemaParts: String,
-      cols: Option[Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]])
-  : Unit = {
-    alterTable(tableIdentifier, schemaParts, cols)
-  }
-
-  override def createPartitions(
-      tableName: TableIdentifier,
-      parts: Seq[CatalogTablePartition],
-      ignoreIfExists: Boolean): Unit = {
-    try {
-      val table = CarbonEnv.getCarbonTable(tableName)(sparkSession)
-      val updatedParts = CarbonScalaUtil.updatePartitions(parts, table)
-      super.createPartitions(tableName, updatedParts, ignoreIfExists)
-    } catch {
-      case e: Exception =>
-        super.createPartitions(tableName, parts, ignoreIfExists)
-    }
-  }
-
-  /**
-   * This is alternate way of getting partition information. It first fetches all partitions from
-   * hive and then apply filter instead of querying hive along with filters.
-   * @param partitionFilters
-   * @param sparkSession
-   * @param identifier
-   * @return
-   */
-  override def getPartitionsAlternate(partitionFilters: Seq[Expression],
-      sparkSession: SparkSession,
-      identifier: TableIdentifier) = {
-    CarbonSessionUtil.prunePartitionsByFilter(partitionFilters, sparkSession, identifier)
-  }
-
-  /**
-   * Update the storageformat with new location information
-   */
-  override def updateStorageLocation(
-      path: Path,
-      storage: CatalogStorageFormat,
-      newTableName: String,
-      dbName: String): CatalogStorageFormat = {
-    storage.copy(locationUri = Some(path.toUri))
-  }
-}
-
-/**
- * Session state implementation to override sql parser and adding strategies
- *
- * @param sparkSession
- */
-class CarbonSessionStateBuilder(sparkSession: SparkSession,
-    parentState: Option[SessionState] = None)
-  extends HiveSessionStateBuilder(sparkSession, parentState) {
-
-  override lazy val sqlParser: ParserInterface = new CarbonSparkSqlParser(conf, sparkSession)
-
-  experimentalMethods.extraStrategies =
-    Seq(new StreamingTableStrategy(sparkSession),
-        new CarbonLateDecodeStrategy,
-        new DDLStrategy(sparkSession)
-    )
-  experimentalMethods.extraOptimizations = Seq(new CarbonIUDRule,
-    new CarbonUDFTransformRule,
-    new CarbonLateDecodeRule)
-
-  /**
-   * Internal catalog for managing table and database states.
-   */
-  /**
-   * Create a [[CarbonSessionStateBuilder]].
-   */
-  override protected lazy val catalog: CarbonHiveSessionCatalog = {
-    val catalog = new CarbonHiveSessionCatalog(
-      externalCatalog,
-      session.sharedState.globalTempViewManager,
-      functionRegistry,
-      sparkSession,
-      conf,
-      SessionState.newHadoopConf(session.sparkContext.hadoopConfiguration, conf),
-      sqlParser,
-      resourceLoader)
-    parentState.foreach(_.catalog.copyStateTo(catalog))
-    catalog
-  }
-
-  private def externalCatalog: HiveExternalCatalog =
-    session.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog]
-
-  /**
-   * Create a Hive aware resource loader.
-   */
-  override protected lazy val resourceLoader: HiveSessionResourceLoader = {
-    val client: HiveClient = externalCatalog.client.newSession()
-    new HiveSessionResourceLoader(session, client)
-  }
-
-  override lazy val optimizer: Optimizer = new CarbonOptimizer(catalog, conf, experimentalMethods)
-
-  override protected def analyzer: Analyzer = new CarbonAnalyzer(catalog, conf, sparkSession,
-    new Analyzer(catalog, conf) {
-
-      override val extendedResolutionRules: Seq[Rule[LogicalPlan]] =
-        new ResolveHiveSerdeTable(session) +:
-        new FindDataSourceTable(session) +:
-        new ResolveSQLOnFile(session) +:
-        new CarbonIUDAnalysisRule(sparkSession) +:
-        new CarbonPreInsertionCasts(sparkSession) +: customResolutionRules
-
-      override val extendedCheckRules: Seq[LogicalPlan => Unit] =
-      PreWriteCheck :: HiveOnlyCheck :: Nil
-
-      override val postHocResolutionRules: Seq[Rule[LogicalPlan]] =
-        new DetermineTableStats(session) +:
-        RelationConversions(conf, catalog) +:
-        PreprocessTableCreation(session) +:
-        PreprocessTableInsertion(conf) +:
-        DataSourceAnalysis(conf) +:
-        HiveAnalysis +:
-        customPostHocResolutionRules
-    }
-  )
-
-  override protected def newBuilder: NewBuilder = new CarbonSessionStateBuilder(_, _)
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonSessionUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonSessionUtil.scala b/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonSessionUtil.scala
deleted file mode 100644
index ac702ea..0000000
--- a/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonSessionUtil.scala
+++ /dev/null
@@ -1,97 +0,0 @@
-/*
-* Licensed to the Apache Software Foundation (ASF) under one or more
-* contributor license agreements.  See the NOTICE file distributed with
-* this work for additional information regarding copyright ownership.
-* The ASF licenses this file to You under the Apache License, Version 2.0
-* (the "License"); you may not use this file except in compliance with
-* the License.  You may obtain a copy of the License at
-*
-*    http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-package org.apache.spark.sql.hive
-
-import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTablePartition}
-import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
-import org.apache.spark.sql.execution.datasources.LogicalRelation
-import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, CarbonEnv, SparkSession}
-import org.apache.spark.util.CarbonReflectionUtils
-import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils
-import org.apache.spark.sql.catalyst.expressions.Expression
-
-import org.apache.carbondata.common.logging.LogServiceFactory
-
-/**
- * This class refresh the relation from cache if the carbontable in
- * carbon catalog is not same as cached carbon relation's carbon table.
- */
-object CarbonSessionUtil {
-
-  val LOGGER = LogServiceFactory.getLogService("CarbonSessionUtil")
-
-  /**
-   * The method refreshes the cache entry
-   *
-   * @param rtnRelation [[LogicalPlan]] represents the given table or view.
-   * @param name        tableName
-   * @param sparkSession
-   * @return
-   */
-  def refreshRelation(rtnRelation: LogicalPlan, name: TableIdentifier)
-    (sparkSession: SparkSession): Boolean = {
-    var isRelationRefreshed = false
-    rtnRelation match {
-      case SubqueryAlias(_,
-      LogicalRelation(_: CarbonDatasourceHadoopRelation, _, _)
-      ) =>
-        isRelationRefreshed = CarbonEnv.refreshRelationFromCache(name)(sparkSession)
-      case LogicalRelation(_: CarbonDatasourceHadoopRelation, _, _) =>
-        isRelationRefreshed = CarbonEnv.refreshRelationFromCache(name)(sparkSession)
-      case SubqueryAlias(_, relation) if
-      relation.getClass.getName.equals("org.apache.spark.sql.catalyst.catalog.CatalogRelation") ||
-      relation.getClass.getName
-        .equals("org.apache.spark.sql.catalyst.catalog.HiveTableRelation") ||
-      relation.getClass.getName.equals(
-        "org.apache.spark.sql.catalyst.catalog.UnresolvedCatalogRelation"
-      ) =>
-        val catalogTable =
-          CarbonReflectionUtils.getFieldOfCatalogTable(
-            "tableMeta",
-            relation
-          ).asInstanceOf[CatalogTable]
-        isRelationRefreshed =
-          CarbonEnv.refreshRelationFromCache(catalogTable.identifier)(sparkSession)
-      case _ =>
-    }
-    isRelationRefreshed
-  }
-
-  /**
-   * This is alternate way of getting partition information. It first fetches all partitions from
-   * hive and then apply filter instead of querying hive along with filters.
-   *
-   * @param partitionFilters
-   * @param sparkSession
-   * @param identifier
-   * @return
-   */
-  def prunePartitionsByFilter(partitionFilters: Seq[Expression],
-      sparkSession: SparkSession,
-      identifier: TableIdentifier): Seq[CatalogTablePartition] = {
-    val allPartitions = sparkSession.sessionState.catalog.listPartitions(identifier)
-    ExternalCatalogUtils.prunePartitionsByFilter(
-      sparkSession.sessionState.catalog.getTableMetadata(identifier),
-      allPartitions,
-      partitionFilters,
-      sparkSession.sessionState.conf.sessionLocalTimeZone
-    )
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonSqlAstBuilder.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonSqlAstBuilder.scala b/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonSqlAstBuilder.scala
index b8bf56d..7177f1a 100644
--- a/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonSqlAstBuilder.scala
+++ b/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonSqlAstBuilder.scala
@@ -18,24 +18,15 @@
 package org.apache.spark.sql.hive
 
 import org.apache.spark.sql.SparkSession
-import org.apache.spark.sql.catalyst.parser.ParserUtils.{string, withOrigin}
-import org.apache.spark.sql.catalyst.parser.SqlBaseParser
-import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{AddTableColumnsContext, ChangeColumnContext, CreateHiveTableContext, CreateTableContext, ShowTablesContext}
+import org.apache.spark.sql.catalyst.parser.ParserUtils.string
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{AddTableColumnsContext, CreateHiveTableContext}
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.execution.SparkSqlAstBuilder
-import org.apache.spark.sql.execution.command.{AlterTableAddColumnsModel, AlterTableDataTypeChangeModel}
-import org.apache.spark.sql.execution.command.schema.{CarbonAlterTableAddColumnCommand, CarbonAlterTableDataTypeChangeCommand}
-import org.apache.spark.sql.execution.command.table.{CarbonExplainCommand, CarbonShowTablesCommand}
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.parser.{CarbonHelperSqlAstBuilder, CarbonSpark2SqlParser, CarbonSparkSqlParserUtil}
-import org.apache.spark.sql.types.DecimalType
-
-import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.util.CarbonProperties
 
 class CarbonSqlAstBuilder(conf: SQLConf, parser: CarbonSpark2SqlParser, sparkSession: SparkSession)
-  extends SparkSqlAstBuilder(conf) {
+  extends SparkSqlAstBuilder(conf) with SqlAstBuilderHelper {
 
   val helper = new CarbonHelperSqlAstBuilder(conf, parser, sparkSession)
 
@@ -55,75 +46,7 @@ class CarbonSqlAstBuilder(conf: SQLConf, parser: CarbonSpark2SqlParser, sparkSes
     }
   }
 
-  override def visitChangeColumn(ctx: ChangeColumnContext): LogicalPlan = {
-
-    val newColumn = visitColType(ctx.colType)
-    if (!ctx.identifier.getText.equalsIgnoreCase(newColumn.name)) {
-      throw new MalformedCarbonCommandException(
-        "Column names provided are different. Both the column names should be same")
-    }
-
-    val (typeString, values) : (String, Option[List[(Int, Int)]]) = newColumn.dataType match {
-      case d:DecimalType => ("decimal", Some(List((d.precision, d.scale))))
-      case _ => (newColumn.dataType.typeName.toLowerCase, None)
-    }
-
-    val alterTableChangeDataTypeModel =
-      AlterTableDataTypeChangeModel(new CarbonSpark2SqlParser().parseDataType(typeString, values),
-        new CarbonSpark2SqlParser()
-          .convertDbNameToLowerCase(Option(ctx.tableIdentifier().db).map(_.getText)),
-        ctx.tableIdentifier().table.getText.toLowerCase,
-        ctx.identifier.getText.toLowerCase,
-        newColumn.name.toLowerCase)
-
-    CarbonAlterTableDataTypeChangeCommand(alterTableChangeDataTypeModel)
-  }
-
-
   override def visitAddTableColumns(ctx: AddTableColumnsContext): LogicalPlan = {
-    val cols = Option(ctx.columns).toSeq.flatMap(visitColTypeList)
-    val fields = parser.getFields(cols)
-    val tblProperties = scala.collection.mutable.Map.empty[String, String]
-    val tableModel = new CarbonSpark2SqlParser().prepareTableModel (false,
-      new CarbonSpark2SqlParser().convertDbNameToLowerCase(Option(ctx.tableIdentifier().db)
-        .map(_.getText)),
-      ctx.tableIdentifier.table.getText.toLowerCase,
-      fields,
-      Seq.empty,
-      tblProperties,
-      None,
-      true)
-
-    val alterTableAddColumnsModel = AlterTableAddColumnsModel(
-      Option(ctx.tableIdentifier().db).map(_.getText),
-      ctx.tableIdentifier.table.getText,
-      tblProperties.toMap,
-      tableModel.dimCols,
-      tableModel.msrCols,
-      tableModel.highcardinalitydims.getOrElse(Seq.empty))
-
-    CarbonAlterTableAddColumnCommand(alterTableAddColumnsModel)
-  }
-
-  override def visitCreateTable(ctx: CreateTableContext): LogicalPlan = {
-    super.visitCreateTable(ctx)
-  }
-
-  override def visitShowTables(ctx: ShowTablesContext): LogicalPlan = {
-    withOrigin(ctx) {
-      if (CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.CARBON_SHOW_DATAMAPS,
-          CarbonCommonConstants.CARBON_SHOW_DATAMAPS_DEFAULT).toBoolean) {
-        super.visitShowTables(ctx)
-      } else {
-        CarbonShowTablesCommand(
-          Option(ctx.db).map(_.getText),
-          Option(ctx.pattern).map(string))
-      }
-    }
-  }
-
-  override def visitExplain(ctx: SqlBaseParser.ExplainContext): LogicalPlan = {
-    CarbonExplainCommand(super.visitExplain(ctx))
+    visitAddTableColumns(parser,ctx)
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonSqlConf.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonSqlConf.scala b/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonSqlConf.scala
deleted file mode 100644
index 2128ffd..0000000
--- a/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonSqlConf.scala
+++ /dev/null
@@ -1,148 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.spark.sql.hive
-
-import org.apache.spark.sql.SparkSession
-import org.apache.spark.sql.internal.SQLConf.buildConf
-
-import org.apache.carbondata.core.constants.{CarbonCommonConstants, CarbonLoadOptionConstants}
-import org.apache.carbondata.core.util.CarbonProperties
-
-/**
- * To initialize dynamic values default param
- */
-class CarbonSQLConf(sparkSession: SparkSession) {
-
-  val carbonProperties = CarbonProperties.getInstance()
-
-  /**
-   * To initialize dynamic param defaults along with usage docs
-   */
-  def addDefaultCarbonParams(): Unit = {
-    val ENABLE_UNSAFE_SORT =
-      buildConf(CarbonCommonConstants.ENABLE_UNSAFE_SORT)
-        .doc("To enable/ disable unsafe sort.")
-        .booleanConf
-        .createWithDefault(carbonProperties.getProperty(CarbonCommonConstants.ENABLE_UNSAFE_SORT,
-          CarbonCommonConstants.ENABLE_UNSAFE_SORT_DEFAULT).toBoolean)
-    val CARBON_CUSTOM_BLOCK_DISTRIBUTION =
-      buildConf(CarbonCommonConstants.CARBON_CUSTOM_BLOCK_DISTRIBUTION)
-        .doc("To set carbon task distribution.")
-        .stringConf
-        .createWithDefault(carbonProperties
-          .getProperty(CarbonCommonConstants.CARBON_TASK_DISTRIBUTION,
-            CarbonCommonConstants.CARBON_TASK_DISTRIBUTION_DEFAULT))
-    val BAD_RECORDS_LOGGER_ENABLE =
-      buildConf(CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORDS_LOGGER_ENABLE)
-        .doc("To enable/ disable carbon bad record logger.")
-        .booleanConf
-        .createWithDefault(CarbonLoadOptionConstants
-          .CARBON_OPTIONS_BAD_RECORDS_LOGGER_ENABLE_DEFAULT.toBoolean)
-    val BAD_RECORDS_ACTION =
-      buildConf(CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORDS_ACTION)
-        .doc("To configure the bad records action.")
-        .stringConf
-        .createWithDefault(carbonProperties
-          .getProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION,
-            CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION_DEFAULT))
-    val IS_EMPTY_DATA_BAD_RECORD =
-      buildConf(CarbonLoadOptionConstants.CARBON_OPTIONS_IS_EMPTY_DATA_BAD_RECORD)
-        .doc("Property to decide weather empty data to be considered bad/ good record.")
-        .booleanConf
-        .createWithDefault(CarbonLoadOptionConstants.CARBON_OPTIONS_IS_EMPTY_DATA_BAD_RECORD_DEFAULT
-          .toBoolean)
-    val SORT_SCOPE =
-      buildConf(CarbonLoadOptionConstants.CARBON_OPTIONS_SORT_SCOPE)
-        .doc("Property to specify sort scope.")
-        .stringConf
-        .createWithDefault(carbonProperties.getProperty(CarbonCommonConstants.LOAD_SORT_SCOPE,
-          CarbonCommonConstants.LOAD_SORT_SCOPE_DEFAULT))
-    val BATCH_SORT_SIZE_INMB =
-      buildConf(CarbonLoadOptionConstants.CARBON_OPTIONS_BATCH_SORT_SIZE_INMB)
-        .doc("Property to specify batch sort size in MB.")
-        .stringConf
-        .createWithDefault(carbonProperties
-          .getProperty(CarbonCommonConstants.LOAD_BATCH_SORT_SIZE_INMB,
-            CarbonCommonConstants.LOAD_BATCH_SORT_SIZE_INMB_DEFAULT))
-    val SINGLE_PASS =
-      buildConf(CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS)
-        .doc("Property to enable/disable single_pass.")
-        .booleanConf
-        .createWithDefault(CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS_DEFAULT.toBoolean)
-    val BAD_RECORD_PATH =
-      buildConf(CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORD_PATH)
-        .doc("Property to configure the bad record location.")
-        .stringConf
-        .createWithDefault(carbonProperties.getProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC,
-          CarbonCommonConstants.CARBON_BADRECORDS_LOC_DEFAULT_VAL))
-    val GLOBAL_SORT_PARTITIONS =
-      buildConf(CarbonLoadOptionConstants.CARBON_OPTIONS_GLOBAL_SORT_PARTITIONS)
-        .doc("Property to configure the global sort partitions.")
-        .stringConf
-        .createWithDefault(carbonProperties
-          .getProperty(CarbonCommonConstants.LOAD_GLOBAL_SORT_PARTITIONS,
-            CarbonCommonConstants.LOAD_GLOBAL_SORT_PARTITIONS_DEFAULT))
-    val DATEFORMAT =
-      buildConf(CarbonLoadOptionConstants.CARBON_OPTIONS_DATEFORMAT)
-        .doc("Property to configure data format for date type columns.")
-        .stringConf
-        .createWithDefault(CarbonLoadOptionConstants.CARBON_OPTIONS_DATEFORMAT_DEFAULT)
-    val CARBON_INPUT_SEGMENTS = buildConf(
-      "carbon.input.segments.<database_name>.<table_name>")
-      .doc("Property to configure the list of segments to query.").stringConf
-      .createWithDefault(carbonProperties
-        .getProperty("carbon.input.segments.<database_name>.<table_name>", "*"))
-  }
-  /**
-   * to set the dynamic properties default values
-   */
-  def addDefaultCarbonSessionParams(): Unit = {
-    sparkSession.conf.set(CarbonCommonConstants.ENABLE_UNSAFE_SORT,
-      carbonProperties.getProperty(CarbonCommonConstants.ENABLE_UNSAFE_SORT,
-        CarbonCommonConstants.ENABLE_UNSAFE_SORT_DEFAULT).toBoolean)
-    sparkSession.conf.set(CarbonCommonConstants.CARBON_CUSTOM_BLOCK_DISTRIBUTION,
-      carbonProperties
-        .getProperty(CarbonCommonConstants.CARBON_TASK_DISTRIBUTION,
-          CarbonCommonConstants.CARBON_TASK_DISTRIBUTION_DEFAULT))
-    sparkSession.conf.set(CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORDS_LOGGER_ENABLE,
-      CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORDS_LOGGER_ENABLE_DEFAULT.toBoolean)
-    sparkSession.conf.set(CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORDS_ACTION,
-      carbonProperties.getProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION,
-        CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION_DEFAULT))
-    sparkSession.conf.set(CarbonLoadOptionConstants.CARBON_OPTIONS_IS_EMPTY_DATA_BAD_RECORD,
-      CarbonLoadOptionConstants.CARBON_OPTIONS_IS_EMPTY_DATA_BAD_RECORD_DEFAULT.toBoolean)
-    sparkSession.conf.set(CarbonLoadOptionConstants.CARBON_OPTIONS_SORT_SCOPE,
-      carbonProperties.getProperty(CarbonCommonConstants.LOAD_SORT_SCOPE,
-        CarbonCommonConstants.LOAD_SORT_SCOPE_DEFAULT))
-    sparkSession.conf.set(CarbonLoadOptionConstants.CARBON_OPTIONS_BATCH_SORT_SIZE_INMB,
-      carbonProperties.getProperty(CarbonCommonConstants.LOAD_BATCH_SORT_SIZE_INMB,
-        CarbonCommonConstants.LOAD_BATCH_SORT_SIZE_INMB_DEFAULT))
-    sparkSession.conf.set(CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS,
-      CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS_DEFAULT.toBoolean)
-    sparkSession.conf.set(CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORD_PATH,
-      carbonProperties.getProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC,
-        CarbonCommonConstants.CARBON_BADRECORDS_LOC_DEFAULT_VAL))
-    sparkSession.conf.set(CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORD_PATH,
-      carbonProperties.getProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC,
-        CarbonCommonConstants.CARBON_BADRECORDS_LOC_DEFAULT_VAL))
-    sparkSession.conf.set(CarbonLoadOptionConstants.CARBON_OPTIONS_GLOBAL_SORT_PARTITIONS,
-      carbonProperties.getProperty(CarbonCommonConstants.LOAD_GLOBAL_SORT_PARTITIONS,
-        CarbonCommonConstants.LOAD_GLOBAL_SORT_PARTITIONS_DEFAULT))
-    sparkSession.conf.set(CarbonLoadOptionConstants.CARBON_OPTIONS_DATEFORMAT,
-      CarbonLoadOptionConstants.CARBON_OPTIONS_DATEFORMAT_DEFAULT)
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CreateCarbonSourceTableAsSelectCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CreateCarbonSourceTableAsSelectCommand.scala b/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CreateCarbonSourceTableAsSelectCommand.scala
deleted file mode 100644
index 4e22d13..0000000
--- a/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CreateCarbonSourceTableAsSelectCommand.scala
+++ /dev/null
@@ -1,122 +0,0 @@
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.hive
-
-import java.net.URI
-
-import org.apache.spark.sql.{AnalysisException, Dataset, Row, SaveMode, SparkSession}
-import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType, CatalogUtils}
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.execution.command.{AlterTableRecoverPartitionsCommand, RunnableCommand}
-import org.apache.spark.sql.execution.datasources.{DataSource, HadoopFsRelation}
-import org.apache.spark.sql.sources.BaseRelation
-
-/**
- * Create table 'using carbondata' and insert the query result into it.
- *
- * @param table the Catalog Table
- * @param mode  SaveMode:Ignore,OverWrite,ErrorIfExists,Append
- * @param query the query whose result will be insert into the new relation
- *
- */
-
-case class CreateCarbonSourceTableAsSelectCommand(
-    table: CatalogTable,
-    mode: SaveMode,
-    query: LogicalPlan)
-  extends RunnableCommand {
-
-  override protected def innerChildren: Seq[LogicalPlan] = Seq(query)
-
-  override def run(sparkSession: SparkSession): Seq[Row] = {
-    assert(table.tableType != CatalogTableType.VIEW)
-    assert(table.provider.isDefined)
-
-    val sessionState = sparkSession.sessionState
-    val db = table.identifier.database.getOrElse(sessionState.catalog.getCurrentDatabase)
-    val tableIdentWithDB = table.identifier.copy(database = Some(db))
-    val tableName = tableIdentWithDB.unquotedString
-
-    if (sessionState.catalog.tableExists(tableIdentWithDB)) {
-      assert(mode != SaveMode.Overwrite,
-        s"Expect the table $tableName has been dropped when the save mode is Overwrite")
-
-      if (mode == SaveMode.ErrorIfExists) {
-        throw new AnalysisException(s"Table $tableName already exists. You need to drop it first.")
-      }
-      if (mode == SaveMode.Ignore) {
-        // Since the table already exists and the save mode is Ignore, we will just return.
-        return Seq.empty
-      }
-
-      saveDataIntoTable(
-        sparkSession, table, table.storage.locationUri, query, SaveMode.Append, tableExists = true)
-    } else {
-      assert(table.schema.isEmpty)
-
-      val tableLocation = if (table.tableType == CatalogTableType.MANAGED) {
-        Some(sessionState.catalog.defaultTablePath(table.identifier))
-      } else {
-        table.storage.locationUri
-      }
-      val result = saveDataIntoTable(
-        sparkSession, table, tableLocation, query, SaveMode.Overwrite, tableExists = false)
-
-      result match {
-        case fs: HadoopFsRelation if table.partitionColumnNames.nonEmpty &&
-                                     sparkSession.sqlContext.conf.manageFilesourcePartitions =>
-          // Need to recover partitions into the metastore so our saved data is visible.
-          sessionState.executePlan(AlterTableRecoverPartitionsCommand(table.identifier)).toRdd
-        case _ =>
-      }
-    }
-
-    Seq.empty[Row]
-  }
-
-  private def saveDataIntoTable(
-      session: SparkSession,
-      table: CatalogTable,
-      tableLocation: Option[URI],
-      data: LogicalPlan,
-      mode: SaveMode,
-      tableExists: Boolean): BaseRelation = {
-    // Create the relation based on the input logical plan: `data`.
-    val pathOption = tableLocation.map("path" -> CatalogUtils.URIToString(_))
-    val dataSource = DataSource(
-      session,
-      className = table.provider.get,
-      partitionColumns = table.partitionColumnNames,
-      bucketSpec = table.bucketSpec,
-      options = table.storage.properties ++ pathOption,
-      catalogTable = if (tableExists) {
-        Some(table)
-      } else {
-        None
-      })
-
-    try {
-      dataSource.writeAndRead(mode, Dataset.ofRows(session, query))
-    } catch {
-      case ex: AnalysisException =>
-        logError(s"Failed to write to table ${ table.identifier.unquotedString }", ex)
-        throw ex
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/spark2.3/org/apache/spark/sql/CarbonToSparkAdapater.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/spark2.3/org/apache/spark/sql/CarbonToSparkAdapater.scala b/integration/spark2/src/main/spark2.3/org/apache/spark/sql/CarbonToSparkAdapater.scala
new file mode 100644
index 0000000..cec4c36
--- /dev/null
+++ b/integration/spark2/src/main/spark2.3/org/apache/spark/sql/CarbonToSparkAdapater.scala
@@ -0,0 +1,84 @@
+/*
+ * 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.spark.sql
+
+import org.apache.spark.SparkContext
+import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
+import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, AttributeSet, ExprId, Expression, ExpressionSet, NamedExpression, SubqueryExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.command.ExplainCommand
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{DataType, Metadata}
+
+object CarbonToSparkAdapater {
+
+  def addSparkListener(sparkContext: SparkContext) = {
+    sparkContext.addSparkListener(new SparkListener {
+      override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = {
+        SparkSession.setDefaultSession(null)
+      }
+    })
+  }
+
+  def createAttributeReference(name: String, dataType: DataType, nullable: Boolean,
+                               metadata: Metadata, exprId: ExprId, qualifier: Option[String],
+                               attrRef : NamedExpression): AttributeReference = {
+    AttributeReference(
+      name,
+      dataType,
+      nullable,
+      metadata)(exprId, qualifier)
+  }
+
+  def createAliasRef(child: Expression,
+                     name: String,
+                     exprId: ExprId = NamedExpression.newExprId,
+                     qualifier: Option[String] = None,
+                     explicitMetadata: Option[Metadata] = None,
+                     namedExpr : Option[NamedExpression] = None ) : Alias = {
+
+      Alias(child, name)(exprId, qualifier, explicitMetadata)
+  }
+
+  def getExplainCommandObj() : ExplainCommand = {
+    ExplainCommand(OneRowRelation())
+  }
+
+  /**
+   * As a part of SPARK-24085 Hive tables supports scala subquery for
+   * parition tables,so Carbon also needs to supports
+   * @param partitionSet
+   * @param filterPredicates
+   * @return
+   */
+  def getPartitionKeyFilter(
+      partitionSet: AttributeSet,
+      filterPredicates: Seq[Expression]): ExpressionSet = {
+    ExpressionSet(
+      ExpressionSet(filterPredicates)
+        .filterNot(SubqueryExpression.hasSubquery)
+        .filter(_.references.subsetOf(partitionSet)))
+  }
+
+  // As per SPARK-22520 OptimizeCodegen is removed in 2.3.1
+  def getOptimizeCodegenRule(conf :SQLConf): Seq[Rule[LogicalPlan]] = {
+    Seq.empty
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/spark2.3/org/apache/spark/sql/CarbonVectorProxy.java
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/spark2.3/org/apache/spark/sql/CarbonVectorProxy.java b/integration/spark2/src/main/spark2.3/org/apache/spark/sql/CarbonVectorProxy.java
deleted file mode 100644
index 783a528..0000000
--- a/integration/spark2/src/main/spark2.3/org/apache/spark/sql/CarbonVectorProxy.java
+++ /dev/null
@@ -1,255 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.spark.sql;
-
-import java.math.BigInteger;
-
-import org.apache.spark.memory.MemoryMode;
-import org.apache.spark.sql.ColumnVectorFactory;
-import org.apache.spark.sql.catalyst.InternalRow;
-import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
-import org.apache.spark.sql.types.*;
-import org.apache.spark.sql.vectorized.ColumnVector;
-import org.apache.spark.sql.vectorized.ColumnarBatch;
-import org.apache.spark.unsafe.types.CalendarInterval;
-import org.apache.spark.unsafe.types.UTF8String;
-
-/**
- * Adapter class which handles the columnar vector reading of the carbondata
- * based on the spark ColumnVector and ColumnarBatch API. This proxy class
- * handles the complexity of spark 2.3 version related api changes since
- * spark ColumnVector and ColumnarBatch interfaces are still evolving.
- */
-public class CarbonVectorProxy {
-
-    private ColumnarBatch columnarBatch;
-    private WritableColumnVector[] writableColumnVectors;
-
-    /**
-     * Adapter class which handles the columnar vector reading of the carbondata
-     * based on the spark ColumnVector and ColumnarBatch API. This proxy class
-     * handles the complexity of spark 2.3 version related api changes since
-     * spark ColumnVector and ColumnarBatch interfaces are still evolving.
-     *
-     * @param memMode       which represent the type onheap or offheap vector.
-     * @param rowNum        rows number for vector reading
-     * @param structFileds, metadata related to current schema of table.
-     */
-    public CarbonVectorProxy(MemoryMode memMode, int rowNum, StructField[] structFileds) {
-        writableColumnVectors = ColumnVectorFactory
-                .getColumnVector(memMode, new StructType(structFileds), rowNum);
-        columnarBatch = new ColumnarBatch(writableColumnVectors);
-        columnarBatch.setNumRows(rowNum);
-    }
-
-    public CarbonVectorProxy(MemoryMode memMode, StructType outputSchema, int rowNum) {
-        writableColumnVectors = ColumnVectorFactory
-                .getColumnVector(memMode, outputSchema, rowNum);
-        columnarBatch = new ColumnarBatch(writableColumnVectors);
-        columnarBatch.setNumRows(rowNum);
-    }
-
-    /**
-     * Returns the number of rows for read, including filtered rows.
-     */
-    public int numRows() {
-        return columnarBatch.numRows();
-    }
-
-    public Object reserveDictionaryIds(int capacity, int ordinal) {
-        return writableColumnVectors[ordinal].reserveDictionaryIds(capacity);
-    }
-
-    /**
-     * This API will return a columnvector from a batch of column vector rows
-     * based on the ordinal
-     *
-     * @param ordinal
-     * @return
-     */
-    public ColumnVector column(int ordinal) {
-        return columnarBatch.column(ordinal);
-    }
-
-    /**
-     * Resets this column for writing. The currently stored values are no longer accessible.
-     */
-    public void reset() {
-        for (WritableColumnVector col : writableColumnVectors) {
-            col.reset();
-        }
-    }
-
-    public void resetDictionaryIds(int ordinal) {
-        writableColumnVectors[ordinal].getDictionaryIds().reset();
-    }
-
-    /**
-     * Returns the row in this batch at `rowId`. Returned row is reused across calls.
-     */
-    public InternalRow getRow(int rowId) {
-        return columnarBatch.getRow(rowId);
-    }
-
-
-    /**
-     * Returns the row in this batch at `rowId`. Returned row is reused across calls.
-     */
-    public Object getColumnarBatch() {
-        return columnarBatch;
-    }
-
-    /**
-     * Called to close all the columns in this batch. It is not valid to access the data after
-     * calling this. This must be called at the end to clean up memory allocations.
-     */
-    public void close() {
-        columnarBatch.close();
-    }
-
-    /**
-     * Sets the number of rows in this batch.
-     */
-    public void setNumRows(int numRows) {
-        columnarBatch.setNumRows(numRows);
-    }
-
-    /**
-     * This method will add the row to the corresponding column vector object
-     *
-     * @param rowId
-     * @param value
-     */
-    public void putRowToColumnBatch(int rowId, Object value, int offset) {
-            org.apache.spark.sql.types.DataType t = dataType(offset);
-            if (null == value) {
-                putNull(rowId, offset);
-            } else {
-                if (t == org.apache.spark.sql.types.DataTypes.BooleanType) {
-                    putBoolean(rowId, (boolean) value, offset);
-                } else if (t == org.apache.spark.sql.types.DataTypes.ByteType) {
-                    putByte(rowId, (byte) value, offset);
-                } else if (t == org.apache.spark.sql.types.DataTypes.ShortType) {
-                    putShort(rowId, (short) value, offset);
-                } else if (t == org.apache.spark.sql.types.DataTypes.IntegerType) {
-                    putInt(rowId, (int) value, offset);
-                } else if (t == org.apache.spark.sql.types.DataTypes.LongType) {
-                    putLong(rowId, (long) value, offset);
-                } else if (t == org.apache.spark.sql.types.DataTypes.FloatType) {
-                    putFloat(rowId, (float) value, offset);
-                } else if (t == org.apache.spark.sql.types.DataTypes.DoubleType) {
-                    putDouble(rowId, (double) value, offset);
-                } else if (t == org.apache.spark.sql.types.DataTypes.StringType) {
-                    UTF8String v = (UTF8String) value;
-                    putByteArray(rowId, v.getBytes(), offset);
-                } else if (t instanceof DecimalType) {
-                    DecimalType dt = (DecimalType) t;
-                    Decimal d = Decimal.fromDecimal(value);
-                    if (dt.precision() <= Decimal.MAX_INT_DIGITS()) {
-                        putInt(rowId, (int) d.toUnscaledLong(), offset);
-                    } else if (dt.precision() <= Decimal.MAX_LONG_DIGITS()) {
-                        putLong(rowId, d.toUnscaledLong(), offset);
-                    } else {
-                        final BigInteger integer = d.toJavaBigDecimal().unscaledValue();
-                        byte[] bytes = integer.toByteArray();
-                        putByteArray(rowId, bytes, 0, bytes.length, offset);
-                    }
-                } else if (t instanceof CalendarIntervalType) {
-                    CalendarInterval c = (CalendarInterval) value;
-                    writableColumnVectors[offset].getChild(0).putInt(rowId, c.months);
-                    writableColumnVectors[offset].getChild(1).putLong(rowId, c.microseconds);
-                } else if (t instanceof org.apache.spark.sql.types.DateType) {
-                    putInt(rowId, (int) value, offset);
-                } else if (t instanceof org.apache.spark.sql.types.TimestampType) {
-                    putLong(rowId, (long) value, offset);
-                }
-            }
-    }
-
-    public void putBoolean(int rowId, boolean value, int ordinal) {
-        writableColumnVectors[ordinal].putBoolean(rowId, (boolean) value);
-    }
-
-    public void putByte(int rowId, byte value, int ordinal) {
-        writableColumnVectors[ordinal].putByte(rowId, (byte) value);
-    }
-
-    public void putShort(int rowId, short value, int ordinal) {
-        writableColumnVectors[ordinal].putShort(rowId, (short) value);
-    }
-
-    public void putInt(int rowId, int value, int ordinal) {
-        writableColumnVectors[ordinal].putInt(rowId, (int) value);
-    }
-
-    public void putFloat(int rowId, float value, int ordinal) {
-        writableColumnVectors[ordinal].putFloat(rowId, (float) value);
-    }
-
-    public void putLong(int rowId, long value, int ordinal) {
-        writableColumnVectors[ordinal].putLong(rowId, (long) value);
-    }
-
-    public void putDouble(int rowId, double value, int ordinal) {
-        writableColumnVectors[ordinal].putDouble(rowId, (double) value);
-    }
-
-    public void putByteArray(int rowId, byte[] value, int ordinal) {
-        writableColumnVectors[ordinal].putByteArray(rowId, (byte[]) value);
-    }
-
-    public void putInts(int rowId, int count, int value, int ordinal) {
-        writableColumnVectors[ordinal].putInts(rowId, count, value);
-    }
-
-    public void putShorts(int rowId, int count, short value, int ordinal) {
-        writableColumnVectors[ordinal].putShorts(rowId, count, value);
-    }
-
-    public void putLongs(int rowId, int count, long value, int ordinal) {
-        writableColumnVectors[ordinal].putLongs(rowId, count, value);
-    }
-
-    public void putDecimal(int rowId, Decimal value, int precision, int ordinal) {
-        writableColumnVectors[ordinal].putDecimal(rowId, value, precision);
-
-    }
-
-    public void putDoubles(int rowId, int count, double value, int ordinal) {
-        writableColumnVectors[ordinal].putDoubles(rowId, count, value);
-    }
-
-    public void putByteArray(int rowId, byte[] value, int offset, int length, int ordinal) {
-        writableColumnVectors[ordinal].putByteArray(rowId, (byte[]) value, offset, length);
-    }
-
-    public void putNull(int rowId, int ordinal) {
-        writableColumnVectors[ordinal].putNull(rowId);
-    }
-
-    public void putNulls(int rowId, int count, int ordinal) {
-        writableColumnVectors[ordinal].putNulls(rowId, count);
-    }
-
-    public boolean isNullAt(int rowId, int ordinal) {
-        return writableColumnVectors[ordinal].isNullAt(rowId);
-    }
-
-    public DataType dataType(int ordinal) {
-        return writableColumnVectors[ordinal].dataType();
-    }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/spark2.3/org/apache/spark/sql/ColumnVectorFactory.java
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/spark2.3/org/apache/spark/sql/ColumnVectorFactory.java b/integration/spark2/src/main/spark2.3/org/apache/spark/sql/ColumnVectorFactory.java
deleted file mode 100644
index 6fe5ede..0000000
--- a/integration/spark2/src/main/spark2.3/org/apache/spark/sql/ColumnVectorFactory.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.spark.sql;
-
-import org.apache.spark.memory.MemoryMode;
-import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
-import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
-import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
-import org.apache.spark.sql.types.StructType;
-import org.apache.spark.sql.vectorized.ColumnarBatch;
-
-public class ColumnVectorFactory {
-
-
-    public static WritableColumnVector[] getColumnVector(MemoryMode memMode, StructType outputSchema, int rowNums) {
-
-
-        WritableColumnVector[] writableColumnVectors = null;
-        switch (memMode) {
-            case ON_HEAP:
-                writableColumnVectors = OnHeapColumnVector
-                        .allocateColumns(rowNums, outputSchema);
-                break;
-            case OFF_HEAP:
-                writableColumnVectors = OffHeapColumnVector
-                        .allocateColumns(rowNums, outputSchema);
-                break;
-        }
-        return writableColumnVectors;
-    }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/spark2.3/org/apache/spark/sql/hive/CarbonOptimizer.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/spark2.3/org/apache/spark/sql/hive/CarbonOptimizer.scala b/integration/spark2/src/main/spark2.3/org/apache/spark/sql/hive/CarbonOptimizer.scala
new file mode 100644
index 0000000..f1b632b
--- /dev/null
+++ b/integration/spark2/src/main/spark2.3/org/apache/spark/sql/hive/CarbonOptimizer.scala
@@ -0,0 +1,37 @@
+/*
+ * 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.spark.sql.hive
+
+import org.apache.spark.sql.ExperimentalMethods
+import org.apache.spark.sql.catalyst.catalog.SessionCatalog
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.SparkOptimizer
+import org.apache.spark.sql.internal.SQLConf
+
+
+class CarbonOptimizer(
+    catalog: SessionCatalog,
+    conf: SQLConf,
+    experimentalMethods: ExperimentalMethods)
+  extends SparkOptimizer(catalog, experimentalMethods) {
+
+  override def execute(plan: LogicalPlan): LogicalPlan = {
+    val transFormedPlan: LogicalPlan = CarbonOptimizerUtil.transformForScalarSubQuery(plan)
+    super.execute(transFormedPlan)
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/main/spark2.3/org/apache/spark/sql/hive/CarbonSqlAstBuilder.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/spark2.3/org/apache/spark/sql/hive/CarbonSqlAstBuilder.scala b/integration/spark2/src/main/spark2.3/org/apache/spark/sql/hive/CarbonSqlAstBuilder.scala
new file mode 100644
index 0000000..73b6790
--- /dev/null
+++ b/integration/spark2/src/main/spark2.3/org/apache/spark/sql/hive/CarbonSqlAstBuilder.scala
@@ -0,0 +1,52 @@
+/*
+ * 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.spark.sql.hive
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.parser.ParserUtils.string
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{AddTableColumnsContext, CreateHiveTableContext}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.SparkSqlAstBuilder
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.parser.{CarbonHelperSqlAstBuilder, CarbonSpark2SqlParser, CarbonSparkSqlParserUtil}
+
+class CarbonSqlAstBuilder(conf: SQLConf, parser: CarbonSpark2SqlParser, sparkSession: SparkSession)
+  extends SparkSqlAstBuilder(conf) with SqlAstBuilderHelper {
+
+  val helper = new CarbonHelperSqlAstBuilder(conf, parser, sparkSession)
+
+  override def visitCreateHiveTable(ctx: CreateHiveTableContext): LogicalPlan = {
+    val fileStorage = CarbonSparkSqlParserUtil.getFileStorage(ctx.createFileFormat(0))
+
+    if (fileStorage.equalsIgnoreCase("'carbondata'") ||
+        fileStorage.equalsIgnoreCase("carbondata") ||
+        fileStorage.equalsIgnoreCase("'carbonfile'") ||
+        fileStorage.equalsIgnoreCase("'org.apache.carbondata.format'")) {
+      val createTableTuple = (ctx.createTableHeader, ctx.skewSpec(0),
+        ctx.bucketSpec(0), ctx.partitionColumns, ctx.columns, ctx.tablePropertyList(0),ctx.locationSpec(0),
+        Option(ctx.STRING(0)).map(string), ctx.AS, ctx.query, fileStorage)
+      helper.createCarbonTable(createTableTuple)
+    } else {
+      super.visitCreateHiveTable(ctx)
+    }
+  }
+
+  override def visitAddTableColumns(ctx: AddTableColumnsContext): LogicalPlan = {
+    visitAddTableColumns(parser,ctx)
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/test/java/org/apache/carbondata/stream/CarbonStreamRecordReaderTest.java
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/java/org/apache/carbondata/stream/CarbonStreamRecordReaderTest.java b/integration/spark2/src/test/java/org/apache/carbondata/stream/CarbonStreamRecordReaderTest.java
new file mode 100644
index 0000000..f357e41
--- /dev/null
+++ b/integration/spark2/src/test/java/org/apache/carbondata/stream/CarbonStreamRecordReaderTest.java
@@ -0,0 +1,100 @@
+/*
+ * 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.stream;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import java.util.UUID;
+
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.core.statusmanager.FileFormat;
+import org.apache.carbondata.hadoop.CarbonInputSplit;
+import org.apache.carbondata.hadoop.CarbonMultiBlockSplit;
+import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil;
+import org.apache.carbondata.streaming.CarbonStreamInputFormat;
+
+import junit.framework.TestCase;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class CarbonStreamRecordReaderTest extends TestCase {
+
+  private TaskAttemptID taskAttemptId;
+  private TaskAttemptContext taskAttemptContext;
+  private Configuration hadoopConf;
+  private AbsoluteTableIdentifier identifier;
+  private String tablePath;
+
+
+  @Override protected void setUp() throws Exception {
+    tablePath = new File("target/stream_input").getCanonicalPath();
+    String dbName = "default";
+    String tableName = "stream_table_input";
+    identifier = AbsoluteTableIdentifier.from(
+        tablePath,
+        new CarbonTableIdentifier(dbName, tableName, UUID.randomUUID().toString()));
+
+    JobID jobId = CarbonInputFormatUtil.getJobId(new Date(), 0);
+    TaskID taskId = new TaskID(jobId, TaskType.MAP, 0);
+    taskAttemptId = new TaskAttemptID(taskId, 0);
+
+    hadoopConf = new Configuration();
+    taskAttemptContext = new TaskAttemptContextImpl(hadoopConf, taskAttemptId);
+  }
+
+  private InputSplit buildInputSplit() throws IOException {
+    CarbonInputSplit carbonInputSplit = new CarbonInputSplit();
+    List<CarbonInputSplit> splitList = new ArrayList<>();
+    splitList.add(carbonInputSplit);
+    return new CarbonMultiBlockSplit(splitList, new String[] { "localhost" },
+        FileFormat.ROW_V1);
+  }
+
+  @Test public void testCreateRecordReader() {
+    try {
+      InputSplit inputSplit = buildInputSplit();
+      CarbonStreamInputFormat inputFormat = new CarbonStreamInputFormat();
+      RecordReader recordReader = inputFormat.createRecordReader(inputSplit, taskAttemptContext);
+      Assert.assertNotNull("Failed to create record reader", recordReader);
+    } catch (Exception e) {
+      e.printStackTrace();
+      Assert.assertTrue(e.getMessage(), false);
+    }
+  }
+
+  @Override protected void tearDown() throws Exception {
+    super.tearDown();
+    if (tablePath != null) {
+      FileFactory.deleteAllFilesOfDir(new File(tablePath));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesFilterTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesFilterTest.scala b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesFilterTest.scala
index cdcf018..466475b 100644
--- a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesFilterTest.scala
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesFilterTest.scala
@@ -80,7 +80,7 @@ class BooleanDataTypesFilterTest extends QueryTest with BeforeAndAfterEach with
     checkAnswer(sql("select count(*) from carbon_table where booleanField = true"),
       Row(4))
 
-    if (!Spark2TestQueryExecutor.spark.version.startsWith("2.2")) {
+    if (Spark2TestQueryExecutor.spark.version.startsWith("2.1")) {
       checkAnswer(sql("select count(*) from carbon_table where booleanField = 'true'"),
         Row(0))
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesLoadTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesLoadTest.scala b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesLoadTest.scala
index c220f1c..4050fd8 100644
--- a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesLoadTest.scala
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesLoadTest.scala
@@ -283,7 +283,7 @@ class BooleanDataTypesLoadTest extends QueryTest with BeforeAndAfterEach with Be
     checkAnswer(sql("select count(*) from boolean_table where booleanField = false or booleanField = true"),
       Row(10))
 
-    if (!Spark2TestQueryExecutor.spark.version.startsWith("2.2")) {
+    if (Spark2TestQueryExecutor.spark.version.startsWith("2.1")) {
       checkAnswer(sql("select count(*) from boolean_table where booleanField = 'true'"),
         Row(0))
 
@@ -373,7 +373,7 @@ class BooleanDataTypesLoadTest extends QueryTest with BeforeAndAfterEach with Be
     checkAnswer(sql("select count(*) from boolean_table where booleanField = false or booleanField = true"),
       Row(10))
 
-    if (!Spark2TestQueryExecutor.spark.version.startsWith("2.2")) {
+    if (Spark2TestQueryExecutor.spark.version.startsWith("2.1")) {
       checkAnswer(sql("select count(*) from boolean_table where booleanField = 'true'"),
         Row(0))
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/test/scala/org/apache/spark/carbondata/bucketing/TableBucketingTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/bucketing/TableBucketingTestCase.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/bucketing/TableBucketingTestCase.scala
index 363db65..8aa0cf6 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/bucketing/TableBucketingTestCase.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/bucketing/TableBucketingTestCase.scala
@@ -19,7 +19,7 @@ package org.apache.spark.carbondata.bucketing
 
 import org.apache.spark.sql.CarbonEnv
 import org.apache.spark.sql.common.util.Spark2QueryTest
-import org.apache.spark.sql.execution.exchange.ShuffleExchange
+import org.apache.spark.sql.execution.exchange.Exchange
 import org.scalatest.BeforeAndAfterAll
 
 import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
@@ -128,7 +128,11 @@ class TableBucketingTestCase extends Spark2QueryTest with BeforeAndAfterAll {
       """.stripMargin).queryExecution.executedPlan
     var shuffleExists = false
     plan.collect {
-      case s: ShuffleExchange => shuffleExists = true
+      case s: Exchange if (s.getClass.getName.equals
+      ("org.apache.spark.sql.execution.exchange.ShuffleExchange") ||
+        s.getClass.getName.equals
+        ("org.apache.spark.sql.execution.exchange.ShuffleExchangeExec"))
+      => shuffleExists = true
     }
     assert(shuffleExists, "shuffle should exist on non bucket tables")
   }
@@ -146,7 +150,11 @@ class TableBucketingTestCase extends Spark2QueryTest with BeforeAndAfterAll {
       """.stripMargin).queryExecution.executedPlan
     var shuffleExists = false
     plan.collect {
-      case s: ShuffleExchange => shuffleExists = true
+      case s: Exchange if (s.getClass.getName.equals
+      ("org.apache.spark.sql.execution.exchange.ShuffleExchange") ||
+        s.getClass.getName.equals
+        ("org.apache.spark.sql.execution.exchange.ShuffleExchangeExec"))
+      => shuffleExists = true
     }
     assert(!shuffleExists, "shuffle should not exist on bucket tables")
   }
@@ -171,7 +179,11 @@ class TableBucketingTestCase extends Spark2QueryTest with BeforeAndAfterAll {
       """.stripMargin).queryExecution.executedPlan
     var shuffleExists = false
     plan.collect {
-      case s: ShuffleExchange => shuffleExists = true
+      case s: Exchange if (s.getClass.getName.equals
+      ("org.apache.spark.sql.execution.exchange.ShuffleExchange") ||
+        s.getClass.getName.equals
+        ("org.apache.spark.sql.execution.exchange.ShuffleExchangeExec"))
+      => shuffleExists = true
     }
     assert(!shuffleExists, "shuffle should not exist on bucket tables")
     sql("DROP TABLE bucketed_parquet_table")
@@ -196,7 +208,11 @@ class TableBucketingTestCase extends Spark2QueryTest with BeforeAndAfterAll {
       """.stripMargin).queryExecution.executedPlan
     var shuffleExists = false
     plan.collect {
-      case s: ShuffleExchange => shuffleExists = true
+      case s: Exchange if (s.getClass.getName.equals
+      ("org.apache.spark.sql.execution.exchange.ShuffleExchange") ||
+        s.getClass.getName.equals
+        ("org.apache.spark.sql.execution.exchange.ShuffleExchangeExec"))
+      => shuffleExists = true
     }
     assert(shuffleExists, "shuffle should exist on non bucket tables")
     sql("DROP TABLE parquet_table")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark2/src/test/scala/org/apache/spark/carbondata/query/SubQueryTestSuite.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/query/SubQueryTestSuite.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/query/SubQueryTestSuite.scala
index 66cf675..089dfd2 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/query/SubQueryTestSuite.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/query/SubQueryTestSuite.scala
@@ -19,7 +19,6 @@ package org.apache.spark.carbondata.query
 
 import org.apache.spark.sql.Row
 import org.apache.spark.sql.common.util.Spark2QueryTest
-import org.apache.spark.sql.execution.exchange.ShuffleExchange
 import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec
 import org.scalatest.BeforeAndAfterAll
 


[2/8] carbondata git commit: [CARBONDATA-2532][Integration] Carbon to support spark 2.3.1 version (Refactored streaming module and column vector to support spark 2.3.1)

Posted by ra...@apache.org.
[CARBONDATA-2532][Integration] Carbon to support spark 2.3.1 version (Refactored streaming module and column vector to support spark 2.3.1)

[CARBONDATA-2532][Integration] Carbon to support spark 2.3.1 version (Refactored streaming module and column vector to support spark 2.3.1)

a3f162f5bc8f02a5d1b0ce4feb35d47257ba537e [CARBONDATA-2532][Integration] Carbon to support spark 2.3.1 version(Make API changes in carbon to be compatible with spark 2.3)

e760f4ab3f43a1dac1a0478e60d7ab7afe94138d  [HOTFIX]Fixed test case failure due to bad record handling

In this PR inorder to hide the compatibility issues of columnar vector API's from the existing common classes, i introduced a proxy vector class, All the common classes of the carbon-spark intergration layer will use this proxy class to interct with ColumnarBatch/ColumnVector API of spark datasource.
thus proxy vector class will take care the compatibility issues with respect to spark different versions.

Column vector and Columnar Batch interface compatibility issues has been addressed in this PR, The changes were related to below modifications done in spark interface.

Highlights:
a) This is a refactoring of ColumnVector hierarchy and related classes.
b) make ColumnVector read-only.
c) introduce WritableColumnVector with write interface
d) remove ReadOnlyColumnVector

This closes #2642


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

Branch: refs/heads/master
Commit: bcef656dd6124b3962c4b4889dbd97a5f60233b2
Parents: cf74c88
Author: sujith71955 <su...@gmail.com>
Authored: Wed May 23 22:51:50 2018 -0700
Committer: ravipesala <ra...@gmail.com>
Committed: Wed Sep 5 18:07:43 2018 +0530

----------------------------------------------------------------------
 .../carbondata/spark/rdd/CarbonScanRDD.scala    |  12 +-
 .../carbondata/spark/rdd/StreamHandoffRDD.scala |  18 +-
 .../vectorreader/ColumnarVectorWrapper.java     |  67 +-
 .../VectorizedCarbonRecordReader.java           |  41 +-
 .../stream/CarbonStreamRecordReader.java        | 761 ++++++++++++++++++
 .../org/apache/spark/sql/CarbonVectorProxy.java | 226 ++++++
 .../org/apache/spark/sql/CarbonVectorProxy.java | 229 ++++++
 .../org/apache/spark/sql/CarbonVectorProxy.java | 255 ++++++
 .../apache/spark/sql/ColumnVectorFactory.java   |  45 ++
 .../streaming/CarbonStreamInputFormat.java      |  46 +-
 .../streaming/CarbonStreamRecordReader.java     | 772 -------------------
 .../carbondata/streaming/CarbonStreamUtils.java |  40 +
 .../streaming/StreamBlockletReader.java         |  39 +-
 13 files changed, 1693 insertions(+), 858 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/bcef656d/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
index e88ae81..b712109 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
@@ -62,7 +62,7 @@ import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil
 import org.apache.carbondata.processing.util.CarbonLoaderUtil
 import org.apache.carbondata.spark.InitInputMetrics
 import org.apache.carbondata.spark.util.Util
-import org.apache.carbondata.streaming.{CarbonStreamInputFormat, CarbonStreamRecordReader}
+import org.apache.carbondata.streaming.CarbonStreamInputFormat
 
 /**
  * This RDD is used to perform query on CarbonData file. Before sending tasks to scan
@@ -421,13 +421,13 @@ class CarbonScanRDD[T: ClassTag](
           // create record reader for row format
           DataTypeUtil.setDataTypeConverter(dataTypeConverterClz.newInstance())
           val inputFormat = new CarbonStreamInputFormat
-          val streamReader = inputFormat.createRecordReader(inputSplit, attemptContext)
-            .asInstanceOf[CarbonStreamRecordReader]
-          streamReader.setVectorReader(vectorReader)
-          streamReader.setInputMetricsStats(inputMetricsStats)
+          inputFormat.setVectorReader(vectorReader)
+          inputFormat.setInputMetricsStats(inputMetricsStats)
           model.setStatisticsRecorder(
             CarbonTimeStatisticsFactory.createExecutorRecorder(model.getQueryId))
-          streamReader.setQueryModel(model)
+          inputFormat.setModel(model)
+          val streamReader = inputFormat.createRecordReader(inputSplit, attemptContext)
+            .asInstanceOf[RecordReader[Void, Object]]
           streamReader
         case _ =>
           // create record reader for CarbonData file format

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bcef656d/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/StreamHandoffRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/StreamHandoffRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/StreamHandoffRDD.scala
index 994cb3d..3cbfab2 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/StreamHandoffRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/StreamHandoffRDD.scala
@@ -22,7 +22,7 @@ import java.util
 import java.util.{Date, UUID}
 
 import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.mapreduce.{Job, TaskAttemptID, TaskType}
+import org.apache.hadoop.mapreduce.{Job, RecordReader, TaskAttemptID, TaskType}
 import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
 import org.apache.spark.{Partition, SerializableWritable, SparkContext, TaskContext}
 import org.apache.spark.sql.SparkSession
@@ -36,6 +36,7 @@ import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.locks.{CarbonLockFactory, LockUsage}
 import org.apache.carbondata.core.metadata.CarbonMetadata
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.scan.model.QueryModel
 import org.apache.carbondata.core.scan.result.iterator.RawResultIterator
 import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatus, SegmentStatusManager}
 import org.apache.carbondata.core.util.{CarbonUtil, DataTypeUtil}
@@ -48,8 +49,13 @@ import org.apache.carbondata.processing.loading.model.CarbonLoadModel
 import org.apache.carbondata.processing.merger.{CompactionResultSortProcessor, CompactionType}
 import org.apache.carbondata.processing.util.CarbonLoaderUtil
 import org.apache.carbondata.spark.{HandoffResult, HandoffResultImpl}
+<<<<<<< 2f537b724f6f03ab40c95f7ecc8ebd38f6500099
 import org.apache.carbondata.spark.util.CommonUtil
 import org.apache.carbondata.streaming.{CarbonStreamInputFormat, CarbonStreamRecordReader}
+=======
+import org.apache.carbondata.spark.util.{CommonUtil, SparkDataTypeConverterImpl}
+import org.apache.carbondata.streaming.CarbonStreamInputFormat
+>>>>>>> [CARBONDATA-2532][Integration] Carbon to support spark 2.3 version, ColumnVector Interface
 
 
 /**
@@ -75,7 +81,7 @@ class HandoffPartition(
  * and we can extract it later
  */
 class StreamingRawResultIterator(
-    recordReader: CarbonStreamRecordReader
+    recordReader: RecordReader[Void, Any]
 ) extends RawResultIterator(null, null, null) {
 
   override def hasNext: Boolean = {
@@ -162,10 +168,10 @@ class StreamHandoffRDD[K, V](
     val model = format.createQueryModel(inputSplit, attemptContext)
     val inputFormat = new CarbonStreamInputFormat
     val streamReader = inputFormat.createRecordReader(inputSplit, attemptContext)
-      .asInstanceOf[CarbonStreamRecordReader]
-    streamReader.setVectorReader(false)
-    streamReader.setQueryModel(model)
-    streamReader.setUseRawRow(true)
+      .asInstanceOf[RecordReader[Void, Any]]
+    inputFormat.setVectorReader(false)
+    inputFormat.setModel(model)
+    inputFormat.setUseRawRow(true)
     streamReader.initialize(inputSplit, attemptContext)
     val iteratorList = new util.ArrayList[RawResultIterator](1)
     iteratorList.add(new StreamingRawResultIterator(streamReader))

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bcef656d/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/ColumnarVectorWrapper.java
----------------------------------------------------------------------
diff --git a/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/ColumnarVectorWrapper.java b/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/ColumnarVectorWrapper.java
index 4f34650..7bab117 100644
--- a/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/ColumnarVectorWrapper.java
+++ b/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/ColumnarVectorWrapper.java
@@ -24,48 +24,46 @@ import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
 import org.apache.carbondata.core.scan.result.vector.CarbonDictionary;
 
 import org.apache.parquet.column.Encoding;
+import org.apache.spark.sql.CarbonVectorProxy;
 import org.apache.spark.sql.carbondata.execution.datasources.CarbonSparkDataSourceUtil;
-import org.apache.spark.sql.execution.vectorized.ColumnVector;
 import org.apache.spark.sql.types.Decimal;
 
 class ColumnarVectorWrapper implements CarbonColumnVector {
 
-  private ColumnVector columnVector;
+  private CarbonVectorProxy writableColumnVector;
 
   private boolean[] filteredRows;
 
   private int counter;
 
+  private int ordinal;
+
   private boolean filteredRowsExist;
 
   private DataType blockDataType;
 
-  private CarbonColumnVector dictionaryVector;
-
-  ColumnarVectorWrapper(ColumnVector columnVector, boolean[] filteredRows) {
-    this.columnVector = columnVector;
+  ColumnarVectorWrapper(CarbonVectorProxy writableColumnVector,
+                        boolean[] filteredRows, int ordinal) {
+    this.writableColumnVector = writableColumnVector;
     this.filteredRows = filteredRows;
-    if (columnVector.getDictionaryIds() != null) {
-      this.dictionaryVector =
-          new ColumnarVectorWrapper(columnVector.getDictionaryIds(), filteredRows);
-    }
+    this.ordinal = ordinal;
   }
 
   @Override public void putBoolean(int rowId, boolean value) {
     if (!filteredRows[rowId]) {
-      columnVector.putBoolean(counter++, value);
+      writableColumnVector.putBoolean(counter++, value, ordinal);
     }
   }
 
   @Override public void putFloat(int rowId, float value) {
     if (!filteredRows[rowId]) {
-      columnVector.putFloat(counter++, value);
+      writableColumnVector.putFloat(counter++, value,ordinal);
     }
   }
 
   @Override public void putShort(int rowId, short value) {
     if (!filteredRows[rowId]) {
-      columnVector.putShort(counter++, value);
+      writableColumnVector.putShort(counter++, value, ordinal);
     }
   }
 
@@ -73,18 +71,18 @@ class ColumnarVectorWrapper implements CarbonColumnVector {
     if (filteredRowsExist) {
       for (int i = 0; i < count; i++) {
         if (!filteredRows[rowId]) {
-          columnVector.putShort(counter++, value);
+          writableColumnVector.putShort(counter++, value, ordinal);
         }
         rowId++;
       }
     } else {
-      columnVector.putShorts(rowId, count, value);
+      writableColumnVector.putShorts(rowId, count, value, ordinal);
     }
   }
 
   @Override public void putInt(int rowId, int value) {
     if (!filteredRows[rowId]) {
-      columnVector.putInt(counter++, value);
+      writableColumnVector.putInt(counter++, value, ordinal);
     }
   }
 
@@ -92,18 +90,18 @@ class ColumnarVectorWrapper implements CarbonColumnVector {
     if (filteredRowsExist) {
       for (int i = 0; i < count; i++) {
         if (!filteredRows[rowId]) {
-          columnVector.putInt(counter++, value);
+          writableColumnVector.putInt(counter++, value, ordinal);
         }
         rowId++;
       }
     } else {
-      columnVector.putInts(rowId, count, value);
+      writableColumnVector.putInts(rowId, count, value, ordinal);
     }
   }
 
   @Override public void putLong(int rowId, long value) {
     if (!filteredRows[rowId]) {
-      columnVector.putLong(counter++, value);
+      writableColumnVector.putLong(counter++, value, ordinal);
     }
   }
 
@@ -111,19 +109,19 @@ class ColumnarVectorWrapper implements CarbonColumnVector {
     if (filteredRowsExist) {
       for (int i = 0; i < count; i++) {
         if (!filteredRows[rowId]) {
-          columnVector.putLong(counter++, value);
+          writableColumnVector.putLong(counter++, value, ordinal);
         }
         rowId++;
       }
     } else {
-      columnVector.putLongs(rowId, count, value);
+      writableColumnVector.putLongs(rowId, count, value, ordinal);
     }
   }
 
   @Override public void putDecimal(int rowId, BigDecimal value, int precision) {
     if (!filteredRows[rowId]) {
       Decimal toDecimal = Decimal.apply(value);
-      columnVector.putDecimal(counter++, toDecimal, precision);
+      writableColumnVector.putDecimal(counter++, toDecimal, precision, ordinal);
     }
   }
 
@@ -131,7 +129,7 @@ class ColumnarVectorWrapper implements CarbonColumnVector {
     Decimal decimal = Decimal.apply(value);
     for (int i = 0; i < count; i++) {
       if (!filteredRows[rowId]) {
-        columnVector.putDecimal(counter++, decimal, precision);
+        writableColumnVector.putDecimal(counter++, decimal, precision, ordinal);
       }
       rowId++;
     }
@@ -139,7 +137,7 @@ class ColumnarVectorWrapper implements CarbonColumnVector {
 
   @Override public void putDouble(int rowId, double value) {
     if (!filteredRows[rowId]) {
-      columnVector.putDouble(counter++, value);
+      writableColumnVector.putDouble(counter++, value, ordinal);
     }
   }
 
@@ -147,25 +145,25 @@ class ColumnarVectorWrapper implements CarbonColumnVector {
     if (filteredRowsExist) {
       for (int i = 0; i < count; i++) {
         if (!filteredRows[rowId]) {
-          columnVector.putDouble(counter++, value);
+          writableColumnVector.putDouble(counter++, value, ordinal);
         }
         rowId++;
       }
     } else {
-      columnVector.putDoubles(rowId, count, value);
+      writableColumnVector.putDoubles(rowId, count, value, ordinal);
     }
   }
 
   @Override public void putBytes(int rowId, byte[] value) {
     if (!filteredRows[rowId]) {
-      columnVector.putByteArray(counter++, value);
+      writableColumnVector.putByteArray(counter++, value, ordinal);
     }
   }
 
   @Override public void putBytes(int rowId, int count, byte[] value) {
     for (int i = 0; i < count; i++) {
       if (!filteredRows[rowId]) {
-        columnVector.putByteArray(counter++, value);
+        writableColumnVector.putByteArray(counter++, value, ordinal);
       }
       rowId++;
     }
@@ -173,13 +171,13 @@ class ColumnarVectorWrapper implements CarbonColumnVector {
 
   @Override public void putBytes(int rowId, int offset, int length, byte[] value) {
     if (!filteredRows[rowId]) {
-      columnVector.putByteArray(counter++, value, offset, length);
+      writableColumnVector.putByteArray(counter++, value, offset, length, ordinal);
     }
   }
 
   @Override public void putNull(int rowId) {
     if (!filteredRows[rowId]) {
-      columnVector.putNull(counter++);
+      writableColumnVector.putNull(counter++, ordinal);
     }
   }
 
@@ -187,12 +185,12 @@ class ColumnarVectorWrapper implements CarbonColumnVector {
     if (filteredRowsExist) {
       for (int i = 0; i < count; i++) {
         if (!filteredRows[rowId]) {
-          columnVector.putNull(counter++);
+          writableColumnVector.putNull(counter++, ordinal);
         }
         rowId++;
       }
     } else {
-      columnVector.putNulls(rowId, count);
+      writableColumnVector.putNulls(rowId, count,ordinal);
     }
   }
 
@@ -216,7 +214,7 @@ class ColumnarVectorWrapper implements CarbonColumnVector {
   }
 
   @Override public boolean isNull(int rowId) {
-    return columnVector.isNullAt(rowId);
+    return writableColumnVector.isNullAt(rowId,ordinal);
   }
 
   @Override public void putObject(int rowId, Object obj) {
@@ -237,7 +235,8 @@ class ColumnarVectorWrapper implements CarbonColumnVector {
   }
 
   @Override public DataType getType() {
-    return CarbonSparkDataSourceUtil.convertSparkToCarbonDataType(columnVector.dataType());
+    return CarbonSparkDataSourceUtil
+        .convertSparkToCarbonDataType(writableColumnVector.dataType(ordinal));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bcef656d/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
----------------------------------------------------------------------
diff --git a/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java b/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
index f237552..7c98608 100644
--- a/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
+++ b/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
@@ -51,13 +51,16 @@ import org.apache.commons.lang3.exception.ExceptionUtils;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.spark.memory.MemoryMode;
+<<<<<<< 2f537b724f6f03ab40c95f7ecc8ebd38f6500099:integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
 import org.apache.spark.sql.carbondata.execution.datasources.CarbonSparkDataSourceUtil;
 import org.apache.spark.sql.catalyst.InternalRow;
 import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
 import org.apache.spark.sql.execution.vectorized.ColumnarBatch;
+=======
+import org.apache.spark.sql.CarbonVectorProxy;
+>>>>>>> [CARBONDATA-2532][Integration] Carbon to support spark 2.3 version, ColumnVector Interface:integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
 import org.apache.spark.sql.types.DecimalType;
 import org.apache.spark.sql.types.StructField;
-import org.apache.spark.sql.types.StructType;
 
 /**
  * A specialized RecordReader that reads into InternalRows or ColumnarBatches directly using the
@@ -70,9 +73,11 @@ public class VectorizedCarbonRecordReader extends AbstractRecordReader<Object> {
 
   private int batchIdx = 0;
 
+  private static final int DEFAULT_BATCH_SIZE = 4 * 1024;
+
   private int numBatched = 0;
 
-  private ColumnarBatch columnarBatch;
+  private CarbonVectorProxy vectorProxy;
 
   private CarbonColumnarBatch carbonColumnarBatch;
 
@@ -159,9 +164,9 @@ public class VectorizedCarbonRecordReader extends AbstractRecordReader<Object> {
   @Override
   public void close() throws IOException {
     logStatistics(rowCount, queryModel.getStatisticsRecorder());
-    if (columnarBatch != null) {
-      columnarBatch.close();
-      columnarBatch = null;
+    if (vectorProxy != null) {
+      vectorProxy.close();
+      vectorProxy = null;
     }
     // clear dictionary cache
     Map<String, Dictionary> columnToDictionaryMapping = queryModel.getColumnToDictionaryMapping();
@@ -195,15 +200,15 @@ public class VectorizedCarbonRecordReader extends AbstractRecordReader<Object> {
   @Override
   public Object getCurrentValue() throws IOException, InterruptedException {
     if (returnColumnarBatch) {
-      int value = columnarBatch.numValidRows();
+      int value = vectorProxy.numRows();
       rowCount += value;
       if (inputMetricsStats != null) {
         inputMetricsStats.incrementRecordRead((long) value);
       }
-      return columnarBatch;
+      return vectorProxy.getColumnarBatch();
     }
     rowCount += 1;
-    return columnarBatch.getRow(batchIdx - 1);
+    return vectorProxy.getRow(batchIdx - 1);
   }
 
   @Override
@@ -271,7 +276,6 @@ public class VectorizedCarbonRecordReader extends AbstractRecordReader<Object> {
             CarbonSparkDataSourceUtil.convertCarbonToSparkDataType(DataTypes.DOUBLE), true, null);
       }
     }
-
     StructType schema = new StructType(fields);
     if (partitionColumns != null) {
       for (StructField field : partitionColumns.fields()) {
@@ -286,15 +290,16 @@ public class VectorizedCarbonRecordReader extends AbstractRecordReader<Object> {
         columnarBatch.column(i + partitionIdx).setIsConstant();
       }
     }
+    vectorProxy = new CarbonVectorProxy(MemoryMode.OFF_HEAP,DEFAULT_BATCH_SIZE,fields);
     CarbonColumnVector[] vectors = new CarbonColumnVector[fields.length];
-    boolean[] filteredRows = new boolean[columnarBatch.capacity()];
+    boolean[] filteredRows = new boolean[vectorProxy.numRows()];
     for (int i = 0; i < fields.length; i++) {
-      if (isNoDictStringField[i]) {
-        columnarBatch.column(i).reserveDictionaryIds(columnarBatch.capacity());
+    if (isNoDictStringField[i]) {
+      vectorProxy.reserveDictionaryIds(vectorProxy.numRows(), i);
       }
-      vectors[i] = new ColumnarVectorWrapper(columnarBatch.column(i), filteredRows);
+      vectors[i] = new ColumnarVectorWrapper(vectorProxy, filteredRows, i);
     }
-    carbonColumnarBatch = new CarbonColumnarBatch(vectors, columnarBatch.capacity(), filteredRows);
+    carbonColumnarBatch = new CarbonColumnarBatch(vectors, vectorProxy.numRows(), filteredRows);
   }
 
   private void initBatch() {
@@ -302,7 +307,7 @@ public class VectorizedCarbonRecordReader extends AbstractRecordReader<Object> {
   }
 
   private void resultBatch() {
-    if (columnarBatch == null) initBatch();
+    if (vectorProxy == null) initBatch();
   }
 
 
@@ -314,16 +319,16 @@ public class VectorizedCarbonRecordReader extends AbstractRecordReader<Object> {
     if (null != isNoDictStringField) {
       for (int i = 0; i < isNoDictStringField.length; i++) {
         if (isNoDictStringField[i]) {
-          columnarBatch.column(i).getDictionaryIds().reset();
+          vectorProxy.resetDictionaryIds(i);
         }
       }
     }
-    columnarBatch.reset();
+    vectorProxy.reset();
     carbonColumnarBatch.reset();
     if (iterator.hasNext()) {
       iterator.processNextBatch(carbonColumnarBatch);
       int actualSize = carbonColumnarBatch.getActualSize();
-      columnarBatch.setNumRows(actualSize);
+      vectorProxy.setNumRows(actualSize);
       numBatched = actualSize;
       batchIdx = 0;
       return true;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bcef656d/integration/spark2/src/main/scala/org/apache/carbondata/stream/CarbonStreamRecordReader.java
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/stream/CarbonStreamRecordReader.java b/integration/spark2/src/main/scala/org/apache/carbondata/stream/CarbonStreamRecordReader.java
new file mode 100644
index 0000000..124413d
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/stream/CarbonStreamRecordReader.java
@@ -0,0 +1,761 @@
+/*
+ * 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.stream;
+
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Method;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.cache.Cache;
+import org.apache.carbondata.core.cache.CacheProvider;
+import org.apache.carbondata.core.cache.CacheType;
+import org.apache.carbondata.core.cache.dictionary.Dictionary;
+import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.block.SegmentProperties;
+import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
+import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
+import org.apache.carbondata.core.metadata.blocklet.index.BlockletMinMaxIndex;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.metadata.encoder.Encoding;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
+import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.core.reader.CarbonHeaderReader;
+import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
+import org.apache.carbondata.core.scan.filter.FilterUtil;
+import org.apache.carbondata.core.scan.filter.GenericQueryType;
+import org.apache.carbondata.core.scan.filter.executer.FilterExecuter;
+import org.apache.carbondata.core.scan.filter.intf.RowImpl;
+import org.apache.carbondata.core.scan.filter.intf.RowIntf;
+import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
+import org.apache.carbondata.core.scan.model.QueryModel;
+import org.apache.carbondata.core.util.CarbonMetadataUtil;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.DataTypeUtil;
+import org.apache.carbondata.format.BlockletHeader;
+import org.apache.carbondata.format.FileHeader;
+import org.apache.carbondata.hadoop.CarbonInputSplit;
+import org.apache.carbondata.hadoop.CarbonMultiBlockSplit;
+import org.apache.carbondata.hadoop.InputMetricsStats;
+import org.apache.carbondata.hadoop.api.CarbonTableInputFormat;
+import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
+import org.apache.carbondata.streaming.CarbonStreamInputFormat;
+import org.apache.carbondata.streaming.CarbonStreamUtils;
+import org.apache.carbondata.streaming.StreamBlockletReader;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+
+import org.apache.spark.memory.MemoryMode;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * Stream record reader
+ */
+public class CarbonStreamRecordReader extends RecordReader<Void, Object> {
+  // vector reader
+  private boolean isVectorReader;
+
+  // metadata
+  private CarbonTable carbonTable;
+  private CarbonColumn[] storageColumns;
+  private boolean[] isRequired;
+  private DataType[] measureDataTypes;
+  private int dimensionCount;
+  private int measureCount;
+
+  // input
+  private FileSplit fileSplit;
+  private Configuration hadoopConf;
+  private StreamBlockletReader input;
+  private boolean isFirstRow = true;
+  private QueryModel model;
+
+  // decode data
+  private BitSet allNonNull;
+  private boolean[] isNoDictColumn;
+  private DirectDictionaryGenerator[] directDictionaryGenerators;
+  private CacheProvider cacheProvider;
+  private Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache;
+  private GenericQueryType[] queryTypes;
+
+  // vectorized reader
+  private StructType outputSchema;
+  private Object vectorProxy;
+  private boolean isFinished = false;
+
+  // filter
+  private FilterExecuter filter;
+  private boolean[] isFilterRequired;
+  private Object[] filterValues;
+  private RowIntf filterRow;
+  private int[] filterMap;
+
+  // output
+  private CarbonColumn[] projection;
+  private boolean[] isProjectionRequired;
+  private int[] projectionMap;
+  private Object[] outputValues;
+  private InternalRow outputRow;
+
+  // empty project, null filter
+  private boolean skipScanData;
+
+  // return raw row for handoff
+  private boolean useRawRow = false;
+
+  // InputMetricsStats
+  private InputMetricsStats inputMetricsStats;
+
+  private static final LogService LOGGER =
+          LogServiceFactory.getLogService(CarbonStreamRecordReader.class.getName());
+
+  public CarbonStreamRecordReader(boolean isVectorReader, InputMetricsStats inputMetricsStats,
+      QueryModel mdl, boolean useRawRow) {
+    this.isVectorReader = isVectorReader;
+    this.inputMetricsStats = inputMetricsStats;
+    this.model = mdl;
+    this.useRawRow = useRawRow;
+
+  }
+  @Override public void initialize(InputSplit split, TaskAttemptContext context)
+      throws IOException, InterruptedException {
+    // input
+    if (split instanceof CarbonInputSplit) {
+      fileSplit = (CarbonInputSplit) split;
+    } else if (split instanceof CarbonMultiBlockSplit) {
+      fileSplit = ((CarbonMultiBlockSplit) split).getAllSplits().get(0);
+    } else {
+      fileSplit = (FileSplit) split;
+    }
+
+    // metadata
+    hadoopConf = context.getConfiguration();
+    if (model == null) {
+      CarbonTableInputFormat format = new CarbonTableInputFormat<Object>();
+      model = format.createQueryModel(split, context);
+    }
+    carbonTable = model.getTable();
+    List<CarbonDimension> dimensions =
+        carbonTable.getDimensionByTableName(carbonTable.getTableName());
+    dimensionCount = dimensions.size();
+    List<CarbonMeasure> measures =
+        carbonTable.getMeasureByTableName(carbonTable.getTableName());
+    measureCount = measures.size();
+    List<CarbonColumn> carbonColumnList =
+        carbonTable.getStreamStorageOrderColumn(carbonTable.getTableName());
+    storageColumns = carbonColumnList.toArray(new CarbonColumn[carbonColumnList.size()]);
+    isNoDictColumn = CarbonDataProcessorUtil.getNoDictionaryMapping(storageColumns);
+    directDictionaryGenerators = new DirectDictionaryGenerator[storageColumns.length];
+    for (int i = 0; i < storageColumns.length; i++) {
+      if (storageColumns[i].hasEncoding(Encoding.DIRECT_DICTIONARY)) {
+        directDictionaryGenerators[i] = DirectDictionaryKeyGeneratorFactory
+            .getDirectDictionaryGenerator(storageColumns[i].getDataType());
+      }
+    }
+    measureDataTypes = new DataType[measureCount];
+    for (int i = 0; i < measureCount; i++) {
+      measureDataTypes[i] = storageColumns[dimensionCount + i].getDataType();
+    }
+
+    // decode data
+    allNonNull = new BitSet(storageColumns.length);
+    projection = model.getProjectionColumns();
+
+    isRequired = new boolean[storageColumns.length];
+    boolean[] isFiltlerDimensions = model.getIsFilterDimensions();
+    boolean[] isFiltlerMeasures = model.getIsFilterMeasures();
+    isFilterRequired = new boolean[storageColumns.length];
+    filterMap = new int[storageColumns.length];
+    for (int i = 0; i < storageColumns.length; i++) {
+      if (storageColumns[i].isDimension()) {
+        if (isFiltlerDimensions[storageColumns[i].getOrdinal()]) {
+          isRequired[i] = true;
+          isFilterRequired[i] = true;
+          filterMap[i] = storageColumns[i].getOrdinal();
+        }
+      } else {
+        if (isFiltlerMeasures[storageColumns[i].getOrdinal()]) {
+          isRequired[i] = true;
+          isFilterRequired[i] = true;
+          filterMap[i] = carbonTable.getDimensionOrdinalMax() + storageColumns[i].getOrdinal();
+        }
+      }
+    }
+
+    isProjectionRequired = new boolean[storageColumns.length];
+    projectionMap = new int[storageColumns.length];
+    for (int j = 0; j < projection.length; j++) {
+      for (int i = 0; i < storageColumns.length; i++) {
+        if (storageColumns[i].getColName().equals(projection[j].getColName())) {
+          isRequired[i] = true;
+          isProjectionRequired[i] = true;
+          projectionMap[i] = j;
+          break;
+        }
+      }
+    }
+
+    // initialize filter
+    if (null != model.getFilterExpressionResolverTree()) {
+      initializeFilter();
+    } else if (projection.length == 0) {
+      skipScanData = true;
+    }
+
+  }
+
+  private void initializeFilter() {
+
+    List<ColumnSchema> wrapperColumnSchemaList = CarbonUtil
+        .getColumnSchemaList(carbonTable.getDimensionByTableName(carbonTable.getTableName()),
+            carbonTable.getMeasureByTableName(carbonTable.getTableName()));
+    int[] dimLensWithComplex = new int[wrapperColumnSchemaList.size()];
+    for (int i = 0; i < dimLensWithComplex.length; i++) {
+      dimLensWithComplex[i] = Integer.MAX_VALUE;
+    }
+
+    int[] dictionaryColumnCardinality =
+        CarbonUtil.getFormattedCardinality(dimLensWithComplex, wrapperColumnSchemaList);
+    SegmentProperties segmentProperties =
+        new SegmentProperties(wrapperColumnSchemaList, dictionaryColumnCardinality);
+    Map<Integer, GenericQueryType> complexDimensionInfoMap = new HashMap<>();
+
+    FilterResolverIntf resolverIntf = model.getFilterExpressionResolverTree();
+    filter = FilterUtil.getFilterExecuterTree(resolverIntf, segmentProperties,
+        complexDimensionInfoMap);
+    // for row filter, we need update column index
+    FilterUtil.updateIndexOfColumnExpression(resolverIntf.getFilterExpression(),
+        carbonTable.getDimensionOrdinalMax());
+
+  }
+
+  private byte[] getSyncMarker(String filePath) throws IOException {
+    CarbonHeaderReader headerReader = new CarbonHeaderReader(filePath);
+    FileHeader header = headerReader.readHeader();
+    return header.getSync_marker();
+  }
+
+  private void initializeAtFirstRow() throws IOException {
+    filterValues = new Object[carbonTable.getDimensionOrdinalMax() + measureCount];
+    filterRow = new RowImpl();
+    filterRow.setValues(filterValues);
+
+    outputValues = new Object[projection.length];
+    outputRow = new GenericInternalRow(outputValues);
+
+    Path file = fileSplit.getPath();
+
+    byte[] syncMarker = getSyncMarker(file.toString());
+
+    FileSystem fs = file.getFileSystem(hadoopConf);
+
+    int bufferSize = Integer.parseInt(hadoopConf.get(CarbonStreamInputFormat.READ_BUFFER_SIZE,
+        CarbonStreamInputFormat.READ_BUFFER_SIZE_DEFAULT));
+
+    FSDataInputStream fileIn = fs.open(file, bufferSize);
+    fileIn.seek(fileSplit.getStart());
+    input = new StreamBlockletReader(syncMarker, fileIn, fileSplit.getLength(),
+        fileSplit.getStart() == 0);
+
+    cacheProvider = CacheProvider.getInstance();
+    cache = cacheProvider.createCache(CacheType.FORWARD_DICTIONARY);
+    queryTypes = CarbonStreamInputFormat.getComplexDimensions(carbonTable, storageColumns, cache);
+
+    outputSchema = new StructType((StructField[])
+        DataTypeUtil.getDataTypeConverter().convertCarbonSchemaToSparkSchema(projection));
+  }
+
+  @Override public boolean nextKeyValue() throws IOException, InterruptedException {
+    if (isFirstRow) {
+      isFirstRow = false;
+      initializeAtFirstRow();
+    }
+    if (isFinished) {
+      return false;
+    }
+
+    if (isVectorReader) {
+      return nextColumnarBatch();
+    }
+
+    return nextRow();
+  }
+
+  /**
+   * for vector reader, check next columnar batch
+   */
+  private boolean nextColumnarBatch() throws IOException {
+    boolean hasNext;
+    boolean scanMore = false;
+    do {
+      // move to the next blocklet
+      hasNext = input.nextBlocklet();
+      if (hasNext) {
+        // read blocklet header
+        BlockletHeader header = input.readBlockletHeader();
+        if (isScanRequired(header)) {
+          scanMore = !scanBlockletAndFillVector(header);
+        } else {
+          input.skipBlockletData(true);
+          scanMore = true;
+        }
+      } else {
+        isFinished = true;
+        scanMore = false;
+      }
+    } while (scanMore);
+    return hasNext;
+  }
+
+  /**
+   * check next Row
+   */
+  private boolean nextRow() throws IOException {
+    // read row one by one
+    try {
+      boolean hasNext;
+      boolean scanMore = false;
+      do {
+        hasNext = input.hasNext();
+        if (hasNext) {
+          if (skipScanData) {
+            input.nextRow();
+            scanMore = false;
+          } else {
+            if (useRawRow) {
+              // read raw row for streaming handoff which does not require decode raw row
+              readRawRowFromStream();
+            } else {
+              readRowFromStream();
+            }
+            if (null != filter) {
+              scanMore = !filter.applyFilter(filterRow, carbonTable.getDimensionOrdinalMax());
+            } else {
+              scanMore = false;
+            }
+          }
+        } else {
+          if (input.nextBlocklet()) {
+            BlockletHeader header = input.readBlockletHeader();
+            if (isScanRequired(header)) {
+              if (skipScanData) {
+                input.skipBlockletData(false);
+              } else {
+                input.readBlockletData(header);
+              }
+            } else {
+              input.skipBlockletData(true);
+            }
+            scanMore = true;
+          } else {
+            isFinished = true;
+            scanMore = false;
+          }
+        }
+      } while (scanMore);
+      return hasNext;
+    } catch (FilterUnsupportedException e) {
+      throw new IOException("Failed to filter row in detail reader", e);
+    }
+  }
+
+  @Override public Void getCurrentKey() throws IOException, InterruptedException {
+    return null;
+  }
+
+  @Override public Object getCurrentValue() throws IOException, InterruptedException {
+    if (isVectorReader) {
+      Method method = null;
+      try {
+        method = vectorProxy.getClass().getMethod("numRows");
+        int value = (int) method.invoke(vectorProxy);
+        if (inputMetricsStats != null) {
+          inputMetricsStats.incrementRecordRead((long) value);
+        }
+        method = vectorProxy.getClass().getMethod("getColumnarBatch");
+        return method.invoke(vectorProxy);
+      } catch (Exception e) {
+        throw new IOException(e);
+      }
+    }
+
+    if (inputMetricsStats != null) {
+      inputMetricsStats.incrementRecordRead(1L);
+    }
+
+    return outputRow;
+  }
+
+  private boolean isScanRequired(BlockletHeader header) {
+    if (filter != null && header.getBlocklet_index() != null) {
+      BlockletMinMaxIndex minMaxIndex = CarbonMetadataUtil.convertExternalMinMaxIndex(
+          header.getBlocklet_index().getMin_max_index());
+      if (minMaxIndex != null) {
+        BitSet bitSet =
+            filter.isScanRequired(minMaxIndex.getMaxValues(), minMaxIndex.getMinValues());
+        if (bitSet.isEmpty()) {
+          return false;
+        } else {
+          return true;
+        }
+      }
+    }
+    return true;
+  }
+
+  private boolean scanBlockletAndFillVector(BlockletHeader header) throws IOException {
+    Constructor cons = null;
+    // if filter is null and output projection is empty, use the row number of blocklet header
+    int rowNum = 0;
+    String methodName = "setNumRows";
+    try {
+      String vectorReaderClassName = "org.apache.spark.sql.CarbonVectorProxy";
+      cons = CarbonStreamUtils.getConstructorWithReflection(vectorReaderClassName, MemoryMode.class,
+              StructType.class, int.class);
+      if (skipScanData) {
+
+        int rowNums = header.getBlocklet_info().getNum_rows();
+        vectorProxy = cons.newInstance(MemoryMode.OFF_HEAP, outputSchema, rowNums);
+        Method setNumRowsMethod = vectorProxy.getClass().getMethod(methodName, int.class);
+        setNumRowsMethod.invoke(vectorProxy, rowNums);
+        input.skipBlockletData(true);
+        return rowNums > 0;
+      }
+      input.readBlockletData(header);
+      vectorProxy = cons.newInstance(MemoryMode.OFF_HEAP,outputSchema, input.getRowNums());
+      if (null == filter) {
+        while (input.hasNext()) {
+          readRowFromStream();
+          putRowToColumnBatch(rowNum++);
+        }
+      } else {
+        try {
+          while (input.hasNext()) {
+            readRowFromStream();
+            if (filter.applyFilter(filterRow, carbonTable.getDimensionOrdinalMax())) {
+              putRowToColumnBatch(rowNum++);
+            }
+          }
+        } catch (FilterUnsupportedException e) {
+          throw new IOException("Failed to filter row in vector reader", e);
+        }
+      }
+      Method setNumRowsMethod = vectorProxy.getClass().getMethod(methodName, int.class);
+      setNumRowsMethod.invoke(vectorProxy, rowNum);
+    } catch (Exception e) {
+      throw new IOException("Failed to fill row in  vector reader", e);
+    }
+    return rowNum > 0;
+  }
+
+  private void readRowFromStream() {
+    input.nextRow();
+    short nullLen = input.readShort();
+    BitSet nullBitSet = allNonNull;
+    if (nullLen > 0) {
+      nullBitSet = BitSet.valueOf(input.readBytes(nullLen));
+    }
+    int colCount = 0;
+    // primitive type dimension
+    for (; colCount < isNoDictColumn.length; colCount++) {
+      if (nullBitSet.get(colCount)) {
+        if (isFilterRequired[colCount]) {
+          filterValues[filterMap[colCount]] = CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY;
+        }
+        if (isProjectionRequired[colCount]) {
+          outputValues[projectionMap[colCount]] = null;
+        }
+      } else {
+        if (isNoDictColumn[colCount]) {
+          int v = input.readShort();
+          if (isRequired[colCount]) {
+            byte[] b = input.readBytes(v);
+            if (isFilterRequired[colCount]) {
+              filterValues[filterMap[colCount]] = b;
+            }
+            if (isProjectionRequired[colCount]) {
+              outputValues[projectionMap[colCount]] =
+                  DataTypeUtil.getDataBasedOnDataTypeForNoDictionaryColumn(b,
+                      storageColumns[colCount].getDataType());
+            }
+          } else {
+            input.skipBytes(v);
+          }
+        } else if (null != directDictionaryGenerators[colCount]) {
+          if (isRequired[colCount]) {
+            if (isFilterRequired[colCount]) {
+              filterValues[filterMap[colCount]] = input.copy(4);
+            }
+            if (isProjectionRequired[colCount]) {
+              outputValues[projectionMap[colCount]] =
+                  directDictionaryGenerators[colCount].getValueFromSurrogate(input.readInt());
+            } else {
+              input.skipBytes(4);
+            }
+          } else {
+            input.skipBytes(4);
+          }
+        } else {
+          if (isRequired[colCount]) {
+            if (isFilterRequired[colCount]) {
+              filterValues[filterMap[colCount]] = input.copy(4);
+            }
+            if (isProjectionRequired[colCount]) {
+              outputValues[projectionMap[colCount]] = input.readInt();
+            } else {
+              input.skipBytes(4);
+            }
+          } else {
+            input.skipBytes(4);
+          }
+        }
+      }
+    }
+    // complex type dimension
+    for (; colCount < dimensionCount; colCount++) {
+      if (nullBitSet.get(colCount)) {
+        if (isFilterRequired[colCount]) {
+          filterValues[filterMap[colCount]] = null;
+        }
+        if (isProjectionRequired[colCount]) {
+          outputValues[projectionMap[colCount]] = null;
+        }
+      } else {
+        short v = input.readShort();
+        if (isRequired[colCount]) {
+          byte[] b = input.readBytes(v);
+          if (isFilterRequired[colCount]) {
+            filterValues[filterMap[colCount]] = b;
+          }
+          if (isProjectionRequired[colCount]) {
+            outputValues[projectionMap[colCount]] = queryTypes[colCount]
+                .getDataBasedOnDataType(ByteBuffer.wrap(b));
+          }
+        } else {
+          input.skipBytes(v);
+        }
+      }
+    }
+    // measure
+    DataType dataType;
+    for (int msrCount = 0; msrCount < measureCount; msrCount++, colCount++) {
+      if (nullBitSet.get(colCount)) {
+        if (isFilterRequired[colCount]) {
+          filterValues[filterMap[colCount]] = null;
+        }
+        if (isProjectionRequired[colCount]) {
+          outputValues[projectionMap[colCount]] = null;
+        }
+      } else {
+        dataType = measureDataTypes[msrCount];
+        if (dataType == DataTypes.BOOLEAN) {
+          if (isRequired[colCount]) {
+            boolean v = input.readBoolean();
+            if (isFilterRequired[colCount]) {
+              filterValues[filterMap[colCount]] = v;
+            }
+            if (isProjectionRequired[colCount]) {
+              outputValues[projectionMap[colCount]] = v;
+            }
+          } else {
+            input.skipBytes(1);
+          }
+        } else if (dataType == DataTypes.SHORT) {
+          if (isRequired[colCount]) {
+            short v = input.readShort();
+            if (isFilterRequired[colCount]) {
+              filterValues[filterMap[colCount]] = v;
+            }
+            if (isProjectionRequired[colCount]) {
+              outputValues[projectionMap[colCount]] = v;
+            }
+          } else {
+            input.skipBytes(2);
+          }
+        } else if (dataType == DataTypes.INT) {
+          if (isRequired[colCount]) {
+            int v = input.readInt();
+            if (isFilterRequired[colCount]) {
+              filterValues[filterMap[colCount]] = v;
+            }
+            if (isProjectionRequired[colCount]) {
+              outputValues[projectionMap[colCount]] = v;
+            }
+          } else {
+            input.skipBytes(4);
+          }
+        } else if (dataType == DataTypes.LONG) {
+          if (isRequired[colCount]) {
+            long v = input.readLong();
+            if (isFilterRequired[colCount]) {
+              filterValues[filterMap[colCount]] = v;
+            }
+            if (isProjectionRequired[colCount]) {
+              outputValues[projectionMap[colCount]] = v;
+            }
+          } else {
+            input.skipBytes(8);
+          }
+        } else if (dataType == DataTypes.DOUBLE) {
+          if (isRequired[colCount]) {
+            double v = input.readDouble();
+            if (isFilterRequired[colCount]) {
+              filterValues[filterMap[colCount]] = v;
+            }
+            if (isProjectionRequired[colCount]) {
+              outputValues[projectionMap[colCount]] = v;
+            }
+          } else {
+            input.skipBytes(8);
+          }
+        } else if (DataTypes.isDecimal(dataType)) {
+          int len = input.readShort();
+          if (isRequired[colCount]) {
+            BigDecimal v = DataTypeUtil.byteToBigDecimal(input.readBytes(len));
+            if (isFilterRequired[colCount]) {
+              filterValues[filterMap[colCount]] = v;
+            }
+            if (isProjectionRequired[colCount]) {
+              outputValues[projectionMap[colCount]] =
+                  DataTypeUtil.getDataTypeConverter().convertFromBigDecimalToDecimal(v);
+            }
+          } else {
+            input.skipBytes(len);
+          }
+        }
+      }
+    }
+  }
+
+  private void readRawRowFromStream() {
+    input.nextRow();
+    short nullLen = input.readShort();
+    BitSet nullBitSet = allNonNull;
+    if (nullLen > 0) {
+      nullBitSet = BitSet.valueOf(input.readBytes(nullLen));
+    }
+    int colCount = 0;
+    // primitive type dimension
+    for (; colCount < isNoDictColumn.length; colCount++) {
+      if (nullBitSet.get(colCount)) {
+        outputValues[colCount] = CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY;
+      } else {
+        if (isNoDictColumn[colCount]) {
+          int v = input.readShort();
+          outputValues[colCount] = input.readBytes(v);
+        } else {
+          outputValues[colCount] = input.readInt();
+        }
+      }
+    }
+    // complex type dimension
+    for (; colCount < dimensionCount; colCount++) {
+      if (nullBitSet.get(colCount)) {
+        outputValues[colCount] = null;
+      } else {
+        short v = input.readShort();
+        outputValues[colCount] = input.readBytes(v);
+      }
+    }
+    // measure
+    DataType dataType;
+    for (int msrCount = 0; msrCount < measureCount; msrCount++, colCount++) {
+      if (nullBitSet.get(colCount)) {
+        outputValues[colCount] = null;
+      } else {
+        dataType = measureDataTypes[msrCount];
+        if (dataType == DataTypes.BOOLEAN) {
+          outputValues[colCount] = input.readBoolean();
+        } else if (dataType == DataTypes.SHORT) {
+          outputValues[colCount] = input.readShort();
+        } else if (dataType == DataTypes.INT) {
+          outputValues[colCount] = input.readInt();
+        } else if (dataType == DataTypes.LONG) {
+          outputValues[colCount] = input.readLong();
+        } else if (dataType == DataTypes.DOUBLE) {
+          outputValues[colCount] = input.readDouble();
+        } else if (DataTypes.isDecimal(dataType)) {
+          int len = input.readShort();
+          outputValues[colCount] = DataTypeUtil.byteToBigDecimal(input.readBytes(len));
+        }
+      }
+    }
+  }
+
+  private void putRowToColumnBatch(int rowId) {
+    Class<?>[] paramTypes = {int.class, Object.class, int.class};
+    Method putRowToColumnBatch = null;
+    try {
+      putRowToColumnBatch = vectorProxy.getClass().getMethod("putRowToColumnBatch", paramTypes);
+
+    } catch (Exception e) {
+      LOGGER.error(
+              "Unable to put the row in the vector" + "rowid: " + rowId + e);
+    }
+    for (int i = 0; i < projection.length; i++) {
+      Object value = outputValues[i];
+      try {
+        putRowToColumnBatch.invoke(vectorProxy, rowId, value, i);
+      } catch (Exception e) {
+        LOGGER.error(
+                "Unable to put the row in the vector" + "rowid: " + rowId + e);
+      }
+    }
+  }
+
+  @Override public float getProgress() throws IOException, InterruptedException {
+    return 0;
+  }
+
+  @Override public void close() throws IOException {
+    if (null != input) {
+      input.close();
+    }
+    if (null != vectorProxy) {
+      try {
+        Method closeMethod = vectorProxy.getClass().getMethod("close");
+        closeMethod.invoke(vectorProxy);
+      } catch (Exception e) {
+        LOGGER.error(
+                "Unable to close the stream vector reader" + e);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bcef656d/integration/spark2/src/main/spark2.1/org/apache/spark/sql/CarbonVectorProxy.java
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/spark2.1/org/apache/spark/sql/CarbonVectorProxy.java b/integration/spark2/src/main/spark2.1/org/apache/spark/sql/CarbonVectorProxy.java
new file mode 100644
index 0000000..7fa01e9
--- /dev/null
+++ b/integration/spark2/src/main/spark2.1/org/apache/spark/sql/CarbonVectorProxy.java
@@ -0,0 +1,226 @@
+/*
+ * 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.spark.sql;
+
+import java.math.BigInteger;
+
+import org.apache.spark.memory.MemoryMode;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.execution.vectorized.ColumnarBatch;
+import org.apache.spark.sql.types.CalendarIntervalType;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.Decimal;
+import org.apache.spark.sql.types.DecimalType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.CalendarInterval;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/**
+ * Adapter class which handles the columnar vector reading of the carbondata
+ * based on the spark ColumnVector and ColumnarBatch API. This proxy class
+ * handles the complexity of spark 2.1 version related api changes since
+ * spark ColumnVector and ColumnarBatch interfaces are still evolving.
+ */
+public class CarbonVectorProxy {
+
+    private ColumnarBatch columnarBatch;
+
+    /**
+     * Adapter class which handles the columnar vector reading of the carbondata
+     * based on the spark ColumnVector and ColumnarBatch API. This proxy class
+     * handles the complexity of spark 2.3 version related api changes since
+     * spark ColumnVector and ColumnarBatch interfaces are still evolving.
+     *
+     * @param memMode       which represent the type onheap or offheap vector.
+     * @param rowNum        rows number for vector reading
+     * @param structFileds, metadata related to current schema of table.
+     */
+    public CarbonVectorProxy(MemoryMode memMode, int rowNum, StructField[] structFileds) {
+        columnarBatch = ColumnarBatch.allocate(new StructType(structFileds), memMode, rowNum);
+    }
+
+    public CarbonVectorProxy(MemoryMode memMode, StructType outputSchema, int rowNum) {
+        columnarBatch = ColumnarBatch.allocate(outputSchema, memMode, rowNum);
+    }
+
+    /**
+     * Sets the number of rows in this batch.
+     */
+    public void setNumRows(int numRows) {
+        columnarBatch.setNumRows(numRows);
+    }
+
+    /**
+     * Returns the number of rows for read, including filtered rows.
+     */
+    public int numRows() {
+        return columnarBatch.capacity();
+    }
+
+    /**
+     * Called to close all the columns in this batch. It is not valid to access the data after
+     * calling this. This must be called at the end to clean up memory allocations.
+     */
+    public void close() {
+        columnarBatch.close();
+    }
+
+    /**
+     * Returns the row in this batch at `rowId`. Returned row is reused across calls.
+     */
+    public InternalRow getRow(int rowId) {
+        return columnarBatch.getRow(rowId);
+    }
+
+    /**
+     * Returns the row in this batch at `rowId`. Returned row is reused across calls.
+     */
+    public Object getColumnarBatch() {
+        return columnarBatch;
+    }
+
+    public void resetDictionaryIds(int ordinal) {
+        columnarBatch.column(ordinal).getDictionaryIds().reset();
+    }
+
+    /**
+     * Resets this column for writing. The currently stored values are no longer accessible.
+     */
+    public void reset() {
+        columnarBatch.reset();
+    }
+
+    public void putRowToColumnBatch(int rowId, Object value, int offset) {
+        org.apache.spark.sql.types.DataType t = dataType(offset);
+        if (null == value) {
+            putNull(rowId, offset);
+        } else {
+            if (t == org.apache.spark.sql.types.DataTypes.BooleanType) {
+                putBoolean(rowId, (boolean) value, offset);
+            } else if (t == org.apache.spark.sql.types.DataTypes.ByteType) {
+                putByte(rowId, (byte) value, offset);
+            } else if (t == org.apache.spark.sql.types.DataTypes.ShortType) {
+                putShort(rowId, (short) value, offset);
+            } else if (t == org.apache.spark.sql.types.DataTypes.IntegerType) {
+                putInt(rowId, (int) value, offset);
+            } else if (t == org.apache.spark.sql.types.DataTypes.LongType) {
+                putLong(rowId, (long) value, offset);
+            } else if (t == org.apache.spark.sql.types.DataTypes.FloatType) {
+                putFloat(rowId, (float) value, offset);
+            } else if (t == org.apache.spark.sql.types.DataTypes.DoubleType) {
+                putDouble(rowId, (double) value, offset);
+            } else if (t == org.apache.spark.sql.types.DataTypes.StringType) {
+                UTF8String v = (UTF8String) value;
+                putByteArray(rowId, v.getBytes(), offset);
+            } else if (t instanceof org.apache.spark.sql.types.DecimalType) {
+                DecimalType dt = (DecimalType) t;
+                Decimal d = Decimal.fromDecimal(value);
+                if (dt.precision() <= Decimal.MAX_INT_DIGITS()) {
+                    putInt(rowId, (int) d.toUnscaledLong(), offset);
+                } else if (dt.precision() <= Decimal.MAX_LONG_DIGITS()) {
+                    putLong(rowId, d.toUnscaledLong(), offset);
+                } else {
+                    final BigInteger integer = d.toJavaBigDecimal().unscaledValue();
+                    byte[] bytes = integer.toByteArray();
+                    putByteArray(rowId, bytes, 0, bytes.length, offset);
+                }
+            } else if (t instanceof CalendarIntervalType) {
+                CalendarInterval c = (CalendarInterval) value;
+                columnarBatch.column(offset).getChildColumn(0).putInt(rowId, c.months);
+                columnarBatch.column(offset).getChildColumn(1).putLong(rowId, c.microseconds);
+            } else if (t instanceof org.apache.spark.sql.types.DateType) {
+                putInt(rowId, (int) value, offset);
+            } else if (t instanceof org.apache.spark.sql.types.TimestampType) {
+                putLong(rowId, (long) value, offset);
+            }
+        }
+    }
+
+    public void putBoolean(int rowId, boolean value, int ordinal) {
+        columnarBatch.column(ordinal).putBoolean(rowId, (boolean) value);
+    }
+
+    public void putByte(int rowId, byte value, int ordinal) {
+        columnarBatch.column(ordinal).putByte(rowId, (byte) value);
+    }
+
+    public void putShort(int rowId, short value, int ordinal) {
+        columnarBatch.column(ordinal).putShort(rowId, (short) value);
+    }
+
+    public void putInt(int rowId, int value, int ordinal) {
+        columnarBatch.column(ordinal).putInt(rowId, (int) value);
+    }
+
+    public void putFloat(int rowId, float value, int ordinal) {
+        columnarBatch.column(ordinal).putFloat(rowId, (float) value);
+    }
+
+    public void putLong(int rowId, long value, int ordinal) {
+        columnarBatch.column(ordinal).putLong(rowId, (long) value);
+    }
+
+    public void putDouble(int rowId, double value, int ordinal) {
+        columnarBatch.column(ordinal).putDouble(rowId, (double) value);
+    }
+
+    public void putByteArray(int rowId, byte[] value, int ordinal) {
+        columnarBatch.column(ordinal).putByteArray(rowId, (byte[]) value);
+    }
+
+    public void putInts(int rowId, int count, int value, int ordinal) {
+        columnarBatch.column(ordinal).putInts(rowId, count, value);
+    }
+
+    public void putShorts(int rowId, int count, short value, int ordinal) {
+        columnarBatch.column(ordinal).putShorts(rowId, count, value);
+    }
+
+    public void putLongs(int rowId, int count, long value, int ordinal) {
+        columnarBatch.column(ordinal).putLongs(rowId, count, value);
+    }
+
+    public void putDecimal(int rowId, Decimal value, int precision, int ordinal) {
+        columnarBatch.column(ordinal).putDecimal(rowId, value, precision);
+
+    }
+
+    public void putDoubles(int rowId, int count, double value, int ordinal) {
+        columnarBatch.column(ordinal).putDoubles(rowId, count, value);
+    }
+
+    public void putByteArray(int rowId, byte[] value, int offset, int length, int ordinal) {
+        columnarBatch.column(ordinal).putByteArray(rowId, (byte[]) value, offset, length);
+    }
+
+    public void putNull(int rowId, int ordinal) {
+        columnarBatch.column(ordinal).putNull(rowId);
+    }
+
+    public void putNulls(int rowId, int count, int ordinal) {
+        columnarBatch.column(ordinal).putNulls(rowId, count);
+    }
+
+    public boolean isNullAt(int rowId, int ordinal) {
+        return columnarBatch.column(ordinal).isNullAt(rowId);
+    }
+
+    public DataType dataType(int ordinal) {
+        return columnarBatch.column(ordinal).dataType();
+    }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bcef656d/integration/spark2/src/main/spark2.2/org/apache/spark/sql/CarbonVectorProxy.java
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/spark2.2/org/apache/spark/sql/CarbonVectorProxy.java b/integration/spark2/src/main/spark2.2/org/apache/spark/sql/CarbonVectorProxy.java
new file mode 100644
index 0000000..944b32e
--- /dev/null
+++ b/integration/spark2/src/main/spark2.2/org/apache/spark/sql/CarbonVectorProxy.java
@@ -0,0 +1,229 @@
+/*
+ * 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.spark.sql;
+
+import java.math.BigInteger;
+
+import org.apache.spark.memory.MemoryMode;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.execution.vectorized.ColumnarBatch;
+import org.apache.spark.sql.types.CalendarIntervalType;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.Decimal;
+import org.apache.spark.sql.types.DecimalType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.CalendarInterval;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/**
+ * Adapter class which handles the columnar vector reading of the carbondata
+ * based on the spark ColumnVector and ColumnarBatch API. This proxy class
+ * handles the complexity of spark 2.2 version related api changes since
+ * spark ColumnVector and ColumnarBatch interfaces are still evolving.
+ */
+public class CarbonVectorProxy {
+
+    private ColumnarBatch columnarBatch;
+
+    /**
+     * Adapter class which handles the columnar vector reading of the carbondata
+     * based on the spark ColumnVector and ColumnarBatch API. This proxy class
+     * handles the complexity of spark 2.3 version related api changes since
+     * spark ColumnVector and ColumnarBatch interfaces are still evolving.
+     *
+     * @param memMode       which represent the type onheap or offheap vector.
+     * @param rowNum        rows number for vector reading
+     * @param structFileds, metadata related to current schema of table.
+     */
+    public CarbonVectorProxy(MemoryMode memMode, int rowNum, StructField[] structFileds) {
+        columnarBatch = ColumnarBatch.allocate(new StructType(structFileds), memMode, rowNum);
+    }
+
+    public CarbonVectorProxy(MemoryMode memMode, StructType outputSchema, int rowNum) {
+        columnarBatch = ColumnarBatch.allocate(outputSchema, memMode, rowNum);
+    }
+
+    /**
+     * Sets the number of rows in this batch.
+     */
+    public void setNumRows(int numRows) {
+        columnarBatch.setNumRows(numRows);
+    }
+
+    /**
+     * Returns the number of rows for read, including filtered rows.
+     */
+    public int numRows() {
+        return columnarBatch.capacity();
+    }
+
+    /**
+     * Called to close all the columns in this batch. It is not valid to access the data after
+     * calling this. This must be called at the end to clean up memory allocations.
+     */
+    public void close() {
+        columnarBatch.close();
+    }
+
+    /**
+     * Returns the row in this batch at `rowId`. Returned row is reused across calls.
+     */
+    public InternalRow getRow(int rowId) {
+        return columnarBatch.getRow(rowId);
+    }
+
+    /**
+     * Returns the row in this batch at `rowId`. Returned row is reused across calls.
+     */
+    public Object getColumnarBatch() {
+        return columnarBatch;
+    }
+
+    public Object reserveDictionaryIds(int capacity , int dummyOrdinal) {
+        return columnarBatch.column(ordinal).reserveDictionaryIds(capacity);
+    }
+
+    public void resetDictionaryIds(int ordinal) {
+        columnarBatch.column(ordinal).getDictionaryIds().reset();
+    }
+
+    /**
+     * Resets this column for writing. The currently stored values are no longer accessible.
+     */
+    public void reset() {
+        columnarBatch.reset();
+    }
+
+    public void putRowToColumnBatch(int rowId, Object value, int offset) {
+        org.apache.spark.sql.types.DataType t = dataType(offset);
+        if (null == value) {
+            putNull(rowId, offset);
+        } else {
+            if (t == org.apache.spark.sql.types.DataTypes.BooleanType) {
+                putBoolean(rowId, (boolean) value, offset);
+            } else if (t == org.apache.spark.sql.types.DataTypes.ByteType) {
+                putByte(rowId, (byte) value, offset);
+            } else if (t == org.apache.spark.sql.types.DataTypes.ShortType) {
+                putShort(rowId, (short) value, offset);
+            } else if (t == org.apache.spark.sql.types.DataTypes.IntegerType) {
+                putInt(rowId, (int) value, offset);
+            } else if (t == org.apache.spark.sql.types.DataTypes.LongType) {
+                putLong(rowId, (long) value, offset);
+            } else if (t == org.apache.spark.sql.types.DataTypes.FloatType) {
+                putFloat(rowId, (float) value, offset);
+            } else if (t == org.apache.spark.sql.types.DataTypes.DoubleType) {
+                putDouble(rowId, (double) value, offset);
+            } else if (t == org.apache.spark.sql.types.DataTypes.StringType) {
+                UTF8String v = (UTF8String) value;
+                putByteArray(rowId, v.getBytes(), offset);
+            } else if (t instanceof org.apache.spark.sql.types.DecimalType) {
+                DecimalType dt = (DecimalType) t;
+                Decimal d = Decimal.fromDecimal(value);
+                if (dt.precision() <= Decimal.MAX_INT_DIGITS()) {
+                    putInt(rowId, (int) d.toUnscaledLong(), offset);
+                } else if (dt.precision() <= Decimal.MAX_LONG_DIGITS()) {
+                    putLong(rowId, d.toUnscaledLong(), offset);
+                } else {
+                    final BigInteger integer = d.toJavaBigDecimal().unscaledValue();
+                    byte[] bytes = integer.toByteArray();
+                    putByteArray(rowId, bytes, 0, bytes.length, offset);
+                }
+            } else if (t instanceof CalendarIntervalType) {
+                CalendarInterval c = (CalendarInterval) value;
+                columnarBatch.column(offset).getChildColumn(0).putInt(rowId, c.months);
+                columnarBatch.column(offset).getChildColumn(1).putLong(rowId, c.microseconds);
+            } else if (t instanceof org.apache.spark.sql.types.DateType) {
+                putInt(rowId, (int) value, offset);
+            } else if (t instanceof org.apache.spark.sql.types.TimestampType) {
+                putLong(rowId, (long) value, offset);
+            }
+        }
+    }
+
+    public void putBoolean(int rowId, boolean value, int ordinal) {
+        columnarBatch.column(ordinal).putBoolean(rowId, (boolean) value);
+    }
+
+    public void putByte(int rowId, byte value, int ordinal) {
+        columnarBatch.column(ordinal).putByte(rowId, (byte) value);
+    }
+
+    public void putShort(int rowId, short value, int ordinal) {
+        columnarBatch.column(ordinal).putShort(rowId, (short) value);
+    }
+
+    public void putInt(int rowId, int value, int ordinal) {
+        columnarBatch.column(ordinal).putInt(rowId, (int) value);
+    }
+
+    public void putFloat(int rowId, float value, int ordinal) {
+        columnarBatch.column(ordinal).putFloat(rowId, (float) value);
+    }
+
+    public void putLong(int rowId, long value, int ordinal) {
+        columnarBatch.column(ordinal).putLong(rowId, (long) value);
+    }
+
+    public void putDouble(int rowId, double value, int ordinal) {
+        columnarBatch.column(ordinal).putDouble(rowId, (double) value);
+    }
+
+    public void putByteArray(int rowId, byte[] value, int ordinal) {
+        columnarBatch.column(ordinal).putByteArray(rowId, (byte[]) value);
+    }
+
+    public void putInts(int rowId, int count, int value, int ordinal) {
+        columnarBatch.column(ordinal).putInts(rowId, count, value);
+    }
+
+    public void putShorts(int rowId, int count, short value, int ordinal) {
+        columnarBatch.column(ordinal).putShorts(rowId, count, value);
+    }
+
+    public void putLongs(int rowId, int count, long value, int ordinal) {
+        columnarBatch.column(ordinal).putLongs(rowId, count, value);
+    }
+
+    public void putDecimal(int rowId, Decimal value, int precision, int ordinal) {
+        columnarBatch.column(ordinal).putDecimal(rowId, value, precision);
+    }
+
+    public void putDoubles(int rowId, int count, double value, int ordinal) {
+        columnarBatch.column(ordinal).putDoubles(rowId, count, value);
+    }
+
+    public void putByteArray(int rowId, byte[] value, int offset, int length, int ordinal) {
+        columnarBatch.column(ordinal).putByteArray(rowId, (byte[]) value, offset, length);
+    }
+
+    public void putNull(int rowId, int ordinal) {
+        columnarBatch.column(ordinal).putNull(rowId);
+    }
+
+    public void putNulls(int rowId, int count, int ordinal) {
+        columnarBatch.column(ordinal).putNulls(rowId, count);
+    }
+
+    public boolean isNullAt(int rowId, int ordinal) {
+        return columnarBatch.column(ordinal).isNullAt(rowId);
+    }
+
+    public DataType dataType(int ordinal) {
+        return columnarBatch.column(ordinal).dataType();
+    }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bcef656d/integration/spark2/src/main/spark2.3/org/apache/spark/sql/CarbonVectorProxy.java
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/spark2.3/org/apache/spark/sql/CarbonVectorProxy.java b/integration/spark2/src/main/spark2.3/org/apache/spark/sql/CarbonVectorProxy.java
new file mode 100644
index 0000000..783a528
--- /dev/null
+++ b/integration/spark2/src/main/spark2.3/org/apache/spark/sql/CarbonVectorProxy.java
@@ -0,0 +1,255 @@
+/*
+ * 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.spark.sql;
+
+import java.math.BigInteger;
+
+import org.apache.spark.memory.MemoryMode;
+import org.apache.spark.sql.ColumnVectorFactory;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
+import org.apache.spark.sql.types.*;
+import org.apache.spark.sql.vectorized.ColumnVector;
+import org.apache.spark.sql.vectorized.ColumnarBatch;
+import org.apache.spark.unsafe.types.CalendarInterval;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/**
+ * Adapter class which handles the columnar vector reading of the carbondata
+ * based on the spark ColumnVector and ColumnarBatch API. This proxy class
+ * handles the complexity of spark 2.3 version related api changes since
+ * spark ColumnVector and ColumnarBatch interfaces are still evolving.
+ */
+public class CarbonVectorProxy {
+
+    private ColumnarBatch columnarBatch;
+    private WritableColumnVector[] writableColumnVectors;
+
+    /**
+     * Adapter class which handles the columnar vector reading of the carbondata
+     * based on the spark ColumnVector and ColumnarBatch API. This proxy class
+     * handles the complexity of spark 2.3 version related api changes since
+     * spark ColumnVector and ColumnarBatch interfaces are still evolving.
+     *
+     * @param memMode       which represent the type onheap or offheap vector.
+     * @param rowNum        rows number for vector reading
+     * @param structFileds, metadata related to current schema of table.
+     */
+    public CarbonVectorProxy(MemoryMode memMode, int rowNum, StructField[] structFileds) {
+        writableColumnVectors = ColumnVectorFactory
+                .getColumnVector(memMode, new StructType(structFileds), rowNum);
+        columnarBatch = new ColumnarBatch(writableColumnVectors);
+        columnarBatch.setNumRows(rowNum);
+    }
+
+    public CarbonVectorProxy(MemoryMode memMode, StructType outputSchema, int rowNum) {
+        writableColumnVectors = ColumnVectorFactory
+                .getColumnVector(memMode, outputSchema, rowNum);
+        columnarBatch = new ColumnarBatch(writableColumnVectors);
+        columnarBatch.setNumRows(rowNum);
+    }
+
+    /**
+     * Returns the number of rows for read, including filtered rows.
+     */
+    public int numRows() {
+        return columnarBatch.numRows();
+    }
+
+    public Object reserveDictionaryIds(int capacity, int ordinal) {
+        return writableColumnVectors[ordinal].reserveDictionaryIds(capacity);
+    }
+
+    /**
+     * This API will return a columnvector from a batch of column vector rows
+     * based on the ordinal
+     *
+     * @param ordinal
+     * @return
+     */
+    public ColumnVector column(int ordinal) {
+        return columnarBatch.column(ordinal);
+    }
+
+    /**
+     * Resets this column for writing. The currently stored values are no longer accessible.
+     */
+    public void reset() {
+        for (WritableColumnVector col : writableColumnVectors) {
+            col.reset();
+        }
+    }
+
+    public void resetDictionaryIds(int ordinal) {
+        writableColumnVectors[ordinal].getDictionaryIds().reset();
+    }
+
+    /**
+     * Returns the row in this batch at `rowId`. Returned row is reused across calls.
+     */
+    public InternalRow getRow(int rowId) {
+        return columnarBatch.getRow(rowId);
+    }
+
+
+    /**
+     * Returns the row in this batch at `rowId`. Returned row is reused across calls.
+     */
+    public Object getColumnarBatch() {
+        return columnarBatch;
+    }
+
+    /**
+     * Called to close all the columns in this batch. It is not valid to access the data after
+     * calling this. This must be called at the end to clean up memory allocations.
+     */
+    public void close() {
+        columnarBatch.close();
+    }
+
+    /**
+     * Sets the number of rows in this batch.
+     */
+    public void setNumRows(int numRows) {
+        columnarBatch.setNumRows(numRows);
+    }
+
+    /**
+     * This method will add the row to the corresponding column vector object
+     *
+     * @param rowId
+     * @param value
+     */
+    public void putRowToColumnBatch(int rowId, Object value, int offset) {
+            org.apache.spark.sql.types.DataType t = dataType(offset);
+            if (null == value) {
+                putNull(rowId, offset);
+            } else {
+                if (t == org.apache.spark.sql.types.DataTypes.BooleanType) {
+                    putBoolean(rowId, (boolean) value, offset);
+                } else if (t == org.apache.spark.sql.types.DataTypes.ByteType) {
+                    putByte(rowId, (byte) value, offset);
+                } else if (t == org.apache.spark.sql.types.DataTypes.ShortType) {
+                    putShort(rowId, (short) value, offset);
+                } else if (t == org.apache.spark.sql.types.DataTypes.IntegerType) {
+                    putInt(rowId, (int) value, offset);
+                } else if (t == org.apache.spark.sql.types.DataTypes.LongType) {
+                    putLong(rowId, (long) value, offset);
+                } else if (t == org.apache.spark.sql.types.DataTypes.FloatType) {
+                    putFloat(rowId, (float) value, offset);
+                } else if (t == org.apache.spark.sql.types.DataTypes.DoubleType) {
+                    putDouble(rowId, (double) value, offset);
+                } else if (t == org.apache.spark.sql.types.DataTypes.StringType) {
+                    UTF8String v = (UTF8String) value;
+                    putByteArray(rowId, v.getBytes(), offset);
+                } else if (t instanceof DecimalType) {
+                    DecimalType dt = (DecimalType) t;
+                    Decimal d = Decimal.fromDecimal(value);
+                    if (dt.precision() <= Decimal.MAX_INT_DIGITS()) {
+                        putInt(rowId, (int) d.toUnscaledLong(), offset);
+                    } else if (dt.precision() <= Decimal.MAX_LONG_DIGITS()) {
+                        putLong(rowId, d.toUnscaledLong(), offset);
+                    } else {
+                        final BigInteger integer = d.toJavaBigDecimal().unscaledValue();
+                        byte[] bytes = integer.toByteArray();
+                        putByteArray(rowId, bytes, 0, bytes.length, offset);
+                    }
+                } else if (t instanceof CalendarIntervalType) {
+                    CalendarInterval c = (CalendarInterval) value;
+                    writableColumnVectors[offset].getChild(0).putInt(rowId, c.months);
+                    writableColumnVectors[offset].getChild(1).putLong(rowId, c.microseconds);
+                } else if (t instanceof org.apache.spark.sql.types.DateType) {
+                    putInt(rowId, (int) value, offset);
+                } else if (t instanceof org.apache.spark.sql.types.TimestampType) {
+                    putLong(rowId, (long) value, offset);
+                }
+            }
+    }
+
+    public void putBoolean(int rowId, boolean value, int ordinal) {
+        writableColumnVectors[ordinal].putBoolean(rowId, (boolean) value);
+    }
+
+    public void putByte(int rowId, byte value, int ordinal) {
+        writableColumnVectors[ordinal].putByte(rowId, (byte) value);
+    }
+
+    public void putShort(int rowId, short value, int ordinal) {
+        writableColumnVectors[ordinal].putShort(rowId, (short) value);
+    }
+
+    public void putInt(int rowId, int value, int ordinal) {
+        writableColumnVectors[ordinal].putInt(rowId, (int) value);
+    }
+
+    public void putFloat(int rowId, float value, int ordinal) {
+        writableColumnVectors[ordinal].putFloat(rowId, (float) value);
+    }
+
+    public void putLong(int rowId, long value, int ordinal) {
+        writableColumnVectors[ordinal].putLong(rowId, (long) value);
+    }
+
+    public void putDouble(int rowId, double value, int ordinal) {
+        writableColumnVectors[ordinal].putDouble(rowId, (double) value);
+    }
+
+    public void putByteArray(int rowId, byte[] value, int ordinal) {
+        writableColumnVectors[ordinal].putByteArray(rowId, (byte[]) value);
+    }
+
+    public void putInts(int rowId, int count, int value, int ordinal) {
+        writableColumnVectors[ordinal].putInts(rowId, count, value);
+    }
+
+    public void putShorts(int rowId, int count, short value, int ordinal) {
+        writableColumnVectors[ordinal].putShorts(rowId, count, value);
+    }
+
+    public void putLongs(int rowId, int count, long value, int ordinal) {
+        writableColumnVectors[ordinal].putLongs(rowId, count, value);
+    }
+
+    public void putDecimal(int rowId, Decimal value, int precision, int ordinal) {
+        writableColumnVectors[ordinal].putDecimal(rowId, value, precision);
+
+    }
+
+    public void putDoubles(int rowId, int count, double value, int ordinal) {
+        writableColumnVectors[ordinal].putDoubles(rowId, count, value);
+    }
+
+    public void putByteArray(int rowId, byte[] value, int offset, int length, int ordinal) {
+        writableColumnVectors[ordinal].putByteArray(rowId, (byte[]) value, offset, length);
+    }
+
+    public void putNull(int rowId, int ordinal) {
+        writableColumnVectors[ordinal].putNull(rowId);
+    }
+
+    public void putNulls(int rowId, int count, int ordinal) {
+        writableColumnVectors[ordinal].putNulls(rowId, count);
+    }
+
+    public boolean isNullAt(int rowId, int ordinal) {
+        return writableColumnVectors[ordinal].isNullAt(rowId);
+    }
+
+    public DataType dataType(int ordinal) {
+        return writableColumnVectors[ordinal].dataType();
+    }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bcef656d/integration/spark2/src/main/spark2.3/org/apache/spark/sql/ColumnVectorFactory.java
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/spark2.3/org/apache/spark/sql/ColumnVectorFactory.java b/integration/spark2/src/main/spark2.3/org/apache/spark/sql/ColumnVectorFactory.java
new file mode 100644
index 0000000..6fe5ede
--- /dev/null
+++ b/integration/spark2/src/main/spark2.3/org/apache/spark/sql/ColumnVectorFactory.java
@@ -0,0 +1,45 @@
+/*
+ * 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.spark.sql;
+
+import org.apache.spark.memory.MemoryMode;
+import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
+import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
+import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.sql.vectorized.ColumnarBatch;
+
+public class ColumnVectorFactory {
+
+
+    public static WritableColumnVector[] getColumnVector(MemoryMode memMode, StructType outputSchema, int rowNums) {
+
+
+        WritableColumnVector[] writableColumnVectors = null;
+        switch (memMode) {
+            case ON_HEAP:
+                writableColumnVectors = OnHeapColumnVector
+                        .allocateColumns(rowNums, outputSchema);
+                break;
+            case OFF_HEAP:
+                writableColumnVectors = OffHeapColumnVector
+                        .allocateColumns(rowNums, outputSchema);
+                break;
+        }
+        return writableColumnVectors;
+    }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bcef656d/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamInputFormat.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamInputFormat.java b/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamInputFormat.java
index 644ac4c..c0e270c 100644
--- a/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamInputFormat.java
+++ b/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamInputFormat.java
@@ -18,6 +18,7 @@
 package org.apache.carbondata.streaming;
 
 import java.io.IOException;
+import java.lang.reflect.Constructor;
 
 import org.apache.carbondata.core.cache.Cache;
 import org.apache.carbondata.core.cache.dictionary.Dictionary;
@@ -33,7 +34,10 @@ import org.apache.carbondata.core.scan.complextypes.ArrayQueryType;
 import org.apache.carbondata.core.scan.complextypes.PrimitiveQueryType;
 import org.apache.carbondata.core.scan.complextypes.StructQueryType;
 import org.apache.carbondata.core.scan.filter.GenericQueryType;
+import org.apache.carbondata.core.scan.model.QueryModel;
 import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.hadoop.InputMetricsStats;
+import org.apache.carbondata.streaming.CarbonStreamUtils;
 
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.RecordReader;
@@ -47,10 +51,46 @@ public class CarbonStreamInputFormat extends FileInputFormat<Void, Object> {
 
   public static final String READ_BUFFER_SIZE = "carbon.stream.read.buffer.size";
   public static final String READ_BUFFER_SIZE_DEFAULT = "65536";
+  public static final String STREAM_RECORD_READER_INSTANCE =
+      "org.apache.carbondata.stream.CarbonStreamRecordReader";
+  // return raw row for handoff
+  private boolean useRawRow = false;
 
-  @Override public RecordReader<Void, Object> createRecordReader(InputSplit split,
-      TaskAttemptContext context) throws IOException, InterruptedException {
-    return new CarbonStreamRecordReader();
+  public void setUseRawRow(boolean useRawRow) {
+    this.useRawRow = useRawRow;
+  }
+
+  public void setInputMetricsStats(InputMetricsStats inputMetricsStats) {
+    this.inputMetricsStats = inputMetricsStats;
+  }
+
+  public void setVectorReader(boolean vectorReader) {
+    isVectorReader = vectorReader;
+  }
+
+  public void setModel(QueryModel model) {
+    this.model = model;
+  }
+
+  // InputMetricsStats
+  private InputMetricsStats inputMetricsStats;
+  // vector reader
+  private boolean isVectorReader;
+  private QueryModel model;
+
+  @Override
+  public RecordReader<Void, Object> createRecordReader(InputSplit split, TaskAttemptContext context)
+      throws IOException, InterruptedException {
+    try {
+      Constructor cons = CarbonStreamUtils
+          .getConstructorWithReflection(STREAM_RECORD_READER_INSTANCE, boolean.class,
+              InputMetricsStats.class, QueryModel.class, boolean.class);
+      return (RecordReader) CarbonStreamUtils
+          .getInstanceWithReflection(cons, isVectorReader, inputMetricsStats, model, useRawRow);
+
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
   }
 
   public static GenericQueryType[] getComplexDimensions(CarbonTable carbontable,


[7/8] carbondata git commit: [CARBONDATA-2532][Integration] Carbon to support spark 2.3.1 version(Make API changes in carbon to be compatible with spark 2.3)

Posted by ra...@apache.org.
[CARBONDATA-2532][Integration] Carbon to support spark 2.3.1 version(Make API changes in carbon to be compatible with spark 2.3)

Fixed spark-carbon integration API compatibility issues
f) Corrected the testcases based on spark 2.3.0 behaviour change
g) Excluded following dependency from pom.xml files net.jpountzlz4 as spark 2.3.0 changed
it to org.lz4, so removed from the test class path of spark2,spark-common-test,spark2-examples

This closes #2642


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

Branch: refs/heads/master
Commit: 74c3eb10b860845dacd3542ab72595617dc0df57
Parents: bcef656
Author: sandeep-katta <sa...@gmail.com>
Authored: Thu Jul 5 21:31:29 2018 -0700
Committer: ravipesala <ra...@gmail.com>
Committed: Wed Sep 5 18:09:16 2018 +0530

----------------------------------------------------------------------
 datamap/mv/core/pom.xml                         |   5 +
 .../apache/carbondata/mv/datamap/MVHelper.scala |  44 ++-
 datamap/mv/plan/pom.xml                         |   5 +
 .../mv/plans/modular/ModularPlan.scala          |   3 +-
 .../mv/plans/modular/ModularRelation.scala      |   5 +-
 .../mv/plans/util/BirdcageOptimizer.scala       |  13 +-
 .../util/LogicalPlanSignatureGenerator.scala    |   5 +-
 .../mv/testutil/ModularPlanTest.scala           |  36 +--
 .../mv/plans/ExtractJoinConditionsSuite.scala   |   4 +-
 .../mv/plans/LogicalToModularPlanSuite.scala    |  18 +-
 examples/spark2/pom.xml                         |  20 ++
 integration/spark-common-test/pom.xml           |  32 +++
 .../testsuite/bigdecimal/TestBigDecimal.scala   |  10 +-
 .../filterexpr/AllDataTypesTestCaseFilter.scala |   7 +-
 .../StandardPartitionGlobalSortTestCase.scala   |  22 +-
 .../sql/commands/StoredAsCarbondataSuite.scala  |   3 +-
 .../carbondata/spark/rdd/CarbonScanRDD.scala    |   2 +-
 .../carbondata/spark/rdd/StreamHandoffRDD.scala |  11 +-
 .../streaming/CarbonAppendableStreamSink.scala  |   2 +-
 .../apache/spark/sql/test/util/PlanTest.scala   |   5 -
 .../apache/spark/sql/util/SparkSQLUtil.scala    |  72 ++++-
 .../spark/util/CarbonReflectionUtils.scala      | 121 +++++---
 .../scala/org/apache/spark/util/SparkUtil.scala |  42 ---
 .../org/apache/spark/util/SparkUtilTest.scala   |  58 ++++
 integration/spark-datasource/pom.xml            | 132 +++++++++
 .../vectorreader/ColumnarVectorWrapper.java     |  88 +++---
 .../VectorizedCarbonRecordReader.java           |  20 +-
 .../scala/org/apache/spark/util/SparkUtil.scala |  60 ++++
 .../org/apache/spark/sql/CarbonVectorProxy.java | 272 ++++++++++++++++++
 .../org/apache/spark/sql/CarbonVectorProxy.java | 276 ++++++++++++++++++
 .../apache/spark/sql/ColumnVectorFactory.java   |  45 +++
 ...tCreateTableUsingSparkCarbonFileFormat.scala |  26 +-
 integration/spark2/pom.xml                      |  80 ++++++
 .../apache/spark/sql/hive/CarbonAnalyzer.scala  |  51 ++++
 .../sql/hive/CarbonInMemorySessionState.scala   | 278 +++++++++++++++++++
 .../spark/sql/hive/CarbonOptimizerUtil.scala    |  44 +++
 .../spark/sql/hive/CarbonSessionState.scala     | 269 ++++++++++++++++++
 .../spark/sql/hive/CarbonSessionUtil.scala      |  96 +++++++
 .../apache/spark/sql/hive/CarbonSqlConf.scala   | 148 ++++++++++
 ...CreateCarbonSourceTableAsSelectCommand.scala | 130 +++++++++
 .../spark/sql/hive/SqlAstBuilderHelper.scala    | 110 ++++++++
 .../stream/CarbonStreamRecordReader.java        | 155 ++++-------
 .../org/apache/spark/CarbonInputMetrics.scala   |   1 -
 .../spark/sql/CarbonCatalystOperators.scala     |  57 ++--
 .../org/apache/spark/sql/CarbonCountStar.scala  |  10 +-
 .../org/apache/spark/sql/CarbonSession.scala    |   7 +-
 .../sql/CustomDeterministicExpression.scala     |   2 -
 .../management/CarbonLoadDataCommand.scala      |  19 +-
 .../strategy/CarbonLateDecodeStrategy.scala     |  82 ++++--
 .../sql/execution/strategy/DDLStrategy.scala    |  17 +-
 .../spark/sql/hive/CarbonAnalysisRules.scala    |  17 +-
 .../spark/sql/hive/CarbonFileMetastore.scala    |  26 +-
 .../sql/hive/CarbonPreAggregateRules.scala      | 147 +++++++---
 .../spark/sql/optimizer/CarbonFilters.scala     |  26 +-
 .../sql/optimizer/CarbonLateDecodeRule.scala    |  16 +-
 .../sql/parser/CarbonSpark2SqlParser.scala      |   4 +-
 .../spark/sql/CarbonToSparkAdapater.scala       |  75 +++++
 .../org/apache/spark/sql/CarbonVectorProxy.java | 226 ---------------
 ...CreateCarbonSourceTableAsSelectCommand.scala |   2 -
 .../spark/sql/CarbonToSparkAdapater.scala       |  82 ++++++
 .../org/apache/spark/sql/CarbonVectorProxy.java | 229 ---------------
 .../apache/spark/sql/hive/CarbonAnalyzer.scala  |  51 ----
 .../sql/hive/CarbonInMemorySessionState.scala   | 278 -------------------
 .../apache/spark/sql/hive/CarbonOptimizer.scala |  44 +--
 .../spark/sql/hive/CarbonSessionState.scala     | 277 ------------------
 .../spark/sql/hive/CarbonSessionUtil.scala      |  97 -------
 .../spark/sql/hive/CarbonSqlAstBuilder.scala    |  85 +-----
 .../apache/spark/sql/hive/CarbonSqlConf.scala   | 148 ----------
 ...CreateCarbonSourceTableAsSelectCommand.scala | 122 --------
 .../spark/sql/CarbonToSparkAdapater.scala       |  84 ++++++
 .../org/apache/spark/sql/CarbonVectorProxy.java | 255 -----------------
 .../apache/spark/sql/ColumnVectorFactory.java   |  45 ---
 .../apache/spark/sql/hive/CarbonOptimizer.scala |  37 +++
 .../spark/sql/hive/CarbonSqlAstBuilder.scala    |  52 ++++
 .../stream/CarbonStreamRecordReaderTest.java    | 100 +++++++
 .../BooleanDataTypesFilterTest.scala            |   2 +-
 .../booleantype/BooleanDataTypesLoadTest.scala  |   4 +-
 .../bucketing/TableBucketingTestCase.scala      |  26 +-
 .../carbondata/query/SubQueryTestSuite.scala    |   1 -
 pom.xml                                         |  60 ++++
 .../scala/org/apache/spark/rpc/Master.scala     |   2 +-
 .../scala/org/apache/spark/rpc/RpcUtil.scala    |  56 ++++
 .../scala/org/apache/spark/rpc/Worker.scala     |   4 +-
 .../streaming/CarbonStreamInputFormat.java      |   2 +-
 .../streaming/CarbonStreamInputFormatTest.java  |  99 -------
 85 files changed, 3368 insertions(+), 2433 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/datamap/mv/core/pom.xml
----------------------------------------------------------------------
diff --git a/datamap/mv/core/pom.xml b/datamap/mv/core/pom.xml
index 77b5cc7..f5e5715 100644
--- a/datamap/mv/core/pom.xml
+++ b/datamap/mv/core/pom.xml
@@ -50,6 +50,11 @@
       <artifactId>scalatest_${scala.binary.version}</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-core_${scala.binary.version}</artifactId>
+      <version>${spark.version}</version>
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVHelper.scala
----------------------------------------------------------------------
diff --git a/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVHelper.scala b/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVHelper.scala
index f677826..9da109a 100644
--- a/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVHelper.scala
+++ b/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVHelper.scala
@@ -22,7 +22,7 @@ import scala.collection.JavaConverters._
 import scala.collection.mutable
 import scala.collection.mutable.ArrayBuffer
 
-import org.apache.spark.sql.{CarbonEnv, SparkSession}
+import org.apache.spark.sql.{CarbonEnv, CarbonToSparkAdapater, SparkSession}
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.catalog.CatalogTable
 import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, Cast, Expression, NamedExpression, ScalaUDF, SortOrder}
@@ -300,10 +300,13 @@ object MVHelper {
         case Alias(agg: AggregateExpression, name) =>
           agg.aggregateFunction.collect {
             case attr: AttributeReference =>
-              AttributeReference(attr.name, attr.dataType, attr.nullable, attr
-                .metadata)(attr.exprId,
+              CarbonToSparkAdapater.createAttributeReference(attr.name,
+                attr.dataType,
+                attr.nullable,
+                attr.metadata,
+                attr.exprId,
                 aliasName,
-                attr.isGenerated)
+                attr)
           }.head
         case Alias(child, name) =>
           child
@@ -314,18 +317,25 @@ object MVHelper {
     expressions.map {
         case alias@Alias(agg: AggregateExpression, name) =>
           attrMap.get(AttributeKey(agg)).map { exp =>
-            Alias(getAttribute(exp), name)(alias.exprId,
+            CarbonToSparkAdapater.createAliasRef(
+              getAttribute(exp),
+              name,
+              alias.exprId,
               alias.qualifier,
               alias.explicitMetadata,
-              alias.isGenerated)
+              Some(alias))
           }.getOrElse(alias)
 
         case attr: AttributeReference =>
           val uattr = attrMap.get(AttributeKey(attr)).map{a =>
             if (keepAlias) {
-              AttributeReference(a.name, a.dataType, a.nullable, a.metadata)(a.exprId,
+              CarbonToSparkAdapater.createAttributeReference(a.name,
+                a.dataType,
+                a.nullable,
+                a.metadata,
+                a.exprId,
                 attr.qualifier,
-                a.isGenerated)
+                a)
             } else {
               a
             }
@@ -333,10 +343,9 @@ object MVHelper {
           uattr
         case alias@Alias(expression: Expression, name) =>
           attrMap.get(AttributeKey(expression)).map { exp =>
-            Alias(getAttribute(exp), name)(alias.exprId,
-              alias.qualifier,
-              alias.explicitMetadata,
-              alias.isGenerated)
+            CarbonToSparkAdapater
+              .createAliasRef(getAttribute(exp), name, alias.exprId, alias.qualifier,
+                alias.explicitMetadata, Some(alias))
           }.getOrElse(alias)
         case expression: Expression =>
           val uattr = attrMap.get(AttributeKey(expression))
@@ -376,9 +385,14 @@ object MVHelper {
         case attr: AttributeReference =>
           val uattr = attrMap.get(AttributeKey(attr)).map{a =>
             if (keepAlias) {
-              AttributeReference(a.name, a.dataType, a.nullable, a.metadata)(a.exprId,
-                attr.qualifier,
-                a.isGenerated)
+              CarbonToSparkAdapater
+                .createAttributeReference(a.name,
+                  a.dataType,
+                  a.nullable,
+                  a.metadata,
+                  a.exprId,
+                  attr.qualifier,
+                  a)
             } else {
               a
             }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/datamap/mv/plan/pom.xml
----------------------------------------------------------------------
diff --git a/datamap/mv/plan/pom.xml b/datamap/mv/plan/pom.xml
index fcf0e51..ff6976d 100644
--- a/datamap/mv/plan/pom.xml
+++ b/datamap/mv/plan/pom.xml
@@ -44,6 +44,11 @@
       <artifactId>scalatest_${scala.binary.version}</artifactId>
       <scope>provided</scope>
     </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-core_${scala.binary.version}</artifactId>
+      <version>${spark.version}</version>
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/datamap/mv/plan/src/main/scala/org/apache/carbondata/mv/plans/modular/ModularPlan.scala
----------------------------------------------------------------------
diff --git a/datamap/mv/plan/src/main/scala/org/apache/carbondata/mv/plans/modular/ModularPlan.scala b/datamap/mv/plan/src/main/scala/org/apache/carbondata/mv/plans/modular/ModularPlan.scala
index 1420522..6c82598 100644
--- a/datamap/mv/plan/src/main/scala/org/apache/carbondata/mv/plans/modular/ModularPlan.scala
+++ b/datamap/mv/plan/src/main/scala/org/apache/carbondata/mv/plans/modular/ModularPlan.scala
@@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.plans.{JoinType, QueryPlan}
 import org.apache.spark.sql.catalyst.plans.logical.Statistics
 import org.apache.spark.sql.catalyst.trees.TreeNode
 import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.util.SparkSQLUtil
 
 import org.apache.carbondata.mv.plans._
 import org.apache.carbondata.mv.plans.util.{Printers, Signature, SQLBuilder}
@@ -64,7 +65,7 @@ abstract class ModularPlan
     //    spark.conf.set("spark.sql.cbo.enabled", true)
     val sqlStmt = asOneLineSQL
     val plan = spark.sql(sqlStmt).queryExecution.optimizedPlan
-    plan.stats(conf)
+    SparkSQLUtil.invokeStatsMethod(plan, conf)
   }
 
   override def fastEquals(other: TreeNode[_]): Boolean = {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/datamap/mv/plan/src/main/scala/org/apache/carbondata/mv/plans/modular/ModularRelation.scala
----------------------------------------------------------------------
diff --git a/datamap/mv/plan/src/main/scala/org/apache/carbondata/mv/plans/modular/ModularRelation.scala b/datamap/mv/plan/src/main/scala/org/apache/carbondata/mv/plans/modular/ModularRelation.scala
index 3349835..7e1eb05 100644
--- a/datamap/mv/plan/src/main/scala/org/apache/carbondata/mv/plans/modular/ModularRelation.scala
+++ b/datamap/mv/plan/src/main/scala/org/apache/carbondata/mv/plans/modular/ModularRelation.scala
@@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.plans.JoinType
 import org.apache.spark.sql.catalyst.plans.logical
 import org.apache.spark.sql.catalyst.plans.logical.Statistics
 import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.util.SparkSQLUtil
 
 import org.apache.carbondata.mv.plans.modular.Flags._
 
@@ -45,7 +46,7 @@ case class ModularRelation(databaseName: String,
     rest: Seq[Seq[Any]]) extends LeafNode {
   override def computeStats(spark: SparkSession, conf: SQLConf): Statistics = {
     val plan = spark.table(s"${ databaseName }.${ tableName }").queryExecution.optimizedPlan
-    val stats = plan.stats(conf)
+    val stats = SparkSQLUtil.invokeStatsMethod(plan, conf)
     val output = outputList.map(_.toAttribute)
     val mapSeq = plan.collect { case n: logical.LeafNode => n }.map {
       table => AttributeMap(table.output.zip(output))
@@ -136,7 +137,7 @@ case class HarmonizedRelation(source: ModularPlan) extends LeafNode {
   // (spark, conf)
   override def computeStats(spark: SparkSession, conf: SQLConf): Statistics = {
     val plan = spark.table(s"${ databaseName }.${ tableName }").queryExecution.optimizedPlan
-    val stats = plan.stats(conf)
+    val stats = SparkSQLUtil.invokeStatsMethod(plan, conf)
     val output = source.asInstanceOf[GroupBy].child.children(0).asInstanceOf[ModularRelation]
       .outputList.map(_.toAttribute)
     val mapSeq = plan.collect { case n: logical.LeafNode => n }.map {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/datamap/mv/plan/src/main/scala/org/apache/carbondata/mv/plans/util/BirdcageOptimizer.scala
----------------------------------------------------------------------
diff --git a/datamap/mv/plan/src/main/scala/org/apache/carbondata/mv/plans/util/BirdcageOptimizer.scala b/datamap/mv/plan/src/main/scala/org/apache/carbondata/mv/plans/util/BirdcageOptimizer.scala
index 6363089..e1e891a 100644
--- a/datamap/mv/plan/src/main/scala/org/apache/carbondata/mv/plans/util/BirdcageOptimizer.scala
+++ b/datamap/mv/plan/src/main/scala/org/apache/carbondata/mv/plans/util/BirdcageOptimizer.scala
@@ -17,12 +17,14 @@
 
 package org.apache.carbondata.mv.plans.util
 
+import org.apache.spark.sql.CarbonToSparkAdapater
 import org.apache.spark.sql.catalyst.analysis._
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.optimizer._
 import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, _}
 import org.apache.spark.sql.catalyst.rules.{RuleExecutor, _}
 import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.util.SparkSQLUtil
 
 object BirdcageOptimizer extends RuleExecutor[LogicalPlan] {
 
@@ -74,8 +76,8 @@ object BirdcageOptimizer extends RuleExecutor[LogicalPlan] {
       "Operator Optimizations", fixedPoint, Seq(
         // Operator push down
         PushProjectionThroughUnion,
-        ReorderJoin(conf),
-        EliminateOuterJoin(conf),
+        SparkSQLUtil.getReorderJoinObj(conf),
+        SparkSQLUtil.getEliminateOuterJoinObj(conf),
         PushPredicateThroughJoin,
         PushDownPredicate,
         //      LimitPushDown(conf),
@@ -89,7 +91,7 @@ object BirdcageOptimizer extends RuleExecutor[LogicalPlan] {
         CombineLimits,
         CombineUnions,
         // Constant folding and strength reduction
-        NullPropagation(conf),
+        SparkSQLUtil.getNullPropagationObj(conf),
         FoldablePropagation,
         //      OptimizeIn(conf),
         ConstantFolding,
@@ -113,7 +115,7 @@ object BirdcageOptimizer extends RuleExecutor[LogicalPlan] {
                                             extendedOperatorOptimizationRules: _*) ::
     Batch(
       "Check Cartesian Products", Once,
-      CheckCartesianProducts(conf)) ::
+      SparkSQLUtil.getCheckCartesianProductsObj(conf)) ::
     //    Batch("Join Reorder", Once,
     //      CostBasedJoinReorder(conf)) ::
     //    Batch("Decimal Optimizations", fixedPoint,
@@ -126,8 +128,7 @@ object BirdcageOptimizer extends RuleExecutor[LogicalPlan] {
     //      ConvertToLocalRelation,
     //      PropagateEmptyRelation) ::
     Batch(
-      "OptimizeCodegen", Once,
-      OptimizeCodegen(conf)) ::
+      "OptimizeCodegen", Once, CarbonToSparkAdapater.getOptimizeCodegenRule(conf): _*) ::
     Batch(
       "RewriteSubquery", Once,
       RewritePredicateSubquery,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/datamap/mv/plan/src/main/scala/org/apache/carbondata/mv/plans/util/LogicalPlanSignatureGenerator.scala
----------------------------------------------------------------------
diff --git a/datamap/mv/plan/src/main/scala/org/apache/carbondata/mv/plans/util/LogicalPlanSignatureGenerator.scala b/datamap/mv/plan/src/main/scala/org/apache/carbondata/mv/plans/util/LogicalPlanSignatureGenerator.scala
index 2aff5c0..eaa8b04 100644
--- a/datamap/mv/plan/src/main/scala/org/apache/carbondata/mv/plans/util/LogicalPlanSignatureGenerator.scala
+++ b/datamap/mv/plan/src/main/scala/org/apache/carbondata/mv/plans/util/LogicalPlanSignatureGenerator.scala
@@ -30,7 +30,8 @@ object CheckSPJG {
       case a: Aggregate =>
         a.child.collect {
           case Join(_, _, _, _) | Project(_, _) | Filter(_, _) |
-               HiveTableRelation(_, _, _) | LogicalRelation(_, _, _) | LocalRelation(_, _) => true
+               HiveTableRelation(_, _, _) => true
+          case  l: LogicalRelation => true
           case _ => false
         }.forall(identity)
       case _ => false
@@ -55,7 +56,7 @@ object LogicalPlanRule extends SignatureRule[LogicalPlan] {
   def apply(plan: LogicalPlan, childSignatures: Seq[Option[Signature]]): Option[Signature] = {
 
     plan match {
-      case LogicalRelation(_, _, _) =>
+      case l: LogicalRelation =>
         // TODO: implement this (link to BaseRelation)
         None
       case HiveTableRelation(tableMeta, _, _) =>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/datamap/mv/plan/src/main/scala/org/apache/carbondata/mv/testutil/ModularPlanTest.scala
----------------------------------------------------------------------
diff --git a/datamap/mv/plan/src/main/scala/org/apache/carbondata/mv/testutil/ModularPlanTest.scala b/datamap/mv/plan/src/main/scala/org/apache/carbondata/mv/testutil/ModularPlanTest.scala
index 0c68a1f..9d4735c 100644
--- a/datamap/mv/plan/src/main/scala/org/apache/carbondata/mv/testutil/ModularPlanTest.scala
+++ b/datamap/mv/plan/src/main/scala/org/apache/carbondata/mv/testutil/ModularPlanTest.scala
@@ -17,13 +17,11 @@
 
 package org.apache.carbondata.mv.testutil
 
-import java.io.File
-
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.util._
-import org.apache.spark.sql.test.util.{PlanTest, QueryTest}
+import org.apache.spark.sql.test.util.QueryTest
 
 import org.apache.carbondata.mv.plans.modular
 import org.apache.carbondata.mv.plans.modular.{ModularPlan, OneRowTable, Select}
@@ -56,30 +54,6 @@ abstract class ModularPlanTest extends QueryTest with PredicateHelper {
   }
 
   /**
-   * Normalizes plans:
-   * - Filter the filter conditions that appear in a plan. For instance,
-   * ((expr 1 && expr 2) && expr 3), (expr 1 && expr 2 && expr 3), (expr 3 && (expr 1 && expr 2)
-   *   etc., will all now be equivalent.
-   * - Sample the seed will replaced by 0L.
-   * - Join conditions will be resorted by hashCode.
-   */
-  protected def normalizePlan(plan: LogicalPlan): LogicalPlan = {
-    plan transform {
-      case filter@Filter(condition: Expression, child: LogicalPlan) =>
-        Filter(
-          splitConjunctivePredicates(condition).map(rewriteEqual(_)).sortBy(_.hashCode())
-            .reduce(And), child)
-      case sample: Sample =>
-        sample.copy(seed = 0L)(true)
-      case join@Join(left, right, joinType, condition) if condition.isDefined =>
-        val newCondition =
-          splitConjunctivePredicates(condition.get).map(rewriteEqual(_)).sortBy(_.hashCode())
-            .reduce(And)
-        Join(left, right, joinType, Some(newCondition))
-    }
-  }
-
-  /**
    * Rewrite [[EqualTo]] and [[EqualNullSafe]] operator to keep order. The following cases will be
    * equivalent:
    * 1. (a = b), (b = a);
@@ -177,4 +151,12 @@ abstract class ModularPlanTest extends QueryTest with PredicateHelper {
            """.stripMargin)
     }
   }
+
+  object MatchLocalRelation {
+    def unapply(localRelation: LocalRelation): Option[(Seq[Attribute], Any)] = localRelation match {
+      case l: LocalRelation => Some(l.output, l.data)
+      case _ => None
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/datamap/mv/plan/src/test/scala/org/apache/carbondata/mv/plans/ExtractJoinConditionsSuite.scala
----------------------------------------------------------------------
diff --git a/datamap/mv/plan/src/test/scala/org/apache/carbondata/mv/plans/ExtractJoinConditionsSuite.scala b/datamap/mv/plan/src/test/scala/org/apache/carbondata/mv/plans/ExtractJoinConditionsSuite.scala
index e1a3d9f..34ee2ed 100644
--- a/datamap/mv/plan/src/test/scala/org/apache/carbondata/mv/plans/ExtractJoinConditionsSuite.scala
+++ b/datamap/mv/plan/src/test/scala/org/apache/carbondata/mv/plans/ExtractJoinConditionsSuite.scala
@@ -39,7 +39,7 @@ class ExtractJoinConditionsSuite extends ModularPlanTest {
     val extracted = modularPlan.extractJoinConditions(modularPlan.children(0),modularPlan.children(1))
     
     val correctAnswer = originalQuery match {
-      case logical.Join(logical.Filter(cond1,logical.LocalRelation(tbl1,_)),logical.LocalRelation(tbl2,_),Inner,Some(cond2)) =>
+      case logical.Join(logical.Filter(cond1,MatchLocalRelation(tbl1,_)),MatchLocalRelation(tbl2,_),Inner,Some(cond2)) =>
         Seq(cond2)
     }
     
@@ -58,7 +58,7 @@ class ExtractJoinConditionsSuite extends ModularPlanTest {
       left.join(right,condition = Some("l.c".attr === "r.c".attr)).analyze
       
     val correctAnswer = originalQuery1 match {
-      case logical.Join(logical.Filter(cond1,logical.LocalRelation(tbl1,_)),logical.Filter(cond2,logical.LocalRelation(tbl2,_)),Inner,Some(cond3)) =>
+      case logical.Join(logical.Filter(cond1,MatchLocalRelation(tbl1,_)),logical.Filter(cond2,MatchLocalRelation(tbl2,_)),Inner,Some(cond3)) =>
         Seq(cond3)
     }    
     

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/datamap/mv/plan/src/test/scala/org/apache/carbondata/mv/plans/LogicalToModularPlanSuite.scala
----------------------------------------------------------------------
diff --git a/datamap/mv/plan/src/test/scala/org/apache/carbondata/mv/plans/LogicalToModularPlanSuite.scala b/datamap/mv/plan/src/test/scala/org/apache/carbondata/mv/plans/LogicalToModularPlanSuite.scala
index 082c325..144721c 100644
--- a/datamap/mv/plan/src/test/scala/org/apache/carbondata/mv/plans/LogicalToModularPlanSuite.scala
+++ b/datamap/mv/plan/src/test/scala/org/apache/carbondata/mv/plans/LogicalToModularPlanSuite.scala
@@ -43,7 +43,7 @@ class LogicalToModularPlanSuite extends ModularPlanTest {
         .analyze
     val modularized = originalQuery.modularize 
     val correctAnswer = originalQuery match {
-      case logical.Project(proj,logical.LocalRelation(tbl,_)) =>
+      case logical.Project(proj,MatchLocalRelation(tbl,_)) =>
         ModularRelation(null,null,tbl,NoFlags,Seq.empty).select(proj:_*)(tbl:_*)()(Map.empty)()
     }
     comparePlans(modularized, correctAnswer)
@@ -58,7 +58,7 @@ class LogicalToModularPlanSuite extends ModularPlanTest {
         
     val modularized = originalQuery.modularize
     val correctAnswer = originalQuery match {
-      case logical.Project(proj1,logical.Aggregate(grp,agg,logical.Project(proj2,logical.LocalRelation(tbl,_)))) =>
+      case logical.Project(proj1,logical.Aggregate(grp,agg,logical.Project(proj2,MatchLocalRelation(tbl,_)))) =>
         ModularRelation(null,null,tbl,NoFlags,Seq.empty).select(proj2:_*)(tbl:_*)()(Map.empty)().groupBy(agg:_*)(proj2:_*)(grp:_*).select(proj1:_*)(proj1:_*)()(Map.empty)() 
     }   
     comparePlans(modularized, correctAnswer)
@@ -73,7 +73,7 @@ class LogicalToModularPlanSuite extends ModularPlanTest {
         
     val modularized = originalQuery.modularize
     val correctAnswer = originalQuery match {
-      case logical.Project(proj,logical.Filter(cond,logical.LocalRelation(tbl,_))) =>
+      case logical.Project(proj,logical.Filter(cond,MatchLocalRelation(tbl,_))) =>
         ModularRelation(null,null,tbl,NoFlags,Seq.empty).select(proj:_*)(tbl:_*)(cond)(Map.empty)()  
     }
     comparePlans(modularized, correctAnswer)
@@ -87,7 +87,7 @@ class LogicalToModularPlanSuite extends ModularPlanTest {
       
     val modularized = originalQuery.modularize 
     val correctAnswer = originalQuery match {
-      case logical.Join(logical.Filter(cond1,logical.LocalRelation(tbl1,_)),logical.LocalRelation(tbl2,_),Inner,Some(cond2)) =>
+      case logical.Join(logical.Filter(cond1,MatchLocalRelation(tbl1,_)),MatchLocalRelation(tbl2,_),Inner,Some(cond2)) =>
         Seq(ModularRelation(null,null,tbl1,NoFlags,Seq.empty),ModularRelation(null,null,tbl2,NoFlags,Seq.empty)).select(tbl1++tbl2:_*)(tbl1++tbl2:_*)(Seq(cond1,cond2):_*)(Map.empty)(JoinEdge(0,1,Inner)) 
     } 
     comparePlans(modularized, correctAnswer)
@@ -101,7 +101,7 @@ class LogicalToModularPlanSuite extends ModularPlanTest {
       
     val modularized = originalQuery.modularize 
     val correctAnswer = originalQuery match {
-      case logical.Join(logical.Filter(cond1,logical.LocalRelation(tbl1,_)),logical.LocalRelation(tbl2,_),LeftOuter,Some(cond2)) =>
+      case logical.Join(logical.Filter(cond1,MatchLocalRelation(tbl1,_)),MatchLocalRelation(tbl2,_),LeftOuter,Some(cond2)) =>
         Seq(ModularRelation(null,null,tbl1,NoFlags,Seq.empty),ModularRelation(null,null,tbl2,NoFlags,Seq.empty)).select(tbl1++tbl2:_*)(tbl1++tbl2:_*)(Seq(cond1,cond2):_*)(Map.empty)(JoinEdge(0,1,LeftOuter)) 
     } 
     comparePlans(modularized, correctAnswer)
@@ -115,7 +115,7 @@ class LogicalToModularPlanSuite extends ModularPlanTest {
       
     val modularized = originalQuery.modularize 
     val correctAnswer = originalQuery match {
-      case logical.Join(logical.Filter(cond1,logical.LocalRelation(tbl1,_)),logical.LocalRelation(tbl2,_),RightOuter,Some(cond2)) =>
+      case logical.Join(logical.Filter(cond1,MatchLocalRelation(tbl1,_)),MatchLocalRelation(tbl2,_),RightOuter,Some(cond2)) =>
         Seq(ModularRelation(null,null,tbl1,NoFlags,Seq.empty),ModularRelation(null,null,tbl2,NoFlags,Seq.empty)).select(tbl1++tbl2:_*)(tbl1++tbl2:_*)(Seq(cond1,cond2):_*)(Map.empty)(JoinEdge(0,1,RightOuter))  
     } 
     comparePlans(modularized, correctAnswer)
@@ -129,7 +129,7 @@ class LogicalToModularPlanSuite extends ModularPlanTest {
     
     val modularized = analysis.EliminateSubqueryAliases(originalQuery).modularize
     val correctAnswer = originalQuery match {
-      case logical.Join(logical.Filter(cond1,logical.LocalRelation(tbl1,_)),logical.LocalRelation(tbl2,_),Inner,Some(cond2)) =>
+      case logical.Join(logical.Filter(cond1,MatchLocalRelation(tbl1,_)),MatchLocalRelation(tbl2,_),Inner,Some(cond2)) =>
         Seq(ModularRelation(null,null,tbl1,NoFlags,Seq.empty),ModularRelation(null,null,tbl2,NoFlags,Seq.empty)).select(tbl1++tbl2:_*)(tbl1++tbl2:_*)(Seq(cond1,cond2):_*)(Map.empty)(JoinEdge(0,1,Inner))  
     } 
     comparePlans(modularized, correctAnswer)
@@ -147,7 +147,7 @@ class LogicalToModularPlanSuite extends ModularPlanTest {
         
     val modularized = originalQuery.modularize
     val correctAnswer = originalQuery match {
-      case logical.Join(logical.Filter(cond1,logical.LocalRelation(tbl1,_)),logical.Join(logical.Filter(cond2,logical.LocalRelation(tbl2,_)),logical.LocalRelation(tbl3,_),Inner,Some(cond3)),Inner,Some(cond4)) =>
+      case logical.Join(logical.Filter(cond1,MatchLocalRelation(tbl1,_)),logical.Join(logical.Filter(cond2,MatchLocalRelation(tbl2,_)),MatchLocalRelation(tbl3,_),Inner,Some(cond3)),Inner,Some(cond4)) =>
         Seq(ModularRelation(null,null,tbl1,NoFlags,Seq.empty),ModularRelation(null,null,tbl2,NoFlags,Seq.empty),ModularRelation(null,null,tbl3,NoFlags,Seq.empty)).select(tbl1++tbl2++tbl3:_*)(tbl1++tbl2++tbl3:_*)(Seq(cond1,cond2,cond3,cond4):_*)(Map.empty)(JoinEdge(0,1,Inner),JoinEdge(1,2,Inner))  
     } 
     comparePlans(modularized, correctAnswer)
@@ -165,7 +165,7 @@ class LogicalToModularPlanSuite extends ModularPlanTest {
         
     val modularized = originalQuery.modularize
     val correctAnswer = originalQuery match {
-      case logical.Project(proj0, logical.Filter(cond1, logical.Project(proj1, logical.Aggregate(grp,agg,logical.Join(logical.Filter(cond2,logical.LocalRelation(tbl1,_)),logical.Filter(cond3,logical.LocalRelation(tbl2,_)),Inner,Some(cond4)))))) =>
+      case logical.Project(proj0, logical.Filter(cond1, logical.Project(proj1, logical.Aggregate(grp,agg,logical.Join(logical.Filter(cond2,MatchLocalRelation(tbl1,_)),logical.Filter(cond3,MatchLocalRelation(tbl2,_)),Inner,Some(cond4)))))) =>
         Seq(ModularRelation(null,null,tbl1,NoFlags,Seq.empty),ModularRelation(null,null,tbl2,NoFlags,Seq.empty)).select(tbl1++tbl2:_*)(tbl1++tbl2:_*)(Seq(cond2,cond3,cond4):_*)(Map.empty)(JoinEdge(0,1,Inner)).groupBy(agg:_*)(tbl1++tbl2:_*)(grp:_*).select(proj0:_*)(proj1:_*)(cond1)(Map.empty)()  
     }   
     comparePlans(modularized, correctAnswer)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/examples/spark2/pom.xml
----------------------------------------------------------------------
diff --git a/examples/spark2/pom.xml b/examples/spark2/pom.xml
index 9d1adde..91cb20f 100644
--- a/examples/spark2/pom.xml
+++ b/examples/spark2/pom.xml
@@ -77,6 +77,26 @@
       <artifactId>scalatest_${scala.binary.version}</artifactId>
       <scope>test</scope>
     </dependency>
+    <!-- in spark 2.3 spark catalyst added dependency on spark-core-->
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-core_${scala.binary.version}</artifactId>
+      <version>${spark.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.carbondata</groupId>
+      <artifactId>carbondata-core</artifactId>
+      <version>${project.version}</version>
+      <exclusions>
+        <!-- need to Exclude net.jpountz jar from this project.
+         Spark has changed this jar to org.lz4:lz4-java
+         net.jpountz and org.lz4 has same class Name -->
+        <exclusion>
+          <groupId>net.jpountz.lz4</groupId>
+          <artifactId>lz4</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark-common-test/pom.xml
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/pom.xml b/integration/spark-common-test/pom.xml
index ee87d92..50e5830 100644
--- a/integration/spark-common-test/pom.xml
+++ b/integration/spark-common-test/pom.xml
@@ -127,12 +127,27 @@
       <artifactId>carbondata-lucene</artifactId>
       <version>${project.version}</version>
       <scope>test</scope>
+      <exclusions>
+        <!-- need to Exclude net.jpountz jar from this project.
+         Spark has changed this jar to org.lz4:lz4-java
+         net.jpountz and org.lz4 has same class Name -->
+        <exclusion>
+          <groupId>net.jpountz.lz4</groupId>
+          <artifactId>lz4</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.apache.carbondata</groupId>
       <artifactId>carbondata-bloom</artifactId>
       <version>${project.version}</version>
       <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>net.jpountz.lz4</groupId>
+          <artifactId>lz4</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.apache.carbondata</groupId>
@@ -146,6 +161,23 @@
       <scope>test</scope>
     </dependency>
     <dependency>
+      <!-- spark catalyst added runtime dependency on spark-core,so
+      while executing the testcases spark-core should be present else it
+      will fail to execute -->
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-core_${scala.binary.version}</artifactId>
+      <version>${spark.version}</version>
+      <scope>test</scope>
+      <exclusions>
+        <!-- need to Exclude Avro jar from this project,spark core is using
+        the version 1.7.4 which is not compatible with Carbon -->
+        <exclusion>
+          <groupId>org.apache.avro</groupId>
+          <artifactId>avro</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/bigdecimal/TestBigDecimal.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/bigdecimal/TestBigDecimal.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/bigdecimal/TestBigDecimal.scala
index 0dba467..1f7aafe 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/bigdecimal/TestBigDecimal.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/bigdecimal/TestBigDecimal.scala
@@ -73,7 +73,6 @@ class TestBigDecimal extends QueryTest with BeforeAndAfterAll {
     checkAnswer(sql("select min(salary) from carbonTable"),
       sql("select min(salary) from hiveTable"))
   }
-  
   test("test min datatype on big decimal column") {
     val output = sql("select min(salary) from carbonTable").collectAsList().get(0).get(0)
     assert(output.isInstanceOf[java.math.BigDecimal])
@@ -83,7 +82,6 @@ class TestBigDecimal extends QueryTest with BeforeAndAfterAll {
     val output = sql("select max(salary) from carbonTable").collectAsList().get(0).get(0)
     assert(output.isInstanceOf[java.math.BigDecimal])
   }
-  
   test("test count function on big decimal column") {
     checkAnswer(sql("select count(salary) from carbonTable"),
       sql("select count(salary) from hiveTable"))
@@ -152,8 +150,8 @@ class TestBigDecimal extends QueryTest with BeforeAndAfterAll {
   }
 
   test("test sum*10 aggregation on big decimal column with high precision") {
-    checkAnswer(sql("select sum(salary)*10 from carbonBigDecimal_2"),
-      sql("select sum(salary)*10 from hiveBigDecimal"))
+    checkAnswer(sql("select cast(sum(salary)*10 as double) from carbonBigDecimal_2"),
+      sql("select cast(sum(salary)*10 as double) from hiveBigDecimal"))
   }
 
   test("test sum/10 aggregation on big decimal column with high precision") {
@@ -167,8 +165,8 @@ class TestBigDecimal extends QueryTest with BeforeAndAfterAll {
   }
 
   test("test sum-distinct*10 aggregation on big decimal column with high precision") {
-    checkAnswer(sql("select sum(distinct(salary))*10 from carbonBigDecimal_2"),
-      sql("select sum(distinct(salary))*10 from hiveBigDecimal"))
+    checkAnswer(sql("select cast(sum(distinct(salary))*10 as decimal)from carbonBigDecimal_2"),
+      sql("select cast(sum(distinct(salary))*10 as decimal) from hiveBigDecimal"))
   }
 
   test("test sum-distinct/10 aggregation on big decimal column with high precision") {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/filterexpr/AllDataTypesTestCaseFilter.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/filterexpr/AllDataTypesTestCaseFilter.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/filterexpr/AllDataTypesTestCaseFilter.scala
index 15ac1f4..73786c8 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/filterexpr/AllDataTypesTestCaseFilter.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/filterexpr/AllDataTypesTestCaseFilter.scala
@@ -17,6 +17,7 @@
 
 package org.apache.carbondata.spark.testsuite.filterexpr
 
+import org.apache.spark.sql.execution.strategy.CarbonDataSourceScan
 import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.BeforeAndAfterAll
 
@@ -56,13 +57,15 @@ class AllDataTypesTestCaseFilter extends QueryTest with BeforeAndAfterAll {
   test("verify like query ends with filter push down") {
     val df = sql("select * from alldatatypestableFilter where empname like '%nandh'").queryExecution
       .sparkPlan
-    assert(df.metadata.get("PushedFilters").get.contains("CarbonEndsWith"))
+    assert(df.asInstanceOf[CarbonDataSourceScan].metadata
+      .get("PushedFilters").get.contains("CarbonEndsWith"))
   }
 
   test("verify like query contains with filter push down") {
     val df = sql("select * from alldatatypestableFilter where empname like '%nand%'").queryExecution
       .sparkPlan
-    assert(df.metadata.get("PushedFilters").get.contains("CarbonContainsWith"))
+    assert(df.asInstanceOf[CarbonDataSourceScan].metadata
+      .get("PushedFilters").get.contains("CarbonContainsWith"))
   }
   
   override def afterAll {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionGlobalSortTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionGlobalSortTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionGlobalSortTestCase.scala
index 150bc36..564c5f3 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionGlobalSortTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionGlobalSortTestCase.scala
@@ -21,6 +21,7 @@ import java.util.concurrent.{Callable, ExecutorService, Executors}
 
 import org.apache.spark.sql.Row
 import org.apache.spark.sql.test.util.QueryTest
+import org.apache.spark.util.SparkUtil
 import org.scalatest.BeforeAndAfterAll
 
 import org.apache.carbondata.common.constants.LoggerAction
@@ -859,11 +860,22 @@ class StandardPartitionGlobalSortTestCase extends QueryTest with BeforeAndAfterA
     sql("create table scalarissue_hive(a int,salary double) using parquet partitioned by (salary) ")
     sql("set hive.exec.dynamic.partition.mode=nonstrict")
     sql("insert into scalarissue_hive values(23,21.2)")
-    intercept[Exception] {
-      sql(s"select * from scalarissue_hive where salary = (select max(salary) from scalarissue_hive)").show()
-    }
-    intercept[Exception] {
-      sql(s"select * from scalarissue where salary = (select max(salary) from scalarissue)").show()
+    if (SparkUtil.isSparkVersionEqualTo("2.1") || SparkUtil.isSparkVersionEqualTo("2.2")) {
+      intercept[Exception] {
+        sql(s"select * from scalarissue_hive where salary = (select max(salary) from " +
+            s"scalarissue_hive)")
+          .show()
+      }
+      intercept[Exception] {
+        sql(s"select * from scalarissue where salary = (select max(salary) from scalarissue)")
+          .show()
+      }
+    } else {
+      checkAnswer(sql(s"select * from scalarissue_hive where salary = (select max(salary) from " +
+                      s"scalarissue_hive)"), Seq(Row(23, 21.2)))
+      checkAnswer(sql(s"select * from scalarissue where salary = (select max(salary) from " +
+                      s"scalarissue)"),
+        Seq(Row(23, 21.2)))
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark-common-test/src/test/scala/org/apache/carbondata/sql/commands/StoredAsCarbondataSuite.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/sql/commands/StoredAsCarbondataSuite.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/sql/commands/StoredAsCarbondataSuite.scala
index 2029e93..6400ed1 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/sql/commands/StoredAsCarbondataSuite.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/sql/commands/StoredAsCarbondataSuite.scala
@@ -87,7 +87,8 @@ class StoredAsCarbondataSuite extends QueryTest with BeforeAndAfterEach {
       sql("CREATE TABLE carbon_table(key INT, value STRING) STORED AS  ")
     } catch {
       case e: Exception =>
-        assert(e.getMessage.contains("no viable alternative at input"))
+        assert(e.getMessage.contains("no viable alternative at input") ||
+        e.getMessage.contains("mismatched input '<EOF>' expecting "))
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
index b712109..01c77f0 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
@@ -421,7 +421,7 @@ class CarbonScanRDD[T: ClassTag](
           // create record reader for row format
           DataTypeUtil.setDataTypeConverter(dataTypeConverterClz.newInstance())
           val inputFormat = new CarbonStreamInputFormat
-          inputFormat.setVectorReader(vectorReader)
+          inputFormat.setIsVectorReader(vectorReader)
           inputFormat.setInputMetricsStats(inputMetricsStats)
           model.setStatisticsRecorder(
             CarbonTimeStatisticsFactory.createExecutorRecorder(model.getQueryId))

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/StreamHandoffRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/StreamHandoffRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/StreamHandoffRDD.scala
index 3cbfab2..ab6e320 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/StreamHandoffRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/StreamHandoffRDD.scala
@@ -49,13 +49,8 @@ import org.apache.carbondata.processing.loading.model.CarbonLoadModel
 import org.apache.carbondata.processing.merger.{CompactionResultSortProcessor, CompactionType}
 import org.apache.carbondata.processing.util.CarbonLoaderUtil
 import org.apache.carbondata.spark.{HandoffResult, HandoffResultImpl}
-<<<<<<< 2f537b724f6f03ab40c95f7ecc8ebd38f6500099
 import org.apache.carbondata.spark.util.CommonUtil
-import org.apache.carbondata.streaming.{CarbonStreamInputFormat, CarbonStreamRecordReader}
-=======
-import org.apache.carbondata.spark.util.{CommonUtil, SparkDataTypeConverterImpl}
 import org.apache.carbondata.streaming.CarbonStreamInputFormat
->>>>>>> [CARBONDATA-2532][Integration] Carbon to support spark 2.3 version, ColumnVector Interface
 
 
 /**
@@ -167,11 +162,11 @@ class StreamHandoffRDD[K, V](
     val format = new CarbonTableInputFormat[Array[Object]]()
     val model = format.createQueryModel(inputSplit, attemptContext)
     val inputFormat = new CarbonStreamInputFormat
-    val streamReader = inputFormat.createRecordReader(inputSplit, attemptContext)
-      .asInstanceOf[RecordReader[Void, Any]]
-    inputFormat.setVectorReader(false)
+    inputFormat.setIsVectorReader(false)
     inputFormat.setModel(model)
     inputFormat.setUseRawRow(true)
+    val streamReader = inputFormat.createRecordReader(inputSplit, attemptContext)
+      .asInstanceOf[RecordReader[Void, Any]]
     streamReader.initialize(inputSplit, attemptContext)
     val iteratorList = new util.ArrayList[RawResultIterator](1)
     iteratorList.add(new StreamingRawResultIterator(streamReader))

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
index 196baa6..5762906 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
@@ -121,7 +121,7 @@ class CarbonAppendableStreamSink(
         className = sparkSession.sessionState.conf.streamingFileCommitProtocolClass,
         jobId = batchId.toString,
         outputPath = fileLogPath,
-        isAppend = false)
+        false)
 
       committer match {
         case manifestCommitter: ManifestFileCommitProtocol =>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark-common/src/main/scala/org/apache/spark/sql/test/util/PlanTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/test/util/PlanTest.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/test/util/PlanTest.scala
index 9883607..00156e9 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/test/util/PlanTest.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/test/util/PlanTest.scala
@@ -26,11 +26,6 @@ import org.apache.spark.sql.catalyst.util._
  */
 class PlanTest extends CarbonFunSuite {
 
-  /** Fails the test if the two expressions do not match */
-  protected def compareExpressions(e1: Expression, e2: Expression): Unit = {
-    comparePlans(Filter(e1, OneRowRelation), Filter(e2, OneRowRelation))
-  }
-
   /** Fails the test if the two plans do not match */
   protected def comparePlans(plan1: LogicalPlan, plan2: LogicalPlan) {
     val normalized1 = normalizeExprIds(plan1)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark-common/src/main/scala/org/apache/spark/sql/util/SparkSQLUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/util/SparkSQLUtil.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/util/SparkSQLUtil.scala
index b5fda85..b7d47a0 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/util/SparkSQLUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/util/SparkSQLUtil.scala
@@ -17,9 +17,13 @@
 
 package org.apache.spark.sql.util
 
+import java.lang.reflect.Method
+
 import org.apache.spark.sql.{DataFrame, Dataset, SparkSession}
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.internal.SessionState
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.{SessionState, SQLConf}
+import org.apache.spark.util.{CarbonReflectionUtils, SparkUtil}
 
 object SparkSQLUtil {
   def sessionState(sparkSession: SparkSession): SessionState = sparkSession.sessionState
@@ -31,4 +35,68 @@ object SparkSQLUtil {
   def getSparkSession: SparkSession = {
     SparkSession.getDefaultSession.get
   }
+
+  def invokeStatsMethod(logicalPlanObj: LogicalPlan, conf: SQLConf): Statistics = {
+    if (SparkUtil.isSparkVersionEqualTo("2.2")) {
+      val method: Method = logicalPlanObj.getClass.getMethod("stats", classOf[SQLConf])
+      method.invoke(logicalPlanObj, conf).asInstanceOf[Statistics]
+    } else if (SparkUtil.isSparkVersionXandAbove("2.3")) {
+      val method: Method = logicalPlanObj.getClass.getMethod("stats")
+      method.invoke(logicalPlanObj).asInstanceOf[Statistics]
+    } else {
+      throw new UnsupportedOperationException("Spark version not supported")
+    }
+  }
+
+  def getReorderJoinObj(conf: SQLConf): Rule[LogicalPlan] = {
+    if (SparkUtil.isSparkVersionEqualTo("2.2")) {
+      val className = "org.apache.spark.sql.catalyst.optimizer.ReorderJoin";
+      CarbonReflectionUtils.createObject(className, conf)._1.asInstanceOf[Rule[LogicalPlan]]
+    } else if (SparkUtil.isSparkVersionXandAbove("2.3")) {
+      val className = "org.apache.spark.sql.catalyst.optimizer.ReorderJoin$";
+      CarbonReflectionUtils.createObjectOfPrivateConstructor(className)._1
+        .asInstanceOf[Rule[LogicalPlan]]
+    } else {
+      throw new UnsupportedOperationException("Spark version not supported")
+    }
+  }
+
+  def getEliminateOuterJoinObj(conf: SQLConf): Rule[LogicalPlan] = {
+    if (SparkUtil.isSparkVersionEqualTo("2.2")) {
+      val className = "org.apache.spark.sql.catalyst.optimizer.EliminateOuterJoin";
+      CarbonReflectionUtils.createObject(className, conf)._1.asInstanceOf[Rule[LogicalPlan]]
+    } else if (SparkUtil.isSparkVersionXandAbove("2.3")) {
+      val className = "org.apache.spark.sql.catalyst.optimizer.EliminateOuterJoin$";
+      CarbonReflectionUtils.createObjectOfPrivateConstructor(className)._1
+        .asInstanceOf[Rule[LogicalPlan]]
+    } else {
+      throw new UnsupportedOperationException("Spark version not supported")
+    }
+  }
+
+  def getNullPropagationObj(conf: SQLConf): Rule[LogicalPlan] = {
+    if (SparkUtil.isSparkVersionEqualTo("2.2")) {
+      val className = "org.apache.spark.sql.catalyst.optimizer.NullPropagation";
+      CarbonReflectionUtils.createObject(className, conf)._1.asInstanceOf[Rule[LogicalPlan]]
+    } else if (SparkUtil.isSparkVersionXandAbove("2.3")) {
+      val className = "org.apache.spark.sql.catalyst.optimizer.NullPropagation$";
+      CarbonReflectionUtils.createObjectOfPrivateConstructor(className)._1
+        .asInstanceOf[Rule[LogicalPlan]]
+    } else {
+      throw new UnsupportedOperationException("Spark version not supported")
+    }
+  }
+
+  def getCheckCartesianProductsObj(conf: SQLConf): Rule[LogicalPlan] = {
+    if (SparkUtil.isSparkVersionEqualTo("2.2")) {
+      val className = "org.apache.spark.sql.catalyst.optimizer.CheckCartesianProducts";
+      CarbonReflectionUtils.createObject(className, conf)._1.asInstanceOf[Rule[LogicalPlan]]
+    } else if (SparkUtil.isSparkVersionXandAbove("2.3")) {
+      val className = "org.apache.spark.sql.catalyst.optimizer.CheckCartesianProducts$";
+      CarbonReflectionUtils.createObjectOfPrivateConstructor(className)._1
+        .asInstanceOf[Rule[LogicalPlan]]
+    } else {
+      throw new UnsupportedOperationException("Spark version not supported")
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark-common/src/main/scala/org/apache/spark/util/CarbonReflectionUtils.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/util/CarbonReflectionUtils.scala b/integration/spark-common/src/main/scala/org/apache/spark/util/CarbonReflectionUtils.scala
index 34eeded..32cd201 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/util/CarbonReflectionUtils.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/util/CarbonReflectionUtils.scala
@@ -17,24 +17,25 @@
 
 package org.apache.spark.util
 
-import java.lang.reflect.Field
+import java.lang.reflect.Method
 
 import scala.reflect.runtime._
 import scala.reflect.runtime.universe._
 
-import org.apache.spark.SPARK_VERSION
-import org.apache.spark.SparkContext
-import org.apache.spark.sql.SparkSession
-import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.{SPARK_VERSION, SparkContext}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession}
+import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
 import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
 import org.apache.spark.sql.catalyst.catalog.CatalogTable
 import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression}
 import org.apache.spark.sql.catalyst.parser.AstBuilder
 import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan, SubqueryAlias}
-import org.apache.spark.sql.execution.datasources.LogicalRelation
-import org.apache.spark.sql.sources.BaseRelation
+import org.apache.spark.sql.catalyst.plans.physical.Partitioning
+import org.apache.spark.sql.execution.{RowDataSourceScanExec, SparkPlan}
+import org.apache.spark.sql.execution.datasources.{DataSource, LogicalRelation}
+import org.apache.spark.sql.sources.{BaseRelation, Filter}
 
-import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 
 /**
@@ -61,12 +62,12 @@ object CarbonReflectionUtils {
       tableIdentifier: TableIdentifier,
       tableAlias: Option[String] = None): UnresolvedRelation = {
     val className = "org.apache.spark.sql.catalyst.analysis.UnresolvedRelation"
-    if (SPARK_VERSION.startsWith("2.1")) {
+    if (SparkUtil.isSparkVersionEqualTo("2.1")) {
       createObject(
         className,
         tableIdentifier,
         tableAlias)._1.asInstanceOf[UnresolvedRelation]
-    } else if (SPARK_VERSION.startsWith("2.2")) {
+    } else if (SparkUtil.isSparkVersionXandAbove("2.2")) {
       createObject(
         className,
         tableIdentifier)._1.asInstanceOf[UnresolvedRelation]
@@ -79,13 +80,13 @@ object CarbonReflectionUtils {
       relation: LogicalPlan,
       view: Option[TableIdentifier]): SubqueryAlias = {
     val className = "org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias"
-    if (SPARK_VERSION.startsWith("2.1")) {
+    if (SparkUtil.isSparkVersionEqualTo("2.1")) {
       createObject(
         className,
         alias.getOrElse(""),
         relation,
         Option(view))._1.asInstanceOf[SubqueryAlias]
-    } else if (SPARK_VERSION.startsWith("2.2")) {
+    } else if (SparkUtil.isSparkVersionXandAbove("2.2")) {
       createObject(
         className,
         alias.getOrElse(""),
@@ -101,7 +102,7 @@ object CarbonReflectionUtils {
       overwrite: Boolean,
       ifPartitionNotExists: Boolean): InsertIntoTable = {
     val className = "org.apache.spark.sql.catalyst.plans.logical.InsertIntoTable"
-    if (SPARK_VERSION.startsWith("2.1")) {
+    if (SparkUtil.isSparkVersionEqualTo("2.1")) {
       val overwriteOptions = createObject(
         "org.apache.spark.sql.catalyst.plans.logical.OverwriteOptions",
         overwrite.asInstanceOf[Object], Map.empty.asInstanceOf[Object])._1.asInstanceOf[Object]
@@ -112,7 +113,7 @@ object CarbonReflectionUtils {
         query,
         overwriteOptions,
         ifPartitionNotExists.asInstanceOf[Object])._1.asInstanceOf[InsertIntoTable]
-    } else if (SPARK_VERSION.startsWith("2.2")) {
+    } else if (SparkUtil.isSparkVersionXandAbove("2.2") ) {
       createObject(
         className,
         table,
@@ -127,20 +128,28 @@ object CarbonReflectionUtils {
 
   def getLogicalRelation(relation: BaseRelation,
       expectedOutputAttributes: Seq[Attribute],
-      catalogTable: Option[CatalogTable]): LogicalRelation = {
+      catalogTable: Option[CatalogTable],
+      isStreaming: Boolean): LogicalRelation = {
     val className = "org.apache.spark.sql.execution.datasources.LogicalRelation"
-    if (SPARK_VERSION.startsWith("2.1")) {
+    if (SparkUtil.isSparkVersionEqualTo("2.1")) {
       createObject(
         className,
         relation,
         Some(expectedOutputAttributes),
         catalogTable)._1.asInstanceOf[LogicalRelation]
-    } else if (SPARK_VERSION.startsWith("2.2")) {
+    } else if (SparkUtil.isSparkVersionEqualTo("2.2")) {
       createObject(
         className,
         relation,
         expectedOutputAttributes,
         catalogTable)._1.asInstanceOf[LogicalRelation]
+    } else if (SparkUtil.isSparkVersionEqualTo("2.3")) {
+      createObject(
+        className,
+        relation,
+        expectedOutputAttributes,
+        catalogTable,
+        isStreaming.asInstanceOf[Object])._1.asInstanceOf[LogicalRelation]
     } else {
       throw new UnsupportedOperationException("Unsupported Spark version")
     }
@@ -183,27 +192,23 @@ object CarbonReflectionUtils {
   def getAstBuilder(conf: Object,
       sqlParser: Object,
       sparkSession: SparkSession): AstBuilder = {
-    if (SPARK_VERSION.startsWith("2.1") || SPARK_VERSION.startsWith("2.2")) {
-      val className = sparkSession.sparkContext.conf.get(
-        CarbonCommonConstants.CARBON_SQLASTBUILDER_CLASSNAME,
-        "org.apache.spark.sql.hive.CarbonSqlAstBuilder")
-      createObject(className,
-        conf,
-        sqlParser, sparkSession)._1.asInstanceOf[AstBuilder]
-    } else {
-      throw new UnsupportedOperationException("Spark version not supported")
-    }
+    val className = sparkSession.sparkContext.conf.get(
+      CarbonCommonConstants.CARBON_SQLASTBUILDER_CLASSNAME,
+      "org.apache.spark.sql.hive.CarbonSqlAstBuilder")
+    createObject(className,
+      conf,
+      sqlParser, sparkSession)._1.asInstanceOf[AstBuilder]
   }
 
   def getSessionState(sparkContext: SparkContext,
       carbonSession: Object,
       useHiveMetaStore: Boolean): Any = {
-    if (SPARK_VERSION.startsWith("2.1")) {
+    if (SparkUtil.isSparkVersionEqualTo("2.1")) {
       val className = sparkContext.conf.get(
         CarbonCommonConstants.CARBON_SESSIONSTATE_CLASSNAME,
         "org.apache.spark.sql.hive.CarbonSessionState")
       createObject(className, carbonSession)._1
-    } else if (SPARK_VERSION.startsWith("2.2")) {
+    } else if (SparkUtil.isSparkVersionXandAbove("2.2")) {
       if (useHiveMetaStore) {
         val className = sparkContext.conf.get(
           CarbonCommonConstants.CARBON_SESSIONSTATE_CLASSNAME,
@@ -225,12 +230,12 @@ object CarbonReflectionUtils {
   }
 
   def hasPredicateSubquery(filterExp: Expression) : Boolean = {
-    if (SPARK_VERSION.startsWith("2.1")) {
+    if (SparkUtil.isSparkVersionEqualTo("2.1")) {
       val tuple = Class.forName("org.apache.spark.sql.catalyst.expressions.PredicateSubquery")
       val method = tuple.getMethod("hasPredicateSubquery", classOf[Expression])
       val hasSubquery : Boolean = method.invoke(tuple, filterExp).asInstanceOf[Boolean]
       hasSubquery
-    } else if (SPARK_VERSION.startsWith("2.2")) {
+    } else if (SparkUtil.isSparkVersionXandAbove("2.2")) {
       val tuple = Class.forName("org.apache.spark.sql.catalyst.expressions.SubqueryExpression")
       val method = tuple.getMethod("hasInOrExistsSubquery", classOf[Expression])
       val hasSubquery : Boolean = method.invoke(tuple, filterExp).asInstanceOf[Boolean]
@@ -248,6 +253,54 @@ object CarbonReflectionUtils {
     isFormatted
   }
 
+  def getRowDataSourceScanExecObj(relation: LogicalRelation,
+      output: Seq[Attribute],
+      pushedFilters: Seq[Filter],
+      handledFilters: Seq[Filter],
+      rdd: RDD[InternalRow],
+      partition: Partitioning,
+      metadata: Map[String, String]): RowDataSourceScanExec = {
+    val className = "org.apache.spark.sql.execution.RowDataSourceScanExec"
+    if (SparkUtil.isSparkVersionEqualTo("2.1") || SparkUtil.isSparkVersionEqualTo("2.2")) {
+      createObject(className, output, rdd, relation.relation,
+        partition, metadata,
+        relation.catalogTable.map(_.identifier))._1.asInstanceOf[RowDataSourceScanExec]
+    } else if (SparkUtil.isSparkVersionXandAbove("2.3")) {
+      createObject(className, output, output.map(output.indexOf),
+        pushedFilters.toSet, handledFilters.toSet, rdd,
+        relation.relation,
+        relation.catalogTable.map(_.identifier))._1.asInstanceOf[RowDataSourceScanExec]
+    } else {
+      throw new UnsupportedOperationException("Spark version not supported")
+    }
+  }
+
+  def invokewriteAndReadMethod(dataSourceObj: DataSource,
+      dataFrame: DataFrame,
+      data: LogicalPlan,
+      session: SparkSession,
+      mode: SaveMode,
+      query: LogicalPlan,
+      physicalPlan: SparkPlan): BaseRelation = {
+    if (SparkUtil.isSparkVersionEqualTo("2.2")) {
+      val method: Method = dataSourceObj.getClass
+        .getMethod("writeAndRead", classOf[SaveMode], classOf[DataFrame])
+      method.invoke(dataSourceObj, mode, dataFrame)
+        .asInstanceOf[BaseRelation]
+    } else if (SparkUtil.isSparkVersionEqualTo("2.3")) {
+      val method: Method = dataSourceObj.getClass
+        .getMethod("writeAndRead",
+          classOf[SaveMode],
+          classOf[LogicalPlan],
+          classOf[Seq[Attribute]],
+          classOf[SparkPlan])
+      method.invoke(dataSourceObj, mode, query, query.output, physicalPlan)
+        .asInstanceOf[BaseRelation]
+    } else {
+      throw new UnsupportedOperationException("Spark version not supported")
+    }
+  }
+
   def createObject(className: String, conArgs: Object*): (Any, Class[_]) = {
     val clazz = Utils.classForName(className)
     val ctor = clazz.getConstructors.head
@@ -255,4 +308,10 @@ object CarbonReflectionUtils {
     (ctor.newInstance(conArgs: _*), clazz)
   }
 
+  def createObjectOfPrivateConstructor(className: String, conArgs: Object*): (Any, Class[_]) = {
+    val clazz = Utils.classForName(className)
+    val ctor = clazz.getDeclaredConstructors.head
+    ctor.setAccessible(true)
+    (ctor.newInstance(conArgs: _*), clazz)
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark-common/src/main/scala/org/apache/spark/util/SparkUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/util/SparkUtil.scala b/integration/spark-common/src/main/scala/org/apache/spark/util/SparkUtil.scala
deleted file mode 100644
index 4635fc7..0000000
--- a/integration/spark-common/src/main/scala/org/apache/spark/util/SparkUtil.scala
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.util
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.io.{LongWritable, Text}
-import org.apache.hadoop.mapred.JobConf
-import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat, FileSplit}
-import org.apache.spark.{SparkContext, TaskContext}
-import org.apache.spark.deploy.SparkHadoopUtil
-import org.apache.spark.rdd.{NewHadoopPartition, NewHadoopRDD}
-
-import org.apache.carbondata.processing.loading.csvinput.BlockDetails
-
-/*
- * this object use to handle file splits
- */
-object SparkUtil {
-
-  def setTaskContext(context: TaskContext): Unit = {
-    val localThreadContext = TaskContext.get()
-    if (localThreadContext == null) {
-      TaskContext.setTaskContext(context)
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark-common/src/test/scala/org/apache/spark/util/SparkUtilTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/test/scala/org/apache/spark/util/SparkUtilTest.scala b/integration/spark-common/src/test/scala/org/apache/spark/util/SparkUtilTest.scala
new file mode 100644
index 0000000..4810db1
--- /dev/null
+++ b/integration/spark-common/src/test/scala/org/apache/spark/util/SparkUtilTest.scala
@@ -0,0 +1,58 @@
+/*
+ * 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.spark.util
+
+import org.apache.spark.SPARK_VERSION
+import org.scalatest.FunSuite
+
+class SparkUtilTest extends FunSuite{
+
+  test("Test Spark Version API with X and Above") {
+    if (SPARK_VERSION.startsWith("2.1")) {
+      assert(SparkUtil.isSparkVersionXandAbove("2.1"))
+      assert(!SparkUtil.isSparkVersionXandAbove("2.2"))
+      assert(!SparkUtil.isSparkVersionXandAbove("2.3"))
+    } else if (SPARK_VERSION.startsWith("2.2")) {
+      assert(SparkUtil.isSparkVersionXandAbove("2.1"))
+      assert(SparkUtil.isSparkVersionXandAbove("2.2"))
+      assert(!SparkUtil.isSparkVersionXandAbove("2.3"))
+    } else {
+      assert(SparkUtil.isSparkVersionXandAbove("2.1"))
+      assert(SparkUtil.isSparkVersionXandAbove("2.2"))
+      assert(SparkUtil.isSparkVersionXandAbove("2.3"))
+      assert(!SparkUtil.isSparkVersionXandAbove("2.4"))
+    }
+  }
+
+  test("Test Spark Version API Equal to X") {
+    if (SPARK_VERSION.startsWith("2.1")) {
+      assert(SparkUtil.isSparkVersionEqualTo("2.1"))
+      assert(!SparkUtil.isSparkVersionEqualTo("2.2"))
+      assert(!SparkUtil.isSparkVersionEqualTo("2.3"))
+    } else if (SPARK_VERSION.startsWith("2.2")) {
+      assert(!SparkUtil.isSparkVersionEqualTo("2.1"))
+      assert(SparkUtil.isSparkVersionEqualTo("2.2"))
+      assert(!SparkUtil.isSparkVersionEqualTo("2.3"))
+    } else {
+      assert(!SparkUtil.isSparkVersionEqualTo("2.1"))
+      assert(!SparkUtil.isSparkVersionEqualTo("2.2"))
+      assert(SparkUtil.isSparkVersionEqualTo("2.3"))
+      assert(!SparkUtil.isSparkVersionEqualTo("2.4"))
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark-datasource/pom.xml
----------------------------------------------------------------------
diff --git a/integration/spark-datasource/pom.xml b/integration/spark-datasource/pom.xml
index 38cf629..3d017c8 100644
--- a/integration/spark-datasource/pom.xml
+++ b/integration/spark-datasource/pom.xml
@@ -39,6 +39,15 @@
       <groupId>org.apache.carbondata</groupId>
       <artifactId>carbondata-hadoop</artifactId>
       <version>${project.version}</version>
+      <exclusions>
+        <!-- need to Exclude net.jpountz jar from this project.
+         Spark has changed this jar to org.lz4:lz4-java
+         net.jpountz and org.lz4 has same class Name -->
+        <exclusion>
+          <groupId>net.jpountz.lz4</groupId>
+          <artifactId>lz4</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.apache.carbondata</groupId>
@@ -192,5 +201,128 @@
         <maven.test.skip>true</maven.test.skip>
       </properties>
     </profile>
+    <profile>
+      <id>spark-2.1</id>
+      <properties>
+        <spark.version>2.1.0</spark.version>
+        <scala.binary.version>2.11</scala.binary.version>
+        <scala.version>2.11.8</scala.version>
+      </properties>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-compiler-plugin</artifactId>
+            <configuration>
+              <excludes>
+                <exclude>src/main/spark2.3plus</exclude>
+              </excludes>
+            </configuration>
+          </plugin>
+          <plugin>
+            <groupId>org.codehaus.mojo</groupId>
+            <artifactId>build-helper-maven-plugin</artifactId>
+            <version>3.0.0</version>
+            <executions>
+              <execution>
+                <id>add-source</id>
+                <phase>generate-sources</phase>
+                <goals>
+                  <goal>add-source</goal>
+                </goals>
+                <configuration>
+                  <sources>
+                    <source>src/main/spark2.1andspark2.2</source>
+                  </sources>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+    <profile>
+      <id>spark-2.2</id>
+      <activation>
+        <activeByDefault>true</activeByDefault>
+      </activation>
+      <properties>
+        <spark.version>2.2.1</spark.version>
+        <scala.binary.version>2.11</scala.binary.version>
+        <scala.version>2.11.8</scala.version>
+      </properties>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-compiler-plugin</artifactId>
+            <configuration>
+              <excludes>
+                <exclude>src/main/spark2.3plus</exclude>
+              </excludes>
+            </configuration>
+          </plugin>
+          <plugin>
+            <groupId>org.codehaus.mojo</groupId>
+            <artifactId>build-helper-maven-plugin</artifactId>
+            <version>3.0.0</version>
+            <executions>
+              <execution>
+                <id>add-source</id>
+                <phase>generate-sources</phase>
+                <goals>
+                  <goal>add-source</goal>
+                </goals>
+                <configuration>
+                  <sources>
+                    <source>src/main/spark2.1andspark2.2</source>
+                  </sources>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+    <profile>
+      <id>spark-2.3</id>
+      <properties>
+        <spark.version>2.3.1</spark.version>
+        <scala.binary.version>2.11</scala.binary.version>
+        <scala.version>2.11.8</scala.version>
+      </properties>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-compiler-plugin</artifactId>
+            <configuration>
+              <excludes>
+                <exclude>src/main/spark2.1andspark2.2</exclude>
+              </excludes>
+            </configuration>
+          </plugin>
+          <plugin>
+            <groupId>org.codehaus.mojo</groupId>
+            <artifactId>build-helper-maven-plugin</artifactId>
+            <version>3.0.0</version>
+            <executions>
+              <execution>
+                <id>add-source</id>
+                <phase>generate-sources</phase>
+                <goals>
+                  <goal>add-source</goal>
+                </goals>
+                <configuration>
+                  <sources>
+                    <source>src/main/spark2.3plus</source>
+                  </sources>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
   </profiles>
 </project>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/ColumnarVectorWrapper.java
----------------------------------------------------------------------
diff --git a/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/ColumnarVectorWrapper.java b/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/ColumnarVectorWrapper.java
index 7bab117..bc5a387 100644
--- a/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/ColumnarVectorWrapper.java
+++ b/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/ColumnarVectorWrapper.java
@@ -30,7 +30,7 @@ import org.apache.spark.sql.types.Decimal;
 
 class ColumnarVectorWrapper implements CarbonColumnVector {
 
-  private CarbonVectorProxy writableColumnVector;
+  private CarbonVectorProxy sparkColumnVectorProxy;
 
   private boolean[] filteredRows;
 
@@ -38,32 +38,36 @@ class ColumnarVectorWrapper implements CarbonColumnVector {
 
   private int ordinal;
 
+  private boolean isDictionary;
+
   private boolean filteredRowsExist;
 
   private DataType blockDataType;
 
+  private CarbonColumnVector dictionaryVector;
+
   ColumnarVectorWrapper(CarbonVectorProxy writableColumnVector,
-                        boolean[] filteredRows, int ordinal) {
-    this.writableColumnVector = writableColumnVector;
+      boolean[] filteredRows, int ordinal) {
+    this.sparkColumnVectorProxy = writableColumnVector;
     this.filteredRows = filteredRows;
     this.ordinal = ordinal;
   }
 
   @Override public void putBoolean(int rowId, boolean value) {
     if (!filteredRows[rowId]) {
-      writableColumnVector.putBoolean(counter++, value, ordinal);
+      sparkColumnVectorProxy.putBoolean(counter++, value, ordinal);
     }
   }
 
   @Override public void putFloat(int rowId, float value) {
     if (!filteredRows[rowId]) {
-      writableColumnVector.putFloat(counter++, value,ordinal);
+      sparkColumnVectorProxy.putFloat(counter++, value,ordinal);
     }
   }
 
   @Override public void putShort(int rowId, short value) {
     if (!filteredRows[rowId]) {
-      writableColumnVector.putShort(counter++, value, ordinal);
+      sparkColumnVectorProxy.putShort(counter++, value, ordinal);
     }
   }
 
@@ -71,18 +75,22 @@ class ColumnarVectorWrapper implements CarbonColumnVector {
     if (filteredRowsExist) {
       for (int i = 0; i < count; i++) {
         if (!filteredRows[rowId]) {
-          writableColumnVector.putShort(counter++, value, ordinal);
+          sparkColumnVectorProxy.putShort(counter++, value, ordinal);
         }
         rowId++;
       }
     } else {
-      writableColumnVector.putShorts(rowId, count, value, ordinal);
+      sparkColumnVectorProxy.putShorts(rowId, count, value, ordinal);
     }
   }
 
   @Override public void putInt(int rowId, int value) {
     if (!filteredRows[rowId]) {
-      writableColumnVector.putInt(counter++, value, ordinal);
+      if (isDictionary) {
+        sparkColumnVectorProxy.putDictionaryInt(counter++, value, ordinal);
+      } else {
+        sparkColumnVectorProxy.putInt(counter++, value, ordinal);
+      }
     }
   }
 
@@ -90,18 +98,18 @@ class ColumnarVectorWrapper implements CarbonColumnVector {
     if (filteredRowsExist) {
       for (int i = 0; i < count; i++) {
         if (!filteredRows[rowId]) {
-          writableColumnVector.putInt(counter++, value, ordinal);
+          sparkColumnVectorProxy.putInt(counter++, value, ordinal);
         }
         rowId++;
       }
     } else {
-      writableColumnVector.putInts(rowId, count, value, ordinal);
+      sparkColumnVectorProxy.putInts(rowId, count, value, ordinal);
     }
   }
 
   @Override public void putLong(int rowId, long value) {
     if (!filteredRows[rowId]) {
-      writableColumnVector.putLong(counter++, value, ordinal);
+      sparkColumnVectorProxy.putLong(counter++, value, ordinal);
     }
   }
 
@@ -109,19 +117,19 @@ class ColumnarVectorWrapper implements CarbonColumnVector {
     if (filteredRowsExist) {
       for (int i = 0; i < count; i++) {
         if (!filteredRows[rowId]) {
-          writableColumnVector.putLong(counter++, value, ordinal);
+          sparkColumnVectorProxy.putLong(counter++, value, ordinal);
         }
         rowId++;
       }
     } else {
-      writableColumnVector.putLongs(rowId, count, value, ordinal);
+      sparkColumnVectorProxy.putLongs(rowId, count, value, ordinal);
     }
   }
 
   @Override public void putDecimal(int rowId, BigDecimal value, int precision) {
     if (!filteredRows[rowId]) {
       Decimal toDecimal = Decimal.apply(value);
-      writableColumnVector.putDecimal(counter++, toDecimal, precision, ordinal);
+      sparkColumnVectorProxy.putDecimal(counter++, toDecimal, precision, ordinal);
     }
   }
 
@@ -129,7 +137,7 @@ class ColumnarVectorWrapper implements CarbonColumnVector {
     Decimal decimal = Decimal.apply(value);
     for (int i = 0; i < count; i++) {
       if (!filteredRows[rowId]) {
-        writableColumnVector.putDecimal(counter++, decimal, precision, ordinal);
+        sparkColumnVectorProxy.putDecimal(counter++, decimal, precision, ordinal);
       }
       rowId++;
     }
@@ -137,7 +145,7 @@ class ColumnarVectorWrapper implements CarbonColumnVector {
 
   @Override public void putDouble(int rowId, double value) {
     if (!filteredRows[rowId]) {
-      writableColumnVector.putDouble(counter++, value, ordinal);
+      sparkColumnVectorProxy.putDouble(counter++, value, ordinal);
     }
   }
 
@@ -145,25 +153,25 @@ class ColumnarVectorWrapper implements CarbonColumnVector {
     if (filteredRowsExist) {
       for (int i = 0; i < count; i++) {
         if (!filteredRows[rowId]) {
-          writableColumnVector.putDouble(counter++, value, ordinal);
+          sparkColumnVectorProxy.putDouble(counter++, value, ordinal);
         }
         rowId++;
       }
     } else {
-      writableColumnVector.putDoubles(rowId, count, value, ordinal);
+      sparkColumnVectorProxy.putDoubles(rowId, count, value, ordinal);
     }
   }
 
   @Override public void putBytes(int rowId, byte[] value) {
     if (!filteredRows[rowId]) {
-      writableColumnVector.putByteArray(counter++, value, ordinal);
+      sparkColumnVectorProxy.putByteArray(counter++, value, ordinal);
     }
   }
 
   @Override public void putBytes(int rowId, int count, byte[] value) {
     for (int i = 0; i < count; i++) {
       if (!filteredRows[rowId]) {
-        writableColumnVector.putByteArray(counter++, value, ordinal);
+        sparkColumnVectorProxy.putByteArray(counter++, value, ordinal);
       }
       rowId++;
     }
@@ -171,13 +179,13 @@ class ColumnarVectorWrapper implements CarbonColumnVector {
 
   @Override public void putBytes(int rowId, int offset, int length, byte[] value) {
     if (!filteredRows[rowId]) {
-      writableColumnVector.putByteArray(counter++, value, offset, length, ordinal);
+      sparkColumnVectorProxy.putByteArray(counter++, value, offset, length, ordinal);
     }
   }
 
   @Override public void putNull(int rowId) {
     if (!filteredRows[rowId]) {
-      writableColumnVector.putNull(counter++, ordinal);
+      sparkColumnVectorProxy.putNull(counter++, ordinal);
     }
   }
 
@@ -185,18 +193,18 @@ class ColumnarVectorWrapper implements CarbonColumnVector {
     if (filteredRowsExist) {
       for (int i = 0; i < count; i++) {
         if (!filteredRows[rowId]) {
-          writableColumnVector.putNull(counter++, ordinal);
+          sparkColumnVectorProxy.putNull(counter++, ordinal);
         }
         rowId++;
       }
     } else {
-      writableColumnVector.putNulls(rowId, count,ordinal);
+      sparkColumnVectorProxy.putNulls(rowId, count,ordinal);
     }
   }
 
   @Override public void putNotNull(int rowId) {
     if (!filteredRows[rowId]) {
-      columnVector.putNotNull(counter++);
+      sparkColumnVectorProxy.putNotNull(counter++,ordinal);
     }
   }
 
@@ -204,17 +212,17 @@ class ColumnarVectorWrapper implements CarbonColumnVector {
     if (filteredRowsExist) {
       for (int i = 0; i < count; i++) {
         if (!filteredRows[rowId]) {
-          columnVector.putNotNull(counter++);
+          sparkColumnVectorProxy.putNotNull(counter++, ordinal);
         }
         rowId++;
       }
     } else {
-      columnVector.putNotNulls(rowId, count);
+      sparkColumnVectorProxy.putNotNulls(rowId, count, ordinal);
     }
   }
 
   @Override public boolean isNull(int rowId) {
-    return writableColumnVector.isNullAt(rowId,ordinal);
+    return sparkColumnVectorProxy.isNullAt(rowId,ordinal);
   }
 
   @Override public void putObject(int rowId, Object obj) {
@@ -236,7 +244,7 @@ class ColumnarVectorWrapper implements CarbonColumnVector {
 
   @Override public DataType getType() {
     return CarbonSparkDataSourceUtil
-        .convertSparkToCarbonDataType(writableColumnVector.dataType(ordinal));
+        .convertSparkToCarbonDataType(sparkColumnVectorProxy.dataType(ordinal));
   }
 
   @Override
@@ -249,20 +257,32 @@ class ColumnarVectorWrapper implements CarbonColumnVector {
     this.blockDataType = blockDataType;
   }
 
-  @Override public void setFilteredRowsExist(boolean filteredRowsExist) {
+  @Override
+  public void setFilteredRowsExist(boolean filteredRowsExist) {
     this.filteredRowsExist = filteredRowsExist;
   }
 
   @Override public void setDictionary(CarbonDictionary dictionary) {
     if (dictionary == null) {
-      columnVector.setDictionary(null);
+      sparkColumnVectorProxy.setDictionary(null, ordinal);
     } else {
-      columnVector.setDictionary(new CarbonDictionaryWrapper(Encoding.PLAIN, dictionary));
+      sparkColumnVectorProxy
+          .setDictionary(new CarbonDictionaryWrapper(Encoding.PLAIN, dictionary),ordinal);
     }
   }
 
+  private void  setDictionaryType(boolean type) {
+    this.isDictionary = type;
+  }
+
   @Override public boolean hasDictionary() {
-    return columnVector.hasDictionary();
+    return sparkColumnVectorProxy.hasDictionary(ordinal);
+  }
+
+  public void reserveDictionaryIds() {
+    sparkColumnVectorProxy.reserveDictionaryIds(sparkColumnVectorProxy.numRows(), ordinal);
+    dictionaryVector = new ColumnarVectorWrapper(sparkColumnVectorProxy, filteredRows, ordinal);
+    ((ColumnarVectorWrapper) dictionaryVector).isDictionary = true;
   }
 
   @Override public CarbonColumnVector getDictionaryVector() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74c3eb10/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
----------------------------------------------------------------------
diff --git a/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java b/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
index 7c98608..d6b6b7e 100644
--- a/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
+++ b/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
@@ -51,16 +51,13 @@ import org.apache.commons.lang3.exception.ExceptionUtils;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.spark.memory.MemoryMode;
-<<<<<<< 2f537b724f6f03ab40c95f7ecc8ebd38f6500099:integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
 import org.apache.spark.sql.carbondata.execution.datasources.CarbonSparkDataSourceUtil;
 import org.apache.spark.sql.catalyst.InternalRow;
 import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
-import org.apache.spark.sql.execution.vectorized.ColumnarBatch;
-=======
 import org.apache.spark.sql.CarbonVectorProxy;
->>>>>>> [CARBONDATA-2532][Integration] Carbon to support spark 2.3 version, ColumnVector Interface:integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
 import org.apache.spark.sql.types.DecimalType;
 import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
 
 /**
  * A specialized RecordReader that reads into InternalRows or ColumnarBatches directly using the
@@ -282,22 +279,23 @@ public class VectorizedCarbonRecordReader extends AbstractRecordReader<Object> {
         schema = schema.add(field);
       }
     }
-    columnarBatch = ColumnarBatch.allocate(schema, memMode);
+    vectorProxy = new CarbonVectorProxy(DEFAULT_MEMORY_MODE,schema,DEFAULT_BATCH_SIZE);
     if (partitionColumns != null) {
       int partitionIdx = fields.length;
       for (int i = 0; i < partitionColumns.fields().length; i++) {
-        ColumnVectorUtils.populate(columnarBatch.column(i + partitionIdx), partitionValues, i);
-        columnarBatch.column(i + partitionIdx).setIsConstant();
+        ColumnVectorUtils.populate(vectorProxy.column(i + partitionIdx), partitionValues, i);
+        vectorProxy.column(i + partitionIdx).setIsConstant();
       }
     }
-    vectorProxy = new CarbonVectorProxy(MemoryMode.OFF_HEAP,DEFAULT_BATCH_SIZE,fields);
     CarbonColumnVector[] vectors = new CarbonColumnVector[fields.length];
     boolean[] filteredRows = new boolean[vectorProxy.numRows()];
     for (int i = 0; i < fields.length; i++) {
-    if (isNoDictStringField[i]) {
-      vectorProxy.reserveDictionaryIds(vectorProxy.numRows(), i);
-      }
       vectors[i] = new ColumnarVectorWrapper(vectorProxy, filteredRows, i);
+      if (isNoDictStringField[i]) {
+        if (vectors[i] instanceof ColumnarVectorWrapper) {
+          ((ColumnarVectorWrapper) vectors[i]).reserveDictionaryIds();
+        }
+      }
     }
     carbonColumnarBatch = new CarbonColumnarBatch(vectors, vectorProxy.numRows(), filteredRows);
   }


[8/8] carbondata git commit: [HOTFIX]Fixed test case failure due to bad record handling

Posted by ra...@apache.org.
[HOTFIX]Fixed test case failure due to bad record handling

This closes #2642


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

Branch: refs/heads/master
Commit: b611a861a9b0f997a58eb3eee3c9606576352217
Parents: 74c3eb1
Author: sandeep-katta <sa...@gmail.com>
Authored: Tue Sep 4 20:00:19 2018 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Wed Sep 5 18:09:45 2018 +0530

----------------------------------------------------------------------
 .../rdd/InsertTaskCompletionListener.scala      | 20 ++++++++++++++++----
 .../spark/rdd/NewCarbonDataLoadRDD.scala        |  7 ++++---
 2 files changed, 20 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/b611a861/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/InsertTaskCompletionListener.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/InsertTaskCompletionListener.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/InsertTaskCompletionListener.scala
index 9439ae5..dfdbd19 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/InsertTaskCompletionListener.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/InsertTaskCompletionListener.scala
@@ -18,16 +18,28 @@
 package org.apache.carbondata.spark.rdd
 
 import org.apache.spark.TaskContext
+import org.apache.spark.sql.execution.command.ExecutionErrors
 import org.apache.spark.util.TaskCompletionListener
 
 import org.apache.carbondata.core.util.ThreadLocalTaskInfo
-import org.apache.carbondata.processing.loading.DataLoadExecutor
+import org.apache.carbondata.processing.loading.{DataLoadExecutor, FailureCauses}
 import org.apache.carbondata.spark.util.CommonUtil
 
-class InsertTaskCompletionListener(dataLoadExecutor: DataLoadExecutor)
+class InsertTaskCompletionListener(dataLoadExecutor: DataLoadExecutor,
+    executorErrors: ExecutionErrors)
   extends TaskCompletionListener {
   override def onTaskCompletion(context: TaskContext): Unit = {
-    dataLoadExecutor.close()
-    CommonUtil.clearUnsafeMemory(ThreadLocalTaskInfo.getCarbonTaskInfo.getTaskId)
+    try {
+      dataLoadExecutor.close()
+    }
+    catch {
+      case e: Exception =>
+        if (executorErrors.failureCauses != FailureCauses.BAD_RECORDS) {
+          throw e
+        }
+    }
+    finally {
+      CommonUtil.clearUnsafeMemory(ThreadLocalTaskInfo.getCarbonTaskInfo.getTaskId)
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/b611a861/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
index 38d6b48..87c8e4c 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
@@ -207,8 +207,8 @@ class NewCarbonDataLoadRDD[K, V](
         loader.initialize()
         val executor = new DataLoadExecutor()
         // in case of success, failure or cancelation clear memory and stop execution
-        context.addTaskCompletionListener { context => executor.close()
-          CommonUtil.clearUnsafeMemory(ThreadLocalTaskInfo.getCarbonTaskInfo.getTaskId)}
+        context
+          .addTaskCompletionListener { new InsertTaskCompletionListener(executor, executionErrors) }
         executor.execute(model,
           loader.storeLocation,
           recordReaders)
@@ -347,7 +347,8 @@ class NewDataFrameLoaderRDD[K, V](
         loader.initialize()
         val executor = new DataLoadExecutor
         // in case of success, failure or cancelation clear memory and stop execution
-        context.addTaskCompletionListener (new InsertTaskCompletionListener(executor))
+        context
+          .addTaskCompletionListener(new InsertTaskCompletionListener(executor, executionErrors))
         executor.execute(model, loader.storeLocation, recordReaders.toArray)
       } catch {
         case e: NoRetryException =>