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

Performance Analysis and Improvement of LogStream Primary-Backup Replication #795

Open
3 tasks done
ijsong opened this issue May 26, 2024 · 0 comments · May be fixed by #808
Open
3 tasks done

Performance Analysis and Improvement of LogStream Primary-Backup Replication #795

ijsong opened this issue May 26, 2024 · 0 comments · May be fixed by #808
Assignees

Comments

@ijsong
Copy link
Member

ijsong commented May 26, 2024

Background

This issue aims to analyze and improve the performance of LogStream's Primary-Backup Replication. First, we need to define the metrics that accurately reflect replication performance. Currently, there are metrics such as sn.replicate.logs, sn.replicate.bytes, and sn.replicate.duration. However, these metrics only provide average values for replication throughput and response time. These metrics need to be redefined to conduct a detailed analysis using percentiles.

Problem: Current Structure

The replication gRPC operates as follows: the Primary Replica invokes the Backup Replica's Replicate RPC. In this setup, the Backup Replica acts as the gRPC server, while the Primary Replica acts as the client. The Primary Replica sends batches of LogEntry to the Backup Replica using the Replicate RPC.

  1. Primary Replica: Sends batches of LogEntry via Replicate RPC.
  2. Backup Replica:
    • Receiver Goroutine: This goroutine acts as the gRPC Receiver, receiving ReplicateRequest through Replicate RPC and placing it into a queue as ReplicationRequestTask.
    • Replicator Goroutine: This goroutine takes ReplicationRequestTask from the queue and uses it to call LogStreamExecutor's Replicate method.

The Receiver goroutine defines ReplicateRequest once and uses it continuously. However, the gRPC Unmarshal code repeatedly generates slices for some fields, such as LLSN and Data. Although the protobuf message ReplicateRequest is reused, the slices included are not, causing degraded performance.

type ReplicateRequest struct {
TopicID github_com_kakao_varlog_pkg_types.TopicID `protobuf:"varint,1,opt,name=topic_id,json=topicId,proto3,casttype=github.com/kakao/varlog/pkg/types.TopicID" json:"topic_id,omitempty"`
LogStreamID github_com_kakao_varlog_pkg_types.LogStreamID `protobuf:"varint,2,opt,name=log_stream_id,json=logStreamId,proto3,casttype=github.com/kakao/varlog/pkg/types.LogStreamID" json:"log_stream_id,omitempty"`
LLSN []github_com_kakao_varlog_pkg_types.LLSN `protobuf:"varint,3,rep,packed,name=llsn,proto3,casttype=github.com/kakao/varlog/pkg/types.LLSN" json:"llsn,omitempty"`
Data [][]byte `protobuf:"bytes,4,rep,name=data,proto3" json:"data,omitempty"`
}
func (m *ReplicateRequest) Reset() { *m = ReplicateRequest{} }

func (rs *replicationServer) recv(ctx context.Context, stream snpb.Replicator_ReplicateServer, wg *sync.WaitGroup) <-chan *replicationServerTask {
wg.Add(1)
// TODO: add configuration
c := make(chan *replicationServerTask, 4096)
go func() {
defer wg.Done()
defer close(c)
req := &snpb.ReplicateRequest{}
for {
req.Reset()
err := stream.RecvMsg(req)
rst := newReplicationServerTask(*req, err)
select {
case c <- rst:
if err != nil {
return
}
case <-ctx.Done():
rst.release()
return
}
}
}()
return c
}

Proposed Optimizations

The following ideas are proposed to optimize the replication process:

  1. Use sync.Pool for ReplicateRequest: Utilize sync.Pool to reuse ReplicateRequest structures.
  2. Reuse Slices: Extend the Reset method generated in gRPC to reuse slice-type fields - LLSN and Data - of the ReplicateRequest struct.

The proposed optimization aims to reuse these fields to enhance Unmarshal's performance. The longer the batch size, the more allocations for the slices there are, so reusing them seems promising. Furthermore, append entries into the slices cause copy overhead whenever the underlying heaps enlarge. This optimization may greatly reduce this overhead.

