|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | + |
| 19 | +package org.apache.iceberg.mr.hive.compaction.evaluator; |
| 20 | + |
| 21 | +import java.io.IOException; |
| 22 | +import java.io.UncheckedIOException; |
| 23 | +import java.util.Collections; |
| 24 | +import java.util.Map; |
| 25 | +import java.util.Optional; |
| 26 | +import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo; |
| 27 | +import org.apache.hadoop.hive.ql.txn.compactor.CompactorContext; |
| 28 | +import org.apache.iceberg.DataFile; |
| 29 | +import org.apache.iceberg.FileScanTask; |
| 30 | +import org.apache.iceberg.MetadataTableType; |
| 31 | +import org.apache.iceberg.MetadataTableUtils; |
| 32 | +import org.apache.iceberg.PartitionData; |
| 33 | +import org.apache.iceberg.PartitionSpec; |
| 34 | +import org.apache.iceberg.Partitioning; |
| 35 | +import org.apache.iceberg.PartitionsTable; |
| 36 | +import org.apache.iceberg.StructLike; |
| 37 | +import org.apache.iceberg.Table; |
| 38 | +import org.apache.iceberg.io.CloseableIterable; |
| 39 | +import org.apache.iceberg.mr.hive.IcebergTableUtil; |
| 40 | +import org.apache.iceberg.mr.hive.compaction.IcebergCompactionUtil; |
| 41 | +import org.apache.iceberg.mr.hive.compaction.evaluator.amoro.CommonPartitionEvaluator; |
| 42 | +import org.apache.iceberg.mr.hive.compaction.evaluator.amoro.IcebergTableFileScanHelper; |
| 43 | +import org.apache.iceberg.mr.hive.compaction.evaluator.amoro.OptimizingConfig; |
| 44 | +import org.apache.iceberg.mr.hive.compaction.evaluator.amoro.TableConfiguration; |
| 45 | +import org.apache.iceberg.mr.hive.compaction.evaluator.amoro.TableFileScanHelper; |
| 46 | +import org.apache.iceberg.mr.hive.compaction.evaluator.amoro.TableFormat; |
| 47 | +import org.apache.iceberg.mr.hive.compaction.evaluator.amoro.TableProperties; |
| 48 | +import org.apache.iceberg.mr.hive.compaction.evaluator.amoro.TableRuntime; |
| 49 | +import org.apache.iceberg.mr.hive.compaction.evaluator.amoro.TableRuntimeMeta; |
| 50 | +import org.apache.iceberg.relocated.com.google.common.collect.FluentIterable; |
| 51 | +import org.apache.iceberg.relocated.com.google.common.collect.Maps; |
| 52 | +import org.apache.iceberg.util.Pair; |
| 53 | +import org.apache.iceberg.util.StructProjection; |
| 54 | +import org.slf4j.Logger; |
| 55 | +import org.slf4j.LoggerFactory; |
| 56 | + |
| 57 | +public class CompactionEvaluator extends CommonPartitionEvaluator { |
| 58 | + |
| 59 | + private static final long LAST_OPTIMIZE_TIME = 0; |
| 60 | + private static final int TRIGGER_INTERVAL = -1; |
| 61 | + private final Table table; |
| 62 | + private final CompactionInfo ci; |
| 63 | + |
| 64 | + private static final Logger LOG = LoggerFactory.getLogger(CompactionEvaluator.class); |
| 65 | + |
| 66 | + public CompactionEvaluator(Table table, CompactionInfo ci, Map<String, String> parameters) throws IOException { |
| 67 | + super( |
| 68 | + createTableRuntime(table, parameters), |
| 69 | + getPartitionSpecStructPair(table, ci.partName), |
| 70 | + System.currentTimeMillis() |
| 71 | + ); |
| 72 | + this.table = table; |
| 73 | + this.ci = ci; |
| 74 | + addFiles(); |
| 75 | + } |
| 76 | + |
| 77 | + public boolean isEligibleForCompaction() { |
| 78 | + |
| 79 | + if (table.currentSnapshot() == null) { |
| 80 | + LOG.info("Table {}{} doesn't require compaction because it is empty", table, |
| 81 | + ci.partName == null ? "" : " partition " + ci.partName); |
| 82 | + return false; |
| 83 | + } |
| 84 | + |
| 85 | + addFiles(); |
| 86 | + |
| 87 | + switch (ci.type) { |
| 88 | + case MINOR: |
| 89 | + return isMinorNecessary(); |
| 90 | + case MAJOR: |
| 91 | + return isMajorNecessary(); |
| 92 | + default: |
| 93 | + return false; |
| 94 | + } |
| 95 | + } |
| 96 | + |
| 97 | + private static TableRuntime createTableRuntime(Table icebergTable, Map<String, String> parameters) { |
| 98 | + OptimizingConfig optimizingConfig = OptimizingConfig.parse(Collections.emptyMap()); |
| 99 | + optimizingConfig.setTargetSize(getTargetSizeBytes(parameters)); |
| 100 | + optimizingConfig.setFragmentRatio(getFragmentRatio(parameters)); |
| 101 | + optimizingConfig.setMinTargetSizeRatio(getMinTargetSizeRatio(parameters)); |
| 102 | + optimizingConfig.setMinorLeastFileCount(getMinInputFiles(parameters)); |
| 103 | + optimizingConfig.setMajorDuplicateRatio(getDeleteFileRatio(parameters)); |
| 104 | + optimizingConfig.setFullTriggerInterval(TRIGGER_INTERVAL); |
| 105 | + optimizingConfig.setMinorLeastInterval(TRIGGER_INTERVAL); |
| 106 | + |
| 107 | + TableConfiguration tableConfig = new TableConfiguration(); |
| 108 | + tableConfig.setOptimizingConfig(optimizingConfig); |
| 109 | + |
| 110 | + TableRuntimeMeta tableRuntimeMeta = new TableRuntimeMeta(); |
| 111 | + tableRuntimeMeta.setTableName(icebergTable.name()); |
| 112 | + tableRuntimeMeta.setFormat(TableFormat.ICEBERG); |
| 113 | + tableRuntimeMeta.setLastFullOptimizingTime(LAST_OPTIMIZE_TIME); |
| 114 | + tableRuntimeMeta.setLastMinorOptimizingTime(LAST_OPTIMIZE_TIME); |
| 115 | + tableRuntimeMeta.setTableConfig(tableConfig); |
| 116 | + |
| 117 | + return new HiveTableRuntime(tableRuntimeMeta); |
| 118 | + } |
| 119 | + |
| 120 | + private void addFiles() { |
| 121 | + TableFileScanHelper tableFileScanHelper = new IcebergTableFileScanHelper(table, |
| 122 | + table.currentSnapshot().snapshotId()); |
| 123 | + try (CloseableIterable<TableFileScanHelper.FileScanResult> results = |
| 124 | + tableFileScanHelper.scan()) { |
| 125 | + for (TableFileScanHelper.FileScanResult fileScanResult : results) { |
| 126 | + DataFile file = fileScanResult.file(); |
| 127 | + if (IcebergCompactionUtil.shouldIncludeForCompaction(table, ci.partName, file)) { |
| 128 | + addFile(fileScanResult.file(), fileScanResult.deleteFiles()); |
| 129 | + } |
| 130 | + } |
| 131 | + } catch (IOException e) { |
| 132 | + throw new UncheckedIOException(e); |
| 133 | + } |
| 134 | + } |
| 135 | + |
| 136 | + public static long getTargetSizeBytes(Map<String, String> parameters) { |
| 137 | + return Optional.ofNullable(parameters.get(CompactorContext.COMPACTION_TARGET_SIZE)) |
| 138 | + .map(Long::parseLong) |
| 139 | + .orElse(TableProperties.SELF_OPTIMIZING_TARGET_SIZE_DEFAULT); |
| 140 | + } |
| 141 | + |
| 142 | + public static double getMinTargetSizeRatio(Map<String, String> parameters) { |
| 143 | + return Optional.ofNullable(parameters.get(CompactorContext.COMPACTION_MIN_TARGET_SIZE_RATIO)) |
| 144 | + .map(Double::parseDouble) |
| 145 | + .orElse(TableProperties.SELF_OPTIMIZING_MIN_TARGET_SIZE_RATIO_DEFAULT); |
| 146 | + } |
| 147 | + |
| 148 | + public static int getFragmentRatio(Map<String, String> parameters) { |
| 149 | + return Optional.ofNullable(parameters.get(CompactorContext.COMPACTION_MIN_FRAGMENT_RATIO)) |
| 150 | + .map(x -> (int) (1 / Double.parseDouble(x))) |
| 151 | + .orElse(TableProperties.SELF_OPTIMIZING_FRAGMENT_RATIO_DEFAULT); |
| 152 | + } |
| 153 | + |
| 154 | + public static int getFragmentSizeBytes(Map<String, String> parameters) { |
| 155 | + return (int) (getTargetSizeBytes(parameters) * getMinTargetSizeRatio(parameters)); |
| 156 | + } |
| 157 | + |
| 158 | + public static int getMinInputFiles(Map<String, String> parameters) { |
| 159 | + return Optional.ofNullable(parameters.get(CompactorContext.COMPACTION_MIN_INPUT_FILES)) |
| 160 | + .map(Integer::parseInt) |
| 161 | + .orElse(TableProperties.SELF_OPTIMIZING_MINOR_TRIGGER_FILE_CNT_DEFAULT); |
| 162 | + } |
| 163 | + |
| 164 | + public static double getDeleteFileRatio(Map<String, String> parameters) { |
| 165 | + return Optional.ofNullable(parameters.get(CompactorContext.COMPACTION_DELETE_FILE_RATIO)) |
| 166 | + .map(Double::parseDouble) |
| 167 | + .orElse(TableProperties.SELF_OPTIMIZING_MAJOR_TRIGGER_DUPLICATE_RATIO_DEFAULT); |
| 168 | + } |
| 169 | + |
| 170 | + private static Pair<Integer, StructLike> getPartitionSpecStructPair(Table table, String partitionPath) |
| 171 | + throws IOException { |
| 172 | + if (!table.spec().isPartitioned() || partitionPath == null) { |
| 173 | + return null; |
| 174 | + } |
| 175 | + PartitionsTable partitionsTable = (PartitionsTable) MetadataTableUtils |
| 176 | + .createMetadataTableInstance(table, MetadataTableType.PARTITIONS); |
| 177 | + try (CloseableIterable<FileScanTask> fileScanTasks = partitionsTable.newScan().planFiles()) { |
| 178 | + return FluentIterable.from(fileScanTasks) |
| 179 | + .transformAndConcat(task -> task.asDataTask().rows()) |
| 180 | + .transform(row -> { |
| 181 | + StructLike data = row.get(IcebergTableUtil.PART_IDX, StructProjection.class); |
| 182 | + PartitionSpec spec = table.specs().get(row.get(IcebergTableUtil.SPEC_IDX, Integer.class)); |
| 183 | + PartitionData partitionData = IcebergTableUtil.toPartitionData(data, |
| 184 | + Partitioning.partitionType(table), spec.partitionType()); |
| 185 | + String path = spec.partitionToPath(partitionData); |
| 186 | + return Maps.immutableEntry(path, Pair.of(spec.specId(), data)); |
| 187 | + }) |
| 188 | + .filter(e -> e.getKey().equals(partitionPath)) |
| 189 | + .transform(Map.Entry::getValue) |
| 190 | + .get(0); |
| 191 | + } |
| 192 | + } |
| 193 | +} |
0 commit comments