You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by GitBox <gi...@apache.org> on 2020/10/27 09:55:44 UTC

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3916: [CARBONDATA-3935]Support partition table transactional write in presto

ajantha-bhat commented on a change in pull request #3916:
URL: https://github.com/apache/carbondata/pull/3916#discussion_r512505051



##########
File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##########
@@ -369,7 +369,8 @@ private CarbonCommonConstants() {
   public static final String CARBON_MERGE_INDEX_IN_SEGMENT =
       "carbon.merge.index.in.segment";
 
-  public static final String CARBON_MERGE_INDEX_IN_SEGMENT_DEFAULT = "true";
+  // TODO: revert this after proper fix in this PR

Review comment:
       please revert this

##########
File path: integration/hive/src/main/java/org/apache/carbondata/hive/MapredCarbonOutputCommitter.java
##########
@@ -123,13 +132,46 @@ public void commitJob(JobContext jobContext) throws IOException {
     try {
       Configuration configuration = jobContext.getConfiguration();
       CarbonLoadModel carbonLoadModel = MapredCarbonOutputFormat.getLoadModel(configuration);
-      ThreadLocalSessionInfo.unsetAll();
-      SegmentFileStore.writeSegmentFile(carbonLoadModel.getCarbonDataLoadSchema().getCarbonTable(),
-          carbonLoadModel.getSegmentId(), String.valueOf(carbonLoadModel.getFactTimeStamp()));
-      SegmentFileStore
-          .mergeIndexAndWriteSegmentFile(carbonLoadModel.getCarbonDataLoadSchema().getCarbonTable(),
-              carbonLoadModel.getSegmentId(), String.valueOf(carbonLoadModel.getFactTimeStamp()));
-      CarbonTableOutputFormat.setLoadModel(configuration, carbonLoadModel);
+      if (!carbonLoadModel.getCarbonDataLoadSchema().getCarbonTable().isHivePartitionTable()) {
+        ThreadLocalSessionInfo.unsetAll();
+        SegmentFileStore
+            .writeSegmentFile(carbonLoadModel.getCarbonDataLoadSchema().getCarbonTable(),
+                carbonLoadModel.getSegmentId(), String.valueOf(carbonLoadModel.getFactTimeStamp()));
+        SegmentFileStore.mergeIndexAndWriteSegmentFile(
+            carbonLoadModel.getCarbonDataLoadSchema().getCarbonTable(),
+            carbonLoadModel.getSegmentId(), String.valueOf(carbonLoadModel.getFactTimeStamp()));
+        CarbonTableOutputFormat.setLoadModel(configuration, carbonLoadModel);
+      } else {
+        String tableFactLocation =
+            carbonLoadModel.getCarbonDataLoadSchema().getCarbonTable().getTablePath();
+        List<CarbonFile> carbonFiles =
+            FileFactory.getCarbonFile(tableFactLocation).listFiles(true, new CarbonFileFilter() {

Review comment:
       list files of whole table can be very slow when multiple segments are present. we list previous load segments also here. I think we need to keep list of index/merge index created for current load in memory and write in the segment file here.

##########
File path: integration/hive/src/main/java/org/apache/carbondata/hive/MapredCarbonOutputFormat.java
##########
@@ -115,9 +115,20 @@ public void checkOutputSpecs(FileSystem fileSystem, JobConf jobConf) throws IOEx
         carbonLoadModel.getCarbonDataLoadSchema().getCarbonTable().getPartitionInfo();
     final int partitionColumn =
         partitionInfo != null ? partitionInfo.getColumnSchemaList().size() : 0;
+    final String finalOutputPath;
     if (carbonLoadModel.getCarbonDataLoadSchema().getCarbonTable().isHivePartitionTable()) {
-      carbonLoadModel.getMetrics().addToPartitionPath(finalOutPath.toString());
-      context.getConfiguration().set("carbon.outputformat.writepath", finalOutPath.toString());
+      String[] outputPathSplits = finalOutPath.toString().split("/");
+      StringBuilder partitionDirs = new StringBuilder();
+      for (int i = partitionColumn; i > 0;  i--) {

Review comment:
       so, how carbondata-hive partition write is working ?  

##########
File path: integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoInsertIntoPartitionTableTestCase.scala
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.integrationtest
+
+import java.io.File
+import java.util
+import java.util.concurrent.atomic.AtomicInteger
+
+import scala.collection.JavaConverters._
+
+import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuiteLike}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.filesystem.{CarbonFile, CarbonFileFilter}
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.metadata.datatype.{DataTypes, StructField}
+import org.apache.carbondata.core.metadata.schema.{PartitionInfo, SchemaReader}
+import org.apache.carbondata.core.metadata.schema.partition.PartitionType
+import org.apache.carbondata.core.metadata.schema.table.TableSchemaBuilder
+import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
+import org.apache.carbondata.core.statusmanager.SegmentStatusManager
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.presto.server.PrestoServer
+
+/**
+ * Tests for partition tables transational write in presto
+ */
+class PrestoInsertIntoPartitionTableTestCase
+  extends FunSuiteLike with BeforeAndAfterAll with BeforeAndAfterEach {
+
+  private val logger = LogServiceFactory
+    .getLogService(classOf[PrestoAllDataTypeTest].getCanonicalName)
+
+  private val rootPath = new File(this.getClass.getResource("/").getPath
+                                  + "../../../..").getCanonicalPath
+  private val storePath = s"$rootPath/integration/presto/target/store"
+  private val prestoServer = new PrestoServer
+
+  override def beforeAll: Unit = {
+    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.CARBON_WRITTEN_BY_APPNAME,

Review comment:
       remove written by, it has to take care internally




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org