diff --git a/src/main/java/net/snowflake/ingest/streaming/internal/BlobBuilder.java b/src/main/java/net/snowflake/ingest/streaming/internal/BlobBuilder.java index 8e65ed677..939e6d255 100644 --- a/src/main/java/net/snowflake/ingest/streaming/internal/BlobBuilder.java +++ b/src/main/java/net/snowflake/ingest/streaming/internal/BlobBuilder.java @@ -84,7 +84,7 @@ static Blob constructBlobAndMetadata( Flusher flusher = channelsDataPerTable.get(0).createFlusher(); Flusher.SerializationResult serializedChunk = - flusher.serialize(channelsDataPerTable, filePath); + flusher.serialize(channelsDataPerTable, filePath, curDataSize); if (!serializedChunk.channelsMetadataList.isEmpty()) { final byte[] compressedChunkData; diff --git a/src/main/java/net/snowflake/ingest/streaming/internal/Flusher.java b/src/main/java/net/snowflake/ingest/streaming/internal/Flusher.java index 3bb339975..0cf8220bb 100644 --- a/src/main/java/net/snowflake/ingest/streaming/internal/Flusher.java +++ b/src/main/java/net/snowflake/ingest/streaming/internal/Flusher.java @@ -20,12 +20,15 @@ public interface Flusher { /** * Serialize buffered rows into the underlying format. * + * @param fullyQualifiedTableName * @param channelsDataPerTable buffered rows * @param filePath file path + * @param chunkStartOffset * @return {@link SerializationResult} * @throws IOException */ - SerializationResult serialize(List> channelsDataPerTable, String filePath) + SerializationResult serialize( + List> channelsDataPerTable, String filePath, long chunkStartOffset) throws IOException; /** Holds result of the buffered rows conversion: channel metadata and stats. */ diff --git a/src/main/java/net/snowflake/ingest/streaming/internal/ParquetFlusher.java b/src/main/java/net/snowflake/ingest/streaming/internal/ParquetFlusher.java index ddfca4a42..cfb60f94f 100644 --- a/src/main/java/net/snowflake/ingest/streaming/internal/ParquetFlusher.java +++ b/src/main/java/net/snowflake/ingest/streaming/internal/ParquetFlusher.java @@ -15,6 +15,7 @@ import net.snowflake.ingest.utils.Logging; import net.snowflake.ingest.utils.Pair; import net.snowflake.ingest.utils.SFException; +import org.apache.parquet.Preconditions; import org.apache.parquet.hadoop.BdecParquetWriter; import org.apache.parquet.schema.MessageType; @@ -41,13 +42,17 @@ public ParquetFlusher( @Override public SerializationResult serialize( - List> channelsDataPerTable, String filePath) + List> channelsDataPerTable, + String filePath, + long chunkStartOffset) throws IOException { - return serializeFromJavaObjects(channelsDataPerTable, filePath); + return serializeFromJavaObjects(channelsDataPerTable, filePath, chunkStartOffset); } private SerializationResult serializeFromJavaObjects( - List> channelsDataPerTable, String filePath) + List> channelsDataPerTable, + String filePath, + long chunkStartOffset) throws IOException { List channelsMetadataList = new ArrayList<>(); long rowCount = 0L; @@ -113,10 +118,7 @@ private SerializationResult serializeFromJavaObjects( } Map metadata = channelsDataPerTable.get(0).getVectors().metadata; - // We insert the filename in the file itself as metadata so that streams can work on replicated - // mixed tables. For a more detailed discussion on the topic see SNOW-561447 and - // http://go/streams-on-replicated-mixed-tables - metadata.put(Constants.PRIMARY_FILE_ID_KEY, StreamingIngestUtils.getShortname(filePath)); + addFileIdToMetadata(filePath, chunkStartOffset, metadata); parquetWriter = new BdecParquetWriter( mergedData, @@ -139,6 +141,26 @@ private SerializationResult serializeFromJavaObjects( chunkMinMaxInsertTimeInMs); } + private static void addFileIdToMetadata( + String filePath, long chunkStartOffset, Map metadata) { + // We insert the filename in the file itself as metadata so that streams can work on replicated + // mixed tables. For a more detailed discussion on the topic see SNOW-561447 and + // http://go/streams-on-replicated-mixed-tables + // Using chunk offset as suffix ensures that for interleaved tables, the file + // id key is unique for each chunk. Each chunk is logically a separate Parquet file that happens + // to be bundled together. + if (chunkStartOffset == 0) { + metadata.put(Constants.PRIMARY_FILE_ID_KEY, StreamingIngestUtils.getShortname(filePath)); + } else { + String shortName = StreamingIngestUtils.getShortname(filePath); + final String[] parts = shortName.split("\\."); + Preconditions.checkState(parts.length == 2, "Invalid file name format"); + metadata.put( + Constants.PRIMARY_FILE_ID_KEY, + String.format("%s_%d.%s", parts[0], chunkStartOffset, parts[1])); + } + } + /** * Validates that rows count in metadata matches the row count in Parquet footer and the row count * written by the parquet writer diff --git a/src/main/java/net/snowflake/ingest/streaming/internal/ParquetRowBuffer.java b/src/main/java/net/snowflake/ingest/streaming/internal/ParquetRowBuffer.java index b35a7e999..395bea98f 100644 --- a/src/main/java/net/snowflake/ingest/streaming/internal/ParquetRowBuffer.java +++ b/src/main/java/net/snowflake/ingest/streaming/internal/ParquetRowBuffer.java @@ -18,9 +18,11 @@ import java.util.Set; import java.util.function.Consumer; import net.snowflake.client.jdbc.internal.google.common.collect.Sets; +import net.snowflake.ingest.connection.RequestBuilder; import net.snowflake.ingest.connection.TelemetryService; import net.snowflake.ingest.streaming.OffsetTokenVerificationFunction; import net.snowflake.ingest.streaming.OpenChannelRequest; +import net.snowflake.ingest.utils.Constants; import net.snowflake.ingest.utils.ErrorCode; import net.snowflake.ingest.utils.SFException; import org.apache.parquet.schema.MessageType; @@ -74,6 +76,7 @@ public void setupSchema(List columns) { fieldIndex.clear(); metadata.clear(); metadata.put("sfVer", "1,1"); + metadata.put(Constants.SDK_VERSION_KEY, RequestBuilder.DEFAULT_VERSION); List parquetTypes = new ArrayList<>(); int id = 1; for (ColumnMetadata column : columns) { diff --git a/src/main/java/net/snowflake/ingest/utils/Constants.java b/src/main/java/net/snowflake/ingest/utils/Constants.java index 83394abf7..35eed3469 100644 --- a/src/main/java/net/snowflake/ingest/utils/Constants.java +++ b/src/main/java/net/snowflake/ingest/utils/Constants.java @@ -35,6 +35,7 @@ public class Constants { public static final String SNOWFLAKE_OAUTH_TOKEN_ENDPOINT = "/oauth/token-request"; public static final String PRIMARY_FILE_ID_KEY = "primaryFileId"; // Don't change, should match Parquet Scanner + public static final String SDK_VERSION_KEY = "sdkVersion"; public static final long RESPONSE_SUCCESS = 0L; // Don't change, should match server side public static final long RESPONSE_ERR_GENERAL_EXCEPTION_RETRY_REQUEST = 10L; // Don't change, should match server side diff --git a/src/test/java/net/snowflake/ingest/streaming/internal/FlushServiceTest.java b/src/test/java/net/snowflake/ingest/streaming/internal/FlushServiceTest.java index f85cfbc56..2d1bd9646 100644 --- a/src/test/java/net/snowflake/ingest/streaming/internal/FlushServiceTest.java +++ b/src/test/java/net/snowflake/ingest/streaming/internal/FlushServiceTest.java @@ -140,7 +140,7 @@ ChannelData flushChannel(String name) { BlobMetadata buildAndUpload() throws Exception { List>> blobData = Collections.singletonList(channelData); return flushService.buildAndUpload( - BlobPath.fileNameWithoutToken("file_name"), + BlobPath.fileNameWithoutToken("file_name.bdec"), blobData, blobData.get(0).get(0).getChannelContext().getFullyQualifiedTableName()); } @@ -940,7 +940,7 @@ public void testBuildAndUpload() throws Exception { blobCaptor.capture(), metadataCaptor.capture(), ArgumentMatchers.any()); - Assert.assertEquals("file_name", nameCaptor.getValue().fileName); + Assert.assertEquals("file_name.bdec", nameCaptor.getValue().fileName); ChunkMetadata metadataResult = metadataCaptor.getValue().get(0); List channelMetadataResult = metadataResult.getChannels(); diff --git a/src/test/java/net/snowflake/ingest/streaming/internal/RowBufferTest.java b/src/test/java/net/snowflake/ingest/streaming/internal/RowBufferTest.java index 30ef7dbbd..70b950fda 100644 --- a/src/test/java/net/snowflake/ingest/streaming/internal/RowBufferTest.java +++ b/src/test/java/net/snowflake/ingest/streaming/internal/RowBufferTest.java @@ -20,6 +20,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; +import net.snowflake.ingest.connection.RequestBuilder; import net.snowflake.ingest.streaming.InsertValidationResponse; import net.snowflake.ingest.streaming.OpenChannelRequest; import net.snowflake.ingest.utils.Constants; @@ -1905,11 +1906,30 @@ public void testParquetFileNameMetadata() throws IOException { data.setChannelContext(new ChannelFlushContext("name", "db", "schema", "table", 1L, "key", 0L)); ParquetFlusher flusher = (ParquetFlusher) bufferUnderTest.createFlusher(); - Flusher.SerializationResult result = - flusher.serialize(Collections.singletonList(data), filePath); + { + Flusher.SerializationResult result = + flusher.serialize(Collections.singletonList(data), filePath, 0); - BdecParquetReader reader = new BdecParquetReader(result.chunkData.toByteArray()); - Assert.assertEquals(filePath, reader.getKeyValueMetadata().get(Constants.PRIMARY_FILE_ID_KEY)); + BdecParquetReader reader = new BdecParquetReader(result.chunkData.toByteArray()); + Assert.assertEquals( + "testParquetFileNameMetadata.bdec", + reader.getKeyValueMetadata().get(Constants.PRIMARY_FILE_ID_KEY)); + Assert.assertEquals( + RequestBuilder.DEFAULT_VERSION, + reader.getKeyValueMetadata().get(Constants.SDK_VERSION_KEY)); + } + { + Flusher.SerializationResult result = + flusher.serialize(Collections.singletonList(data), filePath, 13); + + BdecParquetReader reader = new BdecParquetReader(result.chunkData.toByteArray()); + Assert.assertEquals( + "testParquetFileNameMetadata_13.bdec", + reader.getKeyValueMetadata().get(Constants.PRIMARY_FILE_ID_KEY)); + Assert.assertEquals( + RequestBuilder.DEFAULT_VERSION, + reader.getKeyValueMetadata().get(Constants.SDK_VERSION_KEY)); + } } private static Thread getThreadThatWaitsForLockReleaseAndFlushes(