mirror of
https://github.com/nats-io/nats.go.git
synced 2025-10-05 16:48:43 +08:00
js: Add js.PullSubscribe and sub.Fetch APIs for pull consumers
Signed-off-by: Waldemar Quevedo <wally@synadia.com>
This commit is contained in:
649
test/js_test.go
649
test/js_test.go
@@ -457,18 +457,30 @@ func TestJetStreamSubscribe(t *testing.T) {
|
||||
// Now try pull based subscribers.
|
||||
|
||||
// Check some error conditions first.
|
||||
if _, err := js.Subscribe("bar", func(m *nats.Msg) {}, nats.Pull(1)); err != nats.ErrPullModeNotAllowed {
|
||||
t.Fatalf("Expected an error trying to do PullMode on callback based subscriber, got %v", err)
|
||||
if _, err := js.Subscribe("bar", nil); err != nats.ErrBadSubscription {
|
||||
t.Fatalf("Expected an error trying to create subscriber with nil callback, got %v", err)
|
||||
}
|
||||
|
||||
// Durable name is required for now.
|
||||
sub, err = js.PullSubscribe("bar")
|
||||
if err == nil {
|
||||
t.Fatalf("Unexpected success")
|
||||
}
|
||||
if err != nil && err.Error() != `consumer in pull mode requires a durable name` {
|
||||
t.Errorf("Expected consumer in pull mode error, got %v", err)
|
||||
}
|
||||
|
||||
batch := 5
|
||||
sub, err = js.SubscribeSync("bar", nats.Durable("rip"), nats.Pull(batch))
|
||||
sub, err = js.PullSubscribe("bar", nats.Durable("rip"))
|
||||
if err != nil {
|
||||
t.Fatalf("Unexpected error: %v", err)
|
||||
}
|
||||
|
||||
// The first batch if available should be delivered and queued up.
|
||||
waitForPending(t, batch)
|
||||
bmsgs, err := sub.Fetch(batch)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
if info, _ := sub.ConsumerInfo(); info.NumAckPending != batch || info.NumPending != uint64(batch) {
|
||||
t.Fatalf("Expected %d pending ack, and %d still waiting to be delivered, got %d and %d", batch, batch, info.NumAckPending, info.NumPending)
|
||||
@@ -476,27 +488,39 @@ func TestJetStreamSubscribe(t *testing.T) {
|
||||
|
||||
// Now go ahead and consume these and ack, but not ack+next.
|
||||
for i := 0; i < batch; i++ {
|
||||
m, err := sub.NextMsg(10 * time.Millisecond)
|
||||
m := bmsgs[i]
|
||||
err = m.Ack()
|
||||
if err != nil {
|
||||
t.Fatalf("Unexpected error: %v", err)
|
||||
t.Fatal(err)
|
||||
}
|
||||
m.Respond(nats.AckAck)
|
||||
}
|
||||
if info, _ := sub.ConsumerInfo(); info.AckFloor.Consumer != uint64(batch) {
|
||||
t.Fatalf("Expected ack floor to be %d, got %d", batch, info.AckFloor.Consumer)
|
||||
}
|
||||
|
||||
// Now we are stuck so to speak. So we can unstick the sub by calling poll.
|
||||
waitForPending(t, 0)
|
||||
sub.Poll()
|
||||
waitForPending(t, batch)
|
||||
|
||||
// Make a request for 10 but should only receive a few.
|
||||
bmsgs, err = sub.Fetch(10, nats.MaxWait(2*time.Second))
|
||||
if err != nil {
|
||||
t.Errorf("Unexpected error: %v", err)
|
||||
}
|
||||
got := len(bmsgs)
|
||||
expected := 5
|
||||
if got != expected {
|
||||
t.Errorf("Expected: %v, got: %v", expected, got)
|
||||
}
|
||||
|
||||
for _, msg := range bmsgs {
|
||||
msg.Ack()
|
||||
}
|
||||
|
||||
sub.Drain()
|
||||
|
||||
// Now test attaching to a pull based durable.
|
||||
|
||||
// Test that if we are attaching that the subjects will match up. rip from
|
||||
// above was created with a filtered subject of bar, so this should fail.
|
||||
_, err = js.SubscribeSync("baz", nats.Durable("rip"), nats.Pull(batch))
|
||||
_, err = js.PullSubscribe("baz", nats.Durable("rip"))
|
||||
if err != nats.ErrSubjectMismatch {
|
||||
t.Fatalf("Expected a %q error but got %q", nats.ErrSubjectMismatch, err)
|
||||
}
|
||||
@@ -506,49 +530,30 @@ func TestJetStreamSubscribe(t *testing.T) {
|
||||
js.Publish("bar", msg)
|
||||
}
|
||||
|
||||
sub, err = js.SubscribeSync("bar", nats.Durable("rip"), nats.Pull(batch))
|
||||
sub, err = js.PullSubscribe("bar", nats.Durable("rip"))
|
||||
if err != nil {
|
||||
t.Fatalf("Unexpected error: %v", err)
|
||||
}
|
||||
defer sub.Unsubscribe()
|
||||
|
||||
waitForPending(t, batch)
|
||||
// Fetch messages a couple of times.
|
||||
expected = 5
|
||||
bmsgs, err = sub.Fetch(expected, nats.MaxWait(2*time.Second))
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
got = len(bmsgs)
|
||||
if got != expected {
|
||||
t.Errorf("Expected: %v, got: %v", expected, got)
|
||||
}
|
||||
|
||||
info, err = sub.ConsumerInfo()
|
||||
if err != nil {
|
||||
t.Fatalf("Unexpected error: %v", err)
|
||||
}
|
||||
if info.NumAckPending != batch*2 || info.NumPending != uint64(toSend-batch) {
|
||||
t.Fatalf("Expected ack pending of %d and pending to be %d, got %d %d", batch*2, toSend-batch, info.NumAckPending, info.NumPending)
|
||||
}
|
||||
|
||||
// Create a new pull based consumer.
|
||||
batch = 1
|
||||
msgs := make(chan *nats.Msg, 100)
|
||||
sub, err = js.ChanSubscribe("baz", msgs, nats.Durable("dlc"), nats.Pull(batch))
|
||||
if err != nil {
|
||||
t.Fatalf("Unexpected error: %v", err)
|
||||
}
|
||||
|
||||
// Since this sub is on 'baz' no messages are waiting for us to start.
|
||||
waitForPending(t, 0)
|
||||
|
||||
// Now send in 10 messages to baz.
|
||||
for i := 0; i < toSend; i++ {
|
||||
js.Publish("baz", msg)
|
||||
}
|
||||
// We should get 1 queued up.
|
||||
waitForPending(t, batch)
|
||||
|
||||
for received := 0; received < toSend; {
|
||||
select {
|
||||
case m := <-msgs:
|
||||
received++
|
||||
// This will do the AckNext version since it knows we are pull based.
|
||||
m.Ack()
|
||||
case <-time.After(time.Second):
|
||||
t.Fatalf("Timeout waiting for messages")
|
||||
}
|
||||
if info.NumAckPending != batch || info.NumPending != uint64(toSend-batch) {
|
||||
t.Fatalf("Expected ack pending of %d and pending to be %d, got %d %d", batch, toSend-batch, info.NumAckPending, info.NumPending)
|
||||
}
|
||||
|
||||
// Prevent invalid durable names
|
||||
@@ -612,12 +617,13 @@ func TestJetStreamAckPending_Pull(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
sub, err := js.SubscribeSync("foo",
|
||||
ackPendingLimit := 3
|
||||
|
||||
sub, err := js.PullSubscribe("foo",
|
||||
nats.Durable("dname-pull-ack-wait"),
|
||||
nats.AckWait(100*time.Millisecond),
|
||||
nats.MaxDeliver(5),
|
||||
nats.MaxAckPending(3),
|
||||
nats.Pull(15),
|
||||
nats.MaxAckPending(ackPendingLimit),
|
||||
)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
@@ -626,15 +632,29 @@ func TestJetStreamAckPending_Pull(t *testing.T) {
|
||||
|
||||
// 3 messages delivered 5 times.
|
||||
expected := 15
|
||||
timeout := time.Now().Add(2 * time.Second)
|
||||
|
||||
// Fetching more than ack pending is an error.
|
||||
_, err = sub.Fetch(expected)
|
||||
if err == nil {
|
||||
t.Fatalf("Unexpected success fetching more messages than ack pending limit")
|
||||
}
|
||||
|
||||
pending := 0
|
||||
msgs := make([]*nats.Msg, 0)
|
||||
timeout := time.Now().Add(2 * time.Second)
|
||||
for time.Now().Before(timeout) {
|
||||
if pending, _, _ = sub.Pending(); pending >= expected {
|
||||
ms, err := sub.Fetch(ackPendingLimit)
|
||||
if err != nil || (ms != nil && len(ms) == 0) {
|
||||
continue
|
||||
}
|
||||
|
||||
msgs = append(msgs, ms...)
|
||||
if len(msgs) >= expected {
|
||||
break
|
||||
}
|
||||
time.Sleep(10 * time.Millisecond)
|
||||
}
|
||||
if pending < expected {
|
||||
if len(msgs) < expected {
|
||||
t.Errorf("Expected %v, got %v", expected, pending)
|
||||
}
|
||||
|
||||
@@ -680,10 +700,8 @@ func TestJetStreamAckPending_Pull(t *testing.T) {
|
||||
}
|
||||
|
||||
acks := map[int]int{}
|
||||
|
||||
ackPending := 3
|
||||
timeout = time.Now().Add(2 * time.Second)
|
||||
for time.Now().Before(timeout) {
|
||||
for _, m := range msgs {
|
||||
info, err := sub.ConsumerInfo()
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
@@ -692,17 +710,6 @@ func TestJetStreamAckPending_Pull(t *testing.T) {
|
||||
t.Fatalf("unexpected num ack pending: got=%d, want=%d", got, want)
|
||||
}
|
||||
|
||||
// Continue to ack all messages until no more pending.
|
||||
pending, _, _ = sub.Pending()
|
||||
if pending == 0 {
|
||||
break
|
||||
}
|
||||
|
||||
m, err := sub.NextMsg(100 * time.Millisecond)
|
||||
if err != nil {
|
||||
t.Fatalf("Error getting next message: %v", err)
|
||||
}
|
||||
|
||||
if err := m.AckSync(); err != nil {
|
||||
t.Fatalf("Error on ack message: %v", err)
|
||||
}
|
||||
@@ -734,7 +741,7 @@ func TestJetStreamAckPending_Pull(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
_, err = sub.NextMsg(100 * time.Millisecond)
|
||||
_, err = sub.Fetch(1, nats.MaxWait(100*time.Millisecond))
|
||||
if err != nats.ErrTimeout {
|
||||
t.Errorf("Expected timeout, got: %v", err)
|
||||
}
|
||||
@@ -1695,26 +1702,6 @@ func TestJetStreamImportDirectOnly(t *testing.T) {
|
||||
t.Errorf("Unexpected error: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
// Now pull based consumer.
|
||||
batch := 10
|
||||
sub, err = js.SubscribeSync("ORDERS", nats.PullDirect("ORDERS", "d1", batch))
|
||||
if err != nil {
|
||||
t.Fatalf("Unexpected error: %v", err)
|
||||
}
|
||||
waitForPending(t, batch)
|
||||
|
||||
for i := 0; i < toSend; i++ {
|
||||
m, err := sub.NextMsg(100 * time.Millisecond)
|
||||
if err != nil {
|
||||
t.Fatalf("Unexpected error: %v", err)
|
||||
}
|
||||
// Tests that acks flow since we need these to do AckNext for this to work.
|
||||
err = m.Ack()
|
||||
if err != nil {
|
||||
t.Errorf("Unexpected error: %v", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestJetStreamCrossAccountMirrorsAndSources(t *testing.T) {
|
||||
@@ -2010,28 +1997,6 @@ func TestJetStreamPullBasedStall(t *testing.T) {
|
||||
nc.Publish("STALL", msg)
|
||||
}
|
||||
nc.Flush()
|
||||
|
||||
batch := 100
|
||||
msgs := make(chan *nats.Msg, batch-2)
|
||||
sub, err := js.ChanSubscribe("STALL", msgs, nats.Durable("dlc"), nats.Pull(batch))
|
||||
if err != nil {
|
||||
t.Fatalf("Unexpected error: %v", err)
|
||||
}
|
||||
defer sub.Unsubscribe()
|
||||
|
||||
for received := 0; received < toSend; {
|
||||
select {
|
||||
case m := <-msgs:
|
||||
received++
|
||||
meta, _ := m.MetaData()
|
||||
if meta.Consumer != uint64(received) {
|
||||
t.Fatalf("Missed something, wanted %d but got %d", received, meta.Consumer)
|
||||
}
|
||||
m.Ack()
|
||||
case <-time.After(time.Second):
|
||||
t.Fatalf("Timeout waiting for messages, last received was %d", received)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestJetStreamSubscribe_DeliverPolicy(t *testing.T) {
|
||||
@@ -2589,16 +2554,17 @@ func TestJetStream_Unsubscribe(t *testing.T) {
|
||||
}); err != nil {
|
||||
t.Fatalf("Unexpected error: %v", err)
|
||||
}
|
||||
subC, err := js.SubscribeSync("foo.C", nats.Durable("wq"), nats.Pull(1))
|
||||
subC, err := js.PullSubscribe("foo.C", nats.Durable("wq"))
|
||||
if err != nil {
|
||||
t.Fatalf("Unexpected error: %v", err)
|
||||
}
|
||||
fetchConsumers(t, 1)
|
||||
|
||||
msg, err := subC.NextMsg(2 * time.Second)
|
||||
msgs, err := subC.Fetch(1, nats.MaxWait(2*time.Second))
|
||||
if err != nil {
|
||||
t.Errorf("Unexpected error getting message: %v", err)
|
||||
}
|
||||
msg := msgs[0]
|
||||
got := string(msg.Data)
|
||||
expected := "C"
|
||||
if got != expected {
|
||||
@@ -3563,18 +3529,6 @@ func TestJetStream_ClusterReconnect(t *testing.T) {
|
||||
replicas := []int{1, 3}
|
||||
|
||||
t.Run("pull sub", func(t *testing.T) {
|
||||
for _, r := range replicas {
|
||||
t.Run(fmt.Sprintf("n=%d r=%d", n, r), func(t *testing.T) {
|
||||
stream := &nats.StreamConfig{
|
||||
Name: fmt.Sprintf("foo-r%d", r),
|
||||
Replicas: r,
|
||||
}
|
||||
withJSClusterAndStream(t, fmt.Sprintf("PULLR%d", r), n, stream, testJetStream_ClusterReconnectPullSubscriber)
|
||||
})
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("pull qsub", func(t *testing.T) {
|
||||
for _, r := range replicas {
|
||||
t.Run(fmt.Sprintf("n=%d r=%d", n, r), func(t *testing.T) {
|
||||
stream := &nats.StreamConfig{
|
||||
@@ -3599,142 +3553,16 @@ func TestJetStream_ClusterReconnect(t *testing.T) {
|
||||
})
|
||||
|
||||
t.Run("qsub durable", func(t *testing.T) {
|
||||
r := 3
|
||||
t.Run(fmt.Sprintf("n=%d r=%d", n, r), func(t *testing.T) {
|
||||
stream := &nats.StreamConfig{
|
||||
Name: fmt.Sprintf("bar-r%d", r),
|
||||
Replicas: r,
|
||||
}
|
||||
withJSClusterAndStream(t, fmt.Sprintf("QSUBR%d", r), n, stream, testJetStream_ClusterReconnectDurableQueueSubscriber)
|
||||
})
|
||||
})
|
||||
}
|
||||
|
||||
func testJetStream_ClusterReconnectPullSubscriber(t *testing.T, subject string, srvs ...*jsServer) {
|
||||
var (
|
||||
recvd int
|
||||
srvA = srvs[0]
|
||||
totalMsgs = 20
|
||||
durable = nats.Durable("d1")
|
||||
reconnected = make(chan struct{}, 2)
|
||||
reconnectDone bool
|
||||
)
|
||||
nc, err := nats.Connect(srvA.ClientURL(),
|
||||
nats.ReconnectHandler(func(nc *nats.Conn) {
|
||||
reconnected <- struct{}{}
|
||||
|
||||
// Bring back the server after the reconnect event.
|
||||
if !reconnectDone {
|
||||
reconnectDone = true
|
||||
srvA.Restart()
|
||||
}
|
||||
}),
|
||||
)
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
defer nc.Close()
|
||||
|
||||
js, err := nc.JetStream()
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
|
||||
for i := 0; i < 10; i++ {
|
||||
payload := fmt.Sprintf("i:%d", i)
|
||||
_, err := js.Publish(subject, []byte(payload))
|
||||
if err != nil {
|
||||
t.Errorf("Unexpected error: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
sub, err := js.SubscribeSync(subject, durable, nats.Pull(1))
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
|
||||
for i := 10; i < totalMsgs; i++ {
|
||||
payload := fmt.Sprintf("i:%d", i)
|
||||
_, err := js.Publish(subject, []byte(payload))
|
||||
if err != nil {
|
||||
t.Errorf("Unexpected error: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
ctx, done := context.WithTimeout(context.Background(), 10*time.Second)
|
||||
defer done()
|
||||
|
||||
NextMsg:
|
||||
for recvd < totalMsgs {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
t.Fatalf("Timeout waiting for messages, expected: %d, got: %d", totalMsgs, recvd)
|
||||
default:
|
||||
}
|
||||
|
||||
pending, _, _ := sub.Pending()
|
||||
if pending == 0 {
|
||||
err = sub.Poll()
|
||||
if err != nil {
|
||||
t.Errorf("Unexpected error: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
// Server will shutdown after a couple of messages which will result
|
||||
// in empty messages with an status unavailable error.
|
||||
msg, err := sub.NextMsg(2 * time.Second)
|
||||
if err == nats.ErrNoResponders || err == nats.ErrTimeout {
|
||||
// Backoff before asking for more messages.
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
continue NextMsg
|
||||
} else if err != nil {
|
||||
t.Errorf("Unexpected error: %v", err)
|
||||
continue NextMsg
|
||||
}
|
||||
|
||||
if len(msg.Data) == 0 && msg.Header.Get("Status") == "503" {
|
||||
t.Fatal("Got 503 JetStream API message!")
|
||||
}
|
||||
|
||||
got := string(msg.Data)
|
||||
expected := fmt.Sprintf("i:%d", recvd)
|
||||
if got != expected {
|
||||
// Missed a message, but continue checking for the rest.
|
||||
recvd++
|
||||
t.Logf("WARN: Expected %v, got: %v", expected, got)
|
||||
}
|
||||
|
||||
// Add a few retries since there can be errors during the reconnect.
|
||||
timeout := time.Now().Add(5 * time.Second)
|
||||
RetryAck:
|
||||
for time.Now().Before(timeout) {
|
||||
err = msg.AckSync()
|
||||
if err != nil {
|
||||
// During the reconnection, both of these errors can occur.
|
||||
if err == nats.ErrNoResponders || err == nats.ErrTimeout {
|
||||
// Wait for reconnection event to occur to continue.
|
||||
select {
|
||||
case <-reconnected:
|
||||
continue RetryAck
|
||||
case <-time.After(100 * time.Millisecond):
|
||||
continue RetryAck
|
||||
case <-ctx.Done():
|
||||
t.Fatal("Timed out waiting for reconnect")
|
||||
}
|
||||
for _, r := range replicas {
|
||||
t.Run(fmt.Sprintf("n=%d r=%d", n, r), func(t *testing.T) {
|
||||
stream := &nats.StreamConfig{
|
||||
Name: fmt.Sprintf("bar-r%d", r),
|
||||
Replicas: r,
|
||||
}
|
||||
|
||||
t.Errorf("Unexpected error: %v", err)
|
||||
continue RetryAck
|
||||
}
|
||||
break RetryAck
|
||||
withJSClusterAndStream(t, fmt.Sprintf("QSUBR%d", r), n, stream, testJetStream_ClusterReconnectDurableQueueSubscriber)
|
||||
})
|
||||
}
|
||||
recvd++
|
||||
|
||||
// Shutdown the server after a couple of messages.
|
||||
if recvd == 2 {
|
||||
srvA.Shutdown()
|
||||
}
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
func testJetStream_ClusterReconnectDurableQueueSubscriber(t *testing.T, subject string, srvs ...*jsServer) {
|
||||
@@ -4044,7 +3872,7 @@ func testJetStream_ClusterReconnectPullQueueSubscriber(t *testing.T, subject str
|
||||
subs := make([]*nats.Subscription, 0)
|
||||
|
||||
for i := 0; i < 5; i++ {
|
||||
sub, err := js.QueueSubscribeSync(subject, "wq", durable, nats.Pull(1))
|
||||
sub, err := js.PullSubscribe(subject, durable, nats.PullMaxWaiting(5))
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
@@ -4071,16 +3899,9 @@ NextMsg:
|
||||
}
|
||||
|
||||
for qsub, sub := range subs {
|
||||
if pending, _, _ := sub.Pending(); pending == 0 {
|
||||
err = sub.Poll()
|
||||
if err != nil {
|
||||
t.Errorf("Unexpected error: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
// Server will shutdown after a couple of messages which will result
|
||||
// in empty messages with an status unavailable error.
|
||||
msg, err := sub.NextMsg(2 * time.Second)
|
||||
msgs, err := sub.Fetch(1, nats.MaxWait(2*time.Second))
|
||||
if err == nats.ErrNoResponders || err == nats.ErrTimeout {
|
||||
// Backoff before asking for more messages.
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
@@ -4089,6 +3910,11 @@ NextMsg:
|
||||
t.Errorf("Unexpected error: %v", err)
|
||||
continue NextMsg
|
||||
}
|
||||
msg := msgs[0]
|
||||
if len(msg.Data) == 0 && msg.Header.Get("Status") == "503" {
|
||||
t.Fatal("Got 503 JetStream API message!")
|
||||
}
|
||||
|
||||
recvd[string(msg.Data)]++
|
||||
recvdQ[qsub] = append(recvdQ[qsub], msg)
|
||||
|
||||
@@ -4158,3 +3984,296 @@ func checkFor(t *testing.T, totalWait, sleepDur time.Duration, f func() error) {
|
||||
t.Fatal(err.Error())
|
||||
}
|
||||
}
|
||||
|
||||
func TestJetStreamPullSubscribeOptions(t *testing.T) {
|
||||
withJSCluster(t, "FOPTS", 3, testJetStreamFetchOptions)
|
||||
}
|
||||
|
||||
func testJetStreamFetchOptions(t *testing.T, srvs ...*jsServer) {
|
||||
srv := srvs[0]
|
||||
nc, err := nats.Connect(srv.ClientURL())
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
defer nc.Close()
|
||||
|
||||
js, err := nc.JetStream()
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
subject := "WQ"
|
||||
_, err = js.AddStream(&nats.StreamConfig{
|
||||
Name: subject,
|
||||
Replicas: 1,
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
sendMsgs := func(t *testing.T, totalMsgs int) {
|
||||
t.Helper()
|
||||
for i := 0; i < totalMsgs; i++ {
|
||||
payload := fmt.Sprintf("i:%d", i)
|
||||
_, err := js.Publish(subject, []byte(payload))
|
||||
if err != nil {
|
||||
t.Errorf("Unexpected error: %v", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
t.Run("batch size", func(t *testing.T) {
|
||||
defer js.PurgeStream(subject)
|
||||
|
||||
expected := 10
|
||||
sendMsgs(t, expected)
|
||||
sub, err := js.PullSubscribe(subject, nats.Durable("batch-size"))
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer sub.Unsubscribe()
|
||||
|
||||
msgs, err := sub.Fetch(expected, nats.MaxWait(2*time.Second))
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
for _, msg := range msgs {
|
||||
msg.AckSync()
|
||||
}
|
||||
|
||||
got := len(msgs)
|
||||
if got != expected {
|
||||
t.Fatalf("Got %v messages, expected at least: %v", got, expected)
|
||||
}
|
||||
|
||||
// Next fetch will timeout since no more messages.
|
||||
_, err = sub.Fetch(1, nats.MaxWait(250*time.Millisecond))
|
||||
if err != nats.ErrTimeout {
|
||||
t.Errorf("Expected timeout fetching next message, got: %v", err)
|
||||
}
|
||||
|
||||
expected = 5
|
||||
sendMsgs(t, expected)
|
||||
msgs, err = sub.Fetch(expected, nats.MaxWait(1*time.Second))
|
||||
if err != nil {
|
||||
t.Fatalf("Unexpected error: %v", err)
|
||||
}
|
||||
got = len(msgs)
|
||||
if got != expected {
|
||||
t.Fatalf("Got %v messages, expected at least: %v", got, expected)
|
||||
}
|
||||
|
||||
for _, msg := range msgs {
|
||||
msg.Ack()
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("sub drain is no op", func(t *testing.T) {
|
||||
defer js.PurgeStream(subject)
|
||||
|
||||
expected := 10
|
||||
sendMsgs(t, expected)
|
||||
sub, err := js.PullSubscribe(subject, nats.Durable("batch-ctx"))
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer sub.Unsubscribe()
|
||||
|
||||
msgs, err := sub.Fetch(expected, nats.MaxWait(2*time.Second))
|
||||
if err != nil {
|
||||
t.Fatalf("Unexpected error: %v", err)
|
||||
}
|
||||
got := len(msgs)
|
||||
if got != expected {
|
||||
t.Fatalf("Got %v messages, expected at least: %v", got, expected)
|
||||
}
|
||||
err = sub.Drain()
|
||||
if err != nil {
|
||||
t.Errorf("Unexpected error: %v", err)
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("pull with context", func(t *testing.T) {
|
||||
defer js.PurgeStream(subject)
|
||||
|
||||
expected := 10
|
||||
sendMsgs(t, expected)
|
||||
sub, err := js.PullSubscribe(subject, nats.Durable("batch-ctx"))
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer sub.Unsubscribe()
|
||||
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
cancel()
|
||||
|
||||
// Should fail with expired context.
|
||||
_, err = sub.Fetch(expected, nats.Context(ctx))
|
||||
if err == nil {
|
||||
t.Fatal("Unexpected success")
|
||||
}
|
||||
if err != context.Canceled {
|
||||
t.Errorf("Expected context deadline exceeded error, got: %v", err)
|
||||
}
|
||||
|
||||
ctx, cancel = context.WithTimeout(context.Background(), 2*time.Second)
|
||||
defer cancel()
|
||||
|
||||
msgs, err := sub.Fetch(expected, nats.Context(ctx))
|
||||
if err != nil {
|
||||
t.Fatalf("Unexpected error: %v", err)
|
||||
}
|
||||
|
||||
got := len(msgs)
|
||||
if got != expected {
|
||||
t.Fatalf("Got %v messages, expected at least: %v", got, expected)
|
||||
}
|
||||
|
||||
for _, msg := range msgs {
|
||||
msg.AckSync()
|
||||
}
|
||||
|
||||
// Next fetch will timeout since no more messages.
|
||||
_, err = sub.Fetch(1, nats.MaxWait(250*time.Millisecond))
|
||||
if err != nats.ErrTimeout {
|
||||
t.Errorf("Expected timeout fetching next message, got: %v", err)
|
||||
}
|
||||
|
||||
expected = 5
|
||||
sendMsgs(t, expected)
|
||||
msgs, err = sub.Fetch(expected, nats.MaxWait(1*time.Second))
|
||||
if err != nil {
|
||||
t.Fatalf("Unexpected error: %v", err)
|
||||
}
|
||||
got = len(msgs)
|
||||
if got != expected {
|
||||
t.Fatalf("Got %v messages, expected at least: %v", got, expected)
|
||||
}
|
||||
|
||||
for _, msg := range msgs {
|
||||
msg.Ack()
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("fetch after unsubscribe", func(t *testing.T) {
|
||||
defer js.PurgeStream(subject)
|
||||
|
||||
expected := 10
|
||||
sendMsgs(t, expected)
|
||||
sub, err := js.PullSubscribe(subject, nats.Durable("fetch-unsub"))
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
err = sub.Unsubscribe()
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
_, err = sub.Fetch(1, nats.MaxWait(500*time.Millisecond))
|
||||
if err == nil {
|
||||
t.Fatal("Unexpected success")
|
||||
}
|
||||
if err != nil && (err != nats.ErrTimeout && err != nats.ErrNoResponders) {
|
||||
t.Fatalf("Unexpected error: %v", err)
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("max waiting timeout", func(t *testing.T) {
|
||||
defer js.PurgeStream(subject)
|
||||
|
||||
expected := 10
|
||||
sendMsgs(t, expected)
|
||||
|
||||
sub, err := js.PullSubscribe(subject, nats.Durable("max-waiting"))
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer sub.Unsubscribe()
|
||||
|
||||
// Poll more than the default max of waiting/inflight pull requests,
|
||||
// so that We will get only 408 timeout errors.
|
||||
errCh := make(chan error, 1024)
|
||||
defer close(errCh)
|
||||
var wg sync.WaitGroup
|
||||
for i := 0; i < 1024; i++ {
|
||||
wg.Add(1)
|
||||
|
||||
go func() {
|
||||
_, err := sub.Fetch(1, nats.MaxWait(500*time.Millisecond))
|
||||
defer wg.Done()
|
||||
if err != nil {
|
||||
errCh <- err
|
||||
}
|
||||
}()
|
||||
}
|
||||
wg.Wait()
|
||||
|
||||
select {
|
||||
case <-time.After(1 * time.Second):
|
||||
t.Fatal("Expected RequestTimeout (408) error due to many inflight pulls")
|
||||
case err := <-errCh:
|
||||
if err != nil && (err.Error() != `Request Timeout` && err != nats.ErrTimeout) {
|
||||
t.Errorf("Expected request timeout fetching next message, got: %+v", err)
|
||||
}
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("no wait", func(t *testing.T) {
|
||||
defer js.PurgeStream(subject)
|
||||
|
||||
expected := 10
|
||||
sendMsgs(t, expected)
|
||||
sub, err := js.PullSubscribe(subject, nats.Durable("no-wait"))
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer sub.Unsubscribe()
|
||||
|
||||
ctx, done := context.WithTimeout(context.Background(), 5*time.Second)
|
||||
defer done()
|
||||
recvd := make([]*nats.Msg, 0)
|
||||
|
||||
Loop:
|
||||
for range time.NewTicker(100 * time.Millisecond).C {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
break Loop
|
||||
default:
|
||||
}
|
||||
|
||||
msgs, err := sub.Fetch(1, nats.MaxWait(250*time.Millisecond))
|
||||
if err != nil {
|
||||
t.Fatalf("Unexpected error: %v", err)
|
||||
}
|
||||
recvd = append(recvd, msgs[0])
|
||||
|
||||
for _, msg := range msgs {
|
||||
err = msg.AckSync()
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
}
|
||||
|
||||
if len(recvd) == expected {
|
||||
done()
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
got := len(recvd)
|
||||
if got != expected {
|
||||
t.Fatalf("Got %v messages, expected at least: %v", got, expected)
|
||||
}
|
||||
|
||||
// There should only be timeout errors since no more messages.
|
||||
msgs, err := sub.Fetch(expected, nats.MaxWait(2*time.Second))
|
||||
if err == nil {
|
||||
t.Fatal("Unexpected success", len(msgs))
|
||||
}
|
||||
if err != nats.ErrTimeout {
|
||||
t.Fatalf("Expected timeout error, got: %v", err)
|
||||
}
|
||||
})
|
||||
}
|
||||
|
Reference in New Issue
Block a user