Closed davseitsev closed 7 months ago
I remember we had this issue earlier. Here is the PR with some discussion. The idea suggested there was either contribute a custom FileIO
implementation in Iceberg or use Hadoop FS that handles it already.
cc @rdblue
I feel like we need to agree on the best solution and document it.
Here is the snippet from create
in S3AfileSystem
:
try {
// get the status or throw an FNFE
status = getFileStatus(path);
// if the thread reaches here, there is something at the path
if (status.isDirectory()) {
// path references a directory: automatic error
throw new FileAlreadyExistsException(path + " is a directory");
}
if (!overwrite) {
// path references a file and overwrite is disabled
throw new FileAlreadyExistsException(path + " already exists");
}
LOG.debug("Overwriting file {}", path);
} catch (FileNotFoundException e) {
// this means the file is not found
}
Looks like getFileStatus
breaks this logic and even setting the overwrite flag does not help. At least, in S3AFileSystem
.
Our S3 file system doesn't do the getFileStatus
check when overwriting, which is how we avoid this. I think the best way to handle this is to add a certain amount of retries at the FileIO
layer.
We already have retries for table.refresh()
(in BaseMetastoreTableOperations
) that take care of most of these issues for the root metadata. The reason why we added the retries there was to avoid adding extra latency when there actually is a problem -- when a data file is missing, for example. But, it looks like it would be better to have retries for at least metadata files when using S3.
I think it makes sense to add retries to HadoopInputFile
and to make the number of retries and total retry timeout configurable either in table properties or Hadoop Configuration. The downside to adding retries there is that we'd have retries for data files as well, but that seems like a reasonable trade-off to me.
Hi @davseitsev, we're also using s3a (version 2.8.4) and I'm wondering if you see this frequently or within a single operation? We're starting to have an issue which we can't identify within a single create() but it sounds like this happens from a create() and then open()?
This issue has been automatically marked as stale because it has been open for 180 days with no activity. It will be closed in next 14 days if no further activity occurs. To permanently prevent this issue from being considered stale, add the label 'not-stale', but commenting on the issue is preferred when possible.
This issue has been closed because it has not received any activity in the last 14 days since being marked as 'stale'
I'm building Spark structured streaming application which writes data to Amazon S3 in Iceberg format. Sometimes the query fails without reties dues to S3 eventual consistency.
As far as I understand it happens in
SnapshotProducer
during commit operation.FastAppend
tries to read manifest list inapply()
method and getjava.io.FileNotFoundException
inS3AFileSystem.getFileStatus(S3AFileSystem.java:1642)
. But actually file exists.It's a known issue with S3AFileSystem, it checks whether file exists before creating a file and it breaks read-after-write strong consistency. And when another client want to read newly created file it can get FNF exception. The problem is that
SnapshotProducer
reties onlyCommitFailedException
and doesn't retry anyIOException
.