-
Notifications
You must be signed in to change notification settings - Fork 2.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Remove unneeded metadata read during update event generation #11829
base: main
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,79 @@ | ||
/* | ||
* 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.iceberg; | ||
|
||
import static org.assertj.core.api.Assertions.assertThat; | ||
|
||
import java.util.Arrays; | ||
import java.util.List; | ||
import org.junit.jupiter.api.TestTemplate; | ||
import org.junit.jupiter.api.extension.ExtendWith; | ||
|
||
@ExtendWith(ParameterizedTestExtension.class) | ||
public class TestPendingUpdateUpdateEvents extends TestBase { | ||
@Parameters(name = "formatVersion = {0}") | ||
protected static List<Object> parameters() { | ||
return Arrays.asList(1, 2, 3); | ||
} | ||
|
||
@TestTemplate | ||
public void testFastAppendDoesNotPerformExtraMetadataReads() { | ||
table.newFastAppend().appendFile(FILE_A).commit(); | ||
long startSnapshotId = table.currentSnapshot().snapshotId(); | ||
|
||
AppendFiles appendFiles = table.newFastAppend().appendFile(FILE_A); | ||
int commitOnlyFetchCount = recordMetadataFetchesForCommitOnly(appendFiles); | ||
|
||
assertThat(table.currentSnapshot().snapshotId()).isNotEqualTo(startSnapshotId); | ||
// fetch metadata once | ||
assertThat(commitOnlyFetchCount).isEqualTo(1); | ||
} | ||
|
||
@TestTemplate | ||
public void testCherryPickOperationDoesNotPerformExtraMetadataReads() { | ||
table.newFastAppend().appendFile(FILE_A).commit(); | ||
long startSnapshotId = table.currentSnapshot().snapshotId(); | ||
|
||
table.newAppend().appendFile(FILE_B).stageOnly().commit(); | ||
Snapshot stagedSnapshot = readMetadata().snapshots().get(1); | ||
|
||
ManageSnapshots manageSnapshots = | ||
table.manageSnapshots().cherrypick(stagedSnapshot.snapshotId()); | ||
int commitOnlyFetchCount = recordMetadataFetchesForCommitOnly(manageSnapshots); | ||
|
||
assertThat(table.currentSnapshot().snapshotId()).isNotEqualTo(startSnapshotId); | ||
// fetch metadata twice, once in BaseTransaction#applyUpdates and once normally | ||
assertThat(commitOnlyFetchCount).isEqualTo(2); | ||
} | ||
|
||
private <T> int recordMetadataFetchesForCommitOnly(PendingUpdate<T> pendingUpdate) { | ||
pendingUpdate.apply(); | ||
|
||
// determine number of metadata fetches during apply | ||
int beforeApplyFetchCount = table.ops().getMetadataFetchCount(); | ||
pendingUpdate.apply(); | ||
int afterApplyFetchCount = table.ops().getMetadataFetchCount(); | ||
int applyOnlyFetchCount = afterApplyFetchCount - beforeApplyFetchCount; | ||
|
||
// determine number of metadata fetches during commit | ||
pendingUpdate.commit(); | ||
int afterCommitFetchCount = table.ops().getMetadataFetchCount(); | ||
return afterCommitFetchCount - afterApplyFetchCount - applyOnlyFetchCount; | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -22,6 +22,7 @@ | |
|
||
import java.io.File; | ||
import java.util.Map; | ||
import java.util.concurrent.atomic.AtomicInteger; | ||
import org.apache.iceberg.exceptions.AlreadyExistsException; | ||
import org.apache.iceberg.exceptions.CommitFailedException; | ||
import org.apache.iceberg.exceptions.CommitStateUnknownException; | ||
|
@@ -220,6 +221,7 @@ public static class TestTableOperations implements TableOperations { | |
private TableMetadata current = null; | ||
private long lastSnapshotId = 0; | ||
private int failCommits = 0; | ||
private final AtomicInteger metadataFetchCount = new AtomicInteger(); | ||
|
||
public TestTableOperations(String tableName, File location) { | ||
this.tableName = tableName; | ||
|
@@ -255,8 +257,13 @@ void failCommits(int numFailures) { | |
this.failCommits = numFailures; | ||
} | ||
|
||
int getMetadataFetchCount() { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Decided to count calls to current()/refresh() as a proxy for reading metadata from objectstorage. For TestTables (which uses in memory metadata) that is the best we can do. Is there some other test I could write that uses real file system metadata (not TestTables)? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why current/refresh counting is not great: cached metadata fetches are free while uncached fetches are not, and this does not differentiate between the two. Instrumenting at the fileio level would let us validate indeed only a single uncached metadata request is needed for the whole commit. |
||
return metadataFetchCount.get(); | ||
} | ||
|
||
@Override | ||
public TableMetadata current() { | ||
metadataFetchCount.incrementAndGet(); | ||
return current; | ||
} | ||
|
||
|
@@ -265,6 +272,7 @@ public TableMetadata refresh() { | |
synchronized (METADATA) { | ||
this.current = METADATA.get(tableName); | ||
} | ||
metadataFetchCount.incrementAndGet(); | ||
return current; | ||
} | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -114,7 +114,10 @@ public void testMetadataFileLocationsWithMissingFiles() { | |
// delete v3.metadata.json making v2.metadata.json and v1.metadata.json inaccessible | ||
table.io().deleteFile(location); | ||
|
||
Set<String> metadataFileLocations = ReachableFileUtil.metadataFileLocations(table, true); | ||
// original hadoop table will not see the file deletion | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is side effect of removing This is mostly a problem for hadoop/filesystem tables and these are deprecated: https://iceberg.apache.org/spec/#file-system-tables For metastore catalogs for example, |
||
Table refreshedTable = TABLES.load(tableDir.toURI().toString()); | ||
Set<String> metadataFileLocations = | ||
ReachableFileUtil.metadataFileLocations(refreshedTable, true); | ||
assertThat(metadataFileLocations).hasSize(2); | ||
} | ||
|
||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
PendingUpdate.updateEvent
only usage is in SnapshotProducer currently so could change the interface directly toupdateEvent(Snapshot committedSnapshot)
, but did not want to make a backwards incompatible API changeThere was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
SnapshotProducer
is package private, so I think we're OK in terms of backwards compatibility since it's not like a public API is being broken.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Was trying to provide context why I didn't modify
PendingUpdate.updateEvent
directly (it is a public API)But agree with you is safe to modify SnapshotProducer here.