Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,6 @@
import org.apache.pulsar.client.api.PulsarClientException;
import org.apache.pulsar.client.api.Reader;
import org.apache.pulsar.client.api.Schema;
import org.apache.pulsar.client.api.TableView;
import org.apache.pulsar.client.api.TopicMessageId;
import org.apache.pulsar.common.naming.TopicDomain;
import org.apache.pulsar.common.naming.TopicName;
Expand All @@ -67,7 +66,7 @@
import org.testng.annotations.Test;

/**
* Unit test for {@link org.apache.pulsar.client.impl.TableViewImpl}.
* Unit test for {@link TableView}.
*/
@Slf4j
@Test(groups = "broker-impl")
Expand Down Expand Up @@ -169,7 +168,7 @@ public void testRefreshAPI(int partition) throws Exception {
}

@Cleanup
TableView<byte[]> tv = pulsarClient.newTableView(Schema.BYTES)
org.apache.pulsar.client.api.TableView<byte[]> tv = pulsarClient.newTableView(Schema.BYTES)
.topic(topic)
.create();
// Verify refresh can handle the case when the topic is empty
Expand Down Expand Up @@ -225,11 +224,11 @@ public void testRefreshTaskCanBeCompletedWhenReaderClosed() throws Exception {
String topic2 = "persistent://public/default/testRefreshTaskCanBeCompletedWhenReaderClosed-2";
admin.topics().createNonPartitionedTopic(topic2);
@Cleanup
TableView<byte[]> tv1 = pulsarClient.newTableView(Schema.BYTES)
org.apache.pulsar.client.api.TableView<byte[]> tv1 = pulsarClient.newTableView(Schema.BYTES)
.topic(topic1)
.create();
@Cleanup
TableView<byte[]> tv2 = pulsarClient.newTableView(Schema.BYTES)
org.apache.pulsar.client.api.TableView<byte[]> tv2 = pulsarClient.newTableView(Schema.BYTES)
.topic(topic1)
.create();
// 2. Slow down the rate of reading messages.
Expand Down Expand Up @@ -269,7 +268,7 @@ public void testTableView() throws Exception {
int count = 20;
Set<String> keys = this.publishMessages(topic, count, false);
@Cleanup
TableView<byte[]> tv = pulsarClient.newTableViewBuilder(Schema.BYTES)
org.apache.pulsar.client.api.TableView<byte[]> tv = pulsarClient.newTableViewBuilder(Schema.BYTES)
.topic(topic)
.autoUpdatePartitionsInterval(60, TimeUnit.SECONDS)
.create();
Expand Down Expand Up @@ -316,7 +315,7 @@ public void testNewTableView() throws Exception {
admin.topics().createPartitionedTopic(topic, 2);
Set<String> keys = this.publishMessages(topic, 10, false);
@Cleanup
TableView<byte[]> tv = pulsarClient.newTableView()
org.apache.pulsar.client.api.TableView<byte[]> tv = pulsarClient.newTableView()
.topic(topic)
.autoUpdatePartitionsInterval(60, TimeUnit.SECONDS)
.create();
Expand All @@ -336,7 +335,7 @@ public void testTableViewUpdatePartitions(String topicDomain) throws Exception {
// For non-persistent topic, this keys will never be received.
Set<String> keys = this.publishMessages(topic, count, false);
@Cleanup
TableView<byte[]> tv = pulsarClient.newTableViewBuilder(Schema.BYTES)
org.apache.pulsar.client.api.TableView<byte[]> tv = pulsarClient.newTableViewBuilder(Schema.BYTES)
.topic(topic)
.autoUpdatePartitionsInterval(5, TimeUnit.SECONDS)
.create();
Expand Down Expand Up @@ -374,7 +373,7 @@ public void testPublishNullValue(String topicDomain) throws Exception {
String topic = topicDomain + "://public/default/tableview-test-publish-null-value";
admin.topics().createPartitionedTopic(topic, 3);

final TableView<String> tv = pulsarClient.newTableViewBuilder(Schema.STRING)
final org.apache.pulsar.client.api.TableView<String> tv = pulsarClient.newTableViewBuilder(Schema.STRING)
.topic(topic)
.autoUpdatePartitionsInterval(5, TimeUnit.SECONDS)
.create();
Expand All @@ -397,7 +396,7 @@ public void testPublishNullValue(String topicDomain) throws Exception {
tv.close();

@Cleanup
TableView<String> tv1 = pulsarClient.newTableView(Schema.STRING)
org.apache.pulsar.client.api.TableView<String> tv1 = pulsarClient.newTableView(Schema.STRING)
.topic(topic)
.autoUpdatePartitionsInterval(5, TimeUnit.SECONDS)
.create();
Expand Down Expand Up @@ -427,7 +426,7 @@ public void testAck(boolean partitionedTopic) throws Exception {
}

@Cleanup
TableView<String> tv1 = pulsarClient.newTableViewBuilder(Schema.STRING)
org.apache.pulsar.client.api.TableView<String> tv1 = pulsarClient.newTableViewBuilder(Schema.STRING)
.topic(topic)
.autoUpdatePartitionsInterval(5, TimeUnit.SECONDS)
.create();
Expand Down Expand Up @@ -476,7 +475,7 @@ public void testListen() throws Exception {
}

@Cleanup
TableView<String> tv = pulsarClient.newTableViewBuilder(Schema.STRING)
org.apache.pulsar.client.api.TableView<String> tv = pulsarClient.newTableViewBuilder(Schema.STRING)
.topic(topic)
.autoUpdatePartitionsInterval(5, TimeUnit.SECONDS)
.create();
Expand Down Expand Up @@ -521,7 +520,7 @@ public void testTableViewWithEncryptedMessages() throws Exception {

// TableView can read them using the private key
@Cleanup
TableView<byte[]> tv = pulsarClient.newTableViewBuilder(Schema.BYTES)
org.apache.pulsar.client.api.TableView<byte[]> tv = pulsarClient.newTableViewBuilder(Schema.BYTES)
.topic(topic)
.autoUpdatePartitionsInterval(60, TimeUnit.SECONDS)
.defaultCryptoKeyReader("file:" + ECDSA_PRIVATE_KEY)
Expand All @@ -540,7 +539,7 @@ public void testTableViewTailMessageReadRetry() throws Exception {
String topic = "persistent://public/default/tableview-is-interrupted-test";
admin.topics().createNonPartitionedTopic(topic);
@Cleanup
TableView<byte[]> tv = pulsarClient.newTableView(Schema.BYTES)
org.apache.pulsar.client.api.TableView<byte[]> tv = pulsarClient.newTableView(Schema.BYTES)
.topic(topic)
.autoUpdatePartitionsInterval(60, TimeUnit.SECONDS)
.create();
Expand Down Expand Up @@ -609,12 +608,12 @@ public void testBuildTableViewWithMessagesAlwaysAvailable() throws Exception {
return CompletableFuture.completedFuture(message);
});
@Cleanup
TableViewImpl<byte[]> tableView = (TableViewImpl<byte[]>) pulsarClient.newTableView()
TableView<byte[]> tableView = (TableView<byte[]>) pulsarClient.newTableView()
.topic(topic)
.createAsync()
.get();
TableViewImpl<byte[]> mockTableView = spy(tableView);
Method readAllExistingMessagesMethod = TableViewImpl.class
TableView<byte[]> mockTableView = spy(tableView);
Method readAllExistingMessagesMethod = TableView.class
.getDeclaredMethod("readAllExistingMessages", Reader.class);
readAllExistingMessagesMethod.setAccessible(true);
CompletableFuture<Reader<?>> future =
Expand All @@ -624,4 +623,91 @@ public void testBuildTableViewWithMessagesAlwaysAvailable() throws Exception {
future.get(3, TimeUnit.SECONDS);
assertTrue(index.get() <= 0);
}

@Test
public void testCreateMapped() throws Exception {
String topic = "persistent://public/default/testCreateMapped";
admin.topics().createNonPartitionedTopic(topic);

@Cleanup
Producer<String> producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create();

@Cleanup
org.apache.pulsar.client.api.TableView<String> tableView = pulsarClient.newTableViewBuilder(Schema.STRING)
.topic(topic)
.createMapped(m -> {
if (m.getValue().equals("delete-me")) {
return null;
}
return m.getValue() + ":" + m.getProperty("myProp");
});

// Send a message to be mapped
String testKey = "key1";
String testValue = "value1";
producer.newMessage()
.key(testKey)
.value(testValue)
.property("myProp", "myValue")
.send();

Awaitility.await().atMost(5, TimeUnit.SECONDS).until(() -> tableView.size() == 1);

String mappedValue = tableView.get(testKey);
assertEquals(mappedValue, "value1:myValue");

// Send another message to update the value
producer.newMessage()
.key(testKey)
.value("value2")
.property("myProp", "myValue2")
.send();

Awaitility.await().atMost(5, TimeUnit.SECONDS)
.until(() -> "value2:myValue2".equals(tableView.get(testKey)));
assertEquals(tableView.size(), 1);

// Send a message that maps to null (tombstone)
producer.newMessage()
.key(testKey)
.value("delete-me")
.send();

Awaitility.await().atMost(5, TimeUnit.SECONDS).until(() -> tableView.size() == 0);
Assert.assertNull(tableView.get(testKey), "Value should be null after tombstone message");
}

@Test
public void testCreateMappedWithIdentityMapper() throws Exception {
String topic = "persistent://public/default/testCreateMappedWithIdentityMapper";
admin.topics().createNonPartitionedTopic(topic);

@Cleanup
Producer<String> producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create();

String testKey = "key1";
String testValue = "value1";
producer.newMessage()
.key(testKey)
.value(testValue)
.property("myProp", "myValue")
.send();

@Cleanup
org.apache.pulsar.client.api.TableView<Message<String>> tableView = pulsarClient.newTableViewBuilder(Schema.STRING)
.topic(topic)
.createMapped(java.util.function.Function.identity());

Awaitility.await().atMost(5, TimeUnit.SECONDS).until(() -> tableView.size() == 1);

Message<String> message = tableView.get(testKey);
Assert.assertNotNull(message, "Message should not be null for key: " + testKey);
assertEquals(message.getKey(), testKey);
assertEquals(message.getValue(), testValue);
assertEquals(message.getProperty("myProp"), "myValue");

Message<String> missingMessage = tableView.get("missingKey");
Assert.assertNull(missingMessage, "Message should be null for missing key");
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,7 @@ public interface TableView<T> extends Closeable {
*/
T get(String key);


/**
* Returns a Set view of the mappings contained in this map.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,8 @@
import java.util.Map;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
import java.util.function.Function;

import org.apache.pulsar.common.classification.InterfaceAudience;
import org.apache.pulsar.common.classification.InterfaceStability;

Expand Down Expand Up @@ -76,6 +78,45 @@ public interface TableViewBuilder<T> {
*/
CompletableFuture<TableView<T>> createAsync();

/**
* Creates a {@link TableView} instance where the values are the result of applying a user-defined
* `mapper` function to each message.
*
* <p>This provides a flexible way to create a key-value view over a topic, allowing users to extract data
* from the message payload, properties, and other metadata into a custom object {@code V}.
*
* <p>To get a view of the full {@link Message<T>} objects, {@code java.util.function.Function.identity()}
* can be used as the mapper.
*
* <p>If the `mapper` function returns `null`, it is treated as a tombstone message, and the
* corresponding key will be removed from the `TableView`.
*
* @param mapper A function that takes a {@link Message<T>} and returns a custom object of type {@code V}.
* @param <V> The type of the value in the TableView.
* @return the {@link TableView} instance
* @throws PulsarClientException if the tableView creation fails
*/
<V> TableView<V> createMapped(Function<Message<T>, V> mapper) throws PulsarClientException;

/**
* Creates a {@link TableView} instance in asynchronous mode where the values are the result of applying
* a user-defined `mapper` function to each message.
*
* <p>This provides a flexible way to create a key-value view over a topic, allowing users to extract data
* from the message payload, properties, and other metadata into a custom object {@code V}.
*
* <p>To get a view of the full {@link Message<T>} objects, {@code java.util.function.Function.identity()}
* can be used as the mapper.
*
* <p>If the `mapper` function returns `null`, it is treated as a tombstone message, and the
* corresponding key will be removed from the `TableView`.
*
* @param mapper A function that takes a {@link Message<T>} and returns a custom object of type {@code V}.
* @param <V> The type of the value in the TableView.
* @return a future that can be used to access the {@link TableView} instance when it's ready
*/
<V> CompletableFuture<TableView<V>> createMappedAsync(Function<Message<T>, V> mapper);

/**
* Set the topic name of the {@link TableView}.
*
Expand Down
Loading