-
Notifications
You must be signed in to change notification settings - Fork 3.6k
Synchronize Delta log writing in tests too #28092
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
Merged
Merged
Changes from all commits
Commits
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
61 changes: 0 additions & 61 deletions
61
...java/io/trino/plugin/deltalake/transactionlog/writer/LocalTransactionLogSynchronizer.java
This file was deleted.
Oops, something went wrong.
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
86 changes: 86 additions & 0 deletions
86
.../trino/plugin/deltalake/transactionlog/writer/TestingLocalTransactionLogSynchronizer.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,86 @@ | ||
| /* | ||
| * Licensed 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 io.trino.plugin.deltalake.transactionlog.writer; | ||
|
|
||
| import io.trino.filesystem.Location; | ||
| import io.trino.filesystem.TrinoFileSystem; | ||
| import io.trino.filesystem.local.LocalFileSystem; | ||
| import io.trino.plugin.deltalake.DeltaLakeFileSystemFactory; | ||
| import io.trino.plugin.deltalake.metastore.VendedCredentialsHandle; | ||
| import io.trino.spi.connector.ConnectorSession; | ||
|
|
||
| import java.io.IOException; | ||
| import java.io.UncheckedIOException; | ||
| import java.nio.file.FileAlreadyExistsException; | ||
| import java.nio.file.Files; | ||
| import java.nio.file.Path; | ||
|
|
||
| import static io.trino.plugin.base.util.Closables.closeAllSuppress; | ||
| import static java.nio.file.StandardCopyOption.ATOMIC_MOVE; | ||
| import static java.nio.file.StandardOpenOption.CREATE_NEW; | ||
| import static java.util.Objects.requireNonNull; | ||
| import static java.util.UUID.randomUUID; | ||
|
|
||
| public class TestingLocalTransactionLogSynchronizer | ||
| implements TransactionLogSynchronizer | ||
| { | ||
| private final DeltaLakeFileSystemFactory fileSystemFactory; | ||
|
|
||
| public TestingLocalTransactionLogSynchronizer(DeltaLakeFileSystemFactory fileSystemFactory) | ||
|
findepi marked this conversation as resolved.
|
||
| { | ||
| this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null"); | ||
| } | ||
|
|
||
| @Override | ||
| public void write(ConnectorSession session, VendedCredentialsHandle credentialsHandle, String clusterId, Location newLogEntryPath, byte[] entryContents) | ||
| { | ||
| try { | ||
| TrinoFileSystem fileSystem = fileSystemFactory.create(session, credentialsHandle); | ||
| Path targetPath = ((LocalFileSystem) fileSystem).toFilePath(newLogEntryPath); | ||
| Files.createDirectories(targetPath.getParent()); | ||
| Path tmpPath = targetPath.resolveSibling(".tmp.%s.%s".formatted(targetPath.getFileName().toString(), randomUUID())); | ||
|
findepi marked this conversation as resolved.
|
||
| Files.write(tmpPath, entryContents, CREATE_NEW); | ||
| try { | ||
| // It's important that file is renamed atomically (hence the ATOMIC_MOVE option), as | ||
| // reads do not synchronize with the log writer in any way. | ||
| // Files.move with ATOMIC_MOVE is not guaranteed to be exclusive. | ||
|
findepi marked this conversation as resolved.
|
||
| // In fact, it can be seen to overwrite the target file if it already exists. | ||
| // In-memory synchronization should be sufficient for tests, | ||
| // assuming single TestingDeltaLakePlugin instance is used. | ||
| synchronized (this) { | ||
| if (Files.exists(targetPath)) { | ||
| throw new FileAlreadyExistsException(targetPath.toString()); | ||
| } | ||
| Files.move(tmpPath, targetPath, ATOMIC_MOVE); | ||
| } | ||
| } | ||
| catch (Throwable t) { | ||
| closeAllSuppress(t, () -> Files.deleteIfExists(tmpPath)); | ||
| throw t; | ||
| } | ||
| } | ||
| catch (FileAlreadyExistsException e) { | ||
| throw new TransactionConflictException("Conflict detected while writing Transaction Log entry " + newLogEntryPath, e); | ||
| } | ||
| catch (IOException e) { | ||
| throw new UncheckedIOException(e); | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public boolean isUnsafe() | ||
| { | ||
| return false; | ||
| } | ||
| } | ||
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
This test failed on master https://github.com/trinodb/trino/actions/runs/22487358862/job/65140354090