From c7e30bd2cb6e3e6e0d656efe2c8c26b1b6f0b1f3 Mon Sep 17 00:00:00 2001 From: rangao Date: Wed, 1 Feb 2023 23:51:34 +0800 Subject: [PATCH 01/15] Support query compaction data using Pulsar SQL. --- .../mledger/impl/ManagedCursorImpl.java | 1 + .../LeastResourceUsageWithWeightTest.java | 1 + .../src/main/resources/conf/jvm.config | 8 + .../sql/presto/PulsarConnectorCache.java | 43 ++ .../sql/presto/PulsarInternalColumn.java | 4 + .../pulsar/sql/presto/PulsarRecordCursor.java | 393 ++++++++++++++++-- .../apache/pulsar/sql/presto/PulsarSplit.java | 13 +- .../pulsar/sql/presto/PulsarSplitManager.java | 58 ++- .../presto/PulsarSqlSchemaInfoProvider.java | 3 + .../sql/presto/util/ReadCompactedType.java | 34 ++ .../org/apache/pulsar/sql/presto/Demo.java | 114 +++++ .../sql/presto/TestCacheSizeAllocator.java | 1 + .../pulsar/sql/presto/TestCompactedQuery.java | 265 ++++++++++++ .../sql/presto/TestPulsarConnector.java | 2 +- .../sql/presto/TestPulsarRecordCursor.java | 2 +- .../sql/presto/TestPulsarSplitManager.java | 26 +- .../sql/presto/TestReadChunkedMessages.java | 2 + 17 files changed, 917 insertions(+), 53 deletions(-) create mode 100644 pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/util/ReadCompactedType.java create mode 100644 pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/Demo.java create mode 100644 pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestCompactedQuery.java diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 5851395b08566..0e10df6365948 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -1048,6 +1048,7 @@ public boolean hasMoreEntries() { // is // at the last entry in the previous ledger PositionImpl writerPosition = ledger.getLastPosition(); + System.out.println("xxxx writer position is " + writerPosition); if (writerPosition.getEntryId() != -1) { return readPosition.compareTo(writerPosition) <= 0; } else { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/LeastResourceUsageWithWeightTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/LeastResourceUsageWithWeightTest.java index 2856dde892a8f..2a9e556b1df10 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/LeastResourceUsageWithWeightTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/LeastResourceUsageWithWeightTest.java @@ -36,6 +36,7 @@ import org.apache.pulsar.broker.loadbalance.extensions.LoadManagerContext; import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLoadData; import org.apache.pulsar.broker.loadbalance.extensions.store.LoadDataStore; +import org.apache.pulsar.client.api.TableView; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.ServiceUnitId; import org.apache.pulsar.common.naming.TopicName; diff --git a/pulsar-sql/presto-distribution/src/main/resources/conf/jvm.config b/pulsar-sql/presto-distribution/src/main/resources/conf/jvm.config index 86c9d0613b233..9ae7e633f8ae5 100644 --- a/pulsar-sql/presto-distribution/src/main/resources/conf/jvm.config +++ b/pulsar-sql/presto-distribution/src/main/resources/conf/jvm.config @@ -26,3 +26,11 @@ -XX:+ExitOnOutOfMemoryError -Dpresto-temporarily-allow-java8=true -Djdk.attach.allowAttachSelf=true + +-Dzookeeper.serverCnxnFactory=org.apache.zookeeper.server.NettyServerCnxnFactory +-Dzookeeper.ssl.keyStore.location=/Users/ran/Work/pulsar-test/apache-pulsar-2.11.0-SNAPSHOT/certWithPrivateKey.pem +#-Dzookeeper.ssl.keyStore.password=testpass +-Dzookeeper.ssl.trustStore.location=/Users/ran/Work/pulsar-test/apache-pulsar-2.11.0-SNAPSHOT/ca-cert.pem +#-Dzookeeper.ssl.trustStore.password=testpass +-Dzookeeper.ssl.keyStore.type=PEM +-Dzookeeper.ssl.trustStore.type=PEM diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarConnectorCache.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarConnectorCache.java index 20b00b59e5ab8..0b91ba83616bf 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarConnectorCache.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarConnectorCache.java @@ -26,6 +26,12 @@ import java.util.Map; import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; + +import lombok.Getter; +import org.apache.bookkeeper.client.AsyncCallback; +import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.common.util.OrderedScheduler; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.mledger.LedgerOffloader; @@ -38,10 +44,12 @@ import org.apache.bookkeeper.mledger.impl.NullLedgerOffloader; import org.apache.bookkeeper.mledger.offload.Offloaders; import org.apache.bookkeeper.mledger.offload.OffloadersCache; +import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsProvider; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.common.policies.data.EnsemblePlacementPolicyConfig; import org.apache.pulsar.common.policies.data.OffloadPoliciesImpl; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.metadata.api.MetadataStoreConfig; @@ -68,15 +76,33 @@ public class PulsarConnectorCache { private LedgerOffloader defaultOffloader; private Map offloaderMap = new ConcurrentHashMap<>(); + @Getter + private final BookKeeper bookKeeper; + private static final String OFFLOADERS_DIRECTOR = "offloadersDirectory"; private static final String MANAGED_LEDGER_OFFLOAD_DRIVER = "managedLedgerOffloadDriver"; private static final String MANAGED_LEDGER_OFFLOAD_MAX_THREADS = "managedLedgerOffloadMaxThreads"; + static class DefaultBkFactory implements ManagedLedgerFactoryImpl.BookkeeperFactoryForCustomEnsemblePlacementPolicy { + + private final BookKeeper bkClient; + + public DefaultBkFactory(ClientConfiguration bkClientConfiguration) + throws InterruptedException, BKException, IOException { + bkClient = new BookKeeper(bkClientConfiguration); + } + + @Override + public BookKeeper get(EnsemblePlacementPolicyConfig policy) { + return bkClient; + } + } private PulsarConnectorCache(PulsarConnectorConfig pulsarConnectorConfig) throws Exception { this.metadataStore = MetadataStoreExtended.create(pulsarConnectorConfig.getMetadataUrl(), MetadataStoreConfig.builder().metadataStoreName(MetadataStoreConfig.METADATA_STORE).build()); this.managedLedgerFactory = initManagedLedgerFactory(pulsarConnectorConfig); + this.bookKeeper = initBookKeeper(pulsarConnectorConfig); this.statsProvider = PulsarConnectorUtils.createInstance(pulsarConnectorConfig.getStatsProvider(), StatsProvider.class, getClass().getClassLoader()); @@ -131,6 +157,23 @@ private ManagedLedgerFactory initManagedLedgerFactory(PulsarConnectorConfig puls return new ManagedLedgerFactoryImpl(metadataStore, bkClientConfiguration, managedLedgerFactoryConfig); } + private BookKeeper initBookKeeper(PulsarConnectorConfig pulsarConnectorConfig) throws Exception { + PulsarMetadataClientDriver.init(); + + ClientConfiguration bkClientConfiguration = new ClientConfiguration() + .setMetadataServiceUri("metadata-store:" + pulsarConnectorConfig.getMetadataUrl()) + .setClientTcpNoDelay(false) + .setUseV2WireProtocol(pulsarConnectorConfig.getBookkeeperUseV2Protocol()) + .setExplictLacInterval(pulsarConnectorConfig.getBookkeeperExplicitInterval()) + .setStickyReadsEnabled(false) + .setReadEntryTimeout(60) + .setThrottleValue(pulsarConnectorConfig.getBookkeeperThrottleValue()) + .setNumIOThreads(pulsarConnectorConfig.getBookkeeperNumIOThreads()) + .setNumWorkerThreads(pulsarConnectorConfig.getBookkeeperNumWorkerThreads()) + .setNettyMaxFrameSizeBytes(pulsarConnectorConfig.getMaxMessageSize() + Commands.MESSAGE_SIZE_FRAME_PADDING); + return new BookKeeper(bkClientConfiguration); + } + public ManagedLedgerConfig getManagedLedgerConfig(NamespaceName namespaceName, OffloadPoliciesImpl offloadPolicies, PulsarConnectorConfig pulsarConnectorConfig) { ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarInternalColumn.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarInternalColumn.java index a0812085f04e1..46f93165e537a 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarInternalColumn.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarInternalColumn.java @@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import io.trino.spi.type.BigintType; +import io.trino.spi.type.BooleanType; import io.trino.spi.type.IntegerType; import io.trino.spi.type.TimestampType; import io.trino.spi.type.Type; @@ -62,6 +63,9 @@ public class PulsarInternalColumn { public static final PulsarInternalColumn PROPERTIES = new PulsarInternalColumn("__properties__", VarcharType.VARCHAR, "User defined properties"); + public static final PulsarInternalColumn READ_COMPACTED = new PulsarInternalColumn("__read_compacted__", + VarcharType.VARCHAR, "Read compacted flag"); + private static Set internalFields = ImmutableSet.of(PARTITION, EVENT_TIME, PUBLISH_TIME, MESSAGE_ID, SEQUENCE_ID, PRODUCER_NAME, KEY, PROPERTIES); diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java index f86335ae3780b..e520c9f681db5 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java @@ -38,7 +38,9 @@ import io.trino.spi.connector.RecordCursor; import io.trino.spi.type.Type; import java.io.IOException; +import java.util.Enumeration; import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Objects; @@ -48,6 +50,14 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; + +import lombok.Getter; +import org.apache.bookkeeper.client.AsyncCallback; +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.client.LedgerEntry; +import org.apache.bookkeeper.client.LedgerHandle; +import org.apache.bookkeeper.client.api.DigestType; +import org.apache.bookkeeper.client.api.LedgerEntries; import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; @@ -59,14 +69,23 @@ import org.apache.bookkeeper.mledger.impl.ReadOnlyCursorImpl; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.impl.BatchMessageIdImpl; import org.apache.pulsar.client.impl.schema.KeyValueSchemaInfo; +import org.apache.pulsar.common.api.proto.CompressionType; +import org.apache.pulsar.common.api.proto.MessageIdData; +import org.apache.pulsar.common.api.proto.MessageMetadata; +import org.apache.pulsar.common.api.proto.SingleMessageMetadata; import org.apache.pulsar.common.api.raw.MessageParser; import org.apache.pulsar.common.api.raw.RawMessage; import org.apache.pulsar.common.api.raw.RawMessageIdImpl; import org.apache.pulsar.common.api.raw.RawMessageImpl; +import org.apache.pulsar.common.compression.CompressionCodec; +import org.apache.pulsar.common.compression.CompressionCodecProvider; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.OffloadPoliciesImpl; +import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; +import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.protocol.schema.BytesSchemaVersion; import org.apache.pulsar.common.schema.KeyValue; import org.apache.pulsar.common.schema.KeyValueEncodingType; @@ -76,6 +95,7 @@ import org.apache.pulsar.sql.presto.util.CacheSizeAllocator; import org.apache.pulsar.sql.presto.util.NoStrictCacheSizeAllocator; import org.apache.pulsar.sql.presto.util.NullCacheSizeAllocator; +import org.apache.pulsar.sql.presto.util.ReadCompactedType; import org.jctools.queues.MessagePassingQueue; import org.jctools.queues.SpscArrayQueue; @@ -85,6 +105,7 @@ */ public class PulsarRecordCursor implements RecordCursor { + @Getter private List columnHandles; private PulsarSplit pulsarSplit; private PulsarConnectorConfig pulsarConnectorConfig; @@ -123,6 +144,19 @@ public class PulsarRecordCursor implements RecordCursor { protected ConcurrentOpenHashMap chunkedMessagesMap = ConcurrentOpenHashMap.newBuilder().build(); + private PulsarConnectorCache pulsarConnectorCache; + private OffloadPoliciesImpl offloadPolicies; + private volatile Throwable compactedHandleError; + private final ReadCompactedType readCompactedType; + private long compactedLedgerId = -1; +// private Position compactedHorizon; + protected ConcurrentOpenHashMap compactedMessage = + ConcurrentOpenHashMap.newBuilder().build(); + private LedgerHandle compactedLedgerHandle; + private BookKeeper bookKeeper; + private ManagedLedgerFactory managedLedgerFactory; + private ManagedLedgerConfig managedLedgerConfig; + private static final Logger log = Logger.get(PulsarRecordCursor.class); public PulsarRecordCursor(List columnHandles, PulsarSplit pulsarSplit, @@ -131,7 +165,6 @@ public PulsarRecordCursor(List columnHandles, PulsarSplit pu this.splitSize = pulsarSplit.getSplitSize(); // Set start time for split this.startTime = System.nanoTime(); - PulsarConnectorCache pulsarConnectorCache; try { pulsarConnectorCache = PulsarConnectorCache.getConnectorCache(pulsarConnectorConfig); } catch (Exception e) { @@ -140,12 +173,13 @@ public PulsarRecordCursor(List columnHandles, PulsarSplit pu throw new RuntimeException(e); } - OffloadPoliciesImpl offloadPolicies = pulsarSplit.getOffloadPolicies(); + offloadPolicies = pulsarSplit.getOffloadPolicies(); if (offloadPolicies != null) { offloadPolicies.setOffloadersDirectory(pulsarConnectorConfig.getOffloadersDirectory()); offloadPolicies.setManagedLedgerOffloadMaxThreads( pulsarConnectorConfig.getManagedLedgerOffloadMaxThreads()); } + this.readCompactedType = pulsarSplit.getReadCompactedType(); initialize(columnHandles, pulsarSplit, pulsarConnectorConfig, pulsarConnectorCache.getManagedLedgerFactory(), pulsarConnectorCache.getManagedLedgerConfig( @@ -154,7 +188,6 @@ public PulsarRecordCursor(List columnHandles, PulsarSplit pu pulsarConnectorConfig), new PulsarConnectorMetricsTracker(pulsarConnectorCache.getStatsProvider())); this.decoderFactory = decoderFactory; - initEntryCacheSizeAllocator(pulsarConnectorConfig); } // Exposed for testing purposes @@ -163,11 +196,26 @@ public PulsarRecordCursor(List columnHandles, PulsarSplit pu PulsarConnectorMetricsTracker pulsarConnectorMetricsTracker, PulsarDispatchingRowDecoderFactory decoderFactory) { this.splitSize = pulsarSplit.getSplitSize(); + this.readCompactedType = pulsarSplit.getReadCompactedType(); initialize(columnHandles, pulsarSplit, pulsarConnectorConfig, managedLedgerFactory, managedLedgerConfig, pulsarConnectorMetricsTracker); this.decoderFactory = decoderFactory; } + PulsarRecordCursor(List columnHandles, PulsarSplit pulsarSplit, PulsarConnectorConfig + pulsarConnectorConfig, ManagedLedgerFactory managedLedgerFactory, ManagedLedgerConfig managedLedgerConfig, + PulsarConnectorMetricsTracker pulsarConnectorMetricsTracker, + PulsarDispatchingRowDecoderFactory decoderFactory, BookKeeper bookKeeper) { + this.splitSize = pulsarSplit.getSplitSize(); + this.readCompactedType = pulsarSplit.getReadCompactedType(); + this.bookKeeper = bookKeeper; + this.managedLedgerFactory = managedLedgerFactory; + this.managedLedgerConfig = managedLedgerConfig; + initialize(columnHandles, pulsarSplit, pulsarConnectorConfig, managedLedgerFactory, managedLedgerConfig, + pulsarConnectorMetricsTracker); + this.decoderFactory = decoderFactory; + } + private void initialize(List columnHandles, PulsarSplit pulsarSplit, PulsarConnectorConfig pulsarConnectorConfig, ManagedLedgerFactory managedLedgerFactory, ManagedLedgerConfig managedLedgerConfig, PulsarConnectorMetricsTracker pulsarConnectorMetricsTracker) { @@ -198,9 +246,13 @@ private void initialize(List columnHandles, PulsarSplit puls log.info("Initializing split with parameters: %s", pulsarSplit); try { - this.cursor = getCursor(TopicName.get("persistent", NamespaceName.get(pulsarSplit.getSchemaName()), - pulsarSplit.getTableName()), pulsarSplit.getStartPosition(), managedLedgerFactory, managedLedgerConfig); - } catch (ManagedLedgerException | InterruptedException e) { + if (readCompactedType != null) { + initCompactedRead(); + } else { + this.cursor = getCursor(topicName, + pulsarSplit.getStartPosition(), managedLedgerFactory, managedLedgerConfig); + } + } catch (Exception e) { log.error(e, "Failed to get read only cursor"); close(); throw new RuntimeException(e); @@ -273,6 +325,8 @@ public void run() { @Override public void accept(Entry entry) { + System.out.println("xxxx deserialize entry " + + entry.getLedgerId() + ":" + entry.getEntryId() + ", size: " + entry.getLength()); try { entryQueueCacheSizeAllocator.release(entry.getLength()); @@ -306,15 +360,30 @@ public void accept(Entry entry) { message = null; } if (message != null) { - while (true) { - if (!haveAvailableCacheSize( - messageQueueCacheSizeAllocator, messageQueue) - || !messageQueue.offer(message)) { - Thread.sleep(1); - } else { - messageQueueCacheSizeAllocator.allocate( - message.getData().readableBytes()); - break; + if (readCompactedType != null && message.getKey().isPresent()) { + RawMessageIdImpl messageId = + (RawMessageIdImpl) message.getMessageId(); + System.out.println("xxxx parse message with id " + messageId); + if (message.getKey().equals("stock-7")) { + System.out.println(""); + } + compactMessage(message.getKey().get(), + new BatchMessageIdImpl( + messageId.getLedgerId(), + messageId.getEntryId(), + partition, + (int) messageId.getBatchIndex())); + } else { + while (true) { + if (!haveAvailableCacheSize( + messageQueueCacheSizeAllocator, messageQueue) + || !messageQueue.offer(message)) { + Thread.sleep(1); + } else { + messageQueueCacheSizeAllocator.allocate( + message.getData().readableBytes()); + break; + } } } } @@ -481,15 +550,23 @@ private boolean haveAvailableCacheSize(CacheSizeAllocator cacheSizeAllocator, Sp public boolean advanceNextPosition() { if (readEntries == null) { - // start deserialize thread - deserializeEntries = new DeserializeEntries(); - deserializeEntries.setUncaughtExceptionHandler((t, ex) -> { - deserializingError = ex; - }); - deserializeEntries.start(); - - readEntries = new ReadEntries(); - readEntries.run(); + if (readCompactedType != null) { + Thread compactedHandleThread = new Thread(new CompactedLedgerReader()); + compactedHandleThread.setUncaughtExceptionHandler((t, ex) -> { + compactedHandleError = ex; + }); + compactedHandleThread.start(); + } else { + // start deserialize thread + deserializeEntries = new DeserializeEntries(); + deserializeEntries.setUncaughtExceptionHandler((t, ex) -> { + deserializingError = ex; + }); + deserializeEntries.start(); + + readEntries = new ReadEntries(); + readEntries.run(); + } } if (currentMessage != null) { @@ -498,12 +575,14 @@ public boolean advanceNextPosition() { } while (true) { - if (readEntries.hasFinished()) { + if (readEntries != null && readEntries.hasFinished() && compactedMessage.isEmpty()) { return false; } if ((messageQueue.capacity() - messageQueue.size()) > 0) { - readEntries.run(); + if (readEntries != null) { + readEntries.run(); + } } currentMessage = messageQueue.poll(); @@ -512,11 +591,17 @@ public boolean advanceNextPosition() { break; } else if (deserializingError != null) { throw new RuntimeException(deserializingError); + } else if (compactedHandleError != null) { + throw new RuntimeException(compactedHandleError); } else { try { - Thread.sleep(1); + long waitMills = 1; + if (readCompactedType != null) { + waitMills = 10; + } + Thread.sleep(waitMills); // stats for time spent wait to read from message queue because its empty - metricsTracker.register_MESSAGE_QUEUE_DEQUEUE_WAIT_TIME(1); + metricsTracker.register_MESSAGE_QUEUE_DEQUEUE_WAIT_TIME(waitMills); } catch (InterruptedException e) { throw new RuntimeException(e); } @@ -589,6 +674,9 @@ public boolean advanceNextPosition() { .filter(col -> PulsarColumnHandle.HandleKeyValueType.NONE .equals(col.getHandleKeyValueType())) .collect(toImmutableSet())); + if (this.currentMessage.getData().readableBytes() == 0) { + System.out.println(""); + } Optional> decodedValue = messageDecoder.decodeRow(this.currentMessage.getData()); decodedValue.ifPresent(currentRowValuesMap::putAll); @@ -748,6 +836,14 @@ public void close() { } } + if (this.compactedLedgerHandle != null) { + try { + this.compactedLedgerHandle.close(); + } catch (Exception e) { + log.error(e, "Failed to close compacted leger handle."); + } + } + // set stat for total execution time of split if (this.metricsTracker != null) { this.metricsTracker.register_TOTAL_EXECUTION_TIME(System.nanoTime() - startTime); @@ -867,4 +963,245 @@ public void recycle() { } } + private void initCompactedRead() throws Exception { + String topic = topicName.getPartition(partition).toString(); + PersistentTopicInternalStats internalStats = + schemaInfoProvider.getPulsarAdmin().topics().getInternalStats(topic); + if (internalStats.compactedLedger == null + || internalStats.compactedLedger.ledgerId == -1 + || internalStats.compactedLedger.entries == 0) { + return; + } + this.compactedLedgerId = internalStats.compactedLedger.ledgerId; +// if (internalStats.cursors.containsKey("Compaction")) { +// String compactedHorizonStr = internalStats.cursors.get("Compaction").markDeletePosition; +// String[] arr = compactedHorizonStr.split(":"); +// compactedHorizon = PositionImpl.get(Long.parseLong(arr[0]), Long.parseLong(arr[1])); +// } + this.compactedLedgerHandle = bookKeeper.openLedger(compactedLedgerId, + BookKeeper.DigestType.fromApiDigestType(DigestType.CRC32), "".getBytes()); + } + + private void compactMessage(String key, BatchMessageIdImpl messageId) { + if (key == null) { + return; + } + switch (readCompactedType) { + case COMPACTED_LATEST -> compactedMessage.put(key, messageId); + case COMPACTED_EARLIEST -> compactedMessage.computeIfAbsent(key, __ -> messageId); + } + } + + class CompactedLedgerReader implements AsyncCallback.ReadCallback, Runnable { + + private final int readSize = 1000; + private long startEntry = 0; + private long readEntry = 0; + private SpscArrayQueue queue; + private MessageIdData lastMessageId; + private AtomicBoolean havePendingRead = new AtomicBoolean(false); + + CompactedLedgerReader() { + queue = new SpscArrayQueue<>((int) (readSize * 1.5)); + lastMessageId = new MessageIdData(); + lastMessageId.setLedgerId(-1); + lastMessageId.setEntryId(-1); + } + + private void readMoreEntriesIfNeed() { + if (havePendingRead.get()) { + return; + } + havePendingRead.set(true); + if (queue.size() < readSize / 2) { + long endEntry = Math.min(compactedLedgerHandle.getLastAddConfirmed(), startEntry + readSize); + compactedLedgerHandle.asyncReadEntries(startEntry, endEntry, this, null); + } + } + + @Override + public void readComplete(int i, LedgerHandle ledgerHandle, Enumeration enumeration, Object o) { + while (enumeration.hasMoreElements()) { + LedgerEntry ledgerEntry = enumeration.nextElement(); + queue.offer(ledgerEntry); + startEntry = ledgerEntry.getEntryId() + 1; + System.out.println("increase read entry " + startEntry); + } + havePendingRead.set(false); + } + + @Override + public void run() { + readCompactedData(); + readNewerData(); + readOriginalData(); + } + + private void readCompactedData() { + if (compactedLedgerId == -1 || compactedLedgerHandle == null) { + return; + } + while (readEntry < compactedLedgerHandle.getLastAddConfirmed()) { + readMoreEntriesIfNeed(); + LedgerEntry ledgerEntry = queue.poll(); + if (ledgerEntry == null) { + try { + Thread.sleep(1); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + System.out.println("ledger entry is null, startEntry: " + startEntry); + continue; + } + + ByteBuf buffer = ledgerEntry.getEntryBuffer(); + int idSize = buffer.readInt(); + + lastMessageId = new MessageIdData(); + lastMessageId.parseFrom(buffer, idSize); + int payloadAndMetadataSize = buffer.readInt(); + ByteBuf metadataAndPayload = buffer.slice(buffer.readerIndex(), payloadAndMetadataSize); + MessageMetadata messageMetadata = Commands.parseMessageMetadata(metadataAndPayload); + + if (messageMetadata.hasNumMessagesInBatch()) { + try { + compactBatchMessage( + ledgerEntry.getLedgerId(), ledgerEntry.getEntryId(), ledgerEntry.getEntryBuffer()); + } catch (IOException e) { + throw new RuntimeException(e); + } + } else { + compactMessage(messageMetadata.getPartitionKey(), new BatchMessageIdImpl( + ledgerEntry.getLedgerId(), ledgerEntry.getEntryId(), partition, 0)); + } + readEntry = ledgerEntry.getEntryId(); + System.out.println("read entry is " + readEntry); + } + } + + private void compactBatchMessage(long ledgerId, long entryId, ByteBuf payload) throws IOException { + MessageMetadata metadata = Commands.parseMessageMetadata(payload); + int batchSize = metadata.getNumMessagesInBatch(); + + CompressionType compressionType = metadata.getCompression(); + CompressionCodec codec = CompressionCodecProvider.getCompressionCodec(compressionType); + int uncompressedSize = metadata.getUncompressedSize(); + ByteBuf uncompressedPayload = codec.decode(payload, uncompressedSize); + + SingleMessageMetadata smm = new SingleMessageMetadata(); + for (int i = 0; i < batchSize; i++) { + ByteBuf singleMessagePayload = Commands.deSerializeSingleMessageInBatch(uncompressedPayload, + smm, + 0, batchSize); + BatchMessageIdImpl id = new BatchMessageIdImpl(ledgerId, entryId, partition, i); + if (!smm.isCompactedOut()) { + compactMessage(smm.getPartitionKey(), id); + } + singleMessagePayload.release(); + } + uncompressedPayload.release(); + } + + private void readNewerData() { + try { + PositionImpl startPosition; + if (lastMessageId.getLedgerId() == -1 && lastMessageId.getEntryId() == -1) { + startPosition = PositionImpl.EARLIEST; + } else { + startPosition = PositionImpl.get(lastMessageId.getLedgerId(), lastMessageId.getEntryId()).getNext(); + } + cursor = getCursor(topicName, startPosition, managedLedgerFactory, managedLedgerConfig); + + readEntries = new ReadEntries(); + readEntries.run(); + + deserializeEntries = new DeserializeEntries(); + deserializeEntries.setUncaughtExceptionHandler((t, ex) -> { + deserializingError = ex; + }); + deserializeEntries.start(); + } catch (Exception e) { + throw new RuntimeException(e); + } + + while (!readEntries.hasFinished()) { + try { + Thread.sleep(1); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + } + + private void readOriginalData() { + Iterator messageIdIterator = compactedMessage.values().stream().sorted().iterator(); + LedgerHandle readLedgerHandle = null; + long readLedgerId = -1; + boolean isCompactedLedger = false; + while (messageIdIterator.hasNext()) { + BatchMessageIdImpl messageId = messageIdIterator.next(); + if (messageId.getLedgerId() == compactedLedgerId) { + readLedgerHandle = compactedLedgerHandle; + isCompactedLedger = true; + } else { + try { + if (readLedgerHandle == null || readLedgerId != messageId.getLedgerId()) { + if (readLedgerHandle != null) { + readLedgerHandle.close(); + } + readLedgerId = messageId.getLedgerId(); + isCompactedLedger = false; + readLedgerHandle = bookKeeper.openLedger(readLedgerId, + BookKeeper.DigestType.CRC32C, "".getBytes()); + } + } catch (Exception e) { + log.error("Failed to read entry "); + throw new RuntimeException(e); + } + } + readEntries(readLedgerHandle, messageId.getLedgerId(), messageId.getEntryId(), + messageId.getBatchIndex(), isCompactedLedger); + } + } + + private void readEntries(LedgerHandle ledgerHandle, long ledgerId, long entryId, int batchIndex, boolean isCompactedLedger) { + try { + LedgerEntries ledgerEntries = + ledgerHandle.read(entryId, entryId); + Iterator iterator = ledgerEntries.iterator(); + if (iterator.hasNext()) { + org.apache.bookkeeper.client.api.LedgerEntry ledgerEntry = iterator.next(); + ByteBuf buffer = ledgerEntry.getEntryBuffer(); + + if (isCompactedLedger) { + int idSize = buffer.readInt(); + MessageIdData lastMessageId = new MessageIdData(); + lastMessageId.parseFrom(buffer, idSize); + int payloadAndMetadataSize = buffer.readInt(); + buffer = buffer.slice(buffer.readerIndex(), payloadAndMetadataSize); + } + + parseAndAddMessageToQueue(ledgerId, entryId, batchIndex, buffer); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + private void parseAndAddMessageToQueue( + long ledgerId, long entryId, int batchIndex, ByteBuf entryBuffer) throws IOException { + MessageParser.parseMessage(topicName, ledgerId, + entryId, entryBuffer, message -> { + if (((RawMessageIdImpl) message.getMessageId()).getBatchIndex() == batchIndex) { + messageQueue.offer(message); + if (message.getKey().isPresent()) { + compactedMessage.remove(message.getKey().get()); + } + } + }, pulsarConnectorConfig.getMaxMessageSize()); + } + + } + + } diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplit.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplit.java index 1967ec5e436b6..e9d4d30df4675 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplit.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplit.java @@ -35,6 +35,7 @@ import org.apache.pulsar.common.policies.data.OffloadPoliciesImpl; import org.apache.pulsar.common.schema.SchemaInfo; import org.apache.pulsar.common.schema.SchemaType; +import org.apache.pulsar.sql.presto.util.ReadCompactedType; /** * This class represents information for a split. @@ -63,6 +64,7 @@ public class PulsarSplit implements ConnectorSplit { private final String schemaInfoProperties; private final OffloadPoliciesImpl offloadPolicies; + private final ReadCompactedType readCompactedType; @JsonCreator public PulsarSplit( @@ -80,7 +82,8 @@ public PulsarSplit( @JsonProperty("endPositionLedgerId") long endPositionLedgerId, @JsonProperty("tupleDomain") TupleDomain tupleDomain, @JsonProperty("schemaInfoProperties") String schemaInfoProperties, - @JsonProperty("offloadPolicies") OffloadPoliciesImpl offloadPolicies) throws IOException { + @JsonProperty("offloadPolicies") OffloadPoliciesImpl offloadPolicies, + @JsonProperty("readCompactedType") ReadCompactedType readCompactedType) throws IOException { this.splitId = splitId; requireNonNull(schemaName, "schema name is null"); this.originSchemaName = originSchemaName; @@ -107,6 +110,8 @@ public PulsarSplit( .schema(schema.getBytes("ISO8859-1")) .properties(objectMapper.readValue(schemaInfoProperties, Map.class)) .build(); + + this.readCompactedType = readCompactedType; } @JsonProperty @@ -192,6 +197,11 @@ public OffloadPoliciesImpl getOffloadPolicies() { return offloadPolicies; } + @JsonProperty + public ReadCompactedType getReadCompactedType() { + return readCompactedType; + } + @Override public boolean isRemotelyAccessible() { return true; @@ -223,6 +233,7 @@ public String toString() { + ", startPositionLedgerId=" + startPositionLedgerId + ", endPositionLedgerId=" + endPositionLedgerId + ", schemaInfoProperties=" + schemaInfoProperties + + ", readCompactedType=" + readCompactedType + (offloadPolicies == null ? "" : offloadPolicies.toString()) + '}'; } diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplitManager.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplitManager.java index 464e70b18dddd..68153e76493d9 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplitManager.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplitManager.java @@ -45,6 +45,7 @@ import java.util.Collections; import java.util.LinkedList; import java.util.List; +import java.util.concurrent.atomic.AtomicReference; import javax.inject.Inject; import lombok.Data; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; @@ -59,8 +60,11 @@ import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.OffloadPoliciesImpl; +import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; +import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.schema.SchemaInfo; +import org.apache.pulsar.sql.presto.util.ReadCompactedType; /** * The class helping to manage splits. @@ -122,6 +126,33 @@ public ConnectorSplitSource getSplits(ConnectorTransactionHandle transactionHand } } + AtomicReference readCompactedTypeReference = new AtomicReference<>(); + tupleDomain.getDomains().ifPresent(__ -> { + Domain readCompactedDomain = tupleDomain.getDomains().get() + .get(PulsarInternalColumn.READ_COMPACTED.getColumnHandle(connectorId, true)); + readCompactedTypeReference.set(ReadCompactedType.valueOf(readCompactedDomain.getSingleValue().toString())); + }); + ReadCompactedType readCompactedType = readCompactedTypeReference.get(); + + long compactedLedgerId; + PositionImpl compactedHorizon = PositionImpl.EARLIEST; + List leftLedgers = new ArrayList<>(); + if (readCompactedType != null) { + try { + String topic = String.format("%s/%s", namespace, tableHandle.getTopicName()); + PersistentTopicInternalStats internalStats = pulsarAdmin.topics().getInternalStats(topic); + compactedLedgerId = internalStats.compactedLedger.ledgerId; + if (internalStats.cursors.containsKey("Compaction")) { + String[] compactedHorizonArr = internalStats.cursors.get("Compaction").markDeletePosition.split(":"); + compactedHorizon = PositionImpl.get( + Long.parseLong(compactedHorizonArr[0]), Long.parseLong(compactedHorizonArr[1])); + } + TopicStats stats = pulsarAdmin.topics().getStats(topic); + } catch (PulsarAdminException e) { + throw new RuntimeException(e); + } + } + Collection splits; try { OffloadPoliciesImpl offloadPolicies = (OffloadPoliciesImpl) this.pulsarAdmin.namespaces() @@ -133,11 +164,11 @@ public ConnectorSplitSource getSplits(ConnectorTransactionHandle transactionHand } if (!PulsarConnectorUtils.isPartitionedTopic(topicName, this.pulsarAdmin)) { splits = getSplitsNonPartitionedTopic( - numSplits, topicName, tableHandle, schemaInfo, tupleDomain, offloadPolicies); + numSplits, topicName, tableHandle, schemaInfo, tupleDomain, offloadPolicies, readCompactedType, compactedHorizon); log.debug("Splits for non-partitioned topic %s: %s", topicName, splits); } else { splits = getSplitsPartitionedTopic( - numSplits, topicName, tableHandle, schemaInfo, tupleDomain, offloadPolicies); + numSplits, topicName, tableHandle, schemaInfo, tupleDomain, offloadPolicies, readCompactedType, compactedHorizon); log.debug("Splits for partitioned topic %s: %s", topicName, splits); } } catch (Exception e) { @@ -150,7 +181,8 @@ public ConnectorSplitSource getSplits(ConnectorTransactionHandle transactionHand @VisibleForTesting Collection getSplitsPartitionedTopic(int numSplits, TopicName topicName, PulsarTableHandle tableHandle, SchemaInfo schemaInfo, TupleDomain tupleDomain, - OffloadPoliciesImpl offloadPolicies) throws Exception { + OffloadPoliciesImpl offloadPolicies, ReadCompactedType readCompactedType, + PositionImpl compactedHorizon) throws Exception { List predicatedPartitions = getPredicatedPartitions(topicName, tupleDomain); if (log.isDebugEnabled()) { @@ -181,7 +213,9 @@ Collection getSplitsPartitionedTopic(int numSplits, TopicName topic schemaInfo, topicName.getPartition(predicatedPartitions.get(i)).getLocalName(), tupleDomain, - offloadPolicies)); + offloadPolicies, + readCompactedType, + compactedHorizon)); } return splits; } @@ -238,7 +272,8 @@ private List getPredicatedPartitions(TopicName topicName, TupleDomain getSplitsNonPartitionedTopic(int numSplits, TopicName topicName, PulsarTableHandle tableHandle, SchemaInfo schemaInfo, TupleDomain tupleDomain, - OffloadPoliciesImpl offloadPolicies) throws Exception { + OffloadPoliciesImpl offloadPolicies, ReadCompactedType readCompactedType, + PositionImpl compactedHorizon) throws Exception { PulsarConnectorCache pulsarConnectorCache = PulsarConnectorCache.getConnectorCache(pulsarConnectorConfig); ManagedLedgerFactory managedLedgerFactory = pulsarConnectorCache.getManagedLedgerFactory(); ManagedLedgerConfig managedLedgerConfig = pulsarConnectorCache.getManagedLedgerConfig( @@ -253,7 +288,9 @@ Collection getSplitsNonPartitionedTopic(int numSplits, TopicName to schemaInfo, topicName.getLocalName(), tupleDomain, - offloadPolicies); + offloadPolicies, + readCompactedType, + compactedHorizon); } @VisibleForTesting @@ -264,14 +301,16 @@ Collection getSplitsForTopic(String topicNamePersistenceEncoding, PulsarTableHandle tableHandle, SchemaInfo schemaInfo, String tableName, TupleDomain tupleDomain, - OffloadPoliciesImpl offloadPolicies) + OffloadPoliciesImpl offloadPolicies, + ReadCompactedType readCompactedType, + PositionImpl compactedHorizon) throws ManagedLedgerException, InterruptedException, IOException { ReadOnlyCursor readOnlyCursor = null; try { readOnlyCursor = managedLedgerFactory.openReadOnlyCursor( topicNamePersistenceEncoding, - PositionImpl.EARLIEST, managedLedgerConfig); + compactedHorizon, managedLedgerConfig); long numEntries = readOnlyCursor.getNumberOfEntries(); if (numEntries <= 0) { @@ -324,7 +363,8 @@ Collection getSplitsForTopic(String topicNamePersistenceEncoding, endPosition.getLedgerId(), tupleDomain, objectMapper.writeValueAsString(schemaInfo.getProperties()), - offloadPolicies); + offloadPolicies, + readCompactedType); splits.add(pulsarSplit); } return splits; diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSqlSchemaInfoProvider.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSqlSchemaInfoProvider.java index d8f7db96b83d7..97a567290af31 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSqlSchemaInfoProvider.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSqlSchemaInfoProvider.java @@ -27,6 +27,8 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; + +import lombok.Getter; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Schema; @@ -48,6 +50,7 @@ public class PulsarSqlSchemaInfoProvider implements SchemaInfoProvider { private final TopicName topicName; + @Getter private final PulsarAdmin pulsarAdmin; private final LoadingCache cache = CacheBuilder.newBuilder().maximumSize(100000) diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/util/ReadCompactedType.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/util/ReadCompactedType.java new file mode 100644 index 0000000000000..668038cb2af00 --- /dev/null +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/util/ReadCompactedType.java @@ -0,0 +1,34 @@ +package org.apache.pulsar.sql.presto.util; + +import org.apache.commons.lang3.StringUtils; + +public enum ReadCompactedType { + + COMPACTED_LATEST("COMPACTED_LATEST"), + + COMPACTED_EARLIEST("COMPACTED_EARLIEST"); + + private final String type; + + ReadCompactedType(String type) { + this.type = type; + } + + public String getType() { + return type; + } + +// public ReadCompactedType valueOf(String type) { +// if (StringUtils.isEmpty(type)) { +// return null; +// } +// +// for (ReadCompactedType readCompactedType : ReadCompactedType.values()) { +// if (readCompactedType.type.equalsIgnoreCase(type)) { +// return readCompactedType; +// } +// } +// return null; +// } + +} diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/Demo.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/Demo.java new file mode 100644 index 0000000000000..6d1a7553ff53f --- /dev/null +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/Demo.java @@ -0,0 +1,114 @@ +package org.apache.pulsar.sql.presto; + +import lombok.Data; +import lombok.NoArgsConstructor; +import lombok.ToString; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.Schema; + +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; + +public class Demo { + + @Data + @NoArgsConstructor + @ToString + static class User { + private String name; + private Integer age; + } + + public void sendData() throws Exception { + PulsarClient pulsarClient = PulsarClient.builder() + .serviceUrl("pulsar://localhost:6650") + .build(); + + String topic = "user"; + Producer producer = pulsarClient.newProducer(Schema.AVRO(User.class)) + .topic(topic) + .create(); + + Consumer consumer = pulsarClient.newConsumer(Schema.AVRO(User.class)) + .topic(topic) + .subscriptionName("sub") + .subscribe(); + consumer.close(); + + for (int i = 0; i < 10; i++) { + User user = new User(); + user.setName("user-" + i); + user.setAge(10 + i); + producer.newMessage().value(user).send(); + } + + producer.close(); + pulsarClient.close(); + } + + public void sendDataWithRepeatedKey() throws Exception { + PulsarClient pulsarClient = PulsarClient.builder() + .serviceUrl("pulsar://localhost:6650") + .build(); + + String topic = "pt-10"; + Producer producer = pulsarClient.newProducer(Schema.AVRO(User.class)) + .topic(topic) + .create(); + + Consumer consumer = pulsarClient.newConsumer(Schema.AVRO(User.class)) + .topic(topic) + .subscriptionName("sub") + .readCompacted(true) + .subscribe(); + consumer.close(); + + for (int i = 0; i < 100; i++) { + User user = new User(); + user.setName("user-" + i); + user.setAge(10 + i); + producer.newMessage().key("" + i % 10).value(user).send(); + } + + producer.close(); + pulsarClient.close(); + } + + public void readCompactedData() throws Exception { + PulsarClient pulsarClient = PulsarClient.builder() + .serviceUrl("pulsar://localhost:6650") + .build(); + + String topic = "pt-10"; + Consumer consumer = pulsarClient.newConsumer(Schema.AVRO(User.class)) + .topic(topic) + .subscriptionName("sub") + .readCompacted(true) + .subscribe(); + + long receiveCount = 0; + while (true) { + Message message = consumer.receive(5, TimeUnit.SECONDS); + if (message == null) { + break; + } + receiveCount++; + System.out.println("receive message " + receiveCount + + ", key: " + message.getKey() + + ", value: " + message.getValue().toString() + + ", messageId: " + message.getMessageId()); + } + + consumer.close(); + pulsarClient.close(); + } + + public static void main(String[] args) throws Exception { + Demo demo = new Demo(); + demo.readCompactedData(); + } + +} diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestCacheSizeAllocator.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestCacheSizeAllocator.java index 716cd2cf9d663..57079190a8891 100644 --- a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestCacheSizeAllocator.java +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestCacheSizeAllocator.java @@ -114,6 +114,7 @@ public void cacheSizeAllocatorTest(long entryQueueSizeBytes) throws Exception { lastPosition.getLedgerId(), TupleDomain.all(), objectMapper.writeValueAsString(new HashMap<>()), + null, null); List pulsarColumnHandles = TestPulsarConnector.getColumnColumnHandles( diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestCompactedQuery.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestCompactedQuery.java new file mode 100644 index 0000000000000..274cdf4366c7b --- /dev/null +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestCompactedQuery.java @@ -0,0 +1,265 @@ +package org.apache.pulsar.sql.presto; + +import static org.testng.AssertJUnit.assertEquals; +import static org.testng.AssertJUnit.assertTrue; + +import com.google.common.collect.Sets; +import io.airlift.slice.Slice; +import io.trino.spi.connector.ConnectorContext; +import io.trino.spi.predicate.TupleDomain; +import io.trino.testing.TestingConnectorContext; +import java.math.BigDecimal; +import java.math.RoundingMode; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.ManagedLedgerConfig; +import org.apache.bookkeeper.stats.NullStatsProvider; +import org.apache.commons.lang3.RandomStringUtils; +import org.apache.commons.lang3.RandomUtils; +import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.apache.pulsar.client.admin.LongRunningProcessStatus; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.impl.MessageIdImpl; +import org.apache.pulsar.common.naming.TopicDomain; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.common.schema.SchemaType; +import org.apache.pulsar.sql.presto.util.ReadCompactedType; +import org.testcontainers.shaded.com.fasterxml.jackson.databind.ObjectMapper; +import org.testcontainers.shaded.org.awaitility.Awaitility; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + + +@Slf4j +public class TestCompactedQuery extends MockedPulsarServiceBaseTest { + + @BeforeClass + @Override + public void setup() throws Exception { + super.internalSetup(); + + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); + + // so that clients can test short names + admin.tenants().createTenant("public", + new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("test"))); + admin.namespaces().createNamespace("public/default"); + admin.namespaces().setNamespaceReplicationClusters("public/default", Sets.newHashSet("test")); + } + + @AfterClass + @Override + public void cleanup() throws Exception { + super.internalCleanup(); + } + + @Data + @NoArgsConstructor + @AllArgsConstructor + static class Stock { + private String no; + private Double price; + } + + @DataProvider(name = "compactInfoProvider") + public Object[][] compactInfoProvider() { + return new Object[][]{ + {0, 0}, + {0, 100}, + {100, 0}, + {100, 100}, + }; + } + + @Test(dataProvider = "compactInfoProvider") + public void compactQueryForBatchMessages(int compactedMsgNum, int unCompactedMsgNum) throws Exception { + compactQuery(true, compactedMsgNum, unCompactedMsgNum); + } + + @Test(dataProvider = "compactInfoProvider") + public void compactQueryForNonBatchMessages(int compactedMsgNum, int unCompactedMsgNum) throws Exception { + compactQuery(false, compactedMsgNum, unCompactedMsgNum); + } + + private void compactQuery(boolean enableBatch, int compactedMsgNum, int unCompactedMsgNum) throws Exception { + TopicName topicName = TopicName.get( + TopicDomain.persistent.toString(), "public", "default", + RandomStringUtils.randomAlphabetic(5)); + + pulsarClient.newConsumer(Schema.AVRO(Stock.class)) + .topic(topicName.toString()) + .readCompacted(true) + .subscriptionName("sub") + .subscribe() + .close(); + + Producer producer = pulsarClient.newProducer(Schema.AVRO(Stock.class)) + .topic(topicName.toString()) + .enableBatching(enableBatch) + .batchingMaxMessages(5) + .batchingMaxPublishDelay(1, TimeUnit.SECONDS) + .create(); + + AtomicInteger sendReceiptsCount = new AtomicInteger(0); + Map latestPrice = new HashMap<>(); + AtomicReference firstMessageId = new AtomicReference<>((MessageIdImpl) MessageId.earliest); + AtomicReference lastMessageId = new AtomicReference<>((MessageIdImpl) MessageId.latest); + AtomicBoolean setValueForKey7 = new AtomicBoolean(false); + for (int i = 0; i < compactedMsgNum; i++) { + String name = "stock-" + i % 10; + Double price = BigDecimal.valueOf( + RandomUtils.nextDouble(10, 100)).setScale(4, RoundingMode.HALF_UP).doubleValue(); + final int index = i; + producer.newMessage().key(name).value(getStock(name, price, name, setValueForKey7)).sendAsync() + .thenAccept(messageId -> { + if (index == 0) { + firstMessageId.set((MessageIdImpl) messageId); + } + if (index == compactedMsgNum - 1) { + lastMessageId.set(getNextMessageId(messageId)); + } + latestPrice.put(name, price); + sendReceiptsCount.incrementAndGet(); + System.out.println("xxxx send message with id " + messageId); + }); + } + + admin.topics().triggerCompaction(topicName.toString()); + Awaitility.await().until(() -> { + LongRunningProcessStatus status = admin.topics().compactionStatus(topicName.toString()); + return Objects.equals(LongRunningProcessStatus.Status.SUCCESS, status.status); + }); + + setValueForKey7 = new AtomicBoolean(false); + Set entrySet = new HashSet<>(); + for (int i = 0; i < unCompactedMsgNum; i++) { + String name = "stock-" + i % 10; + Double price = BigDecimal.valueOf( + RandomUtils.nextDouble(10, 100)).setScale(4, RoundingMode.HALF_UP).doubleValue(); + final int index = i; + producer.newMessage().key(name).value(getStock(name, price, name, setValueForKey7)).sendAsync() + .thenAccept(messageId -> { + if (index == 0 && firstMessageId.get() == null) { + firstMessageId.set((MessageIdImpl) messageId); + } + if (index == unCompactedMsgNum - 1) { + lastMessageId.set(getNextMessageId(messageId)); + } + + latestPrice.put(name, price); + entrySet.add(((MessageIdImpl) messageId).getLedgerId() + ":" + + ((MessageIdImpl) messageId).getEntryId()); + sendReceiptsCount.incrementAndGet(); + System.out.println("xxxx 2 send message with id " + messageId); + }); + } + + Awaitility.await() + .atMost(5, TimeUnit.SECONDS) + .until(() -> sendReceiptsCount.get() == (compactedMsgNum + unCompactedMsgNum)); + log.info("finish to prepare compaction data"); +// MessageId messageId = producer.newMessage().value(new Stock("", 0.0)).send(); +// System.out.println("xxxx the last message id is " + messageId); + + ObjectMapper objectMapper = new ObjectMapper(); + PulsarSplit pulsarSplit = new PulsarSplit( + 0, + "connector-id", + topicName.getNamespace(), + topicName.getLocalName(), + topicName.getLocalName(), + entrySet.size(), + new String(Schema.AVRO(Stock.class).getSchemaInfo().getSchema()), + SchemaType.AVRO, + firstMessageId.get().getEntryId(), + lastMessageId.get().getEntryId(), + firstMessageId.get().getLedgerId(), + lastMessageId.get().getLedgerId(), + TupleDomain.all(), + objectMapper.writeValueAsString(new HashMap<>()), + null, + ReadCompactedType.COMPACTED_LATEST); + + List pulsarColumnHandles = TestPulsarConnector.getColumnColumnHandles( + topicName, Schema.AVRO(Stock.class).getSchemaInfo(), PulsarColumnHandle.HandleKeyValueType.NONE, true); + + PulsarConnectorConfig connectorConfig = new PulsarConnectorConfig(); + connectorConfig.setBrokerServiceUrl(admin.getServiceUrl()); + connectorConfig.setMetadataUrl("zk:localhost:2181"); + ConnectorContext prestoConnectorContext = new TestingConnectorContext(); + PulsarRecordCursor pulsarRecordCursor = new PulsarRecordCursor( + pulsarColumnHandles, pulsarSplit, connectorConfig, pulsar.getManagedLedgerFactory(), + new ManagedLedgerConfig(), new PulsarConnectorMetricsTracker(new NullStatsProvider()), + new PulsarDispatchingRowDecoderFactory(prestoConnectorContext.getTypeManager()), + this.mockBookKeeper); + + List columns = pulsarRecordCursor.getColumnHandles(); + while (pulsarRecordCursor.advanceNextPosition()) { + String name = null; + String key = null; + Double price = null; + for (int i = 0; i < columns.size(); i++) { + if (columns.get(i).getName().equalsIgnoreCase("no")) { + Slice slice = pulsarRecordCursor.getSlice(i); + name = slice == null ? null : new String(slice.getBytes()); + } else if (columns.get(i).getName().equalsIgnoreCase("price")) { + price = pulsarRecordCursor.getDouble(i); + } else if (columns.get(i).getName().equalsIgnoreCase(PulsarInternalColumn.KEY.getName())) { + Slice slice = pulsarRecordCursor.getSlice(i); + key = slice == null ? null : new String(slice.getBytes()); + } + } + Assert.assertNotNull(key); + if (key.equals("stock-7")) { + Assert.assertNull(name); + Assert.assertNull(price); + continue; + } + Assert.assertNotNull(name); + Assert.assertNotNull(price); + assertTrue(latestPrice.containsKey(key)); + assertEquals(name, key); + assertEquals(latestPrice.remove(key), price); + System.out.println("read message key: " + key + ", price: " + price); + } + Assert.assertTrue(latestPrice.isEmpty()); + } + + private MessageIdImpl getNextMessageId(MessageId messageId) { + MessageIdImpl id = (MessageIdImpl) messageId; + return new MessageIdImpl(id.getLedgerId(), id.getEntryId() + 1, id.getPartitionIndex()); + } + + private Stock getStock(String name, Double price, String key, AtomicBoolean setValueForKey7) { + Stock stock = new Stock(name, price); + // If the key is 7, set a value first, then delete the value for the key + if (key.equals("stock-7")) { + if (!setValueForKey7.get()) { + setValueForKey7.set(true); + } else { + stock = null; + } + } + return stock; + } + +} diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarConnector.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarConnector.java index 61f6edd38530c..51591247c91b1 100644 --- a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarConnector.java +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarConnector.java @@ -296,7 +296,7 @@ public static class Bar { 0, topicsToNumEntries.get(topicName.getSchemaName()), 0, 0, TupleDomain.all(), objectMapper.writeValueAsString( - topicsToSchemas.get(topicName.getSchemaName()).getProperties()), null)); + topicsToSchemas.get(topicName.getSchemaName()).getProperties()), null, null)); } } diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarRecordCursor.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarRecordCursor.java index 7eaa2da498f45..681bcccc3be1d 100644 --- a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarRecordCursor.java +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarRecordCursor.java @@ -420,7 +420,7 @@ public Long answer(InvocationOnMock invocationOnMock) throws Throwable { 0, entriesNum, 0, 0, TupleDomain.all(), objectMapper.writeValueAsString( - schema.getSchemaInfo().getProperties()), null); + schema.getSchemaInfo().getProperties()), null, null); PulsarRecordCursor pulsarRecordCursor = spy(new PulsarRecordCursor( ColumnHandles, split, diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarSplitManager.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarSplitManager.java index 86b2ee56c85fe..09233a919f9ea 100644 --- a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarSplitManager.java +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarSplitManager.java @@ -96,7 +96,7 @@ public void testTopic(String delimiter) throws Exception { PulsarTableLayoutHandle pulsarTableLayoutHandle = new PulsarTableLayoutHandle(pulsarTableHandle, TupleDomain.all()); final ResultCaptor> resultCaptor = new ResultCaptor<>(); - doAnswer(resultCaptor).when(this.pulsarSplitManager).getSplitsNonPartitionedTopic(anyInt(), any(), any(), any(), any(), any()); + doAnswer(resultCaptor).when(this.pulsarSplitManager).getSplitsNonPartitionedTopic(anyInt(), any(), any(), any(), any(), any(), any(), any()); ConnectorSplitSource connectorSplitSource = this.pulsarSplitManager.getSplits( @@ -104,7 +104,7 @@ public void testTopic(String delimiter) throws Exception { pulsarTableLayoutHandle, null); verify(this.pulsarSplitManager, times(1)) - .getSplitsNonPartitionedTopic(anyInt(), any(), any(), any(), any(), any()); + .getSplitsNonPartitionedTopic(anyInt(), any(), any(), any(), any(), any(), any(), any()); int totalSize = 0; for (PulsarSplit pulsarSplit : resultCaptor.getResult()) { @@ -143,13 +143,13 @@ public void testPartitionedTopic(String delimiter) throws Exception { PulsarTableLayoutHandle pulsarTableLayoutHandle = new PulsarTableLayoutHandle(pulsarTableHandle, TupleDomain.all()); final ResultCaptor> resultCaptor = new ResultCaptor<>(); - doAnswer(resultCaptor).when(this.pulsarSplitManager).getSplitsPartitionedTopic(anyInt(), any(), any(), any(), any(), any()); + doAnswer(resultCaptor).when(this.pulsarSplitManager).getSplitsPartitionedTopic(anyInt(), any(), any(), any(), any(), any(), any(), any()); this.pulsarSplitManager.getSplits(mock(ConnectorTransactionHandle.class), mock(ConnectorSession.class), pulsarTableLayoutHandle, null); verify(this.pulsarSplitManager, times(1)) - .getSplitsPartitionedTopic(anyInt(), any(), any(), any(), any(), any()); + .getSplitsPartitionedTopic(anyInt(), any(), any(), any(), any(), any(), any(), any()); int partitions = partitionedTopicsToPartitions.get(topicName.toString()); @@ -210,7 +210,7 @@ public void testPublishTimePredicatePushdown(String delimiter) throws Exception final ResultCaptor> resultCaptor = new ResultCaptor<>(); doAnswer(resultCaptor).when(this.pulsarSplitManager) - .getSplitsNonPartitionedTopic(anyInt(), any(), any(), any(), any(), any()); + .getSplitsNonPartitionedTopic(anyInt(), any(), any(), any(), any(), any(), any(), any()); ConnectorSplitSource connectorSplitSource = this.pulsarSplitManager.getSplits( mock(ConnectorTransactionHandle.class), mock(ConnectorSession.class), @@ -218,7 +218,7 @@ public void testPublishTimePredicatePushdown(String delimiter) throws Exception verify(this.pulsarSplitManager, times(1)) - .getSplitsNonPartitionedTopic(anyInt(), any(), any(), any(), any(), any()); + .getSplitsNonPartitionedTopic(anyInt(), any(), any(), any(), any(), any(), any(), any()); int totalSize = 0; int initalStart = 1; @@ -267,7 +267,7 @@ public void testPublishTimePredicatePushdownPartitionedTopic(String delimiter) t final ResultCaptor> resultCaptor = new ResultCaptor<>(); doAnswer(resultCaptor).when(this.pulsarSplitManager) - .getSplitsPartitionedTopic(anyInt(), any(), any(), any(), any(), any()); + .getSplitsPartitionedTopic(anyInt(), any(), any(), any(), any(), any(), any(), any()); ConnectorSplitSource connectorSplitSource = this.pulsarSplitManager.getSplits( mock(ConnectorTransactionHandle.class), mock(ConnectorSession.class), @@ -275,7 +275,7 @@ public void testPublishTimePredicatePushdownPartitionedTopic(String delimiter) t verify(this.pulsarSplitManager, times(1)) - .getSplitsPartitionedTopic(anyInt(), any(), any(), any(), any(), any()); + .getSplitsPartitionedTopic(anyInt(), any(), any(), any(), any(), any(), any(), any()); int partitions = partitionedTopicsToPartitions.get(topicName.toString()); @@ -324,7 +324,7 @@ public void testPartitionFilter(String delimiter) throws Exception { domainMap.put(PulsarInternalColumn.PARTITION.getColumnHandle(pulsarConnectorId.toString(), false), domain); TupleDomain tupleDomain = TupleDomain.withColumnDomains(domainMap); Collection splits = this.pulsarSplitManager.getSplitsPartitionedTopic(2, topicName, pulsarTableHandle, - schemas.getSchemaInfo(topicName.getSchemaName()), tupleDomain, null); + schemas.getSchemaInfo(topicName.getSchemaName()), tupleDomain, null, null, null); if (topicsToNumEntries.get(topicName.getSchemaName()) > 1) { Assert.assertEquals(splits.size(), 2); } @@ -341,7 +341,7 @@ public void testPartitionFilter(String delimiter) throws Exception { domainMap.put(PulsarInternalColumn.PARTITION.getColumnHandle(pulsarConnectorId.toString(), false), domain); tupleDomain = TupleDomain.withColumnDomains(domainMap); splits = this.pulsarSplitManager.getSplitsPartitionedTopic(1, topicName, pulsarTableHandle, - schemas.getSchemaInfo(topicName.getSchemaName()), tupleDomain, null); + schemas.getSchemaInfo(topicName.getSchemaName()), tupleDomain, null, null, null); if (topicsToNumEntries.get(topicName.getSchemaName()) > 1) { Assert.assertEquals(splits.size(), 2); } @@ -357,7 +357,7 @@ public void testPartitionFilter(String delimiter) throws Exception { domainMap.put(PulsarInternalColumn.PARTITION.getColumnHandle(pulsarConnectorId.toString(), false), domain); tupleDomain = TupleDomain.withColumnDomains(domainMap); splits = this.pulsarSplitManager.getSplitsPartitionedTopic(2, topicName, pulsarTableHandle, - schemas.getSchemaInfo(topicName.getSchemaName()), tupleDomain, null); + schemas.getSchemaInfo(topicName.getSchemaName()), tupleDomain, null, null, null); if (topicsToNumEntries.get(topicName.getSchemaName()) > 1) { Assert.assertEquals(splits.size(), 3); } @@ -376,7 +376,7 @@ public void testPartitionFilter(String delimiter) throws Exception { domainMap.put(PulsarInternalColumn.PARTITION.getColumnHandle(pulsarConnectorId.toString(), false), domain); tupleDomain = TupleDomain.withColumnDomains(domainMap); splits = this.pulsarSplitManager.getSplitsPartitionedTopic(2, topicName, pulsarTableHandle, - schemas.getSchemaInfo(topicName.getSchemaName()), tupleDomain, null); + schemas.getSchemaInfo(topicName.getSchemaName()), tupleDomain, null, null, null); if (topicsToNumEntries.get(topicName.getSchemaName()) > 1) { Assert.assertEquals(splits.size(), 4); } @@ -454,7 +454,7 @@ public void pulsarSplitJsonCodecTest() throws JsonProcessingException, Unsupport PulsarSplit pulsarSplit = new PulsarSplit( splitId, connectorId, schemaName, originSchemaName, tableName, splitSize, schema, schemaType, startPositionEntryId, endPositionEntryId, startPositionLedgerId, - endPositionLedgerId, tupleDomain, schemaInfoProperties, offloadPolicies); + endPositionLedgerId, tupleDomain, schemaInfoProperties, offloadPolicies, null); pulsarSplitData = jsonCodec.toJsonBytes(pulsarSplit); } catch (Exception e) { e.printStackTrace(); diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestReadChunkedMessages.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestReadChunkedMessages.java index 1e959e9b83059..662811a06094b 100644 --- a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestReadChunkedMessages.java +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestReadChunkedMessages.java @@ -115,6 +115,8 @@ public void queryTest() throws Exception { schemaInfo, topic, TupleDomain.all(), + null, + null, null); List columnHandleList = TestPulsarConnector.getColumnColumnHandles( From a0a1264f1211dd5ecaf419bf8ef959ade9be40fe Mon Sep 17 00:00:00 2001 From: rangao Date: Fri, 10 Feb 2023 11:23:48 +0800 Subject: [PATCH 02/15] fix test, check style --- .../sql/presto/PulsarConnectorCache.java | 10 ++- .../sql/presto/PulsarInternalColumn.java | 1 - .../pulsar/sql/presto/PulsarRecordCursor.java | 7 +-- .../pulsar/sql/presto/PulsarSplitManager.java | 61 ++++++++----------- .../presto/PulsarSqlSchemaInfoProvider.java | 1 - .../sql/presto/util/ReadCompactedType.java | 19 +++++- .../org/apache/pulsar/sql/presto/Demo.java | 21 ++++++- .../pulsar/sql/presto/TestCompactedQuery.java | 21 ++++++- .../sql/presto/TestPulsarSplitManager.java | 24 ++++---- .../sql/presto/TestReadChunkedMessages.java | 1 - 10 files changed, 102 insertions(+), 64 deletions(-) diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarConnectorCache.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarConnectorCache.java index 0b91ba83616bf..84020f3cf8a95 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarConnectorCache.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarConnectorCache.java @@ -26,12 +26,9 @@ import java.util.Map; import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; - import lombok.Getter; -import org.apache.bookkeeper.client.AsyncCallback; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper; -import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.common.util.OrderedScheduler; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.mledger.LedgerOffloader; @@ -44,7 +41,6 @@ import org.apache.bookkeeper.mledger.impl.NullLedgerOffloader; import org.apache.bookkeeper.mledger.offload.Offloaders; import org.apache.bookkeeper.mledger.offload.OffloadersCache; -import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsProvider; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.PulsarVersion; @@ -83,7 +79,8 @@ public class PulsarConnectorCache { private static final String MANAGED_LEDGER_OFFLOAD_DRIVER = "managedLedgerOffloadDriver"; private static final String MANAGED_LEDGER_OFFLOAD_MAX_THREADS = "managedLedgerOffloadMaxThreads"; - static class DefaultBkFactory implements ManagedLedgerFactoryImpl.BookkeeperFactoryForCustomEnsemblePlacementPolicy { + static class DefaultBkFactory implements + ManagedLedgerFactoryImpl.BookkeeperFactoryForCustomEnsemblePlacementPolicy { private final BookKeeper bkClient; @@ -170,7 +167,8 @@ private BookKeeper initBookKeeper(PulsarConnectorConfig pulsarConnectorConfig) t .setThrottleValue(pulsarConnectorConfig.getBookkeeperThrottleValue()) .setNumIOThreads(pulsarConnectorConfig.getBookkeeperNumIOThreads()) .setNumWorkerThreads(pulsarConnectorConfig.getBookkeeperNumWorkerThreads()) - .setNettyMaxFrameSizeBytes(pulsarConnectorConfig.getMaxMessageSize() + Commands.MESSAGE_SIZE_FRAME_PADDING); + .setNettyMaxFrameSizeBytes( + pulsarConnectorConfig.getMaxMessageSize() + Commands.MESSAGE_SIZE_FRAME_PADDING); return new BookKeeper(bkClientConfiguration); } diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarInternalColumn.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarInternalColumn.java index 46f93165e537a..916f29e4d77c9 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarInternalColumn.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarInternalColumn.java @@ -24,7 +24,6 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import io.trino.spi.type.BigintType; -import io.trino.spi.type.BooleanType; import io.trino.spi.type.IntegerType; import io.trino.spi.type.TimestampType; import io.trino.spi.type.Type; diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java index e520c9f681db5..eaa1ec8e2d51a 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java @@ -50,7 +50,6 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; - import lombok.Getter; import org.apache.bookkeeper.client.AsyncCallback; import org.apache.bookkeeper.client.BookKeeper; @@ -363,9 +362,8 @@ public void accept(Entry entry) { if (readCompactedType != null && message.getKey().isPresent()) { RawMessageIdImpl messageId = (RawMessageIdImpl) message.getMessageId(); - System.out.println("xxxx parse message with id " + messageId); if (message.getKey().equals("stock-7")) { - System.out.println(""); + System.out.println("stock-7"); } compactMessage(message.getKey().get(), new BatchMessageIdImpl( @@ -1164,7 +1162,8 @@ private void readOriginalData() { } } - private void readEntries(LedgerHandle ledgerHandle, long ledgerId, long entryId, int batchIndex, boolean isCompactedLedger) { + private void readEntries(LedgerHandle ledgerHandle, long ledgerId, long entryId, int batchIndex, + boolean isCompactedLedger) { try { LedgerEntries ledgerEntries = ledgerHandle.read(entryId, entryId); diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplitManager.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplitManager.java index 68153e76493d9..84b4ecc229502 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplitManager.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplitManager.java @@ -51,6 +51,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.ManagedLedgerFactory; +import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.ReadOnlyCursor; import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.lang3.exception.ExceptionUtils; @@ -61,7 +62,6 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.OffloadPoliciesImpl; import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; -import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.schema.SchemaInfo; import org.apache.pulsar.sql.presto.util.ReadCompactedType; @@ -134,25 +134,6 @@ public ConnectorSplitSource getSplits(ConnectorTransactionHandle transactionHand }); ReadCompactedType readCompactedType = readCompactedTypeReference.get(); - long compactedLedgerId; - PositionImpl compactedHorizon = PositionImpl.EARLIEST; - List leftLedgers = new ArrayList<>(); - if (readCompactedType != null) { - try { - String topic = String.format("%s/%s", namespace, tableHandle.getTopicName()); - PersistentTopicInternalStats internalStats = pulsarAdmin.topics().getInternalStats(topic); - compactedLedgerId = internalStats.compactedLedger.ledgerId; - if (internalStats.cursors.containsKey("Compaction")) { - String[] compactedHorizonArr = internalStats.cursors.get("Compaction").markDeletePosition.split(":"); - compactedHorizon = PositionImpl.get( - Long.parseLong(compactedHorizonArr[0]), Long.parseLong(compactedHorizonArr[1])); - } - TopicStats stats = pulsarAdmin.topics().getStats(topic); - } catch (PulsarAdminException e) { - throw new RuntimeException(e); - } - } - Collection splits; try { OffloadPoliciesImpl offloadPolicies = (OffloadPoliciesImpl) this.pulsarAdmin.namespaces() @@ -164,11 +145,11 @@ public ConnectorSplitSource getSplits(ConnectorTransactionHandle transactionHand } if (!PulsarConnectorUtils.isPartitionedTopic(topicName, this.pulsarAdmin)) { splits = getSplitsNonPartitionedTopic( - numSplits, topicName, tableHandle, schemaInfo, tupleDomain, offloadPolicies, readCompactedType, compactedHorizon); + numSplits, topicName, tableHandle, schemaInfo, tupleDomain, offloadPolicies, readCompactedType); log.debug("Splits for non-partitioned topic %s: %s", topicName, splits); } else { splits = getSplitsPartitionedTopic( - numSplits, topicName, tableHandle, schemaInfo, tupleDomain, offloadPolicies, readCompactedType, compactedHorizon); + numSplits, topicName, tableHandle, schemaInfo, tupleDomain, offloadPolicies, readCompactedType); log.debug("Splits for partitioned topic %s: %s", topicName, splits); } } catch (Exception e) { @@ -181,8 +162,7 @@ public ConnectorSplitSource getSplits(ConnectorTransactionHandle transactionHand @VisibleForTesting Collection getSplitsPartitionedTopic(int numSplits, TopicName topicName, PulsarTableHandle tableHandle, SchemaInfo schemaInfo, TupleDomain tupleDomain, - OffloadPoliciesImpl offloadPolicies, ReadCompactedType readCompactedType, - PositionImpl compactedHorizon) throws Exception { + OffloadPoliciesImpl offloadPolicies, ReadCompactedType readCompactedType) throws Exception { List predicatedPartitions = getPredicatedPartitions(topicName, tupleDomain); if (log.isDebugEnabled()) { @@ -195,6 +175,11 @@ Collection getSplitsPartitionedTopic(int numSplits, TopicName topic int splitRemainder = actualNumSplits % predicatedPartitions.size(); + if (readCompactedType != null) { + splitsPerPartition = 1; + splitRemainder = 0; + } + PulsarConnectorCache pulsarConnectorCache = PulsarConnectorCache.getConnectorCache(pulsarConnectorConfig); ManagedLedgerFactory managedLedgerFactory = pulsarConnectorCache.getManagedLedgerFactory(); ManagedLedgerConfig managedLedgerConfig = pulsarConnectorCache.getManagedLedgerConfig( @@ -214,8 +199,7 @@ Collection getSplitsPartitionedTopic(int numSplits, TopicName topic topicName.getPartition(predicatedPartitions.get(i)).getLocalName(), tupleDomain, offloadPolicies, - readCompactedType, - compactedHorizon)); + readCompactedType)); } return splits; } @@ -272,8 +256,7 @@ private List getPredicatedPartitions(TopicName topicName, TupleDomain getSplitsNonPartitionedTopic(int numSplits, TopicName topicName, PulsarTableHandle tableHandle, SchemaInfo schemaInfo, TupleDomain tupleDomain, - OffloadPoliciesImpl offloadPolicies, ReadCompactedType readCompactedType, - PositionImpl compactedHorizon) throws Exception { + OffloadPoliciesImpl offloadPolicies, ReadCompactedType readCompactedType) throws Exception { PulsarConnectorCache pulsarConnectorCache = PulsarConnectorCache.getConnectorCache(pulsarConnectorConfig); ManagedLedgerFactory managedLedgerFactory = pulsarConnectorCache.getManagedLedgerFactory(); ManagedLedgerConfig managedLedgerConfig = pulsarConnectorCache.getManagedLedgerConfig( @@ -289,8 +272,7 @@ Collection getSplitsNonPartitionedTopic(int numSplits, TopicName to topicName.getLocalName(), tupleDomain, offloadPolicies, - readCompactedType, - compactedHorizon); + readCompactedType); } @VisibleForTesting @@ -302,15 +284,26 @@ Collection getSplitsForTopic(String topicNamePersistenceEncoding, SchemaInfo schemaInfo, String tableName, TupleDomain tupleDomain, OffloadPoliciesImpl offloadPolicies, - ReadCompactedType readCompactedType, - PositionImpl compactedHorizon) - throws ManagedLedgerException, InterruptedException, IOException { + ReadCompactedType readCompactedType) + throws ManagedLedgerException, InterruptedException, IOException, PulsarAdminException { ReadOnlyCursor readOnlyCursor = null; try { + Position cursorStartPos = PositionImpl.EARLIEST; + if (readCompactedType != null) { + PersistentTopicInternalStats internalStats = pulsarAdmin.topics() + .getInternalStats(topicNamePersistenceEncoding); + if (internalStats.cursors.containsKey("Compaction")) { + String[] compactedHorizonArr = + internalStats.cursors.get("Compaction").markDeletePosition.split(":"); + cursorStartPos = PositionImpl.get( + Long.parseLong(compactedHorizonArr[0]), Long.parseLong(compactedHorizonArr[1])); + } + } + readOnlyCursor = managedLedgerFactory.openReadOnlyCursor( topicNamePersistenceEncoding, - compactedHorizon, managedLedgerConfig); + cursorStartPos, managedLedgerConfig); long numEntries = readOnlyCursor.getNumberOfEntries(); if (numEntries <= 0) { diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSqlSchemaInfoProvider.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSqlSchemaInfoProvider.java index 97a567290af31..90271f7870ab0 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSqlSchemaInfoProvider.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSqlSchemaInfoProvider.java @@ -27,7 +27,6 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; - import lombok.Getter; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/util/ReadCompactedType.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/util/ReadCompactedType.java index 668038cb2af00..853574054c76b 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/util/ReadCompactedType.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/util/ReadCompactedType.java @@ -1,6 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package org.apache.pulsar.sql.presto.util; -import org.apache.commons.lang3.StringUtils; public enum ReadCompactedType { diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/Demo.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/Demo.java index 6d1a7553ff53f..1aac760d8a944 100644 --- a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/Demo.java +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/Demo.java @@ -1,5 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package org.apache.pulsar.sql.presto; +import java.util.concurrent.TimeUnit; import lombok.Data; import lombok.NoArgsConstructor; import lombok.ToString; @@ -9,8 +28,6 @@ import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.Schema; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; public class Demo { diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestCompactedQuery.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestCompactedQuery.java index 274cdf4366c7b..0fbb73adbba45 100644 --- a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestCompactedQuery.java +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestCompactedQuery.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package org.apache.pulsar.sql.presto; import static org.testng.AssertJUnit.assertEquals; @@ -48,7 +66,6 @@ import org.testng.annotations.DataProvider; import org.testng.annotations.Test; - @Slf4j public class TestCompactedQuery extends MockedPulsarServiceBaseTest { @@ -210,7 +227,7 @@ private void compactQuery(boolean enableBatch, int compactedMsgNum, int unCompac pulsarColumnHandles, pulsarSplit, connectorConfig, pulsar.getManagedLedgerFactory(), new ManagedLedgerConfig(), new PulsarConnectorMetricsTracker(new NullStatsProvider()), new PulsarDispatchingRowDecoderFactory(prestoConnectorContext.getTypeManager()), - this.mockBookKeeper); + this.pulsarTestContext.getMockBookKeeper()); List columns = pulsarRecordCursor.getColumnHandles(); while (pulsarRecordCursor.advanceNextPosition()) { diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarSplitManager.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarSplitManager.java index 09233a919f9ea..1dbdc86d6f583 100644 --- a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarSplitManager.java +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarSplitManager.java @@ -96,7 +96,7 @@ public void testTopic(String delimiter) throws Exception { PulsarTableLayoutHandle pulsarTableLayoutHandle = new PulsarTableLayoutHandle(pulsarTableHandle, TupleDomain.all()); final ResultCaptor> resultCaptor = new ResultCaptor<>(); - doAnswer(resultCaptor).when(this.pulsarSplitManager).getSplitsNonPartitionedTopic(anyInt(), any(), any(), any(), any(), any(), any(), any()); + doAnswer(resultCaptor).when(this.pulsarSplitManager).getSplitsNonPartitionedTopic(anyInt(), any(), any(), any(), any(), any(), any()); ConnectorSplitSource connectorSplitSource = this.pulsarSplitManager.getSplits( @@ -104,7 +104,7 @@ public void testTopic(String delimiter) throws Exception { pulsarTableLayoutHandle, null); verify(this.pulsarSplitManager, times(1)) - .getSplitsNonPartitionedTopic(anyInt(), any(), any(), any(), any(), any(), any(), any()); + .getSplitsNonPartitionedTopic(anyInt(), any(), any(), any(), any(), any(), any()); int totalSize = 0; for (PulsarSplit pulsarSplit : resultCaptor.getResult()) { @@ -143,13 +143,13 @@ public void testPartitionedTopic(String delimiter) throws Exception { PulsarTableLayoutHandle pulsarTableLayoutHandle = new PulsarTableLayoutHandle(pulsarTableHandle, TupleDomain.all()); final ResultCaptor> resultCaptor = new ResultCaptor<>(); - doAnswer(resultCaptor).when(this.pulsarSplitManager).getSplitsPartitionedTopic(anyInt(), any(), any(), any(), any(), any(), any(), any()); + doAnswer(resultCaptor).when(this.pulsarSplitManager).getSplitsPartitionedTopic(anyInt(), any(), any(), any(), any(), any(), any()); this.pulsarSplitManager.getSplits(mock(ConnectorTransactionHandle.class), mock(ConnectorSession.class), pulsarTableLayoutHandle, null); verify(this.pulsarSplitManager, times(1)) - .getSplitsPartitionedTopic(anyInt(), any(), any(), any(), any(), any(), any(), any()); + .getSplitsPartitionedTopic(anyInt(), any(), any(), any(), any(), any(), any()); int partitions = partitionedTopicsToPartitions.get(topicName.toString()); @@ -210,7 +210,7 @@ public void testPublishTimePredicatePushdown(String delimiter) throws Exception final ResultCaptor> resultCaptor = new ResultCaptor<>(); doAnswer(resultCaptor).when(this.pulsarSplitManager) - .getSplitsNonPartitionedTopic(anyInt(), any(), any(), any(), any(), any(), any(), any()); + .getSplitsNonPartitionedTopic(anyInt(), any(), any(), any(), any(), any(), any()); ConnectorSplitSource connectorSplitSource = this.pulsarSplitManager.getSplits( mock(ConnectorTransactionHandle.class), mock(ConnectorSession.class), @@ -218,7 +218,7 @@ public void testPublishTimePredicatePushdown(String delimiter) throws Exception verify(this.pulsarSplitManager, times(1)) - .getSplitsNonPartitionedTopic(anyInt(), any(), any(), any(), any(), any(), any(), any()); + .getSplitsNonPartitionedTopic(anyInt(), any(), any(), any(), any(), any(), any()); int totalSize = 0; int initalStart = 1; @@ -267,7 +267,7 @@ public void testPublishTimePredicatePushdownPartitionedTopic(String delimiter) t final ResultCaptor> resultCaptor = new ResultCaptor<>(); doAnswer(resultCaptor).when(this.pulsarSplitManager) - .getSplitsPartitionedTopic(anyInt(), any(), any(), any(), any(), any(), any(), any()); + .getSplitsPartitionedTopic(anyInt(), any(), any(), any(), any(), any(), any()); ConnectorSplitSource connectorSplitSource = this.pulsarSplitManager.getSplits( mock(ConnectorTransactionHandle.class), mock(ConnectorSession.class), @@ -275,7 +275,7 @@ public void testPublishTimePredicatePushdownPartitionedTopic(String delimiter) t verify(this.pulsarSplitManager, times(1)) - .getSplitsPartitionedTopic(anyInt(), any(), any(), any(), any(), any(), any(), any()); + .getSplitsPartitionedTopic(anyInt(), any(), any(), any(), any(), any(), any()); int partitions = partitionedTopicsToPartitions.get(topicName.toString()); @@ -324,7 +324,7 @@ public void testPartitionFilter(String delimiter) throws Exception { domainMap.put(PulsarInternalColumn.PARTITION.getColumnHandle(pulsarConnectorId.toString(), false), domain); TupleDomain tupleDomain = TupleDomain.withColumnDomains(domainMap); Collection splits = this.pulsarSplitManager.getSplitsPartitionedTopic(2, topicName, pulsarTableHandle, - schemas.getSchemaInfo(topicName.getSchemaName()), tupleDomain, null, null, null); + schemas.getSchemaInfo(topicName.getSchemaName()), tupleDomain, null, null); if (topicsToNumEntries.get(topicName.getSchemaName()) > 1) { Assert.assertEquals(splits.size(), 2); } @@ -341,7 +341,7 @@ public void testPartitionFilter(String delimiter) throws Exception { domainMap.put(PulsarInternalColumn.PARTITION.getColumnHandle(pulsarConnectorId.toString(), false), domain); tupleDomain = TupleDomain.withColumnDomains(domainMap); splits = this.pulsarSplitManager.getSplitsPartitionedTopic(1, topicName, pulsarTableHandle, - schemas.getSchemaInfo(topicName.getSchemaName()), tupleDomain, null, null, null); + schemas.getSchemaInfo(topicName.getSchemaName()), tupleDomain, null, null); if (topicsToNumEntries.get(topicName.getSchemaName()) > 1) { Assert.assertEquals(splits.size(), 2); } @@ -357,7 +357,7 @@ public void testPartitionFilter(String delimiter) throws Exception { domainMap.put(PulsarInternalColumn.PARTITION.getColumnHandle(pulsarConnectorId.toString(), false), domain); tupleDomain = TupleDomain.withColumnDomains(domainMap); splits = this.pulsarSplitManager.getSplitsPartitionedTopic(2, topicName, pulsarTableHandle, - schemas.getSchemaInfo(topicName.getSchemaName()), tupleDomain, null, null, null); + schemas.getSchemaInfo(topicName.getSchemaName()), tupleDomain, null, null); if (topicsToNumEntries.get(topicName.getSchemaName()) > 1) { Assert.assertEquals(splits.size(), 3); } @@ -376,7 +376,7 @@ public void testPartitionFilter(String delimiter) throws Exception { domainMap.put(PulsarInternalColumn.PARTITION.getColumnHandle(pulsarConnectorId.toString(), false), domain); tupleDomain = TupleDomain.withColumnDomains(domainMap); splits = this.pulsarSplitManager.getSplitsPartitionedTopic(2, topicName, pulsarTableHandle, - schemas.getSchemaInfo(topicName.getSchemaName()), tupleDomain, null, null, null); + schemas.getSchemaInfo(topicName.getSchemaName()), tupleDomain, null, null); if (topicsToNumEntries.get(topicName.getSchemaName()) > 1) { Assert.assertEquals(splits.size(), 4); } diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestReadChunkedMessages.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestReadChunkedMessages.java index 662811a06094b..f9595ade1b8c1 100644 --- a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestReadChunkedMessages.java +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestReadChunkedMessages.java @@ -116,7 +116,6 @@ public void queryTest() throws Exception { topic, TupleDomain.all(), null, - null, null); List columnHandleList = TestPulsarConnector.getColumnColumnHandles( From 8bd0f6890017f5a173d40e35be74f934289ec638 Mon Sep 17 00:00:00 2001 From: rangao Date: Tue, 14 Feb 2023 10:37:39 +0800 Subject: [PATCH 03/15] add integration test --- .../sql/presto/PulsarInternalColumn.java | 7 +- .../pulsar/sql/presto/PulsarRecordCursor.java | 112 +++++++++--------- .../apache/pulsar/sql/presto/PulsarSplit.java | 13 +- .../pulsar/sql/presto/PulsarSplitManager.java | 47 ++++---- .../org/apache/pulsar/sql/presto/Demo.java | 31 ++++- .../pulsar/sql/presto/TestCompactedQuery.java | 26 ++-- .../sql/presto/TestPulsarSplitManager.java | 9 +- .../sql/presto/TestReadChunkedMessages.java | 4 +- .../integration/presto/TestBasicPresto.java | 83 +++++++++++++ 9 files changed, 225 insertions(+), 107 deletions(-) diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarInternalColumn.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarInternalColumn.java index 916f29e4d77c9..f33eb9be5f8fd 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarInternalColumn.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarInternalColumn.java @@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import io.trino.spi.type.BigintType; +import io.trino.spi.type.BooleanType; import io.trino.spi.type.IntegerType; import io.trino.spi.type.TimestampType; import io.trino.spi.type.Type; @@ -62,11 +63,11 @@ public class PulsarInternalColumn { public static final PulsarInternalColumn PROPERTIES = new PulsarInternalColumn("__properties__", VarcharType.VARCHAR, "User defined properties"); - public static final PulsarInternalColumn READ_COMPACTED = new PulsarInternalColumn("__read_compacted__", - VarcharType.VARCHAR, "Read compacted flag"); + public static final PulsarInternalColumn COMPACTED_QUERY = new PulsarInternalColumn("__compacted_query__", + BooleanType.BOOLEAN, "Compacted query flag"); private static Set internalFields = ImmutableSet.of(PARTITION, EVENT_TIME, PUBLISH_TIME, - MESSAGE_ID, SEQUENCE_ID, PRODUCER_NAME, KEY, PROPERTIES); + MESSAGE_ID, SEQUENCE_ID, PRODUCER_NAME, KEY, PROPERTIES, COMPACTED_QUERY); private final String name; diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java index eaa1ec8e2d51a..6cb1b6580fd29 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java @@ -20,8 +20,10 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.collect.ImmutableSet.toImmutableSet; +import static io.trino.decoder.FieldValueProviders.booleanValueProvider; import static io.trino.decoder.FieldValueProviders.bytesValueProvider; import static io.trino.decoder.FieldValueProviders.longValueProvider; +import static org.apache.pulsar.sql.presto.PulsarInternalColumn.COMPACTED_QUERY; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; @@ -94,7 +96,6 @@ import org.apache.pulsar.sql.presto.util.CacheSizeAllocator; import org.apache.pulsar.sql.presto.util.NoStrictCacheSizeAllocator; import org.apache.pulsar.sql.presto.util.NullCacheSizeAllocator; -import org.apache.pulsar.sql.presto.util.ReadCompactedType; import org.jctools.queues.MessagePassingQueue; import org.jctools.queues.SpscArrayQueue; @@ -146,9 +147,8 @@ public class PulsarRecordCursor implements RecordCursor { private PulsarConnectorCache pulsarConnectorCache; private OffloadPoliciesImpl offloadPolicies; private volatile Throwable compactedHandleError; - private final ReadCompactedType readCompactedType; + private final boolean compactedQuery; private long compactedLedgerId = -1; -// private Position compactedHorizon; protected ConcurrentOpenHashMap compactedMessage = ConcurrentOpenHashMap.newBuilder().build(); private LedgerHandle compactedLedgerHandle; @@ -172,19 +172,23 @@ public PulsarRecordCursor(List columnHandles, PulsarSplit pu throw new RuntimeException(e); } + bookKeeper = pulsarConnectorCache.getBookKeeper(); + managedLedgerFactory = pulsarConnectorCache.getManagedLedgerFactory(); + managedLedgerConfig = pulsarConnectorCache.getManagedLedgerConfig( + TopicName.get("persistent", NamespaceName.get(pulsarSplit.getSchemaName()), + pulsarSplit.getTableName()).getNamespaceObject(), offloadPolicies, + pulsarConnectorConfig); + offloadPolicies = pulsarSplit.getOffloadPolicies(); if (offloadPolicies != null) { offloadPolicies.setOffloadersDirectory(pulsarConnectorConfig.getOffloadersDirectory()); offloadPolicies.setManagedLedgerOffloadMaxThreads( pulsarConnectorConfig.getManagedLedgerOffloadMaxThreads()); } - this.readCompactedType = pulsarSplit.getReadCompactedType(); + this.compactedQuery = pulsarSplit.getCompactedQuery(); initialize(columnHandles, pulsarSplit, pulsarConnectorConfig, - pulsarConnectorCache.getManagedLedgerFactory(), - pulsarConnectorCache.getManagedLedgerConfig( - TopicName.get("persistent", NamespaceName.get(pulsarSplit.getSchemaName()), - pulsarSplit.getTableName()).getNamespaceObject(), offloadPolicies, - pulsarConnectorConfig), + managedLedgerFactory, + managedLedgerConfig, new PulsarConnectorMetricsTracker(pulsarConnectorCache.getStatsProvider())); this.decoderFactory = decoderFactory; } @@ -195,7 +199,7 @@ public PulsarRecordCursor(List columnHandles, PulsarSplit pu PulsarConnectorMetricsTracker pulsarConnectorMetricsTracker, PulsarDispatchingRowDecoderFactory decoderFactory) { this.splitSize = pulsarSplit.getSplitSize(); - this.readCompactedType = pulsarSplit.getReadCompactedType(); + this.compactedQuery = pulsarSplit.getCompactedQuery(); initialize(columnHandles, pulsarSplit, pulsarConnectorConfig, managedLedgerFactory, managedLedgerConfig, pulsarConnectorMetricsTracker); this.decoderFactory = decoderFactory; @@ -206,7 +210,7 @@ public PulsarRecordCursor(List columnHandles, PulsarSplit pu PulsarConnectorMetricsTracker pulsarConnectorMetricsTracker, PulsarDispatchingRowDecoderFactory decoderFactory, BookKeeper bookKeeper) { this.splitSize = pulsarSplit.getSplitSize(); - this.readCompactedType = pulsarSplit.getReadCompactedType(); + this.compactedQuery = pulsarSplit.getCompactedQuery(); this.bookKeeper = bookKeeper; this.managedLedgerFactory = managedLedgerFactory; this.managedLedgerConfig = managedLedgerConfig; @@ -245,7 +249,7 @@ private void initialize(List columnHandles, PulsarSplit puls log.info("Initializing split with parameters: %s", pulsarSplit); try { - if (readCompactedType != null) { + if (compactedQuery) { initCompactedRead(); } else { this.cursor = getCursor(topicName, @@ -324,8 +328,6 @@ public void run() { @Override public void accept(Entry entry) { - System.out.println("xxxx deserialize entry " - + entry.getLedgerId() + ":" + entry.getEntryId() + ", size: " + entry.getLength()); try { entryQueueCacheSizeAllocator.release(entry.getLength()); @@ -359,18 +361,16 @@ public void accept(Entry entry) { message = null; } if (message != null) { - if (readCompactedType != null && message.getKey().isPresent()) { + if (compactedQuery && message.getKey().isPresent()) { RawMessageIdImpl messageId = (RawMessageIdImpl) message.getMessageId(); - if (message.getKey().equals("stock-7")) { - System.out.println("stock-7"); - } compactMessage(message.getKey().get(), new BatchMessageIdImpl( messageId.getLedgerId(), messageId.getEntryId(), partition, - (int) messageId.getBatchIndex())); + (int) messageId.getBatchIndex()), + message.getData().readableBytes()); } else { while (true) { if (!haveAvailableCacheSize( @@ -548,7 +548,7 @@ private boolean haveAvailableCacheSize(CacheSizeAllocator cacheSizeAllocator, Sp public boolean advanceNextPosition() { if (readEntries == null) { - if (readCompactedType != null) { + if (compactedQuery) { Thread compactedHandleThread = new Thread(new CompactedLedgerReader()); compactedHandleThread.setUncaughtExceptionHandler((t, ex) -> { compactedHandleError = ex; @@ -594,7 +594,7 @@ public boolean advanceNextPosition() { } else { try { long waitMills = 1; - if (readCompactedType != null) { + if (compactedQuery) { waitMills = 10; } Thread.sleep(waitMills); @@ -606,6 +606,7 @@ public boolean advanceNextPosition() { } } + log.info("deserialize message " + currentMessage.getMessageId()); //start time for deserializing record metricsTracker.start_RECORD_DESERIALIZE_TIME(); @@ -672,9 +673,6 @@ public boolean advanceNextPosition() { .filter(col -> PulsarColumnHandle.HandleKeyValueType.NONE .equals(col.getHandleKeyValueType())) .collect(toImmutableSet())); - if (this.currentMessage.getData().readableBytes() == 0) { - System.out.println(""); - } Optional> decodedValue = messageDecoder.decodeRow(this.currentMessage.getData()); decodedValue.ifPresent(currentRowValuesMap::putAll); @@ -708,6 +706,8 @@ public boolean advanceNextPosition() { } catch (JsonProcessingException e) { throw new RuntimeException(e); } + } else if (COMPACTED_QUERY.getName().equals(columnHandle.getName())) { + currentRowValuesMap.put(columnHandle, booleanValueProvider(compactedQuery)); } else { throw new IllegalArgumentException("unknown internal field " + columnHandle.getName()); } @@ -971,22 +971,20 @@ private void initCompactedRead() throws Exception { return; } this.compactedLedgerId = internalStats.compactedLedger.ledgerId; -// if (internalStats.cursors.containsKey("Compaction")) { -// String compactedHorizonStr = internalStats.cursors.get("Compaction").markDeletePosition; -// String[] arr = compactedHorizonStr.split(":"); -// compactedHorizon = PositionImpl.get(Long.parseLong(arr[0]), Long.parseLong(arr[1])); -// } this.compactedLedgerHandle = bookKeeper.openLedger(compactedLedgerId, BookKeeper.DigestType.fromApiDigestType(DigestType.CRC32), "".getBytes()); } - private void compactMessage(String key, BatchMessageIdImpl messageId) { + private void compactMessage(String key, BatchMessageIdImpl messageId, long payloadSize) { if (key == null) { return; } - switch (readCompactedType) { - case COMPACTED_LATEST -> compactedMessage.put(key, messageId); - case COMPACTED_EARLIEST -> compactedMessage.computeIfAbsent(key, __ -> messageId); + if (compactedQuery) { + if (payloadSize > 0) { + compactedMessage.put(key, messageId); + } else { + compactedMessage.remove(key); + } } } @@ -1023,7 +1021,6 @@ public void readComplete(int i, LedgerHandle ledgerHandle, Enumeration { - if (((RawMessageIdImpl) message.getMessageId()).getBatchIndex() == batchIndex) { - messageQueue.offer(message); - if (message.getKey().isPresent()) { - compactedMessage.remove(message.getKey().get()); - } - } - }, pulsarConnectorConfig.getMaxMessageSize()); + if (((RawMessageIdImpl) message.getMessageId()).getBatchIndex() == batchIndex) { + messageQueue.offer(message); + if (message.getKey().isPresent()) { + compactedMessage.remove(message.getKey().get()); + } + } + }, pulsarConnectorConfig.getMaxMessageSize()); } } diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplit.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplit.java index e9d4d30df4675..6ee4c105d08b5 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplit.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplit.java @@ -35,7 +35,6 @@ import org.apache.pulsar.common.policies.data.OffloadPoliciesImpl; import org.apache.pulsar.common.schema.SchemaInfo; import org.apache.pulsar.common.schema.SchemaType; -import org.apache.pulsar.sql.presto.util.ReadCompactedType; /** * This class represents information for a split. @@ -64,7 +63,7 @@ public class PulsarSplit implements ConnectorSplit { private final String schemaInfoProperties; private final OffloadPoliciesImpl offloadPolicies; - private final ReadCompactedType readCompactedType; + private final Boolean compactedQuery; @JsonCreator public PulsarSplit( @@ -83,7 +82,7 @@ public PulsarSplit( @JsonProperty("tupleDomain") TupleDomain tupleDomain, @JsonProperty("schemaInfoProperties") String schemaInfoProperties, @JsonProperty("offloadPolicies") OffloadPoliciesImpl offloadPolicies, - @JsonProperty("readCompactedType") ReadCompactedType readCompactedType) throws IOException { + @JsonProperty("compactedQuery") Boolean compactedQuery) throws IOException { this.splitId = splitId; requireNonNull(schemaName, "schema name is null"); this.originSchemaName = originSchemaName; @@ -111,7 +110,7 @@ public PulsarSplit( .properties(objectMapper.readValue(schemaInfoProperties, Map.class)) .build(); - this.readCompactedType = readCompactedType; + this.compactedQuery = compactedQuery; } @JsonProperty @@ -198,8 +197,8 @@ public OffloadPoliciesImpl getOffloadPolicies() { } @JsonProperty - public ReadCompactedType getReadCompactedType() { - return readCompactedType; + public Boolean getCompactedQuery() { + return compactedQuery; } @Override @@ -233,7 +232,7 @@ public String toString() { + ", startPositionLedgerId=" + startPositionLedgerId + ", endPositionLedgerId=" + endPositionLedgerId + ", schemaInfoProperties=" + schemaInfoProperties - + ", readCompactedType=" + readCompactedType + + ", compactedQuery=" + compactedQuery + (offloadPolicies == null ? "" : offloadPolicies.toString()) + '}'; } diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplitManager.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplitManager.java index 84b4ecc229502..cc0c1c67c5429 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplitManager.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplitManager.java @@ -64,7 +64,6 @@ import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.schema.SchemaInfo; -import org.apache.pulsar.sql.presto.util.ReadCompactedType; /** * The class helping to manage splits. @@ -126,13 +125,19 @@ public ConnectorSplitSource getSplits(ConnectorTransactionHandle transactionHand } } - AtomicReference readCompactedTypeReference = new AtomicReference<>(); - tupleDomain.getDomains().ifPresent(__ -> { - Domain readCompactedDomain = tupleDomain.getDomains().get() - .get(PulsarInternalColumn.READ_COMPACTED.getColumnHandle(connectorId, true)); - readCompactedTypeReference.set(ReadCompactedType.valueOf(readCompactedDomain.getSingleValue().toString())); + AtomicReference readCompactedTypeReference = new AtomicReference<>(false); + tupleDomain.getDomains().ifPresent(domainMap -> { + log.info("tuple domain map " + domainMap); + Domain readCompactedDomain = domainMap.get( + PulsarInternalColumn.COMPACTED_QUERY.getColumnHandle(connectorId, false)); + log.info("tuple readCompactedDomain " + readCompactedDomain); + if (readCompactedDomain != null) { + log.info("tuple readCompactedDomain value " + readCompactedDomain.getSingleValue()); + readCompactedTypeReference.set((Boolean) readCompactedDomain.getSingleValue()); + } + }); - ReadCompactedType readCompactedType = readCompactedTypeReference.get(); + boolean readCompactedType = readCompactedTypeReference.get(); Collection splits; try { @@ -162,7 +167,7 @@ public ConnectorSplitSource getSplits(ConnectorTransactionHandle transactionHand @VisibleForTesting Collection getSplitsPartitionedTopic(int numSplits, TopicName topicName, PulsarTableHandle tableHandle, SchemaInfo schemaInfo, TupleDomain tupleDomain, - OffloadPoliciesImpl offloadPolicies, ReadCompactedType readCompactedType) throws Exception { + OffloadPoliciesImpl offloadPolicies, boolean readCompacted) throws Exception { List predicatedPartitions = getPredicatedPartitions(topicName, tupleDomain); if (log.isDebugEnabled()) { @@ -175,7 +180,7 @@ Collection getSplitsPartitionedTopic(int numSplits, TopicName topic int splitRemainder = actualNumSplits % predicatedPartitions.size(); - if (readCompactedType != null) { + if (readCompacted) { splitsPerPartition = 1; splitRemainder = 0; } @@ -190,7 +195,7 @@ Collection getSplitsPartitionedTopic(int numSplits, TopicName topic int splitsForThisPartition = (splitRemainder > i) ? splitsPerPartition + 1 : splitsPerPartition; splits.addAll( getSplitsForTopic( - topicName.getPartition(predicatedPartitions.get(i)).getPersistenceNamingEncoding(), + topicName.getPartition(predicatedPartitions.get(i)), managedLedgerFactory, managedLedgerConfig, splitsForThisPartition, @@ -199,7 +204,7 @@ Collection getSplitsPartitionedTopic(int numSplits, TopicName topic topicName.getPartition(predicatedPartitions.get(i)).getLocalName(), tupleDomain, offloadPolicies, - readCompactedType)); + readCompacted)); } return splits; } @@ -256,14 +261,14 @@ private List getPredicatedPartitions(TopicName topicName, TupleDomain getSplitsNonPartitionedTopic(int numSplits, TopicName topicName, PulsarTableHandle tableHandle, SchemaInfo schemaInfo, TupleDomain tupleDomain, - OffloadPoliciesImpl offloadPolicies, ReadCompactedType readCompactedType) throws Exception { + OffloadPoliciesImpl offloadPolicies, boolean readCompacted) throws Exception { PulsarConnectorCache pulsarConnectorCache = PulsarConnectorCache.getConnectorCache(pulsarConnectorConfig); ManagedLedgerFactory managedLedgerFactory = pulsarConnectorCache.getManagedLedgerFactory(); ManagedLedgerConfig managedLedgerConfig = pulsarConnectorCache.getManagedLedgerConfig( topicName.getNamespaceObject(), offloadPolicies, pulsarConnectorConfig); return getSplitsForTopic( - topicName.getPersistenceNamingEncoding(), + topicName, managedLedgerFactory, managedLedgerConfig, numSplits, @@ -272,11 +277,11 @@ Collection getSplitsNonPartitionedTopic(int numSplits, TopicName to topicName.getLocalName(), tupleDomain, offloadPolicies, - readCompactedType); + readCompacted); } @VisibleForTesting - Collection getSplitsForTopic(String topicNamePersistenceEncoding, + Collection getSplitsForTopic(TopicName topicName, ManagedLedgerFactory managedLedgerFactory, ManagedLedgerConfig managedLedgerConfig, int numSplits, @@ -284,25 +289,27 @@ Collection getSplitsForTopic(String topicNamePersistenceEncoding, SchemaInfo schemaInfo, String tableName, TupleDomain tupleDomain, OffloadPoliciesImpl offloadPolicies, - ReadCompactedType readCompactedType) + boolean readCompacted) throws ManagedLedgerException, InterruptedException, IOException, PulsarAdminException { ReadOnlyCursor readOnlyCursor = null; try { Position cursorStartPos = PositionImpl.EARLIEST; - if (readCompactedType != null) { + if (readCompacted) { PersistentTopicInternalStats internalStats = pulsarAdmin.topics() - .getInternalStats(topicNamePersistenceEncoding); + .getInternalStats(topicName.toString()); if (internalStats.cursors.containsKey("Compaction")) { String[] compactedHorizonArr = internalStats.cursors.get("Compaction").markDeletePosition.split(":"); cursorStartPos = PositionImpl.get( Long.parseLong(compactedHorizonArr[0]), Long.parseLong(compactedHorizonArr[1])); } + numSplits = 1; } + String topicNamePersistenceEncoding = topicName.getPersistenceNamingEncoding(); readOnlyCursor = managedLedgerFactory.openReadOnlyCursor( - topicNamePersistenceEncoding, + topicName.getPersistenceNamingEncoding(), cursorStartPos, managedLedgerConfig); long numEntries = readOnlyCursor.getNumberOfEntries(); @@ -357,7 +364,7 @@ Collection getSplitsForTopic(String topicNamePersistenceEncoding, tupleDomain, objectMapper.writeValueAsString(schemaInfo.getProperties()), offloadPolicies, - readCompactedType); + readCompacted); splits.add(pulsarSplit); } return splits; diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/Demo.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/Demo.java index 1aac760d8a944..740b0c69f6175 100644 --- a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/Demo.java +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/Demo.java @@ -55,11 +55,11 @@ public void sendData() throws Exception { .subscribe(); consumer.close(); - for (int i = 0; i < 10; i++) { + for (int i = 0; i < 100; i++) { User user = new User(); user.setName("user-" + i); user.setAge(10 + i); - producer.newMessage().value(user).send(); + producer.newMessage().key("" + (i % 10)).value(user).send(); } producer.close(); @@ -71,9 +71,11 @@ public void sendDataWithRepeatedKey() throws Exception { .serviceUrl("pulsar://localhost:6650") .build(); - String topic = "pt-10"; + String topic = "pt-5"; Producer producer = pulsarClient.newProducer(Schema.AVRO(User.class)) .topic(topic) + .batchingMaxMessages(100) + .batchingMaxPublishDelay(1, TimeUnit.SECONDS) .create(); Consumer consumer = pulsarClient.newConsumer(Schema.AVRO(User.class)) @@ -83,11 +85,12 @@ public void sendDataWithRepeatedKey() throws Exception { .subscribe(); consumer.close(); - for (int i = 0; i < 100; i++) { + for (int i = 0; i < 10; i++) { User user = new User(); user.setName("user-" + i); user.setAge(10 + i); - producer.newMessage().key("" + i % 10).value(user).send(); + producer.newMessage().key("" + i % 10).value(user).sendAsync(); +// producer.newMessage().key("5").value(user).sendAsync(); } producer.close(); @@ -123,9 +126,25 @@ public void readCompactedData() throws Exception { pulsarClient.close(); } + public void removeKey(String key) throws Exception { + PulsarClient pulsarClient = PulsarClient.builder() + .serviceUrl("pulsar://localhost:6650") + .build(); + + String topic = "user"; + Producer producer = pulsarClient.newProducer(Schema.AVRO(User.class)) + .topic(topic) + .create(); + + producer.newMessage().key(key).value(null).send(); + + producer.close(); + pulsarClient.close(); + } + public static void main(String[] args) throws Exception { Demo demo = new Demo(); - demo.readCompactedData(); + demo.sendDataWithRepeatedKey(); } } diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestCompactedQuery.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestCompactedQuery.java index 0fbb73adbba45..b72697996d34d 100644 --- a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestCompactedQuery.java +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestCompactedQuery.java @@ -57,7 +57,6 @@ import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.schema.SchemaType; -import org.apache.pulsar.sql.presto.util.ReadCompactedType; import org.testcontainers.shaded.com.fasterxml.jackson.databind.ObjectMapper; import org.testcontainers.shaded.org.awaitility.Awaitility; import org.testng.Assert; @@ -146,7 +145,8 @@ private void compactQuery(boolean enableBatch, int compactedMsgNum, int unCompac Double price = BigDecimal.valueOf( RandomUtils.nextDouble(10, 100)).setScale(4, RoundingMode.HALF_UP).doubleValue(); final int index = i; - producer.newMessage().key(name).value(getStock(name, price, name, setValueForKey7)).sendAsync() + final Stock stock = getStock(name, price, name, setValueForKey7); + producer.newMessage().key(name).value(stock).sendAsync() .thenAccept(messageId -> { if (index == 0) { firstMessageId.set((MessageIdImpl) messageId); @@ -154,9 +154,13 @@ private void compactQuery(boolean enableBatch, int compactedMsgNum, int unCompac if (index == compactedMsgNum - 1) { lastMessageId.set(getNextMessageId(messageId)); } - latestPrice.put(name, price); + if (stock == null) { + latestPrice.remove(name); + } else { + latestPrice.put(name, price); + } sendReceiptsCount.incrementAndGet(); - System.out.println("xxxx send message with id " + messageId); + System.out.println("xxxx send message, id: " + messageId + ", name: " + name + ", price: " + price); }); } @@ -173,7 +177,8 @@ private void compactQuery(boolean enableBatch, int compactedMsgNum, int unCompac Double price = BigDecimal.valueOf( RandomUtils.nextDouble(10, 100)).setScale(4, RoundingMode.HALF_UP).doubleValue(); final int index = i; - producer.newMessage().key(name).value(getStock(name, price, name, setValueForKey7)).sendAsync() + final Stock stock = getStock(name, price, name, setValueForKey7); + producer.newMessage().key(name).value(stock).sendAsync() .thenAccept(messageId -> { if (index == 0 && firstMessageId.get() == null) { firstMessageId.set((MessageIdImpl) messageId); @@ -182,7 +187,11 @@ private void compactQuery(boolean enableBatch, int compactedMsgNum, int unCompac lastMessageId.set(getNextMessageId(messageId)); } - latestPrice.put(name, price); + if (stock == null) { + latestPrice.remove(name); + } else { + latestPrice.put(name, price); + } entrySet.add(((MessageIdImpl) messageId).getLedgerId() + ":" + ((MessageIdImpl) messageId).getEntryId()); sendReceiptsCount.incrementAndGet(); @@ -194,8 +203,6 @@ private void compactQuery(boolean enableBatch, int compactedMsgNum, int unCompac .atMost(5, TimeUnit.SECONDS) .until(() -> sendReceiptsCount.get() == (compactedMsgNum + unCompactedMsgNum)); log.info("finish to prepare compaction data"); -// MessageId messageId = producer.newMessage().value(new Stock("", 0.0)).send(); -// System.out.println("xxxx the last message id is " + messageId); ObjectMapper objectMapper = new ObjectMapper(); PulsarSplit pulsarSplit = new PulsarSplit( @@ -214,7 +221,7 @@ private void compactQuery(boolean enableBatch, int compactedMsgNum, int unCompac TupleDomain.all(), objectMapper.writeValueAsString(new HashMap<>()), null, - ReadCompactedType.COMPACTED_LATEST); + true); List pulsarColumnHandles = TestPulsarConnector.getColumnColumnHandles( topicName, Schema.AVRO(Stock.class).getSchemaInfo(), PulsarColumnHandle.HandleKeyValueType.NONE, true); @@ -256,7 +263,6 @@ private void compactQuery(boolean enableBatch, int compactedMsgNum, int unCompac assertTrue(latestPrice.containsKey(key)); assertEquals(name, key); assertEquals(latestPrice.remove(key), price); - System.out.println("read message key: " + key + ", price: " + price); } Assert.assertTrue(latestPrice.isEmpty()); } diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarSplitManager.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarSplitManager.java index 1dbdc86d6f583..f0e3302cf24a1 100644 --- a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarSplitManager.java +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarSplitManager.java @@ -324,7 +324,7 @@ public void testPartitionFilter(String delimiter) throws Exception { domainMap.put(PulsarInternalColumn.PARTITION.getColumnHandle(pulsarConnectorId.toString(), false), domain); TupleDomain tupleDomain = TupleDomain.withColumnDomains(domainMap); Collection splits = this.pulsarSplitManager.getSplitsPartitionedTopic(2, topicName, pulsarTableHandle, - schemas.getSchemaInfo(topicName.getSchemaName()), tupleDomain, null, null); + schemas.getSchemaInfo(topicName.getSchemaName()), tupleDomain, null, false); if (topicsToNumEntries.get(topicName.getSchemaName()) > 1) { Assert.assertEquals(splits.size(), 2); } @@ -341,7 +341,7 @@ public void testPartitionFilter(String delimiter) throws Exception { domainMap.put(PulsarInternalColumn.PARTITION.getColumnHandle(pulsarConnectorId.toString(), false), domain); tupleDomain = TupleDomain.withColumnDomains(domainMap); splits = this.pulsarSplitManager.getSplitsPartitionedTopic(1, topicName, pulsarTableHandle, - schemas.getSchemaInfo(topicName.getSchemaName()), tupleDomain, null, null); + schemas.getSchemaInfo(topicName.getSchemaName()), tupleDomain, null, false); if (topicsToNumEntries.get(topicName.getSchemaName()) > 1) { Assert.assertEquals(splits.size(), 2); } @@ -357,7 +357,7 @@ public void testPartitionFilter(String delimiter) throws Exception { domainMap.put(PulsarInternalColumn.PARTITION.getColumnHandle(pulsarConnectorId.toString(), false), domain); tupleDomain = TupleDomain.withColumnDomains(domainMap); splits = this.pulsarSplitManager.getSplitsPartitionedTopic(2, topicName, pulsarTableHandle, - schemas.getSchemaInfo(topicName.getSchemaName()), tupleDomain, null, null); + schemas.getSchemaInfo(topicName.getSchemaName()), tupleDomain, null, false); if (topicsToNumEntries.get(topicName.getSchemaName()) > 1) { Assert.assertEquals(splits.size(), 3); } @@ -376,7 +376,7 @@ public void testPartitionFilter(String delimiter) throws Exception { domainMap.put(PulsarInternalColumn.PARTITION.getColumnHandle(pulsarConnectorId.toString(), false), domain); tupleDomain = TupleDomain.withColumnDomains(domainMap); splits = this.pulsarSplitManager.getSplitsPartitionedTopic(2, topicName, pulsarTableHandle, - schemas.getSchemaInfo(topicName.getSchemaName()), tupleDomain, null, null); + schemas.getSchemaInfo(topicName.getSchemaName()), tupleDomain, null, false); if (topicsToNumEntries.get(topicName.getSchemaName()) > 1) { Assert.assertEquals(splits.size(), 4); } @@ -388,7 +388,6 @@ public void testPartitionFilter(String delimiter) throws Exception { } } - } @Test(dataProvider = "rewriteNamespaceDelimiter", singleThreaded = true) diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestReadChunkedMessages.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestReadChunkedMessages.java index f9595ade1b8c1..8f857563e36f3 100644 --- a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestReadChunkedMessages.java +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestReadChunkedMessages.java @@ -107,7 +107,7 @@ public void queryTest() throws Exception { connectorConfig.setWebServiceUrl(pulsar.getWebServiceAddress()); PulsarSplitManager pulsarSplitManager = new PulsarSplitManager(new PulsarConnectorId("1"), connectorConfig); Collection splits = pulsarSplitManager.getSplitsForTopic( - topicName.getPersistenceNamingEncoding(), + topicName, pulsar.getManagedLedgerFactory(), new ManagedLedgerConfig(), 3, @@ -116,7 +116,7 @@ public void queryTest() throws Exception { topic, TupleDomain.all(), null, - null); + false); List columnHandleList = TestPulsarConnector.getColumnColumnHandles( topicName, schemaInfo, PulsarColumnHandle.HandleKeyValueType.NONE, true); diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestBasicPresto.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestBasicPresto.java index 7658883441f5b..38ef7e467d240 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestBasicPresto.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestBasicPresto.java @@ -23,9 +23,18 @@ import static org.testng.Assert.assertTrue; import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.RandomUtils; +import org.apache.pulsar.client.admin.LongRunningProcessStatus; import org.apache.pulsar.client.api.CompressionType; +import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; @@ -40,6 +49,7 @@ import org.apache.pulsar.common.schema.SchemaType; import org.apache.pulsar.tests.integration.docker.ContainerExecException; import org.apache.pulsar.tests.integration.docker.ContainerExecResult; +import org.testcontainers.shaded.org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @@ -363,4 +373,77 @@ public void testQueueBigEntry() throws Exception { Assert.assertEquals(count, messageCnt); } + @DataProvider(name = "compactedQueryProvider") + public Object[][] compactedQueryProvider() { + return new Object[][] { + {0, 0}, + {100, 0}, + {0, 100}, + {100, 100} + }; + } + + @Test(timeOut = 1000 * 30, dataProvider = "compactedQueryProvider") + public void testCompactedQueryForNoBatchData(int compactedCount, int noCompactedCount) throws Exception { + testCompactedQuery(false, compactedCount, noCompactedCount); + } + + @Test(timeOut = 1000 * 30, dataProvider = "compactedQueryProvider") + public void testCompactedQueryForBatchData(int compactedCount, int noCompactedCount) throws Exception { + testCompactedQuery(true, compactedCount, noCompactedCount); + } + + private void testCompactedQuery(boolean enableBatch, int compactedCount, int noCompactedCount) throws Exception { + String tableName = "compacted-topic-" + randomName(5); + String topic = "public/default/" + tableName; + + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.AVRO(Stock.class)) + .topic(topic) + .enableBatching(enableBatch) + .maxPendingMessages(10) + .create(); + + int divisor = 8; + Map latestStocks = new HashMap<>(); + prepareDataForCompactedQuery(producer, latestStocks, compactedCount, divisor); + if (compactedCount > 0) { + pulsarAdmin.topics().triggerCompaction(topic); + Awaitility.await().until(() -> { + LongRunningProcessStatus status = pulsarAdmin.topics().compactionStatus(topic); + return Objects.equals(LongRunningProcessStatus.Status.SUCCESS, status.status); + }); + } + prepareDataForCompactedQuery(producer, latestStocks, noCompactedCount, divisor); + + Assert.assertEquals(selectCount("public/default", tableName), divisor); + ContainerExecResult result = execQuery( + "select * from pulsar.\"public/default\".\"" + tableName + "\" where __compacted_query__=true"); + assertThat(result.getExitCode()).isEqualTo(0); + log.info("select sql query output \n{}", result.getStdout()); + String[] split = result.getStdout().split("\n"); + assertThat(split.length).isEqualTo(divisor); + String[] contentArr = result.getStdout().split("\n|,"); + } + + private void prepareDataForCompactedQuery(Producer producer, Map latestStocks, + int messageCount, int divisor) { + AtomicInteger sendCount = new AtomicInteger(); + for (int i = 0; i < messageCount; i++) { + int entryId = i % divisor; + String key = "" + entryId; + Stock stock = new Stock(entryId, "stock-" + entryId, RandomUtils.nextDouble(50, 90)); + CompletableFuture future = producer.newMessage().key(key).value(stock).sendAsync(); + future.thenApply(messageId -> { + latestStocks.put(key, stock); + sendCount.incrementAndGet(); + return null; + }); + } + Awaitility.await() + .atMost(5, TimeUnit.SECONDS) + .pollInterval(100, TimeUnit.MILLISECONDS) + .until(() -> sendCount.get() == messageCount); + } + } From 6155aaa394745dd2478b8d8fedd0421128877857 Mon Sep 17 00:00:00 2001 From: rangao Date: Tue, 14 Feb 2023 18:40:04 +0800 Subject: [PATCH 04/15] fix --- .../mledger/impl/ManagedCursorImpl.java | 1 - .../pulsar/sql/presto/PulsarRecordCursor.java | 198 ++++++++++-------- .../apache/pulsar/sql/presto/PulsarSplit.java | 2 +- .../pulsar/sql/presto/TestCompactedQuery.java | 141 +++++-------- .../integration/presto/TestBasicPresto.java | 27 ++- 5 files changed, 188 insertions(+), 181 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 0e10df6365948..5851395b08566 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -1048,7 +1048,6 @@ public boolean hasMoreEntries() { // is // at the last entry in the previous ledger PositionImpl writerPosition = ledger.getLastPosition(); - System.out.println("xxxx writer position is " + writerPosition); if (writerPosition.getEntryId() != -1) { return readPosition.compareTo(writerPosition) <= 0; } else { diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java index 6cb1b6580fd29..4b95f9b866ffe 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java @@ -71,6 +71,7 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.BatchMessageIdImpl; +import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.client.impl.schema.KeyValueSchemaInfo; import org.apache.pulsar.common.api.proto.CompressionType; import org.apache.pulsar.common.api.proto.MessageIdData; @@ -144,14 +145,13 @@ public class PulsarRecordCursor implements RecordCursor { protected ConcurrentOpenHashMap chunkedMessagesMap = ConcurrentOpenHashMap.newBuilder().build(); - private PulsarConnectorCache pulsarConnectorCache; private OffloadPoliciesImpl offloadPolicies; + + private CompactedLedgerReader compactedLedgerReader; private volatile Throwable compactedHandleError; - private final boolean compactedQuery; - private long compactedLedgerId = -1; - protected ConcurrentOpenHashMap compactedMessage = + private final boolean isCompactedQuery; + protected ConcurrentOpenHashMap compactedMessageIds = ConcurrentOpenHashMap.newBuilder().build(); - private LedgerHandle compactedLedgerHandle; private BookKeeper bookKeeper; private ManagedLedgerFactory managedLedgerFactory; private ManagedLedgerConfig managedLedgerConfig; @@ -164,6 +164,7 @@ public PulsarRecordCursor(List columnHandles, PulsarSplit pu this.splitSize = pulsarSplit.getSplitSize(); // Set start time for split this.startTime = System.nanoTime(); + PulsarConnectorCache pulsarConnectorCache; try { pulsarConnectorCache = PulsarConnectorCache.getConnectorCache(pulsarConnectorConfig); } catch (Exception e) { @@ -185,10 +186,8 @@ public PulsarRecordCursor(List columnHandles, PulsarSplit pu offloadPolicies.setManagedLedgerOffloadMaxThreads( pulsarConnectorConfig.getManagedLedgerOffloadMaxThreads()); } - this.compactedQuery = pulsarSplit.getCompactedQuery(); - initialize(columnHandles, pulsarSplit, pulsarConnectorConfig, - managedLedgerFactory, - managedLedgerConfig, + this.isCompactedQuery = pulsarSplit.isCompactedQuery(); + initialize(columnHandles, pulsarSplit, pulsarConnectorConfig, managedLedgerFactory, managedLedgerConfig, new PulsarConnectorMetricsTracker(pulsarConnectorCache.getStatsProvider())); this.decoderFactory = decoderFactory; } @@ -199,7 +198,7 @@ public PulsarRecordCursor(List columnHandles, PulsarSplit pu PulsarConnectorMetricsTracker pulsarConnectorMetricsTracker, PulsarDispatchingRowDecoderFactory decoderFactory) { this.splitSize = pulsarSplit.getSplitSize(); - this.compactedQuery = pulsarSplit.getCompactedQuery(); + this.isCompactedQuery = pulsarSplit.isCompactedQuery(); initialize(columnHandles, pulsarSplit, pulsarConnectorConfig, managedLedgerFactory, managedLedgerConfig, pulsarConnectorMetricsTracker); this.decoderFactory = decoderFactory; @@ -210,7 +209,7 @@ public PulsarRecordCursor(List columnHandles, PulsarSplit pu PulsarConnectorMetricsTracker pulsarConnectorMetricsTracker, PulsarDispatchingRowDecoderFactory decoderFactory, BookKeeper bookKeeper) { this.splitSize = pulsarSplit.getSplitSize(); - this.compactedQuery = pulsarSplit.getCompactedQuery(); + this.isCompactedQuery = pulsarSplit.isCompactedQuery(); this.bookKeeper = bookKeeper; this.managedLedgerFactory = managedLedgerFactory; this.managedLedgerConfig = managedLedgerConfig; @@ -249,9 +248,7 @@ private void initialize(List columnHandles, PulsarSplit puls log.info("Initializing split with parameters: %s", pulsarSplit); try { - if (compactedQuery) { - initCompactedRead(); - } else { + if (!isCompactedQuery) { this.cursor = getCursor(topicName, pulsarSplit.getStartPosition(), managedLedgerFactory, managedLedgerConfig); } @@ -361,10 +358,10 @@ public void accept(Entry entry) { message = null; } if (message != null) { - if (compactedQuery && message.getKey().isPresent()) { + if (isCompactedQuery && message.getKey().isPresent()) { RawMessageIdImpl messageId = (RawMessageIdImpl) message.getMessageId(); - compactMessage(message.getKey().get(), + cacheCmpactedMessageIds(message.getKey().get(), new BatchMessageIdImpl( messageId.getLedgerId(), messageId.getEntryId(), @@ -547,14 +544,17 @@ private boolean haveAvailableCacheSize(CacheSizeAllocator cacheSizeAllocator, Sp @Override public boolean advanceNextPosition() { - if (readEntries == null) { - if (compactedQuery) { + if (isCompactedQuery) { + if (compactedLedgerReader == null) { + compactedLedgerReader = new CompactedLedgerReader(); Thread compactedHandleThread = new Thread(new CompactedLedgerReader()); compactedHandleThread.setUncaughtExceptionHandler((t, ex) -> { compactedHandleError = ex; }); compactedHandleThread.start(); - } else { + } + } else { + if (readEntries == null) { // start deserialize thread deserializeEntries = new DeserializeEntries(); deserializeEntries.setUncaughtExceptionHandler((t, ex) -> { @@ -573,7 +573,7 @@ public boolean advanceNextPosition() { } while (true) { - if (readEntries != null && readEntries.hasFinished() && compactedMessage.isEmpty()) { + if (readEntries != null && readEntries.hasFinished() && compactedMessageIds.isEmpty()) { return false; } @@ -594,7 +594,9 @@ public boolean advanceNextPosition() { } else { try { long waitMills = 1; - if (compactedQuery) { + if (isCompactedQuery) { + // Wait more time when executing compacted query, + // because compacted query requires read all data to get the latest data of the key. waitMills = 10; } Thread.sleep(waitMills); @@ -606,7 +608,6 @@ public boolean advanceNextPosition() { } } - log.info("deserialize message " + currentMessage.getMessageId()); //start time for deserializing record metricsTracker.start_RECORD_DESERIALIZE_TIME(); @@ -673,9 +674,11 @@ public boolean advanceNextPosition() { .filter(col -> PulsarColumnHandle.HandleKeyValueType.NONE .equals(col.getHandleKeyValueType())) .collect(toImmutableSet())); - Optional> decodedValue = - messageDecoder.decodeRow(this.currentMessage.getData()); - decodedValue.ifPresent(currentRowValuesMap::putAll); + if (this.currentMessage.getData().readableBytes() > 0) { + Optional> decodedValue = + messageDecoder.decodeRow(this.currentMessage.getData()); + decodedValue.ifPresent(currentRowValuesMap::putAll); + } } for (DecoderColumnHandle columnHandle : columnHandles) { @@ -707,7 +710,7 @@ public boolean advanceNextPosition() { throw new RuntimeException(e); } } else if (COMPACTED_QUERY.getName().equals(columnHandle.getName())) { - currentRowValuesMap.put(columnHandle, booleanValueProvider(compactedQuery)); + currentRowValuesMap.put(columnHandle, booleanValueProvider(isCompactedQuery)); } else { throw new IllegalArgumentException("unknown internal field " + columnHandle.getName()); } @@ -834,12 +837,8 @@ public void close() { } } - if (this.compactedLedgerHandle != null) { - try { - this.compactedLedgerHandle.close(); - } catch (Exception e) { - log.error(e, "Failed to close compacted leger handle."); - } + if (this.compactedLedgerReader != null) { + this.compactedLedgerReader.close(); } // set stat for total execution time of split @@ -961,47 +960,53 @@ public void recycle() { } } - private void initCompactedRead() throws Exception { - String topic = topicName.getPartition(partition).toString(); - PersistentTopicInternalStats internalStats = - schemaInfoProvider.getPulsarAdmin().topics().getInternalStats(topic); - if (internalStats.compactedLedger == null - || internalStats.compactedLedger.ledgerId == -1 - || internalStats.compactedLedger.entries == 0) { - return; - } - this.compactedLedgerId = internalStats.compactedLedger.ledgerId; - this.compactedLedgerHandle = bookKeeper.openLedger(compactedLedgerId, - BookKeeper.DigestType.fromApiDigestType(DigestType.CRC32), "".getBytes()); - } - - private void compactMessage(String key, BatchMessageIdImpl messageId, long payloadSize) { + private void cacheCmpactedMessageIds(String key, BatchMessageIdImpl messageId, long payloadSize) { if (key == null) { return; } - if (compactedQuery) { - if (payloadSize > 0) { - compactedMessage.put(key, messageId); - } else { - compactedMessage.remove(key); - } + if (payloadSize > 0) { + compactedMessageIds.put(key, messageId); + } else { + compactedMessageIds.remove(key); } } class CompactedLedgerReader implements AsyncCallback.ReadCallback, Runnable { private final int readSize = 1000; + private final SpscArrayQueue queue; + private final AtomicBoolean havePendingRead = new AtomicBoolean(false); private long startEntry = 0; private long readEntry = 0; - private SpscArrayQueue queue; private MessageIdData lastMessageId; - private AtomicBoolean havePendingRead = new AtomicBoolean(false); + private long compactedLedgerId = -1; + private LedgerHandle compactedLedgerHandle; CompactedLedgerReader() { queue = new SpscArrayQueue<>((int) (readSize * 1.5)); lastMessageId = new MessageIdData(); lastMessageId.setLedgerId(-1); lastMessageId.setEntryId(-1); + try { + initCompactedRead(); + } catch (Exception e) { + log.error(e, "Failed to init compacted reader."); + throw new RuntimeException(e); + } + } + + private void initCompactedRead() throws Exception { + String topic = topicName.getPartition(partition).toString(); + PersistentTopicInternalStats internalStats = + schemaInfoProvider.getPulsarAdmin().topics().getInternalStats(topic); + if (internalStats.compactedLedger == null + || internalStats.compactedLedger.ledgerId == -1 + || internalStats.compactedLedger.entries == 0) { + return; + } + this.compactedLedgerId = internalStats.compactedLedger.ledgerId; + this.compactedLedgerHandle = bookKeeper.openLedger(compactedLedgerId, + BookKeeper.DigestType.fromApiDigestType(DigestType.CRC32), "".getBytes()); } private void readMoreEntriesIfNeed() { @@ -1028,8 +1033,8 @@ public void readComplete(int i, LedgerHandle ledgerHandle, Enumeration messageIdIterator = compactedMessage.values().stream().sorted().iterator(); + private void readEntireData() { + Iterator messageIdIterator = compactedMessageIds.values().stream().sorted().iterator(); LedgerHandle readLedgerHandle = null; - long readLedgerId = -1; - boolean isCompactedLedger = false; + AtomicBoolean isCompactedLedger = new AtomicBoolean(false); while (messageIdIterator.hasNext()) { BatchMessageIdImpl messageId = messageIdIterator.next(); - if (messageId.getLedgerId() == compactedLedgerId) { - readLedgerHandle = compactedLedgerHandle; - isCompactedLedger = true; - } else { - try { - if (readLedgerId != messageId.getLedgerId()) { - if (readLedgerHandle != null) { - readLedgerHandle.close(); - } - readLedgerId = messageId.getLedgerId(); - isCompactedLedger = false; - readLedgerHandle = bookKeeper.openLedger( - readLedgerId, BookKeeper.DigestType.CRC32C, "".getBytes()); - } - } catch (Exception e) { - log.error(e, "Failed to open ledger handle."); - throw new RuntimeException(e); - } - } + readLedgerHandle = getLedgerHandle(messageId, isCompactedLedger, readLedgerHandle); readEntries(readLedgerHandle, messageId.getLedgerId(), messageId.getEntryId(), - messageId.getBatchIndex(), isCompactedLedger); + messageId.getBatchIndex(), isCompactedLedger.get()); } if (readLedgerHandle != null) { try { readLedgerHandle.close(); } catch (Exception e) { - log.warn("Failed to close ledger handle."); + log.warn("Failed to close ledger handle for ledger %s.", readLedgerHandle.getId()); + } + } + } + + private LedgerHandle getLedgerHandle(MessageIdImpl messageId, + AtomicBoolean isCompactedLedger, + LedgerHandle readLedgerHandle) { + if (messageId.getLedgerId() == compactedLedgerId) { + isCompactedLedger.set(true); + return compactedLedgerHandle; + } else { + try { + if (readLedgerHandle == null || readLedgerHandle.getId() != messageId.getLedgerId()) { + if (readLedgerHandle != null) { + readLedgerHandle.close(); + } + isCompactedLedger.set(false); + return bookKeeper.openLedger( + messageId.getLedgerId(), BookKeeper.DigestType.CRC32C, "".getBytes()); + } else { + return readLedgerHandle; + } + } catch (Exception e) { + log.error(e, "Failed to open ledger handle for ledger %s.", messageId.getLedgerId()); + throw new RuntimeException(e); } } } @@ -1198,12 +1210,22 @@ private void parseAndAddMessageToQueue( if (((RawMessageIdImpl) message.getMessageId()).getBatchIndex() == batchIndex) { messageQueue.offer(message); if (message.getKey().isPresent()) { - compactedMessage.remove(message.getKey().get()); + compactedMessageIds.remove(message.getKey().get()); } } }, pulsarConnectorConfig.getMaxMessageSize()); } + private void close() { + if (this.compactedLedgerHandle != null) { + try { + this.compactedLedgerHandle.close(); + } catch (Exception e) { + log.error(e, "Failed to close compacted leger handle."); + } + } + } + } diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplit.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplit.java index 6ee4c105d08b5..646ad8e048273 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplit.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplit.java @@ -197,7 +197,7 @@ public OffloadPoliciesImpl getOffloadPolicies() { } @JsonProperty - public Boolean getCompactedQuery() { + public Boolean isCompactedQuery() { return compactedQuery; } diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestCompactedQuery.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestCompactedQuery.java index b72697996d34d..2d5a21a6e86cd 100644 --- a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestCompactedQuery.java +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestCompactedQuery.java @@ -18,8 +18,9 @@ */ package org.apache.pulsar.sql.presto; -import static org.testng.AssertJUnit.assertEquals; -import static org.testng.AssertJUnit.assertTrue; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; import com.google.common.collect.Sets; import io.airlift.slice.Slice; @@ -35,7 +36,6 @@ import java.util.Objects; import java.util.Set; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import lombok.AllArgsConstructor; @@ -50,6 +50,7 @@ import org.apache.pulsar.client.admin.LongRunningProcessStatus; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.common.naming.TopicDomain; @@ -59,7 +60,6 @@ import org.apache.pulsar.common.schema.SchemaType; import org.testcontainers.shaded.com.fasterxml.jackson.databind.ObjectMapper; import org.testcontainers.shaded.org.awaitility.Awaitility; -import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.DataProvider; @@ -135,34 +135,12 @@ private void compactQuery(boolean enableBatch, int compactedMsgNum, int unCompac .batchingMaxPublishDelay(1, TimeUnit.SECONDS) .create(); - AtomicInteger sendReceiptsCount = new AtomicInteger(0); Map latestPrice = new HashMap<>(); AtomicReference firstMessageId = new AtomicReference<>((MessageIdImpl) MessageId.earliest); AtomicReference lastMessageId = new AtomicReference<>((MessageIdImpl) MessageId.latest); - AtomicBoolean setValueForKey7 = new AtomicBoolean(false); - for (int i = 0; i < compactedMsgNum; i++) { - String name = "stock-" + i % 10; - Double price = BigDecimal.valueOf( - RandomUtils.nextDouble(10, 100)).setScale(4, RoundingMode.HALF_UP).doubleValue(); - final int index = i; - final Stock stock = getStock(name, price, name, setValueForKey7); - producer.newMessage().key(name).value(stock).sendAsync() - .thenAccept(messageId -> { - if (index == 0) { - firstMessageId.set((MessageIdImpl) messageId); - } - if (index == compactedMsgNum - 1) { - lastMessageId.set(getNextMessageId(messageId)); - } - if (stock == null) { - latestPrice.remove(name); - } else { - latestPrice.put(name, price); - } - sendReceiptsCount.incrementAndGet(); - System.out.println("xxxx send message, id: " + messageId + ", name: " + name + ", price: " + price); - }); - } + int divisor = 8; + + prepareData(producer, compactedMsgNum, divisor, latestPrice, firstMessageId, lastMessageId); admin.topics().triggerCompaction(topicName.toString()); Awaitility.await().until(() -> { @@ -170,38 +148,11 @@ private void compactQuery(boolean enableBatch, int compactedMsgNum, int unCompac return Objects.equals(LongRunningProcessStatus.Status.SUCCESS, status.status); }); - setValueForKey7 = new AtomicBoolean(false); - Set entrySet = new HashSet<>(); - for (int i = 0; i < unCompactedMsgNum; i++) { - String name = "stock-" + i % 10; - Double price = BigDecimal.valueOf( - RandomUtils.nextDouble(10, 100)).setScale(4, RoundingMode.HALF_UP).doubleValue(); - final int index = i; - final Stock stock = getStock(name, price, name, setValueForKey7); - producer.newMessage().key(name).value(stock).sendAsync() - .thenAccept(messageId -> { - if (index == 0 && firstMessageId.get() == null) { - firstMessageId.set((MessageIdImpl) messageId); - } - if (index == unCompactedMsgNum - 1) { - lastMessageId.set(getNextMessageId(messageId)); - } - - if (stock == null) { - latestPrice.remove(name); - } else { - latestPrice.put(name, price); - } - entrySet.add(((MessageIdImpl) messageId).getLedgerId() + ":" - + ((MessageIdImpl) messageId).getEntryId()); - sendReceiptsCount.incrementAndGet(); - System.out.println("xxxx 2 send message with id " + messageId); - }); + Set entrySet = + prepareData(producer, unCompactedMsgNum, divisor, latestPrice, firstMessageId, lastMessageId); + if (compactedMsgNum > 0 || unCompactedMsgNum > 0) { + assertEquals(divisor, latestPrice.size()); } - - Awaitility.await() - .atMost(5, TimeUnit.SECONDS) - .until(() -> sendReceiptsCount.get() == (compactedMsgNum + unCompactedMsgNum)); log.info("finish to prepare compaction data"); ObjectMapper objectMapper = new ObjectMapper(); @@ -252,37 +203,55 @@ private void compactQuery(boolean enableBatch, int compactedMsgNum, int unCompac key = slice == null ? null : new String(slice.getBytes()); } } - Assert.assertNotNull(key); - if (key.equals("stock-7")) { - Assert.assertNull(name); - Assert.assertNull(price); - continue; - } - Assert.assertNotNull(name); - Assert.assertNotNull(price); + assertNotNull(key); + assertNotNull(name); + assertNotNull(price); assertTrue(latestPrice.containsKey(key)); - assertEquals(name, key); - assertEquals(latestPrice.remove(key), price); + assertEquals(key, name); + assertEquals(price, latestPrice.remove(key)); + } + if (compactedMsgNum > 0 || unCompactedMsgNum > 0) { + assertEquals(latestPrice.size(), 1); + assertNotNull(latestPrice.remove("stock-7")); + } else { + assertEquals(latestPrice.size(), 0); } - Assert.assertTrue(latestPrice.isEmpty()); - } - - private MessageIdImpl getNextMessageId(MessageId messageId) { - MessageIdImpl id = (MessageIdImpl) messageId; - return new MessageIdImpl(id.getLedgerId(), id.getEntryId() + 1, id.getPartitionIndex()); } - private Stock getStock(String name, Double price, String key, AtomicBoolean setValueForKey7) { - Stock stock = new Stock(name, price); - // If the key is 7, set a value first, then delete the value for the key - if (key.equals("stock-7")) { - if (!setValueForKey7.get()) { - setValueForKey7.set(true); - } else { - stock = null; - } + private Set prepareData(Producer producer, int messageCount, int divisor, + Map latestPrice, + AtomicReference firstMessageId, + AtomicReference lastMessageId) throws PulsarClientException { + Set entrySet = new HashSet<>(); + AtomicInteger sendCount = new AtomicInteger(); + for (int i = 0; i < messageCount; i++) { + String name = "stock-" + i % divisor; + Double price = BigDecimal.valueOf( + RandomUtils.nextDouble(10, 100)).setScale(4, RoundingMode.HALF_UP).doubleValue(); + final int index = i; + final Stock stock = new Stock(name, price); + producer.newMessage().key(name).value(stock).sendAsync() + .thenAccept(messageId -> { + MessageIdImpl idImpl = (MessageIdImpl) messageId; + if (index == 0) { + firstMessageId.set(idImpl); + } + entrySet.add(idImpl.getLedgerId() + ":" + idImpl.getEntryId()); + latestPrice.put(name, price); + sendCount.incrementAndGet(); + }); + } + if (messageCount > 0) { + MessageIdImpl idImpl = (MessageIdImpl) producer.newMessage().key("stock-" + 7).send(); + entrySet.add(idImpl.getLedgerId() + ":" + idImpl.getEntryId() + 1); + lastMessageId.set(new MessageIdImpl( + idImpl.getLedgerId(), idImpl.getEntryId() + 1, idImpl.getPartitionIndex())); } - return stock; + Awaitility.await() + .atMost(5, TimeUnit.SECONDS) + .pollInterval(100, TimeUnit.MILLISECONDS) + .until(() -> sendCount.get() == messageCount); + return entrySet; } } diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestBasicPresto.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestBasicPresto.java index 38ef7e467d240..13ba5209e4355 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestBasicPresto.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestBasicPresto.java @@ -19,7 +19,9 @@ package org.apache.pulsar.tests.integration.presto; import static org.assertj.core.api.Assertions.assertThat; +import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; import java.nio.ByteBuffer; @@ -418,16 +420,28 @@ private void testCompactedQuery(boolean enableBatch, int compactedCount, int noC Assert.assertEquals(selectCount("public/default", tableName), divisor); ContainerExecResult result = execQuery( - "select * from pulsar.\"public/default\".\"" + tableName + "\" where __compacted_query__=true"); + "select __key__,symbol,sharePrice from pulsar.\"public/default\".\"" + tableName + + "\" where __compacted_query__=true"); assertThat(result.getExitCode()).isEqualTo(0); log.info("select sql query output \n{}", result.getStdout()); - String[] split = result.getStdout().split("\n"); - assertThat(split.length).isEqualTo(divisor); - String[] contentArr = result.getStdout().split("\n|,"); + String[] dataArray = result.getStdout().split("\n"); + for (String data : dataArray) { + String[] columns = data.split(","); + Stock expectedStock = latestStocks.remove(columns[0].trim()); + assertNotNull(expectedStock); + assertEquals(columns[1], expectedStock.getSymbol()); + assertEquals(columns[2], "" + expectedStock.getSharePrice()); + } + if (compactedCount > 0 || noCompactedCount > 0) { + assertEquals(1, latestStocks.size()); + assertNotNull(latestStocks.remove("4")); + } else { + assertTrue(latestStocks.isEmpty()); + } } private void prepareDataForCompactedQuery(Producer producer, Map latestStocks, - int messageCount, int divisor) { + int messageCount, int divisor) throws PulsarClientException { AtomicInteger sendCount = new AtomicInteger(); for (int i = 0; i < messageCount; i++) { int entryId = i % divisor; @@ -440,6 +454,9 @@ private void prepareDataForCompactedQuery(Producer producer, Map 0) { + producer.newMessage().key("4").send(); + } Awaitility.await() .atMost(5, TimeUnit.SECONDS) .pollInterval(100, TimeUnit.MILLISECONDS) From b45cfcbcf51245e4e74b930a07f12e1d3245e5c0 Mon Sep 17 00:00:00 2001 From: rangao Date: Tue, 14 Feb 2023 18:53:49 +0800 Subject: [PATCH 05/15] remove useless code --- .../LeastResourceUsageWithWeightTest.java | 1 - .../src/main/resources/conf/jvm.config | 10 +- .../sql/presto/util/ReadCompactedType.java | 51 ------ .../org/apache/pulsar/sql/presto/Demo.java | 150 ------------------ 4 files changed, 1 insertion(+), 211 deletions(-) delete mode 100644 pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/util/ReadCompactedType.java delete mode 100644 pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/Demo.java diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/LeastResourceUsageWithWeightTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/LeastResourceUsageWithWeightTest.java index 2a9e556b1df10..2856dde892a8f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/LeastResourceUsageWithWeightTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/LeastResourceUsageWithWeightTest.java @@ -36,7 +36,6 @@ import org.apache.pulsar.broker.loadbalance.extensions.LoadManagerContext; import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLoadData; import org.apache.pulsar.broker.loadbalance.extensions.store.LoadDataStore; -import org.apache.pulsar.client.api.TableView; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.ServiceUnitId; import org.apache.pulsar.common.naming.TopicName; diff --git a/pulsar-sql/presto-distribution/src/main/resources/conf/jvm.config b/pulsar-sql/presto-distribution/src/main/resources/conf/jvm.config index 9ae7e633f8ae5..71e9ec1273bc0 100644 --- a/pulsar-sql/presto-distribution/src/main/resources/conf/jvm.config +++ b/pulsar-sql/presto-distribution/src/main/resources/conf/jvm.config @@ -25,12 +25,4 @@ -XX:+HeapDumpOnOutOfMemoryError -XX:+ExitOnOutOfMemoryError -Dpresto-temporarily-allow-java8=true --Djdk.attach.allowAttachSelf=true - --Dzookeeper.serverCnxnFactory=org.apache.zookeeper.server.NettyServerCnxnFactory --Dzookeeper.ssl.keyStore.location=/Users/ran/Work/pulsar-test/apache-pulsar-2.11.0-SNAPSHOT/certWithPrivateKey.pem -#-Dzookeeper.ssl.keyStore.password=testpass --Dzookeeper.ssl.trustStore.location=/Users/ran/Work/pulsar-test/apache-pulsar-2.11.0-SNAPSHOT/ca-cert.pem -#-Dzookeeper.ssl.trustStore.password=testpass --Dzookeeper.ssl.keyStore.type=PEM --Dzookeeper.ssl.trustStore.type=PEM +-Djdk.attach.allowAttachSelf=true \ No newline at end of file diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/util/ReadCompactedType.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/util/ReadCompactedType.java deleted file mode 100644 index 853574054c76b..0000000000000 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/util/ReadCompactedType.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.sql.presto.util; - - -public enum ReadCompactedType { - - COMPACTED_LATEST("COMPACTED_LATEST"), - - COMPACTED_EARLIEST("COMPACTED_EARLIEST"); - - private final String type; - - ReadCompactedType(String type) { - this.type = type; - } - - public String getType() { - return type; - } - -// public ReadCompactedType valueOf(String type) { -// if (StringUtils.isEmpty(type)) { -// return null; -// } -// -// for (ReadCompactedType readCompactedType : ReadCompactedType.values()) { -// if (readCompactedType.type.equalsIgnoreCase(type)) { -// return readCompactedType; -// } -// } -// return null; -// } - -} diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/Demo.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/Demo.java deleted file mode 100644 index 740b0c69f6175..0000000000000 --- a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/Demo.java +++ /dev/null @@ -1,150 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.sql.presto; - -import java.util.concurrent.TimeUnit; -import lombok.Data; -import lombok.NoArgsConstructor; -import lombok.ToString; -import org.apache.pulsar.client.api.Consumer; -import org.apache.pulsar.client.api.Message; -import org.apache.pulsar.client.api.Producer; -import org.apache.pulsar.client.api.PulsarClient; -import org.apache.pulsar.client.api.Schema; - - -public class Demo { - - @Data - @NoArgsConstructor - @ToString - static class User { - private String name; - private Integer age; - } - - public void sendData() throws Exception { - PulsarClient pulsarClient = PulsarClient.builder() - .serviceUrl("pulsar://localhost:6650") - .build(); - - String topic = "user"; - Producer producer = pulsarClient.newProducer(Schema.AVRO(User.class)) - .topic(topic) - .create(); - - Consumer consumer = pulsarClient.newConsumer(Schema.AVRO(User.class)) - .topic(topic) - .subscriptionName("sub") - .subscribe(); - consumer.close(); - - for (int i = 0; i < 100; i++) { - User user = new User(); - user.setName("user-" + i); - user.setAge(10 + i); - producer.newMessage().key("" + (i % 10)).value(user).send(); - } - - producer.close(); - pulsarClient.close(); - } - - public void sendDataWithRepeatedKey() throws Exception { - PulsarClient pulsarClient = PulsarClient.builder() - .serviceUrl("pulsar://localhost:6650") - .build(); - - String topic = "pt-5"; - Producer producer = pulsarClient.newProducer(Schema.AVRO(User.class)) - .topic(topic) - .batchingMaxMessages(100) - .batchingMaxPublishDelay(1, TimeUnit.SECONDS) - .create(); - - Consumer consumer = pulsarClient.newConsumer(Schema.AVRO(User.class)) - .topic(topic) - .subscriptionName("sub") - .readCompacted(true) - .subscribe(); - consumer.close(); - - for (int i = 0; i < 10; i++) { - User user = new User(); - user.setName("user-" + i); - user.setAge(10 + i); - producer.newMessage().key("" + i % 10).value(user).sendAsync(); -// producer.newMessage().key("5").value(user).sendAsync(); - } - - producer.close(); - pulsarClient.close(); - } - - public void readCompactedData() throws Exception { - PulsarClient pulsarClient = PulsarClient.builder() - .serviceUrl("pulsar://localhost:6650") - .build(); - - String topic = "pt-10"; - Consumer consumer = pulsarClient.newConsumer(Schema.AVRO(User.class)) - .topic(topic) - .subscriptionName("sub") - .readCompacted(true) - .subscribe(); - - long receiveCount = 0; - while (true) { - Message message = consumer.receive(5, TimeUnit.SECONDS); - if (message == null) { - break; - } - receiveCount++; - System.out.println("receive message " + receiveCount - + ", key: " + message.getKey() - + ", value: " + message.getValue().toString() - + ", messageId: " + message.getMessageId()); - } - - consumer.close(); - pulsarClient.close(); - } - - public void removeKey(String key) throws Exception { - PulsarClient pulsarClient = PulsarClient.builder() - .serviceUrl("pulsar://localhost:6650") - .build(); - - String topic = "user"; - Producer producer = pulsarClient.newProducer(Schema.AVRO(User.class)) - .topic(topic) - .create(); - - producer.newMessage().key(key).value(null).send(); - - producer.close(); - pulsarClient.close(); - } - - public static void main(String[] args) throws Exception { - Demo demo = new Demo(); - demo.sendDataWithRepeatedKey(); - } - -} From 3174a96b88b6490907e9d34dcfb29025a89b215b Mon Sep 17 00:00:00 2001 From: rangao Date: Thu, 16 Feb 2023 13:53:06 +0800 Subject: [PATCH 06/15] fix --- .../impl/ManagedLedgerFactoryImpl.java | 2 +- .../sql/presto/PulsarConnectorCache.java | 62 +++++-------------- .../pulsar/sql/presto/PulsarRecordCursor.java | 13 ++-- 3 files changed, 24 insertions(+), 53 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java index 9f3fe9bb0c4a7..19d5744498d84 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java @@ -215,7 +215,7 @@ private ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore, metadataStore.registerSessionListener(this::handleMetadataStoreNotification); } - static class DefaultBkFactory implements BookkeeperFactoryForCustomEnsemblePlacementPolicy { + public static class DefaultBkFactory implements BookkeeperFactoryForCustomEnsemblePlacementPolicy { private final BookKeeper bkClient; diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarConnectorCache.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarConnectorCache.java index 84020f3cf8a95..1831a4d9fb575 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarConnectorCache.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarConnectorCache.java @@ -27,8 +27,6 @@ import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; import lombok.Getter; -import org.apache.bookkeeper.client.BKException; -import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.common.util.OrderedScheduler; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.mledger.LedgerOffloader; @@ -38,6 +36,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerFactory; import org.apache.bookkeeper.mledger.ManagedLedgerFactoryConfig; import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl.BookkeeperFactoryForCustomEnsemblePlacementPolicy; import org.apache.bookkeeper.mledger.impl.NullLedgerOffloader; import org.apache.bookkeeper.mledger.offload.Offloaders; import org.apache.bookkeeper.mledger.offload.OffloadersCache; @@ -45,7 +44,6 @@ import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.common.naming.NamespaceName; -import org.apache.pulsar.common.policies.data.EnsemblePlacementPolicyConfig; import org.apache.pulsar.common.policies.data.OffloadPoliciesImpl; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.metadata.api.MetadataStoreConfig; @@ -73,33 +71,17 @@ public class PulsarConnectorCache { private Map offloaderMap = new ConcurrentHashMap<>(); @Getter - private final BookKeeper bookKeeper; + private final BookkeeperFactoryForCustomEnsemblePlacementPolicy bookKeeperFactory; private static final String OFFLOADERS_DIRECTOR = "offloadersDirectory"; private static final String MANAGED_LEDGER_OFFLOAD_DRIVER = "managedLedgerOffloadDriver"; private static final String MANAGED_LEDGER_OFFLOAD_MAX_THREADS = "managedLedgerOffloadMaxThreads"; - static class DefaultBkFactory implements - ManagedLedgerFactoryImpl.BookkeeperFactoryForCustomEnsemblePlacementPolicy { - - private final BookKeeper bkClient; - - public DefaultBkFactory(ClientConfiguration bkClientConfiguration) - throws InterruptedException, BKException, IOException { - bkClient = new BookKeeper(bkClientConfiguration); - } - - @Override - public BookKeeper get(EnsemblePlacementPolicyConfig policy) { - return bkClient; - } - } - private PulsarConnectorCache(PulsarConnectorConfig pulsarConnectorConfig) throws Exception { this.metadataStore = MetadataStoreExtended.create(pulsarConnectorConfig.getMetadataUrl(), MetadataStoreConfig.builder().metadataStoreName(MetadataStoreConfig.METADATA_STORE).build()); + this.bookKeeperFactory = initBookKeeperFactory(pulsarConnectorConfig); this.managedLedgerFactory = initManagedLedgerFactory(pulsarConnectorConfig); - this.bookKeeper = initBookKeeper(pulsarConnectorConfig); this.statsProvider = PulsarConnectorUtils.createInstance(pulsarConnectorConfig.getStatsProvider(), StatsProvider.class, getClass().getClassLoader()); @@ -131,30 +113,8 @@ public static PulsarConnectorCache getConnectorCache(PulsarConnectorConfig pulsa return instance; } - private ManagedLedgerFactory initManagedLedgerFactory(PulsarConnectorConfig pulsarConnectorConfig) - throws Exception { - PulsarMetadataClientDriver.init(); - - ClientConfiguration bkClientConfiguration = new ClientConfiguration() - .setMetadataServiceUri("metadata-store:" + pulsarConnectorConfig.getMetadataUrl()) - .setClientTcpNoDelay(false) - .setUseV2WireProtocol(pulsarConnectorConfig.getBookkeeperUseV2Protocol()) - .setExplictLacInterval(pulsarConnectorConfig.getBookkeeperExplicitInterval()) - .setStickyReadsEnabled(false) - .setReadEntryTimeout(60) - .setThrottleValue(pulsarConnectorConfig.getBookkeeperThrottleValue()) - .setNumIOThreads(pulsarConnectorConfig.getBookkeeperNumIOThreads()) - .setNumWorkerThreads(pulsarConnectorConfig.getBookkeeperNumWorkerThreads()) - .setNettyMaxFrameSizeBytes(pulsarConnectorConfig.getMaxMessageSize() + Commands.MESSAGE_SIZE_FRAME_PADDING); - - ManagedLedgerFactoryConfig managedLedgerFactoryConfig = new ManagedLedgerFactoryConfig(); - managedLedgerFactoryConfig.setMaxCacheSize(pulsarConnectorConfig.getManagedLedgerCacheSizeMB()); - managedLedgerFactoryConfig.setNumManagedLedgerSchedulerThreads( - pulsarConnectorConfig.getManagedLedgerNumSchedulerThreads()); - return new ManagedLedgerFactoryImpl(metadataStore, bkClientConfiguration, managedLedgerFactoryConfig); - } - - private BookKeeper initBookKeeper(PulsarConnectorConfig pulsarConnectorConfig) throws Exception { + private BookkeeperFactoryForCustomEnsemblePlacementPolicy initBookKeeperFactory( + PulsarConnectorConfig pulsarConnectorConfig) throws Exception { PulsarMetadataClientDriver.init(); ClientConfiguration bkClientConfiguration = new ClientConfiguration() @@ -169,7 +129,16 @@ private BookKeeper initBookKeeper(PulsarConnectorConfig pulsarConnectorConfig) t .setNumWorkerThreads(pulsarConnectorConfig.getBookkeeperNumWorkerThreads()) .setNettyMaxFrameSizeBytes( pulsarConnectorConfig.getMaxMessageSize() + Commands.MESSAGE_SIZE_FRAME_PADDING); - return new BookKeeper(bkClientConfiguration); + return new ManagedLedgerFactoryImpl.DefaultBkFactory(bkClientConfiguration); + } + + private ManagedLedgerFactory initManagedLedgerFactory(PulsarConnectorConfig pulsarConnectorConfig) + throws Exception { + ManagedLedgerFactoryConfig managedLedgerFactoryConfig = new ManagedLedgerFactoryConfig(); + managedLedgerFactoryConfig.setMaxCacheSize(pulsarConnectorConfig.getManagedLedgerCacheSizeMB()); + managedLedgerFactoryConfig.setNumManagedLedgerSchedulerThreads( + pulsarConnectorConfig.getManagedLedgerNumSchedulerThreads()); + return new ManagedLedgerFactoryImpl(metadataStore, bookKeeperFactory, managedLedgerFactoryConfig); } public ManagedLedgerConfig getManagedLedgerConfig(NamespaceName namespaceName, OffloadPoliciesImpl offloadPolicies, @@ -247,6 +216,7 @@ public static void shutdown() throws Exception { if (instance != null) { instance.statsProvider.stop(); instance.managedLedgerFactory.shutdown(); + instance.bookKeeperFactory.get().close(); instance.metadataStore.close(); instance.offloaderScheduler.shutdown(); instance.offloadersCache.close(); diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java index 4b95f9b866ffe..435e08f26b8b0 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java @@ -147,14 +147,15 @@ public class PulsarRecordCursor implements RecordCursor { private OffloadPoliciesImpl offloadPolicies; + private BookKeeper bookKeeper; + private ManagedLedgerFactory managedLedgerFactory; + private ManagedLedgerConfig managedLedgerConfig; + private CompactedLedgerReader compactedLedgerReader; private volatile Throwable compactedHandleError; private final boolean isCompactedQuery; protected ConcurrentOpenHashMap compactedMessageIds = ConcurrentOpenHashMap.newBuilder().build(); - private BookKeeper bookKeeper; - private ManagedLedgerFactory managedLedgerFactory; - private ManagedLedgerConfig managedLedgerConfig; private static final Logger log = Logger.get(PulsarRecordCursor.class); @@ -173,7 +174,7 @@ public PulsarRecordCursor(List columnHandles, PulsarSplit pu throw new RuntimeException(e); } - bookKeeper = pulsarConnectorCache.getBookKeeper(); + bookKeeper = pulsarConnectorCache.getBookKeeperFactory().get(); managedLedgerFactory = pulsarConnectorCache.getManagedLedgerFactory(); managedLedgerConfig = pulsarConnectorCache.getManagedLedgerConfig( TopicName.get("persistent", NamespaceName.get(pulsarSplit.getSchemaName()), @@ -1034,7 +1035,7 @@ public void readComplete(int i, LedgerHandle ledgerHandle, Enumeration messageIdIterator = compactedMessageIds.values().stream().sorted().iterator(); LedgerHandle readLedgerHandle = null; AtomicBoolean isCompactedLedger = new AtomicBoolean(false); From 4b1416d25d481c7692d14c9bf6be3817dc552b81 Mon Sep 17 00:00:00 2001 From: rangao Date: Thu, 16 Feb 2023 15:46:04 +0800 Subject: [PATCH 07/15] fix test --- .../pulsar/tests/integration/presto/TestBasicPresto.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestBasicPresto.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestBasicPresto.java index 13ba5209e4355..bf4d1d9df084c 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestBasicPresto.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestBasicPresto.java @@ -418,7 +418,8 @@ private void testCompactedQuery(boolean enableBatch, int compactedCount, int noC } prepareDataForCompactedQuery(producer, latestStocks, noCompactedCount, divisor); - Assert.assertEquals(selectCount("public/default", tableName), divisor); + Assert.assertEquals(selectCount("public/default", tableName), + compactedCount > 0 || noCompactedCount > 0 ? divisor : 0); ContainerExecResult result = execQuery( "select __key__,symbol,sharePrice from pulsar.\"public/default\".\"" + tableName + "\" where __compacted_query__=true"); From ace3932e15552fedfb0546222b5b6bf81285f03e Mon Sep 17 00:00:00 2001 From: rangao Date: Fri, 17 Feb 2023 00:18:26 +0800 Subject: [PATCH 08/15] fix test --- .../integration/presto/TestBasicPresto.java | 24 +++++++++++-------- 1 file changed, 14 insertions(+), 10 deletions(-) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestBasicPresto.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestBasicPresto.java index bf4d1d9df084c..62dfed9173186 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestBasicPresto.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestBasicPresto.java @@ -407,8 +407,9 @@ private void testCompactedQuery(boolean enableBatch, int compactedCount, int noC .create(); int divisor = 8; + String removeKey = "4"; Map latestStocks = new HashMap<>(); - prepareDataForCompactedQuery(producer, latestStocks, compactedCount, divisor); + prepareDataForCompactedQuery(producer, latestStocks, compactedCount, divisor, removeKey); if (compactedCount > 0) { pulsarAdmin.topics().triggerCompaction(topic); Awaitility.await().until(() -> { @@ -416,7 +417,7 @@ private void testCompactedQuery(boolean enableBatch, int compactedCount, int noC return Objects.equals(LongRunningProcessStatus.Status.SUCCESS, status.status); }); } - prepareDataForCompactedQuery(producer, latestStocks, noCompactedCount, divisor); + prepareDataForCompactedQuery(producer, latestStocks, noCompactedCount, divisor, removeKey); Assert.assertEquals(selectCount("public/default", tableName), compactedCount > 0 || noCompactedCount > 0 ? divisor : 0); @@ -425,6 +426,11 @@ private void testCompactedQuery(boolean enableBatch, int compactedCount, int noC + "\" where __compacted_query__=true"); assertThat(result.getExitCode()).isEqualTo(0); log.info("select sql query output \n{}", result.getStdout()); + if (compactedCount == 0 && noCompactedCount == 0) { + assertTrue(latestStocks.isEmpty()); + assertTrue(result.getStdout().trim().isEmpty()); + return; + } String[] dataArray = result.getStdout().split("\n"); for (String data : dataArray) { String[] columns = data.split(","); @@ -433,16 +439,14 @@ private void testCompactedQuery(boolean enableBatch, int compactedCount, int noC assertEquals(columns[1], expectedStock.getSymbol()); assertEquals(columns[2], "" + expectedStock.getSharePrice()); } - if (compactedCount > 0 || noCompactedCount > 0) { - assertEquals(1, latestStocks.size()); - assertNotNull(latestStocks.remove("4")); - } else { - assertTrue(latestStocks.isEmpty()); - } + assertEquals(1, latestStocks.size()); + assertNotNull(latestStocks.remove(removeKey)); } private void prepareDataForCompactedQuery(Producer producer, Map latestStocks, - int messageCount, int divisor) throws PulsarClientException { + int messageCount, int divisor, String removeKey) + throws PulsarClientException { + AtomicInteger sendCount = new AtomicInteger(); for (int i = 0; i < messageCount; i++) { int entryId = i % divisor; @@ -456,7 +460,7 @@ private void prepareDataForCompactedQuery(Producer producer, Map 0) { - producer.newMessage().key("4").send(); + producer.newMessage().key(removeKey).send(); } Awaitility.await() .atMost(5, TimeUnit.SECONDS) From 374088729e6318c427c481111da226716fe15759 Mon Sep 17 00:00:00 2001 From: rangao Date: Fri, 17 Feb 2023 11:14:27 +0800 Subject: [PATCH 09/15] fix test --- .../apache/pulsar/tests/integration/presto/TestBasicPresto.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestBasicPresto.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestBasicPresto.java index 62dfed9173186..3b5c127bb68c6 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestBasicPresto.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestBasicPresto.java @@ -403,7 +403,7 @@ private void testCompactedQuery(boolean enableBatch, int compactedCount, int noC Producer producer = pulsarClient.newProducer(Schema.AVRO(Stock.class)) .topic(topic) .enableBatching(enableBatch) - .maxPendingMessages(10) + .batchingMaxMessages(10) .create(); int divisor = 8; From f94c7668093d7f8e4c64d4184bf517b09a9a2a35 Mon Sep 17 00:00:00 2001 From: rangao Date: Fri, 17 Feb 2023 13:20:00 +0800 Subject: [PATCH 10/15] fix test --- .../tests/integration/presto/TestBasicPresto.java | 4 ++-- .../tests/integration/presto/TestPulsarSQLBase.java | 11 ++++++++--- 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestBasicPresto.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestBasicPresto.java index 3b5c127bb68c6..7b5e18c969aea 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestBasicPresto.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestBasicPresto.java @@ -371,7 +371,7 @@ public void testQueueBigEntry() throws Exception { producer.newMessage().value(data).send(); } - int count = selectCount("public/default", tableName); + int count = selectCount("public/default", tableName, null); Assert.assertEquals(count, messageCnt); } @@ -419,7 +419,7 @@ private void testCompactedQuery(boolean enableBatch, int compactedCount, int noC } prepareDataForCompactedQuery(producer, latestStocks, noCompactedCount, divisor, removeKey); - Assert.assertEquals(selectCount("public/default", tableName), + Assert.assertEquals(selectCount("public/default", tableName, "__compacted_query__=true"), compactedCount > 0 || noCompactedCount > 0 ? divisor : 0); ContainerExecResult result = execQuery( "select __key__,symbol,sharePrice from pulsar.\"public/default\".\"" + tableName diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestPulsarSQLBase.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestPulsarSQLBase.java index 2833be67bc8e3..862aad3b1e515 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestPulsarSQLBase.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestPulsarSQLBase.java @@ -264,7 +264,7 @@ private void validateData(TopicName topicName, int messageNum, Schema schema) log.info("Executing query: result for topic {} returnedTimestamps size: {}", topic, returnedTimestamps.size()); assertThat(returnedTimestamps.size()).isEqualTo(0); - int count = selectCount(namespace, topic); + int count = selectCount(namespace, topic, null); assertThat(count).isGreaterThan(messageNum - 2); } @@ -314,8 +314,13 @@ private static void printCurrent(ResultSet rs) throws SQLException { } - protected int selectCount(String namespace, String tableName) throws SQLException { - String query = String.format("select count(*) from pulsar.\"%s\".\"%s\"", namespace, tableName); + protected int selectCount(String namespace, String tableName, String condition) throws SQLException { + if (condition != null) { + condition = " where " + condition; + } else { + condition = ""; + } + String query = String.format("select count(*) from pulsar.\"%s\".\"%s\"%s", namespace, tableName, condition); log.info("Executing count query: {}", query); ResultSet res = connection.createStatement().executeQuery(query); res.next(); From 3f40576496ae9e51a1bb4bfa67a685db870d89a4 Mon Sep 17 00:00:00 2001 From: rangao Date: Mon, 20 Feb 2023 13:11:49 +0800 Subject: [PATCH 11/15] fix logic --- .../pulsar/sql/presto/PulsarRecordCursor.java | 36 ++++++++++++++----- .../pulsar/sql/presto/PulsarSplitManager.java | 34 ++++++++++-------- .../integration/presto/TestBasicPresto.java | 7 ++-- 3 files changed, 51 insertions(+), 26 deletions(-) diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java index 435e08f26b8b0..b8abc791cf820 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java @@ -979,15 +979,15 @@ class CompactedLedgerReader implements AsyncCallback.ReadCallback, Runnable { private final AtomicBoolean havePendingRead = new AtomicBoolean(false); private long startEntry = 0; private long readEntry = 0; - private MessageIdData lastMessageId; + private MessageIdData firstUnCompactedMessageId; private long compactedLedgerId = -1; private LedgerHandle compactedLedgerHandle; CompactedLedgerReader() { queue = new SpscArrayQueue<>((int) (readSize * 1.5)); - lastMessageId = new MessageIdData(); - lastMessageId.setLedgerId(-1); - lastMessageId.setEntryId(-1); + firstUnCompactedMessageId = new MessageIdData(); + firstUnCompactedMessageId.setLedgerId(-1); + firstUnCompactedMessageId.setEntryId(-1); try { initCompactedRead(); } catch (Exception e) { @@ -1014,8 +1014,11 @@ private void readMoreEntriesIfNeed() { if (havePendingRead.get()) { return; } - havePendingRead.set(true); + if (startEntry >= compactedLedgerHandle.getLastAddConfirmed()) { + return; + } if (queue.size() < readSize / 2) { + havePendingRead.set(true); long endEntry = Math.min(compactedLedgerHandle.getLastAddConfirmed(), startEntry + readSize); compactedLedgerHandle.asyncReadEntries(startEntry, endEntry, this, null); } @@ -1023,6 +1026,10 @@ private void readMoreEntriesIfNeed() { @Override public void readComplete(int i, LedgerHandle ledgerHandle, Enumeration enumeration, Object o) { + if (enumeration == null) { + havePendingRead.set(false); + return; + } while (enumeration.hasMoreElements()) { LedgerEntry ledgerEntry = enumeration.nextElement(); queue.offer(ledgerEntry); @@ -1040,8 +1047,11 @@ public void run() { private void readCompactedData() { if (compactedLedgerId == -1 || compactedLedgerHandle == null) { + log.info("[%s] Compacted ledger is not exist.", topicName); return; } + + log.info("[%s] Start to read compacted data, compacted ledger is %d.", topicName, compactedLedgerId); while (readEntry < compactedLedgerHandle.getLastAddConfirmed()) { readMoreEntriesIfNeed(); LedgerEntry ledgerEntry = queue.poll(); @@ -1057,8 +1067,8 @@ private void readCompactedData() { ByteBuf buffer = ledgerEntry.getEntryBuffer(); int idSize = buffer.readInt(); - lastMessageId = new MessageIdData(); - lastMessageId.parseFrom(buffer, idSize); + firstUnCompactedMessageId = new MessageIdData(); + firstUnCompactedMessageId.parseFrom(buffer, idSize); int payloadAndMetadataSize = buffer.readInt(); ByteBuf metadataAndPayload = buffer.slice(buffer.readerIndex(), payloadAndMetadataSize); MessageMetadata messageMetadata = Commands.parseMessageMetadata(metadataAndPayload); @@ -1077,6 +1087,7 @@ private void readCompactedData() { } readEntry = ledgerEntry.getEntryId(); } + log.info("[%s] Finish to read compacted data, read entry is %d.", topicName, readEntry); } private void handleCompactedBatchData(long ledgerId, long entryId, ByteBuf payload) throws IOException { @@ -1103,12 +1114,15 @@ private void handleCompactedBatchData(long ledgerId, long entryId, ByteBuf paylo } private void readUnCompactedData() { + log.info("[%s] Start to read unCompacted data, first unCompacted message id is %s.", topicName, + firstUnCompactedMessageId.getLedgerId() + ":" + firstUnCompactedMessageId.getEntryId()); try { PositionImpl startPosition; - if (lastMessageId.getLedgerId() == -1 && lastMessageId.getEntryId() == -1) { + if (firstUnCompactedMessageId.getLedgerId() == -1 && firstUnCompactedMessageId.getEntryId() == -1) { startPosition = PositionImpl.EARLIEST; } else { - startPosition = PositionImpl.get(lastMessageId.getLedgerId(), lastMessageId.getEntryId()).getNext(); + startPosition = PositionImpl.get(firstUnCompactedMessageId.getLedgerId(), + firstUnCompactedMessageId.getEntryId()).getNext(); } cursor = getCursor(topicName, startPosition, managedLedgerFactory, managedLedgerConfig); @@ -1132,9 +1146,12 @@ private void readUnCompactedData() { throw new RuntimeException(e); } } + log.info("[%s] Finish to read unCompacted data.", topicName); } private void readCompleteData() { + log.info("[%s] Start to read complete data, compacted messages count %d.", + topicName, compactedMessageIds.size()); Iterator messageIdIterator = compactedMessageIds.values().stream().sorted().iterator(); LedgerHandle readLedgerHandle = null; AtomicBoolean isCompactedLedger = new AtomicBoolean(false); @@ -1151,6 +1168,7 @@ private void readCompleteData() { log.warn("Failed to close ledger handle for ledger %s.", readLedgerHandle.getId()); } } + log.info("[%s] Finish to read complete data.", topicName); } private LedgerHandle getLedgerHandle(MessageIdImpl messageId, diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplitManager.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplitManager.java index cc0c1c67c5429..08ad28f444d96 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplitManager.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplitManager.java @@ -80,6 +80,8 @@ public class PulsarSplitManager implements ConnectorSplitManager { private final ObjectMapper objectMapper = new ObjectMapper(); + private static final String COMPACTION_SUBSCRIPTION = "__compaction"; + @Inject public PulsarSplitManager(PulsarConnectorId connectorId, PulsarConnectorConfig pulsarConnectorConfig) { this.connectorId = requireNonNull(connectorId, "connectorId is null").toString(); @@ -127,12 +129,9 @@ public ConnectorSplitSource getSplits(ConnectorTransactionHandle transactionHand AtomicReference readCompactedTypeReference = new AtomicReference<>(false); tupleDomain.getDomains().ifPresent(domainMap -> { - log.info("tuple domain map " + domainMap); Domain readCompactedDomain = domainMap.get( PulsarInternalColumn.COMPACTED_QUERY.getColumnHandle(connectorId, false)); - log.info("tuple readCompactedDomain " + readCompactedDomain); if (readCompactedDomain != null) { - log.info("tuple readCompactedDomain value " + readCompactedDomain.getSingleValue()); readCompactedTypeReference.set((Boolean) readCompactedDomain.getSingleValue()); } @@ -298,9 +297,9 @@ Collection getSplitsForTopic(TopicName topicName, if (readCompacted) { PersistentTopicInternalStats internalStats = pulsarAdmin.topics() .getInternalStats(topicName.toString()); - if (internalStats.cursors.containsKey("Compaction")) { + if (internalStats.cursors.containsKey(COMPACTION_SUBSCRIPTION)) { String[] compactedHorizonArr = - internalStats.cursors.get("Compaction").markDeletePosition.split(":"); + internalStats.cursors.get(COMPACTION_SUBSCRIPTION).markDeletePosition.split(":"); cursorStartPos = PositionImpl.get( Long.parseLong(compactedHorizonArr[0]), Long.parseLong(compactedHorizonArr[1])); } @@ -313,17 +312,21 @@ Collection getSplitsForTopic(TopicName topicName, cursorStartPos, managedLedgerConfig); long numEntries = readOnlyCursor.getNumberOfEntries(); - if (numEntries <= 0) { + if (numEntries <= 0 && !readCompacted) { return Collections.emptyList(); } - PredicatePushdownInfo predicatePushdownInfo = PredicatePushdownInfo.getPredicatePushdownInfo( - this.connectorId, - tupleDomain, - managedLedgerFactory, - managedLedgerConfig, - topicNamePersistenceEncoding, - numEntries); + PredicatePushdownInfo predicatePushdownInfo = null; + if (numEntries > 0) { + predicatePushdownInfo = PredicatePushdownInfo.getPredicatePushdownInfo( + this.connectorId, + tupleDomain, + managedLedgerFactory, + managedLedgerConfig, + topicNamePersistenceEncoding, + numEntries, + cursorStartPos); + } PositionImpl initialStartPosition; if (predicatePushdownInfo != null) { @@ -396,14 +399,15 @@ public static PredicatePushdownInfo getPredicatePushdownInfo(String connectorId, ManagedLedgerFactory managedLedgerFactory, ManagedLedgerConfig managedLedgerConfig, String topicNamePersistenceEncoding, - long totalNumEntries) throws + long totalNumEntries, + Position startPos) throws ManagedLedgerException, InterruptedException { ReadOnlyCursor readOnlyCursor = null; try { readOnlyCursor = managedLedgerFactory.openReadOnlyCursor( topicNamePersistenceEncoding, - PositionImpl.EARLIEST, managedLedgerConfig); + startPos, managedLedgerConfig); if (tupleDomain.getDomains().isPresent()) { Domain domain = tupleDomain.getDomains().get().get(PulsarInternalColumn.PUBLISH_TIME diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestBasicPresto.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestBasicPresto.java index 7b5e18c969aea..faa0aab9e20b9 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestBasicPresto.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestBasicPresto.java @@ -419,8 +419,11 @@ private void testCompactedQuery(boolean enableBatch, int compactedCount, int noC } prepareDataForCompactedQuery(producer, latestStocks, noCompactedCount, divisor, removeKey); - Assert.assertEquals(selectCount("public/default", tableName, "__compacted_query__=true"), - compactedCount > 0 || noCompactedCount > 0 ? divisor : 0); + Awaitility.await() + .pollInterval(100, TimeUnit.MILLISECONDS) + .atMost(5, TimeUnit.SECONDS) + .until(() -> selectCount("public/default", tableName, "__compacted_query__=true") == + (compactedCount > 0 || noCompactedCount > 0 ? divisor - 1 : 0)); ContainerExecResult result = execQuery( "select __key__,symbol,sharePrice from pulsar.\"public/default\".\"" + tableName + "\" where __compacted_query__=true"); From effb9718cbb7ca1d122b5bbeb46f6d1de3e24e34 Mon Sep 17 00:00:00 2001 From: rangao Date: Tue, 21 Feb 2023 10:25:47 +0800 Subject: [PATCH 12/15] fix logic --- .../pulsar/sql/presto/PulsarRecordCursor.java | 21 +++++++++------- .../pulsar/sql/presto/PulsarSplitManager.java | 4 +++- .../integration/presto/TestBasicPresto.java | 24 +++++++++++++------ 3 files changed, 32 insertions(+), 17 deletions(-) diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java index b8abc791cf820..18435eae0d38f 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java @@ -362,7 +362,7 @@ public void accept(Entry entry) { if (isCompactedQuery && message.getKey().isPresent()) { RawMessageIdImpl messageId = (RawMessageIdImpl) message.getMessageId(); - cacheCmpactedMessageIds(message.getKey().get(), + cacheCompactedMessageIds(message.getKey().get(), new BatchMessageIdImpl( messageId.getLedgerId(), messageId.getEntryId(), @@ -961,7 +961,7 @@ public void recycle() { } } - private void cacheCmpactedMessageIds(String key, BatchMessageIdImpl messageId, long payloadSize) { + private void cacheCompactedMessageIds(String key, BatchMessageIdImpl messageId, long payloadSize) { if (key == null) { return; } @@ -978,7 +978,7 @@ class CompactedLedgerReader implements AsyncCallback.ReadCallback, Runnable { private final SpscArrayQueue queue; private final AtomicBoolean havePendingRead = new AtomicBoolean(false); private long startEntry = 0; - private long readEntry = 0; + private long readEntry = -1; private MessageIdData firstUnCompactedMessageId; private long compactedLedgerId = -1; private LedgerHandle compactedLedgerHandle; @@ -1014,7 +1014,7 @@ private void readMoreEntriesIfNeed() { if (havePendingRead.get()) { return; } - if (startEntry >= compactedLedgerHandle.getLastAddConfirmed()) { + if (startEntry > compactedLedgerHandle.getLastAddConfirmed()) { return; } if (queue.size() < readSize / 2) { @@ -1051,7 +1051,8 @@ private void readCompactedData() { return; } - log.info("[%s] Start to read compacted data, compacted ledger is %d.", topicName, compactedLedgerId); + log.info("[%s] Start to read compacted data, compacted ledger is %d, the LAC of compacted ledger is %d.", + topicName, compactedLedgerId, compactedLedgerHandle.getLastAddConfirmed()); while (readEntry < compactedLedgerHandle.getLastAddConfirmed()) { readMoreEntriesIfNeed(); LedgerEntry ledgerEntry = queue.poll(); @@ -1081,13 +1082,14 @@ private void readCompactedData() { throw new RuntimeException(e); } } else { - cacheCmpactedMessageIds(messageMetadata.getPartitionKey(), new BatchMessageIdImpl( + cacheCompactedMessageIds(messageMetadata.getPartitionKey(), new BatchMessageIdImpl( ledgerEntry.getLedgerId(), ledgerEntry.getEntryId(), partition, 0), messageMetadata.getUncompressedSize()); } readEntry = ledgerEntry.getEntryId(); } - log.info("[%s] Finish to read compacted data, read entry is %d.", topicName, readEntry); + log.info("[%s] Finish to read compacted data, read entry is %s, compacted messages count %d.", + topicName, compactedLedgerId + ":" + readEntry, compactedMessageIds.size()); } private void handleCompactedBatchData(long ledgerId, long entryId, ByteBuf payload) throws IOException { @@ -1107,7 +1109,7 @@ private void handleCompactedBatchData(long ledgerId, long entryId, ByteBuf paylo continue; } BatchMessageIdImpl id = new BatchMessageIdImpl(ledgerId, entryId, partition, i); - cacheCmpactedMessageIds(smm.getPartitionKey(), id, smm.getPayloadSize()); + cacheCompactedMessageIds(smm.getPartitionKey(), id, smm.getPayloadSize()); singleMessagePayload.release(); } uncompressedPayload.release(); @@ -1146,7 +1148,8 @@ private void readUnCompactedData() { throw new RuntimeException(e); } } - log.info("[%s] Finish to read unCompacted data.", topicName); + log.info("[%s] Finish to read unCompacted data, read position is %s.", + topicName, cursor.getReadPosition().toString()); } private void readCompleteData() { diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplitManager.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplitManager.java index 08ad28f444d96..28e07f52bb554 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplitManager.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplitManager.java @@ -350,7 +350,9 @@ Collection getSplitsForTopic(TopicName topicName, for (int i = 0; i < numSplits; i++) { long entriesForSplit = (remainder > i) ? avgEntriesPerSplit + 1 : avgEntriesPerSplit; PositionImpl startPosition = (PositionImpl) readOnlyCursor.getReadPosition(); - readOnlyCursor.skipEntries(Math.toIntExact(entriesForSplit)); + if (entriesForSplit > 0) { + readOnlyCursor.skipEntries(Math.toIntExact(entriesForSplit)); + } PositionImpl endPosition = (PositionImpl) readOnlyCursor.getReadPosition(); PulsarSplit pulsarSplit = new PulsarSplit(i, this.connectorId, diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestBasicPresto.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestBasicPresto.java index faa0aab9e20b9..d3bb94ba88903 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestBasicPresto.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestBasicPresto.java @@ -24,6 +24,7 @@ import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; +import com.fasterxml.jackson.databind.ObjectMapper; import java.nio.ByteBuffer; import java.util.HashMap; import java.util.Map; @@ -45,6 +46,7 @@ import org.apache.pulsar.client.impl.schema.KeyValueSchemaImpl; import org.apache.pulsar.client.impl.schema.ProtobufNativeSchema; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.schema.KeyValue; import org.apache.pulsar.common.schema.KeyValueEncodingType; @@ -379,8 +381,8 @@ public void testQueueBigEntry() throws Exception { public Object[][] compactedQueryProvider() { return new Object[][] { {0, 0}, - {100, 0}, {0, 100}, + {100, 0}, {100, 100} }; } @@ -418,12 +420,20 @@ private void testCompactedQuery(boolean enableBatch, int compactedCount, int noC }); } prepareDataForCompactedQuery(producer, latestStocks, noCompactedCount, divisor, removeKey); + assertEquals(latestStocks.size(), (compactedCount > 0 || noCompactedCount > 0 ? divisor : 0)); + + PersistentTopicInternalStats internalStats = pulsarAdmin.topics().getInternalStats(topic); + log.info("check internal stats for topic {}", topic); + log.info(new ObjectMapper().writerWithDefaultPrettyPrinter().writeValueAsString(internalStats)); Awaitility.await() - .pollInterval(100, TimeUnit.MILLISECONDS) + .pollInterval(1, TimeUnit.SECONDS) .atMost(5, TimeUnit.SECONDS) - .until(() -> selectCount("public/default", tableName, "__compacted_query__=true") == - (compactedCount > 0 || noCompactedCount > 0 ? divisor - 1 : 0)); + .until(() -> { + int count = selectCount("public/default", tableName, "__compacted_query__=true"); + log.info("select count result for table {} is {}.", tableName, count); + return count == (compactedCount > 0 || noCompactedCount > 0 ? divisor - 1 : 0); + }); ContainerExecResult result = execQuery( "select __key__,symbol,sharePrice from pulsar.\"public/default\".\"" + tableName + "\" where __compacted_query__=true"); @@ -437,10 +447,10 @@ private void testCompactedQuery(boolean enableBatch, int compactedCount, int noC String[] dataArray = result.getStdout().split("\n"); for (String data : dataArray) { String[] columns = data.split(","); - Stock expectedStock = latestStocks.remove(columns[0].trim()); + Stock expectedStock = latestStocks.remove(columns[0].replace("\"", "")); assertNotNull(expectedStock); - assertEquals(columns[1], expectedStock.getSymbol()); - assertEquals(columns[2], "" + expectedStock.getSharePrice()); + assertEquals(columns[1].replace("\"", ""), expectedStock.getSymbol()); + assertEquals(columns[2].replace("\"", ""), "" + expectedStock.getSharePrice()); } assertEquals(1, latestStocks.size()); assertNotNull(latestStocks.remove(removeKey)); From 08d0e70ea9709be58813128b5e8cf261bfab51ec Mon Sep 17 00:00:00 2001 From: rangao Date: Tue, 21 Feb 2023 15:28:41 +0800 Subject: [PATCH 13/15] fix logic --- .../java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java index 18435eae0d38f..cc8e361a7e16f 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java @@ -456,7 +456,8 @@ public void run() { ReadOnlyCursorImpl readOnlyCursorImpl = ((ReadOnlyCursorImpl) cursor); // check if ledger is offloaded - if (!readOffloaded && readOnlyCursorImpl.getCurrentLedgerInfo().hasOffloadContext()) { + if (!readOffloaded && readOnlyCursorImpl.getCurrentLedgerInfo() != null && + readOnlyCursorImpl.getCurrentLedgerInfo().hasOffloadContext()) { log.warn( "Ledger %s is offloaded for topic %s. Ignoring it because offloader is not configured", readOnlyCursorImpl.getCurrentLedgerInfo().getLedgerId(), pulsarSplit.getTableName()); From 586cf6f3e94f5c9fcd33dc3ca65b3037a8bb29a6 Mon Sep 17 00:00:00 2001 From: rangao Date: Tue, 21 Feb 2023 16:04:38 +0800 Subject: [PATCH 14/15] fix checkstyle --- .../java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java index cc8e361a7e16f..8645696d3c003 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java @@ -456,8 +456,8 @@ public void run() { ReadOnlyCursorImpl readOnlyCursorImpl = ((ReadOnlyCursorImpl) cursor); // check if ledger is offloaded - if (!readOffloaded && readOnlyCursorImpl.getCurrentLedgerInfo() != null && - readOnlyCursorImpl.getCurrentLedgerInfo().hasOffloadContext()) { + if (!readOffloaded && readOnlyCursorImpl.getCurrentLedgerInfo() != null + && readOnlyCursorImpl.getCurrentLedgerInfo().hasOffloadContext()) { log.warn( "Ledger %s is offloaded for topic %s. Ignoring it because offloader is not configured", readOnlyCursorImpl.getCurrentLedgerInfo().getLedgerId(), pulsarSplit.getTableName()); From 48f08ac820ebddd41ee5b032b59557f859db9977 Mon Sep 17 00:00:00 2001 From: rangao Date: Tue, 21 Feb 2023 21:34:58 +0800 Subject: [PATCH 15/15] fix test --- .../org/apache/pulsar/sql/presto/TestCacheSizeAllocator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestCacheSizeAllocator.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestCacheSizeAllocator.java index 57079190a8891..a04b7396d3e12 100644 --- a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestCacheSizeAllocator.java +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestCacheSizeAllocator.java @@ -115,7 +115,7 @@ public void cacheSizeAllocatorTest(long entryQueueSizeBytes) throws Exception { TupleDomain.all(), objectMapper.writeValueAsString(new HashMap<>()), null, - null); + false); List pulsarColumnHandles = TestPulsarConnector.getColumnColumnHandles( topicName, Schema.BYTES.getSchemaInfo(), PulsarColumnHandle.HandleKeyValueType.NONE, true);