Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Update default router to switch partition on all batching thresholds … #383

Merged
merged 1 commit into from
Oct 21, 2020
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
94 changes: 94 additions & 0 deletions pulsar/default_router.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,94 @@
// 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 pulsar

import (
"math"
"math/rand"
"sync/atomic"
"time"
)

type defaultRouter struct {
currentPartitionCursor uint32

lastChangeTimestamp int64
msgCounter uint32
cumulativeBatchSize uint32
}

// NewDefaultRouter set the message routing mode for the partitioned producer.
// Default routing mode is round-robin routing if no partition key is specified.
// If the batching is enabled, it honors the different thresholds for batching i.e. maximum batch size,
// maximum number of messages, maximum delay to publish a batch. When one of the threshold is reached the next partition
// is used.
func NewDefaultRouter(
hashFunc func(string) uint32,
maxBatchingMessages uint,
maxBatchingSize uint,
maxBatchingDelay time.Duration,
disableBatching bool) func(*ProducerMessage, uint32) int {
state := &defaultRouter{
currentPartitionCursor: rand.Uint32(),
lastChangeTimestamp: math.MinInt64,
}

return func(message *ProducerMessage, numPartitions uint32) int {
if numPartitions == 1 {
// When there are no partitions, don't even bother
return 0
}

if len(message.Key) != 0 {
// When a key is specified, use the hash of that key
return int(hashFunc(message.Key) % numPartitions)
}

// If there's no key, we do round-robin across partition. If no batching go to next partition.
if disableBatching {
p := int(state.currentPartitionCursor % numPartitions)
atomic.AddUint32(&state.currentPartitionCursor, 1)
return p
}

// If there's no key, we do round-robin across partition, sticking with a given
// partition for a certain amount of messages or volume buffered or the max delay to batch is reached so that
// we ensure having a decent amount of batching of the messages.
// Note that it is possible that we skip more than one partition if multiple goroutines increment
// currentPartitionCursor at the same time. If that happens it shouldn't be a problem because we only want to
// spread the data on different partitions but not necessarily in a specific sequence.
size := uint32(len(message.Payload))
previousMessageCount := atomic.LoadUint32(&state.msgCounter)
previousBatchingMaxSize := atomic.LoadUint32(&state.cumulativeBatchSize)
previousLastChange := atomic.LoadInt64(&state.lastChangeTimestamp)
if (previousMessageCount >= uint32(maxBatchingMessages-1)) ||
(size >= uint32(maxBatchingSize)-previousBatchingMaxSize) ||
(time.Now().UnixNano()-previousLastChange >= maxBatchingDelay.Nanoseconds()) {
atomic.AddUint32(&state.currentPartitionCursor, 1)
atomic.StoreInt64(&state.lastChangeTimestamp, time.Now().UnixNano())
atomic.StoreUint32(&state.cumulativeBatchSize, 0)
atomic.StoreUint32(&state.msgCounter, 0)
return int(state.currentPartitionCursor % numPartitions)
}

atomic.StoreInt64(&state.lastChangeTimestamp, time.Now().UnixNano())
atomic.AddUint32(&state.msgCounter, 1)
atomic.AddUint32(&state.cumulativeBatchSize, size)
return int(state.currentPartitionCursor % numPartitions)
}
}
144 changes: 144 additions & 0 deletions pulsar/default_router_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,144 @@
// 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 pulsar

import (
"testing"
"time"

"github.com/apache/pulsar-client-go/pulsar/internal"
"github.com/stretchr/testify/assert"
)

const oneHourPublishMaxDelay = time.Hour

func TestDefaultRouterRoutingBecauseBatchingDisabled(t *testing.T) {
router := NewDefaultRouter(internal.JavaStringHash, 20, 100, oneHourPublishMaxDelay, true)
const numPartitions = uint32(3)
p1 := router(&ProducerMessage{
Payload: []byte("message 1"),
}, numPartitions)
assert.LessOrEqual(t, p1, int(numPartitions))

p2 := router(&ProducerMessage{
Payload: []byte("message 2"),
}, numPartitions)
if p1 == int(numPartitions-1) {
assert.Equal(t, 0, p2)
} else {
assert.Equal(t, p1+1, p2)
}
}

