|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | + |
| 18 | +package org.apache.spark.sql.hudi.command.procedures |
| 19 | + |
| 20 | +import org.apache.avro.generic.GenericRecord |
| 21 | +import org.apache.avro.specific.SpecificData |
| 22 | +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} |
| 23 | +import org.apache.hudi.avro.HoodieAvroUtils |
| 24 | +import org.apache.hudi.avro.model.HoodieArchivedMetaEntry |
| 25 | +import org.apache.hudi.common.fs.FSUtils |
| 26 | +import org.apache.hudi.common.model.HoodieLogFile |
| 27 | +import org.apache.hudi.common.table.HoodieTableMetaClient |
| 28 | +import org.apache.hudi.common.table.log.HoodieLogFormat |
| 29 | +import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock |
| 30 | +import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline, TimelineMetadataUtils} |
| 31 | +import org.apache.hudi.exception.HoodieException |
| 32 | +import org.apache.spark.internal.Logging |
| 33 | +import org.apache.spark.sql.Row |
| 34 | +import org.apache.spark.sql.catalyst.TableIdentifier |
| 35 | +import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable |
| 36 | +import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} |
| 37 | + |
| 38 | +import java.io.File |
| 39 | +import java.util |
| 40 | +import java.util.Collections |
| 41 | +import java.util.function.Supplier |
| 42 | +import scala.collection.JavaConverters._ |
| 43 | +import scala.util.control.Breaks.break |
| 44 | + |
| 45 | +class ExportInstantsProcedure extends BaseProcedure with ProcedureBuilder with Logging { |
| 46 | + var sortByFieldParameter: ProcedureParameter = _ |
| 47 | + |
| 48 | + val defaultActions = "clean,commit,deltacommit,rollback,savepoint,restore" |
| 49 | + |
| 50 | + private val PARAMETERS = Array[ProcedureParameter]( |
| 51 | + ProcedureParameter.required(0, "table", DataTypes.StringType, None), |
| 52 | + ProcedureParameter.required(1, "localFolder", DataTypes.StringType, None), |
| 53 | + ProcedureParameter.optional(2, "limit", DataTypes.IntegerType, -1), |
| 54 | + ProcedureParameter.optional(3, "actions", DataTypes.StringType, defaultActions), |
| 55 | + ProcedureParameter.optional(4, "desc", DataTypes.BooleanType, false) |
| 56 | + ) |
| 57 | + |
| 58 | + private val OUTPUT_TYPE = new StructType(Array[StructField]( |
| 59 | + StructField("export_detail", DataTypes.StringType, nullable = true, Metadata.empty) |
| 60 | + )) |
| 61 | + |
| 62 | + def parameters: Array[ProcedureParameter] = PARAMETERS |
| 63 | + |
| 64 | + def outputType: StructType = OUTPUT_TYPE |
| 65 | + |
| 66 | + override def call(args: ProcedureArgs): Seq[Row] = { |
| 67 | + super.checkArgs(PARAMETERS, args) |
| 68 | + |
| 69 | + val table = getArgValueOrDefault(args, PARAMETERS(0)).get.asInstanceOf[String] |
| 70 | + val localFolder = getArgValueOrDefault(args, PARAMETERS(1)).get.asInstanceOf[String] |
| 71 | + val limit = getArgValueOrDefault(args, PARAMETERS(2)).get.asInstanceOf[Int] |
| 72 | + val actions: String = getArgValueOrDefault(args, PARAMETERS(3)).get.asInstanceOf[String] |
| 73 | + val desc = getArgValueOrDefault(args, PARAMETERS(4)).get.asInstanceOf[Boolean] |
| 74 | + |
| 75 | + val hoodieCatalogTable = HoodieCatalogTable(sparkSession, new TableIdentifier(table)) |
| 76 | + val basePath = hoodieCatalogTable.tableLocation |
| 77 | + val metaClient = HoodieTableMetaClient.builder.setConf(jsc.hadoopConfiguration()).setBasePath(basePath).build |
| 78 | + val archivePath = new Path(basePath + "/.hoodie/.commits_.archive*") |
| 79 | + val actionSet: util.Set[String] = Set(actions.split(","): _*).asJava |
| 80 | + val numExports = if (limit == -1) Integer.MAX_VALUE else limit |
| 81 | + var numCopied = 0 |
| 82 | + |
| 83 | + if (!new File(localFolder).isDirectory) throw new HoodieException(localFolder + " is not a valid local directory") |
| 84 | + |
| 85 | + // The non archived instants can be listed from the Timeline. |
| 86 | + val nonArchivedInstants: util.List[HoodieInstant] = metaClient |
| 87 | + .getActiveTimeline |
| 88 | + .filterCompletedInstants.getInstants.iterator().asScala |
| 89 | + .filter((i: HoodieInstant) => actionSet.contains(i.getAction)) |
| 90 | + .toList.asJava |
| 91 | + |
| 92 | + // Archived instants are in the commit archive files |
| 93 | + val statuses: Array[FileStatus] = FSUtils.getFs(basePath, jsc.hadoopConfiguration()).globStatus(archivePath) |
| 94 | + val archivedStatuses = List(statuses: _*) |
| 95 | + .sortWith((f1, f2) => (f1.getModificationTime - f2.getModificationTime).toInt > 0).asJava |
| 96 | + |
| 97 | + if (desc) { |
| 98 | + Collections.reverse(nonArchivedInstants) |
| 99 | + numCopied = copyNonArchivedInstants(metaClient, nonArchivedInstants, numExports, localFolder) |
| 100 | + if (numCopied < numExports) { |
| 101 | + Collections.reverse(archivedStatuses) |
| 102 | + numCopied += copyArchivedInstants(basePath, archivedStatuses, actionSet, numExports - numCopied, localFolder) |
| 103 | + } |
| 104 | + } else { |
| 105 | + numCopied = copyArchivedInstants(basePath, archivedStatuses, actionSet, numExports, localFolder) |
| 106 | + if (numCopied < numExports) numCopied += copyNonArchivedInstants(metaClient, nonArchivedInstants, numExports - numCopied, localFolder) |
| 107 | + } |
| 108 | + |
| 109 | + Seq(Row("Exported " + numCopied + " Instants to " + localFolder)) |
| 110 | + } |
| 111 | + |
| 112 | + @throws[Exception] |
| 113 | + private def copyArchivedInstants(basePath: String, statuses: util.List[FileStatus], actionSet: util.Set[String], limit: Int, localFolder: String) = { |
| 114 | + import scala.collection.JavaConversions._ |
| 115 | + var copyCount = 0 |
| 116 | + val fileSystem = FSUtils.getFs(basePath, jsc.hadoopConfiguration()) |
| 117 | + for (fs <- statuses) { |
| 118 | + // read the archived file |
| 119 | + val reader = HoodieLogFormat.newReader(fileSystem, new HoodieLogFile(fs.getPath), HoodieArchivedMetaEntry.getClassSchema) |
| 120 | + // read the avro blocks |
| 121 | + while ( { |
| 122 | + reader.hasNext && copyCount < limit |
| 123 | + }) { |
| 124 | + val blk = reader.next.asInstanceOf[HoodieAvroDataBlock] |
| 125 | + try { |
| 126 | + val recordItr = blk.getRecordIterator |
| 127 | + try while ( { |
| 128 | + recordItr.hasNext |
| 129 | + }) { |
| 130 | + val ir = recordItr.next |
| 131 | + // Archived instants are saved as arvo encoded HoodieArchivedMetaEntry records. We need to get the |
| 132 | + // metadata record from the entry and convert it to json. |
| 133 | + val archiveEntryRecord = SpecificData.get.deepCopy(HoodieArchivedMetaEntry.SCHEMA$, ir).asInstanceOf[HoodieArchivedMetaEntry] |
| 134 | + val action = archiveEntryRecord.get("actionType").toString |
| 135 | + if (!actionSet.contains(action)) break() //todo: continue is not supported |
| 136 | + val metadata: GenericRecord = action match { |
| 137 | + case HoodieTimeline.CLEAN_ACTION => |
| 138 | + archiveEntryRecord.getHoodieCleanMetadata |
| 139 | + |
| 140 | + case HoodieTimeline.COMMIT_ACTION => |
| 141 | + archiveEntryRecord.getHoodieCommitMetadata |
| 142 | + |
| 143 | + case HoodieTimeline.DELTA_COMMIT_ACTION => |
| 144 | + archiveEntryRecord.getHoodieCommitMetadata |
| 145 | + |
| 146 | + case HoodieTimeline.ROLLBACK_ACTION => |
| 147 | + archiveEntryRecord.getHoodieRollbackMetadata |
| 148 | + |
| 149 | + case HoodieTimeline.SAVEPOINT_ACTION => |
| 150 | + archiveEntryRecord.getHoodieSavePointMetadata |
| 151 | + |
| 152 | + case HoodieTimeline.COMPACTION_ACTION => |
| 153 | + archiveEntryRecord.getHoodieCompactionMetadata |
| 154 | + |
| 155 | + case _ => logInfo("Unknown type of action " + action) |
| 156 | + null |
| 157 | + } |
| 158 | + val instantTime = archiveEntryRecord.get("commitTime").toString |
| 159 | + val outPath = localFolder + Path.SEPARATOR + instantTime + "." + action |
| 160 | + if (metadata != null) writeToFile(fileSystem, outPath, HoodieAvroUtils.avroToJson(metadata, true)) |
| 161 | + if ( { |
| 162 | + copyCount += 1; |
| 163 | + copyCount |
| 164 | + } == limit) break //todo: break is not supported |
| 165 | + } |
| 166 | + finally if (recordItr != null) recordItr.close() |
| 167 | + } |
| 168 | + } |
| 169 | + reader.close() |
| 170 | + } |
| 171 | + copyCount |
| 172 | + } |
| 173 | + |
| 174 | + @throws[Exception] |
| 175 | + private def copyNonArchivedInstants(metaClient: HoodieTableMetaClient, instants: util.List[HoodieInstant], limit: Int, localFolder: String): Int = { |
| 176 | + import scala.collection.JavaConversions._ |
| 177 | + var copyCount = 0 |
| 178 | + if (instants.nonEmpty) { |
| 179 | + val timeline = metaClient.getActiveTimeline |
| 180 | + val fileSystem = FSUtils.getFs(metaClient.getBasePath, jsc.hadoopConfiguration()) |
| 181 | + for (instant <- instants) { |
| 182 | + val localPath = localFolder + Path.SEPARATOR + instant.getFileName |
| 183 | + val data: Array[Byte] = instant.getAction match { |
| 184 | + case HoodieTimeline.CLEAN_ACTION => |
| 185 | + val metadata = TimelineMetadataUtils.deserializeHoodieCleanMetadata(timeline.getInstantDetails(instant).get) |
| 186 | + HoodieAvroUtils.avroToJson(metadata, true) |
| 187 | + |
| 188 | + case HoodieTimeline.DELTA_COMMIT_ACTION => |
| 189 | + // Already in json format |
| 190 | + timeline.getInstantDetails(instant).get |
| 191 | + |
| 192 | + case HoodieTimeline.COMMIT_ACTION => |
| 193 | + // Already in json format |
| 194 | + timeline.getInstantDetails(instant).get |
| 195 | + |
| 196 | + case HoodieTimeline.COMPACTION_ACTION => |
| 197 | + // Already in json format |
| 198 | + timeline.getInstantDetails(instant).get |
| 199 | + |
| 200 | + case HoodieTimeline.ROLLBACK_ACTION => |
| 201 | + val metadata = TimelineMetadataUtils.deserializeHoodieRollbackMetadata(timeline.getInstantDetails(instant).get) |
| 202 | + HoodieAvroUtils.avroToJson(metadata, true) |
| 203 | + |
| 204 | + case HoodieTimeline.SAVEPOINT_ACTION => |
| 205 | + val metadata = TimelineMetadataUtils.deserializeHoodieSavepointMetadata(timeline.getInstantDetails(instant).get) |
| 206 | + HoodieAvroUtils.avroToJson(metadata, true) |
| 207 | + |
| 208 | + case _ => null |
| 209 | + |
| 210 | + } |
| 211 | + if (data != null) { |
| 212 | + writeToFile(fileSystem, localPath, data) |
| 213 | + copyCount = copyCount + 1 |
| 214 | + } |
| 215 | + } |
| 216 | + } |
| 217 | + copyCount |
| 218 | + } |
| 219 | + |
| 220 | + @throws[Exception] |
| 221 | + private def writeToFile(fs: FileSystem, path: String, data: Array[Byte]): Unit = { |
| 222 | + val out = fs.create(new Path(path)) |
| 223 | + out.write(data) |
| 224 | + out.flush() |
| 225 | + out.close() |
| 226 | + } |
| 227 | + |
| 228 | + override def build = new ExportInstantsProcedure() |
| 229 | +} |
| 230 | + |
| 231 | +object ExportInstantsProcedure { |
| 232 | + val NAME = "export_instants" |
| 233 | + |
| 234 | + def builder: Supplier[ProcedureBuilder] = new Supplier[ProcedureBuilder] { |
| 235 | + override def get() = new ExportInstantsProcedure() |
| 236 | + } |
| 237 | +} |
| 238 | + |
| 239 | + |
0 commit comments