To reuse the slice in the protobuf message, the following code can be used:

func (m *ReplicateRequest) ResetReuse() {
	s := m.LLSN[:0]
	m.Reset()
	m.LLSN = s
}

In the case of snpb.(ReplicateRequest).Data, the outer slice can be reused using the above code. However, we should change the unmarshalling code generated by gRPC to reuse the inner slice.

Plan

@ijsong ijsong self-assigned this May 26, 2024
ijsong added a commit that referenced this issue Jun 8, 2024
This PR adds generated tests for
`github.com/kakao/varlog/proto/snpb.AppendRequest` and
`github.com/kakao/varlog/proto/snpb.ReplicateRequest`. These tests are generated
using the [gogoproto testgen tag](https://pkg.go.dev/github.com/gogo/protobuf/plugin/testgen).

Resolving issue #795 may require changes to the unmarshaling code generated by
gogoproto. Adding these tests will provide confidence when making those changes.
ijsong added a commit that referenced this issue Jun 8, 2024
This PR adds generated tests for
`github.com/kakao/varlog/proto/snpb.AppendRequest` and
`github.com/kakao/varlog/proto/snpb.ReplicateRequest`. These tests are generated
using the [gogoproto testgen tag](https://pkg.go.dev/github.com/gogo/protobuf/plugin/testgen).

Resolving issue #795 may require changes to the unmarshaling code generated by
gogoproto. Adding these tests will provide confidence when making those changes.
ijsong added a commit that referenced this issue Jun 8, 2024
Improve unmarshaling performance by reusing buffers for ReplicateRequest in the
backup replica.

The protobuf message `github.com/kakao/varlog/proto/snpb.(ReplicateRequest)` has
two slice fields—LLSN (`[]uint64`) and Data (`[][]byte`). The backup replica
receives replicated log entries from the primary replica via the gRPC service
`github.com/kakao/varlog/proto/snpb.(ReplicatorServer).Replicate`, which sends
`ReplicateRequest` messages.

Upon receiving a `ReplicateRequest`, the backup replica unmarshals the message,
which involves growing slices for fields such as LLSN and Data. This growth
causes copy overhead whenever the slice capacities need to expand.

To address this, we introduce a new method, `ResetReuse`, for reusing slices
instead of resetting them completely. The `ResetReuse` method shrinks the slice
lengths while preserving their capacities, thus avoiding the overhead of
reallocating memory.

Example implementation:

```go
type Message struct {
    Buffer []byte
    // Other fields
}

func (m *Message) Reset() {
    *m = Message{}
}

func (m *Message) ResetReuse() {
    s := m.Buffer[:0]
    *m = Message{}
    m.Buffer = s
}
```

Risks:

This approach has potential downsides. Since the heap space consumed by the
slices is not reclaimed, the storage node's memory consumption may increase.
Currently, there is no mechanism to shrink the heap usage.

Additionally, this PR changes the generated code. The protobuf compiler can
revert it, which is contrary to our intention. To catch this mistake, this PR
includes a unit test (github.com/kakao/varlog/proto/snpb.TestReplicateRequest)
to verify that the buffer backing the slices is reused.

Resolves: #795
See also: #806
ijsong added a commit that referenced this issue Jun 8, 2024
Improve unmarshaling performance by reusing buffers for ReplicateRequest in the
backup replica.

The protobuf message `github.com/kakao/varlog/proto/snpb.(ReplicateRequest)` has
two slice fields—LLSN (`[]uint64`) and Data (`[][]byte`). The backup replica
receives replicated log entries from the primary replica via the gRPC service
`github.com/kakao/varlog/proto/snpb.(ReplicatorServer).Replicate`, which sends
`ReplicateRequest` messages.

Upon receiving a `ReplicateRequest`, the backup replica unmarshals the message,
which involves growing slices for fields such as LLSN and Data. This growth
causes copy overhead whenever the slice capacities need to expand.

To address this, we introduce a new method, `ResetReuse`, for reusing slices
instead of resetting them completely. The `ResetReuse` method shrinks the slice
lengths while preserving their capacities, thus avoiding the overhead of
reallocating memory.

Example implementation:

```go
type Message struct {
    Buffer []byte
    // Other fields
}

func (m *Message) Reset() {
    *m = Message{}
}

func (m *Message) ResetReuse() {
    s := m.Buffer[:0]
    *m = Message{}
    m.Buffer = s
}
```

Risks:

This approach has potential downsides. Since the heap space consumed by the
slices is not reclaimed, the storage node's memory consumption may increase.
Currently, there is no mechanism to shrink the heap usage.

Additionally, this PR changes the generated code. The protobuf compiler can
revert it, which is contrary to our intention. To catch this mistake, this PR
includes a unit test (github.com/kakao/varlog/proto/snpb.TestReplicateRequest)
to verify that the buffer backing the slices is reused.

Resolves: #795

See also: #806
ijsong added a commit that referenced this issue Jun 13, 2024
This PR adds generated tests for
`github.com/kakao/varlog/proto/snpb.AppendRequest` and
`github.com/kakao/varlog/proto/snpb.ReplicateRequest`. These tests are generated
using the [gogoproto testgen tag](https://pkg.go.dev/github.com/gogo/protobuf/plugin/testgen).

Resolving issue #795 may require changes to the unmarshaling code generated by
gogoproto. Adding these tests will provide confidence when making those changes.
ijsong added a commit that referenced this issue Jun 13, 2024
Improve unmarshaling performance by reusing buffers for ReplicateRequest in the
backup replica.

The protobuf message `github.com/kakao/varlog/proto/snpb.(ReplicateRequest)` has
two slice fields—LLSN (`[]uint64`) and Data (`[][]byte`). The backup replica
receives replicated log entries from the primary replica via the gRPC service
`github.com/kakao/varlog/proto/snpb.(ReplicatorServer).Replicate`, which sends
`ReplicateRequest` messages.

Upon receiving a `ReplicateRequest`, the backup replica unmarshals the message,
which involves growing slices for fields such as LLSN and Data. This growth
causes copy overhead whenever the slice capacities need to expand.

To address this, we introduce a new method, `ResetReuse`, for reusing slices
instead of resetting them completely. The `ResetReuse` method shrinks the slice
lengths while preserving their capacities, thus avoiding the overhead of
reallocating memory.

Example implementation:

```go
type Message struct {
    Buffer []byte
    // Other fields
}

func (m *Message) Reset() {
    *m = Message{}
}

func (m *Message) ResetReuse() {
    s := m.Buffer[:0]
    *m = Message{}
    m.Buffer = s
}
```

Risks:

This approach has potential downsides. Since the heap space consumed by the
slices is not reclaimed, the storage node's memory consumption may increase.
Currently, there is no mechanism to shrink the heap usage.

Additionally, this PR changes the generated code. The protobuf compiler can
revert it, which is contrary to our intention. To catch this mistake, this PR
includes a unit test (github.com/kakao/varlog/proto/snpb.TestReplicateRequest)
to verify that the buffer backing the slices is reused.

Resolves: #795

See also: #806
ijsong added a commit that referenced this issue Jun 13, 2024
Improve unmarshaling performance by reusing buffers for ReplicateRequest in the
backup replica.

The protobuf message `github.com/kakao/varlog/proto/snpb.(ReplicateRequest)` has
two slice fields—LLSN (`[]uint64`) and Data (`[][]byte`). The backup replica
receives replicated log entries from the primary replica via the gRPC service
`github.com/kakao/varlog/proto/snpb.(ReplicatorServer).Replicate`, which sends
`ReplicateRequest` messages.

Upon receiving a `ReplicateRequest`, the backup replica unmarshals the message,
which involves growing slices for fields such as LLSN and Data. This growth
causes copy overhead whenever the slice capacities need to expand.

To address this, we introduce a new method, `ResetReuse`, for reusing slices
instead of resetting them completely. The `ResetReuse` method shrinks the slice
lengths while preserving their capacities, thus avoiding the overhead of
reallocating memory.

Example implementation:

```go
type Message struct {
    Buffer []byte
    // Other fields
}

func (m *Message) Reset() {
    *m = Message{}
}

func (m *Message) ResetReuse() {
    s := m.Buffer[:0]
    *m = Message{}
    m.Buffer = s
}
```

Risks:

This approach has potential downsides. Since the heap space consumed by the
slices is not reclaimed, the storage node's memory consumption may increase.
Currently, there is no mechanism to shrink the heap usage.

Additionally, this PR changes the generated code. The protobuf compiler can
revert it, which is contrary to our intention. To catch this mistake, this PR
includes a unit test (github.com/kakao/varlog/proto/snpb.TestReplicateRequest)
to verify that the buffer backing the slices is reused.

Resolves: #795

See also: #806
ijsong added a commit that referenced this issue Jun 14, 2024
Improve unmarshaling performance by reusing buffers for ReplicateRequest in the
backup replica.

The protobuf message `github.com/kakao/varlog/proto/snpb.(ReplicateRequest)` has
two slice fields—LLSN (`[]uint64`) and Data (`[][]byte`). The backup replica
receives replicated log entries from the primary replica via the gRPC service
`github.com/kakao/varlog/proto/snpb.(ReplicatorServer).Replicate`, which sends
`ReplicateRequest` messages.

Upon receiving a `ReplicateRequest`, the backup replica unmarshals the message,
which involves growing slices for fields such as LLSN and Data. This growth
causes copy overhead whenever the slice capacities need to expand.

To address this, we introduce a new method, `ResetReuse`, for reusing slices
instead of resetting them completely. The `ResetReuse` method shrinks the slice
lengths while preserving their capacities, thus avoiding the overhead of
reallocating memory.

Example implementation:

```go
type Message struct {
    Buffer []byte
    // Other fields
}

func (m *Message) Reset() {
    *m = Message{}
}

func (m *Message) ResetReuse() {
    s := m.Buffer[:0]
    *m = Message{}
    m.Buffer = s
}
```

Risks:

This approach has potential downsides. Since the heap space consumed by the
slices is not reclaimed, the storage node's memory consumption may increase.
Currently, there is no mechanism to shrink the heap usage.

Additionally, this PR changes the generated code. The protobuf compiler can
revert it, which is contrary to our intention. To catch this mistake, this PR
includes a unit test (github.com/kakao/varlog/proto/snpb.TestReplicateRequest)
to verify that the buffer backing the slices is reused.

Resolves: #795

See also: #806
ijsong added a commit that referenced this issue Jun 14, 2024
Improve unmarshaling performance by reusing buffers for ReplicateRequest in the
backup replica.

The protobuf message `github.com/kakao/varlog/proto/snpb.(ReplicateRequest)` has
two slice fields—LLSN (`[]uint64`) and Data (`[][]byte`). The backup replica
receives replicated log entries from the primary replica via the gRPC service
`github.com/kakao/varlog/proto/snpb.(ReplicatorServer).Replicate`, which sends
`ReplicateRequest` messages.

Upon receiving a `ReplicateRequest`, the backup replica unmarshals the message,
which involves growing slices for fields such as LLSN and Data. This growth
causes copy overhead whenever the slice capacities need to expand.

To address this, we introduce a new method, `ResetReuse`, for reusing slices
instead of resetting them completely. The `ResetReuse` method shrinks the slice
lengths while preserving their capacities, thus avoiding the overhead of
reallocating memory.

Example implementation:

```go
type Message struct {
    Buffer []byte
    // Other fields
}

func (m *Message) Reset() {
    *m = Message{}
}

func (m *Message) ResetReuse() {
    s := m.Buffer[:0]
    *m = Message{}
    m.Buffer = s
}
```

Risks:

This approach has potential downsides. Since the heap space consumed by the
slices is not reclaimed, the storage node's memory consumption may increase.
Currently, there is no mechanism to shrink the heap usage.

Additionally, this PR changes the generated code. The protobuf compiler can
revert it, which is contrary to our intention. To catch this mistake, this PR
includes a unit test (github.com/kakao/varlog/proto/snpb.TestReplicateRequest)
to verify that the buffer backing the slices is reused.

Resolves: #795

See also: #806
ijsong added a commit that referenced this issue Jun 15, 2024
Improve unmarshaling performance by reusing buffers for ReplicateRequest in the
backup replica.

The protobuf message `github.com/kakao/varlog/proto/snpb.(ReplicateRequest)` has
two slice fields—LLSN (`[]uint64`) and Data (`[][]byte`). The backup replica
receives replicated log entries from the primary replica via the gRPC service
`github.com/kakao/varlog/proto/snpb.(ReplicatorServer).Replicate`, which sends
`ReplicateRequest` messages.

Upon receiving a `ReplicateRequest`, the backup replica unmarshals the message,
which involves growing slices for fields such as LLSN and Data. This growth
causes copy overhead whenever the slice capacities need to expand.

To address this, we introduce a new method, `ResetReuse`, for reusing slices
instead of resetting them completely. The `ResetReuse` method shrinks the slice
lengths while preserving their capacities, thus avoiding the overhead of
reallocating memory.

Example implementation:

```go
type Message struct {
    Buffer []byte
    // Other fields
}

func (m *Message) Reset() {
    *m = Message{}
}

func (m *Message) ResetReuse() {
    s := m.Buffer[:0]
    *m = Message{}
    m.Buffer = s
}
```

Risks:

This approach has potential downsides. Since the heap space consumed by the
slices is not reclaimed, the storage node's memory consumption may increase.
Currently, there is no mechanism to shrink the heap usage.

Additionally, this PR changes the generated code. The protobuf compiler can
revert it, which is contrary to our intention. To catch this mistake, this PR
includes a unit test (github.com/kakao/varlog/proto/snpb.TestReplicateRequest)
to verify that the buffer backing the slices is reused.

Resolves: #795

See also: #806
ijsong added a commit that referenced this issue Jun 16, 2024
Improve unmarshaling performance by reusing buffers for ReplicateRequest in the
backup replica.

The protobuf message `github.com/kakao/varlog/proto/snpb.(ReplicateRequest)` has
two slice fields—LLSN (`[]uint64`) and Data (`[][]byte`). The backup replica
receives replicated log entries from the primary replica via the gRPC service
`github.com/kakao/varlog/proto/snpb.(ReplicatorServer).Replicate`, which sends
`ReplicateRequest` messages.

Upon receiving a `ReplicateRequest`, the backup replica unmarshals the message,
which involves growing slices for fields such as LLSN and Data. This growth
causes copy overhead whenever the slice capacities need to expand.

To address this, we introduce a new method, `ResetReuse`, for reusing slices
instead of resetting them completely. The `ResetReuse` method shrinks the slice
lengths while preserving their capacities, thus avoiding the overhead of
reallocating memory.

Example implementation:

```go
type Message struct {
    Buffer []byte
    // Other fields
}

func (m *Message) Reset() {
    *m = Message{}
}

func (m *Message) ResetReuse() {
    s := m.Buffer[:0]
    *m = Message{}
    m.Buffer = s
}
```

Risks:

This approach has potential downsides. Since the heap space consumed by the
slices is not reclaimed, the storage node's memory consumption may increase.
Currently, there is no mechanism to shrink the heap usage.

Additionally, this PR changes the generated code. The protobuf compiler can
revert it, which is contrary to our intention. To catch this mistake, this PR
includes a unit test (github.com/kakao/varlog/proto/snpb.TestReplicateRequest)
to verify that the buffer backing the slices is reused.

Resolves: #795

See also: #806
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging a pull request may close this issue.

1 participant