@Override public PubsubClient newClient( @Nullable String timestampAttribute, @Nullable String idAttribute, PubsubOptions options) throws IOException { ManagedChannel channel = NettyChannelBuilder.forAddress(PUBSUB_ADDRESS, PUBSUB_PORT) .negotiationType(NegotiationType.TLS) .sslContext(GrpcSslContexts.forClient().ciphers(null).build()) .build(); return new PubsubGrpcClient( timestampAttribute, idAttribute, DEFAULT_TIMEOUT_S, channel, options.getGcpCredential()); }
/** Return a stub for making a publish request with a timeout. */ private PublisherBlockingStub publisherStub() throws IOException { if (cachedPublisherStub == null) { cachedPublisherStub = PublisherGrpc.newBlockingStub(newChannel()); } return cachedPublisherStub.withDeadlineAfter(timeoutSec, TimeUnit.SECONDS); }
@Override public List<TopicPath> listTopics(ProjectPath project) throws IOException { ListTopicsRequest.Builder request = ListTopicsRequest.newBuilder().setProject(project.getPath()).setPageSize(LIST_BATCH_SIZE); ListTopicsResponse response = publisherStub().listTopics(request.build()); if (response.getTopicsCount() == 0) { return ImmutableList.of(); } List<TopicPath> topics = new ArrayList<>(response.getTopicsCount()); while (true) { for (Topic topic : response.getTopicsList()) { topics.add(topicPathFromPath(topic.getName())); } if (response.getNextPageToken().isEmpty()) { break; } request.setPageToken(response.getNextPageToken()); response = publisherStub().listTopics(request.build()); } return topics; }
@Override public List<SubscriptionPath> listSubscriptions(ProjectPath project, TopicPath topic) throws IOException { ListSubscriptionsRequest.Builder request = ListSubscriptionsRequest.newBuilder() .setProject(project.getPath()) .setPageSize(LIST_BATCH_SIZE); ListSubscriptionsResponse response = subscriberStub().listSubscriptions(request.build()); if (response.getSubscriptionsCount() == 0) { return ImmutableList.of(); } List<SubscriptionPath> subscriptions = new ArrayList<>(response.getSubscriptionsCount()); while (true) { for (Subscription subscription : response.getSubscriptionsList()) { if (subscription.getTopic().equals(topic.getPath())) { subscriptions.add(subscriptionPathFromPath(subscription.getName())); } } if (response.getNextPageToken().isEmpty()) { break; } request.setPageToken(response.getNextPageToken()); response = subscriberStub().listSubscriptions(request.build()); } return subscriptions; }
.setMaxMessages(batchSize) .build(); PullResponse response = subscriberStub().pull(request); if (response.getReceivedMessagesCount() == 0) { return ImmutableList.of(); extractTimestamp(timestampAttribute, pubsubTimestampString, attributes);
@Override public void deleteSubscription(SubscriptionPath subscription) throws IOException { DeleteSubscriptionRequest request = DeleteSubscriptionRequest.newBuilder().setSubscription(subscription.getPath()).build(); subscriberStub().deleteSubscription(request); // ignore Empty result. }
@Override public void deleteTopic(TopicPath topic) throws IOException { DeleteTopicRequest request = DeleteTopicRequest.newBuilder().setTopic(topic.getPath()).build(); publisherStub().deleteTopic(request); // ignore Empty result. }
@Override public void acknowledge(SubscriptionPath subscription, List<String> ackIds) throws IOException { AcknowledgeRequest request = AcknowledgeRequest.newBuilder() .setSubscription(subscription.getPath()) .addAllAckIds(ackIds) .build(); subscriberStub().acknowledge(request); // ignore Empty result. }
@Override public void createTopic(TopicPath topic) throws IOException { Topic request = Topic.newBuilder().setName(topic.getPath()).build(); publisherStub().createTopic(request); // ignore Topic result. }
@Before public void setup() { channelName = String.format( "%s-%s", PubsubGrpcClientTest.class.getName(), ThreadLocalRandom.current().nextInt()); inProcessChannel = InProcessChannelBuilder.forName(channelName).directExecutor().build(); Credentials testCredentials = new TestCredential(); client = new PubsubGrpcClient( TIMESTAMP_ATTRIBUTE, ID_ATTRIBUTE, 10, inProcessChannel, testCredentials); }
@Override public int ackDeadlineSeconds(SubscriptionPath subscription) throws IOException { GetSubscriptionRequest request = GetSubscriptionRequest.newBuilder().setSubscription(subscription.getPath()).build(); Subscription response = subscriberStub().getSubscription(request); return response.getAckDeadlineSeconds(); }
@Override public int publish(TopicPath topic, List<OutgoingMessage> outgoingMessages) throws IOException { PublishRequest.Builder request = PublishRequest.newBuilder().setTopic(topic.getPath()); for (OutgoingMessage outgoingMessage : outgoingMessages) { PubsubMessage.Builder message = PubsubMessage.newBuilder().setData(ByteString.copyFrom(outgoingMessage.elementBytes)); if (outgoingMessage.attributes != null) { message.putAllAttributes(outgoingMessage.attributes); } if (timestampAttribute != null) { message .getMutableAttributes() .put(timestampAttribute, String.valueOf(outgoingMessage.timestampMsSinceEpoch)); } if (idAttribute != null && !Strings.isNullOrEmpty(outgoingMessage.recordId)) { message.getMutableAttributes().put(idAttribute, outgoingMessage.recordId); } request.addMessages(message); } PublishResponse response = publisherStub().publish(request.build()); return response.getMessageIdsCount(); }
/** Return a stub for making a subscribe request with a timeout. */ private SubscriberBlockingStub subscriberStub() throws IOException { if (cachedSubscriberStub == null) { cachedSubscriberStub = SubscriberGrpc.newBlockingStub(newChannel()); } return cachedSubscriberStub.withDeadlineAfter(timeoutSec, TimeUnit.SECONDS); }
@Override public void modifyAckDeadline( SubscriptionPath subscription, List<String> ackIds, int deadlineSeconds) throws IOException { ModifyAckDeadlineRequest request = ModifyAckDeadlineRequest.newBuilder() .setSubscription(subscription.getPath()) .addAllAckIds(ackIds) .setAckDeadlineSeconds(deadlineSeconds) .build(); subscriberStub().modifyAckDeadline(request); // ignore Empty result. }
@Override public void createSubscription( TopicPath topic, SubscriptionPath subscription, int ackDeadlineSeconds) throws IOException { Subscription request = Subscription.newBuilder() .setTopic(topic.getPath()) .setName(subscription.getPath()) .setAckDeadlineSeconds(ackDeadlineSeconds) .build(); subscriberStub().createSubscription(request); // ignore Subscription result. }