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 2017/03/23 08:22:15 UTC

[2/5] incubator-carbondata git commit: Fix groupid, package name, Class name issues

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/32bf2961/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataHandleResolver.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataHandleResolver.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataHandleResolver.java
new file mode 100755
index 0000000..5918b46
--- /dev/null
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataHandleResolver.java
@@ -0,0 +1,49 @@
+/*
+ * 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.presto;
+
+import com.facebook.presto.spi.*;
+import com.facebook.presto.spi.connector.ConnectorTransactionHandle;
+
+public class CarbondataHandleResolver implements ConnectorHandleResolver {
+    @Override
+    public Class<? extends ConnectorTableHandle> getTableHandleClass() {
+        return CarbondataTableHandle.class;
+    }
+
+    @Override
+    public Class<? extends ConnectorTableLayoutHandle> getTableLayoutHandleClass() {
+        return CarbondataTableLayoutHandle.class;
+    }
+
+    @Override
+    public Class<? extends ColumnHandle> getColumnHandleClass() {
+        return CarbondataColumnHandle.class;
+    }
+
+    @Override
+    public Class<? extends ConnectorSplit> getSplitClass() {
+        return CarbondataSplit.class;
+    }
+
+    @Override
+    public Class<? extends ConnectorTransactionHandle> getTransactionHandleClass()
+    {
+        return CarbondataTransactionHandle.class;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/32bf2961/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataMetadata.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataMetadata.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataMetadata.java
new file mode 100755
index 0000000..90eca58
--- /dev/null
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataMetadata.java
@@ -0,0 +1,309 @@
+/*
+ * 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.presto;
+
+import com.facebook.presto.carbondata.impl.CarbonTableReader;
+import com.facebook.presto.spi.*;
+import com.facebook.presto.spi.classloader.ThreadContextClassLoader;
+import com.facebook.presto.spi.connector.ConnectorMetadata;
+import com.facebook.presto.spi.type.*;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+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 javax.inject.Inject;
+import java.util.*;
+
+import static com.facebook.presto.carbondata.Types.checkType;
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
+
+public class CarbondataMetadata
+        implements ConnectorMetadata
+{
+    private final String connectorId;
+    private CarbonTableReader carbonTableReader;
+    private ClassLoader classLoader;
+
+    private Map<String, ColumnHandle> columnHandleMap;
+
+    @Inject
+    public CarbondataMetadata(CarbondataConnectorId connectorId, CarbonTableReader reader)
+    {
+        this.connectorId = requireNonNull(connectorId, "connectorId is null").toString();
+        this.carbonTableReader = requireNonNull(reader, "client is null");
+    }
+
+
+    public void putClassLoader(ClassLoader classLoader)
+    {
+        this.classLoader = classLoader;
+    }
+
+
+    @Override
+    public List<String> listSchemaNames(ConnectorSession session) {
+        return listSchemaNamesInternal();
+    }
+
+
+    public List<String> listSchemaNamesInternal()
+    {
+        List<String> ret;
+        try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) {
+            ret = carbonTableReader.getSchemaNames();
+        }
+        return ret;
+    }
+
+    @Override
+    public List<SchemaTableName> listTables(ConnectorSession session, String schemaNameOrNull) {
+
+        /*List<SchemaTableName> all = carbonTableReader.getTableList();
+        if(schemaNameOrNull != null)
+        {
+            return all.stream().filter(a -> schemaNameOrNull.equals(a.getSchemaName())).collect(Collectors.toList());
+        }
+        return all;*/
+
+        List<String> schemaNames;
+        if (schemaNameOrNull != null) {
+            schemaNames = ImmutableList.of(schemaNameOrNull);
+        }
+        else {
+            schemaNames = carbonTableReader.getSchemaNames();
+        }
+
+        ImmutableList.Builder<SchemaTableName> builder = ImmutableList.builder();
+        for (String schemaName : schemaNames) {
+            for (String tableName : carbonTableReader.getTableNames(schemaName)) {
+                builder.add(new SchemaTableName(schemaName, tableName));
+            }
+        }
+        return builder.build();
+    }
+
+    @Override
+    public Map<SchemaTableName, List<ColumnMetadata>> listTableColumns(ConnectorSession session, SchemaTablePrefix prefix) {
+        requireNonNull(prefix, "SchemaTablePrefix is null");
+
+        ImmutableMap.Builder<SchemaTableName, List<ColumnMetadata>> columns = ImmutableMap.builder();
+        for (SchemaTableName tableName : listTables(session, prefix)) {
+            ConnectorTableMetadata tableMetadata = getTableMetadata(tableName);
+            if (tableMetadata != null) {
+                columns.put(tableName, tableMetadata.getColumns());
+            }
+        }
+        return columns.build();
+    }
+
+    //if prefix is null. return all tables
+    //if prefix is not null, just return this table
+    private List<SchemaTableName> listTables(ConnectorSession session, SchemaTablePrefix prefix)
+    {
+        if (prefix.getSchemaName() == null) {
+            return listTables(session, prefix.getSchemaName());
+        }
+        return ImmutableList.of(new SchemaTableName(prefix.getSchemaName(), prefix.getTableName()));
+    }
+
+    private ConnectorTableMetadata getTableMetadata(SchemaTableName tableName)
+    {
+        if (!listSchemaNamesInternal().contains(tableName.getSchemaName())) {
+            return null;
+        }
+
+        CarbonTable cb = carbonTableReader.getTable(tableName);
+        if (cb == null) {
+            return null;
+        }
+
+        List<ColumnMetadata> spiCols = new LinkedList<>();
+        List<CarbonDimension> cols = cb.getDimensionByTableName(tableName.getTableName());
+        for(CarbonDimension col : cols)
+        {
+            //show columns command will return these data
+            Type spiType = CarbondataType2SpiMapper(col.getColumnSchema().getDataType());
+            ColumnMetadata spiCol = new ColumnMetadata(col.getColumnSchema().getColumnName(), spiType);
+            spiCols.add(spiCol);
+        }
+
+        List<CarbonMeasure> mcols = cb.getMeasureByTableName(tableName.getTableName());
+        for(CarbonMeasure mcol : mcols)
+        {
+            Type spiType = CarbondataType2SpiMapper(mcol.getColumnSchema().getDataType());
+            ColumnMetadata spiCol = new ColumnMetadata(mcol.getColumnSchema().getColumnName(), spiType);
+            spiCols.add(spiCol);
+        }
+
+        //\u5c01\u88c5carbonTable
+        return new ConnectorTableMetadata(tableName, spiCols);
+    }
+
+    @Override
+    public Map<String, ColumnHandle> getColumnHandles(ConnectorSession session, ConnectorTableHandle tableHandle) {
+
+        CarbondataTableHandle handle = checkType(tableHandle, CarbondataTableHandle.class, "tableHandle");
+        checkArgument(handle.getConnectorId().equals(connectorId), "tableHandle is not for this connector");
+
+        String schemaName = handle.getSchemaTableName().getSchemaName();
+        if (!listSchemaNamesInternal().contains(schemaName)) {
+            throw new SchemaNotFoundException(schemaName);
+        }
+
+        //CarbonTable(official struct) is stored in CarbonMetadata(official struct)
+        CarbonTable cb = carbonTableReader.getTable(handle.getSchemaTableName());
+        if (cb == null) {
+            throw new TableNotFoundException(handle.getSchemaTableName());
+        }
+
+        ImmutableMap.Builder<String, ColumnHandle> columnHandles = ImmutableMap.builder();
+        int index = 0;
+        String tableName = handle.getSchemaTableName().getTableName();
+        for (CarbonDimension column : cb.getDimensionByTableName(tableName)) {
+            ColumnSchema cs = column.getColumnSchema();
+
+            int complex = column.getComplexTypeOrdinal();
+            column.getNumberOfChild();
+            column.getListOfChildDimensions();
+
+            Type spiType = CarbondataType2SpiMapper(cs.getDataType());
+            columnHandles.put(
+                    cs.getColumnName(),
+                    new CarbondataColumnHandle(
+                            connectorId,
+                            cs.getColumnName(),
+                            spiType,
+                            index,
+                            column.getKeyOrdinal(),
+                            column.getColumnGroupOrdinal(),
+                            false,
+                            cs.getColumnGroupId(),
+                            cs.getColumnUniqueId(),
+                            cs.isUseInvertedIndex()));
+            index++;
+        }
+
+        for(CarbonMeasure measure : cb.getMeasureByTableName(tableName)){
+            ColumnSchema cs = measure.getColumnSchema();
+
+            Type spiType = CarbondataType2SpiMapper(cs.getDataType());
+            columnHandles.put(
+                    cs.getColumnName(),
+                    new CarbondataColumnHandle(
+                            connectorId,
+                            cs.getColumnName(),
+                            spiType,
+                            index,
+                            measure.getOrdinal(),
+                            cs.getColumnGroupId(),
+                            true,
+                            cs.getColumnGroupId(),
+                            cs.getColumnUniqueId(),
+                            cs.isUseInvertedIndex()));
+            index++;
+        }
+
+        //should i cache it?
+        columnHandleMap = columnHandles.build();
+
+        return columnHandleMap;
+    }
+
+    @Override
+    public ColumnMetadata getColumnMetadata(ConnectorSession session, ConnectorTableHandle tableHandle, ColumnHandle columnHandle) {
+
+        checkType(tableHandle, CarbondataTableHandle.class, "tableHandle");
+        return checkType(columnHandle, CarbondataColumnHandle.class, "columnHandle").getColumnMetadata();
+    }
+
+    @Override
+    public ConnectorTableHandle getTableHandle(ConnectorSession session, SchemaTableName tableName) {
+        //check tablename is valid
+        //schema is exist
+        //tables is exist
+
+        //CarbondataTable  get from jar
+        return new CarbondataTableHandle(connectorId, tableName);
+    }
+
+    @Override
+    public List<ConnectorTableLayoutResult> getTableLayouts(ConnectorSession session, ConnectorTableHandle table, Constraint<ColumnHandle> constraint, Optional<Set<ColumnHandle>> desiredColumns) {
+        CarbondataTableHandle handle = checkType(table, CarbondataTableHandle.class, "table");
+        ConnectorTableLayout layout = new ConnectorTableLayout(new CarbondataTableLayoutHandle(handle,constraint.getSummary()/*, constraint.getPredicateMap(),constraint.getFilterTuples()*/));
+        return ImmutableList.of(new ConnectorTableLayoutResult(layout, constraint.getSummary()));
+    }
+
+    @Override
+    public ConnectorTableLayout getTableLayout(ConnectorSession session, ConnectorTableLayoutHandle handle) {
+        return new ConnectorTableLayout(handle);
+    }
+
+    @Override
+    public ConnectorTableMetadata getTableMetadata(ConnectorSession session, ConnectorTableHandle table) {
+        return getTableMetadataInternal(table);
+    }
+
+    public ConnectorTableMetadata getTableMetadataInternal(ConnectorTableHandle table)
+    {
+        CarbondataTableHandle carbondataTableHandle = checkType(table, CarbondataTableHandle.class, "table");
+        checkArgument(carbondataTableHandle.getConnectorId().equals(connectorId), "tableHandle is not for this connector");
+        return getTableMetadata(carbondataTableHandle.getSchemaTableName());
+    }
+
+
+    public static Type CarbondataType2SpiMapper(DataType colType)
+    {
+        switch (colType)
+        {
+            case BOOLEAN:
+                return BooleanType.BOOLEAN;
+            case SHORT:
+                return SmallintType.SMALLINT;
+            case INT:
+                return IntegerType.INTEGER;
+            case LONG:
+                return BigintType.BIGINT;
+            case FLOAT:
+            case DOUBLE:
+                return DoubleType.DOUBLE;
+
+            case DECIMAL:
+                return DecimalType.createDecimalType();
+            case STRING:
+                return VarcharType.VARCHAR;
+            case DATE:
+                return DateType.DATE;
+            case TIMESTAMP:
+                return TimestampType.TIMESTAMP;
+
+            /*case DataType.MAP:
+            case DataType.ARRAY:
+            case DataType.STRUCT:
+            case DataType.NULL:*/
+
+            default:
+                return VarcharType.VARCHAR;
+        }
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/32bf2961/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataModule.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataModule.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataModule.java
new file mode 100755
index 0000000..b329678
--- /dev/null
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataModule.java
@@ -0,0 +1,83 @@
+/*
+ * 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.presto;
+
+import org.apache.carbondata.presto.impl.CarbonTableConfig;
+import org.apache.carbondata.presto.impl.CarbonTableReader;
+import com.facebook.presto.spi.connector.ConnectorRecordSetProvider;
+import com.facebook.presto.spi.connector.ConnectorSplitManager;
+import com.facebook.presto.spi.type.Type;
+import com.facebook.presto.spi.type.TypeManager;
+import com.fasterxml.jackson.databind.DeserializationContext;
+import com.fasterxml.jackson.databind.deser.std.FromStringDeserializer;
+import com.google.inject.Binder;
+import com.google.inject.Module;
+import com.google.inject.Scopes;
+
+import javax.inject.Inject;
+
+import static com.facebook.presto.spi.type.TypeSignature.parseTypeSignature;
+import static com.google.common.base.Preconditions.checkArgument;
+import static io.airlift.configuration.ConfigBinder.configBinder;
+import static java.util.Objects.requireNonNull;
+
+public class CarbondataModule implements Module {
+
+    private final String connectorId;
+    private final TypeManager typeManager;
+
+    public CarbondataModule(String connectorId, TypeManager typeManager)
+    {
+        this.connectorId = requireNonNull(connectorId, "connector id is null");
+        this.typeManager = requireNonNull(typeManager, "typeManager is null");
+    }
+
+    @Override
+    public void configure(Binder binder) {
+        binder.bind(TypeManager.class).toInstance(typeManager);
+
+        binder.bind(CarbondataConnectorId.class).toInstance(new CarbondataConnectorId(connectorId));
+        binder.bind(CarbondataMetadata.class).in(Scopes.SINGLETON);
+        binder.bind(CarbonTableReader.class).in(Scopes.SINGLETON);
+        binder.bind(ConnectorSplitManager.class).to(CarbondataSplitManager.class).in(Scopes.SINGLETON);
+        binder.bind(ConnectorRecordSetProvider.class).to(CarbondataRecordSetProvider.class).in(Scopes.SINGLETON);
+        binder.bind(CarbondataHandleResolver.class).in(Scopes.SINGLETON);
+        configBinder(binder).bindConfig(CarbonTableConfig.class);
+    }
+
+    public static final class TypeDeserializer
+            extends FromStringDeserializer<Type>
+    {
+        private final TypeManager typeManager;
+
+        @Inject
+        public TypeDeserializer(TypeManager typeManager)
+        {
+            super(Type.class);
+            this.typeManager = requireNonNull(typeManager, "typeManager is null");
+        }
+
+        @Override
+        protected Type _deserialize(String value, DeserializationContext context)
+        {
+            Type type = typeManager.getType(parseTypeSignature(value));
+            checkArgument(type != null, "Unknown type %s", value);
+            return type;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/32bf2961/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataPlugin.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataPlugin.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataPlugin.java
new file mode 100755
index 0000000..3bcfe4f
--- /dev/null
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataPlugin.java
@@ -0,0 +1,36 @@
+/*
+ * 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.presto;
+
+import com.facebook.presto.spi.Plugin;
+import com.facebook.presto.spi.connector.ConnectorFactory;
+import com.google.common.collect.ImmutableList;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+
+public class CarbondataPlugin implements Plugin {
+
+    @Override
+    public Iterable<ConnectorFactory> getConnectorFactories()
+    {
+        return ImmutableList.of(new CarbondataConnectorFactory("carbondata", getClassLoader()));
+    }
+
+    private static ClassLoader getClassLoader() {
+        return FileFactory.class.getClassLoader();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/32bf2961/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordCursor.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordCursor.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordCursor.java
new file mode 100755
index 0000000..7d65efd
--- /dev/null
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordCursor.java
@@ -0,0 +1,156 @@
+/*
+ * 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.presto;
+
+import com.facebook.presto.spi.RecordCursor;
+import com.facebook.presto.spi.type.Type;
+import com.google.common.base.Strings;
+import io.airlift.log.Logger;
+import io.airlift.slice.Slice;
+import io.airlift.slice.Slices;
+import org.apache.carbondata.common.CarbonIterator;
+import org.apache.carbondata.hadoop.readsupport.CarbonReadSupport;
+
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static com.facebook.presto.spi.type.BooleanType.BOOLEAN;
+import static com.facebook.presto.spi.type.DoubleType.DOUBLE;
+import static com.facebook.presto.spi.type.VarcharType.VARCHAR;
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
+
+public class CarbondataRecordCursor implements RecordCursor {
+
+    private static final Logger log = Logger.get(CarbondataRecordCursor.class);
+    private final List<CarbondataColumnHandle> columnHandles;
+
+    private List<String> fields;
+    private CarbondataSplit split;
+    private CarbonIterator<Object[]> rowCursor;
+    private CarbonReadSupport<Object[]> readSupport;
+
+    private long totalBytes;
+    private long nanoStart;
+    private long nanoEnd;
+
+    public CarbondataRecordCursor(CarbonReadSupport<Object[]> readSupport, CarbonIterator<Object[]> carbonIterator, List<CarbondataColumnHandle> columnHandles, CarbondataSplit split) {
+        this.rowCursor = carbonIterator;
+        this.columnHandles = columnHandles;
+        this.readSupport = readSupport;
+        this.totalBytes = 0;
+    }
+
+
+    @Override
+    public long getTotalBytes() {
+        return totalBytes;
+    }
+
+    @Override
+    public long getCompletedBytes() {
+        return totalBytes;
+    }
+
+    @Override
+    public long getReadTimeNanos() {
+        return nanoStart > 0L ? (nanoEnd == 0 ? System.nanoTime() : nanoEnd) - nanoStart : 0L;
+    }
+
+    @Override
+    public Type getType(int field) {
+
+        checkArgument(field < columnHandles.size(), "Invalid field index");
+        return columnHandles.get(field).getColumnType();
+    }
+
+    @Override
+    public boolean advanceNextPosition() {
+
+        if (nanoStart == 0) {
+            nanoStart = System.nanoTime();
+        }
+
+        if(rowCursor.hasNext())
+        {
+            fields = Stream.of(readSupport.readRow(rowCursor.next())).map(a -> a.toString()).collect(Collectors.toList());
+
+            totalBytes += fields.size();
+            return true;
+        }
+        return false;
+    }
+
+    @Override
+    public boolean getBoolean(int field) {
+        checkFieldType(field, BOOLEAN);
+        return Boolean.parseBoolean(getFieldValue(field));
+    }
+
+    @Override
+    public long getLong(int field) {
+        String timeStr = getFieldValue(field);
+        Long milliSec = 0L;
+
+        //suppose the
+        return Math.round(Double.parseDouble(getFieldValue(field)));
+    }
+
+    @Override
+    public double getDouble(int field) {
+        checkFieldType(field, DOUBLE);
+        return Double.parseDouble(getFieldValue(field));
+    }
+
+    @Override
+    public Slice getSlice(int field) {
+        checkFieldType(field, VARCHAR);
+        return Slices.utf8Slice(getFieldValue(field));
+    }
+
+    @Override
+    public Object getObject(int field) {
+        return null;
+    }
+
+    @Override
+    public boolean isNull(int field) {
+        checkArgument(field < columnHandles.size(), "Invalid field index");
+        return Strings.isNullOrEmpty(getFieldValue(field));
+    }
+
+    String getFieldValue(int field)
+    {
+        checkState(fields != null, "Cursor has not been advanced yet");
+        return fields.get(field);
+    }
+
+    private void checkFieldType(int field, Type expected)
+    {
+        Type actual = getType(field);
+        checkArgument(actual.equals(expected), "Expected field %s to be type %s but is %s", field, expected, actual);
+    }
+
+    @Override
+    public void close() {
+        nanoEnd = System.nanoTime();
+
+        //todo  delete cache from readSupport
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/32bf2961/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSet.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSet.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSet.java
new file mode 100755
index 0000000..60e940b
--- /dev/null
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSet.java
@@ -0,0 +1,115 @@
+/*
+ * 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.presto;
+
+import com.facebook.presto.spi.*;
+import com.facebook.presto.spi.predicate.TupleDomain;
+import com.facebook.presto.spi.type.Type;
+import org.apache.carbondata.common.CarbonIterator;
+import org.apache.carbondata.core.datastore.block.TableBlockInfo;
+import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.scan.executor.QueryExecutor;
+import org.apache.carbondata.core.scan.executor.QueryExecutorFactory;
+import org.apache.carbondata.core.scan.executor.exception.QueryExecutionException;
+import org.apache.carbondata.core.scan.expression.Expression;
+import org.apache.carbondata.core.scan.model.QueryModel;
+import org.apache.carbondata.core.scan.result.BatchResult;
+import org.apache.carbondata.core.scan.result.iterator.ChunkRowIterator;
+import org.apache.carbondata.hadoop.readsupport.CarbonReadSupport;
+import org.apache.carbondata.hadoop.readsupport.impl.DictionaryDecodedReadSupportImpl;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static com.facebook.presto.carbondata.Types.checkType;
+
+public class CarbondataRecordSet implements RecordSet {
+
+    private CarbonTable carbonTable;
+    private TupleDomain<ColumnHandle> originalConstraint;
+    private Expression carbonConstraint;
+    private List<CarbondataColumnConstraint> rebuildConstraints;
+    private QueryModel queryModel;
+    private CarbondataSplit split;
+    private List<CarbondataColumnHandle> columns;
+    private QueryExecutor queryExecutor;
+
+    private CarbonReadSupport<Object[]> readSupport;
+
+    public CarbondataRecordSet(
+            CarbonTable carbonTable,
+            ConnectorSession session,
+            ConnectorSplit split,
+            List<CarbondataColumnHandle> columns,
+            QueryModel queryModel){
+        this.carbonTable = carbonTable;
+        this.split = checkType(split, CarbondataSplit.class, "connectorSplit");
+        this.originalConstraint = this.split.getConstraints();
+        this.rebuildConstraints = this.split.getRebuildConstraints();
+        this.queryModel = queryModel;
+        this.columns = columns;
+        this.readSupport = new DictionaryDecodedReadSupportImpl();
+    }
+
+    //todo support later
+    private Expression parseConstraint2Expression(TupleDomain<ColumnHandle> constraints) {
+        return null;
+    }
+
+    @Override
+    public List<Type> getColumnTypes() {
+        return columns.stream().map(a -> a.getColumnType()).collect(Collectors.toList());
+    }
+
+    @Override
+    public RecordCursor cursor() {
+        List<TableBlockInfo> tableBlockInfoList = new ArrayList<TableBlockInfo>();
+
+        //tableBlockInfoList.add(split.getLocalInputSplit().getTableBlockInfo());
+        /*BlockletInfos blockletInfos = new BlockletInfos(split.getLocalInputSplit().getNumberOfBlocklets(), 0,
+                split.getLocalInputSplit().getNumberOfBlocklets());*/
+        tableBlockInfoList.add(
+                new TableBlockInfo(split.getLocalInputSplit().getPath().toString(),
+                        split.getLocalInputSplit().getStart(),
+                        split.getLocalInputSplit().getSegmentId(),
+                        split.getLocalInputSplit().getLocations().toArray(new String[0]),
+                        split.getLocalInputSplit().getLength(),
+                        //blockletInfos,
+                        ColumnarFormatVersion.valueOf(split.getLocalInputSplit().getVersion())));
+        queryModel.setTableBlockInfos(tableBlockInfoList);
+
+        queryExecutor = QueryExecutorFactory.getQueryExecutor(queryModel);
+
+        //queryModel.setQueryId(queryModel.getQueryId() + "_" + split.getLocalInputSplit().getSegmentId());
+        try {
+            readSupport.initialize(queryModel.getProjectionColumns(), queryModel.getAbsoluteTableIdentifier());
+            CarbonIterator<Object[]> carbonIterator = new ChunkRowIterator((CarbonIterator<BatchResult>) queryExecutor.execute(queryModel));
+            RecordCursor rc = new CarbondataRecordCursor(readSupport, carbonIterator, columns, split);
+            return rc;
+        } catch (QueryExecutionException e) {
+            //throw new InterruptedException(e.getMessage());
+            System.out.println(e.getMessage());
+        } catch(Exception ex) {
+            System.out.println(ex.toString());
+        }
+        return null;
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/32bf2961/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSetProvider.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSetProvider.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSetProvider.java
new file mode 100755
index 0000000..4d3ace5
--- /dev/null
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSetProvider.java
@@ -0,0 +1,296 @@
+/*
+ * 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.presto;
+
+import org.apache.carbondata.presto.impl.CarbonTableCacheModel;
+import org.apache.carbondata.presto.impl.CarbonTableReader;
+import com.facebook.presto.spi.ColumnHandle;
+import com.facebook.presto.spi.ConnectorSession;
+import com.facebook.presto.spi.ConnectorSplit;
+import com.facebook.presto.spi.RecordSet;
+import com.facebook.presto.spi.connector.ConnectorRecordSetProvider;
+import com.facebook.presto.spi.connector.ConnectorTransactionHandle;
+import com.facebook.presto.spi.predicate.Domain;
+import com.facebook.presto.spi.predicate.Range;
+import com.facebook.presto.spi.predicate.TupleDomain;
+import com.facebook.presto.spi.type.*;
+import com.google.common.collect.ImmutableList;
+import io.airlift.slice.Slice;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.scan.expression.ColumnExpression;
+import org.apache.carbondata.core.scan.expression.Expression;
+import org.apache.carbondata.core.scan.expression.LiteralExpression;
+import org.apache.carbondata.core.scan.expression.conditional.*;
+import org.apache.carbondata.core.scan.expression.logical.AndExpression;
+import org.apache.carbondata.core.scan.expression.logical.OrExpression;
+import org.apache.carbondata.core.scan.model.CarbonQueryPlan;
+import org.apache.carbondata.core.scan.model.QueryModel;
+import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil;
+
+import javax.inject.Inject;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static com.facebook.presto.carbondata.Types.checkType;
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Preconditions.checkState;
+import static java.util.Objects.requireNonNull;
+
+public class CarbondataRecordSetProvider implements ConnectorRecordSetProvider {
+
+    private final String connectorId;
+    private final CarbonTableReader carbonTableReader;
+
+    @Inject
+    public CarbondataRecordSetProvider(
+            CarbondataConnectorId connectorId,
+            CarbonTableReader reader)
+    {
+        //this.config = requireNonNull(config, "config is null");
+        //this.connector = requireNonNull(connector, "connector is null");
+        this.connectorId = requireNonNull(connectorId, "connectorId is null").toString();
+        this.carbonTableReader = reader;
+    }
+
+    @Override
+    public RecordSet getRecordSet(ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorSplit split, List<? extends ColumnHandle> columns) {
+        requireNonNull(split, "split is null");
+        requireNonNull(columns, "columns is null");
+
+        // Convert split
+        CarbondataSplit cdSplit = checkType(split, CarbondataSplit.class, "split is not class CarbondataSplit");
+        checkArgument(cdSplit.getConnectorId().equals(connectorId), "split is not for this connector");
+
+        // Convert all columns handles
+        ImmutableList.Builder<CarbondataColumnHandle> handles = ImmutableList.builder();
+        for (ColumnHandle handle : columns) {
+            handles.add(checkType(handle, CarbondataColumnHandle.class, "handle"));
+        }
+
+        // Build column projection(check the column order)
+        String targetCols = "";
+        for(ColumnHandle col : columns){
+            targetCols += ((CarbondataColumnHandle)col).getColumnName() + ",";
+        }
+        targetCols = targetCols.substring(0, targetCols.length() -1 );
+        //String cols = String.join(",", columns.stream().map(a -> ((CarbondataColumnHandle)a).getColumnName()).collect(Collectors.toList()));
+
+        CarbonTableCacheModel tableCacheModel = carbonTableReader.getCarbonCache(cdSplit.getSchemaTableName());
+        checkNotNull(tableCacheModel, "tableCacheModel should not be null");
+        checkNotNull(tableCacheModel.carbonTable, "tableCacheModel.carbonTable should not be null");
+        checkNotNull(tableCacheModel.tableInfo, "tableCacheModel.tableInfo should not be null");
+
+        // Build Query Model
+        CarbonTable targetTable = tableCacheModel.carbonTable;
+        CarbonQueryPlan queryPlan = CarbonInputFormatUtil.createQueryPlan(targetTable, targetCols);
+        QueryModel queryModel = QueryModel.createModel(targetTable.getAbsoluteTableIdentifier(), queryPlan, targetTable);
+
+        // Push down filter
+        fillFilter2QueryModel(queryModel, cdSplit.getConstraints(), targetTable);
+
+        // Return new record set
+        return new CarbondataRecordSet(targetTable,/*connector,*/ session, /*config, */cdSplit, handles.build(), queryModel);
+    }
+
+    // Build filter for QueryModel (copy from CarbonInputFormat=> createRecordReader)
+    private void fillFilter2QueryModel(QueryModel queryModel, TupleDomain<ColumnHandle> originalConstraint, CarbonTable carbonTable) {
+
+        //queryModel.setFilterExpressionResolverTree(new FilterResolverIntf());
+
+        //Build Predicate Expression
+        ImmutableList.Builder<Expression> filters = ImmutableList.builder();
+
+        Domain domain = null;
+
+        for (ColumnHandle c : originalConstraint.getDomains().get().keySet()) {
+
+            // Build ColumnExpresstion for Expresstion(Carbondata)
+            CarbondataColumnHandle cdch = (CarbondataColumnHandle) c;
+            Type type = cdch.getColumnType();
+
+            DataType coltype = Spi2CarbondataTypeMapper(type);
+            Expression colExpression = new ColumnExpression(cdch.getColumnName(), coltype);
+
+            domain = originalConstraint.getDomains().get().get(c);
+            checkArgument(domain.getType().isOrderable(), "Domain type must be orderable");
+
+            if (domain.getValues().isNone()) {
+                //return QueryBuilders.filteredQuery(null, FilterBuilders.missingFilter(columnName));
+                //return domain.isNullAllowed() ? columnName + " IS NULL" : "FALSE";
+                //new Expression()
+            }
+
+            if (domain.getValues().isAll()) {
+                //return QueryBuilders.filteredQuery(null, FilterBuilders.existsFilter(columnName));
+                //return domain.isNullAllowed() ? "TRUE" : columnName + " IS NOT NULL";
+            }
+
+            List<Object> singleValues = new ArrayList<>();
+            List<Expression> rangeFilter = new ArrayList<>();
+            for (Range range : domain.getValues().getRanges().getOrderedRanges()) {
+                checkState(!range.isAll()); // Already checked
+                if (range.isSingleValue()) {
+                    singleValues.add(range.getLow().getValue());
+                }
+                else
+                {
+                    List<String> rangeConjuncts = new ArrayList<>();
+                    if (!range.getLow().isLowerUnbounded()) {
+                        Object value = ConvertDataByType(range.getLow().getValue(), type);
+                        switch (range.getLow().getBound()) {
+                            case ABOVE:
+                                if (type == TimestampType.TIMESTAMP) {
+                                    //todo not now
+                                } else {
+                                    GreaterThanExpression greater = new GreaterThanExpression(colExpression, new LiteralExpression(value, coltype));
+                                    //greater.setRangeExpression(true);
+                                    rangeFilter.add(greater);
+                                }
+                                break;
+                            case EXACTLY:
+                                GreaterThanEqualToExpression greater = new GreaterThanEqualToExpression(colExpression, new LiteralExpression(value, coltype));
+                                //greater.setRangeExpression(true);
+                                rangeFilter.add(greater);
+                                break;
+                            case BELOW:
+                                throw new IllegalArgumentException("Low marker should never use BELOW bound");
+                            default:
+                                throw new AssertionError("Unhandled bound: " + range.getLow().getBound());
+                        }
+                    }
+                    if (!range.getHigh().isUpperUnbounded()) {
+                        Object value = ConvertDataByType(range.getHigh().getValue(), type);
+                        switch (range.getHigh().getBound()) {
+                            case ABOVE:
+                                throw new IllegalArgumentException("High marker should never use ABOVE bound");
+                            case EXACTLY:
+                                LessThanEqualToExpression less = new LessThanEqualToExpression(colExpression, new LiteralExpression(value, coltype));
+                                //less.setRangeExpression(true);
+                                rangeFilter.add(less);
+                                break;
+                            case BELOW:
+                                LessThanExpression less2 = new LessThanExpression(colExpression, new LiteralExpression(value, coltype));
+                                //less2.setRangeExpression(true);
+                                rangeFilter.add(less2);
+                                break;
+                            default:
+                                throw new AssertionError("Unhandled bound: " + range.getHigh().getBound());
+                        }
+                    }
+                }
+            }
+
+            if (singleValues.size() == 1) {
+                Expression ex = null;
+                if (coltype.equals(DataType.STRING)) {
+                    ex = new EqualToExpression(colExpression, new LiteralExpression(((Slice) singleValues.get(0)).toStringUtf8(), coltype));
+                } else
+                    ex = new EqualToExpression(colExpression, new LiteralExpression(singleValues.get(0), coltype));
+                filters.add(ex);
+            }
+            else if(singleValues.size() > 1) {
+                ListExpression candidates = null;
+                List<Expression> exs = singleValues.stream().map((a) ->
+                {
+                    return new LiteralExpression(ConvertDataByType(a, type), coltype);
+                }).collect(Collectors.toList());
+                candidates = new ListExpression(exs);
+
+                if(candidates != null)
+                    filters.add(new InExpression(colExpression, candidates));
+            }
+            else if(rangeFilter.size() > 0){
+                if(rangeFilter.size() > 1) {
+                    Expression finalFilters = new OrExpression(rangeFilter.get(0), rangeFilter.get(1));
+                    if(rangeFilter.size() > 2)
+                    {
+                        for(int i = 2; i< rangeFilter.size(); i++)
+                        {
+                            filters.add(new AndExpression(finalFilters, rangeFilter.get(i)));
+                        }
+                    }
+                }
+                else if(rangeFilter.size() == 1)
+                    filters.add(rangeFilter.get(0));
+            }
+        }
+
+        Expression finalFilters;
+        List<Expression> tmp = filters.build();
+        if(tmp.size() > 1) {
+            finalFilters = new AndExpression(tmp.get(0), tmp.get(1));
+            if(tmp.size() > 2)
+            {
+                for(int i = 2; i< tmp.size(); i++)
+                {
+                    finalFilters = new AndExpression(finalFilters, tmp.get(i));
+                }
+            }
+        }
+        else if(tmp.size() == 1)
+            finalFilters = tmp.get(0);
+        else
+            return;
+
+        // todo set into QueryModel
+        CarbonInputFormatUtil.processFilterExpression(finalFilters, carbonTable);
+        queryModel.setFilterExpressionResolverTree(CarbonInputFormatUtil.resolveFilter(finalFilters, queryModel.getAbsoluteTableIdentifier()));
+    }
+
+    public static DataType Spi2CarbondataTypeMapper(Type colType)
+    {
+        if(colType == BooleanType.BOOLEAN)
+            return DataType.BOOLEAN;
+        else if(colType == SmallintType.SMALLINT)
+            return DataType.SHORT;
+        else if(colType == IntegerType.INTEGER)
+            return DataType.INT;
+        else if(colType == BigintType.BIGINT)
+            return DataType.LONG;
+        else if(colType == DoubleType.DOUBLE)
+            return DataType.DOUBLE;
+        else if(colType == DecimalType.createDecimalType())
+            return DataType.DECIMAL;
+        else if(colType == VarcharType.VARCHAR)
+            return DataType.STRING;
+        else if(colType == DateType.DATE)
+            return DataType.DATE;
+        else if(colType == TimestampType.TIMESTAMP)
+            return DataType.TIMESTAMP;
+        else
+            return DataType.STRING;
+    }
+
+
+    public Object ConvertDataByType(Object rawdata, Type type)
+    {
+        if(type.equals(IntegerType.INTEGER))
+            return new Integer((rawdata.toString()));
+        else if(type.equals(BigintType.BIGINT))
+            return (Long)rawdata;
+        else if(type.equals(VarcharType.VARCHAR))
+            return ((Slice)rawdata).toStringUtf8();
+        else if(type.equals(BooleanType.BOOLEAN))
+            return (Boolean)(rawdata);
+
+        return rawdata;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/32bf2961/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataSplit.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataSplit.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataSplit.java
new file mode 100755
index 0000000..741dfcc
--- /dev/null
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataSplit.java
@@ -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.carbondata.presto;
+
+import org.apache.carbondata.presto.impl.CarbonLocalInputSplit;
+import com.facebook.presto.spi.ColumnHandle;
+import com.facebook.presto.spi.ConnectorSplit;
+import com.facebook.presto.spi.HostAddress;
+import com.facebook.presto.spi.SchemaTableName;
+import com.facebook.presto.spi.predicate.TupleDomain;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+import static java.util.Objects.requireNonNull;
+
+public class CarbondataSplit implements ConnectorSplit {
+
+    private final String connectorId;
+    private final SchemaTableName schemaTableName;
+    private final TupleDomain<ColumnHandle> constraints;
+    private final CarbonLocalInputSplit localInputSplit;
+    private final List<CarbondataColumnConstraint> rebuildConstraints;
+    private final ImmutableList<HostAddress> addresses;
+
+    @JsonCreator
+    public CarbondataSplit( @JsonProperty("connectorId") String connectorId,
+                            @JsonProperty("schemaTableName") SchemaTableName schemaTableName,
+                            @JsonProperty("constraints") TupleDomain<ColumnHandle> constraints,
+                            @JsonProperty("localInputSplit") CarbonLocalInputSplit localInputSplit,
+                            @JsonProperty("rebuildConstraints") List<CarbondataColumnConstraint> rebuildConstraints) {
+        this.connectorId = requireNonNull(connectorId, "connectorId is null");
+        this.schemaTableName = requireNonNull(schemaTableName, "schemaTable is null");
+        this.constraints = requireNonNull(constraints, "constraints is null");
+        this.localInputSplit = requireNonNull(localInputSplit, "localInputSplit is null");
+        this.rebuildConstraints = requireNonNull(rebuildConstraints, "rebuildConstraints is null");
+        this.addresses = ImmutableList.of();
+    }
+
+
+    @JsonProperty
+    public String getConnectorId() {
+        return connectorId;
+    }
+
+    @JsonProperty
+    public SchemaTableName getSchemaTableName(){
+        return  schemaTableName;
+    }
+
+    @JsonProperty
+    public TupleDomain<ColumnHandle> getConstraints() {
+        return constraints;
+    }
+
+    @JsonProperty
+    public CarbonLocalInputSplit getLocalInputSplit(){return localInputSplit;}
+
+    @JsonProperty
+    public List<CarbondataColumnConstraint> getRebuildConstraints() {
+        return rebuildConstraints;
+    }
+
+    @Override
+    public boolean isRemotelyAccessible() {
+        return true;
+    }
+
+    @Override
+    public List<HostAddress> getAddresses() {
+        return addresses;
+    }
+
+    @Override
+    public Object getInfo() {
+        return this;
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/32bf2961/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataSplitManager.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataSplitManager.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataSplitManager.java
new file mode 100755
index 0000000..34b075e
--- /dev/null
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataSplitManager.java
@@ -0,0 +1,304 @@
+/*
+ * 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.presto;
+
+import org.apache.carbondata.presto.impl.CarbonLocalInputSplit;
+import org.apache.carbondata.presto.impl.CarbonTableCacheModel;
+import org.apache.carbondata.presto.impl.CarbonTableReader;
+import com.facebook.presto.spi.*;
+import com.facebook.presto.spi.connector.ConnectorSplitManager;
+import com.facebook.presto.spi.connector.ConnectorTransactionHandle;
+import com.facebook.presto.spi.predicate.Domain;
+import com.facebook.presto.spi.predicate.Range;
+import com.facebook.presto.spi.predicate.TupleDomain;
+import com.facebook.presto.spi.type.*;
+import com.google.common.collect.ImmutableList;
+import io.airlift.slice.Slice;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
+import org.apache.carbondata.core.scan.expression.ColumnExpression;
+import org.apache.carbondata.core.scan.expression.Expression;
+import org.apache.carbondata.core.scan.expression.LiteralExpression;
+import org.apache.carbondata.core.scan.expression.conditional.*;
+import org.apache.carbondata.core.scan.expression.logical.AndExpression;
+import org.apache.carbondata.core.scan.expression.logical.OrExpression;
+
+import javax.inject.Inject;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+import static com.facebook.presto.carbondata.Types.checkType;
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
+import static java.util.Objects.requireNonNull;
+
+public class CarbondataSplitManager
+        implements ConnectorSplitManager
+{
+
+    private final String connectorId;
+    private final CarbonTableReader carbonTableReader;
+
+    @Inject
+    public CarbondataSplitManager(CarbondataConnectorId connectorId, CarbonTableReader reader)
+    {
+        this.connectorId = requireNonNull(connectorId, "connectorId is null").toString();
+        this.carbonTableReader = requireNonNull(reader, "client is null");
+    }
+
+    public ConnectorSplitSource getSplits(ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorTableLayoutHandle layout)
+    {
+        CarbondataTableLayoutHandle layoutHandle = (CarbondataTableLayoutHandle)layout;
+        CarbondataTableHandle tableHandle = layoutHandle.getTable();
+        SchemaTableName key = tableHandle.getSchemaTableName();
+
+        //get all filter domain
+        List<CarbondataColumnConstraint> rebuildConstraints = getColumnConstraints(layoutHandle.getConstraint());
+
+        CarbonTableCacheModel cache = carbonTableReader.getCarbonCache(key);
+        Expression filters = parseFilterExpression(layoutHandle.getConstraint(), cache.carbonTable);
+
+        if(cache != null) {
+            try {
+                List<CarbonLocalInputSplit> splits = carbonTableReader.getInputSplits2(cache, filters);
+
+                ImmutableList.Builder<ConnectorSplit> cSplits = ImmutableList.builder();
+                for (CarbonLocalInputSplit split : splits) {
+                    cSplits.add(new CarbondataSplit(
+                            connectorId,
+                            tableHandle.getSchemaTableName(),
+                            layoutHandle.getConstraint(),
+                            split,
+                            rebuildConstraints
+                    ));
+                }
+                return new FixedSplitSource(cSplits.build());
+            } catch (Exception ex) {
+                System.out.println(ex.toString());
+            }
+        }
+        return null;
+    }
+
+
+    public List<CarbondataColumnConstraint> getColumnConstraints(TupleDomain<ColumnHandle> constraint)
+    {
+        ImmutableList.Builder<CarbondataColumnConstraint> constraintBuilder = ImmutableList.builder();
+        for (TupleDomain.ColumnDomain<ColumnHandle> columnDomain : constraint.getColumnDomains().get()) {
+            CarbondataColumnHandle columnHandle = checkType(columnDomain.getColumn(), CarbondataColumnHandle.class, "column handle");
+
+            constraintBuilder.add(new CarbondataColumnConstraint(
+                    columnHandle.getColumnName(),
+                    Optional.of(columnDomain.getDomain()),
+                    columnHandle.isInvertedIndex()));
+        }
+
+        return constraintBuilder.build();
+    }
+
+
+    public Expression parseFilterExpression(TupleDomain<ColumnHandle> originalConstraint, CarbonTable carbonTable)
+    {
+        ImmutableList.Builder<Expression> filters = ImmutableList.builder();
+
+        Domain domain = null;
+
+        for (ColumnHandle c : originalConstraint.getDomains().get().keySet()) {
+
+            CarbondataColumnHandle cdch = (CarbondataColumnHandle) c;
+            Type type = cdch.getColumnType();
+
+            List<CarbonColumn> ccols = carbonTable.getCreateOrderColumn(carbonTable.getFactTableName());
+            Optional<CarbonColumn>  target = ccols.stream().filter(a -> a.getColName().equals(cdch.getColumnName())).findFirst();
+
+            if(target.get() == null)
+                return null;
+
+            DataType coltype = target.get().getDataType();
+            ColumnExpression colExpression = new ColumnExpression(cdch.getColumnName(), target.get().getDataType());
+            //colExpression.setColIndex(cs.getSchemaOrdinal());
+            colExpression.setDimension(target.get().isDimesion());
+            colExpression.setDimension(carbonTable.getDimensionByName(carbonTable.getFactTableName(), cdch.getColumnName()));
+            colExpression.setCarbonColumn(target.get());
+
+            domain = originalConstraint.getDomains().get().get(c);
+            checkArgument(domain.getType().isOrderable(), "Domain type must be orderable");
+
+            if (domain.getValues().isNone()) {
+                //return QueryBuilders.filteredQuery(null, FilterBuilders.missingFilter(columnName));
+                //return domain.isNullAllowed() ? columnName + " IS NULL" : "FALSE";
+                //new Expression()
+            }
+
+            if (domain.getValues().isAll()) {
+                //return QueryBuilders.filteredQuery(null, FilterBuilders.existsFilter(columnName));
+                //return domain.isNullAllowed() ? "TRUE" : columnName + " IS NOT NULL";
+            }
+
+            List<Object> singleValues = new ArrayList<>();
+            List<Expression> rangeFilter = new ArrayList<>();
+            for (Range range : domain.getValues().getRanges().getOrderedRanges()) {
+                checkState(!range.isAll()); // Already checked
+                if (range.isSingleValue()) {
+                    singleValues.add(range.getLow().getValue());
+                }
+                else
+                {
+                    List<String> rangeConjuncts = new ArrayList<>();
+                    if (!range.getLow().isLowerUnbounded()) {
+                        Object value = ConvertDataByType(range.getLow().getValue(), type);
+                        switch (range.getLow().getBound()) {
+                            case ABOVE:
+                                if (type == TimestampType.TIMESTAMP) {
+                                    //todo not now
+                                } else {
+                                    GreaterThanExpression greater = new GreaterThanExpression(colExpression, new LiteralExpression(value, coltype));
+                                    //greater.setRangeExpression(true);
+                                    rangeFilter.add(greater);
+                                }
+                                break;
+                            case EXACTLY:
+                                GreaterThanEqualToExpression greater = new GreaterThanEqualToExpression(colExpression, new LiteralExpression(value, coltype));
+                                //greater.setRangeExpression(true);
+                                rangeFilter.add(greater);
+                                break;
+                            case BELOW:
+                                throw new IllegalArgumentException("Low marker should never use BELOW bound");
+                            default:
+                                throw new AssertionError("Unhandled bound: " + range.getLow().getBound());
+                        }
+                    }
+                    if (!range.getHigh().isUpperUnbounded()) {
+                        Object value = ConvertDataByType(range.getHigh().getValue(), type);
+                        switch (range.getHigh().getBound()) {
+                            case ABOVE:
+                                throw new IllegalArgumentException("High marker should never use ABOVE bound");
+                            case EXACTLY:
+                                LessThanEqualToExpression less = new LessThanEqualToExpression(colExpression, new LiteralExpression(value, coltype));
+                                //less.setRangeExpression(true);
+                                rangeFilter.add(less);
+                                break;
+                            case BELOW:
+                                LessThanExpression less2 = new LessThanExpression(colExpression, new LiteralExpression(value, coltype));
+                                //less2.setRangeExpression(true);
+                                rangeFilter.add(less2);
+                                break;
+                            default:
+                                throw new AssertionError("Unhandled bound: " + range.getHigh().getBound());
+                        }
+                    }
+                }
+            }
+
+            if (singleValues.size() == 1) {
+                Expression ex = null;
+                if (coltype.equals(DataType.STRING)) {
+                    ex = new EqualToExpression(colExpression, new LiteralExpression(((Slice) singleValues.get(0)).toStringUtf8(), coltype));
+                } else
+                    ex = new EqualToExpression(colExpression, new LiteralExpression(singleValues.get(0), coltype));
+                filters.add(ex);
+            }
+            else if(singleValues.size() > 1) {
+                ListExpression candidates = null;
+                List<Expression> exs = singleValues.stream().map((a) ->
+                {
+                    return new LiteralExpression(ConvertDataByType(a, type), coltype);
+                }).collect(Collectors.toList());
+                candidates = new ListExpression(exs);
+
+                if(candidates != null)
+                    filters.add(new InExpression(colExpression, candidates));
+            }
+            else if(rangeFilter.size() > 0){
+                if(rangeFilter.size() > 1) {
+                    Expression finalFilters = new OrExpression(rangeFilter.get(0), rangeFilter.get(1));
+                    if(rangeFilter.size() > 2)
+                    {
+                        for(int i = 2; i< rangeFilter.size(); i++)
+                        {
+                            filters.add(new AndExpression(finalFilters, rangeFilter.get(i)));
+                        }
+                    }
+                }
+                else if(rangeFilter.size() == 1)//only have one value
+                    filters.add(rangeFilter.get(0));
+            }
+        }
+
+        Expression finalFilters;
+        List<Expression> tmp = filters.build();
+        if(tmp.size() > 1) {
+            finalFilters = new AndExpression(tmp.get(0), tmp.get(1));
+            if(tmp.size() > 2)
+            {
+                for(int i = 2; i< tmp.size(); i++)
+                {
+                    finalFilters = new AndExpression(finalFilters, tmp.get(i));
+                }
+            }
+        }
+        else if(tmp.size() == 1)
+            finalFilters = tmp.get(0);
+        else//no filter
+            return null;
+
+        return finalFilters;
+    }
+
+    public static DataType Spi2CarbondataTypeMapper(Type colType)
+    {
+        if(colType == BooleanType.BOOLEAN)
+            return DataType.BOOLEAN;
+        else if(colType == SmallintType.SMALLINT)
+            return DataType.SHORT;
+        else if(colType == IntegerType.INTEGER)
+            return DataType.INT;
+        else if(colType == BigintType.BIGINT)
+            return DataType.LONG;
+        else if(colType == DoubleType.DOUBLE)
+            return DataType.DOUBLE;
+        else if(colType == DecimalType.createDecimalType())
+            return DataType.DECIMAL;
+        else if(colType == VarcharType.VARCHAR)
+            return DataType.STRING;
+        else if(colType == DateType.DATE)
+            return DataType.DATE;
+        else if(colType == TimestampType.TIMESTAMP)
+            return DataType.TIMESTAMP;
+        else
+            return DataType.STRING;
+    }
+
+
+    public Object ConvertDataByType(Object rawdata, Type type)
+    {
+        if(type.equals(IntegerType.INTEGER))
+            return new Integer((rawdata.toString()));
+        else if(type.equals(BigintType.BIGINT))
+            return (Long)rawdata;
+        else if(type.equals(VarcharType.VARCHAR))
+            return ((Slice)rawdata).toStringUtf8();
+        else if(type.equals(BooleanType.BOOLEAN))
+            return (Boolean)(rawdata);
+
+        return rawdata;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/32bf2961/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataTableHandle.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataTableHandle.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataTableHandle.java
new file mode 100755
index 0000000..b0caf52
--- /dev/null
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataTableHandle.java
@@ -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.carbondata.presto;
+
+import com.facebook.presto.spi.ConnectorTableHandle;
+import com.facebook.presto.spi.SchemaTableName;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Joiner;
+
+import java.util.Objects;
+
+import static java.util.Locale.ENGLISH;
+import static java.util.Objects.requireNonNull;
+
+public class CarbondataTableHandle
+        implements ConnectorTableHandle {
+
+    private final String connectorId;
+    private final SchemaTableName schemaTableName;
+
+    @JsonCreator
+    public CarbondataTableHandle(
+            @JsonProperty("connectorId") String connectorId,
+            @JsonProperty("schemaTableName") SchemaTableName schemaTableName)
+    {
+        this.connectorId = requireNonNull(connectorId.toLowerCase(ENGLISH), "connectorId is null");
+        this.schemaTableName = schemaTableName;
+    }
+
+    @JsonProperty
+    public String getConnectorId()
+    {
+        return connectorId;
+    }
+
+    @JsonProperty
+    public SchemaTableName getSchemaTableName()
+    {
+        return schemaTableName;
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(connectorId, schemaTableName);
+    }
+
+    @Override
+    public boolean equals(Object obj)
+    {
+        if (this == obj) {
+            return true;
+        }
+        if ((obj == null) || (getClass() != obj.getClass())) {
+            return false;
+        }
+
+        CarbondataTableHandle other = (CarbondataTableHandle) obj;
+        return Objects.equals(this.connectorId, other.connectorId) && this.schemaTableName.equals(other.getSchemaTableName());
+    }
+
+    @Override
+    public String toString()
+    {
+        return Joiner.on(":").join(connectorId, schemaTableName.toString());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/32bf2961/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataTableLayoutHandle.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataTableLayoutHandle.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataTableLayoutHandle.java
new file mode 100755
index 0000000..bddf9de
--- /dev/null
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataTableLayoutHandle.java
@@ -0,0 +1,87 @@
+/*
+ * 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.presto;
+
+import com.facebook.presto.spi.ColumnHandle;
+import com.facebook.presto.spi.ConnectorTableLayoutHandle;
+import com.facebook.presto.spi.predicate.TupleDomain;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.Objects;
+
+import static com.google.common.base.MoreObjects.toStringHelper;
+import static java.util.Objects.requireNonNull;
+
+public class CarbondataTableLayoutHandle
+        implements ConnectorTableLayoutHandle
+{
+    private final CarbondataTableHandle table;
+    private final TupleDomain<ColumnHandle> constraint;
+
+    @JsonCreator
+    public CarbondataTableLayoutHandle(@JsonProperty("table") CarbondataTableHandle table,
+                                       @JsonProperty("constraint") TupleDomain<ColumnHandle> constraint)
+    {
+        this.table = requireNonNull(table, "table is null");
+        this.constraint = requireNonNull(constraint, "constraint is null");
+    }
+
+    @JsonProperty
+    public CarbondataTableHandle getTable()
+    {
+        return table;
+    }
+
+    @JsonProperty
+    public TupleDomain<ColumnHandle> getConstraint()
+    {
+        return constraint;
+    }
+
+    @Override
+    public boolean equals(Object obj)
+    {
+        if (this == obj) {
+            return true;
+        }
+
+        if (obj == null || getClass() != obj.getClass()) {
+            return false;
+        }
+
+        CarbondataTableLayoutHandle other = (CarbondataTableLayoutHandle) obj;
+        return Objects.equals(table, other.table)
+                && Objects.equals(constraint, other.constraint);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(table, constraint);
+    }
+
+    @Override
+    public String toString()
+    {
+        return toStringHelper(this)
+                .add("table", table)
+                .add("constraint", constraint)
+                .toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/32bf2961/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataTransactionHandle.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataTransactionHandle.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataTransactionHandle.java
new file mode 100755
index 0000000..06a84e2
--- /dev/null
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataTransactionHandle.java
@@ -0,0 +1,26 @@
+/*
+ * 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.presto;
+
+import com.facebook.presto.spi.connector.ConnectorTransactionHandle;
+
+public enum CarbondataTransactionHandle
+        implements ConnectorTransactionHandle
+{
+    INSTANCE
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/32bf2961/integration/presto/src/main/java/org/apache/carbondata/presto/Types.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/Types.java b/integration/presto/src/main/java/org/apache/carbondata/presto/Types.java
new file mode 100755
index 0000000..b7b0d90
--- /dev/null
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/Types.java
@@ -0,0 +1,38 @@
+/*
+ * 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.presto;
+
+import java.util.Locale;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
+
+public class Types {
+    private Types() {}
+
+    public static <A, B extends A> B checkType(A value, Class<B> target, String name)
+    {
+        requireNonNull(value, String.format(Locale.ENGLISH, "%s is null", name));
+        checkArgument(target.isInstance(value),
+                "%s must be of type %s, not %s",
+                name,
+                target.getName(),
+                value.getClass().getName());
+        return target.cast(value);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/32bf2961/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonLocalInputSplit.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonLocalInputSplit.java b/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonLocalInputSplit.java
new file mode 100755
index 0000000..9cde7a6
--- /dev/null
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonLocalInputSplit.java
@@ -0,0 +1,93 @@
+/*
+ * 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.presto.impl;
+
+import java.util.List;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public class CarbonLocalInputSplit {
+
+    private static final long serialVersionUID = 3520344046772190207L;
+    private String segmentId;
+    private String path;
+    private long start;
+    private long length;
+    private List<String> locations;
+    private short version;
+    /**
+     * Number of BlockLets in a block
+     */
+    private int numberOfBlocklets = 0;
+
+
+    @JsonProperty
+    public short getVersion(){
+        return version;
+    }
+
+    @JsonProperty
+    public List<String> getLocations() {
+        return locations;
+    }
+
+    @JsonProperty
+    public long getLength() {
+        return length;
+    }
+
+    @JsonProperty
+    public long getStart() {
+        return start;
+    }
+
+    @JsonProperty
+    public String getPath() {
+        return path;
+    }
+
+    @JsonProperty
+    public String getSegmentId() {
+        return segmentId;
+    }
+
+    @JsonProperty
+    public int getNumberOfBlocklets() {
+        return numberOfBlocklets;
+    }
+
+    @JsonCreator
+    public CarbonLocalInputSplit(@JsonProperty("segmentId") String segmentId,
+                                 @JsonProperty("path") String path,
+                                 @JsonProperty("start") long start,
+                                 @JsonProperty("length") long length,
+                                 @JsonProperty("locations") List<String> locations,
+                                 @JsonProperty("numberOfBlocklets") int numberOfBlocklets/*,
+                                 @JsonProperty("tableBlockInfo") TableBlockInfo tableBlockInfo*/,
+                                 @JsonProperty("version") short version) {
+        this.path = path;
+        this.start = start;
+        this.length = length;
+        this.segmentId = segmentId;
+        this.locations = locations;
+        this.numberOfBlocklets = numberOfBlocklets;
+        //this.tableBlockInfo = tableBlockInfo;
+        this.version = version;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/32bf2961/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableCacheModel.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableCacheModel.java b/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableCacheModel.java
new file mode 100755
index 0000000..ee636b1
--- /dev/null
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableCacheModel.java
@@ -0,0 +1,44 @@
+/*
+ * 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.presto.impl;
+
+import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.TableInfo;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+
+public class CarbonTableCacheModel {
+
+    public CarbonTableIdentifier carbonTableIdentifier;
+    public CarbonTablePath carbonTablePath;
+
+    public TableInfo tableInfo;
+    public CarbonTable carbonTable;
+    public String[] segments;
+
+    public boolean isValid()
+    {
+        if(carbonTable != null
+                && carbonTablePath != null
+                && carbonTableIdentifier != null)
+            return true;
+        else
+            return false;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/32bf2961/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableConfig.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableConfig.java b/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableConfig.java
new file mode 100755
index 0000000..a682b66
--- /dev/null
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableConfig.java
@@ -0,0 +1,62 @@
+/*
+ * 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.presto.impl;
+
+import io.airlift.configuration.Config;
+
+import javax.validation.constraints.NotNull;
+
+public class CarbonTableConfig {
+    //read from config
+    private String dbPtah;
+    private String tablePath;
+    private String storePath;
+
+    @NotNull
+    public String getDbPtah() {
+        return dbPtah;
+    }
+
+    @Config("carbondata-store")
+    public CarbonTableConfig setDbPtah(String dbPtah) {
+        this.dbPtah = dbPtah;
+        return this;
+    }
+
+    @NotNull
+    public String getTablePath() {
+        return tablePath;
+    }
+
+    @Config("carbondata-store")
+    public CarbonTableConfig setTablePath(String tablePath) {
+        this.tablePath = tablePath;
+        return this;
+    }
+
+    @NotNull
+    public String getStorePath() {
+        return storePath;
+    }
+
+    @Config("carbondata-store")
+    public CarbonTableConfig setStorePath(String storePath) {
+        this.storePath = storePath;
+        return this;
+    }
+}