trino icon indicating copy to clipboard operation
trino copied to clipboard

Trino Insert query failing while moving data from temp location to original location

Open BalaMahesh opened this issue 3 years ago • 9 comments

We are running the insert query in trino(version : 366) , we are getting the below error intermittently

Query 20220105_080332_00024_8knjy failed: Error moving data files from gs://xxx-gcs-prod/data0/logs/trino/e1dafcd1-a616-4205-b2ed-73025fd6a828/set=e578fef26a6d451bb6ee268d3c0bc47e/process_date=2022-01-02/hour=10/20220105_080332_00024_8knjy_03e649ad-06ee-4a73-a95f-1a08ac20e542 to final location gs://xxx-gcs-prod/originla_schema/original_table/set=e578fef26a6d451bb6ee268d3c0bc47e/process_date=2022-01-02/hour=10/20220105_080332_00024_8knjy_03e649ad-06ee-4a73-a95f-1a08ac20e542
	at io.trino.plugin.hive.metastore.SemiTransactionalHiveMetastore.lambda$asyncRename$47(SemiTransactionalHiveMetastore.java:2386)
	at java.base/java.util.concurrent.CompletableFuture$AsyncRun.run(CompletableFuture.java:1736)
	at io.airlift.concurrent.BoundedExecutor.drainQueue(BoundedExecutor.java:80)
	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
	at java.base/java.lang.Thread.run(Thread.java:829)

This gets succeeded if we rerun for many times, the error is message is not so clear. Has anyone seen this issue or know the possible reason and fix. Kindly help and suggest. Thanks In advance.

BalaMahesh avatar Jan 05 '22 12:01 BalaMahesh

cc @medb

findepi avatar Jan 05 '22 13:01 findepi

Without original stacktrace it's hard to tell what is happening, but taking into account that asyncRename parallelises file rename and doing this one by one with call to exists before rename, I would guess that flakiness could be caused by GCS overload if there are many files to rename.

@findepi probably there should be better exception handling that logs original exception thrown in asyncRename?

@BalaMahesh may you check GCS dashboards for the bucket and tell what kind of and how many requests to GCS you see there?

medb avatar Jan 05 '22 16:01 medb

@medb I have gone through the usage logs of GCS bucket, I couldn't find the number of requests to GCS metric yet(can you suggest on where I could find that).

There are logs that I could found for the query that has failed moving data.

time_micros cs_method cs_uri sc_status cs_bytes sc_bytes time_taken_micros cs_host cs_referer cs_user_agent cs_operation cs_bucket cs_object 1641434747044639 GET /storage/v1/b/xxx-gcs-prod-xx-002/o/data0%2Flogs%2Ftrino%2F32e50cc1-fb76-40b2-b892-4583ce6f12b5%2Faky=wefhfwfcaccmd7c42cryuyrw55b431dyq36jcgj%2Fprocess_date=2022-01-05%2F000056_0_20220106_014311_00032_8knjy 200 0 1438 10000 www.googleapis.com GHFS/hadoop2-2.0.0 Google-API-Java-Client Google-HTTP-Java-Client/1.30.0 (gzip),gzip(gfe) storage.objects.get xxx-gcs-prod-xx-002 data0/logs/trino/32e50cc1-fb76-40b2-b892-4583ce6f12b5/2Faky=wefhfwfcaccmd7c42cryuyrw55b431dyq36jcgj/process_date=2022-01-05/000056_0_20220106_014311_00032_8knjy 1641434747047881 GET /storage/v1/b/xxx-gcs-prod-xx-002/o/data0%2Flogs%2Ftrino%2F32e50cc1-fb76-40b2-b892-4583ce6f12b5%2F2Faky=wefhfwfcaccmd7c42cryuyrw55b431dyq36jcgj%2Fprocess_date=2022-01-05%2F000056_0_20220106_014311_00032_8knjy%2F 404 0 589 13000 www.googleapis.com GHFS/hadoop2-2.0.0 Google-API-Java-Client Google-HTTP-Java-Client/1.30.0 (gzip),gzip(gfe) storage.objects.get xxx-gcs-prod-xx-002 data0/logs/trino/32e50cc1-fb76-40b2-b892-4583ce6f12b5/2Faky=wefhfwfcaccmd7c42cryuyrw55b431dyq36jcgj/process_date=2022-01-05/000056_0_20220106_014311_00032_8knjy/ 1641434775443599 GET /storage/v1/b/xxx-gcs-prod-xx-002/o/data0%2Flogs%2Ftrino%2F32e50cc1-fb76-40b2-b892-4583ce6f12b5%2F2Faky=wefhfwfcaccmd7c42cryuyrw55b431dyq36jcgj%2Fprocess_date=2022-01-05%2F000056_0_20220106_014311_00032_8knjy 200 0 1438 8000 www.googleapis.com GHFS/hadoop2-2.0.0 Google-API-Java-Client Google-HTTP-Java-Client/1.30.0 (gzip),gzip(gfe) storage.objects.get xxx-gcs-prod-xx-002 data0/logs/trino/32e50cc1-fb76-40b2-b892-4583ce6f12b5/2Faky=wefhfwfcaccmd7c42cryuyrw55b431dyq36jcgj/process_date=2022-01-05/000056_0_20220106_014311_00032_8knjy 1641434775445554 GET /storage/v1/b/xxx-gcs-prod-xx-002/o/data0%2Flogs%2Ftrino%2F32e50cc1-fb76-40b2-b892-4583ce6f12b5%2F2Faky=wefhfwfcaccmd7c42cryuyrw55b431dyq36jcgj%2Fprocess_date=2022-01-05%2F000056_0_20220106_014311_00032_8knjy%2F 404 0 589 9000 www.googleapis.com GHFS/hadoop2-2.0.0 Google-API-Java-Client Google-HTTP-Java-Client/1.30.0 (gzip),gzip(gfe) storage.objects.get xxx-gcs-prod-xx-002 data0/logs/trino/32e50cc1-fb76-40b2-b892-4583ce6f12b5/2Faky=wefhfwfcaccmd7c42cryuyrw55b431dyq36jcgj/process_date=2022-01-05/000056_0_20220106_014311_00032_8knjy/ 1641434775464812 DELETE /storage/v1/b/xxx-gcs-prod-xx-002/o/data0%2Flogs%2Ftrino%2F32e50cc1-fb76-40b2-b892-4583ce6f12b5%2F2Faky=wefhfwfcaccmd7c42cryuyrw55b431dyq36jcgj%2Fprocess_date=2022-01-05%2F000056_0_20220106_014311_00032_8knjy?ifGenerationMatch=1641434605832442 204 0 0 19000 www.googleapis.com GHFS/hadoop2-2.0.0 Google-API-Java-Client Google-HTTP-Java-Client/1.30.0 (gzip),gzip(gfe) storage.objects.delete xxx-gcs-prod-xx-002 data0/logs/trino/32e50cc1-fb76-40b2-b892-4583ce6f12b5/2Faky=wefhfwfcaccmd7c42cryuyrw55b431dyq36jcgj/process_date=2022-01-05/000056_0_20220106_014311_00032_8knjy