func TestDefaultRouterRoutingBecauseMaxPublishDelayReached(t *testing.T) {
maxPublishDelay := time.Nanosecond * 10
router := NewDefaultRouter(internal.JavaStringHash, 20, 100, maxPublishDelay, false)
const numPartitions = uint32(3)
p1 := router(&ProducerMessage{
Payload: []byte("message 1"),
}, 3)
assert.LessOrEqual(t, p1, int(numPartitions))

time.Sleep(maxPublishDelay)

p2 := router(&ProducerMessage{
Payload: []byte("message 2"),
}, numPartitions)
if p1 == int(numPartitions-1) {
assert.Equal(t, 0, p2)
} else {
assert.Equal(t, p1+1, p2)
}
}

func TestDefaultRouterRoutingBecauseMaxNumberOfMessagesReached(t *testing.T) {
router := NewDefaultRouter(internal.JavaStringHash, 2, 100, oneHourPublishMaxDelay, false)
const numPartitions = uint32(3)
p1 := router(&ProducerMessage{
Payload: []byte("message 1"),
}, 3)
assert.LessOrEqual(t, p1, int(numPartitions))

p2 := router(&ProducerMessage{
Payload: []byte("message 2"),
}, numPartitions)
if p1 == int(numPartitions-1) {
assert.Equal(t, 0, p2)
} else {
assert.Equal(t, p1+1, p2)
}
}

func TestDefaultRouterRoutingBecauseMaxVolumeReached(t *testing.T) {
router := NewDefaultRouter(internal.JavaStringHash, 10, 10, oneHourPublishMaxDelay, false)
const numPartitions = uint32(3)
p1 := router(&ProducerMessage{
Payload: []byte("message 1"),
}, 3)
assert.LessOrEqual(t, p1, int(numPartitions))

p2 := router(&ProducerMessage{
Payload: []byte("message 2"),
}, numPartitions)
if p1 == int(numPartitions-1) {
assert.Equal(t, 0, p2)
} else {
assert.Equal(t, p1+1, p2)
}
}

func TestDefaultRouterNoRoutingBecausePartitionKeyIsSpecified(t *testing.T) {
router := NewDefaultRouter(internal.JavaStringHash, 1, 1, 0, false)
p1 := router(&ProducerMessage{
Key: "my-key",
Payload: []byte("message 1"),
}, 3)
assert.Equal(t, 1, p1)

p2 := router(&ProducerMessage{
Key: "my-key",
Payload: []byte("message 2"),
}, 3)
assert.Equal(t, p1, p2)
}

func TestDefaultRouterNoRoutingBecauseOnlyOnePartition(t *testing.T) {

router := NewDefaultRouter(internal.JavaStringHash, 1, 10, oneHourPublishMaxDelay, false)

// partition index should not change regardless of the batching settings
p1 := router(&ProducerMessage{
Key: "",
}, 1)
p2 := router(&ProducerMessage{
Key: "my-key",
}, 1)
p3 := router(&ProducerMessage{
Payload: []byte("this payload is bigger than 10 bytes"),
}, 1)

// we send 2 messages to try trigger the max messages rule
p4 := router(&ProducerMessage{}, 1)
p5 := router(&ProducerMessage{}, 1)

assert.Equal(t, 0, p1)
assert.Equal(t, 0, p2)
assert.Equal(t, 0, p3)
assert.Equal(t, 0, p4)
assert.Equal(t, 0, p5)
}
15 changes: 1 addition & 14 deletions pulsar/internal/batch_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,14 +27,6 @@ import (
"github.com/apache/pulsar-client-go/pulsar/log"
)

const (
// DefaultMaxBatchSize init default for maximum number of bytes per batch
DefaultMaxBatchSize = 128 * 1024

// DefaultMaxMessagesPerBatch init default num of entries in per batch.
DefaultMaxMessagesPerBatch = 1000
)

type BuffersPool interface {
GetBuffer() Buffer
}
Expand Down Expand Up @@ -71,12 +63,7 @@ type BatchBuilder struct {
func NewBatchBuilder(maxMessages uint, maxBatchSize uint, producerName string, producerID uint64,
compressionType pb.CompressionType, level compression.Level,
bufferPool BuffersPool, logger log.Logger) (*BatchBuilder, error) {
if maxMessages == 0 {
maxMessages = DefaultMaxMessagesPerBatch
}
if maxBatchSize == 0 {
maxBatchSize = DefaultMaxBatchSize
}

bb := &BatchBuilder{
buffer: NewBuffer(4096),
numMessages: 0,
Expand Down
80 changes: 0 additions & 80 deletions pulsar/internal/default_router.go

This file was deleted.

Loading