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

carbondata git commit: [CARBONDATA-2956] CarbonReader support use configuration to read S3 data

Repository: carbondata
Updated Branches:
  refs/heads/master 0ad90820a -> e3eb03054


[CARBONDATA-2956] CarbonReader support use configuration to read S3 data

CarbonReader support use configuration to read S3 data
1.with filter
2.without filter function

This closes #2742


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

Branch: refs/heads/master
Commit: e3eb030548fed48d92a363940bd56ac37700e53c
Parents: 0ad9082
Author: xubo245 <xu...@huawei.com>
Authored: Fri Sep 21 12:36:01 2018 +0800
Committer: kunal642 <ku...@gmail.com>
Committed: Mon Sep 24 17:32:45 2018 +0530

----------------------------------------------------------------------
 .../core/metadata/schema/table/CarbonTable.java |  7 +-
 .../LatestFilesReadCommittedScope.java          |  2 +-
 .../examples/sdk/SDKS3ReadExample.java          | 93 ++++++++++++++++++++
 .../sdk/file/CarbonReaderBuilder.java           |  2 +-
 4 files changed, 100 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/e3eb0305/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
index 3d04cca..e57605a 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
@@ -63,6 +63,8 @@ import org.apache.carbondata.core.util.path.CarbonTablePath;
 import static org.apache.carbondata.core.metadata.schema.datamap.DataMapClassProvider.MV;
 import static org.apache.carbondata.core.util.CarbonUtil.thriftColumnSchemaToWrapperColumnSchema;
 
+import org.apache.hadoop.conf.Configuration;
+
 /**
  * Mapping class for Carbon actual table
  */
@@ -236,9 +238,10 @@ public class CarbonTable implements Serializable {
 
   public static CarbonTable buildTable(
       String tablePath,
-      String tableName) throws IOException {
+      String tableName,
+      Configuration configuration) throws IOException {
     TableInfo tableInfoInfer = CarbonUtil.buildDummyTableInfo(tablePath, "null", "null");
-    CarbonFile carbonFile = getFirstIndexFile(FileFactory.getCarbonFile(tablePath));
+    CarbonFile carbonFile = getFirstIndexFile(FileFactory.getCarbonFile(tablePath, configuration));
     if (carbonFile == null) {
       throw new RuntimeException("Carbon index file not exists.");
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e3eb0305/core/src/main/java/org/apache/carbondata/core/readcommitter/LatestFilesReadCommittedScope.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/readcommitter/LatestFilesReadCommittedScope.java b/core/src/main/java/org/apache/carbondata/core/readcommitter/LatestFilesReadCommittedScope.java
index 9dafed9..7817cc8 100644
--- a/core/src/main/java/org/apache/carbondata/core/readcommitter/LatestFilesReadCommittedScope.java
+++ b/core/src/main/java/org/apache/carbondata/core/readcommitter/LatestFilesReadCommittedScope.java
@@ -178,7 +178,7 @@ public class LatestFilesReadCommittedScope implements ReadCommittedScope {
 
   @Override public void takeCarbonIndexFileSnapShot() throws IOException {
     // Read the current file Path get the list of indexes from the path.
-    CarbonFile file = FileFactory.getCarbonFile(carbonFilePath);
+    CarbonFile file = FileFactory.getCarbonFile(carbonFilePath, configuration);
 
     CarbonFile[] carbonIndexFiles = null;
     if (file.isDirectory()) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e3eb0305/examples/spark2/src/main/java/org/apache/carbondata/examples/sdk/SDKS3ReadExample.java
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/java/org/apache/carbondata/examples/sdk/SDKS3ReadExample.java b/examples/spark2/src/main/java/org/apache/carbondata/examples/sdk/SDKS3ReadExample.java
new file mode 100644
index 0000000..a236175
--- /dev/null
+++ b/examples/spark2/src/main/java/org/apache/carbondata/examples/sdk/SDKS3ReadExample.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.examples.sdk;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.scan.expression.ColumnExpression;
+import org.apache.carbondata.core.scan.expression.LiteralExpression;
+import org.apache.carbondata.core.scan.expression.conditional.EqualToExpression;
+import org.apache.carbondata.sdk.file.*;
+
+import org.apache.hadoop.conf.Configuration;
+
+import static org.apache.hadoop.fs.s3a.Constants.ACCESS_KEY;
+import static org.apache.hadoop.fs.s3a.Constants.ENDPOINT;
+import static org.apache.hadoop.fs.s3a.Constants.SECRET_KEY;
+
+/**
+ * Example for testing carbonReader on S3
+ */
+public class SDKS3ReadExample {
+    public static void main(String[] args) throws Exception {
+        LogService logger = LogServiceFactory.getLogService(SDKS3ReadExample.class.getName());
+        if (args == null || args.length < 3) {
+            logger.error("Usage: java CarbonS3Example: <access-key> <secret-key>"
+                + "<s3-endpoint> [table-path-on-s3]");
+            System.exit(0);
+        }
+
+        String path = "s3a://sdk/WriterOutput";
+        if (args.length > 3) {
+            path=args[3];
+        }
+
+        // Read data
+
+        EqualToExpression equalToExpression = new EqualToExpression(
+            new ColumnExpression("name", DataTypes.STRING),
+            new LiteralExpression("robot1", DataTypes.STRING));
+
+        Configuration configuration = new Configuration();
+        configuration.set(ACCESS_KEY, args[0]);
+        configuration.set(SECRET_KEY, args[1]);
+        configuration.set(ENDPOINT, args[2]);
+        CarbonReader reader = CarbonReader
+            .builder(path, "_temp")
+            .projection(new String[]{"name", "age"})
+            .filter(equalToExpression)
+            .build(configuration);
+
+        System.out.println("\nData:");
+        int i = 0;
+        while (i < 20 && reader.hasNext()) {
+            Object[] row = (Object[]) reader.readNextRow();
+            System.out.println(row[0] + " " + row[1]);
+            i++;
+        }
+        System.out.println("\nFinished");
+        reader.close();
+
+        // Read without filter
+        CarbonReader reader2 = CarbonReader
+            .builder(path, "_temp")
+            .projection(new String[]{"name", "age"})
+            .build(configuration);
+
+        System.out.println("\nData:");
+        i = 0;
+        while (i < 20 && reader2.hasNext()) {
+            Object[] row = (Object[]) reader2.readNextRow();
+            System.out.println(row[0] + " " + row[1]);
+            i++;
+        }
+        System.out.println("\nFinished");
+        reader2.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e3eb0305/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReaderBuilder.java
----------------------------------------------------------------------
diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReaderBuilder.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReaderBuilder.java
index 58bf3ab..15066ce 100644
--- a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReaderBuilder.java
+++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReaderBuilder.java
@@ -188,7 +188,7 @@ public class CarbonReaderBuilder {
           .buildFromTablePath(tableName, "default", tablePath, UUID.randomUUID().toString());
     } else {
       if (filterExpression != null) {
-        table = CarbonTable.buildTable(tablePath, tableName);
+        table = CarbonTable.buildTable(tablePath, tableName, configuration);
       } else {
         table = CarbonTable.buildDummyTable(tablePath);
       }