diff --git a/presto-raptor/src/main/java/com/facebook/presto/raptor/RaptorMetadata.java b/presto-raptor/src/main/java/com/facebook/presto/raptor/RaptorMetadata.java index 1f66661d534cd..308e21f817869 100644 --- a/presto-raptor/src/main/java/com/facebook/presto/raptor/RaptorMetadata.java +++ b/presto-raptor/src/main/java/com/facebook/presto/raptor/RaptorMetadata.java @@ -16,8 +16,10 @@ import com.facebook.airlift.json.JsonCodec; import com.facebook.airlift.log.Logger; import com.facebook.presto.raptor.metadata.ColumnInfo; +import com.facebook.presto.raptor.metadata.DeltaInfoPair; import com.facebook.presto.raptor.metadata.Distribution; import com.facebook.presto.raptor.metadata.MetadataDao; +import com.facebook.presto.raptor.metadata.ShardDeleteDelta; import com.facebook.presto.raptor.metadata.ShardDelta; import com.facebook.presto.raptor.metadata.ShardInfo; import com.facebook.presto.raptor.metadata.ShardManager; @@ -134,6 +136,7 @@ public class RaptorMetadata private static final JsonCodec SHARD_INFO_CODEC = jsonCodec(ShardInfo.class); private static final JsonCodec SHARD_DELTA_CODEC = jsonCodec(ShardDelta.class); + private static final JsonCodec SHARD_DELETE_DELTA_CODEC = jsonCodec(ShardDeleteDelta.class); private final IDBI dbi; private final MetadataDao dao; @@ -821,23 +824,35 @@ public void finishDelete(ConnectorSession session, ConnectorTableHandle tableHan .map(RaptorColumnHandle.class::cast) .map(ColumnInfo::fromHandle).collect(toList()); - ImmutableSet.Builder oldShardUuidsBuilder = ImmutableSet.builder(); - ImmutableList.Builder newShardsBuilder = ImmutableList.builder(); + if (table.isTableSupportsDeltaDelete()) { + ImmutableMap.Builder shardMapBuilder = ImmutableMap.builder(); - fragments.stream() - .map(fragment -> SHARD_DELTA_CODEC.fromJson(fragment.getBytes())) - .forEach(delta -> { - oldShardUuidsBuilder.addAll(delta.getOldShardUuids()); - newShardsBuilder.addAll(delta.getNewShards()); - }); + fragments.stream() + .map(fragment -> SHARD_DELETE_DELTA_CODEC.fromJson(fragment.getBytes())) + .forEach(delta -> shardMapBuilder.put(delta.getOldShardUuid(), delta.getDeltaInfoPair())); + OptionalLong updateTime = OptionalLong.of(session.getStartTime()); - Set oldShardUuids = oldShardUuidsBuilder.build(); - List newShards = newShardsBuilder.build(); - OptionalLong updateTime = OptionalLong.of(session.getStartTime()); + log.info("Finishing delete for tableId %s (affected shardUuid: %s)", tableId, shardMapBuilder.build().size()); + shardManager.replaceDeltaUuids(transactionId, tableId, columns, shardMapBuilder.build(), updateTime); + } + else { + ImmutableSet.Builder oldShardUuidsBuilder = ImmutableSet.builder(); + ImmutableList.Builder newShardsBuilder = ImmutableList.builder(); - log.info("Finishing delete for tableId %s (removed: %s, rewritten: %s)", tableId, oldShardUuids.size() - newShards.size(), newShards.size()); - shardManager.replaceShardUuids(transactionId, tableId, columns, oldShardUuids, newShards, updateTime); + fragments.stream() + .map(fragment -> SHARD_DELTA_CODEC.fromJson(fragment.getBytes())) + .forEach(delta -> { + oldShardUuidsBuilder.addAll(delta.getOldShardUuids()); + newShardsBuilder.addAll(delta.getNewShards()); + }); + Set oldShardUuids = oldShardUuidsBuilder.build(); + List newShards = newShardsBuilder.build(); + OptionalLong updateTime = OptionalLong.of(session.getStartTime()); + + log.info("Finishing delete for tableId %s (removed: %s, rewritten: %s)", tableId, oldShardUuids.size() - newShards.size(), newShards.size()); + shardManager.replaceShardUuids(transactionId, tableId, columns, oldShardUuids, newShards, updateTime); + } clearRollback(); } diff --git a/presto-raptor/src/main/java/com/facebook/presto/raptor/RaptorPageSourceProvider.java b/presto-raptor/src/main/java/com/facebook/presto/raptor/RaptorPageSourceProvider.java index bb0d1b66bc473..88bbecfb8e913 100644 --- a/presto-raptor/src/main/java/com/facebook/presto/raptor/RaptorPageSourceProvider.java +++ b/presto-raptor/src/main/java/com/facebook/presto/raptor/RaptorPageSourceProvider.java @@ -60,24 +60,55 @@ public ConnectorPageSource createPageSource(ConnectorTransactionHandle transacti ReaderAttributes attributes = ReaderAttributes.from(session); OptionalLong transactionId = raptorSplit.getTransactionId(); Optional> columnTypes = raptorSplit.getColumnTypes(); + boolean tableSupportsDeltaDelete = raptorSplit.isTableSupportsDeltaDelete(); FileSystemContext context = new FileSystemContext(session); + Map shardDeltaMap = raptorSplit.getShardDeltaMap(); if (raptorSplit.getShardUuids().size() == 1) { UUID shardUuid = raptorSplit.getShardUuids().iterator().next(); - return createPageSource(context, shardUuid, bucketNumber, columns, predicate, attributes, transactionId, columnTypes); + return createPageSource( + context, + shardUuid, + Optional.ofNullable(shardDeltaMap.get(shardUuid)), + tableSupportsDeltaDelete, + bucketNumber, + columns, + predicate, + attributes, + transactionId, + columnTypes); } Iterator iterator = raptorSplit.getShardUuids().stream() - .map(shardUuid -> createPageSource(context, shardUuid, bucketNumber, columns, predicate, attributes, transactionId, columnTypes)) + .map(shardUuid -> createPageSource( + context, + shardUuid, + Optional.ofNullable(shardDeltaMap.get(shardUuid)), + tableSupportsDeltaDelete, + bucketNumber, + columns, + predicate, + attributes, + transactionId, + columnTypes)) .iterator(); return new ConcatPageSource(iterator); } + /** + * + * @param deltaShardUuid delta of one shard + * @param tableSupportsDeltaDelete table property indicating if this table supports delta_delete + * In the future, we could have the concept of delta_delete as session property. + * Having these two parameters at the same time gives us the flexibility and compatibility to future features. + */ private ConnectorPageSource createPageSource( FileSystemContext context, UUID shardUuid, + Optional deltaShardUuid, + boolean tableSupportsDeltaDelete, OptionalInt bucketNumber, List columns, TupleDomain predicate, @@ -89,6 +120,6 @@ private ConnectorPageSource createPageSource( List columnIds = columnHandles.stream().map(RaptorColumnHandle::getColumnId).collect(toList()); List columnTypes = columnHandles.stream().map(RaptorColumnHandle::getColumnType).collect(toList()); - return storageManager.getPageSource(context, shardUuid, bucketNumber, columnIds, columnTypes, predicate, attributes, transactionId, allColumnTypes); + return storageManager.getPageSource(context, shardUuid, deltaShardUuid, tableSupportsDeltaDelete, bucketNumber, columnIds, columnTypes, predicate, attributes, transactionId, allColumnTypes); } } diff --git a/presto-raptor/src/main/java/com/facebook/presto/raptor/RaptorSplit.java b/presto-raptor/src/main/java/com/facebook/presto/raptor/RaptorSplit.java index 1a9433539f300..86a583d8798cc 100644 --- a/presto-raptor/src/main/java/com/facebook/presto/raptor/RaptorSplit.java +++ b/presto-raptor/src/main/java/com/facebook/presto/raptor/RaptorSplit.java @@ -20,6 +20,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import java.util.List; @@ -38,6 +39,8 @@ public class RaptorSplit { private final String connectorId; private final Set shardUuids; + private final Map shardDeltaMap; + private final boolean tableSupportsDeltaDelete; private final OptionalInt bucketNumber; private final List addresses; private final TupleDomain effectivePredicate; @@ -48,40 +51,75 @@ public class RaptorSplit public RaptorSplit( @JsonProperty("connectorId") String connectorId, @JsonProperty("shardUuids") Set shardUuids, + @JsonProperty("shardDeltaMap") Map shardDeltaMap, + @JsonProperty("tableSupportsDeltaDelete") boolean tableSupportsDeltaDelete, @JsonProperty("bucketNumber") OptionalInt bucketNumber, @JsonProperty("effectivePredicate") TupleDomain effectivePredicate, @JsonProperty("transactionId") OptionalLong transactionId, @JsonProperty("columnTypes") Optional> columnTypes) { - this(connectorId, shardUuids, bucketNumber, ImmutableList.of(), effectivePredicate, transactionId, columnTypes); + this( + connectorId, + shardUuids, + shardDeltaMap, + tableSupportsDeltaDelete, + bucketNumber, + ImmutableList.of(), + effectivePredicate, + transactionId, + columnTypes); } public RaptorSplit( String connectorId, UUID shardUuid, + Optional deltaShardUuid, + boolean tableSupportsDeltaDelete, List addresses, TupleDomain effectivePredicate, OptionalLong transactionId, Optional> columnTypes) { - this(connectorId, ImmutableSet.of(shardUuid), OptionalInt.empty(), addresses, effectivePredicate, transactionId, columnTypes); + this( + connectorId, + ImmutableSet.of(shardUuid), + deltaShardUuid.map(deltaUuid -> ImmutableMap.of(shardUuid, deltaUuid)).orElse(ImmutableMap.of()), + tableSupportsDeltaDelete, + OptionalInt.empty(), + addresses, + effectivePredicate, + transactionId, + columnTypes); } public RaptorSplit( String connectorId, Set shardUuids, + Map shardDeltaMap, + boolean tableSupportsDeltaDelete, int bucketNumber, HostAddress address, TupleDomain effectivePredicate, OptionalLong transactionId, Optional> columnTypes) { - this(connectorId, shardUuids, OptionalInt.of(bucketNumber), ImmutableList.of(address), effectivePredicate, transactionId, columnTypes); + this( + connectorId, + shardUuids, + shardDeltaMap, + tableSupportsDeltaDelete, + OptionalInt.of(bucketNumber), + ImmutableList.of(address), + effectivePredicate, + transactionId, + columnTypes); } private RaptorSplit( String connectorId, Set shardUuids, + Map shardDeltaMap, + boolean tableSupportsDeltaDelete, OptionalInt bucketNumber, List addresses, TupleDomain effectivePredicate, @@ -90,6 +128,8 @@ private RaptorSplit( { this.connectorId = requireNonNull(connectorId, "connectorId is null"); this.shardUuids = ImmutableSet.copyOf(requireNonNull(shardUuids, "shardUuid is null")); + this.shardDeltaMap = requireNonNull(shardDeltaMap, "shardUuid is null"); + this.tableSupportsDeltaDelete = tableSupportsDeltaDelete; this.bucketNumber = requireNonNull(bucketNumber, "bucketNumber is null"); this.addresses = ImmutableList.copyOf(requireNonNull(addresses, "addresses is null")); this.effectivePredicate = requireNonNull(effectivePredicate, "effectivePredicate is null"); @@ -121,6 +161,18 @@ public Set getShardUuids() return shardUuids; } + @JsonProperty + public Map getShardDeltaMap() + { + return shardDeltaMap; + } + + @JsonProperty + public boolean isTableSupportsDeltaDelete() + { + return tableSupportsDeltaDelete; + } + @JsonProperty public OptionalInt getBucketNumber() { @@ -156,6 +208,8 @@ public String toString() { return toStringHelper(this) .add("shardUuids", shardUuids) + .add("shardDeltaMap", shardDeltaMap.toString()) + .add("tableSupportsDeltaDelete", tableSupportsDeltaDelete) .add("bucketNumber", bucketNumber.isPresent() ? bucketNumber.getAsInt() : null) .add("hosts", addresses) .omitNullValues() diff --git a/presto-raptor/src/main/java/com/facebook/presto/raptor/RaptorSplitManager.java b/presto-raptor/src/main/java/com/facebook/presto/raptor/RaptorSplitManager.java index 1d14fd599f8ed..da6056deb1c2c 100644 --- a/presto-raptor/src/main/java/com/facebook/presto/raptor/RaptorSplitManager.java +++ b/presto-raptor/src/main/java/com/facebook/presto/raptor/RaptorSplitManager.java @@ -32,6 +32,7 @@ import com.facebook.presto.spi.predicate.TupleDomain; import com.facebook.presto.spi.type.Type; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import org.skife.jdbi.v2.ResultIterator; import javax.annotation.PreDestroy; @@ -111,7 +112,7 @@ public ConnectorSplitSource getSplits( OptionalLong transactionId = table.getTransactionId(); Optional> bucketToNode = handle.getPartitioning().map(RaptorPartitioningHandle::getBucketToNode); verify(bucketed == bucketToNode.isPresent(), "mismatched bucketCount and bucketToNode presence"); - return new RaptorSplitSource(tableId, merged, effectivePredicate, transactionId, table.getColumnTypes(), bucketToNode); + return new RaptorSplitSource(tableId, merged, effectivePredicate, transactionId, table.getColumnTypes(), bucketToNode, handle.getTable().isTableSupportsDeltaDelete()); } private static List getAddressesForNodes(Map nodeMap, Iterable nodeIdentifiers) @@ -148,6 +149,7 @@ private class RaptorSplitSource private final Optional> columnTypes; private final Optional> bucketToNode; private final ResultIterator iterator; + private final boolean tableSupportsDeltaDelete; @GuardedBy("this") private CompletableFuture future; @@ -158,20 +160,22 @@ public RaptorSplitSource( TupleDomain effectivePredicate, OptionalLong transactionId, Optional> columnTypes, - Optional> bucketToNode) + Optional> bucketToNode, + boolean tableSupportsDeltaDelete) { this.tableId = tableId; this.effectivePredicate = requireNonNull(effectivePredicate, "effectivePredicate is null"); this.transactionId = requireNonNull(transactionId, "transactionId is null"); this.columnTypes = requireNonNull(columnTypes, "columnTypesis null"); this.bucketToNode = requireNonNull(bucketToNode, "bucketToNode is null"); + this.tableSupportsDeltaDelete = tableSupportsDeltaDelete; ResultIterator iterator; if (bucketToNode.isPresent()) { - iterator = shardManager.getShardNodesBucketed(tableId, merged, bucketToNode.get(), effectivePredicate); + iterator = shardManager.getShardNodesBucketed(tableId, merged, bucketToNode.get(), effectivePredicate, tableSupportsDeltaDelete); } else { - iterator = shardManager.getShardNodes(tableId, effectivePredicate); + iterator = shardManager.getShardNodes(tableId, effectivePredicate, tableSupportsDeltaDelete); } this.iterator = new SynchronizedResultIterator<>(iterator); } @@ -226,13 +230,14 @@ private ConnectorSplit createSplit(BucketShards bucketShards) verify(bucketShards.getShards().size() == 1, "wrong shard count for non-bucketed table"); ShardNodes shard = getOnlyElement(bucketShards.getShards()); - UUID shardId = shard.getShardUuid(); + UUID shardUuid = shard.getShardUuid(); + Optional deltaShardUuid = shard.getDeltaShardUuid(); Set nodeIds = shard.getNodeIdentifiers(); List addresses = getAddressesForNodes(nodesById, nodeIds); if (addresses.isEmpty()) { if (!backupAvailable) { - throw new PrestoException(RAPTOR_NO_HOST_FOR_SHARD, format("No host for shard %s found: %s", shardId, nodeIds)); + throw new PrestoException(RAPTOR_NO_HOST_FOR_SHARD, format("No host for shard %s found: %s", shardUuid, nodeIds)); } // Pick a random node and optimistically assign the shard to it. @@ -242,11 +247,19 @@ private ConnectorSplit createSplit(BucketShards bucketShards) throw new PrestoException(NO_NODES_AVAILABLE, "No nodes available to run query"); } Node node = selectRandom(availableNodes); - shardManager.replaceShardAssignment(tableId, shardId, node.getNodeIdentifier(), true); + shardManager.replaceShardAssignment(tableId, shardUuid, node.getNodeIdentifier(), true); addresses = ImmutableList.of(node.getHostAndPort()); } - return new RaptorSplit(connectorId, shardId, addresses, effectivePredicate, transactionId, columnTypes); + return new RaptorSplit( + connectorId, + shardUuid, + deltaShardUuid, + tableSupportsDeltaDelete, + addresses, + effectivePredicate, + transactionId, + columnTypes); } private ConnectorSplit createBucketSplit(int bucketNumber, Set shards) @@ -263,9 +276,25 @@ private ConnectorSplit createBucketSplit(int bucketNumber, Set shard Set shardUuids = shards.stream() .map(ShardNodes::getShardUuid) .collect(toSet()); + ImmutableMap.Builder shardMapBuilder = ImmutableMap.builder(); + shards.forEach( + shard -> { + if (shard.getDeltaShardUuid().isPresent()) { + shardMapBuilder.put(shard.getShardUuid(), shard.getDeltaShardUuid().get()); + } + }); HostAddress address = node.getHostAndPort(); - return new RaptorSplit(connectorId, shardUuids, bucketNumber, address, effectivePredicate, transactionId, columnTypes); + return new RaptorSplit( + connectorId, + shardUuids, + shardMapBuilder.build(), + tableSupportsDeltaDelete, + bucketNumber, + address, + effectivePredicate, + transactionId, + columnTypes); } } } diff --git a/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/DatabaseShardManager.java b/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/DatabaseShardManager.java index 922e179596e40..12fd6147b99f7 100644 --- a/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/DatabaseShardManager.java +++ b/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/DatabaseShardManager.java @@ -14,6 +14,7 @@ package com.facebook.presto.raptor.metadata; import com.facebook.airlift.log.Logger; +import com.facebook.airlift.stats.CounterStat; import com.facebook.presto.raptor.NodeSupplier; import com.facebook.presto.raptor.RaptorColumnHandle; import com.facebook.presto.raptor.storage.organization.ShardOrganizerDao; @@ -28,6 +29,7 @@ import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.Maps; @@ -40,6 +42,8 @@ import org.skife.jdbi.v2.exceptions.DBIException; import org.skife.jdbi.v2.tweak.HandleConsumer; import org.skife.jdbi.v2.util.ByteArrayMapper; +import org.weakref.jmx.Flatten; +import org.weakref.jmx.Managed; import javax.inject.Inject; @@ -51,6 +55,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; @@ -82,11 +87,13 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Throwables.throwIfInstanceOf; import static com.google.common.base.Verify.verify; +import static com.google.common.collect.ImmutableSet.toImmutableSet; import static com.google.common.collect.Iterables.partition; import static java.lang.Boolean.TRUE; import static java.lang.Math.multiplyExact; import static java.lang.String.format; import static java.sql.Statement.RETURN_GENERATED_KEYS; +import static java.sql.Types.BINARY; import static java.util.Arrays.asList; import static java.util.Collections.nCopies; import static java.util.Objects.requireNonNull; @@ -103,6 +110,7 @@ public class DatabaseShardManager private static final String INDEX_TABLE_PREFIX = "x_shards_t"; private static final int MAX_ADD_COLUMN_ATTEMPTS = 100; + private final DeltaDeleteStats deltaDeleteStats = new DeltaDeleteStats(); private final IDBI dbi; private final DaoSupplier shardDaoSupplier; private final ShardDao dao; @@ -280,12 +288,12 @@ public void commitShards(long transactionId, long tableId, List colu runCommit(transactionId, (handle) -> { externalBatchId.ifPresent(shardDaoSupplier.attach(handle)::insertExternalBatch); lockTable(handle, tableId); + // 1. Insert new shards insertShardsAndIndex(tableId, columns, shards, nodeIds, handle); - ShardStats stats = shardStats(shards); - MetadataDao metadata = handle.attach(MetadataDao.class); - metadata.updateTableStats(tableId, shards.size(), stats.getRowCount(), stats.getCompressedSize(), stats.getUncompressedSize()); - metadata.updateTableVersion(tableId, updateTime); + + // 2. Update statistics and table version + updateStatsAndVersion(handle, tableId, shards.size(), 0, stats.getRowCount(), stats.getCompressedSize(), stats.getUncompressedSize(), OptionalLong.of(updateTime)); }); } @@ -321,7 +329,7 @@ public void replaceShardUuids(long transactionId, long tableId, List if (!oldShardUuids.isEmpty() || !newShards.isEmpty()) { MetadataDao metadata = handle.attach(MetadataDao.class); - metadata.updateTableStats(tableId, shardCount, rowCount, compressedSize, uncompressedSize); + metadata.updateTableStats(tableId, shardCount, 0, rowCount, compressedSize, uncompressedSize); updateTime.ifPresent(time -> metadata.updateTableVersion(tableId, time)); } }); @@ -493,6 +501,447 @@ private static int batchSize(Connection connection) return (connection instanceof JdbcConnection) ? 1 : 1000; } + // TODO: Will merge these new function with old function once new feature is stable + /** + * two types of oldShardUuidsMap entry + * a. shard1 -> delete shard + * b. shard2 delta2 -> delete shard and delta + * + * see replaceDeltaUuids + * a is essentially equal to A + * b is essentially equal to B + * + * @param tableSupportsDeltaDelete table table_supports_delta_delete properties + */ + @Override + public void replaceShardUuids(long transactionId, long tableId, List columns, Map> oldShardAndDeltaUuids, Collection newShards, OptionalLong updateTime, boolean tableSupportsDeltaDelete) + { + Map nodeIds = toNodeIdMap(newShards); + + runCommit(transactionId, (handle) -> { + lockTable(handle, tableId); + + // For compaction + if (!updateTime.isPresent() && handle.attach(MetadataDao.class).isMaintenanceBlockedLocked(tableId)) { + throw new PrestoException(TRANSACTION_CONFLICT, "Maintenance is blocked for table"); + } + + // 1. Insert new shards + insertShardsAndIndex(tableId, columns, newShards, nodeIds, handle, false); + ShardStats newStats = shardStats(newShards); + long rowCount = newStats.getRowCount(); + long compressedSize = newStats.getCompressedSize(); + long uncompressedSize = newStats.getUncompressedSize(); + + // 2. Delete old shards and old delta + Set oldDeltaUuidSet = oldShardAndDeltaUuids.values().stream().filter(Optional::isPresent).map(Optional::get).collect(toImmutableSet()); + ShardStats stats = deleteShardsAndIndex(tableId, oldShardAndDeltaUuids, oldDeltaUuidSet, handle, tableSupportsDeltaDelete); + rowCount -= stats.getRowCount(); + compressedSize -= stats.getCompressedSize(); + uncompressedSize -= stats.getUncompressedSize(); + + // 3. Update statistics and table version + long deltaCountChange = -oldDeltaUuidSet.size(); + long shardCountChange = newShards.size() - oldShardAndDeltaUuids.size(); + if (!oldShardAndDeltaUuids.isEmpty() || !newShards.isEmpty()) { + updateStatsAndVersion(handle, tableId, shardCountChange, deltaCountChange, rowCount, compressedSize, uncompressedSize, updateTime); + } + }); + } + + /** + * Four types of shardMap + * A. shard1 delete shard + * B. shard2 old_delta2 delete shard and delta + * C. shard3 new_delta3 add delta + * D. shard4 old_delta4 new_delta4 replace delta + * + * Concurrent conflict resolution: + * (A, A) after deleting shard, verify deleted shard count + * (B, B) after deleting shard, verify deleted shard count / after deleting delta, verify deleted delta count + * (C, C) when updating shard's delta, check its old delta, after updating, verify updated shard count + * (D, D) after deleting delta, verify deleted delta count / verify updated shard count + * + * (A, B) won't happen at the same time + * (A, C) + * A first, B: after updating shard's delta, verfiy updated shard count + * B first, A: when deleting shard, check its delta, after deleting, verify deleted shard count + * (A, D) won't happen at the same time + * (B, C) won't happen at the same time + * (B, D) same way as (A,C) + * (C, D) won't happen at the same time + */ + public void replaceDeltaUuids(long transactionId, long tableId, List columns, Map shardMap, OptionalLong updateTime) + { + runCommit(transactionId, (handle) -> { + lockTable(handle, tableId); + + Set newDeltas = new HashSet<>(); + Set oldDeltaUuids = new HashSet<>(); + Map> shardsMapToDelete = new HashMap<>(); + Map shardsToUpdate = new HashMap<>(); + + // Initiate + for (Map.Entry entry : shardMap.entrySet()) { + UUID uuid = entry.getKey(); + DeltaInfoPair deltaInfoPair = entry.getValue(); + // Replace Shard's delta if new deltaShard isn't empty + if (deltaInfoPair.getNewDeltaDeleteShard().isPresent()) { + newDeltas.add(deltaInfoPair.getNewDeltaDeleteShard().get()); + shardsToUpdate.put(uuid, new DeltaUuidPair(deltaInfoPair.getOldDeltaDeleteShard(), deltaInfoPair.getNewDeltaDeleteShard().get().getShardUuid())); + } + // Delete Shard if deltaShard is empty + else { + shardsMapToDelete.put(uuid, deltaInfoPair.getOldDeltaDeleteShard()); + } + + if (deltaInfoPair.getOldDeltaDeleteShard().isPresent()) { + oldDeltaUuids.add(deltaInfoPair.getOldDeltaDeleteShard().get()); + } + } + + // 1. Insert new deltas + Map nodeIds = toNodeIdMap(newDeltas); + insertShardsAndIndex(tableId, columns, newDeltas, nodeIds, handle, true); + ShardStats newStats = shardStats(newDeltas); + long rowCount = -newStats.getRowCount(); + long compressedSize = newStats.getCompressedSize(); + long uncompressedSize = newStats.getUncompressedSize(); + + // 2. Delete toDelete shards and old deltas + // toDelete shards come from situation A + situation B + // old deltas come from situation B + situation D + ShardStats stats = deleteShardsAndIndex(tableId, shardsMapToDelete, oldDeltaUuids, handle, true); + rowCount -= stats.getRowCount(); + compressedSize -= stats.getCompressedSize(); + uncompressedSize -= stats.getUncompressedSize(); + + // 3. Update shard and delta relationship + updateShardsAndIndex(tableId, shardsToUpdate, handle); + + // 4. Update statistics and table version + int shardCountChange = -shardsMapToDelete.size(); + int deltaCountChange = newDeltas.size() - oldDeltaUuids.size(); + if (!newDeltas.isEmpty() || !oldDeltaUuids.isEmpty() || shardsToUpdate.isEmpty() || !shardsMapToDelete.isEmpty()) { + updateStatsAndVersion(handle, tableId, shardCountChange, deltaCountChange, rowCount, compressedSize, uncompressedSize, updateTime); + } + deltaDeleteStats.deletedShards.update(shardsMapToDelete.size()); + deltaDeleteStats.updatedShards.update(shardsToUpdate.size()); + }); + } + + private void updateStatsAndVersion(Handle handle, long tableId, long shardCountChange, long deltaCountChange, long rowCount, long compressedSize, long uncompressedSize, OptionalLong updateTime) + { + MetadataDao metadata = handle.attach(MetadataDao.class); + metadata.updateTableStats(tableId, shardCountChange, deltaCountChange, rowCount, compressedSize, uncompressedSize); + updateTime.ifPresent(time -> metadata.updateTableVersion(tableId, time)); + } + + /** + * Delete old shards and old deltas + * For call from replaceDeltaUuids: old shards and old deltas are not necessarily related, see the comment from the call + */ + private ShardStats deleteShardsAndIndex(long tableId, Map> oldShardUuidsMap, Set oldDeltaUuidSet, Handle handle, boolean tableSupportsDeltaDelete) + throws SQLException + { + if (tableSupportsDeltaDelete) { + ShardStats shardStats = deleteShardsAndIndexWithDelta(tableId, oldShardUuidsMap, handle); + long rowCount = shardStats.getRowCount(); + long compressedSize = shardStats.getCompressedSize(); + long uncompressedSize = shardStats.getUncompressedSize(); + + ShardStats deltaStats = deleteShardsAndIndexSimple(tableId, oldDeltaUuidSet, handle, true); + rowCount -= deltaStats.getRowCount(); // delta + compressedSize += deltaStats.getCompressedSize(); + uncompressedSize += deltaStats.getUncompressedSize(); + + return new ShardStats(rowCount, compressedSize, uncompressedSize); + } + + return deleteShardsAndIndexSimple(tableId, oldShardUuidsMap.keySet(), handle, false); + } + + /** + * For shards and delta + * + * Select id from `shards` table for both shard and delta shards + * - Purpose: 1. check the count as pre-check to avoid conflict 2. get statistics 3. use id to perform delete + * + * Insert into deleted_shards + * + * Delete from `shards_node` table (won't verify delete count: NONE-BUCKETED) for both shards and delta shards + * Delete from `shards` table verify delete count for both shards and delta shards + * Delete from index table verify delete count only for shards + */ + private ShardStats deleteShardsAndIndexSimple(long tableId, Set shardUuids, Handle handle, boolean isDelta) + throws SQLException + { + if (shardUuids.isEmpty()) { + return new ShardStats(0, 0, 0); + } + + long rowCount = 0; + long compressedSize = 0; + long uncompressedSize = 0; + + // for batch execution + for (List uuids : partition(shardUuids, 1000)) { + String args = Joiner.on(",").join(nCopies(uuids.size(), "?")); + ImmutableSet.Builder shardIdSet = ImmutableSet.builder(); + String selectShards = format("" + + "SELECT shard_id, row_count, compressed_size, uncompressed_size\n" + + "FROM shards\n" + + "WHERE shard_uuid IN (%s)", args); + try (PreparedStatement statement = handle.getConnection().prepareStatement(selectShards)) { + bindUuids(statement, uuids); + try (ResultSet rs = statement.executeQuery()) { + while (rs.next()) { + shardIdSet.add(rs.getLong("shard_id")); + rowCount += rs.getLong("row_count"); + compressedSize += rs.getLong("compressed_size"); + uncompressedSize += rs.getLong("uncompressed_size"); + } + } + } + Set shardIds = shardIdSet.build(); + if (shardIds.size() != uuids.size()) { + throw transactionConflict(); + } + + // For background cleaner + ShardDao dao = shardDaoSupplier.attach(handle); + dao.insertDeletedShards(uuids); + + String where = " WHERE shard_id IN (" + args + ")"; + String deleteFromShardNodes = "DELETE FROM shard_nodes " + where; + String deleteFromShards = "DELETE FROM shards " + where; + String deleteFromShardIndex = "DELETE FROM " + shardIndexTable(tableId) + where; + + try (PreparedStatement statement = handle.getConnection().prepareStatement(deleteFromShardNodes)) { + bindLongs(statement, shardIds); + statement.executeUpdate(); + } + + for (String sql : isDelta ? ImmutableList.of(deleteFromShards) : asList(deleteFromShards, deleteFromShardIndex)) { + try (PreparedStatement statement = handle.getConnection().prepareStatement(sql)) { + bindLongs(statement, shardIds); + if (statement.executeUpdate() != shardIds.size()) { + throw transactionConflict(); + } + } + } + } + + return new ShardStats(rowCount, compressedSize, uncompressedSize); + } + + /** + * ONLY for shards (NO delta) + * + * Select id from `shards` table + * - Purpose: 1. check the count as pre-check to avoid conflict 2. get statistics 3. use id to perform delete + * + * Insert into deleted_shards + * + * Delete from `shards_node` table (won't verify delete count: NONE-BUCKETED) + * Delete from `shards` table check delta verify delete count + * Delete from index table check delta verify delete count + */ + private ShardStats deleteShardsAndIndexWithDelta(long tableId, Map> oldShardUuidsMap, Handle handle) + throws SQLException + { + if (oldShardUuidsMap.isEmpty()) { + return new ShardStats(0, 0, 0); + } + String args = Joiner.on(",").join(nCopies(oldShardUuidsMap.size(), "?")); + + ImmutableMap.Builder shardUuidToIdBuilder = ImmutableMap.builder(); + long rowCount = 0; + long compressedSize = 0; + long uncompressedSize = 0; + + String selectShards = format("" + + "SELECT shard_id, shard_uuid, row_count, compressed_size, uncompressed_size\n" + + "FROM shards\n" + + "WHERE shard_uuid IN (%s)", args); + try (PreparedStatement statement = handle.getConnection().prepareStatement(selectShards)) { + bindUuids(statement, oldShardUuidsMap.keySet()); + try (ResultSet rs = statement.executeQuery()) { + while (rs.next()) { + shardUuidToIdBuilder.put(uuidFromBytes(rs.getBytes("shard_uuid")), rs.getLong("shard_id")); + rowCount += rs.getLong("row_count"); + compressedSize += rs.getLong("compressed_size"); + uncompressedSize += rs.getLong("uncompressed_size"); + } + } + } + Map shardUuidToId = shardUuidToIdBuilder.build(); + if (shardUuidToId.size() != oldShardUuidsMap.size()) { + throw transactionConflict(); + } + + // For background cleaner + ShardDao dao = shardDaoSupplier.attach(handle); + dao.insertDeletedShards(oldShardUuidsMap.keySet()); + + String where = " WHERE shard_id IN (" + args + ")"; + String deleteFromShardNodes = "DELETE FROM shard_nodes " + where; + try (PreparedStatement statement = handle.getConnection().prepareStatement(deleteFromShardNodes)) { + bindLongs(statement, shardUuidToId.values()); + statement.executeUpdate(); + } + + Connection connection = handle.getConnection(); + int updatedCount = 0; + try (ShardsAndIndexDeleter shardsAndIndexDeleter = new ShardsAndIndexDeleter(connection, tableId)) { + for (List batch : partition(oldShardUuidsMap.keySet(), batchSize(connection))) { + for (UUID uuid : batch) { + Optional deltaUuid = oldShardUuidsMap.get(uuid); + shardsAndIndexDeleter.delete(shardUuidToId.get(uuid), deltaUuid); + } + updatedCount += shardsAndIndexDeleter.execute(); + } + } + if (updatedCount != oldShardUuidsMap.size()) { + throw transactionConflict(); + } + + return new ShardStats(rowCount, compressedSize, uncompressedSize); + } + + /** + * For shards and delta + * + * Insert into `shards` for both shards and delta shards + * Insert into `shard_nodes` (non-bucketed) for both shards and delta shards + * Insert into index table only for shards + */ + private static void insertShardsAndIndex(long tableId, List columns, Collection shards, Map nodeIds, Handle handle, boolean isDelta) + throws SQLException + { + if (shards.isEmpty()) { + return; + } + boolean bucketed = shards.iterator().next().getBucketNumber().isPresent(); + + Connection connection = handle.getConnection(); + try (IndexInserter indexInserter = new IndexInserter(connection, tableId, columns)) { + for (List batch : partition(shards, batchSize(connection))) { + List shardIds = insertShards(connection, tableId, batch, isDelta); + + if (!bucketed) { + insertShardNodes(connection, nodeIds, shardIds, batch); + } + + if (!isDelta) { + for (int i = 0; i < batch.size(); i++) { + ShardInfo shard = batch.get(i); + Set shardNodes = shard.getNodeIdentifiers().stream() + .map(nodeIds::get) + .collect(toSet()); + indexInserter.insert( + shardIds.get(i), + shard.getShardUuid(), + shard.getBucketNumber(), + shardNodes, + shard.getColumnStats()); + } + indexInserter.execute(); + } + } + } + } + + /** + * For shards + * + * Select id from `shards` table + * - Purpose: 1. check the count as pre-check to avoid conflict 2. get statistics 3. use id to perform update + * + * Update `shards` table check delta verify delete count + * Update index table check delta verify delete count + */ + private void updateShardsAndIndex(long tableId, Map toUpdateShard, Handle handle) + throws SQLException + { + if (toUpdateShard.isEmpty()) { + return; + } + + String args = Joiner.on(",").join(nCopies(toUpdateShard.size(), "?")); + ImmutableMap.Builder shardMapBuilder = ImmutableMap.builder(); + String selectShards = format("" + + "SELECT shard_id, shard_uuid\n" + + "FROM shards\n" + + "WHERE shard_uuid IN (%s)", args); + try (PreparedStatement statement = handle.getConnection().prepareStatement(selectShards)) { + bindUuids(statement, toUpdateShard.keySet()); + try (ResultSet rs = statement.executeQuery()) { + while (rs.next()) { + shardMapBuilder.put(rs.getLong("shard_id"), uuidFromBytes(rs.getBytes("shard_uuid"))); + } + } + } + Map shardIdToUuid = shardMapBuilder.build(); + if (toUpdateShard.size() != shardIdToUuid.size()) { + throw transactionConflict(); + } + + int updatedCount = 0; + try (ShardsAndIndexUpdater shardsAndIndexUpdater = new ShardsAndIndexUpdater(handle.getConnection(), tableId)) { + for (List batch : partition(shardIdToUuid.keySet(), batchSize(handle.getConnection()))) { + for (long shardId : batch) { + shardsAndIndexUpdater.update( + shardId, + toUpdateShard.get(shardIdToUuid.get(shardId)).getOldDeltaUuid(), + toUpdateShard.get(shardIdToUuid.get(shardId)).getNewDeltaUuid()); + } + updatedCount += shardsAndIndexUpdater.execute(); + } + } + if (updatedCount != shardIdToUuid.size()) { + log.error("updatedCount is not equal to shardIdToUuid size"); + throw transactionConflict(); + } + } + + private static List insertShards(Connection connection, long tableId, List shards, boolean isDelta) + throws SQLException + { + String sql = "" + + "INSERT INTO shards (shard_uuid, table_id, is_delta, delta_uuid, create_time, row_count, compressed_size, uncompressed_size, xxhash64, bucket_number)\n" + + "VALUES (?, ?, ?, ?, CURRENT_TIMESTAMP, ?, ?, ?, ?, ?)"; + + try (PreparedStatement statement = connection.prepareStatement(sql, RETURN_GENERATED_KEYS)) { + for (ShardInfo shard : shards) { + statement.setBytes(1, uuidToBytes(shard.getShardUuid())); + statement.setLong(2, tableId); + statement.setBoolean(3, isDelta); + statement.setNull(4, BINARY); + statement.setLong(5, shard.getRowCount()); + statement.setLong(6, shard.getCompressedSize()); + statement.setLong(7, shard.getUncompressedSize()); + statement.setLong(8, shard.getXxhash64()); + bindOptionalInt(statement, 9, shard.getBucketNumber()); + statement.addBatch(); + } + statement.executeBatch(); + + ImmutableList.Builder builder = ImmutableList.builder(); + try (ResultSet keys = statement.getGeneratedKeys()) { + while (keys.next()) { + builder.add(keys.getLong(1)); + } + } + List shardIds = builder.build(); + + if (shardIds.size() != shards.size()) { + throw new PrestoException(RAPTOR_ERROR, "Wrong number of generated keys for inserted shards"); + } + return shardIds; + } + } + private Map toNodeIdMap(Collection shards) { Set identifiers = shards.stream() @@ -521,15 +970,15 @@ public Set getNodeShards(String nodeIdentifier, long tableId) } @Override - public ResultIterator getShardNodes(long tableId, TupleDomain effectivePredicate) + public ResultIterator getShardNodes(long tableId, TupleDomain effectivePredicate, boolean tableSupportsDeltaDelete) { - return new ShardIterator(tableId, false, Optional.empty(), effectivePredicate, dbi); + return new ShardIterator(tableId, false, tableSupportsDeltaDelete, Optional.empty(), effectivePredicate, dbi); } @Override - public ResultIterator getShardNodesBucketed(long tableId, boolean merged, List bucketToNode, TupleDomain effectivePredicate) + public ResultIterator getShardNodesBucketed(long tableId, boolean merged, List bucketToNode, TupleDomain effectivePredicate, boolean tableSupportsDeltaDelete) { - return new ShardIterator(tableId, merged, Optional.of(bucketToNode), effectivePredicate, dbi); + return new ShardIterator(tableId, merged, tableSupportsDeltaDelete, Optional.of(bucketToNode), effectivePredicate, dbi); } @Override @@ -826,7 +1275,7 @@ private static void lockTable(Handle handle, long tableId) } } - private static PrestoException transactionConflict() + public static PrestoException transactionConflict() { return new PrestoException(TRANSACTION_CONFLICT, "Table was updated by a different transaction. Please retry the operation."); } @@ -911,4 +1360,39 @@ public long getUncompressedSize() return uncompressedSize; } } + + private static class DeltaUuidPair + { + private Optional oldDeltaUuid; + private UUID newDeltaUuid; + + public DeltaUuidPair(Optional oldDeltaUuid, UUID newDeltaUuid) + { + this.oldDeltaUuid = oldDeltaUuid; + this.newDeltaUuid = newDeltaUuid; + } + + public Optional getOldDeltaUuid() + { + return oldDeltaUuid; + } + + public UUID getNewDeltaUuid() + { + return newDeltaUuid; + } + } + + private static class DeltaDeleteStats + { + private final CounterStat updatedShards = new CounterStat(); + private final CounterStat deletedShards = new CounterStat(); + } + + @Managed + @Flatten + public DeltaDeleteStats getDeltaDeleteStats() + { + return deltaDeleteStats; + } } diff --git a/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/DeltaInfoPair.java b/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/DeltaInfoPair.java new file mode 100644 index 0000000000000..98b10fdf408d3 --- /dev/null +++ b/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/DeltaInfoPair.java @@ -0,0 +1,64 @@ +/* + * 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 com.facebook.presto.raptor.metadata; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Optional; +import java.util.UUID; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.util.Objects.requireNonNull; + +/** + * Optional oldDeltaDeleteShard is the old delta for the target file. + * Optional newDeltaDeleteShard the new delta for the target file. + * Note: Optional> being Optional.empty() means deleting the target file. + */ +public class DeltaInfoPair +{ + private final Optional oldDeltaDeleteShard; + private final Optional newDeltaDeleteShard; + + @JsonCreator + public DeltaInfoPair( + @JsonProperty("oldDeltaDeleteShard") Optional oldDeltaDeleteShard, + @JsonProperty("newDeltaDeleteShard") Optional newDeltaDeleteShard) + { + this.oldDeltaDeleteShard = requireNonNull(oldDeltaDeleteShard, "Optional oldDeltaDeleteShard is null"); + this.newDeltaDeleteShard = requireNonNull(newDeltaDeleteShard, "Optional newDeltaDeleteShard is null"); + } + + @JsonProperty + public Optional getOldDeltaDeleteShard() + { + return oldDeltaDeleteShard; + } + + @JsonProperty + public Optional getNewDeltaDeleteShard() + { + return newDeltaDeleteShard; + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("oldDeltaDeleteShard", oldDeltaDeleteShard) + .add("newDeltaDeleteShard", newDeltaDeleteShard) + .toString(); + } +} diff --git a/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/MetadataDao.java b/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/MetadataDao.java index ba25904a5d95a..4209a10bd009b 100644 --- a/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/MetadataDao.java +++ b/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/MetadataDao.java @@ -115,11 +115,11 @@ List getViews( @SqlUpdate("INSERT INTO tables (\n" + " schema_name, table_name, compaction_enabled, organization_enabled, distribution_id,\n" + " create_time, update_time, table_version,\n" + - " shard_count, row_count, compressed_size, uncompressed_size, table_supports_delta_delete)\n" + + " shard_count, delta_count, row_count, compressed_size, uncompressed_size, table_supports_delta_delete)\n" + "VALUES (\n" + " :schemaName, :tableName, :compactionEnabled, :organizationEnabled, :distributionId,\n" + " :createTime, :createTime, 0,\n" + - " 0, 0, 0, 0, :tableSupportsDeltaDelete)\n") + " 0, 0, 0, 0, 0, :tableSupportsDeltaDelete)\n") @GetGeneratedKeys long insertTable( @Bind("schemaName") String schemaName, @@ -139,14 +139,16 @@ void updateTableVersion( @Bind("updateTime") long updateTime); @SqlUpdate("UPDATE tables SET\n" + - " shard_count = shard_count + :shardCount \n" + + " shard_count = shard_count + :shardCountChange \n" + + ", delta_count = delta_count + :deltaCountChange \n" + ", row_count = row_count + :rowCount\n" + ", compressed_size = compressed_size + :compressedSize\n" + ", uncompressed_size = uncompressed_size + :uncompressedSize\n" + "WHERE table_id = :tableId") void updateTableStats( @Bind("tableId") long tableId, - @Bind("shardCount") long shardCount, + @Bind("shardCountChange") long shardCountChange, + @Bind("deltaCountChange") long deltaCountChange, @Bind("rowCount") long rowCount, @Bind("compressedSize") long compressedSize, @Bind("uncompressedSize") long uncompressedSize); @@ -273,7 +275,7 @@ List getColumnMetadataRows( @Bind("tableName") String tableName); @SqlQuery("SELECT schema_name, table_name, create_time, update_time, table_version,\n" + - " shard_count, row_count, compressed_size, uncompressed_size\n" + + " shard_count, delta_count, row_count, compressed_size, uncompressed_size\n" + "FROM tables\n" + "WHERE (schema_name = :schemaName OR :schemaName IS NULL)\n" + " AND (table_name = :tableName OR :tableName IS NULL)\n" + diff --git a/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/ShardDao.java b/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/ShardDao.java index 1ff4edeca4434..d51ece99a2d60 100644 --- a/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/ShardDao.java +++ b/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/ShardDao.java @@ -36,7 +36,7 @@ public interface ShardDao int CLEANABLE_SHARDS_BATCH_SIZE = 1000; int CLEANUP_TRANSACTIONS_BATCH_SIZE = 10_000; - String SHARD_METADATA_COLUMNS = "table_id, shard_id, shard_uuid, bucket_number, row_count, compressed_size, uncompressed_size, xxhash64"; + String SHARD_METADATA_COLUMNS = "table_id, shard_id, shard_uuid, is_delta, delta_uuid, bucket_number, row_count, compressed_size, uncompressed_size, xxhash64"; @SqlUpdate("INSERT INTO nodes (node_identifier) VALUES (:nodeIdentifier)") @GetGeneratedKeys diff --git a/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/ShardDeleteDelta.java b/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/ShardDeleteDelta.java new file mode 100644 index 0000000000000..0dafce061a4bf --- /dev/null +++ b/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/ShardDeleteDelta.java @@ -0,0 +1,58 @@ +/* + * 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 com.facebook.presto.raptor.metadata; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.UUID; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.util.Objects.requireNonNull; + +public class ShardDeleteDelta +{ + private final UUID oldShardUuid; + private final DeltaInfoPair deltaInfoPair; + + @JsonCreator + public ShardDeleteDelta( + @JsonProperty("oldShardUuid") UUID oldShardUuid, + @JsonProperty("deltaInfoPair") DeltaInfoPair deltaInfoPair) + { + this.oldShardUuid = requireNonNull(oldShardUuid, "oldShardUuids is null"); + this.deltaInfoPair = requireNonNull(deltaInfoPair, "deltaInfoPair is null"); + } + + @JsonProperty + public UUID getOldShardUuid() + { + return oldShardUuid; + } + + @JsonProperty + public DeltaInfoPair getDeltaInfoPair() + { + return deltaInfoPair; + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("oldShardUuid", oldShardUuid) + .add("deltaInfoPair", deltaInfoPair) + .toString(); + } +} diff --git a/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/ShardIterator.java b/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/ShardIterator.java index 005c0a2b36a7b..3cad6a96ed094 100644 --- a/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/ShardIterator.java +++ b/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/ShardIterator.java @@ -54,6 +54,7 @@ final class ShardIterator private final Map nodeMap = new HashMap<>(); private final boolean merged; + private final boolean tableSupportsDeltaDelete; private final List bucketToNode; private final ShardDao dao; private final Connection connection; @@ -64,21 +65,22 @@ final class ShardIterator public ShardIterator( long tableId, boolean merged, + boolean tableSupportsDeltaDelete, Optional> bucketToNode, TupleDomain effectivePredicate, IDBI dbi) { this.merged = merged; + this.tableSupportsDeltaDelete = tableSupportsDeltaDelete; this.bucketToNode = bucketToNode.orElse(null); ShardPredicate predicate = ShardPredicate.create(effectivePredicate); String sql; - if (bucketToNode.isPresent()) { - sql = "SELECT shard_uuid, bucket_number FROM %s WHERE %s ORDER BY bucket_number"; - } - else { - sql = "SELECT shard_uuid, node_ids FROM %s WHERE %s"; - } + sql = "SELECT shard_uuid, " + + (tableSupportsDeltaDelete ? "delta_shard_uuid, " : "") + + (bucketToNode.isPresent() ? "bucket_number " : "node_ids ") + + "FROM %s WHERE %s " + + (bucketToNode.isPresent() ? "ORDER BY bucket_number" : ""); sql = format(sql, shardIndexTable(tableId), predicate.getPredicate()); dao = onDemandDao(dbi, ShardDao.class); @@ -138,6 +140,7 @@ private BucketShards compute() } UUID shardUuid = uuidFromBytes(resultSet.getBytes("shard_uuid")); + Optional deltaShardUuid = getDeltaShardUuidFromResultSet(); Set nodeIdentifiers; OptionalInt bucketNumber = OptionalInt.empty(); @@ -151,7 +154,7 @@ private BucketShards compute() nodeIdentifiers = getNodeIdentifiers(nodeIds, shardUuid); } - ShardNodes shard = new ShardNodes(shardUuid, nodeIdentifiers); + ShardNodes shard = new ShardNodes(shardUuid, deltaShardUuid, nodeIdentifiers); return new BucketShards(bucketNumber, ImmutableSet.of(shard)); } @@ -176,10 +179,11 @@ private BucketShards computeMerged() do { UUID shardUuid = uuidFromBytes(resultSet.getBytes("shard_uuid")); + Optional deltaShardUuid = getDeltaShardUuidFromResultSet(); int bucket = resultSet.getInt("bucket_number"); Set nodeIdentifiers = ImmutableSet.of(getBucketNode(bucket)); - shards.add(new ShardNodes(shardUuid, nodeIdentifiers)); + shards.add(new ShardNodes(shardUuid, deltaShardUuid, nodeIdentifiers)); } while (resultSet.next() && resultSet.getInt("bucket_number") == bucketNumber); @@ -218,4 +222,13 @@ private void fetchNodes() nodeMap.put(node.getNodeId(), node.getNodeIdentifier()); } } + + private Optional getDeltaShardUuidFromResultSet() + throws SQLException + { + if (tableSupportsDeltaDelete && resultSet.getBytes("delta_shard_uuid") != null) { + return Optional.of(uuidFromBytes(resultSet.getBytes("delta_shard_uuid"))); + } + return Optional.empty(); + } } diff --git a/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/ShardManager.java b/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/ShardManager.java index 797bec3a2e488..7377611853e52 100644 --- a/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/ShardManager.java +++ b/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/ShardManager.java @@ -52,6 +52,21 @@ public interface ShardManager */ void replaceShardUuids(long transactionId, long tableId, List columns, Set oldShardUuids, Collection newShards, OptionalLong updateTime); + /** + * Replace oldShardsUuids with newShards. + * Used by compaction with tableSupportsDeltaDelete: Delete oldShardsUuids with their delta shards and add newShards formed by compaction + * @param oldShardAndDeltaUuids oldShardsUuids with their delta shards + * @param newShards newShards formed from compaction + */ + void replaceShardUuids(long transactionId, long tableId, List columns, Map> oldShardAndDeltaUuids, Collection newShards, OptionalLong updateTime, boolean tableSupportsDeltaDelete); + + /** + * Replace oldDeltaDeleteShard with newDeltaDeleteShard. + * Used by delta delete. + * @param shardMap UUID in the map is the target file. DeltaInfoPair in the map is the change of delta. + */ + void replaceDeltaUuids(long transactionId, long tableId, List columns, Map shardMap, OptionalLong updateTime); + /** * Get shard metadata for a shard. */ @@ -70,12 +85,12 @@ public interface ShardManager /** * Return the shard nodes for a non-bucketed table. */ - ResultIterator getShardNodes(long tableId, TupleDomain effectivePredicate); + ResultIterator getShardNodes(long tableId, TupleDomain effectivePredicate, boolean tableSupportsDeltaDelete); /** * Return the shard nodes for a bucketed table. */ - ResultIterator getShardNodesBucketed(long tableId, boolean merged, List bucketToNode, TupleDomain effectivePredicate); + ResultIterator getShardNodesBucketed(long tableId, boolean merged, List bucketToNode, TupleDomain effectivePredicate, boolean tableSupportsDeltaDelete); /** * Remove all old shard assignments and assign a shard to a node diff --git a/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/ShardMetadata.java b/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/ShardMetadata.java index 86980f6e8186b..11059a310eb87 100644 --- a/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/ShardMetadata.java +++ b/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/ShardMetadata.java @@ -20,6 +20,7 @@ import java.sql.ResultSet; import java.sql.SQLException; import java.util.Objects; +import java.util.Optional; import java.util.OptionalInt; import java.util.OptionalLong; import java.util.UUID; @@ -38,6 +39,8 @@ public class ShardMetadata private final long tableId; private final long shardId; private final UUID shardUuid; + private final boolean isDelta; + private final Optional deltaUuid; private final OptionalInt bucketNumber; private final long rowCount; private final long compressedSize; @@ -50,6 +53,8 @@ public ShardMetadata( long tableId, long shardId, UUID shardUuid, + boolean isDelta, + Optional deltaUuid, OptionalInt bucketNumber, long rowCount, long compressedSize, @@ -67,6 +72,8 @@ public ShardMetadata( this.tableId = tableId; this.shardId = shardId; this.shardUuid = requireNonNull(shardUuid, "shardUuid is null"); + this.isDelta = isDelta; + this.deltaUuid = deltaUuid; this.bucketNumber = requireNonNull(bucketNumber, "bucketNumber is null"); this.rowCount = rowCount; this.compressedSize = compressedSize; @@ -91,6 +98,16 @@ public long getShardId() return shardId; } + public boolean isDelta() + { + return isDelta; + } + + public Optional getDeltaUuid() + { + return deltaUuid; + } + public OptionalInt getBucketNumber() { return bucketNumber; @@ -132,6 +149,8 @@ public ShardMetadata withTimeRange(long rangeStart, long rangeEnd) tableId, shardId, shardUuid, + isDelta, + deltaUuid, bucketNumber, rowCount, compressedSize, @@ -148,10 +167,12 @@ public String toString() .add("tableId", tableId) .add("shardId", shardId) .add("shardUuid", shardUuid) + .add("isDelta", isDelta) .add("rowCount", rowCount) .add("compressedSize", DataSize.succinctBytes(compressedSize)) .add("uncompressedSize", DataSize.succinctBytes(uncompressedSize)); + deltaUuid.ifPresent(uuid -> stringHelper.add("deltaUuid", uuid)); if (bucketNumber.isPresent()) { stringHelper.add("bucketNumber", bucketNumber.getAsInt()); } @@ -179,6 +200,8 @@ public boolean equals(Object o) ShardMetadata that = (ShardMetadata) o; return Objects.equals(tableId, that.tableId) && Objects.equals(shardId, that.shardId) && + Objects.equals(isDelta, that.isDelta) && + Objects.equals(deltaUuid, that.deltaUuid) && Objects.equals(bucketNumber, that.bucketNumber) && Objects.equals(rowCount, that.rowCount) && Objects.equals(compressedSize, that.compressedSize) && @@ -196,6 +219,8 @@ public int hashCode() tableId, shardId, shardUuid, + isDelta, + deltaUuid, bucketNumber, rowCount, compressedSize, @@ -216,6 +241,8 @@ public ShardMetadata map(int index, ResultSet r, StatementContext ctx) r.getLong("table_id"), r.getLong("shard_id"), uuidFromBytes(r.getBytes("shard_uuid")), + r.getBoolean("is_delta"), + r.getBytes("delta_uuid") == null ? Optional.empty() : Optional.of(uuidFromBytes(r.getBytes("delta_uuid"))), getOptionalInt(r, "bucket_number"), r.getLong("row_count"), r.getLong("compressed_size"), diff --git a/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/ShardNodes.java b/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/ShardNodes.java index 1acfde92bc9ef..9da5b11def032 100644 --- a/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/ShardNodes.java +++ b/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/ShardNodes.java @@ -16,6 +16,7 @@ import com.google.common.collect.ImmutableSet; import java.util.Objects; +import java.util.Optional; import java.util.Set; import java.util.UUID; @@ -25,11 +26,13 @@ public class ShardNodes { private final UUID shardUuid; + private final Optional deltaShardUuid; private final Set nodeIdentifiers; - public ShardNodes(UUID shardUuid, Set nodeIdentifiers) + public ShardNodes(UUID shardUuid, Optional deltaShardUuid, Set nodeIdentifiers) { this.shardUuid = requireNonNull(shardUuid, "shardUuid is null"); + this.deltaShardUuid = requireNonNull(deltaShardUuid, "deltaShardUuid is null"); this.nodeIdentifiers = ImmutableSet.copyOf(requireNonNull(nodeIdentifiers, "nodeIdentifiers is null")); } @@ -38,6 +41,11 @@ public UUID getShardUuid() return shardUuid; } + public Optional getDeltaShardUuid() + { + return deltaShardUuid; + } + public Set getNodeIdentifiers() { return nodeIdentifiers; @@ -54,6 +62,7 @@ public boolean equals(Object obj) } ShardNodes other = (ShardNodes) obj; return Objects.equals(this.shardUuid, other.shardUuid) && + Objects.equals(this.deltaShardUuid, other.deltaShardUuid) && Objects.equals(this.nodeIdentifiers, other.nodeIdentifiers); } @@ -68,6 +77,7 @@ public String toString() { return toStringHelper(this) .add("shardUuid", shardUuid) + .add("deltaShardUuid", deltaShardUuid) .add("nodeIdentifiers", nodeIdentifiers) .toString(); } diff --git a/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/ShardsAndIndexDeleter.java b/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/ShardsAndIndexDeleter.java new file mode 100644 index 0000000000000..5391f5326756d --- /dev/null +++ b/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/ShardsAndIndexDeleter.java @@ -0,0 +1,107 @@ +/* + * 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 com.facebook.presto.raptor.metadata; + +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.SQLException; +import java.util.Arrays; +import java.util.Optional; +import java.util.UUID; + +import static com.facebook.presto.raptor.metadata.DatabaseShardManager.shardIndexTable; +import static com.facebook.presto.raptor.util.UuidUtil.uuidToBytes; + +public class ShardsAndIndexDeleter + implements AutoCloseable +{ + private final PreparedStatement deleteShardStatement; + private final PreparedStatement deleteShardEmptyDeltaStatement; + private final PreparedStatement deleteIndexStatement; + private final PreparedStatement deleteIndexEmptyDeltaStatement; + + public ShardsAndIndexDeleter(Connection connection, long tableId) + throws SQLException + { + // DELETE FROM table_name + // WHERE condition; + String deleteIndexSql = "" + + "DELETE FROM " + shardIndexTable(tableId) + " \n" + + " WHERE shard_id = ? AND delta_shard_uuid = ?"; + String deleteIndexSqlEmptyDelta = "" + + "DELETE FROM " + shardIndexTable(tableId) + " \n" + + " WHERE shard_id = ? AND delta_shard_uuid IS NULL"; + String deleteShardSql = "" + + "DELETE FROM shards \n" + + " WHERE shard_id = ? AND delta_uuid = ?"; + String deleteShardSqlEmptyDelta = "" + + "DELETE FROM shards \n" + + " WHERE shard_id = ? AND delta_uuid IS NULL"; + this.deleteIndexStatement = connection.prepareStatement(deleteIndexSql); + this.deleteIndexEmptyDeltaStatement = connection.prepareStatement(deleteIndexSqlEmptyDelta); + this.deleteShardStatement = connection.prepareStatement(deleteShardSql); + this.deleteShardEmptyDeltaStatement = connection.prepareStatement(deleteShardSqlEmptyDelta); + } + + public void delete(Long id, Optional deltaUuid) + throws SQLException + { + if (deltaUuid.isPresent()) { + deleteShardStatement.setLong(1, id); + deleteShardStatement.setBytes(2, uuidToBytes(deltaUuid.get())); + deleteShardStatement.addBatch(); + + deleteIndexStatement.setLong(1, id); + deleteIndexStatement.setBytes(2, uuidToBytes(deltaUuid.get())); + deleteIndexStatement.addBatch(); + } + else { + deleteShardEmptyDeltaStatement.setLong(1, id); + deleteShardEmptyDeltaStatement.addBatch(); + deleteIndexEmptyDeltaStatement.setLong(1, id); + deleteIndexEmptyDeltaStatement.addBatch(); + } + } + + public int execute() + throws SQLException + { + int shardsUpdatedCount = 0; + int indexUpdatedCount = 0; + shardsUpdatedCount += updatedCount(deleteShardStatement.executeBatch()); + shardsUpdatedCount += updatedCount(deleteShardEmptyDeltaStatement.executeBatch()); + indexUpdatedCount += updatedCount(deleteIndexStatement.executeBatch()); + indexUpdatedCount += updatedCount(deleteIndexEmptyDeltaStatement.executeBatch()); + + if (shardsUpdatedCount != indexUpdatedCount) { + throw DatabaseShardManager.transactionConflict(); + } + return shardsUpdatedCount; + } + + @Override + public void close() + throws SQLException + { + deleteShardStatement.close(); + deleteShardEmptyDeltaStatement.close(); + deleteIndexStatement.close(); + deleteIndexEmptyDeltaStatement.close(); + } + + static int updatedCount(int[] executeBatch) + { + return Arrays.stream(executeBatch).sum(); + } +} diff --git a/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/ShardsAndIndexUpdater.java b/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/ShardsAndIndexUpdater.java new file mode 100644 index 0000000000000..a808d5ab792c8 --- /dev/null +++ b/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/ShardsAndIndexUpdater.java @@ -0,0 +1,119 @@ +/* + * 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 com.facebook.presto.raptor.metadata; + +import com.facebook.airlift.log.Logger; + +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.SQLException; +import java.util.Optional; +import java.util.UUID; + +import static com.facebook.presto.raptor.metadata.DatabaseShardManager.shardIndexTable; +import static com.facebook.presto.raptor.metadata.ShardsAndIndexDeleter.updatedCount; +import static com.facebook.presto.raptor.util.UuidUtil.uuidToBytes; + +public class ShardsAndIndexUpdater + implements AutoCloseable +{ + private static final Logger log = Logger.get(ShardsAndIndexUpdater.class); + + private final PreparedStatement updateShardEmptyDeltaStatement; + private final PreparedStatement updateShardStatement; + private final PreparedStatement updateIndexEmptyDeltaStatement; + private final PreparedStatement updateIndexStatement; + + public ShardsAndIndexUpdater(Connection connection, long tableId) + throws SQLException + { + // UPDATE table_name + // SET column1 = value1, column2 = value2, ... + // WHERE condition; + String updateIndexSqlEmptyDelta = "" + + "UPDATE " + shardIndexTable(tableId) + " SET \n" + + " delta_shard_uuid = ?\n" + + " WHERE shard_id = ? AND delta_shard_uuid IS NULL"; + String updateIndexSql = "" + + "UPDATE " + shardIndexTable(tableId) + " SET \n" + + " delta_shard_uuid = ?\n" + + " WHERE shard_id = ? AND delta_shard_uuid = ?"; + String updateShardSqlEmptyDelta = "" + + "UPDATE shards SET \n" + + " delta_uuid = ?\n" + + " WHERE shard_id = ? AND delta_uuid IS NULL"; + String updateShardSql = "" + + "UPDATE shards SET \n" + + " delta_uuid = ?\n" + + " WHERE shard_id = ? AND delta_uuid = ?"; + + this.updateShardEmptyDeltaStatement = connection.prepareStatement(updateShardSqlEmptyDelta); + this.updateIndexEmptyDeltaStatement = connection.prepareStatement(updateIndexSqlEmptyDelta); + this.updateShardStatement = connection.prepareStatement(updateShardSql); + this.updateIndexStatement = connection.prepareStatement(updateIndexSql); + } + + public void update(long oldId, Optional oldUuid, UUID newUuid) + throws SQLException + { + if (oldUuid.isPresent()) { + updateShardStatement.setBytes(1, uuidToBytes(newUuid)); + updateShardStatement.setLong(2, oldId); + updateShardStatement.setBytes(3, uuidToBytes(oldUuid.get())); + updateShardStatement.addBatch(); + + updateIndexStatement.setBytes(1, uuidToBytes(newUuid)); + updateIndexStatement.setLong(2, oldId); + updateIndexStatement.setBytes(3, uuidToBytes(oldUuid.get())); + updateIndexStatement.addBatch(); + } + else { + updateShardEmptyDeltaStatement.setBytes(1, uuidToBytes(newUuid)); + updateShardEmptyDeltaStatement.setLong(2, oldId); + updateShardEmptyDeltaStatement.addBatch(); + + updateIndexEmptyDeltaStatement.setBytes(1, uuidToBytes(newUuid)); + updateIndexEmptyDeltaStatement.setLong(2, oldId); + updateIndexEmptyDeltaStatement.addBatch(); + } + } + + public int execute() + throws SQLException + { + int shardsUpdatedCount = 0; + int indexUpdatedCount = 0; + shardsUpdatedCount += updatedCount(updateShardStatement.executeBatch()); + shardsUpdatedCount += updatedCount(updateShardEmptyDeltaStatement.executeBatch()); + indexUpdatedCount += updatedCount(updateIndexStatement.executeBatch()); + indexUpdatedCount += updatedCount(updateIndexEmptyDeltaStatement.executeBatch()); + log.info("ShardsAndIndexUpdater shardsUpdatedCount:" + shardsUpdatedCount); + log.info("ShardsAndIndexUpdater indexUpdatedCount:" + indexUpdatedCount); + + if (shardsUpdatedCount != indexUpdatedCount) { + throw DatabaseShardManager.transactionConflict(); + } + return shardsUpdatedCount; + } + + @Override + public void close() + throws SQLException + { + updateShardStatement.close(); + updateShardEmptyDeltaStatement.close(); + updateIndexStatement.close(); + updateIndexEmptyDeltaStatement.close(); + } +} diff --git a/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/TableStatsRow.java b/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/TableStatsRow.java index 92da694886b32..7bb1e9938dbab 100644 --- a/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/TableStatsRow.java +++ b/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/TableStatsRow.java @@ -29,6 +29,7 @@ public class TableStatsRow private final long updateTime; private final long tableVersion; private final long shardCount; + private final long deltaCount; private final long rowCount; private final long compressedSize; private final long uncompressedSize; @@ -40,6 +41,7 @@ public TableStatsRow( long updateTime, long tableVersion, long shardCount, + long deltaCount, long rowCount, long compressedSize, long uncompressedSize) @@ -50,6 +52,7 @@ public TableStatsRow( this.updateTime = updateTime; this.tableVersion = tableVersion; this.shardCount = shardCount; + this.deltaCount = deltaCount; this.rowCount = rowCount; this.compressedSize = compressedSize; this.uncompressedSize = uncompressedSize; @@ -85,6 +88,11 @@ public long getShardCount() return shardCount; } + public long getDeltaCount() + { + return deltaCount; + } + public long getRowCount() { return rowCount; @@ -114,6 +122,7 @@ public TableStatsRow map(int index, ResultSet rs, StatementContext context) rs.getLong("update_time"), rs.getLong("table_version"), rs.getLong("shard_count"), + rs.getLong("delta_count"), rs.getLong("row_count"), rs.getLong("compressed_size"), rs.getLong("uncompressed_size")); diff --git a/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/DeltaShardLoader.java b/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/DeltaShardLoader.java new file mode 100644 index 0000000000000..0143bd3624fb2 --- /dev/null +++ b/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/DeltaShardLoader.java @@ -0,0 +1,111 @@ +/* + * 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 com.facebook.presto.raptor.storage; + +import it.unimi.dsi.fastutil.ints.IntArrayList; +import org.apache.hadoop.fs.FileSystem; + +import java.util.BitSet; +import java.util.Optional; +import java.util.UUID; + +import static java.lang.Math.toIntExact; +import static java.util.Objects.requireNonNull; + +public class DeltaShardLoader +{ + private static final RowsToKeepResult KEEP_ALL = new RowsToKeepResult(true, new IntArrayList(0)); + + private final Optional deltaShardUuid; + private final boolean tableSupportsDeltaDelete; + private final OrcStorageManager orcStorageManager; + private final FileSystem fileSystem; + + private boolean loaded; + private Optional rowsDeleted = Optional.empty(); + + public DeltaShardLoader( + Optional deltaShardUuid, + boolean tableSupportsDeltaDelete, + OrcStorageManager orcStorageManager, + FileSystem fileSystem) + { + this.deltaShardUuid = requireNonNull(deltaShardUuid, "deltaShardUuid is null"); + this.tableSupportsDeltaDelete = tableSupportsDeltaDelete; + this.orcStorageManager = requireNonNull(orcStorageManager, "storageManager is null"); + this.fileSystem = requireNonNull(fileSystem, "fileSystem is null"); + } + + public RowsToKeepResult getRowsToKeep(int batchSize, long filePosition) + { + // lazy load rowsDeleted until getNextPage call + // rowsDeleted will only be loaded once + Optional rowsDeleted = getRowsDeleted(); + if (rowsDeleted.isPresent() && rowsDeleted.get().cardinality() > 0) { + IntArrayList rowsToKeep = new IntArrayList(batchSize); + for (int position = 0; position < batchSize; position++) { + if (!rowsDeleted.get().get(toIntExact(filePosition) + position)) { + rowsToKeep.add(position); + } + } + if (rowsToKeep.size() == batchSize) { + return KEEP_ALL; + } + return new RowsToKeepResult(false, rowsToKeep); + } + return KEEP_ALL; + } + + private Optional getRowsDeleted() + { + // Just load once + if (!loaded && tableSupportsDeltaDelete && deltaShardUuid.isPresent()) { + rowsDeleted = orcStorageManager.getRowsFromUuid(fileSystem, deltaShardUuid); + loaded = true; + } + return rowsDeleted; + } + + static class RowsToKeepResult + { + private final boolean keepAll; + private final IntArrayList rowsToKeep; + + private RowsToKeepResult(boolean keepAll, IntArrayList rowsToKeep) + { + this.keepAll = keepAll; + this.rowsToKeep = requireNonNull(rowsToKeep, "rowsToKeep is null"); + } + + public boolean keepAll() + { + return keepAll; + } + + public int size() + { + return rowsToKeep.size(); + } + + public IntArrayList getRowsToKeep() + { + return rowsToKeep; + } + + public int[] elements() + { + return rowsToKeep.elements(); + } + } +} diff --git a/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/DeltaShardRewriter.java b/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/DeltaShardRewriter.java new file mode 100644 index 0000000000000..fd490d5a9d3a8 --- /dev/null +++ b/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/DeltaShardRewriter.java @@ -0,0 +1,149 @@ +/* + * 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 com.facebook.presto.raptor.storage; + +import com.facebook.airlift.json.JsonCodec; +import com.facebook.presto.raptor.filesystem.FileSystemContext; +import com.facebook.presto.raptor.metadata.DeltaInfoPair; +import com.facebook.presto.raptor.metadata.ShardDeleteDelta; +import com.facebook.presto.raptor.metadata.ShardInfo; +import com.facebook.presto.spi.Page; +import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.spi.block.BlockBuilder; +import com.facebook.presto.spi.block.LongArrayBlockBuilder; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import io.airlift.slice.Slice; +import io.airlift.slice.Slices; +import org.apache.hadoop.fs.FileSystem; + +import java.util.BitSet; +import java.util.Collection; +import java.util.List; +import java.util.Optional; +import java.util.OptionalInt; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; + +import static com.facebook.airlift.concurrent.MoreFutures.getFutureValue; +import static com.facebook.airlift.json.JsonCodec.jsonCodec; +import static com.facebook.presto.raptor.RaptorErrorCode.RAPTOR_ERROR; +import static com.facebook.presto.spi.type.BigintType.BIGINT; +import static java.util.Objects.requireNonNull; +import static java.util.concurrent.CompletableFuture.completedFuture; +import static java.util.concurrent.CompletableFuture.supplyAsync; + +public class DeltaShardRewriter + implements ShardRewriter +{ + private static final JsonCodec SHARD_DELETE_DELTA_CODEC = jsonCodec(ShardDeleteDelta.class); + + private final UUID oldShardUuid; + private final int oldShardRowCount; + private final Optional oldDeltaDeleteShardUuid; + private final ExecutorService deletionExecutor; + private final long transactionId; + private final OptionalInt bucketNumber; + private final OrcStorageManager orcStorageManager; + private final FileSystemContext fileSystemContext; + private final FileSystem fileSystem; + + public DeltaShardRewriter( + UUID oldShardUuid, + int oldShardRowCount, + Optional oldDeltaDeleteShardUuid, + ExecutorService deletionExecutor, + long transactionId, + OptionalInt bucketNumber, + OrcStorageManager orcStorageManager, + FileSystemContext fileSystemContext, + FileSystem fileSystem) + { + this.oldShardUuid = requireNonNull(oldShardUuid, "oldShardUuid is null"); + this.oldShardRowCount = oldShardRowCount; + this.oldDeltaDeleteShardUuid = requireNonNull(oldDeltaDeleteShardUuid, "Optional oldDeltaDeleteShardUuid is null"); + this.deletionExecutor = requireNonNull(deletionExecutor, "deletionExecutor is null"); + this.transactionId = transactionId; + this.bucketNumber = requireNonNull(bucketNumber, "bucketNumber is null"); + this.orcStorageManager = requireNonNull(orcStorageManager, "orcStorageManager is null"); + this.fileSystemContext = requireNonNull(fileSystemContext, "fileSystemContext is null"); + this.fileSystem = requireNonNull(fileSystem, "fileSystem is null"); + } + + @Override + public CompletableFuture> rewrite(BitSet rowsToDelete) + { + if (rowsToDelete.isEmpty()) { + return completedFuture(ImmutableList.of()); + } + return supplyAsync(() -> writeDeltaDeleteFile(rowsToDelete), deletionExecutor); + } + + @VisibleForTesting + Collection writeDeltaDeleteFile(BitSet rowsToDelete) + { + if (rowsToDelete.isEmpty()) { + return ImmutableList.of(); + } + + // TODO: Under current implementation, one block can only hold INT_MAX many rows + // which theoretically may not be enough to hold all rows from an ORC file. + // At this point, rowsToDelete couldn't be empty + oldDeltaDeleteShardUuid.ifPresent(oldDeltaDeleteShardUuid -> mergeToRowsToDelete(rowsToDelete, oldDeltaDeleteShardUuid)); + + if (rowsToDelete.cardinality() == oldShardRowCount) { + // Delete original file + return shardDeleteDelta(oldShardUuid, oldDeltaDeleteShardUuid, Optional.empty()); + } + + BlockBuilder blockBuilder = new LongArrayBlockBuilder(null, rowsToDelete.size()); + for (int i = rowsToDelete.nextSetBit(0); i >= 0; i = rowsToDelete.nextSetBit(i + 1)) { + blockBuilder.writeLong(i); + } + // blockToDelete is LongArrayBlock + StoragePageSink pageSink = orcStorageManager.createStoragePageSink(fileSystemContext, transactionId, bucketNumber, ImmutableList.of(0L), ImmutableList.of(BIGINT), true); + pageSink.appendPages(ImmutableList.of(new Page(blockBuilder.build()))); + List shardInfos = getFutureValue(pageSink.commit()); + // Guaranteed that shardInfos only has one element since we only call commit one time + ShardInfo newDeltaDeleteShard = Iterables.getOnlyElement(shardInfos); + return shardDeleteDelta(oldShardUuid, oldDeltaDeleteShardUuid, Optional.of(newDeltaDeleteShard)); + } + + // Note: This function will change rowsToDelete. + // Will merge the BitSet from oldDeltaDeleteShardUuid to rowsToDelete + // rowsToDelete and rowsDeleted must be mutually exclusive + private void mergeToRowsToDelete(BitSet rowsToDelete, UUID oldDeltaDeleteShardUuid) + { + Optional rowsDeleted = orcStorageManager.getRowsFromUuid(fileSystem, Optional.of(oldDeltaDeleteShardUuid)); + if (!rowsDeleted.isPresent()) { + return; + } + + BitSet verify = new BitSet(); + verify.or(rowsToDelete); + verify.and(rowsDeleted.get()); + if (verify.cardinality() != 0) { + throw new PrestoException(RAPTOR_ERROR, "rowsToDelete and rowsDeleted are not mutually exclusive"); + } + rowsToDelete.or(rowsDeleted.get()); + } + + private static Collection shardDeleteDelta(UUID oldShardUuid, Optional oldDeltaDeleteShard, Optional newDeltaDeleteShard) + { + return ImmutableList.of(Slices.wrappedBuffer(SHARD_DELETE_DELTA_CODEC.toJsonBytes( + new ShardDeleteDelta(oldShardUuid, new DeltaInfoPair(oldDeltaDeleteShard, newDeltaDeleteShard))))); + } +} diff --git a/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/InplaceShardRewriter.java b/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/InplaceShardRewriter.java new file mode 100644 index 0000000000000..183aa387096b5 --- /dev/null +++ b/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/InplaceShardRewriter.java @@ -0,0 +1,138 @@ +/* + * 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 com.facebook.presto.raptor.storage; + +import com.facebook.airlift.json.JsonCodec; +import com.facebook.presto.raptor.backup.BackupManager; +import com.facebook.presto.raptor.metadata.ShardDelta; +import com.facebook.presto.raptor.metadata.ShardInfo; +import com.facebook.presto.raptor.metadata.ShardRecorder; +import com.facebook.presto.spi.type.Type; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import io.airlift.slice.Slice; +import io.airlift.slice.Slices; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +import java.util.BitSet; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.OptionalInt; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; + +import static com.facebook.airlift.concurrent.MoreFutures.getFutureValue; +import static com.facebook.airlift.json.JsonCodec.jsonCodec; +import static java.util.Objects.requireNonNull; +import static java.util.concurrent.CompletableFuture.completedFuture; +import static java.util.concurrent.CompletableFuture.supplyAsync; + +public class InplaceShardRewriter + implements ShardRewriter +{ + private static final JsonCodec SHARD_DELTA_CODEC = jsonCodec(ShardDelta.class); + + private final UUID shardUuid; + private final Map columns; + private final ExecutorService deletionExecutor; + private final long transactionId; + private final OptionalInt bucketNumber; + private final String nodeId; + private final OrcStorageManager orcStorageManager; + private final FileSystem fileSystem; + private final StorageService storageService; + private final ShardRecorder shardRecorder; + private final BackupManager backupManager; + + public InplaceShardRewriter( + UUID shardUuid, + Map columns, + ExecutorService deletionExecutor, + long transactionId, + OptionalInt bucketNumber, + String nodeId, + OrcStorageManager orcStorageManager, + FileSystem fileSystem, + StorageService storageService, + ShardRecorder shardRecorder, + BackupManager backupManager) + { + this.shardUuid = requireNonNull(shardUuid, "shardUuid is null"); + this.columns = requireNonNull(columns, "columns is null"); + this.deletionExecutor = requireNonNull(deletionExecutor, "deletionExecutor is null"); + this.transactionId = transactionId; + this.bucketNumber = requireNonNull(bucketNumber, "bucketNumber is null"); + this.nodeId = requireNonNull(nodeId, "nodeId is null"); + this.orcStorageManager = requireNonNull(orcStorageManager, "orcStorageManager is null"); + this.fileSystem = requireNonNull(fileSystem, "fileSystem is null"); + this.storageService = requireNonNull(storageService, "storageService is null"); + this.shardRecorder = requireNonNull(shardRecorder, "shardRecorder is null"); + this.backupManager = requireNonNull(backupManager, "backupManager is null"); + } + + @Override + public CompletableFuture> rewrite(BitSet rowsToDelete) + { + if (rowsToDelete.isEmpty()) { + return completedFuture(ImmutableList.of()); + } + return supplyAsync(() -> rewriteShard(rowsToDelete), deletionExecutor); + } + + @VisibleForTesting + Collection rewriteShard(BitSet rowsToDelete) + { + if (rowsToDelete.isEmpty()) { + return ImmutableList.of(); + } + + UUID newShardUuid = UUID.randomUUID(); + Path input = storageService.getStorageFile(shardUuid); + Path output = storageService.getStagingFile(newShardUuid); + + OrcFileInfo info = orcStorageManager.rewriteFile(fileSystem, columns, input, output, rowsToDelete); + long rowCount = info.getRowCount(); + + if (rowCount == 0) { + return shardDelta(shardUuid, Optional.empty()); + } + + shardRecorder.recordCreatedShard(transactionId, newShardUuid); + + // submit for backup and wait until it finishes + getFutureValue(backupManager.submit(newShardUuid, output)); + + Set nodes = ImmutableSet.of(nodeId); + long uncompressedSize = info.getUncompressedSize(); + + ShardInfo shard = orcStorageManager.createShardInfo(fileSystem, newShardUuid, bucketNumber, output, nodes, rowCount, uncompressedSize); + + orcStorageManager.writeShard(newShardUuid); + + return shardDelta(shardUuid, Optional.of(shard)); + } + + private static Collection shardDelta(UUID oldShardUuid, Optional shardInfo) + { + List newShards = shardInfo.map(ImmutableList::of).orElse(ImmutableList.of()); + ShardDelta delta = new ShardDelta(ImmutableList.of(oldShardUuid), newShards); + return ImmutableList.of(Slices.wrappedBuffer(SHARD_DELTA_CODEC.toJsonBytes(delta))); + } +} diff --git a/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/OrcPageSource.java b/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/OrcPageSource.java index 783674fd00485..5f91157feddbb 100644 --- a/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/OrcPageSource.java +++ b/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/OrcPageSource.java @@ -16,9 +16,10 @@ import com.facebook.presto.memory.context.AggregatedMemoryContext; import com.facebook.presto.orc.OrcBatchRecordReader; import com.facebook.presto.orc.OrcDataSource; +import com.facebook.presto.raptor.storage.DeltaShardLoader.RowsToKeepResult; +import com.facebook.presto.spi.ConnectorPageSource; import com.facebook.presto.spi.Page; import com.facebook.presto.spi.PrestoException; -import com.facebook.presto.spi.UpdatablePageSource; import com.facebook.presto.spi.block.Block; import com.facebook.presto.spi.block.BlockBuilder; import com.facebook.presto.spi.block.LazyBlock; @@ -26,16 +27,11 @@ import com.facebook.presto.spi.block.RunLengthEncodedBlock; import com.facebook.presto.spi.type.Type; import com.google.common.collect.ImmutableList; -import io.airlift.slice.Slice; import java.io.IOException; -import java.util.BitSet; -import java.util.Collection; import java.util.List; -import java.util.Optional; import java.util.OptionalInt; import java.util.UUID; -import java.util.concurrent.CompletableFuture; import static com.facebook.presto.orc.OrcReader.MAX_BATCH_SIZE; import static com.facebook.presto.raptor.RaptorErrorCode.RAPTOR_ERROR; @@ -45,23 +41,21 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; import static io.airlift.slice.Slices.utf8Slice; -import static java.lang.Math.toIntExact; import static java.util.Objects.requireNonNull; public class OrcPageSource - implements UpdatablePageSource + implements ConnectorPageSource { public static final int NULL_COLUMN = -1; public static final int ROWID_COLUMN = -2; public static final int SHARD_UUID_COLUMN = -3; public static final int BUCKET_NUMBER_COLUMN = -4; - private final Optional shardRewriter; - private final OrcBatchRecordReader recordReader; private final OrcDataSource orcDataSource; - private final BitSet rowsToDelete; + // for shard with existing delta + private final DeltaShardLoader deltaShardLoader; private final List columnIds; private final List types; @@ -76,7 +70,6 @@ public class OrcPageSource private boolean closed; public OrcPageSource( - Optional shardRewriter, OrcBatchRecordReader recordReader, OrcDataSource orcDataSource, List columnIds, @@ -84,13 +77,12 @@ public OrcPageSource( List columnIndexes, UUID shardUuid, OptionalInt bucketNumber, - AggregatedMemoryContext systemMemoryContext) + AggregatedMemoryContext systemMemoryContext, + DeltaShardLoader deltaShardLoader) { - this.shardRewriter = requireNonNull(shardRewriter, "shardRewriter is null"); this.recordReader = requireNonNull(recordReader, "recordReader is null"); this.orcDataSource = requireNonNull(orcDataSource, "orcDataSource is null"); - - this.rowsToDelete = new BitSet(toIntExact(recordReader.getFileRowCount())); + this.deltaShardLoader = requireNonNull(deltaShardLoader, "Optional is null"); checkArgument(columnIds.size() == columnTypes.size(), "ids and types mismatch"); checkArgument(columnIds.size() == columnIndexes.size(), "ids and indexes mismatch"); @@ -164,20 +156,23 @@ public Page getNextPage() long filePosition = recordReader.getFilePosition(); + // for every page, will generate its rowsToKeep + RowsToKeepResult rowsToKeep = deltaShardLoader.getRowsToKeep(batchSize, filePosition); + Block[] blocks = new Block[columnIndexes.length]; for (int fieldId = 0; fieldId < blocks.length; fieldId++) { if (constantBlocks[fieldId] != null) { - blocks[fieldId] = constantBlocks[fieldId].getRegion(0, batchSize); + blocks[fieldId] = constantBlocks[fieldId].getRegion(0, rowsToKeep.keepAll() ? batchSize : rowsToKeep.size()); } else if (columnIndexes[fieldId] == ROWID_COLUMN) { - blocks[fieldId] = buildSequenceBlock(filePosition, batchSize); + blocks[fieldId] = buildSequenceBlock(filePosition, batchSize, rowsToKeep); } else { - blocks[fieldId] = new LazyBlock(batchSize, new OrcBlockLoader(columnIndexes[fieldId])); + blocks[fieldId] = new LazyBlock(batchSize, new OrcBlockLoader(columnIndexes[fieldId], rowsToKeep)); } } - return new Page(batchSize, blocks); + return new Page(rowsToKeep.keepAll() ? batchSize : rowsToKeep.size(), blocks); } catch (IOException | RuntimeException e) { closeWithSuppression(e); @@ -207,22 +202,6 @@ public String toString() .toString(); } - @Override - public void deleteRows(Block rowIds) - { - for (int i = 0; i < rowIds.getPositionCount(); i++) { - long rowId = BIGINT.getLong(rowIds, i); - rowsToDelete.set(toIntExact(rowId)); - } - } - - @Override - public CompletableFuture> finish() - { - checkState(shardRewriter.isPresent(), "shardRewriter is missing"); - return shardRewriter.get().rewrite(rowsToDelete); - } - @Override public long getSystemMemoryUsage() { @@ -243,11 +222,13 @@ private void closeWithSuppression(Throwable throwable) } } - private static Block buildSequenceBlock(long start, int count) + private static Block buildSequenceBlock(long start, int count, RowsToKeepResult rowsToKeep) { BlockBuilder builder = BIGINT.createFixedSizeBlockBuilder(count); for (int i = 0; i < count; i++) { - BIGINT.writeLong(builder, start + i); + if (rowsToKeep.keepAll() || rowsToKeep.getRowsToKeep().contains(i)) { + BIGINT.writeLong(builder, start + i); + } } return builder.build(); } @@ -263,11 +244,13 @@ private final class OrcBlockLoader { private final int expectedBatchId = batchId; private final int columnIndex; + private final RowsToKeepResult rowsToKeep; private boolean loaded; - public OrcBlockLoader(int columnIndex) + public OrcBlockLoader(int columnIndex, RowsToKeepResult rowsToKeep) { this.columnIndex = columnIndex; + this.rowsToKeep = rowsToKeep; } @Override @@ -281,7 +264,12 @@ public final void load(LazyBlock lazyBlock) try { Block block = recordReader.readBlock(columnIndex); - lazyBlock.setBlock(block); + if (rowsToKeep.keepAll()) { + lazyBlock.setBlock(block); + } + else { + lazyBlock.setBlock(block.getPositions(rowsToKeep.elements(), 0, rowsToKeep.size())); + } } catch (IOException e) { throw new PrestoException(RAPTOR_ERROR, e); diff --git a/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/OrcStorageManager.java b/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/OrcStorageManager.java index 74d9eb5c8a330..e0775b45114bd 100644 --- a/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/OrcStorageManager.java +++ b/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/OrcStorageManager.java @@ -22,6 +22,7 @@ import com.facebook.presto.orc.OrcReader; import com.facebook.presto.orc.OrcReaderOptions; import com.facebook.presto.orc.OrcWriterStats; +import com.facebook.presto.orc.StorageStripeMetadataSource; import com.facebook.presto.orc.StripeMetadataSource; import com.facebook.presto.orc.TupleDomainOrcPredicate; import com.facebook.presto.orc.TupleDomainOrcPredicate.ColumnReference; @@ -35,7 +36,6 @@ import com.facebook.presto.raptor.filesystem.FileSystemContext; import com.facebook.presto.raptor.metadata.ColumnInfo; import com.facebook.presto.raptor.metadata.ColumnStats; -import com.facebook.presto.raptor.metadata.ShardDelta; import com.facebook.presto.raptor.metadata.ShardInfo; import com.facebook.presto.raptor.metadata.ShardRecorder; import com.facebook.presto.raptor.storage.StorageManagerConfig.OrcOptimizedWriterStage; @@ -43,6 +43,7 @@ import com.facebook.presto.spi.NodeManager; import com.facebook.presto.spi.Page; import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.spi.block.Block; import com.facebook.presto.spi.predicate.TupleDomain; import com.facebook.presto.spi.type.ArrayType; import com.facebook.presto.spi.type.DecimalType; @@ -60,8 +61,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; -import io.airlift.slice.Slice; -import io.airlift.slice.Slices; import io.airlift.units.DataSize; import io.airlift.units.Duration; import org.apache.hadoop.fs.FileSystem; @@ -75,7 +74,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.BitSet; -import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Optional; @@ -91,7 +89,6 @@ import java.util.concurrent.TimeoutException; import static com.facebook.airlift.concurrent.MoreFutures.allAsList; -import static com.facebook.airlift.concurrent.MoreFutures.getFutureValue; import static com.facebook.airlift.concurrent.Threads.daemonThreadsNamed; import static com.facebook.airlift.json.JsonCodec.jsonCodec; import static com.facebook.presto.memory.context.AggregatedMemoryContext.newSimpleAggregatedMemoryContext; @@ -125,10 +122,9 @@ import static com.google.common.collect.ImmutableList.toImmutableList; import static io.airlift.units.DataSize.Unit.PETABYTE; import static java.lang.Math.min; +import static java.lang.Math.toIntExact; import static java.lang.String.format; import static java.util.Objects.requireNonNull; -import static java.util.concurrent.CompletableFuture.completedFuture; -import static java.util.concurrent.CompletableFuture.supplyAsync; import static java.util.concurrent.Executors.newCachedThreadPool; import static java.util.concurrent.Executors.newFixedThreadPool; import static java.util.stream.Collectors.toList; @@ -145,7 +141,6 @@ public class OrcStorageManager public static final DateTimeZone DEFAULT_STORAGE_TIMEZONE = UTC; // TODO: do not limit the max size of blocks to read for now; enable the limit when the Hive connector is ready public static final DataSize HUGE_MAX_READ_BLOCK_SIZE = new DataSize(1, PETABYTE); - private static final JsonCodec SHARD_DELTA_CODEC = jsonCodec(ShardDelta.class); private static final long MAX_ROWS = 1_000_000_000; private static final JsonCodec METADATA_CODEC = jsonCodec(OrcFileMetadata.class); @@ -274,6 +269,8 @@ public void shutdown() public ConnectorPageSource getPageSource( FileSystemContext fileSystemContext, UUID shardUuid, + Optional deltaShardUuid, + boolean tableSupportsDeltaDelete, OptionalInt bucketNumber, List columnIds, List columnTypes, @@ -324,10 +321,33 @@ public ConnectorPageSource getPageSource( Optional shardRewriter = Optional.empty(); if (transactionId.isPresent()) { checkState(allColumnTypes.isPresent()); - shardRewriter = Optional.of(createShardRewriter(fileSystem, transactionId.getAsLong(), bucketNumber, shardUuid, allColumnTypes.get())); + if (reader.getFooter().getNumberOfRows() >= Integer.MAX_VALUE) { + throw new PrestoException(RAPTOR_ERROR, "File has too many rows, failed to read file: " + shardUuid); + } + shardRewriter = Optional.of(createShardRewriter( + fileSystemContext, + fileSystem, + transactionId.getAsLong(), + bucketNumber, + shardUuid, + toIntExact(reader.getFooter().getNumberOfRows()), + deltaShardUuid, + tableSupportsDeltaDelete, + allColumnTypes.get())); } - - return new OrcPageSource(shardRewriter, recordReader, dataSource, columnIds, columnTypes, columnIndexes.build(), shardUuid, bucketNumber, systemMemoryUsage); + return new OrcUpdatablePageSource( + shardRewriter, + recordReader, + new OrcPageSource( + recordReader, + dataSource, + columnIds, + columnTypes, + columnIndexes.build(), + shardUuid, + bucketNumber, + systemMemoryUsage, + new DeltaShardLoader(deltaShardUuid, tableSupportsDeltaDelete, this, fileSystem))); } catch (IOException | RuntimeException e) { closeQuietly(dataSource); @@ -339,6 +359,50 @@ public ConnectorPageSource getPageSource( } } + Optional getRowsFromUuid(FileSystem fileSystem, Optional deltaShardUuid) + { + if (!deltaShardUuid.isPresent()) { + return Optional.empty(); + } + + try (OrcDataSource dataSource = openShard(fileSystem, deltaShardUuid.get(), defaultReaderAttributes)) { + AggregatedMemoryContext systemMemoryUsage = newSimpleAggregatedMemoryContext(); + OrcReader reader = new OrcReader( + dataSource, + ORC, + orcFileTailSource, + new StorageStripeMetadataSource(), + new OrcReaderOptions( + defaultReaderAttributes.getMaxMergeDistance(), + defaultReaderAttributes.getTinyStripeThreshold(), + HUGE_MAX_READ_BLOCK_SIZE, + defaultReaderAttributes.isZstdJniDecompressionEnabled())); + + if (reader.getFooter().getNumberOfRows() >= Integer.MAX_VALUE) { + throw new IOException("File has too many rows"); + } + + try (OrcBatchRecordReader recordReader = reader.createBatchRecordReader( + ImmutableMap.of(0, BIGINT), + OrcPredicate.TRUE, + DEFAULT_STORAGE_TIMEZONE, + systemMemoryUsage, + INITIAL_BATCH_SIZE)) { + BitSet bitSet = new BitSet(); + while (recordReader.nextBatch() > 0) { + Block block = recordReader.readBlock(0); + for (int i = 0; i < block.getPositionCount(); i++) { + bitSet.set(toIntExact(block.getLong(i))); + } + } + return Optional.of(bitSet); + } + } + catch (IOException | RuntimeException e) { + throw new PrestoException(RAPTOR_ERROR, "Failed to read file: " + deltaShardUuid, e); + } + } + private static int toSpecialIndex(long columnId) { if (isShardRowIdColumn(columnId)) { @@ -368,17 +432,44 @@ public StoragePageSink createStoragePageSink( return new OrcStoragePageSink(orcDataEnvironment.getFileSystem(fileSystemContext), transactionId, columnIds, columnTypes, bucketNumber); } - private ShardRewriter createShardRewriter(FileSystem fileSystem, long transactionId, OptionalInt bucketNumber, UUID shardUuid, Map columns) + ShardRewriter createShardRewriter( + FileSystemContext fileSystemContext, + FileSystem fileSystem, + long transactionId, + OptionalInt bucketNumber, + UUID shardUuid, + int shardRowCount, + Optional deltaShardUuid, + boolean tableSupportsDeltaDelete, + Map columns) { - return rowsToDelete -> { - if (rowsToDelete.isEmpty()) { - return completedFuture(ImmutableList.of()); - } - return supplyAsync(() -> rewriteShard(fileSystem, transactionId, bucketNumber, shardUuid, columns, rowsToDelete), deletionExecutor); - }; + if (tableSupportsDeltaDelete) { + return new DeltaShardRewriter( + shardUuid, + shardRowCount, + deltaShardUuid, + deletionExecutor, + transactionId, + bucketNumber, + this, + fileSystemContext, + fileSystem); + } + return new InplaceShardRewriter( + shardUuid, + columns, + deletionExecutor, + transactionId, + bucketNumber, + nodeId, + this, + fileSystem, + storageService, + shardRecorder, + backupManager); } - private void writeShard(UUID shardUuid) + void writeShard(UUID shardUuid) { if (backupStore.isPresent() && !backupStore.get().shardExists(shardUuid)) { throw new PrestoException(RAPTOR_ERROR, "Backup does not exist after write"); @@ -428,7 +519,7 @@ OrcDataSource openShard(FileSystem fileSystem, UUID shardUuid, ReaderAttributes } } - private ShardInfo createShardInfo(FileSystem fileSystem, UUID shardUuid, OptionalInt bucketNumber, Path file, Set nodes, long rowCount, long uncompressedSize) + ShardInfo createShardInfo(FileSystem fileSystem, UUID shardUuid, OptionalInt bucketNumber, Path file, Set nodes, long rowCount, long uncompressedSize) { try { return new ShardInfo(shardUuid, bucketNumber, nodes, computeShardStats(fileSystem, file), rowCount, fileSystem.getFileStatus(file).getLen(), uncompressedSize, xxhash64(fileSystem, file)); @@ -459,47 +550,7 @@ private List computeShardStats(FileSystem fileSystem, Path file) } } - @VisibleForTesting - Collection rewriteShard(FileSystem fileSystem, long transactionId, OptionalInt bucketNumber, UUID shardUuid, Map columns, BitSet rowsToDelete) - { - if (rowsToDelete.isEmpty()) { - return ImmutableList.of(); - } - - UUID newShardUuid = UUID.randomUUID(); - Path input = storageService.getStorageFile(shardUuid); - Path output = storageService.getStagingFile(newShardUuid); - - OrcFileInfo info = rewriteFile(fileSystem, columns, input, output, rowsToDelete); - long rowCount = info.getRowCount(); - - if (rowCount == 0) { - return shardDelta(shardUuid, Optional.empty()); - } - - shardRecorder.recordCreatedShard(transactionId, newShardUuid); - - // submit for backup and wait until it finishes - getFutureValue(backupManager.submit(newShardUuid, output)); - - Set nodes = ImmutableSet.of(nodeId); - long uncompressedSize = info.getUncompressedSize(); - - ShardInfo shard = createShardInfo(fileSystem, newShardUuid, bucketNumber, output, nodes, rowCount, uncompressedSize); - - writeShard(newShardUuid); - - return shardDelta(shardUuid, Optional.of(shard)); - } - - private static Collection shardDelta(UUID oldShardUuid, Optional shardInfo) - { - List newShards = shardInfo.map(ImmutableList::of).orElse(ImmutableList.of()); - ShardDelta delta = new ShardDelta(ImmutableList.of(oldShardUuid), newShards); - return ImmutableList.of(Slices.wrappedBuffer(SHARD_DELTA_CODEC.toJsonBytes(delta))); - } - - private OrcFileInfo rewriteFile(FileSystem fileSystem, Map columns, Path input, Path output, BitSet rowsToDelete) + OrcFileInfo rewriteFile(FileSystem fileSystem, Map columns, Path input, Path output, BitSet rowsToDelete) { try { return fileRewriter.rewrite(fileSystem, columns, input, output, rowsToDelete); diff --git a/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/OrcUpdatablePageSource.java b/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/OrcUpdatablePageSource.java new file mode 100644 index 0000000000000..8f1e601c42d55 --- /dev/null +++ b/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/OrcUpdatablePageSource.java @@ -0,0 +1,113 @@ +/* + * 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 com.facebook.presto.raptor.storage; + +import com.facebook.presto.orc.OrcBatchRecordReader; +import com.facebook.presto.spi.Page; +import com.facebook.presto.spi.UpdatablePageSource; +import com.facebook.presto.spi.block.Block; +import io.airlift.slice.Slice; + +import java.util.BitSet; +import java.util.Collection; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; + +import static com.facebook.presto.spi.type.BigintType.BIGINT; +import static com.google.common.base.Preconditions.checkState; +import static java.lang.Math.toIntExact; +import static java.util.Objects.requireNonNull; + +public class OrcUpdatablePageSource + implements UpdatablePageSource +{ + private final Optional shardRewriter; + private final OrcPageSource parentPageSource; + private final BitSet rowsToDelete; + + public OrcUpdatablePageSource( + Optional shardRewriter, + OrcBatchRecordReader recordReader, + OrcPageSource parentPageSource) + { + this.shardRewriter = requireNonNull(shardRewriter, "shardRewriter is null"); + requireNonNull(recordReader, "recordReader is null"); + this.parentPageSource = requireNonNull(parentPageSource, "parentPageSource is null"); + this.rowsToDelete = new BitSet(toIntExact(recordReader.getFileRowCount())); + } + + @Override + public long getCompletedBytes() + { + return parentPageSource.getCompletedBytes(); + } + + @Override + public long getCompletedPositions() + { + return parentPageSource.getCompletedPositions(); + } + + @Override + public long getReadTimeNanos() + { + return parentPageSource.getReadTimeNanos(); + } + + @Override + public boolean isFinished() + { + return parentPageSource.isFinished(); + } + + @Override + public Page getNextPage() + { + return parentPageSource.getNextPage(); + } + + @Override + public void close() + { + parentPageSource.close(); + } + + @Override + public String toString() + { + return parentPageSource.toString(); + } + + @Override + public void deleteRows(Block rowIds) + { + for (int i = 0; i < rowIds.getPositionCount(); i++) { + long rowId = BIGINT.getLong(rowIds, i); + rowsToDelete.set(toIntExact(rowId)); + } + } + + @Override + public CompletableFuture> finish() + { + checkState(shardRewriter.isPresent(), "shardRewriter is missing"); + return shardRewriter.get().rewrite(rowsToDelete); + } + + @Override + public long getSystemMemoryUsage() + { + return parentPageSource.getSystemMemoryUsage(); + } +} diff --git a/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/StorageManager.java b/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/StorageManager.java index f2147cd285a38..990b0660adfb5 100644 --- a/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/StorageManager.java +++ b/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/StorageManager.java @@ -31,18 +31,22 @@ public interface StorageManager default ConnectorPageSource getPageSource( FileSystemContext fileSystemContext, UUID shardUuid, + Optional deltaShardUuid, + boolean tableSupportsDeltaDelete, OptionalInt bucketNumber, List columnIds, List columnTypes, TupleDomain effectivePredicate, ReaderAttributes readerAttributes) { - return getPageSource(fileSystemContext, shardUuid, bucketNumber, columnIds, columnTypes, effectivePredicate, readerAttributes, OptionalLong.empty(), Optional.empty()); + return getPageSource(fileSystemContext, shardUuid, deltaShardUuid, tableSupportsDeltaDelete, bucketNumber, columnIds, columnTypes, effectivePredicate, readerAttributes, OptionalLong.empty(), Optional.empty()); } ConnectorPageSource getPageSource( FileSystemContext fileSystemContext, UUID shardUuid, + Optional deltaShardUuid, + boolean tableSupportsDeltaDelete, OptionalInt bucketNumber, List columnIds, List columnTypes, diff --git a/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/StorageModule.java b/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/StorageModule.java index f87248cab72a1..8e2a44b43dfc3 100644 --- a/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/StorageModule.java +++ b/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/StorageModule.java @@ -100,6 +100,7 @@ public void configure(Binder binder) binder.bind(AssignmentLimiter.class).in(Scopes.SINGLETON); binder.bind(TemporalFunction.class).in(Scopes.SINGLETON); + newExporter(binder).export(DatabaseShardManager.class).as(generatedNameOf(DatabaseShardManager.class, connectorId)); newExporter(binder).export(ShardRecoveryManager.class).as(generatedNameOf(ShardRecoveryManager.class, connectorId)); newExporter(binder).export(BackupManager.class).as(generatedNameOf(BackupManager.class, connectorId)); newExporter(binder).export(StorageManager.class).as(generatedNameOf(OrcStorageManager.class, connectorId)); diff --git a/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/organization/OrganizationJob.java b/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/organization/OrganizationJob.java index 3dfbc7449ec58..c54916cfc205b 100644 --- a/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/organization/OrganizationJob.java +++ b/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/organization/OrganizationJob.java @@ -25,7 +25,6 @@ import java.io.UncheckedIOException; import java.util.List; import java.util.OptionalInt; -import java.util.OptionalLong; import java.util.Set; import java.util.UUID; @@ -68,7 +67,9 @@ private void runJob(long tableId, OptionalInt bucketNumber, Set shardUuids { long transactionId = shardManager.beginTransaction(); try { - runJob(transactionId, bucketNumber, tableId, shardUuids); + // todo add logic in organization for delta or it may corrupt data + return; + //runJob(transactionId, bucketNumber, tableId, shardUuids); } catch (Throwable e) { shardManager.rollbackTransaction(transactionId); @@ -79,10 +80,10 @@ private void runJob(long tableId, OptionalInt bucketNumber, Set shardUuids private void runJob(long transactionId, OptionalInt bucketNumber, long tableId, Set shardUuids) throws IOException { + // todo add logic in organization for delta or it may corrupt data TableMetadata metadata = getTableMetadata(tableId); List newShards = performCompaction(transactionId, bucketNumber, shardUuids, metadata); log.info("Compacted shards %s into %s", shardUuids, newShards.stream().map(ShardInfo::getShardUuid).collect(toList())); - shardManager.replaceShardUuids(transactionId, tableId, metadata.getColumns(), shardUuids, newShards, OptionalLong.empty()); } private TableMetadata getTableMetadata(long tableId) diff --git a/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/organization/ShardCompactor.java b/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/organization/ShardCompactor.java index 0ceebd0c76668..38b5414f436a8 100644 --- a/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/organization/ShardCompactor.java +++ b/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/organization/ShardCompactor.java @@ -38,6 +38,7 @@ import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; +import java.util.Optional; import java.util.OptionalInt; import java.util.PriorityQueue; import java.util.Queue; @@ -95,7 +96,8 @@ private List compact(StoragePageSink storagePageSink, OptionalInt buc throws IOException { for (UUID uuid : uuids) { - try (ConnectorPageSource pageSource = storageManager.getPageSource(FileSystemContext.DEFAULT_RAPTOR_CONTEXT, uuid, bucketNumber, columnIds, columnTypes, TupleDomain.all(), readerAttributes)) { + // todo add logic in organization for delta or it may corrupt data + try (ConnectorPageSource pageSource = storageManager.getPageSource(FileSystemContext.DEFAULT_RAPTOR_CONTEXT, uuid, Optional.empty(), false, bucketNumber, columnIds, columnTypes, TupleDomain.all(), readerAttributes)) { while (!pageSource.isFinished()) { Page page = pageSource.getNextPage(); if (isNullOrEmptyPage(page)) { @@ -130,8 +132,9 @@ public List compactSorted(long transactionId, OptionalInt bucketNumbe Queue rowSources = new PriorityQueue<>(); StoragePageSink outputPageSink = storageManager.createStoragePageSink(FileSystemContext.DEFAULT_RAPTOR_CONTEXT, transactionId, bucketNumber, columnIds, columnTypes, false); try { + // todo add logic in organization for delta or it may corrupt data for (UUID uuid : uuids) { - ConnectorPageSource pageSource = storageManager.getPageSource(FileSystemContext.DEFAULT_RAPTOR_CONTEXT, uuid, bucketNumber, columnIds, columnTypes, TupleDomain.all(), readerAttributes); + ConnectorPageSource pageSource = storageManager.getPageSource(FileSystemContext.DEFAULT_RAPTOR_CONTEXT, uuid, Optional.empty(), false, bucketNumber, columnIds, columnTypes, TupleDomain.all(), readerAttributes); SortedPageSource rowSource = new SortedPageSource(pageSource, columnTypes, sortIndexes, sortOrders); rowSources.add(rowSource); } diff --git a/presto-raptor/src/main/java/com/facebook/presto/raptor/systemtables/TableStatsSystemTable.java b/presto-raptor/src/main/java/com/facebook/presto/raptor/systemtables/TableStatsSystemTable.java index 8df1bb55311e9..dd4da4c8a3336 100644 --- a/presto-raptor/src/main/java/com/facebook/presto/raptor/systemtables/TableStatsSystemTable.java +++ b/presto-raptor/src/main/java/com/facebook/presto/raptor/systemtables/TableStatsSystemTable.java @@ -63,6 +63,7 @@ public class TableStatsSystemTable .add(new ColumnMetadata("update_time", TIMESTAMP)) .add(new ColumnMetadata("table_version", BIGINT)) .add(new ColumnMetadata("shard_count", BIGINT)) + .add(new ColumnMetadata("delta_count", BIGINT)) .add(new ColumnMetadata("row_count", BIGINT)) .add(new ColumnMetadata("compressed_size", BIGINT)) .add(new ColumnMetadata("uncompressed_size", BIGINT)) @@ -112,6 +113,7 @@ private static List buildPages(MetadataDao dao, TupleDomain tuple TIMESTAMP.writeLong(pageBuilder.nextBlockBuilder(), row.getUpdateTime()); BIGINT.writeLong(pageBuilder.nextBlockBuilder(), row.getTableVersion()); BIGINT.writeLong(pageBuilder.nextBlockBuilder(), row.getShardCount()); + BIGINT.writeLong(pageBuilder.nextBlockBuilder(), row.getDeltaCount()); BIGINT.writeLong(pageBuilder.nextBlockBuilder(), row.getRowCount()); BIGINT.writeLong(pageBuilder.nextBlockBuilder(), row.getCompressedSize()); BIGINT.writeLong(pageBuilder.nextBlockBuilder(), row.getUncompressedSize()); diff --git a/presto-raptor/src/test/java/com/facebook/presto/raptor/integration/TestRaptorIntegrationSmokeTest.java b/presto-raptor/src/test/java/com/facebook/presto/raptor/integration/TestRaptorIntegrationSmokeTest.java index 9f13968abd028..5d6e2e6191cf2 100644 --- a/presto-raptor/src/test/java/com/facebook/presto/raptor/integration/TestRaptorIntegrationSmokeTest.java +++ b/presto-raptor/src/test/java/com/facebook/presto/raptor/integration/TestRaptorIntegrationSmokeTest.java @@ -833,6 +833,76 @@ public void testTableStatsSystemTable() assertUpdate("DROP TABLE test_table_stats"); } + @SuppressWarnings("OverlyStrongTypeCast") + @Test + public void testTableStatsSystemTableWithDeltaDelete() + { + // create empty table + assertUpdate("CREATE TABLE test_table_stats (x bigint) WITH (table_supports_delta_delete = true)"); + + @Language("SQL") String sql = "" + + "SELECT create_time, update_time, table_version," + + " shard_count, row_count, uncompressed_size, delta_count\n" + + "FROM system.table_stats\n" + + "WHERE table_schema = 'tpch'\n" + + " AND table_name = 'test_table_stats'"; + MaterializedRow row = getOnlyElement(computeActual(sql).getMaterializedRows()); + + LocalDateTime createTime = (LocalDateTime) row.getField(0); + LocalDateTime updateTime1 = (LocalDateTime) row.getField(1); + assertEquals(createTime, updateTime1); + + assertEquals(row.getField(2), 1L); // table_version + assertEquals(row.getField(3), 0L); // shard_count + assertEquals(row.getField(4), 0L); // row_count + long size1 = (long) row.getField(5); // uncompressed_size + + // insert + assertUpdate("INSERT INTO test_table_stats VALUES (1), (2), (3), (4)", 4); + row = getOnlyElement(computeActual(sql).getMaterializedRows()); + + assertEquals(row.getField(0), createTime); + LocalDateTime updateTime2 = (LocalDateTime) row.getField(1); + assertLessThan(updateTime1, updateTime2); + + assertEquals(row.getField(2), 2L); // table_version + assertGreaterThanOrEqual((Long) row.getField(3), 1L); // shard_count + assertEquals(row.getField(4), 4L); // row_count + assertGreaterThanOrEqual((Long) row.getField(6), 0L); // delta_count + long size2 = (long) row.getField(5); // uncompressed_size + assertGreaterThan(size2, size1); + + // delete + assertUpdate("DELETE FROM test_table_stats WHERE x IN (2, 4)", 2); + row = getOnlyElement(computeActual(sql).getMaterializedRows()); + + assertEquals(row.getField(0), createTime); + LocalDateTime updateTime3 = (LocalDateTime) row.getField(1); + assertLessThan(updateTime2, updateTime3); + + assertEquals(row.getField(2), 3L); // table_version + assertGreaterThanOrEqual((Long) row.getField(3), 1L); // shard_count + assertEquals(row.getField(4), 2L); // row_count + assertGreaterThanOrEqual((Long) row.getField(6), 1L); // delta_count + long size3 = (long) row.getField(5); // uncompressed_Size + // without compaction, the size will grow with delta delete + assertGreaterThan(size3, size2); + + // add column + assertUpdate("ALTER TABLE test_table_stats ADD COLUMN y bigint"); + row = getOnlyElement(computeActual(sql).getMaterializedRows()); + + assertEquals(row.getField(0), createTime); + assertLessThan(updateTime3, (LocalDateTime) row.getField(1)); + + assertEquals(row.getField(2), 4L); // table_version + assertEquals(row.getField(4), 2L); // row_count + assertEquals(row.getField(5), size3); // uncompressed_size + + // cleanup + assertUpdate("DROP TABLE test_table_stats"); + } + @Test public void testAlterTable() { @@ -894,6 +964,33 @@ public void testDelete() assertUpdate("DROP TABLE test_delete_table"); } + @Test + public void testDeltaDelete() + { + assertUpdate("CREATE TABLE test_delta_delete_table (c1 bigint, c2 bigint) WITH (table_supports_delta_delete = true)"); + assertUpdate("INSERT INTO test_delta_delete_table VALUES (1, 1), (1, 2), (1, 3), (1, 4), (11, 1), (11, 2)", 6); + + assertUpdate("ALTER TABLE test_delta_delete_table ADD COLUMN c3 bigint"); + assertUpdate("INSERT INTO test_delta_delete_table VALUES (2, 1, 1), (2, 2, 2), (2, 3, 3), (2, 4, 4), (22, 1, 1), (22, 2, 2), (22, 4, 4)", 7); + + assertUpdate("DELETE FROM test_delta_delete_table WHERE c1 = 1", 4); + assertQuery("SELECT * FROM test_delta_delete_table", "VALUES (11, 1, NULL), (11, 2, NULL), (2, 1, 1), (2, 2, 2), (2, 3, 3), (2, 4, 4), (22, 1, 1), (22, 2, 2), (22, 4, 4)"); + + assertUpdate("DELETE FROM test_delta_delete_table WHERE c1 = 1", 0); + assertQuery("SELECT * FROM test_delta_delete_table", "VALUES (11, 1, NULL), (11, 2, NULL), (2, 1, 1), (2, 2, 2), (2, 3, 3), (2, 4, 4), (22, 1, 1), (22, 2, 2), (22, 4, 4)"); + + assertUpdate("ALTER TABLE test_delta_delete_table DROP COLUMN c2"); + assertUpdate("INSERT INTO test_delta_delete_table VALUES (3, 1), (3, 2), (3, 3), (3, 4)", 4); + + assertUpdate("DELETE FROM test_delta_delete_table WHERE c1 = 2", 4); + assertQuery("SELECT * FROM test_delta_delete_table", "VALUES (11, NULL), (11, NULL), (22, 1), (22, 2), (22, 4), (3, 1), (3, 2), (3, 3), (3, 4)"); + + assertUpdate("DELETE FROM test_delta_delete_table WHERE c1 % 11 = 0", 5); + assertQuery("SELECT * FROM test_delta_delete_table", "VALUES (3, 1), (3, 2), (3, 3), (3, 4)"); + + assertUpdate("DROP TABLE test_delta_delete_table"); + } + @Test public void testTriggerBucketBalancer() { diff --git a/presto-raptor/src/test/java/com/facebook/presto/raptor/metadata/TestDatabaseShardManager.java b/presto-raptor/src/test/java/com/facebook/presto/raptor/metadata/TestDatabaseShardManager.java index e826f9e1fb7ea..6ad466efa0e09 100644 --- a/presto-raptor/src/test/java/com/facebook/presto/raptor/metadata/TestDatabaseShardManager.java +++ b/presto-raptor/src/test/java/com/facebook/presto/raptor/metadata/TestDatabaseShardManager.java @@ -62,6 +62,8 @@ import java.util.OptionalLong; import java.util.Set; import java.util.UUID; +import java.util.function.Function; +import java.util.stream.Collectors; import static com.facebook.presto.raptor.RaptorErrorCode.RAPTOR_EXTERNAL_BATCH_ALREADY_EXISTS; import static com.facebook.presto.raptor.metadata.DatabaseShardManager.shardIndexTable; @@ -82,6 +84,7 @@ import static com.facebook.presto.spi.type.VarcharType.createVarcharType; import static com.google.common.base.Strings.repeat; import static com.google.common.base.Ticker.systemTicker; +import static com.google.common.collect.ImmutableMap.toImmutableMap; import static com.google.common.collect.Iterables.getOnlyElement; import static com.google.common.collect.Iterators.concat; import static com.google.common.collect.Iterators.transform; @@ -216,7 +219,7 @@ public void testAssignShard() shardManager.commitShards(transactionId, tableId, columns, shardNodes, Optional.empty(), 0); ShardNodes actual = getOnlyElement(getShardNodes(tableId, TupleDomain.all())); - assertEquals(actual, new ShardNodes(shard, ImmutableSet.of("node1"))); + assertEquals(actual, new ShardNodes(shard, Optional.empty(), ImmutableSet.of("node1"))); try { shardManager.replaceShardAssignment(tableId, shard, "node2", true); @@ -230,13 +233,13 @@ public void testAssignShard() shardManager.replaceShardAssignment(tableId, shard, "node2", false); actual = getOnlyElement(getShardNodes(tableId, TupleDomain.all())); - assertEquals(actual, new ShardNodes(shard, ImmutableSet.of("node2"))); + assertEquals(actual, new ShardNodes(shard, Optional.empty(), ImmutableSet.of("node2"))); // replacing shard assignment should be idempotent shardManager.replaceShardAssignment(tableId, shard, "node2", false); actual = getOnlyElement(getShardNodes(tableId, TupleDomain.all())); - assertEquals(actual, new ShardNodes(shard, ImmutableSet.of("node2"))); + assertEquals(actual, new ShardNodes(shard, Optional.empty(), ImmutableSet.of("node2"))); } @Test @@ -258,16 +261,16 @@ public void testGetNodeBytes() shardManager.commitShards(transactionId, tableId, columns, shardNodes, Optional.empty(), 0); assertEquals(getShardNodes(tableId, TupleDomain.all()), ImmutableSet.of( - new ShardNodes(shard1, ImmutableSet.of("node1")), - new ShardNodes(shard2, ImmutableSet.of("node1")))); + new ShardNodes(shard1, Optional.empty(), ImmutableSet.of("node1")), + new ShardNodes(shard2, Optional.empty(), ImmutableSet.of("node1")))); assertEquals(shardManager.getNodeBytes(), ImmutableMap.of("node1", 88L)); shardManager.replaceShardAssignment(tableId, shard1, "node2", false); assertEquals(getShardNodes(tableId, TupleDomain.all()), ImmutableSet.of( - new ShardNodes(shard1, ImmutableSet.of("node2")), - new ShardNodes(shard2, ImmutableSet.of("node1")))); + new ShardNodes(shard1, Optional.empty(), ImmutableSet.of("node2")), + new ShardNodes(shard2, Optional.empty(), ImmutableSet.of("node1")))); assertEquals(shardManager.getNodeBytes(), ImmutableMap.of("node1", 55L, "node2", 33L)); } @@ -318,9 +321,83 @@ public void testGetExistingShards() assertEquals(actual, expected); } + @Test + public void testReplaceShardUuidsFunction() + throws SQLException + { + // node1 shard1 shard4 + // node2 shard2 + // node3 shard3 + + // goal: replace shard1 and shard4 with newUuid5 + + // Initial data + long tableId = createTable("test"); + List columns = ImmutableList.of(new ColumnInfo(1, BIGINT)); + UUID uuid1 = UUID.randomUUID(); + UUID uuid2 = UUID.randomUUID(); + UUID uuid3 = UUID.randomUUID(); + UUID uuid4 = UUID.randomUUID(); + ShardInfo shardInfo1 = new ShardInfo(uuid1, OptionalInt.empty(), ImmutableSet.of("node1"), ImmutableList.of(), 1, 1, 1, 1); + ShardInfo shardInfo2 = new ShardInfo(uuid2, OptionalInt.empty(), ImmutableSet.of("node2"), ImmutableList.of(), 2, 2, 2, 2); + ShardInfo shardInfo3 = new ShardInfo(uuid3, OptionalInt.empty(), ImmutableSet.of("node3"), ImmutableList.of(), 3, 3, 3, 3); + ShardInfo shardInfo4 = new ShardInfo(uuid4, OptionalInt.empty(), ImmutableSet.of("node1"), ImmutableList.of(), 4, 4, 4, 4); + + shardManager.createTable(tableId, columns, false, OptionalLong.empty(), true); + long transactionId = shardManager.beginTransaction(); + shardManager.commitShards(transactionId, tableId, columns, ImmutableList.of(shardInfo1, shardInfo2, shardInfo3, shardInfo4), Optional.empty(), 0); + + // New data + UUID newUuid5 = UUID.randomUUID(); + ShardInfo newShardInfo4 = new ShardInfo(newUuid5, OptionalInt.empty(), ImmutableSet.of("node1"), ImmutableList.of(), 5, 5, 5, 5); + + // toReplace + Set shardMetadata = shardManager.getNodeShards("node1"); + Set replacedUuids = shardMetadata.stream().map(ShardMetadata::getShardUuid).collect(toSet()); + Map> replaceUuidMap = replacedUuids.stream().collect(Collectors.toMap(uuid -> uuid, uuid -> Optional.empty())); + + transactionId = shardManager.beginTransaction(); + shardManager.replaceShardUuids(transactionId, tableId, columns, replaceUuidMap, ImmutableList.of(newShardInfo4), OptionalLong.of(0), true); + + // check shards on this node1 are correct + shardMetadata = shardManager.getNodeShards("node1"); + assertEquals(shardMetadata.size(), 1); + for (ShardMetadata actual : shardMetadata) { + assertEquals(actual.getShardUuid(), newUuid5); + assertEquals(actual.getDeltaUuid(), Optional.empty()); + assertEquals(actual.getRowCount(), 5); + assertEquals(actual.getCompressedSize(), 5); + assertEquals(actual.getUncompressedSize(), 5); + } + + // check that shards are replaced in index table as well + Set shardNodes = ImmutableSet.copyOf(shardManager.getShardNodes(tableId, TupleDomain.all(), true)); + Set actualAllUuids = shardNodes.stream() + .map(BucketShards::getShards) + .flatMap(Collection::stream) + .map(ShardNodes::getShardUuid) + .collect(toSet()); + Set expectedAllUuids = ImmutableSet.of(uuid2, uuid3, newUuid5); + assertEquals(actualAllUuids, expectedAllUuids); + + // Verify statistics + Statement statement = dummyHandle.getConnection().createStatement(); + ResultSet resultSet = statement.executeQuery("SELECT * FROM tables where table_id = " + tableId); + resultSet.next(); + assertEquals(resultSet.getLong("shard_count"), 3); + assertEquals(resultSet.getLong("delta_count"), 0); + assertEquals(resultSet.getLong("row_count"), 10); + assertEquals(resultSet.getLong("compressed_size"), 10); + assertEquals(resultSet.getLong("uncompressed_size"), 10); + resultSet.close(); + statement.close(); + } + @Test public void testReplaceShardUuids() { + // node1 shard1 / node2 shard2 / node3 shard3 + // replace shard1 with two new shard long tableId = createTable("test"); List columns = ImmutableList.of(new ColumnInfo(1, BIGINT)); List nodes = ImmutableList.of("node1", "node2", "node3"); @@ -332,23 +409,27 @@ public void testReplaceShardUuids() .add(shardInfo(originalUuids.get(2), nodes.get(2))) .build(); - shardManager.createTable(tableId, columns, false, OptionalLong.empty(), false); + shardManager.createTable(tableId, columns, false, OptionalLong.empty(), true); long transactionId = shardManager.beginTransaction(); shardManager.commitShards(transactionId, tableId, columns, oldShards, Optional.empty(), 0); + // newShards List expectedUuids = ImmutableList.of(UUID.randomUUID(), UUID.randomUUID()); List newShards = ImmutableList.builder() .add(shardInfo(expectedUuids.get(0), nodes.get(0))) .add(shardInfo(expectedUuids.get(1), nodes.get(0))) .build(); + // toReplace Set shardMetadata = shardManager.getNodeShards(nodes.get(0)); Set replacedUuids = shardMetadata.stream().map(ShardMetadata::getShardUuid).collect(toSet()); + Map> replaceUuidMap = replacedUuids.stream().collect(Collectors.toMap(uuid -> uuid, uuid -> Optional.empty())); transactionId = shardManager.beginTransaction(); - shardManager.replaceShardUuids(transactionId, tableId, columns, replacedUuids, newShards, OptionalLong.of(0)); + shardManager.replaceShardUuids(transactionId, tableId, columns, replaceUuidMap, newShards, OptionalLong.of(0), true); + // check that shards are replaced in shards table for node1 shardMetadata = shardManager.getNodeShards(nodes.get(0)); Set actualUuids = shardMetadata.stream().map(ShardMetadata::getShardUuid).collect(toSet()); assertEquals(actualUuids, ImmutableSet.copyOf(expectedUuids)); @@ -359,24 +440,225 @@ public void testReplaceShardUuids() expectedAllUuids.addAll(expectedUuids); // check that shards are replaced in index table as well - Set shardNodes = ImmutableSet.copyOf(shardManager.getShardNodes(tableId, TupleDomain.all())); + Set shardNodes = ImmutableSet.copyOf(shardManager.getShardNodes(tableId, TupleDomain.all(), true)); + Set actualAllUuids = shardNodes.stream() + .map(BucketShards::getShards) + .flatMap(Collection::stream) + .map(ShardNodes::getShardUuid) + .collect(toSet()); + assertEquals(actualAllUuids, expectedAllUuids); + + // Verify conflict is handled + // Try to replace shard1 with newShards again (shard1 already deleted, delete shards that's already deleted) + try { + newShards = ImmutableList.of(shardInfo(UUID.randomUUID(), nodes.get(0))); + transactionId = shardManager.beginTransaction(); + shardManager.replaceShardUuids(transactionId, tableId, columns, replaceUuidMap, newShards, OptionalLong.of(0), true); + fail("expected exception"); + } + catch (PrestoException e) { + assertEquals(e.getErrorCode(), TRANSACTION_CONFLICT.toErrorCode()); + } + // Try to add new delta to shard1 (shard1 already deleted) + try { + transactionId = shardManager.beginTransaction(); + ShardInfo newDelta = shardInfo(UUID.randomUUID(), nodes.get(0)); + Map shardMap = ImmutableMap.of(originalUuids.get(0), new DeltaInfoPair(Optional.empty(), Optional.of(newDelta))); + shardManager.replaceDeltaUuids(transactionId, tableId, columns, shardMap, OptionalLong.of(0)); + fail("expected exception"); + } + catch (PrestoException e) { + assertEquals(e.getErrorCode(), TRANSACTION_CONFLICT.toErrorCode()); + } + // Try to delete shard1 (shard1 already deleted) + try { + transactionId = shardManager.beginTransaction(); + Map shardMap = ImmutableMap.of(originalUuids.get(0), new DeltaInfoPair(Optional.empty(), Optional.empty())); + shardManager.replaceDeltaUuids(transactionId, tableId, columns, shardMap, OptionalLong.of(0)); + fail("expected exception"); + } + catch (PrestoException e) { + assertEquals(e.getErrorCode(), TRANSACTION_CONFLICT.toErrorCode()); + } + } + + @Test + public void testReplaceDeltaUuidsFunction() + throws SQLException + { + // node1 shard1 shard4 + // node2 shard2 + // node3 shard3 + + // goal: shard4 add delta1 + + // Initial data + long tableId = createTable("test"); + List columns = ImmutableList.of(new ColumnInfo(1, BIGINT)); + UUID uuid1 = UUID.randomUUID(); + UUID uuid2 = UUID.randomUUID(); + UUID uuid3 = UUID.randomUUID(); + UUID uuid4 = UUID.randomUUID(); + ShardInfo shardInfo1 = new ShardInfo(uuid1, OptionalInt.empty(), ImmutableSet.of("node1"), ImmutableList.of(), 1, 1, 1, 1); + ShardInfo shardInfo2 = new ShardInfo(uuid2, OptionalInt.empty(), ImmutableSet.of("node2"), ImmutableList.of(), 2, 2, 2, 2); + ShardInfo shardInfo3 = new ShardInfo(uuid3, OptionalInt.empty(), ImmutableSet.of("node3"), ImmutableList.of(), 3, 3, 3, 3); + ShardInfo shardInfo4 = new ShardInfo(uuid4, OptionalInt.empty(), ImmutableSet.of("node1"), ImmutableList.of(), 4, 4, 4, 4); + + shardManager.createTable(tableId, columns, false, OptionalLong.empty(), true); + long transactionId = shardManager.beginTransaction(); + shardManager.commitShards(transactionId, tableId, columns, ImmutableList.of(shardInfo1, shardInfo2, shardInfo3, shardInfo4), Optional.empty(), 0); + + // delta + UUID delta1 = UUID.randomUUID(); + ShardInfo deltaInfo1 = new ShardInfo(delta1, OptionalInt.empty(), ImmutableSet.of("node1"), ImmutableList.of(), 1, 1, 1, 1); + + // toReplace + Map shardMap = ImmutableMap.of(uuid4, new DeltaInfoPair(Optional.empty(), Optional.of(deltaInfo1))); + transactionId = shardManager.beginTransaction(); + shardManager.replaceDeltaUuids(transactionId, tableId, columns, shardMap, OptionalLong.of(0)); + + // check shards on this node1 are correct + Set shardMetadata = shardManager.getNodeShards("node1"); + assertEquals(shardMetadata.size(), 3); + + // check index table as well + Set shardNodes = ImmutableSet.copyOf(shardManager.getShardNodes(tableId, TupleDomain.all(), true)); Set actualAllUuids = shardNodes.stream() .map(BucketShards::getShards) .flatMap(Collection::stream) .map(ShardNodes::getShardUuid) .collect(toSet()); + Set expectedAllUuids = ImmutableSet.of(uuid1, uuid2, uuid3, uuid4); assertEquals(actualAllUuids, expectedAllUuids); - // verify that conflicting updates are handled - newShards = ImmutableList.of(shardInfo(UUID.randomUUID(), nodes.get(0))); + // Verify statistics + Statement statement = dummyHandle.getConnection().createStatement(); + ResultSet resultSet = statement.executeQuery("SELECT * FROM tables where table_id = " + tableId); + resultSet.next(); + assertEquals(resultSet.getLong("shard_count"), 4); + assertEquals(resultSet.getLong("delta_count"), 1); + assertEquals(resultSet.getLong("row_count"), 9); + assertEquals(resultSet.getLong("compressed_size"), 11); + assertEquals(resultSet.getLong("uncompressed_size"), 11); + resultSet.close(); + statement.close(); + } + + @Test + public void testReplaceDeltaUuids() + { + // node1 shard1 / node2 shard2 / node3 shard3 + // Add delta to shard1 + // Delete shard2 + long tableId = createTable("test"); + List columns = ImmutableList.of(new ColumnInfo(1, BIGINT)); + List nodes = ImmutableList.of("node1", "node2", "node3"); + List originalUuids = ImmutableList.of(UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID()); + + List oldShards = ImmutableList.builder() + .add(shardInfo(originalUuids.get(0), nodes.get(0))) + .add(shardInfo(originalUuids.get(1), nodes.get(1))) + .add(shardInfo(originalUuids.get(2), nodes.get(2))) + .build(); + + shardManager.createTable(tableId, columns, false, OptionalLong.empty(), true); + + long transactionId = shardManager.beginTransaction(); + shardManager.commitShards(transactionId, tableId, columns, oldShards, Optional.empty(), 0); + + UUID newDeltaUuid1 = UUID.randomUUID(); + ShardInfo newDeltaShard1 = shardInfo(newDeltaUuid1, nodes.get(0)); + Map shardMap = new HashMap<>(); + shardMap.put(originalUuids.get(0), new DeltaInfoPair(Optional.empty(), Optional.of(newDeltaShard1))); + shardMap.put(originalUuids.get(1), new DeltaInfoPair(Optional.empty(), Optional.empty())); + + transactionId = shardManager.beginTransaction(); + shardManager.replaceDeltaUuids(transactionId, tableId, columns, shardMap, OptionalLong.of(0)); + + // check that delta shard are added in shards table for node1 + Set shardMetadata = shardManager.getNodeShards(nodes.get(0)); + Map> actualUuidsMap = shardMetadata.stream().collect(toImmutableMap(ShardMetadata::getShardUuid, ShardMetadata::getDeltaUuid)); + Map> expectedUuidsMap = ImmutableMap.of(originalUuids.get(0), Optional.of(newDeltaUuid1), newDeltaUuid1, Optional.empty()); + assertEquals(actualUuidsMap, expectedUuidsMap); + + // check that shard are deleted in shards table for node2 + shardMetadata = shardManager.getNodeShards(nodes.get(1)); + actualUuidsMap = shardMetadata.stream().collect(toImmutableMap(ShardMetadata::getShardUuid, ShardMetadata::getDeltaUuid)); + expectedUuidsMap = ImmutableMap.of(); + assertEquals(actualUuidsMap, expectedUuidsMap); + + // check index table, delta added and shard removed + Set shardNodes = ImmutableSet.copyOf(shardManager.getShardNodes(tableId, TupleDomain.all(), true)); + Set expectedshardNodes = ImmutableSet.of( + new BucketShards(OptionalInt.empty(), ImmutableSet.of(new ShardNodes(originalUuids.get(0), Optional.of(newDeltaUuid1), ImmutableSet.of(nodes.get(0))))), + new BucketShards(OptionalInt.empty(), ImmutableSet.of(new ShardNodes(originalUuids.get(2), Optional.empty(), ImmutableSet.of(nodes.get(2)))))); + assertEquals(shardNodes, expectedshardNodes); + + // Verify conflict is handled + // Try to replace shard1 with newShards without knowing its new delta + // stimulate the other thread didn't catch the change (actually it's already committed up) try { transactionId = shardManager.beginTransaction(); - shardManager.replaceShardUuids(transactionId, tableId, columns, replacedUuids, newShards, OptionalLong.of(0)); + Map> replaceUuidMap = ImmutableMap.of(originalUuids.get(0), Optional.empty()); + Set newShards = ImmutableSet.of(shardInfo(UUID.randomUUID(), nodes.get(0))); + shardManager.replaceShardUuids(transactionId, tableId, columns, replaceUuidMap, newShards, OptionalLong.of(0), true); fail("expected exception"); + // todo check transaction id roll back + // todo shard change roll back } catch (PrestoException e) { assertEquals(e.getErrorCode(), TRANSACTION_CONFLICT.toErrorCode()); } + // Try to delete shard1 with newShards without knowing its new delta + try { + transactionId = shardManager.beginTransaction(); + shardManager.replaceDeltaUuids(transactionId, tableId, columns, + ImmutableMap.of(originalUuids.get(0), new DeltaInfoPair(Optional.empty(), Optional.empty())), OptionalLong.of(0)); + fail("expected exception"); + } + catch (PrestoException e) { + assertEquals(e.getErrorCode(), TRANSACTION_CONFLICT.toErrorCode()); + } + + // node1 shard1 newDelta / node3 shard3 + // replace the newDelta with another new delta + transactionId = shardManager.beginTransaction(); + UUID anotherNewDeltaUuid1 = UUID.randomUUID(); + shardMap = ImmutableMap.of(originalUuids.get(0), new DeltaInfoPair(Optional.of(newDeltaUuid1), Optional.of(shardInfo(anotherNewDeltaUuid1, nodes.get(0))))); + shardManager.replaceDeltaUuids(transactionId, tableId, columns, shardMap, OptionalLong.of(0)); + + // check that delta shard are added in shards table for node1 + shardMetadata = shardManager.getNodeShards(nodes.get(0)); + actualUuidsMap = shardMetadata.stream().collect(toImmutableMap(ShardMetadata::getShardUuid, ShardMetadata::getDeltaUuid)); + expectedUuidsMap = ImmutableMap.of(originalUuids.get(0), Optional.of(anotherNewDeltaUuid1), anotherNewDeltaUuid1, Optional.empty()); + assertEquals(actualUuidsMap, expectedUuidsMap); + + // check index table, delta modified + shardNodes = ImmutableSet.copyOf(shardManager.getShardNodes(tableId, TupleDomain.all(), true)); + expectedshardNodes = ImmutableSet.of( + new BucketShards(OptionalInt.empty(), ImmutableSet.of(new ShardNodes(originalUuids.get(0), Optional.of(anotherNewDeltaUuid1), ImmutableSet.of(nodes.get(0))))), + new BucketShards(OptionalInt.empty(), ImmutableSet.of(new ShardNodes(originalUuids.get(2), Optional.empty(), ImmutableSet.of(nodes.get(2)))))); + assertEquals(shardNodes, expectedshardNodes); + + // node1 shard1 anotherNewDelta / node3 shard3 + // rewrite shard1 to shard4 + transactionId = shardManager.beginTransaction(); + UUID uuid4 = UUID.randomUUID(); + Map> replaceUuidMap = ImmutableMap.of(originalUuids.get(0), Optional.of(anotherNewDeltaUuid1)); + shardManager.replaceShardUuids(transactionId, tableId, columns, replaceUuidMap, ImmutableSet.of(shardInfo(uuid4, nodes.get(0))), OptionalLong.of(0), true); + + // check that new shard are added, old shard and delta are deleted in shards table for node1 + shardMetadata = shardManager.getNodeShards(nodes.get(0)); + actualUuidsMap = shardMetadata.stream().collect(toImmutableMap(ShardMetadata::getShardUuid, ShardMetadata::getDeltaUuid)); + expectedUuidsMap = ImmutableMap.of(uuid4, Optional.empty()); + assertEquals(actualUuidsMap, expectedUuidsMap); + + // check index table, old shard and delta deleted, new shard added + shardNodes = ImmutableSet.copyOf(shardManager.getShardNodes(tableId, TupleDomain.all(), true)); + expectedshardNodes = ImmutableSet.of( + new BucketShards(OptionalInt.empty(), ImmutableSet.of(new ShardNodes(uuid4, Optional.empty(), ImmutableSet.of(nodes.get(0))))), + new BucketShards(OptionalInt.empty(), ImmutableSet.of(new ShardNodes(originalUuids.get(2), Optional.empty(), ImmutableSet.of(nodes.get(2)))))); + assertEquals(shardNodes, expectedshardNodes); } @Test @@ -457,7 +739,7 @@ public void testEmptyTable() List columns = ImmutableList.of(new ColumnInfo(1, BIGINT)); shardManager.createTable(tableId, columns, false, OptionalLong.empty(), false); - try (ResultIterator iterator = shardManager.getShardNodes(tableId, TupleDomain.all())) { + try (ResultIterator iterator = shardManager.getShardNodes(tableId, TupleDomain.all(), false)) { assertFalse(iterator.hasNext()); } } @@ -469,7 +751,7 @@ public void testEmptyTableBucketed() List columns = ImmutableList.of(new ColumnInfo(1, BIGINT)); shardManager.createTable(tableId, columns, true, OptionalLong.empty(), false); - try (ResultIterator iterator = shardManager.getShardNodesBucketed(tableId, true, ImmutableList.of(), TupleDomain.all())) { + try (ResultIterator iterator = shardManager.getShardNodesBucketed(tableId, true, ImmutableList.of(), TupleDomain.all(), false)) { assertFalse(iterator.hasNext()); } } @@ -733,7 +1015,8 @@ public void testMaintenanceBlocked() long transactionId = shardManager.beginTransaction(); try { - shardManager.replaceShardUuids(transactionId, tableId, columns, oldShards, ImmutableSet.of(), OptionalLong.empty()); + Map> oldShardMap = oldShards.stream().collect(toImmutableMap(Function.identity(), uuid -> Optional.empty())); + shardManager.replaceShardUuids(transactionId, tableId, columns, oldShardMap, ImmutableSet.of(), OptionalLong.empty(), false); fail("expected exception"); } catch (PrestoException e) { @@ -744,7 +1027,7 @@ public void testMaintenanceBlocked() private Set getShardNodes(long tableId, TupleDomain predicate) { - try (ResultIterator iterator = shardManager.getShardNodes(tableId, predicate)) { + try (ResultIterator iterator = shardManager.getShardNodes(tableId, predicate, false)) { return ImmutableSet.copyOf(concat(transform(iterator, i -> i.getShards().iterator()))); } } @@ -767,7 +1050,7 @@ public static ShardInfo shardInfo(UUID shardUuid, String nodeId, List toShardNodes(List shards) { return shards.stream() - .map(shard -> new ShardNodes(shard.getShardUuid(), shard.getNodeIdentifiers())) + .map(shard -> new ShardNodes(shard.getShardUuid(), Optional.empty(), shard.getNodeIdentifiers())) .collect(toSet()); } diff --git a/presto-raptor/src/test/java/com/facebook/presto/raptor/metadata/TestRaptorSplitManager.java b/presto-raptor/src/test/java/com/facebook/presto/raptor/metadata/TestRaptorSplitManager.java index 9bf0a587e2624..5cf12d2759511 100644 --- a/presto-raptor/src/test/java/com/facebook/presto/raptor/metadata/TestRaptorSplitManager.java +++ b/presto-raptor/src/test/java/com/facebook/presto/raptor/metadata/TestRaptorSplitManager.java @@ -20,6 +20,7 @@ import com.facebook.presto.raptor.RaptorColumnHandle; import com.facebook.presto.raptor.RaptorConnectorId; import com.facebook.presto.raptor.RaptorMetadata; +import com.facebook.presto.raptor.RaptorSplit; import com.facebook.presto.raptor.RaptorSplitManager; import com.facebook.presto.raptor.RaptorTableHandle; import com.facebook.presto.raptor.RaptorTableLayoutHandle; @@ -73,6 +74,7 @@ import static java.util.concurrent.TimeUnit.MINUTES; import static java.util.stream.Collectors.toList; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; @Test(singleThreaded = true) public class TestRaptorSplitManager @@ -155,7 +157,11 @@ public void testSanity() ConnectorSplitSource splitSource = getSplits(raptorSplitManager, layout); int splitCount = 0; while (!splitSource.isFinished()) { - splitCount += getSplits(splitSource, 1000).size(); + List splits = getSplits(splitSource, 1000); + splitCount += splits.size(); + RaptorSplit split = (RaptorSplit) (splits.get(0)); + assertFalse(split.isTableSupportsDeltaDelete()); + assertEquals(split.getColumnTypes(), Optional.empty()); } assertEquals(splitCount, 4); } diff --git a/presto-raptor/src/test/java/com/facebook/presto/raptor/metadata/TestShardDao.java b/presto-raptor/src/test/java/com/facebook/presto/raptor/metadata/TestShardDao.java index 2208432b4d5e0..e7cc247d95ef3 100644 --- a/presto-raptor/src/test/java/com/facebook/presto/raptor/metadata/TestShardDao.java +++ b/presto-raptor/src/test/java/com/facebook/presto/raptor/metadata/TestShardDao.java @@ -24,6 +24,7 @@ import org.testng.annotations.Test; import java.sql.SQLException; +import java.util.Optional; import java.util.OptionalInt; import java.util.OptionalLong; import java.util.Set; @@ -178,11 +179,11 @@ public void testNodeShards() OptionalInt noBucket = OptionalInt.empty(); OptionalLong noRange = OptionalLong.empty(); - ShardMetadata shard1 = new ShardMetadata(plainTableId, shardId1, shardUuid1, noBucket, 1, 11, 111, OptionalLong.of(888_111), noRange, noRange); - ShardMetadata shard2 = new ShardMetadata(plainTableId, shardId2, shardUuid2, noBucket, 2, 22, 222, OptionalLong.of(888_222), noRange, noRange); - ShardMetadata shard3 = new ShardMetadata(bucketedTableId, shardId3, shardUuid3, OptionalInt.of(8), 3, 33, 333, OptionalLong.of(888_333), noRange, noRange); - ShardMetadata shard4 = new ShardMetadata(bucketedTableId, shardId4, shardUuid4, OptionalInt.of(9), 4, 44, 444, OptionalLong.of(888_444), noRange, noRange); - ShardMetadata shard5 = new ShardMetadata(bucketedTableId, shardId5, shardUuid5, OptionalInt.of(7), 5, 55, 555, OptionalLong.of(888_555), noRange, noRange); + ShardMetadata shard1 = new ShardMetadata(plainTableId, shardId1, shardUuid1, false, Optional.empty(), noBucket, 1, 11, 111, OptionalLong.of(888_111), noRange, noRange); + ShardMetadata shard2 = new ShardMetadata(plainTableId, shardId2, shardUuid2, false, Optional.empty(), noBucket, 2, 22, 222, OptionalLong.of(888_222), noRange, noRange); + ShardMetadata shard3 = new ShardMetadata(bucketedTableId, shardId3, shardUuid3, false, Optional.empty(), OptionalInt.of(8), 3, 33, 333, OptionalLong.of(888_333), noRange, noRange); + ShardMetadata shard4 = new ShardMetadata(bucketedTableId, shardId4, shardUuid4, false, Optional.empty(), OptionalInt.of(9), 4, 44, 444, OptionalLong.of(888_444), noRange, noRange); + ShardMetadata shard5 = new ShardMetadata(bucketedTableId, shardId5, shardUuid5, false, Optional.empty(), OptionalInt.of(7), 5, 55, 555, OptionalLong.of(888_555), noRange, noRange); assertEquals(dao.getShards(plainTableId), ImmutableSet.of(shardUuid1, shardUuid2)); assertEquals(dao.getShards(bucketedTableId), ImmutableSet.of(shardUuid3, shardUuid4, shardUuid5)); diff --git a/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/TestBucketBalancer.java b/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/TestBucketBalancer.java index 2688feadf34a0..697103b048c6d 100644 --- a/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/TestBucketBalancer.java +++ b/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/TestBucketBalancer.java @@ -279,7 +279,7 @@ private long createBucketedTable(String tableName, long distributionId, DataSize List columnsA = ImmutableList.of(new ColumnInfo(1, BIGINT)); shardManager.createTable(tableId, columnsA, false, OptionalLong.empty(), false); - metadataDao.updateTableStats(tableId, 1024, 1024 * 1024 * 1024, compressedSize.toBytes(), compressedSize.toBytes() * 2); + metadataDao.updateTableStats(tableId, 1024, 0, 1024 * 1024 * 1024, compressedSize.toBytes(), compressedSize.toBytes() * 2); return tableId; } diff --git a/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/TestOrcFileRewriter.java b/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/TestOrcFileRewriter.java index dff12aed8ae5a..278492d083573 100644 --- a/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/TestOrcFileRewriter.java +++ b/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/TestOrcFileRewriter.java @@ -58,6 +58,7 @@ import java.util.BitSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.OptionalInt; import java.util.UUID; import java.util.stream.Collectors; @@ -533,6 +534,8 @@ public void testRewriterDropThenAddDifferentColumns() ConnectorPageSource source = storageManager.getPageSource( FileSystemContext.DEFAULT_RAPTOR_CONTEXT, uuid, + Optional.empty(), + false, OptionalInt.empty(), ImmutableList.of(13L, 7L, 18L), ImmutableList.of(createVarcharType(5), createVarcharType(20), INTEGER), @@ -652,6 +655,8 @@ public void testRewriterDropThenAddSameColumns() ConnectorPageSource source = storageManager.getPageSource( FileSystemContext.DEFAULT_RAPTOR_CONTEXT, uuid, + Optional.empty(), + false, OptionalInt.empty(), ImmutableList.of(3L, 7L, 8L), ImmutableList.of(createVarcharType(5), createVarcharType(20), INTEGER), diff --git a/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/TestOrcStorageManager.java b/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/TestOrcStorageManager.java index 0572e6c0b14ca..208b8a1fa83b6 100644 --- a/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/TestOrcStorageManager.java +++ b/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/TestOrcStorageManager.java @@ -26,6 +26,7 @@ import com.facebook.presto.raptor.filesystem.LocalOrcDataEnvironment; import com.facebook.presto.raptor.filesystem.RaptorLocalFileSystem; import com.facebook.presto.raptor.metadata.ColumnStats; +import com.facebook.presto.raptor.metadata.ShardDeleteDelta; import com.facebook.presto.raptor.metadata.ShardDelta; import com.facebook.presto.raptor.metadata.ShardInfo; import com.facebook.presto.raptor.metadata.ShardManager; @@ -34,6 +35,7 @@ import com.facebook.presto.spi.ConnectorPageSource; import com.facebook.presto.spi.NodeManager; import com.facebook.presto.spi.Page; +import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.block.Block; import com.facebook.presto.spi.predicate.NullableValue; import com.facebook.presto.spi.predicate.TupleDomain; @@ -354,14 +356,17 @@ public void testRewriter() // delete one row BitSet rowsToDelete = new BitSet(); rowsToDelete.set(0); - Collection fragments = manager.rewriteShard( + InplaceShardRewriter shardRewriter = (InplaceShardRewriter) manager.createShardRewriter( + FileSystemContext.DEFAULT_RAPTOR_CONTEXT, fileSystem, transactionId, OptionalInt.empty(), shards.get(0).getShardUuid(), - IntStream.range(0, columnIds.size()).boxed().collect(Collectors.toMap(index -> String.valueOf(columnIds.get(index)), columnTypes::get)), - rowsToDelete); - + 2, + Optional.empty(), + false, + IntStream.range(0, columnIds.size()).boxed().collect(Collectors.toMap(index -> String.valueOf(columnIds.get(index)), columnTypes::get))); + Collection fragments = shardRewriter.rewriteShard(rowsToDelete); Slice shardDelta = Iterables.getOnlyElement(fragments); ShardDelta shardDeltas = jsonCodec(ShardDelta.class).fromJson(shardDelta.getBytes()); ShardInfo shardInfo = Iterables.getOnlyElement(shardDeltas.getNewShards()); @@ -381,6 +386,181 @@ public void testRewriter() assertEquals(recordedShards.get(1).getShardUuid(), shardInfo.getShardUuid()); } + @Test + public void testWriteDeltaDelete() + throws Exception + { + FileSystem fileSystem = new LocalOrcDataEnvironment().getFileSystem(FileSystemContext.DEFAULT_RAPTOR_CONTEXT); + + // delete one row + BitSet rowsToDelete = new BitSet(); + rowsToDelete.set(0); + Collection fragments = deltaDelete(rowsToDelete, false); + + Slice shardDelta = Iterables.getOnlyElement(fragments); + ShardDeleteDelta shardDeltas = jsonCodec(ShardDeleteDelta.class).fromJson(shardDelta.getBytes()); + ShardInfo shardInfo = shardDeltas.getDeltaInfoPair().getNewDeltaDeleteShard().get(); + + // Check that output file (new delta file) has one row + assertEquals(shardInfo.getRowCount(), 1); + assertTrue(checkContent(fileSystem, shardInfo.getShardUuid(), rowsToDelete)); + + // Check that storage file is same as backup file + File storageFile = new File(storageService.getStorageFile(shardInfo.getShardUuid()).toString()); + File backupFile = fileBackupStore.getBackupFile(shardInfo.getShardUuid()); + assertFileEquals(storageFile, backupFile); + + // Verify recorded shard + List recordedShards = shardRecorder.getShards(); + assertEquals(recordedShards.size(), 2); // original file + delta file + assertEquals(recordedShards.get(1).getTransactionId(), TRANSACTION_ID); + assertEquals(recordedShards.get(1).getShardUuid(), shardInfo.getShardUuid()); + } + + @Test + public void testWriteDeltaDeleteEmpty() + { + // delete zero row + BitSet rowsToDelete = new BitSet(); + Collection fragments = deltaDelete(rowsToDelete, false); + + assertEquals(ImmutableList.of(), fragments); + List recordedShards = shardRecorder.getShards(); + assertEquals(recordedShards.size(), 1); // no delta file + } + + @Test + public void testWriteDeltaDeleteAll() + { + // delete every row + BitSet rowsToDelete = new BitSet(); + rowsToDelete.set(0); + rowsToDelete.set(1); + rowsToDelete.set(2); + Collection fragments = deltaDelete(rowsToDelete, false); + + Slice shardDelta = Iterables.getOnlyElement(fragments); + ShardDeleteDelta shardDeltas = jsonCodec(ShardDeleteDelta.class).fromJson(shardDelta.getBytes()); + assertEquals(shardDeltas.getDeltaInfoPair().getNewDeltaDeleteShard(), Optional.empty()); + + // verify recorded shard + List recordedShards = shardRecorder.getShards(); + assertEquals(recordedShards.size(), 1); + } + + @Test + // rowsToDelete and rowsDeleted must be mutually exclusive + public void testWriteDeltaDeleteMerge() + throws Exception + { + FileSystem fileSystem = new LocalOrcDataEnvironment().getFileSystem(FileSystemContext.DEFAULT_RAPTOR_CONTEXT); + + BitSet rowsToDelete = new BitSet(); + rowsToDelete.set(0); + Collection fragments = deltaDelete(rowsToDelete, true); + + Slice shardDelta = Iterables.getOnlyElement(fragments); + ShardDeleteDelta shardDeltas = jsonCodec(ShardDeleteDelta.class).fromJson(shardDelta.getBytes()); + ShardInfo shardInfo = shardDeltas.getDeltaInfoPair().getNewDeltaDeleteShard().get(); + + // Check that output file (new delta file) has merged 2 rows + assertEquals(shardInfo.getRowCount(), 2); + assertTrue(checkContent(fileSystem, shardInfo.getShardUuid(), rowsToDelete)); + + // Check that storage file is same as backup file + File storageFile = new File(storageService.getStorageFile(shardInfo.getShardUuid()).toString()); + File backupFile = fileBackupStore.getBackupFile(shardInfo.getShardUuid()); + assertFileEquals(storageFile, backupFile); + + // Verify recorded shard + List recordedShards = shardRecorder.getShards(); + assertEquals(recordedShards.size(), 3); // original file + old delta + new delta + assertEquals(recordedShards.get(2).getTransactionId(), TRANSACTION_ID); + assertEquals(recordedShards.get(2).getShardUuid(), shardInfo.getShardUuid()); + } + + @Test + public void testWriteDeltaDeleteMergeAll() + { + // delete every row + BitSet rowsToDelete = new BitSet(); + rowsToDelete.set(0); + rowsToDelete.set(1); + Collection fragments = deltaDelete(rowsToDelete, true); + + Slice shardDelta = Iterables.getOnlyElement(fragments); + ShardDeleteDelta shardDeltas = jsonCodec(ShardDeleteDelta.class).fromJson(shardDelta.getBytes()); + assertEquals(shardDeltas.getDeltaInfoPair().getNewDeltaDeleteShard(), Optional.empty()); + + // verify recorded shard + List recordedShards = shardRecorder.getShards(); + assertEquals(recordedShards.size(), 2); // original file + old delta + } + + @Test(expectedExceptions = PrestoException.class) + public void testWriteDeltaDeleteMergeConflict() + { + // delete same row + BitSet rowsToDelete = new BitSet(); + rowsToDelete.set(2); + Collection fragments = deltaDelete(rowsToDelete, true); + deltaDelete(rowsToDelete, true); + } + + private Collection deltaDelete(BitSet rowsToDelete, boolean oldDeltaDeleteExist) + { + OrcStorageManager manager = createOrcStorageManager(); + FileSystem fileSystem = new LocalOrcDataEnvironment().getFileSystem(FileSystemContext.DEFAULT_RAPTOR_CONTEXT); + + List columnIds = ImmutableList.of(3L, 7L); + List columnTypes = ImmutableList.of(BIGINT, createVarcharType(10)); + + // create file with 3 rows + StoragePageSink sink = createStoragePageSink(manager, columnIds, columnTypes); + List pages = rowPagesBuilder(columnTypes) + .row(123L, "hello") + .row(456L, "bye") + .row(456L, "test") + .build(); + sink.appendPages(pages); + List shards = getFutureValue(sink.commit()); + assertEquals(shardRecorder.getShards().size(), 1); + + List oldDeltaDeleteShards = null; + if (oldDeltaDeleteExist) { + // create oldDeltaDeleteExist with 1 row + List deltaColumnIds = ImmutableList.of(0L); + List deltaColumnTypes = ImmutableList.of(BIGINT); + StoragePageSink deltaSink = createStoragePageSink(manager, deltaColumnIds, deltaColumnTypes); + List deltaPages = rowPagesBuilder(deltaColumnTypes) + .row(2L) + .build(); + deltaSink.appendPages(deltaPages); + oldDeltaDeleteShards = getFutureValue(deltaSink.commit()); + } + + // delta delete + DeltaShardRewriter shardRewriter = (DeltaShardRewriter) manager.createShardRewriter( + FileSystemContext.DEFAULT_RAPTOR_CONTEXT, + fileSystem, + TRANSACTION_ID, + OptionalInt.empty(), + shards.get(0).getShardUuid(), + 3, + oldDeltaDeleteExist ? Optional.of(oldDeltaDeleteShards.get(0).getShardUuid()) : Optional.empty(), + true, + null); + Collection fragments = shardRewriter.writeDeltaDeleteFile(rowsToDelete); + return fragments; + } + + private boolean checkContent(FileSystem fileSystem, UUID shardUuid, BitSet rowsToDelete) + { + OrcStorageManager manager = createOrcStorageManager(); + Optional rows = manager.getRowsFromUuid(fileSystem, Optional.of(shardUuid)); + return rows.map(r -> r.equals(rowsToDelete)).orElse(false); + } + public void testWriterRollback() { // verify staging directory is empty @@ -576,7 +756,7 @@ private static ConnectorPageSource getPageSource( UUID uuid, TupleDomain tupleDomain) { - return manager.getPageSource(FileSystemContext.DEFAULT_RAPTOR_CONTEXT, uuid, OptionalInt.empty(), columnIds, columnTypes, tupleDomain, READER_ATTRIBUTES); + return manager.getPageSource(FileSystemContext.DEFAULT_RAPTOR_CONTEXT, uuid, Optional.empty(), false, OptionalInt.empty(), columnIds, columnTypes, tupleDomain, READER_ATTRIBUTES); } private static StoragePageSink createStoragePageSink(StorageManager manager, List columnIds, List columnTypes) diff --git a/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/TestShardEjector.java b/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/TestShardEjector.java index 6da0b736e8255..1a8f724284428 100644 --- a/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/TestShardEjector.java +++ b/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/TestShardEjector.java @@ -143,7 +143,7 @@ public void testEjector() ejector.process(); - shardManager.getShardNodes(tableId, TupleDomain.all()); + shardManager.getShardNodes(tableId, TupleDomain.all(), false); Set ejectedShards = shards.subList(0, 4).stream() .map(ShardInfo::getShardUuid) diff --git a/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/organization/TestShardCompactor.java b/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/organization/TestShardCompactor.java index c0c53fba077d8..4f2c2a745ca84 100644 --- a/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/organization/TestShardCompactor.java +++ b/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/organization/TestShardCompactor.java @@ -44,6 +44,7 @@ import java.io.File; import java.io.IOException; import java.util.List; +import java.util.Optional; import java.util.OptionalInt; import java.util.Set; import java.util.UUID; @@ -264,7 +265,7 @@ private MaterializedResult getMaterializedRows(StorageManager storageManager, Li private ConnectorPageSource getPageSource(StorageManager storageManager, List columnIds, List columnTypes, UUID uuid) { - return storageManager.getPageSource(FileSystemContext.DEFAULT_RAPTOR_CONTEXT, uuid, OptionalInt.empty(), columnIds, columnTypes, TupleDomain.all(), READER_ATTRIBUTES); + return storageManager.getPageSource(FileSystemContext.DEFAULT_RAPTOR_CONTEXT, uuid, Optional.empty(), false, OptionalInt.empty(), columnIds, columnTypes, TupleDomain.all(), READER_ATTRIBUTES); } private static List createSortedShards(StorageManager storageManager, List columnIds, List columnTypes, List sortChannels, List sortOrders, int shardCount)