for the other files I could find extra reWriteTo step.

1641434749905940 POST https://www.googleapis.com/storage/v1/b/xxx-gcs-prod-xx-002/o/data0%2Flogs%2Ftrino%2F32e50cc1-fb76-40b2-b892-4583ce6f12b5%2Faky=hsiefofwfnwk4546e43tdmsdwedfwfefe3%2Fprocess_date=2022-01-05%2F000014_0_20220106_014311_00032_8knjy/rewriteTo/b/xxx-gcs-prod-xx-002/o/xxxx_cube%2Fxxxx_table%2Faky=hsiefofwfnwk4546e43tdmsdwedfwfefe3%2Fprocess_date=2022-01-05%2F000014_0_20220106_014311_00032_8knjy?maxBytesRewrittenPerCall=536870912 200 0 1632 29000 www.googleapis.com GHFS/hadoop2-2.0.0 Google-HTTP-Java-Client/1.30.0 (gzip),gzip(gfe) xxx-gcs-prod-xx-002 xxxx_cube/xxxx_table/aky=hsiefofwfnwk4546e43tdmsdwedfwfefe3/process_date=2022-01-05/000014_0_20220106_014311_00032_8knjy

Sorry for the messy logs, these are tab separated and headers are added at the top. Please copy them to sheets.

From my observation reWriteTo request has not been triggered for the failed file move, from the code I could it is executing :
(SemiTransactionalHiveMetastore.java)

try {
                    if (fileSystem.exists(target) || !fileSystem.rename(source, target)) {
                        throw new TrinoException(HIVE_FILESYSTEM_ERROR, format("Error moving data files from %s to final location %s", source, target));
                    }
                }
                catch (IOException e) {
                    throw new TrinoException(HIVE_FILESYSTEM_ERROR, format("Error moving data files from %s to final location %s", source, target), e);
                }

BalaMahesh avatar Jan 06 '22 09:01 BalaMahesh

GCS bucket should have api/request_count metric that you can use to determine number, type and status of the GCS requests to your bucket during query execution: https://cloud.google.com/monitoring/api/metrics_gcp#gcp-storage

medb avatar Jan 06 '22 17:01 medb

@findepi probably there should be better exception handling that logs original exception thrown in asyncRename?

@medb The exception in issue description points at this line https://github.com/trinodb/trino/blob/44af1d5c9afd4a0eddc62452823f8898b7b2ea4e/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/SemiTransactionalHiveMetastore.java#L2386

I filed https://github.com/trinodb/trino/pull/10520 to differentiate exists and rename checks. Not sure if it helps, but shouldn't hurt.

cc @losipiuk

findepi avatar Jan 10 '22 13:01 findepi

@BalaMahesh Did you have any workaround? I am also experiencing this issue with trino version 395

aakashnand avatar Oct 03 '22 11:10 aakashnand

Hi @findepi, As a workaround to this whenever we face this issue we disable the temporary staging directory in the session property and run the job. After your commit we have identified that we are facing this issue in rename checks. Can you please help with the permanent solution here? This still exists in the newer version of trino.

sambhav8695 avatar Feb 07 '24 07:02 sambhav8695

@findepi We are facing this issue with the version 402. It still happens from time to time. Any plans on fixing this?

dheerajathrey avatar Feb 17 '24 07:02 dheerajathrey

I meet this when I use hdfs Federation. it does not support mv files from one namespace to another namespace.

boson234 avatar May 21 '24 11:05 boson234