From 6517fc27969f71809ce5ad9e1e5b941a32c646a7 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 28 Jul 2023 13:31:03 -0700 Subject: [PATCH] Save a metadata call when reading files from CloudObjectInputSource. (#14677) * Save a metadata call when reading files from CloudObjectInputSource. The call to createSplits(inputFormat, null) in formattableReader would use the default split hint spec, MaxSizeSplitHintSpec, which makes getObjectMetadata calls in order to compute its splits. This isn't necessary; we're just trying to unpack the files inside the input source. To fix this, use FilePerSplitHintSpec to extract files without any funny business. * Adjust call. * Fix constant. * Test coverage. --- .../external/ExternalInputSpecSlicer.java | 25 +-------- .../data/input/FilePerSplitHintSpec.java | 53 +++++++++++++++++++ .../input/impl/CloudObjectInputSource.java | 21 +++++++- .../impl/CloudObjectInputSourceTest.java | 20 ++++++- 4 files changed, 92 insertions(+), 27 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/data/input/FilePerSplitHintSpec.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSpecSlicer.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSpecSlicer.java index ca853bb4e65..7c6700ce3a2 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSpecSlicer.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSpecSlicer.java @@ -19,8 +19,8 @@ package org.apache.druid.msq.input.external; -import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Iterators; +import org.apache.druid.data.input.FilePerSplitHintSpec; import org.apache.druid.data.input.InputFileAttribute; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.InputSplit; @@ -238,27 +238,4 @@ public class ExternalInputSpecSlicer implements InputSpecSlicer ); } } - - /** - * Assigns each input file to its own split. - */ - @VisibleForTesting - static class FilePerSplitHintSpec implements SplitHintSpec - { - static FilePerSplitHintSpec INSTANCE = new FilePerSplitHintSpec(); - - private FilePerSplitHintSpec() - { - // Singleton. - } - - @Override - public Iterator> split( - final Iterator inputIterator, - final Function inputAttributeExtractor - ) - { - return Iterators.transform(inputIterator, Collections::singletonList); - } - } } diff --git a/processing/src/main/java/org/apache/druid/data/input/FilePerSplitHintSpec.java b/processing/src/main/java/org/apache/druid/data/input/FilePerSplitHintSpec.java new file mode 100644 index 00000000000..71b1acecc09 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/data/input/FilePerSplitHintSpec.java @@ -0,0 +1,53 @@ +/* + * 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.druid.data.input; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.google.common.collect.Iterators; + +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.function.Function; + +/** + * Assigns each input file to its own split. + * + * Not named as a {@link JsonSubTypes} in {@link SplitHintSpec}, because this class is meant for internal use + * within a server only. It is not serialized for transfer between servers, and is not part of the user-facing API. + */ +public class FilePerSplitHintSpec implements SplitHintSpec +{ + public static final FilePerSplitHintSpec INSTANCE = new FilePerSplitHintSpec(); + + private FilePerSplitHintSpec() + { + // Singleton. + } + + @Override + public Iterator> split( + final Iterator inputIterator, + final Function inputAttributeExtractor + ) + { + return Iterators.transform(inputIterator, Collections::singletonList); + } +} diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/CloudObjectInputSource.java b/processing/src/main/java/org/apache/druid/data/input/impl/CloudObjectInputSource.java index 724cd96745d..e02806fce6c 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/CloudObjectInputSource.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/CloudObjectInputSource.java @@ -26,6 +26,7 @@ import com.google.common.collect.Lists; import com.google.common.primitives.Ints; import org.apache.commons.lang.StringUtils; import org.apache.druid.data.input.AbstractInputSource; +import org.apache.druid.data.input.FilePerSplitHintSpec; import org.apache.druid.data.input.InputEntity; import org.apache.druid.data.input.InputFileAttribute; import org.apache.druid.data.input.InputFormat; @@ -173,11 +174,26 @@ public abstract class CloudObjectInputSource extends AbstractInputSource return new InputEntityIteratingReader( inputRowSchema, inputFormat, - createSplits(inputFormat, null).flatMap(split -> split.get().stream()).map(this::createEntity).iterator(), + getInputEntities(inputFormat), temporaryDirectory ); } + /** + * Return an iterator of {@link InputEntity} corresponding to the objects represented by this input source, as read + * by the provided {@link InputFormat}. + */ + Iterator getInputEntities(final InputFormat inputFormat) + { + // Use createSplits with FilePerSplitHintSpec.INSTANCE as a way of getting the list of objects to read + // out of either "prefixes", "objects", or "uris". The specific splits don't matter because we are going + // to flatten them anyway. + return createSplits(inputFormat, FilePerSplitHintSpec.INSTANCE) + .flatMap(split -> split.get().stream()) + .map(this::createEntity) + .iterator(); + } + @Override public boolean equals(Object o) { @@ -311,7 +327,8 @@ public abstract class CloudObjectInputSource extends AbstractInputSource long size = splitWidget.getObjectSize(o.getLocation()); return new InputFileAttribute( size, - inputFormat != null ? inputFormat.getWeightedSize(o.getLocation().getPath(), size) : size); + inputFormat != null ? inputFormat.getWeightedSize(o.getLocation().getPath(), size) : size + ); } else { return new InputFileAttribute( o.getSize(), diff --git a/processing/src/test/java/org/apache/druid/data/input/impl/CloudObjectInputSourceTest.java b/processing/src/test/java/org/apache/druid/data/input/impl/CloudObjectInputSourceTest.java index a68713d43b6..fc7ea2bbfa6 100644 --- a/processing/src/test/java/org/apache/druid/data/input/impl/CloudObjectInputSourceTest.java +++ b/processing/src/test/java/org/apache/druid/data/input/impl/CloudObjectInputSourceTest.java @@ -19,6 +19,8 @@ package org.apache.druid.data.input.impl; +import com.google.common.collect.Lists; +import org.apache.druid.data.input.InputEntity; import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.MaxSizeSplitHintSpec; import org.apache.druid.java.util.common.parsers.JSONPathSpec; @@ -146,6 +148,10 @@ public class CloudObjectInputSourceTest Assert.assertEquals("**.csv", inputSource.getObjectGlob()); Assert.assertEquals(URIS, returnedLocationUris); + + final List entities = + Lists.newArrayList(inputSource.getInputEntities(new JsonInputFormat(null, null, null, null, null))); + Assert.assertEquals(URIS.size(), entities.size()); } @Test @@ -168,6 +174,10 @@ public class CloudObjectInputSourceTest Assert.assertEquals(null, inputSource.getObjectGlob()); Assert.assertEquals(URIS, returnedLocationUris); + + final List entities = + Lists.newArrayList(inputSource.getInputEntities(new JsonInputFormat(null, null, null, null, null))); + Assert.assertEquals(URIS.size(), entities.size()); } @Test @@ -190,6 +200,10 @@ public class CloudObjectInputSourceTest Assert.assertEquals("**.csv", inputSource.getObjectGlob()); Assert.assertEquals(URIS, returnedLocationUris); + + final List entities = + Lists.newArrayList(inputSource.getInputEntities(new JsonInputFormat(null, null, null, null, null))); + Assert.assertEquals(OBJECTS.size(), entities.size()); } @Test @@ -208,8 +222,12 @@ public class CloudObjectInputSourceTest List returnedLocations = splits.map(InputSplit::get).collect(Collectors.toList()).get(0); - Assert.assertEquals(null, inputSource.getObjectGlob()); + Assert.assertNull(inputSource.getObjectGlob()); Assert.assertEquals(OBJECTS, returnedLocations); + + final List entities = + Lists.newArrayList(inputSource.getInputEntities(new JsonInputFormat(null, null, null, null, null))); + Assert.assertEquals(OBJECTS.size(), entities.size()); } @Test