Skip to content
Merged
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 @@ -222,6 +222,27 @@ static ClientBuilder builder() {
*/
<T> ReaderBuilder<T> newReader(Schema<T> schema);

/**
* Create a table view builder with a specific schema for subscribing on a specific topic.
*
* <p>The TableView provides a key-value map view of a compacted topic. Messages without keys will
* be ignored.
*
* <p>Example:
* <pre>{@code
* TableView<byte[]> tableView = client.newTableView(Schema.BYTES)
* .topic("my-topic")
* .autoUpdatePartitionsInterval(5, TimeUnit.SECONDS)
* .create();
*
* tableView.forEach((k, v) -> System.out.println(k + ":" + v));
* }</pre>
*
* @param schema provide a way to convert between serialized data and domain objects
* @return a {@link TableViewBuilder} object to configure and construct the {@link TableView} instance
*/
<T> TableViewBuilder<T> newTableViewBuilder(Schema<T> schema);

/**
* Update the service URL this client is using.
*
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,106 @@
/**
* 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.client.api;

import java.io.Closeable;
import java.util.Collection;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.function.BiConsumer;

public interface TableView<T> extends Closeable {

/**
* Returns the number of key-value mappings in the {@link TableView}.
*
* @return the number of key-value mappings in this TableView
*/
int size();

/**
* Returns {@code true} if this {@link TableView} contains no key-value mappings.
*
* @return true if this TableView contains no key-value mappings
*/
boolean isEmpty();

/**
* Returns {@code true} if this {@link TableView} contains a mapping for the specified
* key.
*
* @param key key whose presence in this map is to be tested
* @return true if this map contains a mapping for the specified key
*/
boolean containsKey(String key);

/**
* Returns the value to which the specified key is mapped, or null if this map contains
* no mapping for the key.
*
* @param key the key whose associated value is to be returned
* @return the value associated with the key or null if the keys was not found
*/
T get(String key);

/**
* Returns a Set view of the mappings contained in this map.
*
* @return a set view of the mappings contained in this map
*/
Set<Map.Entry<String, T>> entrySet();

/**
* Returns a {@link Set} view of the keys contained in this {@link TableView}.
*
* @return a set view of the keys contained in this map
*/
Set<String> keySet();

/**
* Returns a Collection view of the values contained in this {@link TableView}.
*
* @return a collection view of the values contained in this TableView
*/
Collection<T> values();

/**
* Performs the given action for each entry in this map until all entries
* have been processed or the action throws an exception.
*
* @param action The action to be performed for each entry
*/
void forEach(BiConsumer<String, T> action);

/**
* Performs the give action for each entry in this map until all entries
* have been processed or the action throws an exception.
*
* @param action The action to be performed for each entry
*/
void forEachAndListen(BiConsumer<String, T> action);

/**
* Close the table view and releases resources allocated.
*
* @return a future that can used to track when the table view has been closed.
*/
CompletableFuture<Void> closeAsync();
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,96 @@
/**
* 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.client.api;

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

import java.util.Map;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;

/**
* {@link TableViewBuilder} is used to configure and create instances of {@link TableView}.
*
* @see PulsarClient#newTableViewBuilder(Schema) ()
*
* @since 2.10.0
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public interface TableViewBuilder<T> {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The builder is part of the API so we should also have Javadoc in here


/**
* Load the configuration from provided <tt>config</tt> map.
*
* <p>Example:
*
* <pre>{@code
* Map<String, Object> config = new HashMap<>();
* config.put("topicName", "test-topic");
* config.put("autoUpdatePartitionsSeconds", "300");
*
* TableViewBuilder<byte[]> builder = ...;
* builder = builder.loadConf(config);
*
* TableView<byte[]> tableView = builder.create();
* }</pre>
*
* @param config configuration to load
* @return the {@link TableViewBuilder} instance
*/
TableViewBuilder<T> loadConf(Map<String, Object> config);

/**
* Finalize the creation of the {@link TableView} instance.
*
* <p>This method will block until the tableView is created successfully or an exception is thrown.
*
* @return the {@link TableView} instance
* @throws PulsarClientException
* if the tableView creation fails
*/
TableView<T> create() throws PulsarClientException;

/**
* Finalize the creation of the {@link TableView} instance in asynchronous mode.
*
* <p>This method will return a {@link CompletableFuture} that can be used to access the instance when it's ready.
*
* @return the {@link TableView} instance
*/
CompletableFuture<TableView<T>> createAsync();

/**
* Set the topic name of the {@link TableView}
*
* @param topic the name of the topic to create the {@link TableView}
* @return the {@link TableViewBuilder} builder instance
*/
TableViewBuilder<T> topic(String topic);

/**
* Set the interval of updating partitions <i>(default: 1 minute)</i>
* @param interval the interval of updating partitions
* @param unit the time unit of the interval
* @return the {@link TableViewBuilder} builder instance
*/
TableViewBuilder<T> autoUpdatePartitionsInterval(int interval, TimeUnit unit);
}
Original file line number Diff line number Diff line change
Expand Up @@ -64,6 +64,7 @@
import org.apache.pulsar.client.api.RegexSubscriptionMode;
import org.apache.pulsar.client.api.Schema;
import org.apache.pulsar.client.api.SubscriptionType;
import org.apache.pulsar.client.api.TableViewBuilder;
import org.apache.pulsar.client.api.schema.KeyValueSchema;
import org.apache.pulsar.client.api.schema.SchemaInfoProvider;
import org.apache.pulsar.client.api.transaction.TransactionBuilder;
Expand Down Expand Up @@ -259,6 +260,11 @@ public <T> ReaderBuilder<T> newReader(Schema<T> schema) {
return new ReaderBuilderImpl<>(this, schema);
}

