danielcweeks commented on code in PR #14264:
URL: https://github.com/apache/iceberg/pull/14264#discussion_r2535415803
##########
core/src/main/java/org/apache/iceberg/BaseIncrementalChangelogScan.java:
##########
@@ -133,13 +185,512 @@ private static Map<Long, Integer>
computeSnapshotOrdinals(Deque<Snapshot> snapsh
return snapshotOrdinals;
}
+ /**
+ * Builds a delete file index for existing deletes that were present before
the start snapshot.
+ * These deletes should be applied to data files but should not generate
DELETE changelog rows.
+ * Uses manifest pruning and caching to optimize performance.
+ */
+ private DeleteFileIndex buildExistingDeleteIndex(Long
fromSnapshotIdExclusive) {
+ if (fromSnapshotIdExclusive == null) {
+ return DeleteFileIndex.emptyIndex();
+ }
+ Snapshot fromSnapshot = table().snapshot(fromSnapshotIdExclusive);
+ Preconditions.checkState(
+ fromSnapshot != null, "Cannot find starting snapshot: %s",
fromSnapshotIdExclusive);
+
+ List<ManifestFile> existingDeleteManifests =
fromSnapshot.deleteManifests(table().io());
+ if (existingDeleteManifests.isEmpty()) {
+ return DeleteFileIndex.emptyIndex();
+ }
+
+ // Prune manifests based on partition filter to avoid processing
irrelevant manifests
+ List<ManifestFile> prunedManifests =
pruneManifestsByPartition(existingDeleteManifests);
+ if (prunedManifests.isEmpty()) {
+ return DeleteFileIndex.emptyIndex();
+ }
+
+ // Load delete files from manifests
+ List<DeleteFile> deleteFiles = loadDeleteFiles(prunedManifests);
+
+ return DeleteFileIndex.builderFor(deleteFiles)
+ .specsById(table().specs())
+ .caseSensitive(isCaseSensitive())
+ .build();
+ }
+
+ /**
+ * Wrapper method that tracks build calls and caches the result for reuse.
This ensures we only
+ * build the index once even if called from multiple places.
+ */
+ private DeleteFileIndex buildExistingDeleteIndexTracked(Long
fromSnapshotIdExclusive) {
+ if (cachedExistingDeleteIndex != null) {
+ return cachedExistingDeleteIndex;
+ }
+ existingDeleteIndexBuildCallCount++;
+ cachedExistingDeleteIndex =
buildExistingDeleteIndex(fromSnapshotIdExclusive);
+ return cachedExistingDeleteIndex;
+ }
+
+ // Visible for testing
+ int getExistingDeleteIndexBuildCallCount() {
+ return existingDeleteIndexBuildCallCount;
+ }
+
+ // Visible for testing
+ boolean wasExistingDeleteIndexBuilt() {
+ return existingDeleteIndexBuildCallCount > 0;
+ }
+
+ /**
+ * Builds per-snapshot delete file indexes for newly added delete files in
each changelog
+ * snapshot. These deletes should generate DELETE changelog rows. Uses
caching to avoid re-parsing
+ * manifests.
+ */
+ private Map<Long, DeleteFileIndex> buildAddedDeleteIndexes(Deque<Snapshot>
changelogSnapshots) {
+ Map<Long, DeleteFileIndex> addedDeletesBySnapshot = Maps.newHashMap();
+
+ for (Snapshot snapshot : changelogSnapshots) {
+ List<ManifestFile> snapshotDeleteManifests =
snapshot.deleteManifests(table().io());
+ if (snapshotDeleteManifests.isEmpty()) {
+ addedDeletesBySnapshot.put(snapshot.snapshotId(),
DeleteFileIndex.emptyIndex());
+ continue;
+ }
+
+ // Filter to only include delete files added in this snapshot
+ List<ManifestFile> addedDeleteManifests =
+ FluentIterable.from(snapshotDeleteManifests)
Review Comment:
I don't think we should be using the guava streams where we can use the
native streams implementations. Just use
`snapshotDeleteManifests.stream().filter`
##########
core/src/main/java/org/apache/iceberg/BaseIncrementalChangelogScan.java:
##########
@@ -133,13 +185,512 @@ private static Map<Long, Integer>
computeSnapshotOrdinals(Deque<Snapshot> snapsh
return snapshotOrdinals;
}
+ /**
+ * Builds a delete file index for existing deletes that were present before
the start snapshot.
+ * These deletes should be applied to data files but should not generate
DELETE changelog rows.
+ * Uses manifest pruning and caching to optimize performance.
+ */
+ private DeleteFileIndex buildExistingDeleteIndex(Long
fromSnapshotIdExclusive) {
+ if (fromSnapshotIdExclusive == null) {
+ return DeleteFileIndex.emptyIndex();
+ }
+ Snapshot fromSnapshot = table().snapshot(fromSnapshotIdExclusive);
+ Preconditions.checkState(
+ fromSnapshot != null, "Cannot find starting snapshot: %s",
fromSnapshotIdExclusive);
+
+ List<ManifestFile> existingDeleteManifests =
fromSnapshot.deleteManifests(table().io());
+ if (existingDeleteManifests.isEmpty()) {
+ return DeleteFileIndex.emptyIndex();
+ }
+
+ // Prune manifests based on partition filter to avoid processing
irrelevant manifests
+ List<ManifestFile> prunedManifests =
pruneManifestsByPartition(existingDeleteManifests);
+ if (prunedManifests.isEmpty()) {
+ return DeleteFileIndex.emptyIndex();
+ }
+
+ // Load delete files from manifests
+ List<DeleteFile> deleteFiles = loadDeleteFiles(prunedManifests);
+
+ return DeleteFileIndex.builderFor(deleteFiles)
+ .specsById(table().specs())
+ .caseSensitive(isCaseSensitive())
+ .build();
+ }
+
+ /**
+ * Wrapper method that tracks build calls and caches the result for reuse.
This ensures we only
+ * build the index once even if called from multiple places.
+ */
+ private DeleteFileIndex buildExistingDeleteIndexTracked(Long
fromSnapshotIdExclusive) {
+ if (cachedExistingDeleteIndex != null) {
+ return cachedExistingDeleteIndex;
+ }
+ existingDeleteIndexBuildCallCount++;
+ cachedExistingDeleteIndex =
buildExistingDeleteIndex(fromSnapshotIdExclusive);
+ return cachedExistingDeleteIndex;
+ }
+
+ // Visible for testing
+ int getExistingDeleteIndexBuildCallCount() {
+ return existingDeleteIndexBuildCallCount;
+ }
+
+ // Visible for testing
+ boolean wasExistingDeleteIndexBuilt() {
+ return existingDeleteIndexBuildCallCount > 0;
+ }
+
+ /**
+ * Builds per-snapshot delete file indexes for newly added delete files in
each changelog
+ * snapshot. These deletes should generate DELETE changelog rows. Uses
caching to avoid re-parsing
+ * manifests.
+ */
+ private Map<Long, DeleteFileIndex> buildAddedDeleteIndexes(Deque<Snapshot>
changelogSnapshots) {
+ Map<Long, DeleteFileIndex> addedDeletesBySnapshot = Maps.newHashMap();
+
+ for (Snapshot snapshot : changelogSnapshots) {
+ List<ManifestFile> snapshotDeleteManifests =
snapshot.deleteManifests(table().io());
Review Comment:
Since we have to load the manifest lists for each snapshot, we should be
doing this in parallel using `Tasks`
##########
core/src/main/java/org/apache/iceberg/BaseIncrementalChangelogScan.java:
##########
@@ -89,7 +121,32 @@ protected CloseableIterable<ChangelogScanTask> doPlanFiles(
manifestGroup = manifestGroup.planWith(planExecutor());
}
- return manifestGroup.plan(new
CreateDataFileChangeTasks(changelogSnapshots));
+ // Create a supplier that reuses already-built index or builds lazily when
first DELETED entry
+ // is encountered
+ Supplier<DeleteFileIndex> existingDeleteIndexSupplier =
+ () -> {
+ if (cachedExistingDeleteIndex != null) {
+ return cachedExistingDeleteIndex;
+ }
+ return buildExistingDeleteIndexTracked(fromSnapshotIdExclusive);
+ };
+
+ // Plan data file tasks (ADDED and DELETED)
+ CloseableIterable<ChangelogScanTask> dataFileTasks =
+ manifestGroup.plan(
+ new CreateDataFileChangeTasks(
+ changelogSnapshots,
+ existingDeleteIndexSupplier,
+ addedDeletesBySnapshot,
+ table().specs(),
+ isCaseSensitive()));
+
+ // Find EXISTING data files affected by newly added delete files and
create tasks for them
+ CloseableIterable<ChangelogScanTask> deletedRowsTasks =
+ planDeletedRowsTasks(
+ changelogSnapshots, existingDeleteIndex, addedDeletesBySnapshot,
changelogSnapshotIds);
+
+ return CloseableIterable.concat(ImmutableList.of(dataFileTasks,
deletedRowsTasks));
Review Comment:
It's a little unclear to me whether we're observing the correct orderedness
by just concatenating these together. I'm not sure what the expectations are.
##########
core/src/main/java/org/apache/iceberg/BaseIncrementalChangelogScan.java:
##########
@@ -179,5 +735,56 @@ public CloseableIterable<ChangelogScanTask> apply(
}
});
}
+
+ /**
+ * Gets delete files that apply to an ADDED data file. Only includes
deletes added in the same
+ * snapshot as the file.
+ */
+ private DeleteFile[] getDeletesForAddedFile(
+ ManifestEntry<DataFile> entry, long commitSnapshotId) {
+ DeleteFileIndex addedDeleteIndex =
addedDeletesBySnapshot.get(commitSnapshotId);
+ return addedDeleteIndex == null || addedDeleteIndex.isEmpty()
+ ? NO_DELETES
+ : addedDeleteIndex.forEntry(entry);
+ }
+
+ /**
+ * Gets all delete files that were applied to a DELETED data file up to
the point it was
+ * deleted. This includes existing deletes and all deletes added in the
scan range up to (but
+ * not including) the deletion snapshot.
+ */
+ private DeleteFile[] getDeletesForDeletedFile(
+ ManifestEntry<DataFile> entry, long deletionSnapshotId) {
+
+ List<DeleteFile> allDeletes = Lists.newArrayList();
+
+ // Build existing delete index lazily when first DELETED entry is
encountered
+ DeleteFileIndex existingDeleteIndex = existingDeleteIndexSupplier.get();
+ DeleteFile[] existingDeletes =
+ existingDeleteIndex.isEmpty() ? NO_DELETES :
existingDeleteIndex.forEntry(entry);
+ for (DeleteFile df : existingDeletes) {
+ allDeletes.add(df);
+ }
+
+ // Add all deletes from snapshots in the scan range BEFORE the deletion
+ List<DeleteFile> cumulativeDeletes =
cumulativeDeletesMap.get(deletionSnapshotId);
+ if (cumulativeDeletes != null) {
+ // For each cumulative delete, check if it applies to this entry
+ for (DeleteFile df : cumulativeDeletes) {
Review Comment:
This feels super expensive. Don't we know that it applies? If there's an
existing POS/DV against a removed file, it would have to apply. If there's an
equality delete, you would just include it regardless and let the task apply it?
##########
core/src/main/java/org/apache/iceberg/BaseIncrementalChangelogScan.java:
##########
@@ -133,13 +185,512 @@ private static Map<Long, Integer>
computeSnapshotOrdinals(Deque<Snapshot> snapsh
return snapshotOrdinals;
}
+ /**
+ * Builds a delete file index for existing deletes that were present before
the start snapshot.
+ * These deletes should be applied to data files but should not generate
DELETE changelog rows.
+ * Uses manifest pruning and caching to optimize performance.
+ */
+ private DeleteFileIndex buildExistingDeleteIndex(Long
fromSnapshotIdExclusive) {
+ if (fromSnapshotIdExclusive == null) {
+ return DeleteFileIndex.emptyIndex();
+ }
+ Snapshot fromSnapshot = table().snapshot(fromSnapshotIdExclusive);
+ Preconditions.checkState(
+ fromSnapshot != null, "Cannot find starting snapshot: %s",
fromSnapshotIdExclusive);
+
+ List<ManifestFile> existingDeleteManifests =
fromSnapshot.deleteManifests(table().io());
+ if (existingDeleteManifests.isEmpty()) {
+ return DeleteFileIndex.emptyIndex();
+ }
+
+ // Prune manifests based on partition filter to avoid processing
irrelevant manifests
+ List<ManifestFile> prunedManifests =
pruneManifestsByPartition(existingDeleteManifests);
+ if (prunedManifests.isEmpty()) {
+ return DeleteFileIndex.emptyIndex();
+ }
+
+ // Load delete files from manifests
+ List<DeleteFile> deleteFiles = loadDeleteFiles(prunedManifests);
+
+ return DeleteFileIndex.builderFor(deleteFiles)
+ .specsById(table().specs())
+ .caseSensitive(isCaseSensitive())
+ .build();
+ }
+
+ /**
+ * Wrapper method that tracks build calls and caches the result for reuse.
This ensures we only
+ * build the index once even if called from multiple places.
+ */
+ private DeleteFileIndex buildExistingDeleteIndexTracked(Long
fromSnapshotIdExclusive) {
+ if (cachedExistingDeleteIndex != null) {
+ return cachedExistingDeleteIndex;
+ }
+ existingDeleteIndexBuildCallCount++;
+ cachedExistingDeleteIndex =
buildExistingDeleteIndex(fromSnapshotIdExclusive);
+ return cachedExistingDeleteIndex;
+ }
+
+ // Visible for testing
+ int getExistingDeleteIndexBuildCallCount() {
+ return existingDeleteIndexBuildCallCount;
+ }
+
+ // Visible for testing
+ boolean wasExistingDeleteIndexBuilt() {
+ return existingDeleteIndexBuildCallCount > 0;
+ }
+
+ /**
+ * Builds per-snapshot delete file indexes for newly added delete files in
each changelog
+ * snapshot. These deletes should generate DELETE changelog rows. Uses
caching to avoid re-parsing
+ * manifests.
+ */
+ private Map<Long, DeleteFileIndex> buildAddedDeleteIndexes(Deque<Snapshot>
changelogSnapshots) {
+ Map<Long, DeleteFileIndex> addedDeletesBySnapshot = Maps.newHashMap();
+
+ for (Snapshot snapshot : changelogSnapshots) {
+ List<ManifestFile> snapshotDeleteManifests =
snapshot.deleteManifests(table().io());
+ if (snapshotDeleteManifests.isEmpty()) {
+ addedDeletesBySnapshot.put(snapshot.snapshotId(),
DeleteFileIndex.emptyIndex());
+ continue;
+ }
+
+ // Filter to only include delete files added in this snapshot
+ List<ManifestFile> addedDeleteManifests =
+ FluentIterable.from(snapshotDeleteManifests)
+ .filter(manifest ->
manifest.snapshotId().equals(snapshot.snapshotId()))
+ .toList();
+
+ if (addedDeleteManifests.isEmpty()) {
+ addedDeletesBySnapshot.put(snapshot.snapshotId(),
DeleteFileIndex.emptyIndex());
+ } else {
+ // Load delete files from manifests
+ List<DeleteFile> deleteFiles = loadDeleteFiles(addedDeleteManifests);
+
+ DeleteFileIndex index =
+ DeleteFileIndex.builderFor(deleteFiles)
+ .specsById(table().specs())
+ .caseSensitive(isCaseSensitive())
+ .build();
+ addedDeletesBySnapshot.put(snapshot.snapshotId(), index);
+ }
+ }
+
+ return addedDeletesBySnapshot;
+ }
+
+ /**
+ * Plans tasks for EXISTING data files that are affected by newly added
delete files. These files
+ * were not added or deleted in the changelog snapshot range, but have new
delete files applied to
+ * them.
+ */
+ private CloseableIterable<ChangelogScanTask> planDeletedRowsTasks(
+ Deque<Snapshot> changelogSnapshots,
+ DeleteFileIndex existingDeleteIndex,
+ Map<Long, DeleteFileIndex> addedDeletesBySnapshot,
+ Set<Long> changelogSnapshotIds) {
+
+ Map<Long, Integer> snapshotOrdinals =
computeSnapshotOrdinals(changelogSnapshots);
+ List<ChangelogScanTask> tasks = Lists.newArrayList();
+
+ // Build a map of file statuses for each snapshot
+ Map<Long, Map<String, ManifestEntry.Status>> fileStatusBySnapshot =
+ buildFileStatusBySnapshot(changelogSnapshots, changelogSnapshotIds);
+
+ // Accumulate actual DeleteFile entries chronologically
+ List<DeleteFile> accumulatedDeletes = Lists.newArrayList();
+
+ // Start with deletes from before the changelog range
+ if (!existingDeleteIndex.isEmpty()) {
+ for (DeleteFile df : existingDeleteIndex.referencedDeleteFiles()) {
+ accumulatedDeletes.add(df);
+ }
+ }
+
+ for (Snapshot snapshot : changelogSnapshots) {
+ DeleteFileIndex addedDeleteIndex =
addedDeletesBySnapshot.get(snapshot.snapshotId());
+ if (addedDeleteIndex.isEmpty()) {
+ continue;
+ }
+
+ DeleteFileIndex cumulativeDeleteIndex =
buildDeleteIndex(accumulatedDeletes);
+
+ // Process data files for this snapshot
+ // Use a local set per snapshot to track processed files
+ Set<String> alreadyProcessedPaths = Sets.newHashSet();
+ processSnapshotForDeletedRowsTasks(
+ snapshot,
+ addedDeleteIndex,
+ cumulativeDeleteIndex,
+ fileStatusBySnapshot.get(snapshot.snapshotId()),
+ alreadyProcessedPaths,
+ snapshotOrdinals,
+ tasks);
+
+ // Accumulate this snapshot's added deletes for subsequent snapshots
+ for (DeleteFile df : addedDeleteIndex.referencedDeleteFiles()) {
+ accumulatedDeletes.add(df);
+ }
+ }
+
+ return CloseableIterable.withNoopClose(tasks);
+ }
+
+ /**
+ * Builds a map of file statuses for each snapshot, tracking which files
were added or deleted in
+ * each snapshot.
+ */
+ private Map<Long, Map<String, ManifestEntry.Status>>
buildFileStatusBySnapshot(
+ Deque<Snapshot> changelogSnapshots, Set<Long> changelogSnapshotIds) {
+
+ Map<Long, Map<String, ManifestEntry.Status>> fileStatusBySnapshot =
Maps.newHashMap();
+
+ for (Snapshot snapshot : changelogSnapshots) {
+ Map<String, ManifestEntry.Status> fileStatuses = Maps.newHashMap();
+
+ List<ManifestFile> changedDataManifests =
+ FluentIterable.from(snapshot.dataManifests(table().io()))
+ .filter(manifest ->
manifest.snapshotId().equals(snapshot.snapshotId()))
+ .toList();
+
+ ManifestGroup changedGroup =
+ new ManifestGroup(table().io(), changedDataManifests,
ImmutableList.of())
+ .specsById(table().specs())
+ .caseSensitive(isCaseSensitive())
+ .select(scanColumns())
+ .filterData(filter())
+ .ignoreExisting()
+ .columnsToKeepStats(columnsToKeepStats());
+
+ try (CloseableIterable<ManifestEntry<DataFile>> entries =
changedGroup.entries()) {
+ for (ManifestEntry<DataFile> entry : entries) {
+ if (changelogSnapshotIds.contains(entry.snapshotId())) {
+ fileStatuses.put(entry.file().location(), entry.status());
+ }
+ }
+ } catch (Exception e) {
+ throw new RuntimeException("Failed to collect file statuses", e);
+ }
+
+ fileStatusBySnapshot.put(snapshot.snapshotId(), fileStatuses);
+ }
+
+ return fileStatusBySnapshot;
+ }
+
+ /** Builds a delete index from the accumulated list of delete files. */
+ private DeleteFileIndex buildDeleteIndex(List<DeleteFile>
accumulatedDeletes) {
+ if (accumulatedDeletes.isEmpty()) {
+ return DeleteFileIndex.emptyIndex();
+ }
+
+ return DeleteFileIndex.builderFor(accumulatedDeletes)
+ .specsById(table().specs())
+ .caseSensitive(isCaseSensitive())
+ .build();
+ }
+
+ /**
+ * Processes data files for a snapshot to create DeletedRowsScanTask for
existing files affected
+ * by new delete files.
+ */
+ private void processSnapshotForDeletedRowsTasks(
+ Snapshot snapshot,
+ DeleteFileIndex addedDeleteIndex,
+ DeleteFileIndex cumulativeDeleteIndex,
+ Map<String, ManifestEntry.Status> currentSnapshotFiles,
+ Set<String> alreadyProcessedPaths,
+ Map<Long, Integer> snapshotOrdinals,
+ List<ChangelogScanTask> tasks) {
+
+ // Get all data files that exist in this snapshot
+ List<ManifestFile> allDataManifests = snapshot.dataManifests(table().io());
+ ManifestGroup allDataGroup =
+ new ManifestGroup(table().io(), allDataManifests, ImmutableList.of())
+ .specsById(table().specs())
+ .caseSensitive(isCaseSensitive())
+ .select(scanColumns())
+ .filterData(filter())
+ .ignoreDeleted()
+ .columnsToKeepStats(columnsToKeepStats());
+
+ if (shouldIgnoreResiduals()) {
+ allDataGroup = allDataGroup.ignoreResiduals();
+ }
+
+ String schemaString = SchemaParser.toJson(schema());
+
+ // Cache per specId - same for all files with same specId
+ Map<Integer, String> specStringCache = Maps.newHashMap();
+ Map<Integer, ResidualEvaluator> residualCache = Maps.newHashMap();
+ Expression residualFilter = shouldIgnoreResiduals() ?
Expressions.alwaysTrue() : filter();
+
+ try (CloseableIterable<ManifestEntry<DataFile>> entries =
allDataGroup.entries()) {
+ for (ManifestEntry<DataFile> entry : entries) {
+ DataFile dataFile = entry.file();
+ String filePath = dataFile.location();
+
+ // Skip if this file was ADDED or DELETED in this snapshot
+ // (those are handled by CreateDataFileChangeTasks)
+ if (currentSnapshotFiles.containsKey(filePath)) {
+ continue;
+ }
+
+ // Skip if we already created a task for this file in this snapshot
+ // Note: alreadyProcessedPaths is local to this snapshot's processing
+ if (alreadyProcessedPaths.contains(filePath)) {
+ continue;
+ }
+
+ // Check if this data file is affected by newly added delete files
+ DeleteFile[] addedDeletes = addedDeleteIndex.forEntry(entry);
+ if (addedDeletes.length == 0) {
+ continue;
+ }
+
+ // This data file was EXISTING but has new delete files applied
+ // Get existing deletes from before this snapshot (cumulative)
+ DeleteFile[] existingDeletes =
+ cumulativeDeleteIndex.isEmpty()
+ ? new DeleteFile[0]
+ : cumulativeDeleteIndex.forEntry(entry);
+
+ // Create a DeletedRowsScanTask
+ int changeOrdinal = snapshotOrdinals.get(snapshot.snapshotId());
+
+ // Use cached values (calculate once per specId)
+ int specId = dataFile.specId();
+ String specString =
+ specStringCache.computeIfAbsent(
+ specId, id ->
PartitionSpecParser.toJson(table().specs().get(id)));
+ ResidualEvaluator residuals =
+ residualCache.computeIfAbsent(
+ specId,
+ id -> {
+ PartitionSpec spec = table().specs().get(id);
+ return ResidualEvaluator.of(spec, residualFilter,
isCaseSensitive());
+ });
+
+ tasks.add(
+ new BaseDeletedRowsScanTask(
+ changeOrdinal,
+ snapshot.snapshotId(),
+ dataFile.copy(shouldKeepStats()),
+ addedDeletes,
+ existingDeletes,
+ schemaString,
+ specString,
+ residuals));
+
+ // Mark this file as processed for this snapshot
+ alreadyProcessedPaths.add(filePath);
+ }
+ } catch (Exception e) {
+ throw new RuntimeException("Failed to plan deleted rows tasks", e);
+ }
+ }
+
+ private boolean shouldKeepStats() {
+ Set<Integer> columns = columnsToKeepStats();
+ return columns != null && !columns.isEmpty();
+ }
+
+ /**
+ * Loads delete files from manifests by parsing each manifest.
+ *
+ * @param manifests the delete manifests to load
+ * @return list of delete files
+ */
+ private List<DeleteFile> loadDeleteFiles(List<ManifestFile> manifests) {
Review Comment:
This feels like something we should be parallelizing using `Tasks`
--
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.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]