[HUDI-880] Replace part of spark context by hadoop configuration in HoodieTable. (#1614)
This commit is contained in:
@@ -18,6 +18,7 @@
|
||||
|
||||
package org.apache.hudi.client;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hudi.client.embedded.EmbeddedTimelineService;
|
||||
import org.apache.hudi.client.utils.ClientUtils;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
@@ -43,6 +44,7 @@ public abstract class AbstractHoodieClient implements Serializable, AutoCloseabl
|
||||
|
||||
protected final transient FileSystem fs;
|
||||
protected final transient JavaSparkContext jsc;
|
||||
protected final transient Configuration hadoopConf;
|
||||
protected final HoodieWriteConfig config;
|
||||
protected final String basePath;
|
||||
|
||||
@@ -62,6 +64,7 @@ public abstract class AbstractHoodieClient implements Serializable, AutoCloseabl
|
||||
Option<EmbeddedTimelineService> timelineServer) {
|
||||
this.fs = FSUtils.getFs(clientConfig.getBasePath(), jsc.hadoopConfiguration());
|
||||
this.jsc = jsc;
|
||||
this.hadoopConf = jsc.hadoopConfiguration();
|
||||
this.basePath = clientConfig.getBasePath();
|
||||
this.config = clientConfig;
|
||||
this.timelineServer = timelineServer;
|
||||
|
||||
@@ -99,7 +99,7 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload> e
|
||||
|
||||
LOG.info("Committing " + instantTime);
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable<T> table = HoodieTable.create(config, jsc);
|
||||
HoodieTable<T> table = HoodieTable.create(config, hadoopConf);
|
||||
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
|
||||
@@ -180,7 +180,7 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload> e
|
||||
// TODO : make sure we cannot rollback / archive last commit file
|
||||
try {
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable table = HoodieTable.create(config, jsc);
|
||||
HoodieTable table = HoodieTable.create(config, hadoopConf);
|
||||
// 0. All of the rolling stat management is only done by the DELTA commit for MOR and COMMIT for COW other wise
|
||||
// there may be race conditions
|
||||
HoodieRollingStatMetadata rollingStatMetadata = new HoodieRollingStatMetadata(actionType);
|
||||
@@ -231,7 +231,7 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload> e
|
||||
setWriteSchemaForDeletes(metaClient);
|
||||
}
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable table = HoodieTable.create(metaClient, config, jsc);
|
||||
HoodieTable table = HoodieTable.create(metaClient, config, hadoopConf);
|
||||
if (table.getMetaClient().getCommitActionType().equals(HoodieTimeline.COMMIT_ACTION)) {
|
||||
writeContext = metrics.getCommitCtx();
|
||||
} else {
|
||||
|
||||
@@ -18,6 +18,7 @@
|
||||
|
||||
package org.apache.hudi.client;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
@@ -68,6 +69,7 @@ public class HoodieReadClient<T extends HoodieRecordPayload> implements Serializ
|
||||
private HoodieTable hoodieTable;
|
||||
private transient Option<SQLContext> sqlContextOpt;
|
||||
private final transient JavaSparkContext jsc;
|
||||
private final transient Configuration hadoopConf;
|
||||
|
||||
/**
|
||||
* @param basePath path to Hoodie table
|
||||
@@ -93,10 +95,11 @@ public class HoodieReadClient<T extends HoodieRecordPayload> implements Serializ
|
||||
*/
|
||||
public HoodieReadClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) {
|
||||
this.jsc = jsc;
|
||||
this.hadoopConf = jsc.hadoopConfiguration();
|
||||
final String basePath = clientConfig.getBasePath();
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true);
|
||||
this.hoodieTable = HoodieTable.create(metaClient, clientConfig, jsc);
|
||||
this.hoodieTable = HoodieTable.create(metaClient, clientConfig, hadoopConf);
|
||||
this.index = HoodieIndex.createIndex(clientConfig);
|
||||
this.sqlContextOpt = Option.empty();
|
||||
}
|
||||
|
||||
@@ -139,7 +139,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
*/
|
||||
public JavaRDD<HoodieRecord<T>> filterExists(JavaRDD<HoodieRecord<T>> hoodieRecords) {
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable<T> table = HoodieTable.create(config, jsc);
|
||||
HoodieTable<T> table = HoodieTable.create(config, hadoopConf);
|
||||
Timer.Context indexTimer = metrics.getIndexCtx();
|
||||
JavaRDD<HoodieRecord<T>> recordsWithLocation = getIndex().tagLocation(hoodieRecords, jsc, table);
|
||||
metrics.updateIndexMetrics(LOOKUP_STR, metrics.getDurationInMs(indexTimer == null ? 0L : indexTimer.stop()));
|
||||
@@ -336,7 +336,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
}
|
||||
// We cannot have unbounded commit files. Archive commits if we have to archive
|
||||
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(config, createMetaClient(true));
|
||||
archiveLog.archiveIfRequired(jsc);
|
||||
archiveLog.archiveIfRequired(hadoopConf);
|
||||
if (config.isAutoClean()) {
|
||||
// Call clean to cleanup if there is anything to cleanup after the commit,
|
||||
LOG.info("Auto cleaning is enabled. Running cleaner now");
|
||||
@@ -356,7 +356,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
* @param comment - Comment for the savepoint
|
||||
*/
|
||||
public void savepoint(String user, String comment) {
|
||||
HoodieTable<T> table = HoodieTable.create(config, jsc);
|
||||
HoodieTable<T> table = HoodieTable.create(config, hadoopConf);
|
||||
if (table.getCompletedCommitsTimeline().empty()) {
|
||||
throw new HoodieSavepointException("Could not savepoint. Commit timeline is empty");
|
||||
}
|
||||
@@ -380,7 +380,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
* @param comment - Comment for the savepoint
|
||||
*/
|
||||
public void savepoint(String instantTime, String user, String comment) {
|
||||
HoodieTable<T> table = HoodieTable.create(config, jsc);
|
||||
HoodieTable<T> table = HoodieTable.create(config, hadoopConf);
|
||||
table.savepoint(jsc, instantTime, user, comment);
|
||||
}
|
||||
|
||||
@@ -392,7 +392,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
* @return true if the savepoint was deleted successfully
|
||||
*/
|
||||
public void deleteSavepoint(String savepointTime) {
|
||||
HoodieTable<T> table = HoodieTable.create(config, jsc);
|
||||
HoodieTable<T> table = HoodieTable.create(config, hadoopConf);
|
||||
SavepointHelpers.deleteSavepoint(table, savepointTime);
|
||||
}
|
||||
|
||||
@@ -407,7 +407,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
* @return true if the savepoint was restored to successfully
|
||||
*/
|
||||
public void restoreToSavepoint(String savepointTime) {
|
||||
HoodieTable<T> table = HoodieTable.create(config, jsc);
|
||||
HoodieTable<T> table = HoodieTable.create(config, hadoopConf);
|
||||
SavepointHelpers.validateSavepointPresence(table, savepointTime);
|
||||
restoreToInstant(savepointTime);
|
||||
SavepointHelpers.validateSavepointRestore(table, savepointTime);
|
||||
@@ -424,7 +424,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
final String rollbackInstantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
final Timer.Context context = this.metrics.getRollbackCtx();
|
||||
try {
|
||||
HoodieTable<T> table = HoodieTable.create(config, jsc);
|
||||
HoodieTable<T> table = HoodieTable.create(config, hadoopConf);
|
||||
Option<HoodieInstant> commitInstantOpt = Option.fromJavaOptional(table.getActiveTimeline().getCommitsTimeline().getInstants()
|
||||
.filter(instant -> HoodieActiveTimeline.EQUALS.test(instant.getTimestamp(), commitInstantTime))
|
||||
.findFirst());
|
||||
@@ -455,7 +455,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
final String restoreInstantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
Timer.Context context = metrics.getRollbackCtx();
|
||||
try {
|
||||
HoodieTable<T> table = HoodieTable.create(config, jsc);
|
||||
HoodieTable<T> table = HoodieTable.create(config, hadoopConf);
|
||||
HoodieRestoreMetadata restoreMetadata = table.restore(jsc, restoreInstantTime, instantTime);
|
||||
if (context != null) {
|
||||
final long durationInMs = metrics.getDurationInMs(context.stop());
|
||||
@@ -488,7 +488,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
public HoodieCleanMetadata clean(String cleanInstantTime) throws HoodieIOException {
|
||||
LOG.info("Cleaner started");
|
||||
final Timer.Context context = metrics.getCleanCtx();
|
||||
HoodieCleanMetadata metadata = HoodieTable.create(config, jsc).clean(jsc, cleanInstantTime);
|
||||
HoodieCleanMetadata metadata = HoodieTable.create(config, hadoopConf).clean(jsc, cleanInstantTime);
|
||||
if (context != null) {
|
||||
long durationMs = metrics.getDurationInMs(context.stop());
|
||||
metrics.updateCleanMetrics(durationMs, metadata.getTotalFilesDeleted());
|
||||
@@ -540,7 +540,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
HoodieTimeline.compareTimestamps(latestPending.getTimestamp(), HoodieTimeline.LESSER_THAN, instantTime),
|
||||
"Latest pending compaction instant time must be earlier than this instant time. Latest Compaction :"
|
||||
+ latestPending + ", Ingesting at " + instantTime));
|
||||
HoodieTable<T> table = HoodieTable.create(metaClient, config, jsc);
|
||||
HoodieTable<T> table = HoodieTable.create(metaClient, config, hadoopConf);
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
String commitActionType = table.getMetaClient().getCommitActionType();
|
||||
activeTimeline.createNewInstant(new HoodieInstant(State.REQUESTED, commitActionType, instantTime));
|
||||
@@ -564,7 +564,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
*/
|
||||
public boolean scheduleCompactionAtInstant(String instantTime, Option<Map<String, String>> extraMetadata) throws HoodieIOException {
|
||||
LOG.info("Scheduling compaction at instant time :" + instantTime);
|
||||
Option<HoodieCompactionPlan> plan = HoodieTable.create(config, jsc)
|
||||
Option<HoodieCompactionPlan> plan = HoodieTable.create(config, hadoopConf)
|
||||
.scheduleCompaction(jsc, instantTime, extraMetadata);
|
||||
return plan.isPresent();
|
||||
}
|
||||
@@ -588,7 +588,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
*/
|
||||
public void commitCompaction(String compactionInstantTime, JavaRDD<WriteStatus> writeStatuses,
|
||||
Option<Map<String, String>> extraMetadata) throws IOException {
|
||||
HoodieTable<T> table = HoodieTable.create(config, jsc);
|
||||
HoodieTable<T> table = HoodieTable.create(config, hadoopConf);
|
||||
HoodieCommitMetadata metadata = CompactHelpers.createCompactionMetadata(
|
||||
table, compactionInstantTime, writeStatuses, config.getSchema());
|
||||
extraMetadata.ifPresent(m -> m.forEach(metadata::addMetadata));
|
||||
@@ -634,7 +634,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
* Cleanup all pending commits.
|
||||
*/
|
||||
private void rollbackPendingCommits() {
|
||||
HoodieTable<T> table = HoodieTable.create(config, jsc);
|
||||
HoodieTable<T> table = HoodieTable.create(config, hadoopConf);
|
||||
HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction();
|
||||
List<String> commits = inflightTimeline.getReverseOrderedInstants().map(HoodieInstant::getTimestamp)
|
||||
.collect(Collectors.toList());
|
||||
@@ -650,7 +650,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
* @return RDD of Write Status
|
||||
*/
|
||||
private JavaRDD<WriteStatus> compact(String compactionInstantTime, boolean shouldComplete) {
|
||||
HoodieTable<T> table = HoodieTable.create(config, jsc);
|
||||
HoodieTable<T> table = HoodieTable.create(config, hadoopConf);
|
||||
HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline();
|
||||
HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(compactionInstantTime);
|
||||
if (pendingCompactionTimeline.containsInstant(inflightInstant)) {
|
||||
|
||||
@@ -20,6 +20,7 @@ package org.apache.hudi.table;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hudi.avro.model.HoodieCleanMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.avro.model.HoodieRestoreMetadata;
|
||||
@@ -86,8 +87,8 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieCopyOnWriteTable.class);
|
||||
|
||||
public HoodieCopyOnWriteTable(HoodieWriteConfig config, JavaSparkContext jsc, HoodieTableMetaClient metaClient) {
|
||||
super(config, jsc, metaClient);
|
||||
public HoodieCopyOnWriteTable(HoodieWriteConfig config, Configuration hadoopConf, HoodieTableMetaClient metaClient) {
|
||||
super(config, hadoopConf, metaClient);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -18,6 +18,7 @@
|
||||
|
||||
package org.apache.hudi.table;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.avro.model.HoodieRestoreMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
|
||||
@@ -70,8 +71,8 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieMergeOnReadTable.class);
|
||||
|
||||
HoodieMergeOnReadTable(HoodieWriteConfig config, JavaSparkContext jsc, HoodieTableMetaClient metaClient) {
|
||||
super(config, jsc, metaClient);
|
||||
HoodieMergeOnReadTable(HoodieWriteConfig config, Configuration hadoopConf, HoodieTableMetaClient metaClient) {
|
||||
super(config, hadoopConf, metaClient);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -89,10 +89,10 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
||||
|
||||
protected final SparkTaskContextSupplier sparkTaskContextSupplier = new SparkTaskContextSupplier();
|
||||
|
||||
protected HoodieTable(HoodieWriteConfig config, JavaSparkContext jsc, HoodieTableMetaClient metaClient) {
|
||||
protected HoodieTable(HoodieWriteConfig config, Configuration hadoopConf, HoodieTableMetaClient metaClient) {
|
||||
this.config = config;
|
||||
this.hadoopConfiguration = new SerializableConfiguration(jsc.hadoopConfiguration());
|
||||
this.viewManager = FileSystemViewManager.createViewManager(new SerializableConfiguration(jsc.hadoopConfiguration()),
|
||||
this.hadoopConfiguration = new SerializableConfiguration(hadoopConf);
|
||||
this.viewManager = FileSystemViewManager.createViewManager(new SerializableConfiguration(hadoopConf),
|
||||
config.getViewStorageConfig());
|
||||
this.metaClient = metaClient;
|
||||
this.index = HoodieIndex.createIndex(config);
|
||||
@@ -105,25 +105,25 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
||||
return viewManager;
|
||||
}
|
||||
|
||||
public static <T extends HoodieRecordPayload> HoodieTable<T> create(HoodieWriteConfig config, JavaSparkContext jsc) {
|
||||
public static <T extends HoodieRecordPayload> HoodieTable<T> create(HoodieWriteConfig config, Configuration hadoopConf) {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(
|
||||
jsc.hadoopConfiguration(),
|
||||
hadoopConf,
|
||||
config.getBasePath(),
|
||||
true,
|
||||
config.getConsistencyGuardConfig(),
|
||||
Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))
|
||||
);
|
||||
return HoodieTable.create(metaClient, config, jsc);
|
||||
return HoodieTable.create(metaClient, config, hadoopConf);
|
||||
}
|
||||
|
||||
public static <T extends HoodieRecordPayload> HoodieTable<T> create(HoodieTableMetaClient metaClient,
|
||||
HoodieWriteConfig config,
|
||||
JavaSparkContext jsc) {
|
||||
Configuration hadoopConf) {
|
||||
switch (metaClient.getTableType()) {
|
||||
case COPY_ON_WRITE:
|
||||
return new HoodieCopyOnWriteTable<>(config, jsc, metaClient);
|
||||
return new HoodieCopyOnWriteTable<>(config, hadoopConf, metaClient);
|
||||
case MERGE_ON_READ:
|
||||
return new HoodieMergeOnReadTable<>(config, jsc, metaClient);
|
||||
return new HoodieMergeOnReadTable<>(config, hadoopConf, metaClient);
|
||||
default:
|
||||
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
|
||||
}
|
||||
|
||||
@@ -18,6 +18,7 @@
|
||||
|
||||
package org.apache.hudi.table;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hudi.avro.model.HoodieArchivedMetaEntry;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
|
||||
@@ -53,7 +54,6 @@ import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
@@ -112,9 +112,9 @@ public class HoodieTimelineArchiveLog {
|
||||
/**
|
||||
* Check if commits need to be archived. If yes, archive commits.
|
||||
*/
|
||||
public boolean archiveIfRequired(final JavaSparkContext jsc) throws IOException {
|
||||
public boolean archiveIfRequired(final Configuration hadoopConf) throws IOException {
|
||||
try {
|
||||
List<HoodieInstant> instantsToArchive = getInstantsToArchive(jsc).collect(Collectors.toList());
|
||||
List<HoodieInstant> instantsToArchive = getInstantsToArchive(hadoopConf).collect(Collectors.toList());
|
||||
|
||||
boolean success = true;
|
||||
if (!instantsToArchive.isEmpty()) {
|
||||
@@ -133,13 +133,13 @@ public class HoodieTimelineArchiveLog {
|
||||
}
|
||||
}
|
||||
|
||||
private Stream<HoodieInstant> getInstantsToArchive(JavaSparkContext jsc) {
|
||||
private Stream<HoodieInstant> getInstantsToArchive(Configuration hadoopConf) {
|
||||
|
||||
// TODO : rename to max/minInstantsToKeep
|
||||
int maxCommitsToKeep = config.getMaxCommitsToKeep();
|
||||
int minCommitsToKeep = config.getMinCommitsToKeep();
|
||||
|
||||
HoodieTable table = HoodieTable.create(metaClient, config, jsc);
|
||||
HoodieTable table = HoodieTable.create(metaClient, config, hadoopConf);
|
||||
|
||||
// GroupBy each action and limit each action timeline to maxCommitsToKeep
|
||||
// TODO: Handle ROLLBACK_ACTION in future
|
||||
|
||||
@@ -182,7 +182,7 @@ public abstract class BaseCommitActionExecutor<T extends HoodieRecordPayload<T>>
|
||||
String actionType = table.getMetaClient().getCommitActionType();
|
||||
LOG.info("Committing " + instantTime + ", action Type " + actionType);
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable<T> table = HoodieTable.create(config, jsc);
|
||||
HoodieTable<T> table = HoodieTable.create(config, jsc.hadoopConfiguration());
|
||||
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
|
||||
|
||||
@@ -89,7 +89,7 @@ public class HoodieMergeOnReadTableCompactor implements HoodieCompactor {
|
||||
}
|
||||
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
|
||||
// Compacting is very similar to applying updates to existing file
|
||||
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc, metaClient);
|
||||
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc.hadoopConfiguration(), metaClient);
|
||||
List<CompactionOperation> operations = compactionPlan.getOperations().stream()
|
||||
.map(CompactionOperation::convertFromAvroRecordInstance).collect(toList());
|
||||
LOG.info("Compactor compacting " + operations + " files");
|
||||
|
||||
Reference in New Issue
Block a user