[GitHub] [carbondata] niuge01 opened a new pull request #3602: [CARBONDATA-3676] Support clean carbon data files of stages.

classic Classic list List threaded Threaded
67 messages Options
1234
Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] niuge01 commented on a change in pull request #3602: [CARBONDATA-3676] Support clean carbon data files of stages.

GitBox
niuge01 commented on a change in pull request #3602: [CARBONDATA-3676] Support clean carbon data files of stages.
URL: https://github.com/apache/carbondata/pull/3602#discussion_r376200240
 
 

 ##########
 File path: integration/flink/src/test/scala/org/apache/carbon/flink/TestDeleteStageFiles.scala
 ##########
 @@ -0,0 +1,203 @@
+/*
+ * 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.carbon.flink
+
+import java.io.File
+import java.util.Properties
+
+import org.apache.flink.api.common.restartstrategy.RestartStrategies
+import org.apache.flink.core.fs.Path
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
+import org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.test.util.QueryTest
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+class TestDeleteStageFiles extends QueryTest {
+
+  val tableName = "test_flink_delete_stage_file"
+
+  test("Delete stage file success") {
+    sql(s"DROP TABLE IF EXISTS $tableName").collect()
+    sql(
+      s"""
+         | CREATE TABLE $tableName (stringField string, intField int, shortField short)
 
 Review comment:
   added

----------------------------------------------------------------
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 #3602: [CARBONDATA-3676] Support clean carbon data files of stages.

GitBox
In reply to this post by GitBox
niuge01 commented on a change in pull request #3602: [CARBONDATA-3676] Support clean carbon data files of stages.
URL: https://github.com/apache/carbondata/pull/3602#discussion_r376200389
 
 

 ##########
 File path: integration/flink/src/main/java/org/apache/carbon/flink/CarbonS3Property.java
 ##########
 @@ -27,6 +27,7 @@
 
   static final String DATA_TEMP_PATH = "carbon.writer.s3.data.temp.path";
 
+  @Deprecated
 
 Review comment:
   deleted.

----------------------------------------------------------------
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 #3602: [CARBONDATA-3676] Support clean carbon data files of stages.

GitBox
In reply to this post by GitBox
niuge01 commented on a change in pull request #3602: [CARBONDATA-3676] Support clean carbon data files of stages.
URL: https://github.com/apache/carbondata/pull/3602#discussion_r376200631
 
 

 ##########
 File path: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonDeleteStageFilesCommand.scala
 ##########
 @@ -0,0 +1,175 @@
+/*
+ * 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.management
+
+import java.io.InputStreamReader
+import java.util
+import java.util.Collections
+
+import scala.collection.JavaConverters._
+
+import com.google.gson.Gson
+import org.apache.hadoop.conf.Configuration
+import org.apache.log4j.Logger
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.execution.command.{Checker, DataCommand}
+
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
+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.impl.FileFactory
+import org.apache.carbondata.core.statusmanager.StageInput
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+/**
+ * Delete carbon data files of table stages.
+ *
+ * @param databaseNameOp database name
+ * @param tableName      table name
+ */
+case class CarbonDeleteStageFilesCommand(
+    databaseNameOp: Option[String],
+    tableName: String,
+    options: Map[String, String]
+) extends DataCommand {
+
+  @transient val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+
+  override def processData(spark: SparkSession): Seq[Row] = {
+    Checker.validateTableExists(databaseNameOp, tableName, spark)
+    val table = CarbonEnv.getCarbonTable(databaseNameOp, tableName)(spark)
+    val configuration = spark.sessionState.newHadoopConf()
+    setAuditTable(table)
+    if (!table.getTableInfo.isTransactionalTable) {
+      throw new MalformedCarbonCommandException("Unsupported operation on non transactional table")
+    }
+    if (table.isChildTableForMV) {
+      throw new MalformedCarbonCommandException("Unsupported operation on MV table")
+    }
+    val tablePath = table.getTablePath
+    val startTime = System.currentTimeMillis()
+    val stageDataFileLocation = CarbonTablePath.getStageDataDir(tablePath)
+    val stageDataFileActiveTime = try {
+      Integer.valueOf(options.getOrElse("retain_hour", "0")) * 3600000
+    } catch {
+      case _: NumberFormatException =>
+        throw new MalformedCarbonCommandException(
+          "Option [retain_hour] is not a number.")
+    }
+    if (stageDataFileActiveTime < 0) {
+      throw new MalformedCarbonCommandException(
+        "Option [retain_hour] is negative.")
+    }
+    val stageDataFilesReferenced =
+      listStageDataFilesReferenced(listStageMetadataFiles(tablePath, configuration), configuration)
+    val stageDataFiles = listStageDataFiles(stageDataFileLocation, configuration)
+    stageDataFiles.collect {
+      case stageDataFile: CarbonFile =>
+        // Which file will be deleted:
+        // 1. Not referenced by any stage file;
+        // 2. Has passed retain time.
+        if (!stageDataFilesReferenced.contains(stageDataFile.getCanonicalPath) &&
+            (startTime - stageDataFile.getLastModifiedTime) >= stageDataFileActiveTime) {
+          stageDataFile.delete()
+        }
+    }
+    Seq.empty
+  }
+
+  private def listStageMetadataFiles(
+      tablePath: String,
+      configuration: Configuration
+  ): Seq[CarbonFile] = {
+    val stagePath = CarbonTablePath.getStageDir(tablePath)
+    val stageDirectory = FileFactory.getCarbonFile(stagePath, configuration)
+    if (stageDirectory.exists()) {
+      stageDirectory.listFiles().filter { file =>
+        !file.getName.endsWith(CarbonTablePath.SUCCESS_FILE_SUBFIX)
+      }
+    } else {
+      Seq.empty
+    }
+  }
+
+  private def listStageDataFiles(
+      location: String,
 
 Review comment:
   done.

----------------------------------------------------------------
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 #3602: [CARBONDATA-3676] Support clean carbon data files of stages.

GitBox
In reply to this post by GitBox
niuge01 commented on a change in pull request #3602: [CARBONDATA-3676] Support clean carbon data files of stages.
URL: https://github.com/apache/carbondata/pull/3602#discussion_r376200989
 
 

 ##########
 File path: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonDeleteStageFilesCommand.scala
 ##########
 @@ -0,0 +1,175 @@
+/*
+ * 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.management
+
+import java.io.InputStreamReader
+import java.util
+import java.util.Collections
+
+import scala.collection.JavaConverters._
+
+import com.google.gson.Gson
+import org.apache.hadoop.conf.Configuration
+import org.apache.log4j.Logger
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.execution.command.{Checker, DataCommand}
+
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
+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.impl.FileFactory
+import org.apache.carbondata.core.statusmanager.StageInput
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+/**
+ * Delete carbon data files of table stages.
+ *
+ * @param databaseNameOp database name
+ * @param tableName      table name
+ */
+case class CarbonDeleteStageFilesCommand(
+    databaseNameOp: Option[String],
+    tableName: String,
+    options: Map[String, String]
+) extends DataCommand {
+
+  @transient val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+
+  override def processData(spark: SparkSession): Seq[Row] = {
+    Checker.validateTableExists(databaseNameOp, tableName, spark)
+    val table = CarbonEnv.getCarbonTable(databaseNameOp, tableName)(spark)
+    val configuration = spark.sessionState.newHadoopConf()
+    setAuditTable(table)
+    if (!table.getTableInfo.isTransactionalTable) {
+      throw new MalformedCarbonCommandException("Unsupported operation on non transactional table")
+    }
+    if (table.isChildTableForMV) {
+      throw new MalformedCarbonCommandException("Unsupported operation on MV table")
+    }
+    val tablePath = table.getTablePath
+    val startTime = System.currentTimeMillis()
+    val stageDataFileLocation = CarbonTablePath.getStageDataDir(tablePath)
+    val stageDataFileActiveTime = try {
+      Integer.valueOf(options.getOrElse("retain_hour", "0")) * 3600000
+    } catch {
+      case _: NumberFormatException =>
+        throw new MalformedCarbonCommandException(
+          "Option [retain_hour] is not a number.")
+    }
+    if (stageDataFileActiveTime < 0) {
+      throw new MalformedCarbonCommandException(
+        "Option [retain_hour] is negative.")
+    }
+    val stageDataFilesReferenced =
+      listStageDataFilesReferenced(listStageMetadataFiles(tablePath, configuration), configuration)
+    val stageDataFiles = listStageDataFiles(stageDataFileLocation, configuration)
+    stageDataFiles.collect {
+      case stageDataFile: CarbonFile =>
+        // Which file will be deleted:
+        // 1. Not referenced by any stage file;
+        // 2. Has passed retain time.
+        if (!stageDataFilesReferenced.contains(stageDataFile.getCanonicalPath) &&
+            (startTime - stageDataFile.getLastModifiedTime) >= stageDataFileActiveTime) {
+          stageDataFile.delete()
+        }
+    }
+    Seq.empty
+  }
+
+  private def listStageMetadataFiles(
+      tablePath: String,
+      configuration: Configuration
+  ): Seq[CarbonFile] = {
+    val stagePath = CarbonTablePath.getStageDir(tablePath)
+    val stageDirectory = FileFactory.getCarbonFile(stagePath, configuration)
+    if (stageDirectory.exists()) {
+      stageDirectory.listFiles().filter { file =>
+        !file.getName.endsWith(CarbonTablePath.SUCCESS_FILE_SUBFIX)
+      }
+    } else {
+      Seq.empty
+    }
+  }
+
+  private def listStageDataFiles(
+      location: String,
+      configuration: Configuration
+  ): Seq[CarbonFile] = {
+    val stageDataFileLocation = FileFactory.getCarbonFile(location, configuration)
+    if (!stageDataFileLocation.exists()) {
+      LOGGER.warn("Stage data file location is not exists. " + location)
+      Seq.empty
+    } else {
+      stageDataFileLocation.listFiles(true).asScala
+    }
+  }
+
+  private def listStageDataFilesReferenced(
 
 Review comment:
   added

----------------------------------------------------------------
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 #3602: [CARBONDATA-3676] Support clean carbon data files of stages.

GitBox
In reply to this post by GitBox
CarbonDataQA1 commented on issue #3602: [CARBONDATA-3676] Support clean carbon data files of stages.
URL: https://github.com/apache/carbondata/pull/3602#issuecomment-583222896
 
 
   Build Success with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/173/
   

----------------------------------------------------------------
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 #3602: [CARBONDATA-3676] Support clean carbon data files of stages.

GitBox
In reply to this post by GitBox
CarbonDataQA1 commented on issue #3602: [CARBONDATA-3676] Support clean carbon data files of stages.
URL: https://github.com/apache/carbondata/pull/3602#issuecomment-583233805
 
 
   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/1876/
   

----------------------------------------------------------------
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] asfgit closed pull request #3602: [CARBONDATA-3676] Support clean carbon data files of stages.

GitBox
In reply to this post by GitBox
asfgit closed pull request #3602: [CARBONDATA-3676] Support clean carbon data files of stages.
URL: https://github.com/apache/carbondata/pull/3602
 
 
   

----------------------------------------------------------------
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
1234