[GitHub] [carbondata] niuge01 opened a new pull request #3661: [WIP] Support materialized view

classic Classic list List threaded Threaded
227 messages Options
1 ... 45678910 ... 12
Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view

GitBox
Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396392817
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/index/CarbonCreateIndexCommand.scala
 ##########
 @@ -0,0 +1,195 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.index
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.execution.command._
+import org.apache.spark.sql.secondaryindex.command.IndexModel
+
+import org.apache.carbondata.common.exceptions.sql.{MalformedCarbonCommandException, MalformedIndexCommandException, NoSuchIndexException}
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datamap.DataMapStoreManager
+import org.apache.carbondata.core.datamap.status.DataMapStatusManager
+import org.apache.carbondata.core.metadata.ColumnarFormatVersion
+import org.apache.carbondata.core.metadata.datatype.DataTypes
+import org.apache.carbondata.core.metadata.schema.datamap.{DataMapClassProvider, DataMapProperty}
+import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, DataMapSchema}
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
+import org.apache.carbondata.datamap.IndexProvider
+import org.apache.carbondata.events._
+
+/**
+ * Below command class will be used to create index on table
+ * and updating the parent table about the index information
+ */
+case class CarbonCreateIndexCommand(
+    indexModel: IndexModel,
+    indexProviderName: String,
+    properties: Map[String, String],
+    ifNotExistsSet: Boolean = false,
+    var deferredRebuild: Boolean = false)
+  extends AtomicRunnableCommand {
+
+  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
+  private var provider: IndexProvider = _
+  private var parentTable: CarbonTable = _
+  private var dataMapSchema: DataMapSchema = _
+
+  override def processMetadata(sparkSession: SparkSession): Seq[Row] = {
+    // since streaming segment does not support building index yet,
+    // so streaming table does not support create index
+    parentTable = CarbonEnv.getCarbonTable(indexModel.dbName, indexModel.tableName)(sparkSession)
+    val indexName = indexModel.indexName
+
+    setAuditTable(parentTable)
+    setAuditInfo(Map("provider" -> indexProviderName, "indexName" -> indexName) ++ properties)
+
+    if (!parentTable.getTableInfo.isTransactionalTable) {
+      throw new MalformedCarbonCommandException("Unsupported operation on non transactional table")
+    }
+
+    if (DataMapStoreManager.getInstance().isDataMapExist(parentTable.getTableId, indexName)) {
+      if (!ifNotExistsSet) {
+        throw new NoSuchIndexException(indexName)
 
 Review comment:
   should throw index exists exception

----------------------------------------------------------------
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:
[hidden email]


With regards,
Apache Git Services
Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

GitBox
In reply to this post by GitBox
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-602567797
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2543/
   

----------------------------------------------------------------
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:
[hidden email]


With regards,
Apache Git Services
Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

GitBox
In reply to this post by GitBox
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-602569626
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/836/
   

----------------------------------------------------------------
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:
[hidden email]


With regards,
Apache Git Services
Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

GitBox
In reply to this post by GitBox
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-603005146
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/837/
   

----------------------------------------------------------------
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:
[hidden email]


With regards,
Apache Git Services
Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

GitBox
In reply to this post by GitBox
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-603018451
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2544/
   

----------------------------------------------------------------
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:
[hidden email]


With regards,
Apache Git Services
Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

GitBox
In reply to this post by GitBox
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396953401
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/IndexUtil.java
 ##########
 @@ -185,38 +185,37 @@ public static void pruneSegments(List<Segment> segments, List<ExtendedBlocklet>
    Loads the datamaps in parallel by utilizing executor
    *
    @param carbonTable
-   @param dataMapExprWrapper
+   @param indexExprWrapper
    @param validSegments
    @param partitionsToPrune
    @throws IOException
    */
-  public static void loadDataMaps(CarbonTable carbonTable, DataMapExprWrapper dataMapExprWrapper,
+  public static void loadDataMaps(CarbonTable carbonTable, IndexExprWrapper indexExprWrapper,
       List<Segment> validSegments, List<PartitionSpec> partitionsToPrune) throws IOException {
     if (!CarbonProperties.getInstance()
         .isDistributedPruningEnabled(carbonTable.getDatabaseName(), carbonTable.getTableName())
-        && BlockletDataMapUtil.loadDataMapsParallel(carbonTable)) {
-      String clsName = "org.apache.spark.sql.secondaryindex.Jobs.SparkBlockletDataMapLoaderJob";
-      DataMapJob dataMapJob = (DataMapJob) createDataMapJob(clsName);
-      String className =
-          "org.apache.spark.sql.secondaryindex.Jobs.DistributableBlockletDataMapLoader";
+        && BlockletIndexUtil.loadDataMapsParallel(carbonTable)) {
+      String clsName = "org.apache.carbondata.spark.rdd.SparkBlockletDataMapLoaderJob";
 
 Review comment:
   OK

----------------------------------------------------------------
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:
[hidden email]


With regards,
Apache Git Services
Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

GitBox
In reply to this post by GitBox
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396955981
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapSyncStatus.java
 ##########
 @@ -36,7 +36,7 @@
 /**
  * Interface to check whether datamap can be enabled
  */
-@InterfaceAudience.Developer("DataMap")
+@InterfaceAudience.Developer("Index")
 
 Review comment:
   This class will be delete when clean the old mv implementation

----------------------------------------------------------------
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:
[hidden email]


With regards,
Apache Git Services
Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

GitBox
In reply to this post by GitBox
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396956769
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/status/MVSegmentStatusUtil.java
 ##########
 @@ -32,7 +32,7 @@
 /**
  * Utility class to get updated segment mapping for datamap table
  */
-public class DataMapSegmentStatusUtil {
+public class MVSegmentStatusUtil {
 
 Review comment:
   This method never used by new mv implementation, and will be delete when clean the old mv implementation

----------------------------------------------------------------
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:
[hidden email]


With regards,
Apache Git Services
Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

GitBox
In reply to this post by GitBox
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396957216
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/metadata/schema/datamap/DataMapClassProvider.java
 ##########
 @@ -18,26 +18,20 @@
 package org.apache.carbondata.core.metadata.schema.datamap;
 
 /**
- * type for create datamap
- * The syntax of datamap creation is as follows.
- * CREATE DATAMAP IF NOT EXISTS dataMapName ON TABLE tableName USING 'DataMapClassProvider'
- * DMPROPERTIES('KEY'='VALUE') AS SELECT COUNT(COL1) FROM tableName
- *
  * Please refer {{org.apache.spark.sql.parser.CarbonSpark2SqlParser}}
  */
-
 public enum DataMapClassProvider {
-  LUCENE("org.apache.carbondata.datamap.lucene.LuceneFineGrainDataMapFactory", "lucene"),
-  BLOOMFILTER("org.apache.carbondata.datamap.bloom.BloomCoarseGrainDataMapFactory", "bloomfilter"),
+  LUCENE("org.apache.carbondata.datamap.lucene.LuceneFineGrainIndexFactory", "lucene"),
+  BLOOMFILTER("org.apache.carbondata.datamap.bloom.BloomCoarseGrainIndexFactory", "bloomfilter"),
 
 Review comment:
   OK

----------------------------------------------------------------
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:
[hidden email]


With regards,
Apache Git Services
Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

GitBox
In reply to this post by GitBox
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396958744
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/view/MaterializedViewProvider.java
 ##########
 @@ -0,0 +1,573 @@
+/*
+ * 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.core.view;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.fileoperations.AtomicFileOperationFactory;
+import org.apache.carbondata.core.fileoperations.AtomicFileOperations;
+import org.apache.carbondata.core.fileoperations.FileWriteOperation;
+import org.apache.carbondata.core.locks.CarbonLockFactory;
+import org.apache.carbondata.core.locks.CarbonLockUtil;
+import org.apache.carbondata.core.locks.ICarbonLock;
+import org.apache.carbondata.core.locks.LockUsage;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.RelationIdentifier;
+import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
+import org.apache.carbondata.core.statusmanager.SegmentStatus;
+import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
+import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+
+import com.google.gson.Gson;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.log4j.Logger;
+
+/**
+ * Stores mv schema in disk as json format
+ */
+@InterfaceAudience.Internal
+public class MaterializedViewProvider {
+
+  private static final Logger LOG = LogServiceFactory.getLogService(
+      MaterializedViewProvider.class.getCanonicalName());
+
+  private static final String STATUS_FILE_NAME = "mv_status";
+
+  private final String storeLocation;
+
+  private final Map<String, SchemaProvider> schemaProviders = new ConcurrentHashMap<>();
+
+  private MaterializedViewProvider(String storeLocation) {
+    this.storeLocation = storeLocation;
+  }
+
+  public static MaterializedViewProvider get() {
+    String storeLocation =
+        CarbonProperties.getInstance().getProperty(CarbonCommonConstants.STORE_LOCATION);
+    if (storeLocation == null) {
+      throw new RuntimeException(
+          "Property [" + CarbonCommonConstants.STORE_LOCATION + "] is not set.");
+    }
+    return new MaterializedViewProvider(storeLocation);
+  }
+
+  private static String getSchemaPath(String schemaRoot, String viewName) {
+    return schemaRoot + CarbonCommonConstants.FILE_SEPARATOR + "mv_schema." + viewName;
+  }
+
+  private SchemaProvider getSchemaProvider(String databaseName) {
+    String databaseNameUpper = databaseName.toUpperCase();
+    SchemaProvider schemaProvider = this.schemaProviders.get(databaseNameUpper);
+    if (schemaProvider == null) {
+      synchronized (this.schemaProviders) {
+        schemaProvider = this.schemaProviders.get(databaseNameUpper);
+        if (schemaProvider == null) {
+          String databaseLocation;
+          if (databaseNameUpper.equalsIgnoreCase("default")) {
 
 Review comment:
   OK

----------------------------------------------------------------
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:
[hidden email]


With regards,
Apache Git Services
Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

GitBox
In reply to this post by GitBox
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396965321
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/view/MaterializedViewProvider.java
 ##########
 @@ -0,0 +1,573 @@
+/*
+ * 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.core.view;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.fileoperations.AtomicFileOperationFactory;
+import org.apache.carbondata.core.fileoperations.AtomicFileOperations;
+import org.apache.carbondata.core.fileoperations.FileWriteOperation;
+import org.apache.carbondata.core.locks.CarbonLockFactory;
+import org.apache.carbondata.core.locks.CarbonLockUtil;
+import org.apache.carbondata.core.locks.ICarbonLock;
+import org.apache.carbondata.core.locks.LockUsage;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.RelationIdentifier;
+import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
+import org.apache.carbondata.core.statusmanager.SegmentStatus;
+import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
+import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+
+import com.google.gson.Gson;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.log4j.Logger;
+
+/**
+ * Stores mv schema in disk as json format
+ */
+@InterfaceAudience.Internal
+public class MaterializedViewProvider {
+
+  private static final Logger LOG = LogServiceFactory.getLogService(
+      MaterializedViewProvider.class.getCanonicalName());
+
+  private static final String STATUS_FILE_NAME = "mv_status";
+
+  private final String storeLocation;
+
+  private final Map<String, SchemaProvider> schemaProviders = new ConcurrentHashMap<>();
+
+  private MaterializedViewProvider(String storeLocation) {
+    this.storeLocation = storeLocation;
+  }
+
+  public static MaterializedViewProvider get() {
+    String storeLocation =
+        CarbonProperties.getInstance().getProperty(CarbonCommonConstants.STORE_LOCATION);
+    if (storeLocation == null) {
+      throw new RuntimeException(
+          "Property [" + CarbonCommonConstants.STORE_LOCATION + "] is not set.");
+    }
+    return new MaterializedViewProvider(storeLocation);
+  }
+
+  private static String getSchemaPath(String schemaRoot, String viewName) {
+    return schemaRoot + CarbonCommonConstants.FILE_SEPARATOR + "mv_schema." + viewName;
+  }
+
+  private SchemaProvider getSchemaProvider(String databaseName) {
+    String databaseNameUpper = databaseName.toUpperCase();
+    SchemaProvider schemaProvider = this.schemaProviders.get(databaseNameUpper);
+    if (schemaProvider == null) {
+      synchronized (this.schemaProviders) {
+        schemaProvider = this.schemaProviders.get(databaseNameUpper);
+        if (schemaProvider == null) {
+          String databaseLocation;
+          if (databaseNameUpper.equalsIgnoreCase("default")) {
+            databaseLocation = CarbonUtil.checkAndAppendHDFSUrl(this.storeLocation);
+          } else {
+            databaseLocation = CarbonUtil.checkAndAppendHDFSUrl(this.storeLocation +
+                CarbonCommonConstants.FILE_SEPARATOR + databaseName + ".db");
+          }
+          if (!FileFactory.getCarbonFile(databaseLocation).exists()) {
+            return null;
+          }
+          schemaProvider = new SchemaProvider(databaseLocation);
+          this.schemaProviders.put(databaseNameUpper, schemaProvider);
+        }
+      }
+    }
+    return schemaProvider;
+  }
+
+  public MaterializedViewSchema getSchema(MaterializedViewManager viewManager,
+      String databaseName, String viewName) throws IOException {
+    SchemaProvider schemaProvider = this.getSchemaProvider(databaseName);
+    if (schemaProvider == null) {
+      return null;
+    }
+    return schemaProvider.retrieveSchema(viewManager, viewName);
+  }
+
+  List<MaterializedViewSchema> getSchemas(MaterializedViewManager viewManager,
+      String databaseName, CarbonTable carbonTable) throws IOException {
+    SchemaProvider schemaProvider = this.getSchemaProvider(databaseName);
+    if (schemaProvider == null) {
+      return Collections.emptyList();
+    } else {
+      return schemaProvider.retrieveSchemas(viewManager, carbonTable);
+    }
+  }
+
+  List<MaterializedViewSchema> getSchemas(MaterializedViewManager viewManager,
+      String databaseName) throws IOException {
+    SchemaProvider schemaProvider = this.getSchemaProvider(databaseName);
+    if (schemaProvider == null) {
+      return Collections.emptyList();
+    } else {
+      return schemaProvider.retrieveAllSchemas(viewManager);
+    }
+  }
+
+  void saveSchema(MaterializedViewManager viewManager, String databaseName,
+      MaterializedViewSchema viewSchema) throws IOException {
+    SchemaProvider schemaProvider = this.getSchemaProvider(databaseName);
+    if (schemaProvider == null) {
+      throw new IOException("Database [" + databaseName + "] is not found.");
+    }
+    schemaProvider.saveSchema(viewManager, viewSchema);
+  }
+
+  public void dropSchema(String databaseName, String viewName)
+      throws IOException {
+    SchemaProvider schemaProvider = this.getSchemaProvider(databaseName);
+    if (schemaProvider == null) {
+      throw new IOException("Materialized view with name " + databaseName + "." + viewName +
+          " does not exists in storage");
+    }
+    schemaProvider.dropSchema(viewName);
+  }
+
+  private String getStatusFileName(String databaseName) {
+    if (databaseName.equalsIgnoreCase("default")) {
+      return this.storeLocation +
+          CarbonCommonConstants.FILE_SEPARATOR + "_system" +
+          CarbonCommonConstants.FILE_SEPARATOR + STATUS_FILE_NAME;
+    } else {
+      return this.storeLocation +
+          CarbonCommonConstants.FILE_SEPARATOR + databaseName + ".db" +
+          CarbonCommonConstants.FILE_SEPARATOR + "_system" +
+          CarbonCommonConstants.FILE_SEPARATOR + STATUS_FILE_NAME;
+    }
+  }
+
+  public List<MaterializedViewStatusDetail> getStatusDetails(String databaseName)
+      throws IOException {
+    String statusPath = this.getStatusFileName(databaseName);
+    Gson gsonObjectToRead = new Gson();
+    DataInputStream dataInputStream = null;
+    BufferedReader buffReader = null;
+    InputStreamReader inStream = null;
+    MaterializedViewStatusDetail[] statusDetails;
+    try {
+      if (!FileFactory.isFileExist(statusPath)) {
+        return Collections.emptyList();
+      }
+      dataInputStream = FileFactory.getDataInputStream(statusPath);
+      inStream = new InputStreamReader(dataInputStream,
+          Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+      buffReader = new BufferedReader(inStream);
+      statusDetails = gsonObjectToRead.fromJson(buffReader,
+          MaterializedViewStatusDetail[].class);
+    } catch (IOException e) {
+      LOG.error("Failed to read datamap status", e);
+      throw e;
+    } finally {
+      CarbonUtil.closeStreams(buffReader, inStream, dataInputStream);
+    }
+    // if status details is null, return empty array
+    if (null == statusDetails) {
+      return Collections.emptyList();
+    }
+    return Arrays.asList(statusDetails);
+  }
+
+  private static ICarbonLock getStatusLock(String databaseName) {
+    return CarbonLockFactory
+        .getSystemLevelCarbonLockObj(
+            CarbonProperties.getInstance().getSystemFolderLocation(databaseName),
+            LockUsage.MATERIALIZED_VIEW_STATUS_LOCK);
+  }
+
+  /**
+   * Update or add the status of passed mvs with the given mv status. If the mv status
+   * given is enabled/disabled then updates/adds the mv, in case of drop it just removes it
+   * from the file.
+   * This method always overwrites the old file.
+   * @param schemaList schemas of which are need to be updated in mv status
+   * @param status  status to be updated for the mv schemas
+   */
+  public void updateStatus(List<MaterializedViewSchema> schemaList,
+      MaterializedViewStatus status) throws IOException {
+    if (schemaList == null || schemaList.size() == 0) {
+      // There is nothing to update
+      return;
+    }
+    final Map<String, List<MaterializedViewSchema>> schemasMapByDatabase = new HashMap<>();
+    for (MaterializedViewSchema schema : schemaList) {
+      String databaseName = schema.getIdentifier().getDatabaseName().toLowerCase();
+      List<MaterializedViewSchema> schemas = schemasMapByDatabase.get(databaseName);
+      if (schemas == null) {
+        schemas = new ArrayList<>();
+        schemasMapByDatabase.put(databaseName, schemas);
+      }
+      schemas.add(schema);
+    }
+    for (Map.Entry<String, List<MaterializedViewSchema>> entry : schemasMapByDatabase.entrySet()) {
+      this.updateStatus(entry.getKey(), entry.getValue(), status);
+    }
+  }
+
+  private void updateStatus(String databaseName, List<MaterializedViewSchema> schemaList,
+      MaterializedViewStatus status) throws IOException {
+    ICarbonLock carbonTableStatusLock = getStatusLock(databaseName);
+    boolean locked = false;
+    try {
+      locked = carbonTableStatusLock.lockWithRetries();
+      if (locked) {
+        LOG.info("Materialized view status lock has been successfully acquired.");
+        if (status == MaterializedViewStatus.ENABLED) {
+          // Enable mv only if mv tables and main table are in sync
+          if (!isViewCanBeEnabled(schemaList.get(0))) {
+            return;
+          }
+        }
+        List<MaterializedViewStatusDetail> statusDetailList =
+            new ArrayList<>(getStatusDetails(databaseName));
+        List<MaterializedViewStatusDetail> changedStatusDetails = new ArrayList<>();
+        List<MaterializedViewStatusDetail> newStatusDetails = new ArrayList<>();
+        for (MaterializedViewSchema schema : schemaList) {
+          boolean exists = false;
+          for (MaterializedViewStatusDetail statusDetail : statusDetailList) {
+            if (statusDetail.getIdentifier().equals(schema.getIdentifier())) {
+              statusDetail.setStatus(status);
+              changedStatusDetails.add(statusDetail);
+              exists = true;
+            }
+          }
+          if (!exists) {
+            newStatusDetails
+                .add(new MaterializedViewStatusDetail(schema.getIdentifier(),
+                    status));
+          }
+        }
+        // Add the newly added datamaps to the list.
+        if (newStatusDetails.size() > 0 &&
+            status != MaterializedViewStatus.DROPPED) {
+          statusDetailList.addAll(newStatusDetails);
+        }
+        // In case of dropped datamap, just remove from the list.
+        if (status == MaterializedViewStatus.DROPPED) {
+          statusDetailList.removeAll(changedStatusDetails);
+        }
+        writeLoadDetailsIntoFile(
+            this.getStatusFileName(databaseName),
+            statusDetailList.toArray(
+                new MaterializedViewStatusDetail[statusDetailList.size()]));
+      } else {
+        String errorMsg = "Upadating datamapstatus is failed due to another process taken the lock"
+            + " for updating it";
+        LOG.error(errorMsg);
+        throw new IOException(errorMsg + " Please try after some time.");
+      }
+    } finally {
+      if (locked) {
+        CarbonLockUtil.fileUnlock(carbonTableStatusLock, LockUsage.DATAMAP_STATUS_LOCK);
+      }
+    }
+  }
+
+  /**
+   * writes mv status details
+   */
+  private static void writeLoadDetailsIntoFile(String location,
+      MaterializedViewStatusDetail[] statusDetails) throws IOException {
+    FileFactory.touchFile(FileFactory.getCarbonFile(location),
+        new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL));
+    AtomicFileOperations fileWrite = AtomicFileOperationFactory.getAtomicFileOperations(location);
+    BufferedWriter brWriter = null;
+    DataOutputStream dataOutputStream = null;
+    Gson gsonObjectToWrite = new Gson();
+    // write the updated data into the mv status file.
+    try {
+      dataOutputStream = fileWrite.openForWrite(FileWriteOperation.OVERWRITE);
+      brWriter = new BufferedWriter(new OutputStreamWriter(dataOutputStream,
+          Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)));
+
+      String metadataInstance = gsonObjectToWrite.toJson(statusDetails);
+      brWriter.write(metadataInstance);
+    } catch (IOException ioe) {
+      LOG.error("Error message: " + ioe.getLocalizedMessage());
+      fileWrite.setFailed();
+      throw ioe;
+    } finally {
+      if (null != brWriter) {
+        brWriter.flush();
+      }
+      CarbonUtil.closeStreams(brWriter);
+      fileWrite.close();
+    }
+
+  }
+
+  /**
+   * This method checks if main table and mv table are synchronised or not. If synchronised
+   * return true to enable the mv
+   *
+   * @param schema of mv to be disabled or enabled
+   * @return flag to enable or disable mv
+   * @throws IOException
+   */
+  private static boolean isViewCanBeEnabled(MaterializedViewSchema schema)
+      throws IOException {
+    if (!schema.isRefreshIncremental()) {
+      return true;
+    }
+    boolean isViewCanBeEnabled = true;
+    String viewMetadataPath =
+        CarbonTablePath.getMetadataPath(schema.getIdentifier().getTablePath());
+    LoadMetadataDetails[] viewLoadMetadataDetails =
+        SegmentStatusManager.readLoadMetadata(viewMetadataPath);
+    Map<String, List<String>> viewSegmentMap = new HashMap<>();
+    for (LoadMetadataDetails loadMetadataDetail : viewLoadMetadataDetails) {
+      if (loadMetadataDetail.getSegmentStatus() == SegmentStatus.SUCCESS) {
+        Map<String, List<String>> segmentMap =
+            new Gson().fromJson(loadMetadataDetail.getExtraInfo(), Map.class);
+        if (viewSegmentMap.isEmpty()) {
+          viewSegmentMap.putAll(segmentMap);
+        } else {
+          for (Map.Entry<String, List<String>> entry : segmentMap.entrySet()) {
+            if (null != viewSegmentMap.get(entry.getKey())) {
+              viewSegmentMap.get(entry.getKey()).addAll(entry.getValue());
+            }
+          }
+        }
+      }
+    }
+    List<RelationIdentifier> associatedTables = schema.getAssociatedTables();
+    for (RelationIdentifier associatedTable : associatedTables) {
+      List<String> associatedTableSegmentList =
+          SegmentStatusManager.getValidSegmentList(associatedTable);
+      if (!associatedTableSegmentList.isEmpty()) {
+        if (viewSegmentMap.isEmpty()) {
+          isViewCanBeEnabled = false;
+        } else {
+          isViewCanBeEnabled = viewSegmentMap.get(
+              associatedTable.getDatabaseName() + CarbonCommonConstants.POINT +
+                  associatedTable.getTableName()).containsAll(associatedTableSegmentList);
+        }
+      }
+    }
+    return isViewCanBeEnabled;
+  }
+
+  /**
+   * Data map schema provider of a database.
+   */
+  private static final class SchemaProvider {
+
+    private String systemDirectory;
+
+    private String schemaIndexFilePath;
+
+    private long lastModifiedTime;
+
+    private Set<MaterializedViewSchema> schemas = new HashSet<>();
+
+    SchemaProvider(String databaseLocation) {
+      final String systemDirectory =
+          databaseLocation + CarbonCommonConstants.FILE_SEPARATOR + "_system";
+      this.systemDirectory = systemDirectory;
+      this.schemaIndexFilePath = systemDirectory + CarbonCommonConstants.FILE_SEPARATOR +
+          "mv_schema_index";
+    }
+
+    void saveSchema(MaterializedViewManager viewManager, MaterializedViewSchema viewSchema)
+        throws IOException {
+      BufferedWriter brWriter = null;
+      DataOutputStream dataOutputStream = null;
+      Gson gsonObjectToWrite = new Gson();
+      String schemaPath =
+          getSchemaPath(this.systemDirectory, viewSchema.getIdentifier().getTableName());
+      if (FileFactory.isFileExist(schemaPath)) {
 
 Review comment:
   MaterializedViewManager.createSchema is a public interface, so it should ensuring independence and integrity itself.

----------------------------------------------------------------
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:
[hidden email]


With regards,
Apache Git Services
Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

GitBox
In reply to this post by GitBox
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396965813
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
 ##########
 @@ -1011,14 +1015,25 @@ object CarbonDataRDDFactory {
       throw new Exception(errorMessage)
     } else {
       DataMapStatusManager.disableAllLazyDataMaps(carbonTable)
+      val viewManager = MaterializedViewManagerInSpark.get(session)
+      val viewSchemas = new util.ArrayList[MaterializedViewSchema]()
+      for (viewSchema <- viewManager.getSchemasOnTable(carbonTable).asScala) {
+        if (viewSchema.isRefreshOnManual) {
+          viewSchemas.add(viewSchema)
+        }
+      }
+      viewManager.setStatus(viewSchemas, MaterializedViewStatus.DISABLED)
       if (overwriteTable) {
         val allDataMapSchemas = DataMapStoreManager.getInstance
           .getDataMapSchemasOfTable(carbonTable).asScala
           .filter(dataMapSchema => null != dataMapSchema.getRelationIdentifier &&
-                                   !dataMapSchema.isIndexDataMap).asJava
+                                   !dataMapSchema.isIndex).asJava
 
 Review comment:
   Will be delete when clean the old mv implementation

----------------------------------------------------------------
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:
[hidden email]


With regards,
Apache Git Services
Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

GitBox
In reply to this post by GitBox
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396965973
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/carbondata/view/MaterializedViewRefresher.scala
 ##########
 @@ -0,0 +1,398 @@
+/*
+ * 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.view
+
+import java._
+import java.io.IOException
+
+import scala.collection.JavaConverters._
+
+import com.google.gson.Gson
+import org.apache.log4j.Logger
+import org.apache.spark.sql.{CarbonUtils, SparkSession}
+import org.apache.spark.sql.execution.command.management.CarbonInsertIntoCommand
+import org.apache.spark.sql.parser.MaterializedViewQueryParser
+
+import org.apache.carbondata.common.exceptions.sql.NoSuchMaterializedViewException
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.locks.ICarbonLock
+import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, RelationIdentifier}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatus, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.core.view.{MaterializedViewSchema, MaterializedViewStatus}
+import org.apache.carbondata.processing.util.CarbonLoaderUtil
+
+class MaterializedViewRefresher{
+
+}
+
+object MaterializedViewRefresher {
+
+  private val LOGGER: Logger = LogServiceFactory.getLogService(
+    classOf[MaterializedViewRefresher].getCanonicalName)
+
+  /**
+   * Refresh the mv by loading all existing data from associated table
+   * This is called when refreshing the mv when
+   * 1. after mv creation and no "WITH DEFERRED REBUILD" defined
+   * 2. user manually trigger REFRESH MATERIALIZED VIEW command
+   */
+  @throws[IOException]
+  @throws[NoSuchMaterializedViewException]
+  def refresh(viewSchema: MaterializedViewSchema, session: SparkSession): Boolean = {
+    var newLoadName: String = ""
+    var segmentMap: String = ""
+    val viewTable: CarbonTable = CarbonTable.buildFromTablePath(
+      viewSchema.getIdentifier.getTableName,
+      viewSchema.getIdentifier.getDatabaseName,
+      viewSchema.getIdentifier.getTablePath,
+      viewSchema.getIdentifier.getTableId)
+    val viewIdentifier = viewSchema.getIdentifier
+    val viewTableIdentifier = viewTable.getAbsoluteTableIdentifier
+    // Clean up the old invalid segment data before creating a new entry for new load.
+    SegmentStatusManager.deleteLoadsAndUpdateMetadata(viewTable, false, null)
+    val segmentStatusManager: SegmentStatusManager = new SegmentStatusManager(viewTableIdentifier)
+    // Acquire table status lock to handle concurrent dataloading
+    val lock: ICarbonLock = segmentStatusManager.getTableStatusLock
+    val segmentMapping: util.Map[String, util.List[String]] =
+      new util.HashMap[String, util.List[String]]
+    val viewManager = MaterializedViewManagerInSpark.get(session)
+    try if (lock.lockWithRetries) {
+      LOGGER.info("Acquired lock for mv " + viewIdentifier + " for table status updation")
+      val viewTableMetadataPath: String =
+        CarbonTablePath.getMetadataPath(viewIdentifier.getTablePath)
+      val loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTableMetadataPath)
+      val loadMetadataDetailList: util.List[LoadMetadataDetails] =
+        new util.ArrayList[LoadMetadataDetails](CarbonCommonConstants.DEFAULT_COLLECTION_SIZE)
+      // Mark for delete all stale loadMetadetail
+      for (loadMetadataDetail <- loadMetadataDetails) {
+        if (((loadMetadataDetail.getSegmentStatus eq SegmentStatus.INSERT_IN_PROGRESS) ||
+             (loadMetadataDetail.getSegmentStatus eq SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS)) &&
+            loadMetadataDetail.getVisibility.equalsIgnoreCase("false")) {
+          loadMetadataDetail.setSegmentStatus(SegmentStatus.MARKED_FOR_DELETE)
+        }
+        loadMetadataDetailList.add(loadMetadataDetail)
+      }
+      if (viewSchema.isRefreshOnManual) {
+        // check if rebuild to mv is already in progress and throw exception
+        if (loadMetadataDetails.nonEmpty) {
+          for (loadMetaDetail <- loadMetadataDetails) {
+            if (((loadMetaDetail.getSegmentStatus eq SegmentStatus.INSERT_IN_PROGRESS) ||
+                 (loadMetaDetail.getSegmentStatus eq SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS)) &&
+                SegmentStatusManager.isLoadInProgress(viewTableIdentifier,
+                  loadMetaDetail.getLoadName)) {
+              throw new RuntimeException(
+                "Rebuild to materialized view " + viewSchema.getIdentifier.getTableName +
+                  " is already in progress")
+            }
+          }
+        }
+      }
+      if (viewSchema.isRefreshIncremental) {
+        if (!getSpecificSegmentsTobeLoaded(viewSchema, segmentMapping, loadMetadataDetailList)) {
+          return false
+        }
+      } else {
+          // set segment mapping only for carbondata table
+          val associatedTableIds =
+            viewSchema.getAssociatedTables.asScala.filter(_.isCarbonDataTable)
+          for (associatedTableId <- associatedTableIds) {
+            val associatedTableSegmentList: util.List[String] =
+              SegmentStatusManager.getValidSegmentList(associatedTableId)
+            if (associatedTableSegmentList.isEmpty) {
+              return false
+            }
+            segmentMapping.put(associatedTableId.toString, associatedTableSegmentList)
+          }
+        }
+      segmentMap = new Gson().toJson(segmentMapping)
+      // To handle concurrent dataloading to mv, create new loadMetaEntry and
+      // set segmentMap to new loadMetaEntry and pass new segmentId with load command
+      val loadMetadataDetail: LoadMetadataDetails = new LoadMetadataDetails
+      val segmentId: String = String.valueOf(
+        SegmentStatusManager.createNewSegmentId(loadMetadataDetails))
+      loadMetadataDetail.setLoadName(segmentId)
+      loadMetadataDetail.setSegmentStatus(SegmentStatus.INSERT_IN_PROGRESS)
+      loadMetadataDetail.setExtraInfo(segmentMap)
+      loadMetadataDetailList.add(loadMetadataDetail)
+      newLoadName = segmentId
+      SegmentStatusManager.writeLoadDetailsIntoFile(CarbonTablePath.getTableStatusFilePath(
+        viewSchema.getIdentifier.getTablePath),
+        loadMetadataDetailList.toArray(new Array[LoadMetadataDetails](loadMetadataDetailList
+          .size)))
+    } else {
+      LOGGER.error("Not able to acquire the lock for Table status updation for table " +
+                   viewSchema.getIdentifier.getDatabaseName + "." +
+                   viewSchema.getIdentifier.getTableName)
+      viewManager.setStatus(viewSchema.getIdentifier, MaterializedViewStatus.DISABLED)
+      return false
+    } finally {
+      if (lock.unlock) {
+        LOGGER.info("Table unlocked successfully after table status updation" +
+                    viewSchema.getIdentifier.getDatabaseName + "." +
+                    viewSchema.getIdentifier.getTableName)
+      } else {
+        LOGGER.error("Unable to unlock Table lock for table" +
+                     viewSchema.getIdentifier.getDatabaseName + "." +
+                     viewSchema.getIdentifier.getTableName +
+                     " during table status updation")
+      }
+    }
+    refreshInternal(viewManager, viewSchema, viewTable, newLoadName, segmentMapping, session)
+  }
+
+  @throws[IOException]
+  private def refreshInternal(
+      viewManager: MaterializedViewManagerInSpark,
+      viewSchema: MaterializedViewSchema,
+      viewTable: CarbonTable,
+      newLoadName: String,
+      segmentMap: java.util.Map[String, java.util.List[String]],
+      session: SparkSession): Boolean = {
+    val query = viewSchema.getQuery
+    if (query != null) {
+      val viewIdentifier = viewSchema.getIdentifier
+      val updatedQuery = MaterializedViewQueryParser.getQuery(query, session)
+      val isFullRefresh = !viewSchema.isRefreshIncremental
+      // Set specified segments for incremental load
+      val segmentMapIterator = segmentMap.entrySet().iterator()
+      while (segmentMapIterator.hasNext) {
+        val entry = segmentMapIterator.next()
+        setInputSegments(entry.getKey, entry.getValue)
+      }
+      val header = viewTable.getTableInfo.getFactTable.getListOfColumns.asScala
+        .filter { column =>
+          !column.getColumnName
+            .equalsIgnoreCase(CarbonCommonConstants.DEFAULT_INVISIBLE_DUMMY_MEASURE)
+        }.sortBy(_.getSchemaOrdinal).map(_.getColumnName).mkString(",")
+      val insertIntoCommand = CarbonInsertIntoCommand(
+        databaseNameOp = Some(viewIdentifier.getDatabaseName),
+        tableName = viewIdentifier.getTableName,
+        options = scala.collection.immutable.Map("fileheader" -> header),
+        isFullRefresh,
+        logicalPlan = updatedQuery.queryExecution.analyzed,
+        tableInfo = viewTable.getTableInfo,
+        internalOptions = Map("mergedSegmentName" -> newLoadName,
+          CarbonCommonConstants.IS_INTERNAL_LOAD_CALL -> "true"),
+        partition = Map.empty)
+      try {
+        insertIntoCommand.run(session)
+      } catch {
+        case exception: Exception =>
+          // If load to dataMap table fails, disable the dataMap and if newLoad is still
+          // in INSERT_IN_PROGRESS state, mark for delete the newLoad and update table status file
+          viewManager.setStatus(viewSchema.getIdentifier, MaterializedViewStatus.DISABLED)
+          LOGGER.error("Data Load failed for DataMap: ", exception)
+          CarbonLoaderUtil.updateTableStatusInCaseOfFailure(
+            newLoadName,
+            viewTable.getAbsoluteTableIdentifier,
+            viewTable.getTableName,
+            viewTable.getDatabaseName,
+            viewTable.getTablePath,
+            viewTable.getMetadataPath)
+          throw exception
+      } finally {
+        unsetInputSegments(viewSchema)
+      }
+    }
+    true
+  }
+
+  /**
+   * This method will compare mainTable and dataMapTable segment List and loads only newly added
+   * segment from main table to dataMap table.
+   * In case if mainTable is compacted, then based on dataMap to mainTables segmentMapping, dataMap
+   * will be loaded
+   * Eg:
+   * case 1: Consider mainTableSegmentList: {0, 1, 2}, dataMapToMainTable segmentMap:
+   * { 0 -> 0, 1-> 1,2}. If (1, 2) segments of main table are compacted to 1.1 and new segment (3)
+   * is loaded to main table, then mainTableSegmentList will be updated to{0, 1.1, 3}.
+   * In this case, segment (1) of dataMap table will be marked for delete, and new segment
+   * {2 -> 1.1, 3} will be loaded to dataMap table
+   * case 2: Consider mainTableSegmentList: {0, 1, 2, 3}, dataMapToMainTable segmentMap:
+   * { 0 -> 0,1,2, 1-> 3}. If (1, 2) segments of main table are compacted to 1.1 and new segment
+   * (4) is loaded to main table, then mainTableSegmentList will be updated to {0, 1.1, 3, 4}.
+   * In this case, segment (0) of dataMap table will be marked for delete and segment (0) of
+   * main table will be added to validSegmentList which needs to be loaded again. Now, new dataMap
+   * table segment (2) with main table segmentList{2 -> 1.1, 4, 0} will be loaded to dataMap table.
+   * dataMapToMainTable segmentMap will be updated to {1 -> 3, 2 -> 1.1, 4, 0} after rebuild
+   */
+  @throws[IOException]
+  private def getSpecificSegmentsTobeLoaded(schema: MaterializedViewSchema,
+      segmentMapping: util.Map[String, util.List[String]],
+      listOfLoadFolderDetails: util.List[LoadMetadataDetails]): Boolean = {
+    val relationIdentifiers: util.List[RelationIdentifier] = schema.getAssociatedTables
+    // invalidDataMapSegmentList holds segment list which needs to be marked for delete
+    val invalidDataMapSegmentList: util.HashSet[String] = new util.HashSet[String]
 
 Review comment:
   OK

----------------------------------------------------------------
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:
[hidden email]


With regards,
Apache Git Services
Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

GitBox
In reply to this post by GitBox
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396974227
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonCleanFilesCommand.scala
 ##########
 @@ -65,7 +66,7 @@ case class CarbonCleanFilesCommand(
       val allDataMapSchemas = DataMapStoreManager.getInstance
         .getDataMapSchemasOfTable(carbonTable).asScala
         .filter(dataMapSchema => null != dataMapSchema.getRelationIdentifier &&
-                                 !dataMapSchema.isIndexDataMap)
+                                 !dataMapSchema.isIndex)
 
 Review comment:
   Yes, will be delete when clean the old mv implementation

----------------------------------------------------------------
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:
[hidden email]


With regards,
Apache Git Services
Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

GitBox
In reply to this post by GitBox
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396974320
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForDeleteCommand.scala
 ##########
 @@ -136,11 +137,18 @@ private[sql] case class CarbonProjectForDeleteCommand(
       val allDataMapSchemas = DataMapStoreManager.getInstance
         .getDataMapSchemasOfTable(carbonTable).asScala
         .filter(dataMapSchema => null != dataMapSchema.getRelationIdentifier &&
-                                 !dataMapSchema.isIndexDataMap).asJava
+                                 !dataMapSchema.isIndex).asJava
 
 Review comment:
   Yes, will be delete when clean the old mv implementation

----------------------------------------------------------------
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:
[hidden email]


With regards,
Apache Git Services
Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

GitBox
In reply to this post by GitBox
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396974363
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForUpdateCommand.scala
 ##########
 @@ -203,10 +204,18 @@ private[sql] case class CarbonProjectForUpdateCommand(
       val allDataMapSchemas = DataMapStoreManager.getInstance
         .getDataMapSchemasOfTable(carbonTable).asScala
         .filter(dataMapSchema => null != dataMapSchema.getRelationIdentifier &&
-                                 !dataMapSchema.isIndexDataMap).asJava
+                                 !dataMapSchema.isIndex).asJava
 
 Review comment:
   Yes, will be delete when clean the old mv implementation

----------------------------------------------------------------
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:
[hidden email]


With regards,
Apache Git Services
Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

GitBox
In reply to this post by GitBox
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396974855
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/strategy/DDLHelper.scala
 ##########
 @@ -379,13 +380,17 @@ object DDLHelper {
           throw new MalformedCarbonCommandException(
             "Streaming property value is incorrect")
         }
-        if (carbonTable.hasMVCreated) {
+        if (carbonTable.isMaterializedView) {
+          throw new MalformedCarbonCommandException(
+            "Datamap table does not support set streaming property")
 
 Review comment:
   OK

----------------------------------------------------------------
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:
[hidden email]


With regards,
Apache Git Services
Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

GitBox
In reply to this post by GitBox
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396975357
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
 ##########
 @@ -65,20 +67,30 @@ case class CarbonIUDAnalysisRule(sparkSession: SparkSession) extends Rule[Logica
       val projList = Seq(UnresolvedAlias(UnresolvedStar(alias.map(Seq(_)))), tupleId)
       val carbonTable = CarbonEnv.getCarbonTable(table.tableIdentifier)(sparkSession)
       if (carbonTable != null) {
+        val viewManager = MaterializedViewManagerInSpark.get(sparkSession)
+        val viewSchemas = viewManager.getSchemasOnTable(carbonTable)
+        if (!viewSchemas.isEmpty) {
+          viewSchemas.asScala.foreach { schema =>
+            viewManager.setStatus(
+              schema.getIdentifier,
+              MaterializedViewStatus.DISABLED
+            )
+          }
+        }
         val indexSchemas = DataMapStoreManager.getInstance().getDataMapSchemasOfTable(carbonTable)
         if (carbonTable.hasMVCreated) {
           val allDataMapSchemas = DataMapStoreManager.getInstance
             .getDataMapSchemasOfTable(carbonTable).asScala
             .filter(dataMapSchema => null != dataMapSchema.getRelationIdentifier &&
-                                     !dataMapSchema.isIndexDataMap).asJava
+                                     !dataMapSchema.isIndex).asJava
 
 Review comment:
   Yes, will be delete when clean the old mv implementation

----------------------------------------------------------------
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:
[hidden email]


With regards,
Apache Git Services
Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

GitBox
In reply to this post by GitBox
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396975509
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
 ##########
 @@ -207,16 +219,26 @@ case class CarbonIUDAnalysisRule(sparkSession: SparkSession) extends Rule[Logica
         val projList = Seq(UnresolvedAlias(UnresolvedStar(alias.map(Seq(_)))), tupleId)
         val carbonTable = CarbonEnv.getCarbonTable(table.tableIdentifier)(sparkSession)
         if (carbonTable != null) {
-          if (carbonTable.isChildTableForMV) {
+          if (carbonTable.isMaterializedView) {
             throw new UnsupportedOperationException(
               "Delete operation is not supported for datamap table")
           }
+          val viewManager = MaterializedViewManagerInSpark.get(sparkSession)
+          val viewSchemas = viewManager.getSchemasOnTable(carbonTable)
+          if (!viewSchemas.isEmpty) {
+            viewSchemas.asScala.foreach { schema =>
+              viewManager.setStatus(
+                schema.getIdentifier,
+                MaterializedViewStatus.DISABLED
+              )
+            }
+          }
           val indexSchemas = DataMapStoreManager.getInstance().getDataMapSchemasOfTable(carbonTable)
           if (carbonTable.hasMVCreated) {
             val allDataMapSchemas = DataMapStoreManager.getInstance
               .getDataMapSchemasOfTable(carbonTable).asScala
               .filter(dataMapSchema => null != dataMapSchema.getRelationIdentifier &&
-                                       !dataMapSchema.isIndexDataMap).asJava
+                                       !dataMapSchema.isIndex).asJava
 
 Review comment:
   Yes, will be delete when clean the old mv implementation

----------------------------------------------------------------
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:
[hidden email]


With regards,
Apache Git Services
Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

GitBox
In reply to this post by GitBox
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396975555
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/listeners/DropCacheEventListeners.scala
 ##########
 @@ -31,31 +32,28 @@ import org.apache.carbondata.core.metadata.schema.datamap.DataMapClassProvider
 import org.apache.carbondata.core.metadata.schema.table.DataMapSchema
 import org.apache.carbondata.events.{DropTableCacheEvent, Event, OperationContext, OperationEventListener}
 
-object DropCacheDataMapEventListener extends OperationEventListener {
+object DropCacheMVEventListener extends OperationEventListener {
 
-  val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
 
   /**
    * Called on a specified event occurrence
-   *
-   * @param event
-   * @param operationContext
    */
   override protected def onEvent(event: Event, operationContext: OperationContext): Unit = {
     event match {
       case dropCacheEvent: DropTableCacheEvent =>
         val carbonTable = dropCacheEvent.carbonTable
         val sparkSession = dropCacheEvent.sparkSession
         val internalCall = dropCacheEvent.internalCall
-        if (carbonTable.isChildTableForMV && !internalCall) {
+        if (carbonTable.isMaterializedView && !internalCall) {
           throw new UnsupportedOperationException("Operation not allowed on child table.")
         }
 
         if (carbonTable.hasMVCreated) {
           val childrenSchemas = DataMapStoreManager.getInstance
             .getDataMapSchemasOfTable(carbonTable).asScala
             .filter(dataMapSchema => null != dataMapSchema.getRelationIdentifier &&
-                                     !dataMapSchema.isIndexDataMap)
+                                     !dataMapSchema.isIndex)
 
 Review comment:
   Yes, will be delete when clean the old mv implementation

----------------------------------------------------------------
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:
[hidden email]


With regards,
Apache Git Services
1 ... 45678910 ... 12