@Override
public <T> TableViewBuilder<T> newTableViewBuilder(Schema<T> schema) {
return new TableViewBuilderImpl<>(this, schema);
}

public CompletableFuture<Producer<byte[]>> createProducerAsync(ProducerConfigurationData conf) {
return createProducerAsync(conf, Schema.BYTES, null);
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,81 @@
/**
* 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.client.impl;

import org.apache.commons.lang3.StringUtils;
import org.apache.pulsar.client.api.PulsarClientException;
import org.apache.pulsar.client.api.Schema;
import org.apache.pulsar.client.api.TableView;
import org.apache.pulsar.client.api.TableViewBuilder;
import org.apache.pulsar.client.impl.conf.ConfigurationDataUtils;

import java.util.Map;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;

import static com.google.common.base.Preconditions.checkArgument;

public class TableViewBuilderImpl<T> implements TableViewBuilder<T> {

private final PulsarClientImpl client;
private final Schema<T> schema;
private TableViewConfigurationData conf;

TableViewBuilderImpl(PulsarClientImpl client, Schema<T> schema) {
this.client = client;
this.schema = schema;
this.conf = new TableViewConfigurationData();
}

@Override
public TableViewBuilder<T> loadConf(Map<String, Object> config) {
conf = ConfigurationDataUtils.loadData(
config, conf, TableViewConfigurationData.class);
return this;
}

@Override
public TableView<T> create() throws PulsarClientException {
try {
return createAsync().get();
} catch (Exception e) {
throw PulsarClientException.unwrap(e);
}
}

@Override
public CompletableFuture<TableView<T>> createAsync() {
return new TableViewImpl<>(client, schema, conf).start();
}

@Override
public TableViewBuilder<T> topic(String topic) {
checkArgument(StringUtils.isNotBlank(topic), "topic cannot be blank");
conf.setTopicName(StringUtils.trim(topic));
return this;
}

@Override
public TableViewBuilder<T> autoUpdatePartitionsInterval(int interval, TimeUnit unit) {
checkArgument(unit.toSeconds(interval) >= 1, "minimum is 1 second");
conf.setAutoUpdatePartitionsSeconds(unit.toSeconds(interval));
return this;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,46 @@
/**
* 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.client.impl;

import lombok.Data;
import lombok.NoArgsConstructor;

import java.io.Serializable;

@Data
@NoArgsConstructor
public class TableViewConfigurationData implements Serializable, Cloneable {
private static final long serialVersionUID = 1L;

private String topicName = null;
private long autoUpdatePartitionsSeconds = 60;

@Override
public TableViewConfigurationData clone() {
try {
TableViewConfigurationData clone = (TableViewConfigurationData) super.clone();
clone.setTopicName(topicName);
clone.setAutoUpdatePartitionsSeconds(autoUpdatePartitionsSeconds);
return clone;
} catch (CloneNotSupportedException e) {
throw new AssertionError();
}
}
}
Loading