You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@skywalking.apache.org by "HHoflittlefish777 (via GitHub)" <gi...@apache.org> on 2023/03/19 04:14:31 UTC

[GitHub] [skywalking-banyandb] HHoflittlefish777 opened a new pull request, #261: Implement Write-ahead Logging

HHoflittlefish777 opened a new pull request, #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261

   Close https://github.com/apache/skywalking/issues/10301


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hailin0 commented on pull request #261: Implement Write-ahead Logging

Posted by "hailin0 (via GitHub)" <gi...@apache.org>.
hailin0 commented on PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#issuecomment-1595819929

   Sorry for this PR delay, I'll move on


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hanahmily commented on a diff in pull request #261: Implement Write-ahead Logging

Posted by "hanahmily (via GitHub)" <gi...@apache.org>.
hanahmily commented on code in PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#discussion_r1148685119


##########
pkg/wal/wal_test.go:
##########
@@ -0,0 +1,64 @@
+// Licensed to 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. Apache Software Foundation (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 version can be used to implement embedding versioning details from
+// git branches and tags into the binary importing this package.
+package wal
+
+import (
+	"os"
+	"path/filepath"
+	"testing"
+	"time"
+
+	"github.com/stretchr/testify/assert"
+)
+
+func TestWAL(t *testing.T) {
+	options := &Options{
+		Compression: true,
+		FileSize:    67108864,

Review Comment:
   Could you add a comment showing the file's size in a human-friendly form? 



##########
pkg/wal/wal_test.go:
##########
@@ -0,0 +1,64 @@
+// Licensed to 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. Apache Software Foundation (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 version can be used to implement embedding versioning details from
+// git branches and tags into the binary importing this package.
+package wal
+
+import (
+	"os"
+	"path/filepath"
+	"testing"
+	"time"
+
+	"github.com/stretchr/testify/assert"
+)
+
+func TestWAL(t *testing.T) {
+	options := &Options{
+		Compression: true,
+		FileSize:    67108864,
+		BufferSize:  16,
+	}
+
+	log, _ := New("test", options)
+	go func() {
+		for {
+			log.Write([]byte{0x01}, time.Now(), []byte{0x01})
+		}
+	}()
+	// ReadALL test.
+	segments, _ := log.ReadAllSegments()

Review Comment:
   `ReadAllSegment` is the replaying process of WAL. That means it usually takes place after a log is opened. 
   
   Based on that, you should close the log, then reopen it to test the reading.



##########
pkg/wal/wal_test.go:
##########
@@ -0,0 +1,64 @@
+// Licensed to 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. Apache Software Foundation (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 version can be used to implement embedding versioning details from
+// git branches and tags into the binary importing this package.
+package wal
+
+import (
+	"os"
+	"path/filepath"
+	"testing"
+	"time"
+
+	"github.com/stretchr/testify/assert"
+)
+
+func TestWAL(t *testing.T) {
+	options := &Options{
+		Compression: true,
+		FileSize:    67108864,
+		BufferSize:  16,
+	}
+
+	log, _ := New("test", options)
+	go func() {
+		for {
+			log.Write([]byte{0x01}, time.Now(), []byte{0x01})
+		}
+	}()
+	// ReadALL test.
+	segments, _ := log.ReadAllSegments()
+	for _, segment := range segments {
+		id := segment.GetSegmentID()
+		assert.Equal(t, int(id), 1)
+	}
+	// Rotate test.
+	segment, _ := log.Rotate()
+	segmentID := segment.GetSegmentID()
+	assert.Equal(t, int(segmentID), 1)
+	// Delete test.
+	log.Delete(segmentID)
+	segments, _ = log.ReadAllSegments()
+	for _, segment := range segments {
+		segmentID := segment.GetSegmentID()
+		assert.Equal(t, int(segmentID), 2)
+	}
+	// Delete test dir.
+	path, _ := filepath.Abs("test")
+	os.RemoveAll(path)

Review Comment:
   Closing the log after testing all scenarios.



##########
pkg/wal/wal_test.go:
##########
@@ -0,0 +1,64 @@
+// Licensed to 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. Apache Software Foundation (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 version can be used to implement embedding versioning details from
+// git branches and tags into the binary importing this package.
+package wal

Review Comment:
   The package should be `wal_test`



##########
pkg/wal/wal_test.go:
##########
@@ -0,0 +1,64 @@
+// Licensed to 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. Apache Software Foundation (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 version can be used to implement embedding versioning details from
+// git branches and tags into the binary importing this package.
+package wal
+
+import (
+	"os"
+	"path/filepath"
+	"testing"
+	"time"
+
+	"github.com/stretchr/testify/assert"
+)
+
+func TestWAL(t *testing.T) {
+	options := &Options{
+		Compression: true,
+		FileSize:    67108864,
+		BufferSize:  16,
+	}
+
+	log, _ := New("test", options)
+	go func() {
+		for {
+			log.Write([]byte{0x01}, time.Now(), []byte{0x01})
+		}
+	}()
+	// ReadALL test.
+	segments, _ := log.ReadAllSegments()
+	for _, segment := range segments {
+		id := segment.GetSegmentID()
+		assert.Equal(t, int(id), 1)
+	}
+	// Rotate test.
+	segment, _ := log.Rotate()
+	segmentID := segment.GetSegmentID()
+	assert.Equal(t, int(segmentID), 1)

Review Comment:
   After rotating, you should check all data are purged 



##########
pkg/wal/wal_test.go:
##########
@@ -0,0 +1,64 @@
+// Licensed to 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. Apache Software Foundation (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 version can be used to implement embedding versioning details from
+// git branches and tags into the binary importing this package.
+package wal
+
+import (
+	"os"
+	"path/filepath"
+	"testing"
+	"time"
+
+	"github.com/stretchr/testify/assert"
+)
+
+func TestWAL(t *testing.T) {
+	options := &Options{
+		Compression: true,
+		FileSize:    67108864,
+		BufferSize:  16,
+	}
+
+	log, _ := New("test", options)

Review Comment:
   You can't ignore the error. 



##########
pkg/wal/wal_test.go:
##########
@@ -0,0 +1,64 @@
+// Licensed to 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. Apache Software Foundation (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 version can be used to implement embedding versioning details from
+// git branches and tags into the binary importing this package.
+package wal
+
+import (
+	"os"
+	"path/filepath"
+	"testing"
+	"time"
+
+	"github.com/stretchr/testify/assert"
+)
+
+func TestWAL(t *testing.T) {
+	options := &Options{
+		Compression: true,
+		FileSize:    67108864,
+		BufferSize:  16,
+	}
+
+	log, _ := New("test", options)
+	go func() {
+		for {

Review Comment:
   This goroutine is running after testing reaches the end. That causes a leak.
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hanahmily merged pull request #261: Implement Write-ahead Logging

Posted by "hanahmily (via GitHub)" <gi...@apache.org>.
hanahmily merged PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hanahmily commented on pull request #261: Implement Write-ahead Logging

Posted by "hanahmily (via GitHub)" <gi...@apache.org>.
hanahmily commented on PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#issuecomment-1537274899

   > This PR seems not to update for weeks. Is this still being processed
   
   Certainly, I will processed with it later, after completing load testing.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hanahmily commented on a diff in pull request #261: Implement Write-ahead Logging

Posted by "hanahmily (via GitHub)" <gi...@apache.org>.
hanahmily commented on code in PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#discussion_r1246590390


##########
pkg/wal/wal.go:
##########
@@ -42,19 +75,759 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// log implements the WAL interface.
+type log struct {
+	entryCloser      *run.Closer
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentMap       map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+	writeWaitGroup   sync.WaitGroup
+	flushWaitGroup   sync.WaitGroup
+	workSegmentMutex sync.Mutex
+	segmentMapMutex  sync.RWMutex

Review Comment:
   We ought to use a distinct mutex since the rotation of the work segment always leads to an update in the segment map.



##########
pkg/wal/wal.go:
##########
@@ -42,19 +75,759 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// log implements the WAL interface.
+type log struct {
+	entryCloser      *run.Closer
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentMap       map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+	writeWaitGroup   sync.WaitGroup
+	flushWaitGroup   sync.WaitGroup

Review Comment:
   Use "entryClose" to replace them. You can initialize `closer` by `run.NewCloser(3)`.



##########
pkg/wal/wal.go:
##########
@@ -42,19 +75,759 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// log implements the WAL interface.
+type log struct {
+	entryCloser      *run.Closer
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentMap       map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+	writeWaitGroup   sync.WaitGroup
+	flushWaitGroup   sync.WaitGroup
+	workSegmentMutex sync.Mutex
+	segmentMapMutex  sync.RWMutex
+	closerOnce       sync.Once
+}
+
+type segment struct {
+	file       *os.File
+	path       string
+	logEntries []LogEntry
+	segmentID  SegmentID
+}
+
+type logRequest struct {
+	seriesID  common.SeriesIDV2
+	timestamp time.Time
+	callback  func(common.SeriesIDV2, time.Time, []byte, error)
+	data      []byte
+}
+
+type logEntry struct {
+	timestamps  []time.Time
+	values      *list.List
+	seriesID    common.SeriesIDV2
+	entryLength int64
+	count       int32
+}
+
+type buffer struct {
+	timestampMap map[common.SeriesIDV2][]time.Time
+	valueMap     map[common.SeriesIDV2][]byte
+	callbackMap  map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)
+	count        int
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	walOptions := DefaultOptions
+	if options != nil {
+		fileSize := options.FileSize
+		if fileSize <= 0 {
+			fileSize = DefaultOptions.FileSize
+		}
+		bufferSize := options.BufferSize
+		if bufferSize <= 0 {
+			bufferSize = DefaultOptions.BufferSize
+		}
+		bufferBatchInterval := options.BufferBatchInterval
+		if bufferBatchInterval <= 0 {
+			bufferBatchInterval = DefaultOptions.BufferBatchInterval
+		}
+		walOptions = &Options{
+			FileSize:            fileSize,
+			BufferSize:          bufferSize,
+			BufferBatchInterval: bufferBatchInterval,
+		}
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path: "+path)
+	}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	log := &log{
+		path:             path,
+		options:          *walOptions,
+		logger:           logger.GetLogger(moduleName),
+		writeChannel:     make(chan logRequest),
+		flushChannel:     make(chan buffer, walOptions.FlushQueueSize),
+		bytesBuffer:      bytes.NewBuffer([]byte{}),
+		timestampsBuffer: bytes.NewBuffer([]byte{}),
+		entryCloser:      run.NewCloser(1),
+		buffer: buffer{
+			timestampMap: make(map[common.SeriesIDV2][]time.Time),
+			valueMap:     make(map[common.SeriesIDV2][]byte),
+			callbackMap:  make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)),
+			count:        0,
+		},
+	}
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+
+	log.writeWaitGroup.Add(1)
+	log.flushWaitGroup.Add(1)
+	log.start()
+
+	log.logger.Info().Str("path", path).Msg("WAL has be initialized")
+	return log, nil
+}
+
+// Write a logging entity.
+// It will return immediately when the data is written in the buffer,
+// The callback function will be called when the entity is flushed on the persistent storage.
+func (log *log) Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error)) {
+	if !log.entryCloser.AddRunning() {
+		return
+	}
+	defer log.entryCloser.Done()
+
+	log.writeChannel <- logRequest{
+		seriesID:  seriesID,
+		timestamp: timestamp,
+		data:      data,
+		callback:  callback,
+	}
+}
+
+// Read specified segment by SegmentID.
+func (log *log) Read(segmentID SegmentID) (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segment := log.segmentMap[segmentID]
+	return segment, nil
+}
+
+// ReadAllSegments reads all segments sorted by their creation time in ascending order.
+func (log *log) ReadAllSegments() ([]Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segments := make([]Segment, 0)
+	for _, segment := range log.segmentMap {
+		segments = append(segments, segment)
+	}
+	return segments, nil
+}
+
+// Rotate closes the open segment and opens a new one, returning the closed segment details.
+func (log *log) Rotate() (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	oldWorkSegment, err := log.swapWorkSegment()
+	if err != nil {
+		return nil, err
+	}
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Update segment information.
+	workSegment := log.workSegment
+	log.segmentMap[workSegment.segmentID] = workSegment
+	return oldWorkSegment, nil
+}
+
+// Delete the specified segment.
+func (log *log) Delete(segmentID SegmentID) error {
+	if !log.entryCloser.AddRunning() {
+		return errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Segment which will be deleted must be closed.
+	if segmentID == log.workSegment.segmentID {
+		return errors.New("Can not delete the segment which is working")
+	}
+
+	err := os.Remove(log.segmentMap[segmentID].path)
+	if err != nil {
+		return errors.Wrap(err, "Delete WAL segment error")
+	}
+	delete(log.segmentMap, segmentID)
+	return nil
+}
+
+// Close all of segments and stop WAL work.
+func (log *log) Close() error {
+	log.closerOnce.Do(func() {
+		log.logger.Info().Msg("Closing WAL...")
+
+		log.entryCloser.Done()
+		log.entryCloser.CloseThenWait()
+
+		close(log.writeChannel)
+		log.writeWaitGroup.Wait()
+
+		close(log.flushChannel)
+		log.flushWaitGroup.Wait()
+
+		if err := log.flushBuffer(log.buffer); err != nil {
+			log.logger.Err(err).Msg("Flushing buffer failed")

Review Comment:
   Please return the error more than log it.



##########
pkg/wal/wal.go:
##########
@@ -42,19 +75,759 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// log implements the WAL interface.
+type log struct {
+	entryCloser      *run.Closer

Review Comment:
   You seem to be overusing the `closer`. Its purpose is to ensure the safe closing of the `chan`. Therefore, it should only be utilized in methods that interact with `writeChannel` and `flushChannel`. There is no need to apply the `closer` to other methods, such as `Read`, `ReadAll`, `Rotate`, and `Delete`.
   
   



##########
pkg/wal/wal.go:
##########
@@ -42,19 +75,759 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// log implements the WAL interface.
+type log struct {
+	entryCloser      *run.Closer
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentMap       map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+	writeWaitGroup   sync.WaitGroup
+	flushWaitGroup   sync.WaitGroup
+	workSegmentMutex sync.Mutex
+	segmentMapMutex  sync.RWMutex
+	closerOnce       sync.Once
+}
+
+type segment struct {
+	file       *os.File
+	path       string
+	logEntries []LogEntry
+	segmentID  SegmentID
+}
+
+type logRequest struct {
+	seriesID  common.SeriesIDV2
+	timestamp time.Time
+	callback  func(common.SeriesIDV2, time.Time, []byte, error)
+	data      []byte
+}
+
+type logEntry struct {
+	timestamps  []time.Time
+	values      *list.List
+	seriesID    common.SeriesIDV2
+	entryLength int64
+	count       int32
+}
+
+type buffer struct {
+	timestampMap map[common.SeriesIDV2][]time.Time
+	valueMap     map[common.SeriesIDV2][]byte
+	callbackMap  map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)
+	count        int
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	walOptions := DefaultOptions
+	if options != nil {
+		fileSize := options.FileSize
+		if fileSize <= 0 {
+			fileSize = DefaultOptions.FileSize
+		}
+		bufferSize := options.BufferSize
+		if bufferSize <= 0 {
+			bufferSize = DefaultOptions.BufferSize
+		}
+		bufferBatchInterval := options.BufferBatchInterval
+		if bufferBatchInterval <= 0 {
+			bufferBatchInterval = DefaultOptions.BufferBatchInterval
+		}
+		walOptions = &Options{
+			FileSize:            fileSize,
+			BufferSize:          bufferSize,
+			BufferBatchInterval: bufferBatchInterval,
+		}
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path: "+path)
+	}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	log := &log{
+		path:             path,
+		options:          *walOptions,
+		logger:           logger.GetLogger(moduleName),
+		writeChannel:     make(chan logRequest),
+		flushChannel:     make(chan buffer, walOptions.FlushQueueSize),
+		bytesBuffer:      bytes.NewBuffer([]byte{}),
+		timestampsBuffer: bytes.NewBuffer([]byte{}),
+		entryCloser:      run.NewCloser(1),
+		buffer: buffer{
+			timestampMap: make(map[common.SeriesIDV2][]time.Time),
+			valueMap:     make(map[common.SeriesIDV2][]byte),
+			callbackMap:  make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)),
+			count:        0,
+		},
+	}
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+
+	log.writeWaitGroup.Add(1)
+	log.flushWaitGroup.Add(1)
+	log.start()
+
+	log.logger.Info().Str("path", path).Msg("WAL has be initialized")
+	return log, nil
+}
+
+// Write a logging entity.
+// It will return immediately when the data is written in the buffer,
+// The callback function will be called when the entity is flushed on the persistent storage.
+func (log *log) Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error)) {
+	if !log.entryCloser.AddRunning() {
+		return
+	}
+	defer log.entryCloser.Done()
+
+	log.writeChannel <- logRequest{
+		seriesID:  seriesID,
+		timestamp: timestamp,
+		data:      data,
+		callback:  callback,
+	}
+}
+
+// Read specified segment by SegmentID.
+func (log *log) Read(segmentID SegmentID) (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segment := log.segmentMap[segmentID]
+	return segment, nil
+}
+
+// ReadAllSegments reads all segments sorted by their creation time in ascending order.
+func (log *log) ReadAllSegments() ([]Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segments := make([]Segment, 0)
+	for _, segment := range log.segmentMap {
+		segments = append(segments, segment)
+	}
+	return segments, nil
+}
+
+// Rotate closes the open segment and opens a new one, returning the closed segment details.
+func (log *log) Rotate() (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	oldWorkSegment, err := log.swapWorkSegment()
+	if err != nil {
+		return nil, err
+	}
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Update segment information.
+	workSegment := log.workSegment
+	log.segmentMap[workSegment.segmentID] = workSegment
+	return oldWorkSegment, nil
+}
+
+// Delete the specified segment.
+func (log *log) Delete(segmentID SegmentID) error {
+	if !log.entryCloser.AddRunning() {
+		return errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Segment which will be deleted must be closed.
+	if segmentID == log.workSegment.segmentID {
+		return errors.New("Can not delete the segment which is working")
+	}
+
+	err := os.Remove(log.segmentMap[segmentID].path)
+	if err != nil {
+		return errors.Wrap(err, "Delete WAL segment error")
+	}
+	delete(log.segmentMap, segmentID)
+	return nil
+}
+
+// Close all of segments and stop WAL work.
+func (log *log) Close() error {
+	log.closerOnce.Do(func() {
+		log.logger.Info().Msg("Closing WAL...")
+
+		log.entryCloser.Done()
+		log.entryCloser.CloseThenWait()
+
+		close(log.writeChannel)
+		log.writeWaitGroup.Wait()
+
+		close(log.flushChannel)
+		log.flushWaitGroup.Wait()
+
+		if err := log.flushBuffer(log.buffer); err != nil {
+			log.logger.Err(err).Msg("Flushing buffer failed")
+		}
+		if err := log.workSegment.file.Close(); err != nil {
+			log.logger.Err(err).Msg("Close work segment file error")
+		}
+		log.logger.Info().Msg("Closed WAL")
+	})
+	return nil
+}
+
+func (log *log) start() {
+	go func() {
+		log.logger.Info().Msg("Start batch task...")
+
+		defer log.writeWaitGroup.Done()
+
+		bufferVolume := 0
+		for {
+			timer := time.NewTimer(log.options.BufferBatchInterval)
+			select {
+			case request, chOpen := <-log.writeChannel:
+				if !chOpen {
+					log.logger.Info().Msg("Stop batch task when write-channel closed!")
+					return
+				}
+
+				log.buffer.write(request)
+				if log.logger.Debug().Enabled() {
+					log.logger.Debug().Msg("Write request to buffer. elements: " + strconv.Itoa(log.buffer.count))
+				}
+
+				bufferVolume += len(request.seriesID.Marshal()) + timestampVolumeLength + len(request.data)
+				if bufferVolume > log.options.BufferSize {
+					log.triggerFlushing()
+					bufferVolume = 0
+				}
+				continue

Review Comment:
   ```suggestion
   ```



##########
pkg/wal/wal.go:
##########
@@ -42,19 +75,759 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// log implements the WAL interface.
+type log struct {
+	entryCloser      *run.Closer
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentMap       map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+	writeWaitGroup   sync.WaitGroup
+	flushWaitGroup   sync.WaitGroup
+	workSegmentMutex sync.Mutex
+	segmentMapMutex  sync.RWMutex
+	closerOnce       sync.Once
+}
+
+type segment struct {
+	file       *os.File
+	path       string
+	logEntries []LogEntry
+	segmentID  SegmentID
+}
+
+type logRequest struct {
+	seriesID  common.SeriesIDV2
+	timestamp time.Time
+	callback  func(common.SeriesIDV2, time.Time, []byte, error)
+	data      []byte
+}
+
+type logEntry struct {
+	timestamps  []time.Time
+	values      *list.List
+	seriesID    common.SeriesIDV2
+	entryLength int64
+	count       int32
+}
+
+type buffer struct {
+	timestampMap map[common.SeriesIDV2][]time.Time
+	valueMap     map[common.SeriesIDV2][]byte
+	callbackMap  map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)
+	count        int
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	walOptions := DefaultOptions
+	if options != nil {
+		fileSize := options.FileSize
+		if fileSize <= 0 {
+			fileSize = DefaultOptions.FileSize
+		}
+		bufferSize := options.BufferSize
+		if bufferSize <= 0 {
+			bufferSize = DefaultOptions.BufferSize
+		}
+		bufferBatchInterval := options.BufferBatchInterval
+		if bufferBatchInterval <= 0 {
+			bufferBatchInterval = DefaultOptions.BufferBatchInterval
+		}
+		walOptions = &Options{
+			FileSize:            fileSize,
+			BufferSize:          bufferSize,
+			BufferBatchInterval: bufferBatchInterval,
+		}
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path: "+path)
+	}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	log := &log{
+		path:             path,
+		options:          *walOptions,
+		logger:           logger.GetLogger(moduleName),
+		writeChannel:     make(chan logRequest),
+		flushChannel:     make(chan buffer, walOptions.FlushQueueSize),
+		bytesBuffer:      bytes.NewBuffer([]byte{}),
+		timestampsBuffer: bytes.NewBuffer([]byte{}),
+		entryCloser:      run.NewCloser(1),
+		buffer: buffer{
+			timestampMap: make(map[common.SeriesIDV2][]time.Time),
+			valueMap:     make(map[common.SeriesIDV2][]byte),
+			callbackMap:  make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)),
+			count:        0,
+		},
+	}
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+
+	log.writeWaitGroup.Add(1)
+	log.flushWaitGroup.Add(1)
+	log.start()
+
+	log.logger.Info().Str("path", path).Msg("WAL has be initialized")
+	return log, nil
+}
+
+// Write a logging entity.
+// It will return immediately when the data is written in the buffer,
+// The callback function will be called when the entity is flushed on the persistent storage.
+func (log *log) Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error)) {
+	if !log.entryCloser.AddRunning() {
+		return
+	}
+	defer log.entryCloser.Done()
+
+	log.writeChannel <- logRequest{
+		seriesID:  seriesID,
+		timestamp: timestamp,
+		data:      data,
+		callback:  callback,
+	}
+}
+
+// Read specified segment by SegmentID.
+func (log *log) Read(segmentID SegmentID) (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segment := log.segmentMap[segmentID]
+	return segment, nil
+}
+
+// ReadAllSegments reads all segments sorted by their creation time in ascending order.
+func (log *log) ReadAllSegments() ([]Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segments := make([]Segment, 0)
+	for _, segment := range log.segmentMap {
+		segments = append(segments, segment)
+	}
+	return segments, nil
+}
+
+// Rotate closes the open segment and opens a new one, returning the closed segment details.
+func (log *log) Rotate() (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	oldWorkSegment, err := log.swapWorkSegment()
+	if err != nil {
+		return nil, err
+	}
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Update segment information.
+	workSegment := log.workSegment
+	log.segmentMap[workSegment.segmentID] = workSegment
+	return oldWorkSegment, nil
+}
+
+// Delete the specified segment.
+func (log *log) Delete(segmentID SegmentID) error {
+	if !log.entryCloser.AddRunning() {
+		return errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Segment which will be deleted must be closed.
+	if segmentID == log.workSegment.segmentID {
+		return errors.New("Can not delete the segment which is working")
+	}
+
+	err := os.Remove(log.segmentMap[segmentID].path)
+	if err != nil {
+		return errors.Wrap(err, "Delete WAL segment error")
+	}
+	delete(log.segmentMap, segmentID)
+	return nil
+}
+
+// Close all of segments and stop WAL work.
+func (log *log) Close() error {
+	log.closerOnce.Do(func() {
+		log.logger.Info().Msg("Closing WAL...")
+
+		log.entryCloser.Done()
+		log.entryCloser.CloseThenWait()
+
+		close(log.writeChannel)
+		log.writeWaitGroup.Wait()
+
+		close(log.flushChannel)
+		log.flushWaitGroup.Wait()
+
+		if err := log.flushBuffer(log.buffer); err != nil {
+			log.logger.Err(err).Msg("Flushing buffer failed")
+		}
+		if err := log.workSegment.file.Close(); err != nil {
+			log.logger.Err(err).Msg("Close work segment file error")
+		}
+		log.logger.Info().Msg("Closed WAL")
+	})
+	return nil
+}
+
+func (log *log) start() {
+	go func() {
+		log.logger.Info().Msg("Start batch task...")
+
+		defer log.writeWaitGroup.Done()
+
+		bufferVolume := 0
+		for {
+			timer := time.NewTimer(log.options.BufferBatchInterval)
+			select {
+			case request, chOpen := <-log.writeChannel:
+				if !chOpen {
+					log.logger.Info().Msg("Stop batch task when write-channel closed!")

Review Comment:
   Don't use "!" in the log.



##########
pkg/wal/wal.go:
##########
@@ -42,19 +75,759 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// log implements the WAL interface.
+type log struct {
+	entryCloser      *run.Closer
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentMap       map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+	writeWaitGroup   sync.WaitGroup
+	flushWaitGroup   sync.WaitGroup
+	workSegmentMutex sync.Mutex
+	segmentMapMutex  sync.RWMutex
+	closerOnce       sync.Once
+}
+
+type segment struct {
+	file       *os.File
+	path       string
+	logEntries []LogEntry
+	segmentID  SegmentID
+}
+
+type logRequest struct {
+	seriesID  common.SeriesIDV2
+	timestamp time.Time
+	callback  func(common.SeriesIDV2, time.Time, []byte, error)
+	data      []byte
+}
+
+type logEntry struct {
+	timestamps  []time.Time
+	values      *list.List
+	seriesID    common.SeriesIDV2
+	entryLength int64
+	count       int32
+}
+
+type buffer struct {
+	timestampMap map[common.SeriesIDV2][]time.Time
+	valueMap     map[common.SeriesIDV2][]byte
+	callbackMap  map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)
+	count        int
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	walOptions := DefaultOptions
+	if options != nil {
+		fileSize := options.FileSize
+		if fileSize <= 0 {
+			fileSize = DefaultOptions.FileSize
+		}
+		bufferSize := options.BufferSize
+		if bufferSize <= 0 {
+			bufferSize = DefaultOptions.BufferSize
+		}
+		bufferBatchInterval := options.BufferBatchInterval
+		if bufferBatchInterval <= 0 {
+			bufferBatchInterval = DefaultOptions.BufferBatchInterval
+		}
+		walOptions = &Options{
+			FileSize:            fileSize,
+			BufferSize:          bufferSize,
+			BufferBatchInterval: bufferBatchInterval,
+		}
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path: "+path)
+	}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	log := &log{
+		path:             path,
+		options:          *walOptions,
+		logger:           logger.GetLogger(moduleName),
+		writeChannel:     make(chan logRequest),
+		flushChannel:     make(chan buffer, walOptions.FlushQueueSize),
+		bytesBuffer:      bytes.NewBuffer([]byte{}),
+		timestampsBuffer: bytes.NewBuffer([]byte{}),
+		entryCloser:      run.NewCloser(1),
+		buffer: buffer{
+			timestampMap: make(map[common.SeriesIDV2][]time.Time),
+			valueMap:     make(map[common.SeriesIDV2][]byte),
+			callbackMap:  make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)),
+			count:        0,
+		},
+	}
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+
+	log.writeWaitGroup.Add(1)
+	log.flushWaitGroup.Add(1)
+	log.start()
+
+	log.logger.Info().Str("path", path).Msg("WAL has be initialized")
+	return log, nil
+}
+
+// Write a logging entity.
+// It will return immediately when the data is written in the buffer,
+// The callback function will be called when the entity is flushed on the persistent storage.
+func (log *log) Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error)) {
+	if !log.entryCloser.AddRunning() {
+		return
+	}
+	defer log.entryCloser.Done()
+
+	log.writeChannel <- logRequest{
+		seriesID:  seriesID,
+		timestamp: timestamp,
+		data:      data,
+		callback:  callback,
+	}
+}
+
+// Read specified segment by SegmentID.
+func (log *log) Read(segmentID SegmentID) (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segment := log.segmentMap[segmentID]
+	return segment, nil
+}
+
+// ReadAllSegments reads all segments sorted by their creation time in ascending order.
+func (log *log) ReadAllSegments() ([]Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segments := make([]Segment, 0)
+	for _, segment := range log.segmentMap {
+		segments = append(segments, segment)
+	}
+	return segments, nil
+}
+
+// Rotate closes the open segment and opens a new one, returning the closed segment details.
+func (log *log) Rotate() (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	oldWorkSegment, err := log.swapWorkSegment()
+	if err != nil {
+		return nil, err
+	}
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Update segment information.
+	workSegment := log.workSegment
+	log.segmentMap[workSegment.segmentID] = workSegment
+	return oldWorkSegment, nil
+}
+
+// Delete the specified segment.
+func (log *log) Delete(segmentID SegmentID) error {
+	if !log.entryCloser.AddRunning() {
+		return errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Segment which will be deleted must be closed.
+	if segmentID == log.workSegment.segmentID {
+		return errors.New("Can not delete the segment which is working")
+	}
+
+	err := os.Remove(log.segmentMap[segmentID].path)
+	if err != nil {
+		return errors.Wrap(err, "Delete WAL segment error")
+	}
+	delete(log.segmentMap, segmentID)
+	return nil
+}
+
+// Close all of segments and stop WAL work.
+func (log *log) Close() error {
+	log.closerOnce.Do(func() {
+		log.logger.Info().Msg("Closing WAL...")
+
+		log.entryCloser.Done()
+		log.entryCloser.CloseThenWait()
+
+		close(log.writeChannel)
+		log.writeWaitGroup.Wait()
+
+		close(log.flushChannel)
+		log.flushWaitGroup.Wait()
+
+		if err := log.flushBuffer(log.buffer); err != nil {
+			log.logger.Err(err).Msg("Flushing buffer failed")
+		}
+		if err := log.workSegment.file.Close(); err != nil {
+			log.logger.Err(err).Msg("Close work segment file error")
+		}
+		log.logger.Info().Msg("Closed WAL")
+	})
+	return nil
+}
+
+func (log *log) start() {
+	go func() {
+		log.logger.Info().Msg("Start batch task...")
+
+		defer log.writeWaitGroup.Done()
+
+		bufferVolume := 0
+		for {
+			timer := time.NewTimer(log.options.BufferBatchInterval)
+			select {
+			case request, chOpen := <-log.writeChannel:
+				if !chOpen {
+					log.logger.Info().Msg("Stop batch task when write-channel closed!")
+					return
+				}
+
+				log.buffer.write(request)
+				if log.logger.Debug().Enabled() {
+					log.logger.Debug().Msg("Write request to buffer. elements: " + strconv.Itoa(log.buffer.count))
+				}
+
+				bufferVolume += len(request.seriesID.Marshal()) + timestampVolumeLength + len(request.data)
+				if bufferVolume > log.options.BufferSize {
+					log.triggerFlushing()
+					bufferVolume = 0
+				}
+				continue
+			case <-timer.C:
+				if bufferVolume == 0 {
+					continue
+				}
+				log.triggerFlushing()
+				bufferVolume = 0
+				continue
+			}
+		}
+	}()
+
+	go func() {
+		log.logger.Info().Msg("Start flush task...")
+
+		defer log.flushWaitGroup.Done()
+
+		for batch := range log.flushChannel {
+			startTime := time.Now()
+
+			var err error
+			for i := 0; i < maxRetries; i++ {
+				if err = log.flushBuffer(batch); err != nil {
+					log.logger.Err(err).Msg("Flushing buffer failed. Retrying...")
+					time.Sleep(100 * time.Millisecond)
+					continue
+				}
+				break
+			}
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Flushed buffer to WAL file. elements: " +
+					strconv.Itoa(batch.count) + ", cost: " + time.Since(startTime).String())
+			}
+
+			batch.notifyRequests(err)
+		}
+		log.logger.Info().Msg("Stop flush task when flush-channel closed!")
+	}()
+
+	log.logger.Info().Msg("Started WAL")
+}
+
+func (log *log) triggerFlushing() {
+	for {
+		select {
+		case log.flushChannel <- log.buffer:
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Send buffer to flush-channel. elements: " + strconv.Itoa(log.buffer.count))
+			}
+			log.newBuffer()
+			return
+		default:
+		}
+		time.Sleep(10 * time.Millisecond)
+	}
+}
+
+func (log *log) newBuffer() {
+	log.buffer.timestampMap = make(map[common.SeriesIDV2][]time.Time)
+	log.buffer.valueMap = make(map[common.SeriesIDV2][]byte)
+	log.buffer.callbackMap = make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error))
+	log.buffer.count = 0
+}
+
+func (log *log) flushBuffer(buffer buffer) error {
+	if buffer.count == 0 {
+		return nil
+	}
+
+	defer func() {
+		log.bytesBuffer.Reset()
+		log.timestampsBuffer.Reset()
+	}()
+
+	// placeholder, preset batch length value is 0
+	batchLen := 0
+	if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(batchLen)); err != nil {
+		return errors.Wrap(err, "Write batch length error")
+	}
+	for seriesID, timestamps := range buffer.timestampMap {
+		// Generate seriesID binary
+		seriesIDBytes := seriesID.Marshal()
+		seriesIDBytesLen := len(seriesIDBytes)
+
+		// Generate timestamps compression binary
+		log.timestampsBuffer.Reset()
+		timestampWriter := encoding.NewWriter()
+		timestampEncoder := encoding.NewXOREncoder(timestampWriter)
+		timestampWriter.Reset(log.timestampsBuffer)
+		for _, timestamp := range timestamps {
+			timestampEncoder.Write(timeTouUnixNano(timestamp))
+		}
+		timestampWriter.Flush()
+		timestampsBytes := log.timestampsBuffer.Bytes()
+		timestampsBytesLen := len(timestampsBytes)
+
+		// Generate values compression binary
+		valuesBytes := snappy.Encode(nil, buffer.valueMap[seriesID])
+
+		// Write entry data
+		entryLen := seriesIDLength + seriesIDBytesLen + seriesCountLength + timestampsBinaryLength + timestampsBytesLen + len(valuesBytes)
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(entryLen)); err != nil {

Review Comment:
   You should provide several binary packing/unpacking methods to conceal bytesBuffer and the binary order.



##########
pkg/wal/wal.go:
##########
@@ -42,19 +75,759 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// log implements the WAL interface.
+type log struct {
+	entryCloser      *run.Closer
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentMap       map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+	writeWaitGroup   sync.WaitGroup
+	flushWaitGroup   sync.WaitGroup
+	workSegmentMutex sync.Mutex
+	segmentMapMutex  sync.RWMutex
+	closerOnce       sync.Once
+}
+
+type segment struct {
+	file       *os.File
+	path       string
+	logEntries []LogEntry
+	segmentID  SegmentID
+}
+
+type logRequest struct {
+	seriesID  common.SeriesIDV2
+	timestamp time.Time
+	callback  func(common.SeriesIDV2, time.Time, []byte, error)
+	data      []byte
+}
+
+type logEntry struct {
+	timestamps  []time.Time
+	values      *list.List
+	seriesID    common.SeriesIDV2
+	entryLength int64
+	count       int32
+}
+
+type buffer struct {
+	timestampMap map[common.SeriesIDV2][]time.Time
+	valueMap     map[common.SeriesIDV2][]byte
+	callbackMap  map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)
+	count        int
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	walOptions := DefaultOptions
+	if options != nil {
+		fileSize := options.FileSize
+		if fileSize <= 0 {
+			fileSize = DefaultOptions.FileSize
+		}
+		bufferSize := options.BufferSize
+		if bufferSize <= 0 {
+			bufferSize = DefaultOptions.BufferSize
+		}
+		bufferBatchInterval := options.BufferBatchInterval
+		if bufferBatchInterval <= 0 {
+			bufferBatchInterval = DefaultOptions.BufferBatchInterval
+		}
+		walOptions = &Options{
+			FileSize:            fileSize,
+			BufferSize:          bufferSize,
+			BufferBatchInterval: bufferBatchInterval,
+		}
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path: "+path)
+	}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	log := &log{
+		path:             path,
+		options:          *walOptions,
+		logger:           logger.GetLogger(moduleName),
+		writeChannel:     make(chan logRequest),
+		flushChannel:     make(chan buffer, walOptions.FlushQueueSize),
+		bytesBuffer:      bytes.NewBuffer([]byte{}),
+		timestampsBuffer: bytes.NewBuffer([]byte{}),
+		entryCloser:      run.NewCloser(1),
+		buffer: buffer{
+			timestampMap: make(map[common.SeriesIDV2][]time.Time),
+			valueMap:     make(map[common.SeriesIDV2][]byte),
+			callbackMap:  make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)),
+			count:        0,
+		},
+	}
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+
+	log.writeWaitGroup.Add(1)
+	log.flushWaitGroup.Add(1)
+	log.start()
+
+	log.logger.Info().Str("path", path).Msg("WAL has be initialized")
+	return log, nil
+}
+
+// Write a logging entity.
+// It will return immediately when the data is written in the buffer,
+// The callback function will be called when the entity is flushed on the persistent storage.
+func (log *log) Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error)) {
+	if !log.entryCloser.AddRunning() {
+		return
+	}
+	defer log.entryCloser.Done()
+
+	log.writeChannel <- logRequest{
+		seriesID:  seriesID,
+		timestamp: timestamp,
+		data:      data,
+		callback:  callback,
+	}
+}
+
+// Read specified segment by SegmentID.
+func (log *log) Read(segmentID SegmentID) (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segment := log.segmentMap[segmentID]
+	return segment, nil
+}
+
+// ReadAllSegments reads all segments sorted by their creation time in ascending order.
+func (log *log) ReadAllSegments() ([]Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segments := make([]Segment, 0)
+	for _, segment := range log.segmentMap {
+		segments = append(segments, segment)
+	}
+	return segments, nil
+}
+
+// Rotate closes the open segment and opens a new one, returning the closed segment details.
+func (log *log) Rotate() (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	oldWorkSegment, err := log.swapWorkSegment()
+	if err != nil {
+		return nil, err
+	}
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Update segment information.
+	workSegment := log.workSegment
+	log.segmentMap[workSegment.segmentID] = workSegment
+	return oldWorkSegment, nil
+}
+
+// Delete the specified segment.
+func (log *log) Delete(segmentID SegmentID) error {
+	if !log.entryCloser.AddRunning() {
+		return errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Segment which will be deleted must be closed.
+	if segmentID == log.workSegment.segmentID {
+		return errors.New("Can not delete the segment which is working")
+	}
+
+	err := os.Remove(log.segmentMap[segmentID].path)
+	if err != nil {
+		return errors.Wrap(err, "Delete WAL segment error")
+	}
+	delete(log.segmentMap, segmentID)
+	return nil
+}
+
+// Close all of segments and stop WAL work.
+func (log *log) Close() error {
+	log.closerOnce.Do(func() {
+		log.logger.Info().Msg("Closing WAL...")
+
+		log.entryCloser.Done()
+		log.entryCloser.CloseThenWait()
+
+		close(log.writeChannel)
+		log.writeWaitGroup.Wait()
+
+		close(log.flushChannel)
+		log.flushWaitGroup.Wait()
+
+		if err := log.flushBuffer(log.buffer); err != nil {
+			log.logger.Err(err).Msg("Flushing buffer failed")
+		}
+		if err := log.workSegment.file.Close(); err != nil {
+			log.logger.Err(err).Msg("Close work segment file error")
+		}
+		log.logger.Info().Msg("Closed WAL")
+	})
+	return nil
+}
+
+func (log *log) start() {
+	go func() {
+		log.logger.Info().Msg("Start batch task...")
+
+		defer log.writeWaitGroup.Done()
+
+		bufferVolume := 0
+		for {
+			timer := time.NewTimer(log.options.BufferBatchInterval)
+			select {
+			case request, chOpen := <-log.writeChannel:
+				if !chOpen {
+					log.logger.Info().Msg("Stop batch task when write-channel closed!")
+					return
+				}
+
+				log.buffer.write(request)
+				if log.logger.Debug().Enabled() {
+					log.logger.Debug().Msg("Write request to buffer. elements: " + strconv.Itoa(log.buffer.count))
+				}
+
+				bufferVolume += len(request.seriesID.Marshal()) + timestampVolumeLength + len(request.data)
+				if bufferVolume > log.options.BufferSize {
+					log.triggerFlushing()
+					bufferVolume = 0
+				}
+				continue
+			case <-timer.C:
+				if bufferVolume == 0 {
+					continue
+				}
+				log.triggerFlushing()
+				bufferVolume = 0
+				continue
+			}
+		}
+	}()
+
+	go func() {
+		log.logger.Info().Msg("Start flush task...")
+
+		defer log.flushWaitGroup.Done()
+
+		for batch := range log.flushChannel {
+			startTime := time.Now()
+
+			var err error
+			for i := 0; i < maxRetries; i++ {
+				if err = log.flushBuffer(batch); err != nil {
+					log.logger.Err(err).Msg("Flushing buffer failed. Retrying...")
+					time.Sleep(100 * time.Millisecond)
+					continue
+				}
+				break
+			}
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Flushed buffer to WAL file. elements: " +
+					strconv.Itoa(batch.count) + ", cost: " + time.Since(startTime).String())
+			}
+
+			batch.notifyRequests(err)
+		}
+		log.logger.Info().Msg("Stop flush task when flush-channel closed!")
+	}()
+
+	log.logger.Info().Msg("Started WAL")
+}
+
+func (log *log) triggerFlushing() {
+	for {
+		select {
+		case log.flushChannel <- log.buffer:
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Send buffer to flush-channel. elements: " + strconv.Itoa(log.buffer.count))
+			}
+			log.newBuffer()
+			return
+		default:
+		}
+		time.Sleep(10 * time.Millisecond)
+	}
+}
+
+func (log *log) newBuffer() {
+	log.buffer.timestampMap = make(map[common.SeriesIDV2][]time.Time)
+	log.buffer.valueMap = make(map[common.SeriesIDV2][]byte)
+	log.buffer.callbackMap = make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error))
+	log.buffer.count = 0
+}
+
+func (log *log) flushBuffer(buffer buffer) error {
+	if buffer.count == 0 {
+		return nil
+	}
+
+	defer func() {
+		log.bytesBuffer.Reset()
+		log.timestampsBuffer.Reset()
+	}()
+
+	// placeholder, preset batch length value is 0
+	batchLen := 0
+	if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(batchLen)); err != nil {
+		return errors.Wrap(err, "Write batch length error")
+	}
+	for seriesID, timestamps := range buffer.timestampMap {
+		// Generate seriesID binary
+		seriesIDBytes := seriesID.Marshal()
+		seriesIDBytesLen := len(seriesIDBytes)
+
+		// Generate timestamps compression binary
+		log.timestampsBuffer.Reset()
+		timestampWriter := encoding.NewWriter()
+		timestampEncoder := encoding.NewXOREncoder(timestampWriter)
+		timestampWriter.Reset(log.timestampsBuffer)
+		for _, timestamp := range timestamps {
+			timestampEncoder.Write(timeTouUnixNano(timestamp))
+		}
+		timestampWriter.Flush()
+		timestampsBytes := log.timestampsBuffer.Bytes()
+		timestampsBytesLen := len(timestampsBytes)
+
+		// Generate values compression binary
+		valuesBytes := snappy.Encode(nil, buffer.valueMap[seriesID])
+
+		// Write entry data
+		entryLen := seriesIDLength + seriesIDBytesLen + seriesCountLength + timestampsBinaryLength + timestampsBytesLen + len(valuesBytes)
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(entryLen)); err != nil {
+			return errors.Wrap(err, "Write entry length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int16(seriesIDBytesLen)); err != nil {
+			return errors.Wrap(err, "Write seriesID length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, seriesIDBytes); err != nil {
+			return errors.Wrap(err, "Write seriesID error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int32(len(timestamps))); err != nil {
+			return errors.Wrap(err, "Write series count error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int16(timestampsBytesLen)); err != nil {
+			return errors.Wrap(err, "Write timestamps length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, timestampsBytes); err != nil {
+			return errors.Wrap(err, "Write timestamps error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, valuesBytes); err != nil {
+			return errors.Wrap(err, "Write values error")
+		}
+	}
+	// Rewrite batch length
+	batchBytes := log.bytesBuffer.Bytes()
+	batchLen = len(batchBytes) - batchLength
+	rewriteInt64InBuf(batchBytes, int64(batchLen), 0, binary.LittleEndian)
+
+	return log.writeWorkSegment(batchBytes)
+}
+
+func (log *log) swapWorkSegment() (Segment, error) {
+	log.workSegmentMutex.Lock()
+	defer log.workSegmentMutex.Unlock()
+
+	if err := log.workSegment.file.Close(); err != nil {
+		return nil, errors.Wrap(err, "Close WAL segment error")
+	}
+
+	oldSegment := log.workSegment
+	// Create new segment.
+	segmentID := log.workSegment.segmentID + 1
+	segment := &segment{
+		segmentID: segmentID,
+		path:      filepath.Join(log.path, segmentName(uint64(segmentID))),
+	}
+	var err error
+	segment.file, err = os.OpenFile(segment.path, os.O_CREATE|os.O_RDWR|os.O_TRUNC, os.ModePerm)
+	if err != nil {
+		return nil, errors.Wrap(err, "Open WAL segment error")
+	}
+	log.workSegment = segment
+
+	return oldSegment, nil
+}
+
+func (log *log) writeWorkSegment(data []byte) error {
+	log.workSegmentMutex.Lock()
+	defer log.workSegmentMutex.Unlock()
+
+	// Write batch data to WAL segment file
+	if _, err := log.workSegment.file.Write(data); err != nil {
+		return errors.Wrap(err, "Write WAL segment file error, file: "+log.workSegment.path)
+	}
+	if err := log.workSegment.file.Sync(); err != nil {
+		log.logger.Warn().Msg("Sync WAL segment file to disk failed, file: " + log.workSegment.path)
+	}
+	return nil
+}
+
+func (log *log) load() error {
+	files, err := os.ReadDir(log.path)
+	if err != nil {
+		return errors.Wrap(err, "Can not read dir: "+log.path)
+	}
+	// Load all of WAL segments.
+	var workSegmentID SegmentID
+	log.segmentMap = make(map[SegmentID]*segment)
+	for _, file := range files {
+		name := file.Name()
+		segmentID, parsePathErr := strconv.ParseUint(name[3:19], 10, 64)

Review Comment:
   It's better to create `parseSegmentID` next to `segmentName`. Then add some method comments to explain the format of the segment name.



##########
pkg/wal/wal.go:
##########
@@ -42,19 +75,759 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// log implements the WAL interface.
+type log struct {
+	entryCloser      *run.Closer
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentMap       map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+	writeWaitGroup   sync.WaitGroup
+	flushWaitGroup   sync.WaitGroup
+	workSegmentMutex sync.Mutex
+	segmentMapMutex  sync.RWMutex
+	closerOnce       sync.Once
+}
+
+type segment struct {
+	file       *os.File
+	path       string
+	logEntries []LogEntry
+	segmentID  SegmentID
+}
+
+type logRequest struct {
+	seriesID  common.SeriesIDV2
+	timestamp time.Time
+	callback  func(common.SeriesIDV2, time.Time, []byte, error)
+	data      []byte
+}
+
+type logEntry struct {
+	timestamps  []time.Time
+	values      *list.List
+	seriesID    common.SeriesIDV2
+	entryLength int64
+	count       int32
+}
+
+type buffer struct {
+	timestampMap map[common.SeriesIDV2][]time.Time
+	valueMap     map[common.SeriesIDV2][]byte
+	callbackMap  map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)
+	count        int
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	walOptions := DefaultOptions
+	if options != nil {
+		fileSize := options.FileSize
+		if fileSize <= 0 {
+			fileSize = DefaultOptions.FileSize
+		}
+		bufferSize := options.BufferSize
+		if bufferSize <= 0 {
+			bufferSize = DefaultOptions.BufferSize
+		}
+		bufferBatchInterval := options.BufferBatchInterval
+		if bufferBatchInterval <= 0 {
+			bufferBatchInterval = DefaultOptions.BufferBatchInterval
+		}
+		walOptions = &Options{
+			FileSize:            fileSize,
+			BufferSize:          bufferSize,
+			BufferBatchInterval: bufferBatchInterval,
+		}
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path: "+path)
+	}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	log := &log{
+		path:             path,
+		options:          *walOptions,
+		logger:           logger.GetLogger(moduleName),
+		writeChannel:     make(chan logRequest),
+		flushChannel:     make(chan buffer, walOptions.FlushQueueSize),
+		bytesBuffer:      bytes.NewBuffer([]byte{}),
+		timestampsBuffer: bytes.NewBuffer([]byte{}),
+		entryCloser:      run.NewCloser(1),
+		buffer: buffer{
+			timestampMap: make(map[common.SeriesIDV2][]time.Time),
+			valueMap:     make(map[common.SeriesIDV2][]byte),
+			callbackMap:  make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)),
+			count:        0,
+		},
+	}
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+
+	log.writeWaitGroup.Add(1)
+	log.flushWaitGroup.Add(1)
+	log.start()
+
+	log.logger.Info().Str("path", path).Msg("WAL has be initialized")
+	return log, nil
+}
+
+// Write a logging entity.
+// It will return immediately when the data is written in the buffer,
+// The callback function will be called when the entity is flushed on the persistent storage.
+func (log *log) Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error)) {
+	if !log.entryCloser.AddRunning() {
+		return
+	}
+	defer log.entryCloser.Done()
+
+	log.writeChannel <- logRequest{
+		seriesID:  seriesID,
+		timestamp: timestamp,
+		data:      data,
+		callback:  callback,
+	}
+}
+
+// Read specified segment by SegmentID.
+func (log *log) Read(segmentID SegmentID) (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segment := log.segmentMap[segmentID]
+	return segment, nil
+}
+
+// ReadAllSegments reads all segments sorted by their creation time in ascending order.
+func (log *log) ReadAllSegments() ([]Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segments := make([]Segment, 0)
+	for _, segment := range log.segmentMap {
+		segments = append(segments, segment)
+	}
+	return segments, nil
+}
+
+// Rotate closes the open segment and opens a new one, returning the closed segment details.
+func (log *log) Rotate() (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	oldWorkSegment, err := log.swapWorkSegment()
+	if err != nil {
+		return nil, err
+	}
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Update segment information.
+	workSegment := log.workSegment
+	log.segmentMap[workSegment.segmentID] = workSegment
+	return oldWorkSegment, nil
+}
+
+// Delete the specified segment.
+func (log *log) Delete(segmentID SegmentID) error {
+	if !log.entryCloser.AddRunning() {
+		return errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Segment which will be deleted must be closed.
+	if segmentID == log.workSegment.segmentID {
+		return errors.New("Can not delete the segment which is working")
+	}
+
+	err := os.Remove(log.segmentMap[segmentID].path)
+	if err != nil {
+		return errors.Wrap(err, "Delete WAL segment error")
+	}
+	delete(log.segmentMap, segmentID)
+	return nil
+}
+
+// Close all of segments and stop WAL work.
+func (log *log) Close() error {
+	log.closerOnce.Do(func() {
+		log.logger.Info().Msg("Closing WAL...")
+
+		log.entryCloser.Done()
+		log.entryCloser.CloseThenWait()
+
+		close(log.writeChannel)
+		log.writeWaitGroup.Wait()
+
+		close(log.flushChannel)
+		log.flushWaitGroup.Wait()
+
+		if err := log.flushBuffer(log.buffer); err != nil {
+			log.logger.Err(err).Msg("Flushing buffer failed")
+		}
+		if err := log.workSegment.file.Close(); err != nil {
+			log.logger.Err(err).Msg("Close work segment file error")
+		}
+		log.logger.Info().Msg("Closed WAL")
+	})
+	return nil
+}
+
+func (log *log) start() {
+	go func() {
+		log.logger.Info().Msg("Start batch task...")
+
+		defer log.writeWaitGroup.Done()
+
+		bufferVolume := 0
+		for {
+			timer := time.NewTimer(log.options.BufferBatchInterval)
+			select {
+			case request, chOpen := <-log.writeChannel:
+				if !chOpen {
+					log.logger.Info().Msg("Stop batch task when write-channel closed!")
+					return
+				}
+
+				log.buffer.write(request)
+				if log.logger.Debug().Enabled() {
+					log.logger.Debug().Msg("Write request to buffer. elements: " + strconv.Itoa(log.buffer.count))
+				}
+
+				bufferVolume += len(request.seriesID.Marshal()) + timestampVolumeLength + len(request.data)
+				if bufferVolume > log.options.BufferSize {
+					log.triggerFlushing()
+					bufferVolume = 0
+				}
+				continue
+			case <-timer.C:
+				if bufferVolume == 0 {
+					continue
+				}
+				log.triggerFlushing()
+				bufferVolume = 0
+				continue
+			}
+		}
+	}()
+
+	go func() {
+		log.logger.Info().Msg("Start flush task...")
+
+		defer log.flushWaitGroup.Done()
+
+		for batch := range log.flushChannel {
+			startTime := time.Now()
+
+			var err error
+			for i := 0; i < maxRetries; i++ {
+				if err = log.flushBuffer(batch); err != nil {
+					log.logger.Err(err).Msg("Flushing buffer failed. Retrying...")
+					time.Sleep(100 * time.Millisecond)
+					continue
+				}
+				break
+			}
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Flushed buffer to WAL file. elements: " +
+					strconv.Itoa(batch.count) + ", cost: " + time.Since(startTime).String())
+			}
+
+			batch.notifyRequests(err)
+		}
+		log.logger.Info().Msg("Stop flush task when flush-channel closed!")
+	}()
+
+	log.logger.Info().Msg("Started WAL")
+}
+
+func (log *log) triggerFlushing() {
+	for {
+		select {
+		case log.flushChannel <- log.buffer:
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Send buffer to flush-channel. elements: " + strconv.Itoa(log.buffer.count))
+			}
+			log.newBuffer()
+			return
+		default:
+		}
+		time.Sleep(10 * time.Millisecond)
+	}
+}
+
+func (log *log) newBuffer() {
+	log.buffer.timestampMap = make(map[common.SeriesIDV2][]time.Time)
+	log.buffer.valueMap = make(map[common.SeriesIDV2][]byte)
+	log.buffer.callbackMap = make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error))
+	log.buffer.count = 0
+}
+
+func (log *log) flushBuffer(buffer buffer) error {
+	if buffer.count == 0 {
+		return nil
+	}
+
+	defer func() {
+		log.bytesBuffer.Reset()
+		log.timestampsBuffer.Reset()
+	}()
+
+	// placeholder, preset batch length value is 0
+	batchLen := 0
+	if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(batchLen)); err != nil {
+		return errors.Wrap(err, "Write batch length error")
+	}
+	for seriesID, timestamps := range buffer.timestampMap {
+		// Generate seriesID binary
+		seriesIDBytes := seriesID.Marshal()
+		seriesIDBytesLen := len(seriesIDBytes)
+
+		// Generate timestamps compression binary
+		log.timestampsBuffer.Reset()
+		timestampWriter := encoding.NewWriter()
+		timestampEncoder := encoding.NewXOREncoder(timestampWriter)
+		timestampWriter.Reset(log.timestampsBuffer)
+		for _, timestamp := range timestamps {
+			timestampEncoder.Write(timeTouUnixNano(timestamp))
+		}
+		timestampWriter.Flush()
+		timestampsBytes := log.timestampsBuffer.Bytes()
+		timestampsBytesLen := len(timestampsBytes)
+
+		// Generate values compression binary
+		valuesBytes := snappy.Encode(nil, buffer.valueMap[seriesID])
+
+		// Write entry data
+		entryLen := seriesIDLength + seriesIDBytesLen + seriesCountLength + timestampsBinaryLength + timestampsBytesLen + len(valuesBytes)
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(entryLen)); err != nil {
+			return errors.Wrap(err, "Write entry length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int16(seriesIDBytesLen)); err != nil {
+			return errors.Wrap(err, "Write seriesID length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, seriesIDBytes); err != nil {
+			return errors.Wrap(err, "Write seriesID error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int32(len(timestamps))); err != nil {
+			return errors.Wrap(err, "Write series count error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int16(timestampsBytesLen)); err != nil {
+			return errors.Wrap(err, "Write timestamps length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, timestampsBytes); err != nil {
+			return errors.Wrap(err, "Write timestamps error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, valuesBytes); err != nil {
+			return errors.Wrap(err, "Write values error")
+		}
+	}
+	// Rewrite batch length
+	batchBytes := log.bytesBuffer.Bytes()
+	batchLen = len(batchBytes) - batchLength
+	rewriteInt64InBuf(batchBytes, int64(batchLen), 0, binary.LittleEndian)
+
+	return log.writeWorkSegment(batchBytes)
+}
+
+func (log *log) swapWorkSegment() (Segment, error) {
+	log.workSegmentMutex.Lock()
+	defer log.workSegmentMutex.Unlock()
+
+	if err := log.workSegment.file.Close(); err != nil {
+		return nil, errors.Wrap(err, "Close WAL segment error")
+	}
+
+	oldSegment := log.workSegment
+	// Create new segment.
+	segmentID := log.workSegment.segmentID + 1
+	segment := &segment{
+		segmentID: segmentID,
+		path:      filepath.Join(log.path, segmentName(uint64(segmentID))),
+	}
+	var err error
+	segment.file, err = os.OpenFile(segment.path, os.O_CREATE|os.O_RDWR|os.O_TRUNC, os.ModePerm)
+	if err != nil {
+		return nil, errors.Wrap(err, "Open WAL segment error")
+	}
+	log.workSegment = segment
+
+	return oldSegment, nil
+}
+
+func (log *log) writeWorkSegment(data []byte) error {
+	log.workSegmentMutex.Lock()
+	defer log.workSegmentMutex.Unlock()
+
+	// Write batch data to WAL segment file
+	if _, err := log.workSegment.file.Write(data); err != nil {
+		return errors.Wrap(err, "Write WAL segment file error, file: "+log.workSegment.path)
+	}
+	if err := log.workSegment.file.Sync(); err != nil {
+		log.logger.Warn().Msg("Sync WAL segment file to disk failed, file: " + log.workSegment.path)
+	}
+	return nil
+}
+
+func (log *log) load() error {
+	files, err := os.ReadDir(log.path)
+	if err != nil {
+		return errors.Wrap(err, "Can not read dir: "+log.path)
+	}
+	// Load all of WAL segments.
+	var workSegmentID SegmentID
+	log.segmentMap = make(map[SegmentID]*segment)
+	for _, file := range files {
+		name := file.Name()
+		segmentID, parsePathErr := strconv.ParseUint(name[3:19], 10, 64)
+		if parsePathErr != nil {
+			return errors.Wrap(parsePathErr, "Parse file name error, name: "+name)
+		}
+		if segmentID > uint64(workSegmentID) {
+			workSegmentID = SegmentID(segmentID)
+		}
+		segment := &segment{
+			segmentID: SegmentID(segmentID),
+			path:      filepath.Join(log.path, segmentName(segmentID)),
+		}
+		if err = segment.parseLogEntries(); err != nil {
+			return errors.Wrap(err, "Fail to parse log entries")
+		}
+		log.segmentMap[SegmentID(segmentID)] = segment
+
+		if log.logger.Debug().Enabled() {
+			log.logger.Debug().Msg("Loaded segment file: " + segment.path)
+		}
+	}
+
+	// If load first time.
+	if len(log.segmentMap) == 0 {
+		segment := &segment{
+			segmentID: 1,
+			path:      filepath.Join(log.path, segmentName(1)),
+		}
+		log.segmentMap[1] = segment
+		log.workSegment = segment
+	} else {
+		log.workSegment = log.segmentMap[workSegmentID]
+	}
+	log.workSegment.file, err = os.OpenFile(log.workSegment.path, os.O_CREATE|os.O_RDWR|os.O_APPEND, os.ModePerm)
+	if err != nil {
+		return errors.Wrap(err, "Open WAL segment error, file: "+log.workSegment.path)
+	}
+	return nil
+}
+
+func (segment *segment) GetSegmentID() SegmentID {
+	return segment.segmentID
+}
+
+func (segment *segment) GetLogEntries() []LogEntry {
+	return segment.logEntries
+}
+
+func (segment *segment) parseLogEntries() error {
+	segmentBytes, err := os.ReadFile(segment.path)
+	if err != nil {
+		return errors.Wrap(err, "Read WAL segment failed, path: "+segment.path)
+	}
+
+	var logEntries []LogEntry
+	var data []byte
+	var batchLen int64
+	var entryLen int64
+	var seriesIDLen int16
+	var seriesID common.SeriesIDV2
+	var seriesCount int32
+	var timestampsBinaryLen int16
+	var entryEndPos int
+	var bytesBuf *bytes.Buffer
+
+	oldPos := 0
+	pos := 0
+	parseNextBatchFlag := true
+
+	for {
+		if parseNextBatchFlag {
+			if len(segmentBytes) <= batchLength {
+				break
+			}
+			data = segmentBytes[pos : pos+batchLength]
+			bytesBuf = bytes.NewBuffer(data)
+			if err = binary.Read(bytesBuf, binary.LittleEndian, &batchLen); err != nil {
+				return errors.Wrap(err, "Read batch length fail to convert from bytes")
+			}
+			if len(segmentBytes) <= int(batchLen) {
+				break
+			}
+
+			pos += batchLength
+			oldPos = pos
+			parseNextBatchFlag = false
+		}
+
+		// Parse entryLength.
+		data = segmentBytes[pos : pos+entryLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &entryLen); err != nil {
+			return errors.Wrap(err, "Read entry length fail to convert from byte")
+		}
+		pos += entryLength
+
+		// Mark entry end-position
+		entryEndPos = pos + int(entryLen)
+		if len(segmentBytes) < entryEndPos {
+			break
+		}
+
+		// Parse seriesIDLen.
+		data = segmentBytes[pos : pos+seriesIDLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &seriesIDLen); err != nil {
+			return errors.Wrap(err, "Read seriesID length fail to convert from byte")
+		}
+		pos += seriesIDLength
+
+		// Parse seriesID.
+		data = segmentBytes[pos : pos+int(seriesIDLen)]
+		seriesID = common.ParseSeriesIDV2(data)
+		pos += int(seriesIDLen)
+
+		// Parse series count.
+		data = segmentBytes[pos : pos+seriesCountLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &seriesCount); err != nil {
+			return errors.Wrap(err, "Read series count fail to convert from byte")
+		}
+		pos += seriesCountLength
+
+		// Parse timestamps compression binary.
+		data = segmentBytes[pos : pos+timestampsBinaryLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &timestampsBinaryLen); err != nil {
+			return errors.Wrap(err, "Read timestamps compression binary length fail to convert from byte")
+		}
+		pos += timestampsBinaryLength
+		data = segmentBytes[pos : pos+int(timestampsBinaryLen)]
+		timestampReader := encoding.NewReader(bytes.NewReader(data))
+		timestampDecoder := encoding.NewXORDecoder(timestampReader)
+		var timestamps []time.Time
+		for i := 0; i < int(seriesCount); i++ {
+			if !timestampDecoder.Next() {
+				return errors.Wrap(err, "Timestamps length not match series count, index: "+strconv.Itoa(i))
+			}
+			timestamps = append(timestamps, unixNanoToTime(timestampDecoder.Value()))
+		}
+		pos += int(timestampsBinaryLen)
+
+		// Parse values compression binary.
+		data = segmentBytes[pos:entryEndPos]
+		if data, err = snappy.Decode(nil, data); err != nil {
+			return errors.Wrap(err, "Decode values compression binary fail to snappy decode")
+		}
+		values := parseValuesBinary(data)
+		if values.Len() != len(timestamps) {
+			return errors.New("Timestamps length and values length not match: " + strconv.Itoa(len(timestamps)) + " vs " + strconv.Itoa(values.Len()))
+		}
+		pos = entryEndPos
+
+		logEntry := &logEntry{
+			entryLength: entryLen,
+			seriesID:    seriesID,
+			count:       seriesCount,
+			timestamps:  timestamps,
+			values:      values,
+		}
+		logEntries = append(logEntries, logEntry)
+
+		if pos == len(segmentBytes) {
+			break
+		}
+		if pos-oldPos == int(batchLen) {
+			parseNextBatchFlag = true
+		}
+	}
+	segment.logEntries = logEntries
+	return nil
+}
+
+func (logEntry *logEntry) GetSeriesID() common.SeriesIDV2 {
+	return logEntry.seriesID
+}
+
+func (logEntry *logEntry) GetTimestamps() []time.Time {
+	return logEntry.timestamps
+}
+
+func (logEntry *logEntry) GetValues() *list.List {
+	return logEntry.values
+}
+
+func (buffer *buffer) write(request logRequest) {
+	seriesID := request.seriesID
+	buffer.timestampMap[seriesID] = append(buffer.timestampMap[seriesID], request.timestamp)
+
+	// Value item: binary-length(2-bytes) + binary data(n-bytes)
+	binaryLength := int16ToBytes(int16(len(request.data)))
+	buffer.valueMap[seriesID] = append(buffer.valueMap[seriesID], binaryLength...)
+	buffer.valueMap[seriesID] = append(buffer.valueMap[seriesID], request.data...)
+
+	buffer.callbackMap[seriesID] = append(buffer.callbackMap[seriesID], request.callback)
+	buffer.count++
+}
+
+func (buffer *buffer) notifyRequests(err error) {
+	for seriesID, callbacks := range buffer.callbackMap {
+		timestamps := buffer.timestampMap[seriesID]
+		values := buffer.valueMap[seriesID]
+		valuePos := 0
+		var valueItem []byte
+		for index, callback := range callbacks {
+			valuePos, valueItem = readValuesBinary(values, valuePos, valuesBinaryLength)
+			tryCallback(func() {
+				callback(seriesID, timestamps[index], valueItem, err)
+			})
+		}
+	}
+}
+
+func segmentName(index uint64) string {
+	return fmt.Sprintf("%v%016x%v", segmentNamePrefix, index, segmentNameSuffix)
+}
+
+func tryCallback(callback func()) {
+	defer func() {
+		if err := recover(); err != nil {
+			fmt.Printf("Run callback error is %v\n", err)
+		}
+	}()
+	callback()
+}
+
+func parseValuesBinary(binary []byte) *list.List {
+	values := list.New()
+	position := 0
+	for {
+		nextPosition, value := readValuesBinary(binary, position, valuesBinaryLength)
+		if value == nil {
+			break
+		}
+		values.PushBack(value)
+		position = nextPosition
+	}
+	return values
+}
+
+func readValuesBinary(raw []byte, position int, offsetLen int) (int, []byte) {
+	if position == len(raw) {
+		return position, nil
+	}
+
+	data := raw[position : position+offsetLen]
+	binaryLen := bytesToInt16(data)
+	position += offsetLen
+
+	data = raw[position : position+int(binaryLen)]
+	position += int(binaryLen)
+	return position, data
+}
+
+func rewriteInt64InBuf(buf []byte, value int64, offset int, order binary.ByteOrder) {
+	_ = buf[offset+7] // early bounds check to guarantee safety of writes below
+	if order == binary.LittleEndian {

Review Comment:
   It seems all binary are packed with little endian.



##########
pkg/wal/wal.go:
##########
@@ -42,19 +75,759 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// log implements the WAL interface.
+type log struct {
+	entryCloser      *run.Closer
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentMap       map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+	writeWaitGroup   sync.WaitGroup
+	flushWaitGroup   sync.WaitGroup
+	workSegmentMutex sync.Mutex
+	segmentMapMutex  sync.RWMutex
+	closerOnce       sync.Once
+}
+
+type segment struct {
+	file       *os.File
+	path       string
+	logEntries []LogEntry
+	segmentID  SegmentID
+}
+
+type logRequest struct {
+	seriesID  common.SeriesIDV2
+	timestamp time.Time
+	callback  func(common.SeriesIDV2, time.Time, []byte, error)
+	data      []byte
+}
+
+type logEntry struct {
+	timestamps  []time.Time
+	values      *list.List
+	seriesID    common.SeriesIDV2
+	entryLength int64
+	count       int32
+}
+
+type buffer struct {
+	timestampMap map[common.SeriesIDV2][]time.Time
+	valueMap     map[common.SeriesIDV2][]byte
+	callbackMap  map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)
+	count        int
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	walOptions := DefaultOptions
+	if options != nil {
+		fileSize := options.FileSize
+		if fileSize <= 0 {
+			fileSize = DefaultOptions.FileSize
+		}
+		bufferSize := options.BufferSize
+		if bufferSize <= 0 {
+			bufferSize = DefaultOptions.BufferSize
+		}
+		bufferBatchInterval := options.BufferBatchInterval
+		if bufferBatchInterval <= 0 {
+			bufferBatchInterval = DefaultOptions.BufferBatchInterval
+		}
+		walOptions = &Options{
+			FileSize:            fileSize,
+			BufferSize:          bufferSize,
+			BufferBatchInterval: bufferBatchInterval,
+		}
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path: "+path)
+	}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	log := &log{
+		path:             path,
+		options:          *walOptions,
+		logger:           logger.GetLogger(moduleName),
+		writeChannel:     make(chan logRequest),
+		flushChannel:     make(chan buffer, walOptions.FlushQueueSize),
+		bytesBuffer:      bytes.NewBuffer([]byte{}),
+		timestampsBuffer: bytes.NewBuffer([]byte{}),
+		entryCloser:      run.NewCloser(1),
+		buffer: buffer{
+			timestampMap: make(map[common.SeriesIDV2][]time.Time),
+			valueMap:     make(map[common.SeriesIDV2][]byte),
+			callbackMap:  make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)),
+			count:        0,
+		},
+	}
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+
+	log.writeWaitGroup.Add(1)
+	log.flushWaitGroup.Add(1)
+	log.start()
+
+	log.logger.Info().Str("path", path).Msg("WAL has be initialized")
+	return log, nil
+}
+
+// Write a logging entity.
+// It will return immediately when the data is written in the buffer,
+// The callback function will be called when the entity is flushed on the persistent storage.
+func (log *log) Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error)) {
+	if !log.entryCloser.AddRunning() {
+		return
+	}
+	defer log.entryCloser.Done()
+
+	log.writeChannel <- logRequest{
+		seriesID:  seriesID,
+		timestamp: timestamp,
+		data:      data,
+		callback:  callback,
+	}
+}
+
+// Read specified segment by SegmentID.
+func (log *log) Read(segmentID SegmentID) (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segment := log.segmentMap[segmentID]
+	return segment, nil
+}
+
+// ReadAllSegments reads all segments sorted by their creation time in ascending order.
+func (log *log) ReadAllSegments() ([]Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segments := make([]Segment, 0)
+	for _, segment := range log.segmentMap {
+		segments = append(segments, segment)
+	}
+	return segments, nil
+}
+
+// Rotate closes the open segment and opens a new one, returning the closed segment details.
+func (log *log) Rotate() (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	oldWorkSegment, err := log.swapWorkSegment()
+	if err != nil {
+		return nil, err
+	}
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Update segment information.
+	workSegment := log.workSegment
+	log.segmentMap[workSegment.segmentID] = workSegment
+	return oldWorkSegment, nil
+}
+
+// Delete the specified segment.
+func (log *log) Delete(segmentID SegmentID) error {
+	if !log.entryCloser.AddRunning() {
+		return errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Segment which will be deleted must be closed.
+	if segmentID == log.workSegment.segmentID {
+		return errors.New("Can not delete the segment which is working")
+	}
+
+	err := os.Remove(log.segmentMap[segmentID].path)
+	if err != nil {
+		return errors.Wrap(err, "Delete WAL segment error")
+	}
+	delete(log.segmentMap, segmentID)
+	return nil
+}
+
+// Close all of segments and stop WAL work.
+func (log *log) Close() error {
+	log.closerOnce.Do(func() {
+		log.logger.Info().Msg("Closing WAL...")
+
+		log.entryCloser.Done()
+		log.entryCloser.CloseThenWait()
+
+		close(log.writeChannel)
+		log.writeWaitGroup.Wait()
+
+		close(log.flushChannel)
+		log.flushWaitGroup.Wait()
+
+		if err := log.flushBuffer(log.buffer); err != nil {
+			log.logger.Err(err).Msg("Flushing buffer failed")
+		}
+		if err := log.workSegment.file.Close(); err != nil {
+			log.logger.Err(err).Msg("Close work segment file error")
+		}
+		log.logger.Info().Msg("Closed WAL")
+	})
+	return nil
+}
+
+func (log *log) start() {
+	go func() {
+		log.logger.Info().Msg("Start batch task...")
+
+		defer log.writeWaitGroup.Done()
+
+		bufferVolume := 0
+		for {
+			timer := time.NewTimer(log.options.BufferBatchInterval)
+			select {
+			case request, chOpen := <-log.writeChannel:
+				if !chOpen {
+					log.logger.Info().Msg("Stop batch task when write-channel closed!")
+					return
+				}
+
+				log.buffer.write(request)
+				if log.logger.Debug().Enabled() {
+					log.logger.Debug().Msg("Write request to buffer. elements: " + strconv.Itoa(log.buffer.count))
+				}
+
+				bufferVolume += len(request.seriesID.Marshal()) + timestampVolumeLength + len(request.data)
+				if bufferVolume > log.options.BufferSize {
+					log.triggerFlushing()
+					bufferVolume = 0
+				}
+				continue
+			case <-timer.C:
+				if bufferVolume == 0 {
+					continue
+				}
+				log.triggerFlushing()
+				bufferVolume = 0
+				continue
+			}
+		}
+	}()
+
+	go func() {
+		log.logger.Info().Msg("Start flush task...")
+
+		defer log.flushWaitGroup.Done()
+
+		for batch := range log.flushChannel {
+			startTime := time.Now()
+
+			var err error
+			for i := 0; i < maxRetries; i++ {
+				if err = log.flushBuffer(batch); err != nil {
+					log.logger.Err(err).Msg("Flushing buffer failed. Retrying...")
+					time.Sleep(100 * time.Millisecond)
+					continue
+				}
+				break
+			}
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Flushed buffer to WAL file. elements: " +
+					strconv.Itoa(batch.count) + ", cost: " + time.Since(startTime).String())
+			}
+
+			batch.notifyRequests(err)
+		}
+		log.logger.Info().Msg("Stop flush task when flush-channel closed!")
+	}()
+
+	log.logger.Info().Msg("Started WAL")
+}
+
+func (log *log) triggerFlushing() {
+	for {
+		select {
+		case log.flushChannel <- log.buffer:
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Send buffer to flush-channel. elements: " + strconv.Itoa(log.buffer.count))
+			}
+			log.newBuffer()
+			return
+		default:
+		}
+		time.Sleep(10 * time.Millisecond)
+	}
+}
+
+func (log *log) newBuffer() {
+	log.buffer.timestampMap = make(map[common.SeriesIDV2][]time.Time)
+	log.buffer.valueMap = make(map[common.SeriesIDV2][]byte)
+	log.buffer.callbackMap = make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error))
+	log.buffer.count = 0
+}
+
+func (log *log) flushBuffer(buffer buffer) error {
+	if buffer.count == 0 {
+		return nil
+	}
+
+	defer func() {
+		log.bytesBuffer.Reset()
+		log.timestampsBuffer.Reset()
+	}()
+
+	// placeholder, preset batch length value is 0
+	batchLen := 0
+	if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(batchLen)); err != nil {
+		return errors.Wrap(err, "Write batch length error")
+	}
+	for seriesID, timestamps := range buffer.timestampMap {
+		// Generate seriesID binary
+		seriesIDBytes := seriesID.Marshal()
+		seriesIDBytesLen := len(seriesIDBytes)
+
+		// Generate timestamps compression binary
+		log.timestampsBuffer.Reset()
+		timestampWriter := encoding.NewWriter()
+		timestampEncoder := encoding.NewXOREncoder(timestampWriter)
+		timestampWriter.Reset(log.timestampsBuffer)
+		for _, timestamp := range timestamps {
+			timestampEncoder.Write(timeTouUnixNano(timestamp))
+		}
+		timestampWriter.Flush()
+		timestampsBytes := log.timestampsBuffer.Bytes()
+		timestampsBytesLen := len(timestampsBytes)
+
+		// Generate values compression binary
+		valuesBytes := snappy.Encode(nil, buffer.valueMap[seriesID])
+
+		// Write entry data
+		entryLen := seriesIDLength + seriesIDBytesLen + seriesCountLength + timestampsBinaryLength + timestampsBytesLen + len(valuesBytes)
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(entryLen)); err != nil {
+			return errors.Wrap(err, "Write entry length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int16(seriesIDBytesLen)); err != nil {
+			return errors.Wrap(err, "Write seriesID length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, seriesIDBytes); err != nil {
+			return errors.Wrap(err, "Write seriesID error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int32(len(timestamps))); err != nil {
+			return errors.Wrap(err, "Write series count error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int16(timestampsBytesLen)); err != nil {
+			return errors.Wrap(err, "Write timestamps length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, timestampsBytes); err != nil {
+			return errors.Wrap(err, "Write timestamps error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, valuesBytes); err != nil {
+			return errors.Wrap(err, "Write values error")
+		}
+	}
+	// Rewrite batch length
+	batchBytes := log.bytesBuffer.Bytes()
+	batchLen = len(batchBytes) - batchLength
+	rewriteInt64InBuf(batchBytes, int64(batchLen), 0, binary.LittleEndian)
+
+	return log.writeWorkSegment(batchBytes)
+}
+
+func (log *log) swapWorkSegment() (Segment, error) {
+	log.workSegmentMutex.Lock()
+	defer log.workSegmentMutex.Unlock()
+
+	if err := log.workSegment.file.Close(); err != nil {
+		return nil, errors.Wrap(err, "Close WAL segment error")
+	}
+
+	oldSegment := log.workSegment
+	// Create new segment.
+	segmentID := log.workSegment.segmentID + 1
+	segment := &segment{
+		segmentID: segmentID,
+		path:      filepath.Join(log.path, segmentName(uint64(segmentID))),
+	}
+	var err error
+	segment.file, err = os.OpenFile(segment.path, os.O_CREATE|os.O_RDWR|os.O_TRUNC, os.ModePerm)
+	if err != nil {
+		return nil, errors.Wrap(err, "Open WAL segment error")
+	}
+	log.workSegment = segment
+
+	return oldSegment, nil
+}
+
+func (log *log) writeWorkSegment(data []byte) error {
+	log.workSegmentMutex.Lock()
+	defer log.workSegmentMutex.Unlock()
+
+	// Write batch data to WAL segment file
+	if _, err := log.workSegment.file.Write(data); err != nil {
+		return errors.Wrap(err, "Write WAL segment file error, file: "+log.workSegment.path)
+	}
+	if err := log.workSegment.file.Sync(); err != nil {
+		log.logger.Warn().Msg("Sync WAL segment file to disk failed, file: " + log.workSegment.path)
+	}
+	return nil
+}
+
+func (log *log) load() error {
+	files, err := os.ReadDir(log.path)
+	if err != nil {
+		return errors.Wrap(err, "Can not read dir: "+log.path)
+	}
+	// Load all of WAL segments.
+	var workSegmentID SegmentID
+	log.segmentMap = make(map[SegmentID]*segment)
+	for _, file := range files {
+		name := file.Name()
+		segmentID, parsePathErr := strconv.ParseUint(name[3:19], 10, 64)
+		if parsePathErr != nil {
+			return errors.Wrap(parsePathErr, "Parse file name error, name: "+name)
+		}
+		if segmentID > uint64(workSegmentID) {
+			workSegmentID = SegmentID(segmentID)
+		}
+		segment := &segment{
+			segmentID: SegmentID(segmentID),
+			path:      filepath.Join(log.path, segmentName(segmentID)),
+		}
+		if err = segment.parseLogEntries(); err != nil {
+			return errors.Wrap(err, "Fail to parse log entries")
+		}
+		log.segmentMap[SegmentID(segmentID)] = segment
+
+		if log.logger.Debug().Enabled() {
+			log.logger.Debug().Msg("Loaded segment file: " + segment.path)
+		}
+	}
+
+	// If load first time.
+	if len(log.segmentMap) == 0 {
+		segment := &segment{
+			segmentID: 1,
+			path:      filepath.Join(log.path, segmentName(1)),
+		}
+		log.segmentMap[1] = segment
+		log.workSegment = segment
+	} else {
+		log.workSegment = log.segmentMap[workSegmentID]
+	}
+	log.workSegment.file, err = os.OpenFile(log.workSegment.path, os.O_CREATE|os.O_RDWR|os.O_APPEND, os.ModePerm)
+	if err != nil {
+		return errors.Wrap(err, "Open WAL segment error, file: "+log.workSegment.path)
+	}
+	return nil
+}
+
+func (segment *segment) GetSegmentID() SegmentID {
+	return segment.segmentID
+}
+
+func (segment *segment) GetLogEntries() []LogEntry {
+	return segment.logEntries
+}
+
+func (segment *segment) parseLogEntries() error {
+	segmentBytes, err := os.ReadFile(segment.path)
+	if err != nil {
+		return errors.Wrap(err, "Read WAL segment failed, path: "+segment.path)
+	}
+
+	var logEntries []LogEntry
+	var data []byte
+	var batchLen int64
+	var entryLen int64
+	var seriesIDLen int16
+	var seriesID common.SeriesIDV2
+	var seriesCount int32
+	var timestampsBinaryLen int16
+	var entryEndPos int
+	var bytesBuf *bytes.Buffer
+
+	oldPos := 0
+	pos := 0
+	parseNextBatchFlag := true
+
+	for {
+		if parseNextBatchFlag {
+			if len(segmentBytes) <= batchLength {
+				break
+			}
+			data = segmentBytes[pos : pos+batchLength]
+			bytesBuf = bytes.NewBuffer(data)
+			if err = binary.Read(bytesBuf, binary.LittleEndian, &batchLen); err != nil {
+				return errors.Wrap(err, "Read batch length fail to convert from bytes")
+			}
+			if len(segmentBytes) <= int(batchLen) {
+				break
+			}
+
+			pos += batchLength
+			oldPos = pos
+			parseNextBatchFlag = false
+		}
+
+		// Parse entryLength.
+		data = segmentBytes[pos : pos+entryLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &entryLen); err != nil {
+			return errors.Wrap(err, "Read entry length fail to convert from byte")
+		}
+		pos += entryLength
+
+		// Mark entry end-position
+		entryEndPos = pos + int(entryLen)
+		if len(segmentBytes) < entryEndPos {
+			break
+		}
+
+		// Parse seriesIDLen.
+		data = segmentBytes[pos : pos+seriesIDLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &seriesIDLen); err != nil {
+			return errors.Wrap(err, "Read seriesID length fail to convert from byte")
+		}
+		pos += seriesIDLength
+
+		// Parse seriesID.
+		data = segmentBytes[pos : pos+int(seriesIDLen)]
+		seriesID = common.ParseSeriesIDV2(data)
+		pos += int(seriesIDLen)
+
+		// Parse series count.
+		data = segmentBytes[pos : pos+seriesCountLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &seriesCount); err != nil {
+			return errors.Wrap(err, "Read series count fail to convert from byte")
+		}
+		pos += seriesCountLength
+
+		// Parse timestamps compression binary.
+		data = segmentBytes[pos : pos+timestampsBinaryLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &timestampsBinaryLen); err != nil {
+			return errors.Wrap(err, "Read timestamps compression binary length fail to convert from byte")
+		}
+		pos += timestampsBinaryLength
+		data = segmentBytes[pos : pos+int(timestampsBinaryLen)]
+		timestampReader := encoding.NewReader(bytes.NewReader(data))
+		timestampDecoder := encoding.NewXORDecoder(timestampReader)
+		var timestamps []time.Time
+		for i := 0; i < int(seriesCount); i++ {
+			if !timestampDecoder.Next() {
+				return errors.Wrap(err, "Timestamps length not match series count, index: "+strconv.Itoa(i))
+			}
+			timestamps = append(timestamps, unixNanoToTime(timestampDecoder.Value()))
+		}
+		pos += int(timestampsBinaryLen)
+
+		// Parse values compression binary.
+		data = segmentBytes[pos:entryEndPos]
+		if data, err = snappy.Decode(nil, data); err != nil {
+			return errors.Wrap(err, "Decode values compression binary fail to snappy decode")
+		}
+		values := parseValuesBinary(data)
+		if values.Len() != len(timestamps) {
+			return errors.New("Timestamps length and values length not match: " + strconv.Itoa(len(timestamps)) + " vs " + strconv.Itoa(values.Len()))
+		}
+		pos = entryEndPos
+
+		logEntry := &logEntry{
+			entryLength: entryLen,
+			seriesID:    seriesID,
+			count:       seriesCount,
+			timestamps:  timestamps,
+			values:      values,
+		}
+		logEntries = append(logEntries, logEntry)
+
+		if pos == len(segmentBytes) {
+			break
+		}
+		if pos-oldPos == int(batchLen) {
+			parseNextBatchFlag = true
+		}
+	}
+	segment.logEntries = logEntries
+	return nil
+}
+
+func (logEntry *logEntry) GetSeriesID() common.SeriesIDV2 {
+	return logEntry.seriesID
+}
+
+func (logEntry *logEntry) GetTimestamps() []time.Time {
+	return logEntry.timestamps
+}
+
+func (logEntry *logEntry) GetValues() *list.List {
+	return logEntry.values
+}
+
+func (buffer *buffer) write(request logRequest) {
+	seriesID := request.seriesID
+	buffer.timestampMap[seriesID] = append(buffer.timestampMap[seriesID], request.timestamp)
+
+	// Value item: binary-length(2-bytes) + binary data(n-bytes)
+	binaryLength := int16ToBytes(int16(len(request.data)))
+	buffer.valueMap[seriesID] = append(buffer.valueMap[seriesID], binaryLength...)
+	buffer.valueMap[seriesID] = append(buffer.valueMap[seriesID], request.data...)
+
+	buffer.callbackMap[seriesID] = append(buffer.callbackMap[seriesID], request.callback)
+	buffer.count++
+}
+
+func (buffer *buffer) notifyRequests(err error) {
+	for seriesID, callbacks := range buffer.callbackMap {
+		timestamps := buffer.timestampMap[seriesID]
+		values := buffer.valueMap[seriesID]
+		valuePos := 0
+		var valueItem []byte
+		for index, callback := range callbacks {
+			valuePos, valueItem = readValuesBinary(values, valuePos, valuesBinaryLength)
+			tryCallback(func() {
+				callback(seriesID, timestamps[index], valueItem, err)
+			})
+		}
+	}
+}
+
+func segmentName(index uint64) string {
+	return fmt.Sprintf("%v%016x%v", segmentNamePrefix, index, segmentNameSuffix)
+}
+
+func tryCallback(callback func()) {
+	defer func() {
+		if err := recover(); err != nil {
+			fmt.Printf("Run callback error is %v\n", err)
+		}
+	}()
+	callback()
+}
+
+func parseValuesBinary(binary []byte) *list.List {
+	values := list.New()
+	position := 0
+	for {
+		nextPosition, value := readValuesBinary(binary, position, valuesBinaryLength)
+		if value == nil {
+			break
+		}
+		values.PushBack(value)
+		position = nextPosition
+	}
+	return values
+}
+
+func readValuesBinary(raw []byte, position int, offsetLen int) (int, []byte) {
+	if position == len(raw) {
+		return position, nil
+	}
+
+	data := raw[position : position+offsetLen]
+	binaryLen := bytesToInt16(data)
+	position += offsetLen
+
+	data = raw[position : position+int(binaryLen)]
+	position += int(binaryLen)
+	return position, data
+}
+
+func rewriteInt64InBuf(buf []byte, value int64, offset int, order binary.ByteOrder) {
+	_ = buf[offset+7] // early bounds check to guarantee safety of writes below
+	if order == binary.LittleEndian {
+		buf[offset+0] = byte(value)
+		buf[offset+1] = byte(value >> 8)
+		buf[offset+2] = byte(value >> 16)
+		buf[offset+3] = byte(value >> 24)
+		buf[offset+4] = byte(value >> 32)
+		buf[offset+5] = byte(value >> 40)
+		buf[offset+6] = byte(value >> 48)
+		buf[offset+7] = byte(value >> 56)
+	} else {
+		buf[offset+0] = byte(value >> 56)
+		buf[offset+1] = byte(value >> 48)
+		buf[offset+2] = byte(value >> 40)
+		buf[offset+3] = byte(value >> 32)
+		buf[offset+4] = byte(value >> 24)
+		buf[offset+5] = byte(value >> 16)
+		buf[offset+6] = byte(value >> 8)
+		buf[offset+7] = byte(value)
+	}
+}
+
+func int16ToBytes(i int16) []byte {
+	buf := make([]byte, 2)
+	binary.LittleEndian.PutUint16(buf, uint16(i))
+	return buf
+}
+
+func bytesToInt16(buf []byte) int16 {
+	return int16(binary.LittleEndian.Uint16(buf))
+}
+
+func timeTouUnixNano(time time.Time) uint64 {

Review Comment:
   ```suggestion
   func timeToUnixNano(time time.Time) uint64 {
   ```



##########
pkg/wal/wal.go:
##########
@@ -42,19 +75,759 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// log implements the WAL interface.
+type log struct {
+	entryCloser      *run.Closer
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentMap       map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+	writeWaitGroup   sync.WaitGroup
+	flushWaitGroup   sync.WaitGroup
+	workSegmentMutex sync.Mutex
+	segmentMapMutex  sync.RWMutex
+	closerOnce       sync.Once
+}
+
+type segment struct {
+	file       *os.File
+	path       string
+	logEntries []LogEntry
+	segmentID  SegmentID
+}
+
+type logRequest struct {
+	seriesID  common.SeriesIDV2
+	timestamp time.Time
+	callback  func(common.SeriesIDV2, time.Time, []byte, error)
+	data      []byte
+}
+
+type logEntry struct {
+	timestamps  []time.Time
+	values      *list.List
+	seriesID    common.SeriesIDV2
+	entryLength int64
+	count       int32
+}
+
+type buffer struct {
+	timestampMap map[common.SeriesIDV2][]time.Time
+	valueMap     map[common.SeriesIDV2][]byte
+	callbackMap  map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)
+	count        int
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	walOptions := DefaultOptions
+	if options != nil {
+		fileSize := options.FileSize
+		if fileSize <= 0 {
+			fileSize = DefaultOptions.FileSize
+		}
+		bufferSize := options.BufferSize
+		if bufferSize <= 0 {
+			bufferSize = DefaultOptions.BufferSize
+		}
+		bufferBatchInterval := options.BufferBatchInterval
+		if bufferBatchInterval <= 0 {
+			bufferBatchInterval = DefaultOptions.BufferBatchInterval
+		}
+		walOptions = &Options{
+			FileSize:            fileSize,
+			BufferSize:          bufferSize,
+			BufferBatchInterval: bufferBatchInterval,
+		}
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path: "+path)
+	}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	log := &log{
+		path:             path,
+		options:          *walOptions,
+		logger:           logger.GetLogger(moduleName),
+		writeChannel:     make(chan logRequest),
+		flushChannel:     make(chan buffer, walOptions.FlushQueueSize),
+		bytesBuffer:      bytes.NewBuffer([]byte{}),
+		timestampsBuffer: bytes.NewBuffer([]byte{}),
+		entryCloser:      run.NewCloser(1),
+		buffer: buffer{
+			timestampMap: make(map[common.SeriesIDV2][]time.Time),
+			valueMap:     make(map[common.SeriesIDV2][]byte),
+			callbackMap:  make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)),
+			count:        0,
+		},
+	}
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+
+	log.writeWaitGroup.Add(1)
+	log.flushWaitGroup.Add(1)
+	log.start()
+
+	log.logger.Info().Str("path", path).Msg("WAL has be initialized")
+	return log, nil
+}
+
+// Write a logging entity.
+// It will return immediately when the data is written in the buffer,
+// The callback function will be called when the entity is flushed on the persistent storage.
+func (log *log) Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error)) {
+	if !log.entryCloser.AddRunning() {
+		return
+	}
+	defer log.entryCloser.Done()
+
+	log.writeChannel <- logRequest{
+		seriesID:  seriesID,
+		timestamp: timestamp,
+		data:      data,
+		callback:  callback,
+	}
+}
+
+// Read specified segment by SegmentID.
+func (log *log) Read(segmentID SegmentID) (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segment := log.segmentMap[segmentID]
+	return segment, nil
+}
+
+// ReadAllSegments reads all segments sorted by their creation time in ascending order.
+func (log *log) ReadAllSegments() ([]Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segments := make([]Segment, 0)
+	for _, segment := range log.segmentMap {
+		segments = append(segments, segment)
+	}
+	return segments, nil
+}
+
+// Rotate closes the open segment and opens a new one, returning the closed segment details.
+func (log *log) Rotate() (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	oldWorkSegment, err := log.swapWorkSegment()
+	if err != nil {
+		return nil, err
+	}
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Update segment information.
+	workSegment := log.workSegment
+	log.segmentMap[workSegment.segmentID] = workSegment
+	return oldWorkSegment, nil
+}
+
+// Delete the specified segment.
+func (log *log) Delete(segmentID SegmentID) error {
+	if !log.entryCloser.AddRunning() {
+		return errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Segment which will be deleted must be closed.
+	if segmentID == log.workSegment.segmentID {
+		return errors.New("Can not delete the segment which is working")
+	}
+
+	err := os.Remove(log.segmentMap[segmentID].path)
+	if err != nil {
+		return errors.Wrap(err, "Delete WAL segment error")
+	}
+	delete(log.segmentMap, segmentID)
+	return nil
+}
+
+// Close all of segments and stop WAL work.
+func (log *log) Close() error {
+	log.closerOnce.Do(func() {
+		log.logger.Info().Msg("Closing WAL...")
+
+		log.entryCloser.Done()
+		log.entryCloser.CloseThenWait()
+
+		close(log.writeChannel)
+		log.writeWaitGroup.Wait()
+
+		close(log.flushChannel)
+		log.flushWaitGroup.Wait()
+
+		if err := log.flushBuffer(log.buffer); err != nil {
+			log.logger.Err(err).Msg("Flushing buffer failed")
+		}
+		if err := log.workSegment.file.Close(); err != nil {
+			log.logger.Err(err).Msg("Close work segment file error")
+		}
+		log.logger.Info().Msg("Closed WAL")
+	})
+	return nil
+}
+
+func (log *log) start() {
+	go func() {
+		log.logger.Info().Msg("Start batch task...")
+
+		defer log.writeWaitGroup.Done()
+
+		bufferVolume := 0
+		for {
+			timer := time.NewTimer(log.options.BufferBatchInterval)
+			select {
+			case request, chOpen := <-log.writeChannel:
+				if !chOpen {
+					log.logger.Info().Msg("Stop batch task when write-channel closed!")
+					return
+				}
+
+				log.buffer.write(request)
+				if log.logger.Debug().Enabled() {
+					log.logger.Debug().Msg("Write request to buffer. elements: " + strconv.Itoa(log.buffer.count))
+				}
+
+				bufferVolume += len(request.seriesID.Marshal()) + timestampVolumeLength + len(request.data)
+				if bufferVolume > log.options.BufferSize {
+					log.triggerFlushing()
+					bufferVolume = 0
+				}
+				continue
+			case <-timer.C:
+				if bufferVolume == 0 {
+					continue
+				}
+				log.triggerFlushing()
+				bufferVolume = 0
+				continue
+			}
+		}
+	}()
+
+	go func() {
+		log.logger.Info().Msg("Start flush task...")
+
+		defer log.flushWaitGroup.Done()
+
+		for batch := range log.flushChannel {
+			startTime := time.Now()
+
+			var err error
+			for i := 0; i < maxRetries; i++ {
+				if err = log.flushBuffer(batch); err != nil {
+					log.logger.Err(err).Msg("Flushing buffer failed. Retrying...")
+					time.Sleep(100 * time.Millisecond)
+					continue
+				}
+				break
+			}
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Flushed buffer to WAL file. elements: " +
+					strconv.Itoa(batch.count) + ", cost: " + time.Since(startTime).String())
+			}
+
+			batch.notifyRequests(err)
+		}
+		log.logger.Info().Msg("Stop flush task when flush-channel closed!")
+	}()
+
+	log.logger.Info().Msg("Started WAL")
+}
+
+func (log *log) triggerFlushing() {
+	for {
+		select {
+		case log.flushChannel <- log.buffer:
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Send buffer to flush-channel. elements: " + strconv.Itoa(log.buffer.count))
+			}
+			log.newBuffer()
+			return
+		default:
+		}
+		time.Sleep(10 * time.Millisecond)
+	}
+}
+
+func (log *log) newBuffer() {
+	log.buffer.timestampMap = make(map[common.SeriesIDV2][]time.Time)
+	log.buffer.valueMap = make(map[common.SeriesIDV2][]byte)
+	log.buffer.callbackMap = make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error))
+	log.buffer.count = 0
+}
+
+func (log *log) flushBuffer(buffer buffer) error {
+	if buffer.count == 0 {
+		return nil
+	}
+
+	defer func() {
+		log.bytesBuffer.Reset()
+		log.timestampsBuffer.Reset()
+	}()
+
+	// placeholder, preset batch length value is 0
+	batchLen := 0
+	if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(batchLen)); err != nil {
+		return errors.Wrap(err, "Write batch length error")
+	}
+	for seriesID, timestamps := range buffer.timestampMap {
+		// Generate seriesID binary
+		seriesIDBytes := seriesID.Marshal()
+		seriesIDBytesLen := len(seriesIDBytes)
+
+		// Generate timestamps compression binary
+		log.timestampsBuffer.Reset()
+		timestampWriter := encoding.NewWriter()
+		timestampEncoder := encoding.NewXOREncoder(timestampWriter)
+		timestampWriter.Reset(log.timestampsBuffer)
+		for _, timestamp := range timestamps {
+			timestampEncoder.Write(timeTouUnixNano(timestamp))
+		}
+		timestampWriter.Flush()
+		timestampsBytes := log.timestampsBuffer.Bytes()
+		timestampsBytesLen := len(timestampsBytes)
+
+		// Generate values compression binary
+		valuesBytes := snappy.Encode(nil, buffer.valueMap[seriesID])
+
+		// Write entry data
+		entryLen := seriesIDLength + seriesIDBytesLen + seriesCountLength + timestampsBinaryLength + timestampsBytesLen + len(valuesBytes)
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(entryLen)); err != nil {
+			return errors.Wrap(err, "Write entry length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int16(seriesIDBytesLen)); err != nil {
+			return errors.Wrap(err, "Write seriesID length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, seriesIDBytes); err != nil {
+			return errors.Wrap(err, "Write seriesID error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int32(len(timestamps))); err != nil {
+			return errors.Wrap(err, "Write series count error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int16(timestampsBytesLen)); err != nil {
+			return errors.Wrap(err, "Write timestamps length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, timestampsBytes); err != nil {
+			return errors.Wrap(err, "Write timestamps error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, valuesBytes); err != nil {
+			return errors.Wrap(err, "Write values error")
+		}
+	}
+	// Rewrite batch length
+	batchBytes := log.bytesBuffer.Bytes()
+	batchLen = len(batchBytes) - batchLength
+	rewriteInt64InBuf(batchBytes, int64(batchLen), 0, binary.LittleEndian)
+
+	return log.writeWorkSegment(batchBytes)
+}
+
+func (log *log) swapWorkSegment() (Segment, error) {
+	log.workSegmentMutex.Lock()
+	defer log.workSegmentMutex.Unlock()
+
+	if err := log.workSegment.file.Close(); err != nil {
+		return nil, errors.Wrap(err, "Close WAL segment error")
+	}
+
+	oldSegment := log.workSegment
+	// Create new segment.
+	segmentID := log.workSegment.segmentID + 1
+	segment := &segment{
+		segmentID: segmentID,
+		path:      filepath.Join(log.path, segmentName(uint64(segmentID))),
+	}
+	var err error
+	segment.file, err = os.OpenFile(segment.path, os.O_CREATE|os.O_RDWR|os.O_TRUNC, os.ModePerm)

Review Comment:
   Could you create a function to create the segment file?



##########
pkg/wal/wal.go:
##########
@@ -42,19 +75,759 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// log implements the WAL interface.
+type log struct {
+	entryCloser      *run.Closer
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentMap       map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+	writeWaitGroup   sync.WaitGroup
+	flushWaitGroup   sync.WaitGroup
+	workSegmentMutex sync.Mutex
+	segmentMapMutex  sync.RWMutex
+	closerOnce       sync.Once
+}
+
+type segment struct {
+	file       *os.File
+	path       string
+	logEntries []LogEntry
+	segmentID  SegmentID
+}
+
+type logRequest struct {
+	seriesID  common.SeriesIDV2
+	timestamp time.Time
+	callback  func(common.SeriesIDV2, time.Time, []byte, error)
+	data      []byte
+}
+
+type logEntry struct {
+	timestamps  []time.Time
+	values      *list.List
+	seriesID    common.SeriesIDV2
+	entryLength int64
+	count       int32
+}
+
+type buffer struct {
+	timestampMap map[common.SeriesIDV2][]time.Time
+	valueMap     map[common.SeriesIDV2][]byte
+	callbackMap  map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)
+	count        int
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	walOptions := DefaultOptions
+	if options != nil {
+		fileSize := options.FileSize
+		if fileSize <= 0 {
+			fileSize = DefaultOptions.FileSize
+		}
+		bufferSize := options.BufferSize
+		if bufferSize <= 0 {
+			bufferSize = DefaultOptions.BufferSize
+		}
+		bufferBatchInterval := options.BufferBatchInterval
+		if bufferBatchInterval <= 0 {
+			bufferBatchInterval = DefaultOptions.BufferBatchInterval
+		}
+		walOptions = &Options{
+			FileSize:            fileSize,
+			BufferSize:          bufferSize,
+			BufferBatchInterval: bufferBatchInterval,
+		}
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path: "+path)
+	}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	log := &log{
+		path:             path,
+		options:          *walOptions,
+		logger:           logger.GetLogger(moduleName),
+		writeChannel:     make(chan logRequest),
+		flushChannel:     make(chan buffer, walOptions.FlushQueueSize),
+		bytesBuffer:      bytes.NewBuffer([]byte{}),
+		timestampsBuffer: bytes.NewBuffer([]byte{}),
+		entryCloser:      run.NewCloser(1),
+		buffer: buffer{
+			timestampMap: make(map[common.SeriesIDV2][]time.Time),
+			valueMap:     make(map[common.SeriesIDV2][]byte),
+			callbackMap:  make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)),
+			count:        0,
+		},
+	}
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+
+	log.writeWaitGroup.Add(1)
+	log.flushWaitGroup.Add(1)
+	log.start()
+
+	log.logger.Info().Str("path", path).Msg("WAL has be initialized")
+	return log, nil
+}
+
+// Write a logging entity.
+// It will return immediately when the data is written in the buffer,
+// The callback function will be called when the entity is flushed on the persistent storage.
+func (log *log) Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error)) {
+	if !log.entryCloser.AddRunning() {
+		return
+	}
+	defer log.entryCloser.Done()
+
+	log.writeChannel <- logRequest{
+		seriesID:  seriesID,
+		timestamp: timestamp,
+		data:      data,
+		callback:  callback,
+	}
+}
+
+// Read specified segment by SegmentID.
+func (log *log) Read(segmentID SegmentID) (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segment := log.segmentMap[segmentID]
+	return segment, nil
+}
+
+// ReadAllSegments reads all segments sorted by their creation time in ascending order.
+func (log *log) ReadAllSegments() ([]Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segments := make([]Segment, 0)
+	for _, segment := range log.segmentMap {
+		segments = append(segments, segment)
+	}
+	return segments, nil
+}
+
+// Rotate closes the open segment and opens a new one, returning the closed segment details.
+func (log *log) Rotate() (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	oldWorkSegment, err := log.swapWorkSegment()
+	if err != nil {
+		return nil, err
+	}
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Update segment information.
+	workSegment := log.workSegment
+	log.segmentMap[workSegment.segmentID] = workSegment
+	return oldWorkSegment, nil
+}
+
+// Delete the specified segment.
+func (log *log) Delete(segmentID SegmentID) error {
+	if !log.entryCloser.AddRunning() {
+		return errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Segment which will be deleted must be closed.
+	if segmentID == log.workSegment.segmentID {
+		return errors.New("Can not delete the segment which is working")
+	}
+
+	err := os.Remove(log.segmentMap[segmentID].path)
+	if err != nil {
+		return errors.Wrap(err, "Delete WAL segment error")
+	}
+	delete(log.segmentMap, segmentID)
+	return nil
+}
+
+// Close all of segments and stop WAL work.
+func (log *log) Close() error {
+	log.closerOnce.Do(func() {
+		log.logger.Info().Msg("Closing WAL...")
+
+		log.entryCloser.Done()
+		log.entryCloser.CloseThenWait()
+
+		close(log.writeChannel)
+		log.writeWaitGroup.Wait()
+
+		close(log.flushChannel)
+		log.flushWaitGroup.Wait()
+
+		if err := log.flushBuffer(log.buffer); err != nil {
+			log.logger.Err(err).Msg("Flushing buffer failed")
+		}
+		if err := log.workSegment.file.Close(); err != nil {
+			log.logger.Err(err).Msg("Close work segment file error")
+		}
+		log.logger.Info().Msg("Closed WAL")
+	})
+	return nil
+}
+
+func (log *log) start() {
+	go func() {
+		log.logger.Info().Msg("Start batch task...")
+
+		defer log.writeWaitGroup.Done()
+
+		bufferVolume := 0
+		for {
+			timer := time.NewTimer(log.options.BufferBatchInterval)
+			select {
+			case request, chOpen := <-log.writeChannel:
+				if !chOpen {
+					log.logger.Info().Msg("Stop batch task when write-channel closed!")
+					return
+				}
+
+				log.buffer.write(request)
+				if log.logger.Debug().Enabled() {
+					log.logger.Debug().Msg("Write request to buffer. elements: " + strconv.Itoa(log.buffer.count))
+				}
+
+				bufferVolume += len(request.seriesID.Marshal()) + timestampVolumeLength + len(request.data)
+				if bufferVolume > log.options.BufferSize {
+					log.triggerFlushing()
+					bufferVolume = 0
+				}
+				continue
+			case <-timer.C:
+				if bufferVolume == 0 {
+					continue
+				}
+				log.triggerFlushing()
+				bufferVolume = 0
+				continue
+			}
+		}
+	}()
+
+	go func() {
+		log.logger.Info().Msg("Start flush task...")
+
+		defer log.flushWaitGroup.Done()
+
+		for batch := range log.flushChannel {
+			startTime := time.Now()
+
+			var err error
+			for i := 0; i < maxRetries; i++ {
+				if err = log.flushBuffer(batch); err != nil {
+					log.logger.Err(err).Msg("Flushing buffer failed. Retrying...")
+					time.Sleep(100 * time.Millisecond)
+					continue
+				}
+				break
+			}
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Flushed buffer to WAL file. elements: " +
+					strconv.Itoa(batch.count) + ", cost: " + time.Since(startTime).String())
+			}
+
+			batch.notifyRequests(err)
+		}
+		log.logger.Info().Msg("Stop flush task when flush-channel closed!")
+	}()
+
+	log.logger.Info().Msg("Started WAL")
+}
+
+func (log *log) triggerFlushing() {
+	for {
+		select {
+		case log.flushChannel <- log.buffer:
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Send buffer to flush-channel. elements: " + strconv.Itoa(log.buffer.count))
+			}
+			log.newBuffer()
+			return
+		default:
+		}
+		time.Sleep(10 * time.Millisecond)
+	}
+}
+
+func (log *log) newBuffer() {
+	log.buffer.timestampMap = make(map[common.SeriesIDV2][]time.Time)
+	log.buffer.valueMap = make(map[common.SeriesIDV2][]byte)
+	log.buffer.callbackMap = make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error))
+	log.buffer.count = 0
+}
+
+func (log *log) flushBuffer(buffer buffer) error {
+	if buffer.count == 0 {
+		return nil
+	}
+
+	defer func() {
+		log.bytesBuffer.Reset()
+		log.timestampsBuffer.Reset()
+	}()
+
+	// placeholder, preset batch length value is 0
+	batchLen := 0
+	if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(batchLen)); err != nil {
+		return errors.Wrap(err, "Write batch length error")
+	}
+	for seriesID, timestamps := range buffer.timestampMap {
+		// Generate seriesID binary
+		seriesIDBytes := seriesID.Marshal()
+		seriesIDBytesLen := len(seriesIDBytes)
+
+		// Generate timestamps compression binary
+		log.timestampsBuffer.Reset()
+		timestampWriter := encoding.NewWriter()
+		timestampEncoder := encoding.NewXOREncoder(timestampWriter)
+		timestampWriter.Reset(log.timestampsBuffer)
+		for _, timestamp := range timestamps {
+			timestampEncoder.Write(timeTouUnixNano(timestamp))
+		}
+		timestampWriter.Flush()
+		timestampsBytes := log.timestampsBuffer.Bytes()
+		timestampsBytesLen := len(timestampsBytes)
+
+		// Generate values compression binary
+		valuesBytes := snappy.Encode(nil, buffer.valueMap[seriesID])
+
+		// Write entry data
+		entryLen := seriesIDLength + seriesIDBytesLen + seriesCountLength + timestampsBinaryLength + timestampsBytesLen + len(valuesBytes)
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(entryLen)); err != nil {
+			return errors.Wrap(err, "Write entry length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int16(seriesIDBytesLen)); err != nil {
+			return errors.Wrap(err, "Write seriesID length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, seriesIDBytes); err != nil {
+			return errors.Wrap(err, "Write seriesID error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int32(len(timestamps))); err != nil {
+			return errors.Wrap(err, "Write series count error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int16(timestampsBytesLen)); err != nil {
+			return errors.Wrap(err, "Write timestamps length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, timestampsBytes); err != nil {
+			return errors.Wrap(err, "Write timestamps error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, valuesBytes); err != nil {
+			return errors.Wrap(err, "Write values error")
+		}
+	}
+	// Rewrite batch length
+	batchBytes := log.bytesBuffer.Bytes()
+	batchLen = len(batchBytes) - batchLength
+	rewriteInt64InBuf(batchBytes, int64(batchLen), 0, binary.LittleEndian)
+
+	return log.writeWorkSegment(batchBytes)
+}
+
+func (log *log) swapWorkSegment() (Segment, error) {
+	log.workSegmentMutex.Lock()
+	defer log.workSegmentMutex.Unlock()
+
+	if err := log.workSegment.file.Close(); err != nil {
+		return nil, errors.Wrap(err, "Close WAL segment error")
+	}
+
+	oldSegment := log.workSegment
+	// Create new segment.
+	segmentID := log.workSegment.segmentID + 1
+	segment := &segment{
+		segmentID: segmentID,
+		path:      filepath.Join(log.path, segmentName(uint64(segmentID))),
+	}
+	var err error
+	segment.file, err = os.OpenFile(segment.path, os.O_CREATE|os.O_RDWR|os.O_TRUNC, os.ModePerm)
+	if err != nil {
+		return nil, errors.Wrap(err, "Open WAL segment error")
+	}
+	log.workSegment = segment
+
+	return oldSegment, nil
+}
+
+func (log *log) writeWorkSegment(data []byte) error {
+	log.workSegmentMutex.Lock()
+	defer log.workSegmentMutex.Unlock()
+
+	// Write batch data to WAL segment file
+	if _, err := log.workSegment.file.Write(data); err != nil {
+		return errors.Wrap(err, "Write WAL segment file error, file: "+log.workSegment.path)
+	}
+	if err := log.workSegment.file.Sync(); err != nil {
+		log.logger.Warn().Msg("Sync WAL segment file to disk failed, file: " + log.workSegment.path)
+	}
+	return nil
+}
+
+func (log *log) load() error {
+	files, err := os.ReadDir(log.path)
+	if err != nil {
+		return errors.Wrap(err, "Can not read dir: "+log.path)
+	}
+	// Load all of WAL segments.
+	var workSegmentID SegmentID
+	log.segmentMap = make(map[SegmentID]*segment)
+	for _, file := range files {
+		name := file.Name()
+		segmentID, parsePathErr := strconv.ParseUint(name[3:19], 10, 64)
+		if parsePathErr != nil {
+			return errors.Wrap(parsePathErr, "Parse file name error, name: "+name)
+		}
+		if segmentID > uint64(workSegmentID) {
+			workSegmentID = SegmentID(segmentID)
+		}
+		segment := &segment{
+			segmentID: SegmentID(segmentID),
+			path:      filepath.Join(log.path, segmentName(segmentID)),
+		}
+		if err = segment.parseLogEntries(); err != nil {
+			return errors.Wrap(err, "Fail to parse log entries")
+		}
+		log.segmentMap[SegmentID(segmentID)] = segment
+
+		if log.logger.Debug().Enabled() {
+			log.logger.Debug().Msg("Loaded segment file: " + segment.path)
+		}
+	}
+
+	// If load first time.
+	if len(log.segmentMap) == 0 {
+		segment := &segment{
+			segmentID: 1,
+			path:      filepath.Join(log.path, segmentName(1)),
+		}
+		log.segmentMap[1] = segment
+		log.workSegment = segment
+	} else {
+		log.workSegment = log.segmentMap[workSegmentID]
+	}
+	log.workSegment.file, err = os.OpenFile(log.workSegment.path, os.O_CREATE|os.O_RDWR|os.O_APPEND, os.ModePerm)
+	if err != nil {
+		return errors.Wrap(err, "Open WAL segment error, file: "+log.workSegment.path)
+	}
+	return nil
+}
+
+func (segment *segment) GetSegmentID() SegmentID {
+	return segment.segmentID
+}
+
+func (segment *segment) GetLogEntries() []LogEntry {
+	return segment.logEntries
+}
+
+func (segment *segment) parseLogEntries() error {
+	segmentBytes, err := os.ReadFile(segment.path)
+	if err != nil {
+		return errors.Wrap(err, "Read WAL segment failed, path: "+segment.path)
+	}
+
+	var logEntries []LogEntry
+	var data []byte
+	var batchLen int64
+	var entryLen int64
+	var seriesIDLen int16
+	var seriesID common.SeriesIDV2
+	var seriesCount int32
+	var timestampsBinaryLen int16
+	var entryEndPos int
+	var bytesBuf *bytes.Buffer
+
+	oldPos := 0
+	pos := 0
+	parseNextBatchFlag := true
+
+	for {
+		if parseNextBatchFlag {
+			if len(segmentBytes) <= batchLength {
+				break
+			}
+			data = segmentBytes[pos : pos+batchLength]
+			bytesBuf = bytes.NewBuffer(data)
+			if err = binary.Read(bytesBuf, binary.LittleEndian, &batchLen); err != nil {
+				return errors.Wrap(err, "Read batch length fail to convert from bytes")
+			}
+			if len(segmentBytes) <= int(batchLen) {
+				break
+			}
+
+			pos += batchLength
+			oldPos = pos
+			parseNextBatchFlag = false
+		}
+
+		// Parse entryLength.
+		data = segmentBytes[pos : pos+entryLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &entryLen); err != nil {
+			return errors.Wrap(err, "Read entry length fail to convert from byte")
+		}
+		pos += entryLength
+
+		// Mark entry end-position
+		entryEndPos = pos + int(entryLen)
+		if len(segmentBytes) < entryEndPos {
+			break
+		}
+
+		// Parse seriesIDLen.
+		data = segmentBytes[pos : pos+seriesIDLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &seriesIDLen); err != nil {
+			return errors.Wrap(err, "Read seriesID length fail to convert from byte")
+		}
+		pos += seriesIDLength
+
+		// Parse seriesID.
+		data = segmentBytes[pos : pos+int(seriesIDLen)]
+		seriesID = common.ParseSeriesIDV2(data)
+		pos += int(seriesIDLen)
+
+		// Parse series count.
+		data = segmentBytes[pos : pos+seriesCountLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &seriesCount); err != nil {
+			return errors.Wrap(err, "Read series count fail to convert from byte")
+		}
+		pos += seriesCountLength
+
+		// Parse timestamps compression binary.
+		data = segmentBytes[pos : pos+timestampsBinaryLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &timestampsBinaryLen); err != nil {
+			return errors.Wrap(err, "Read timestamps compression binary length fail to convert from byte")
+		}
+		pos += timestampsBinaryLength
+		data = segmentBytes[pos : pos+int(timestampsBinaryLen)]
+		timestampReader := encoding.NewReader(bytes.NewReader(data))
+		timestampDecoder := encoding.NewXORDecoder(timestampReader)
+		var timestamps []time.Time
+		for i := 0; i < int(seriesCount); i++ {
+			if !timestampDecoder.Next() {
+				return errors.Wrap(err, "Timestamps length not match series count, index: "+strconv.Itoa(i))
+			}
+			timestamps = append(timestamps, unixNanoToTime(timestampDecoder.Value()))
+		}
+		pos += int(timestampsBinaryLen)
+
+		// Parse values compression binary.
+		data = segmentBytes[pos:entryEndPos]
+		if data, err = snappy.Decode(nil, data); err != nil {
+			return errors.Wrap(err, "Decode values compression binary fail to snappy decode")
+		}
+		values := parseValuesBinary(data)
+		if values.Len() != len(timestamps) {
+			return errors.New("Timestamps length and values length not match: " + strconv.Itoa(len(timestamps)) + " vs " + strconv.Itoa(values.Len()))
+		}
+		pos = entryEndPos
+
+		logEntry := &logEntry{
+			entryLength: entryLen,
+			seriesID:    seriesID,
+			count:       seriesCount,
+			timestamps:  timestamps,
+			values:      values,
+		}
+		logEntries = append(logEntries, logEntry)
+
+		if pos == len(segmentBytes) {
+			break
+		}
+		if pos-oldPos == int(batchLen) {
+			parseNextBatchFlag = true
+		}
+	}
+	segment.logEntries = logEntries
+	return nil
+}
+
+func (logEntry *logEntry) GetSeriesID() common.SeriesIDV2 {
+	return logEntry.seriesID
+}
+
+func (logEntry *logEntry) GetTimestamps() []time.Time {
+	return logEntry.timestamps
+}
+
+func (logEntry *logEntry) GetValues() *list.List {
+	return logEntry.values
+}
+
+func (buffer *buffer) write(request logRequest) {
+	seriesID := request.seriesID
+	buffer.timestampMap[seriesID] = append(buffer.timestampMap[seriesID], request.timestamp)
+
+	// Value item: binary-length(2-bytes) + binary data(n-bytes)
+	binaryLength := int16ToBytes(int16(len(request.data)))
+	buffer.valueMap[seriesID] = append(buffer.valueMap[seriesID], binaryLength...)
+	buffer.valueMap[seriesID] = append(buffer.valueMap[seriesID], request.data...)
+
+	buffer.callbackMap[seriesID] = append(buffer.callbackMap[seriesID], request.callback)
+	buffer.count++
+}
+
+func (buffer *buffer) notifyRequests(err error) {
+	for seriesID, callbacks := range buffer.callbackMap {
+		timestamps := buffer.timestampMap[seriesID]
+		values := buffer.valueMap[seriesID]
+		valuePos := 0
+		var valueItem []byte
+		for index, callback := range callbacks {
+			valuePos, valueItem = readValuesBinary(values, valuePos, valuesBinaryLength)
+			tryCallback(func() {
+				callback(seriesID, timestamps[index], valueItem, err)
+			})
+		}
+	}
+}
+
+func segmentName(index uint64) string {
+	return fmt.Sprintf("%v%016x%v", segmentNamePrefix, index, segmentNameSuffix)
+}
+
+func tryCallback(callback func()) {
+	defer func() {
+		if err := recover(); err != nil {
+			fmt.Printf("Run callback error is %v\n", err)
+		}
+	}()
+	callback()
+}
+
+func parseValuesBinary(binary []byte) *list.List {
+	values := list.New()
+	position := 0
+	for {
+		nextPosition, value := readValuesBinary(binary, position, valuesBinaryLength)
+		if value == nil {
+			break
+		}
+		values.PushBack(value)
+		position = nextPosition
+	}
+	return values
+}
+
+func readValuesBinary(raw []byte, position int, offsetLen int) (int, []byte) {
+	if position == len(raw) {
+		return position, nil
+	}
+
+	data := raw[position : position+offsetLen]
+	binaryLen := bytesToInt16(data)
+	position += offsetLen
+
+	data = raw[position : position+int(binaryLen)]
+	position += int(binaryLen)
+	return position, data
+}
+
+func rewriteInt64InBuf(buf []byte, value int64, offset int, order binary.ByteOrder) {
+	_ = buf[offset+7] // early bounds check to guarantee safety of writes below
+	if order == binary.LittleEndian {
+		buf[offset+0] = byte(value)
+		buf[offset+1] = byte(value >> 8)
+		buf[offset+2] = byte(value >> 16)
+		buf[offset+3] = byte(value >> 24)
+		buf[offset+4] = byte(value >> 32)
+		buf[offset+5] = byte(value >> 40)
+		buf[offset+6] = byte(value >> 48)
+		buf[offset+7] = byte(value >> 56)
+	} else {
+		buf[offset+0] = byte(value >> 56)
+		buf[offset+1] = byte(value >> 48)
+		buf[offset+2] = byte(value >> 40)
+		buf[offset+3] = byte(value >> 32)
+		buf[offset+4] = byte(value >> 24)
+		buf[offset+5] = byte(value >> 16)
+		buf[offset+6] = byte(value >> 8)
+		buf[offset+7] = byte(value)
+	}
+}
+
+func int16ToBytes(i int16) []byte {

Review Comment:
   Replace int64 with uint64



##########
pkg/wal/wal.go:
##########
@@ -42,19 +75,759 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// log implements the WAL interface.
+type log struct {
+	entryCloser      *run.Closer
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentMap       map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+	writeWaitGroup   sync.WaitGroup
+	flushWaitGroup   sync.WaitGroup
+	workSegmentMutex sync.Mutex
+	segmentMapMutex  sync.RWMutex
+	closerOnce       sync.Once
+}
+
+type segment struct {
+	file       *os.File
+	path       string
+	logEntries []LogEntry
+	segmentID  SegmentID
+}
+
+type logRequest struct {
+	seriesID  common.SeriesIDV2
+	timestamp time.Time
+	callback  func(common.SeriesIDV2, time.Time, []byte, error)
+	data      []byte
+}
+
+type logEntry struct {
+	timestamps  []time.Time
+	values      *list.List
+	seriesID    common.SeriesIDV2
+	entryLength int64
+	count       int32
+}
+
+type buffer struct {
+	timestampMap map[common.SeriesIDV2][]time.Time
+	valueMap     map[common.SeriesIDV2][]byte
+	callbackMap  map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)
+	count        int
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	walOptions := DefaultOptions
+	if options != nil {
+		fileSize := options.FileSize
+		if fileSize <= 0 {
+			fileSize = DefaultOptions.FileSize
+		}
+		bufferSize := options.BufferSize
+		if bufferSize <= 0 {
+			bufferSize = DefaultOptions.BufferSize
+		}
+		bufferBatchInterval := options.BufferBatchInterval
+		if bufferBatchInterval <= 0 {
+			bufferBatchInterval = DefaultOptions.BufferBatchInterval
+		}
+		walOptions = &Options{
+			FileSize:            fileSize,
+			BufferSize:          bufferSize,
+			BufferBatchInterval: bufferBatchInterval,
+		}
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path: "+path)
+	}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	log := &log{
+		path:             path,
+		options:          *walOptions,
+		logger:           logger.GetLogger(moduleName),
+		writeChannel:     make(chan logRequest),
+		flushChannel:     make(chan buffer, walOptions.FlushQueueSize),
+		bytesBuffer:      bytes.NewBuffer([]byte{}),
+		timestampsBuffer: bytes.NewBuffer([]byte{}),
+		entryCloser:      run.NewCloser(1),
+		buffer: buffer{
+			timestampMap: make(map[common.SeriesIDV2][]time.Time),
+			valueMap:     make(map[common.SeriesIDV2][]byte),
+			callbackMap:  make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)),
+			count:        0,
+		},
+	}
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+
+	log.writeWaitGroup.Add(1)
+	log.flushWaitGroup.Add(1)
+	log.start()
+
+	log.logger.Info().Str("path", path).Msg("WAL has be initialized")
+	return log, nil
+}
+
+// Write a logging entity.
+// It will return immediately when the data is written in the buffer,
+// The callback function will be called when the entity is flushed on the persistent storage.
+func (log *log) Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error)) {
+	if !log.entryCloser.AddRunning() {
+		return
+	}
+	defer log.entryCloser.Done()
+
+	log.writeChannel <- logRequest{
+		seriesID:  seriesID,
+		timestamp: timestamp,
+		data:      data,
+		callback:  callback,
+	}
+}
+
+// Read specified segment by SegmentID.
+func (log *log) Read(segmentID SegmentID) (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segment := log.segmentMap[segmentID]
+	return segment, nil
+}
+
+// ReadAllSegments reads all segments sorted by their creation time in ascending order.
+func (log *log) ReadAllSegments() ([]Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segments := make([]Segment, 0)
+	for _, segment := range log.segmentMap {
+		segments = append(segments, segment)
+	}
+	return segments, nil
+}
+
+// Rotate closes the open segment and opens a new one, returning the closed segment details.
+func (log *log) Rotate() (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	oldWorkSegment, err := log.swapWorkSegment()
+	if err != nil {
+		return nil, err
+	}
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Update segment information.
+	workSegment := log.workSegment
+	log.segmentMap[workSegment.segmentID] = workSegment
+	return oldWorkSegment, nil
+}
+
+// Delete the specified segment.
+func (log *log) Delete(segmentID SegmentID) error {
+	if !log.entryCloser.AddRunning() {
+		return errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Segment which will be deleted must be closed.
+	if segmentID == log.workSegment.segmentID {
+		return errors.New("Can not delete the segment which is working")
+	}
+
+	err := os.Remove(log.segmentMap[segmentID].path)
+	if err != nil {
+		return errors.Wrap(err, "Delete WAL segment error")
+	}
+	delete(log.segmentMap, segmentID)
+	return nil
+}
+
+// Close all of segments and stop WAL work.
+func (log *log) Close() error {
+	log.closerOnce.Do(func() {
+		log.logger.Info().Msg("Closing WAL...")
+
+		log.entryCloser.Done()
+		log.entryCloser.CloseThenWait()
+
+		close(log.writeChannel)
+		log.writeWaitGroup.Wait()
+
+		close(log.flushChannel)
+		log.flushWaitGroup.Wait()
+
+		if err := log.flushBuffer(log.buffer); err != nil {
+			log.logger.Err(err).Msg("Flushing buffer failed")
+		}
+		if err := log.workSegment.file.Close(); err != nil {
+			log.logger.Err(err).Msg("Close work segment file error")
+		}
+		log.logger.Info().Msg("Closed WAL")
+	})
+	return nil
+}
+
+func (log *log) start() {
+	go func() {
+		log.logger.Info().Msg("Start batch task...")
+
+		defer log.writeWaitGroup.Done()
+
+		bufferVolume := 0
+		for {
+			timer := time.NewTimer(log.options.BufferBatchInterval)
+			select {
+			case request, chOpen := <-log.writeChannel:
+				if !chOpen {
+					log.logger.Info().Msg("Stop batch task when write-channel closed!")
+					return
+				}
+
+				log.buffer.write(request)
+				if log.logger.Debug().Enabled() {
+					log.logger.Debug().Msg("Write request to buffer. elements: " + strconv.Itoa(log.buffer.count))
+				}
+
+				bufferVolume += len(request.seriesID.Marshal()) + timestampVolumeLength + len(request.data)
+				if bufferVolume > log.options.BufferSize {
+					log.triggerFlushing()
+					bufferVolume = 0
+				}
+				continue
+			case <-timer.C:
+				if bufferVolume == 0 {
+					continue
+				}
+				log.triggerFlushing()
+				bufferVolume = 0
+				continue
+			}
+		}
+	}()
+
+	go func() {
+		log.logger.Info().Msg("Start flush task...")
+
+		defer log.flushWaitGroup.Done()
+
+		for batch := range log.flushChannel {
+			startTime := time.Now()
+
+			var err error
+			for i := 0; i < maxRetries; i++ {
+				if err = log.flushBuffer(batch); err != nil {
+					log.logger.Err(err).Msg("Flushing buffer failed. Retrying...")
+					time.Sleep(100 * time.Millisecond)
+					continue
+				}
+				break
+			}
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Flushed buffer to WAL file. elements: " +
+					strconv.Itoa(batch.count) + ", cost: " + time.Since(startTime).String())
+			}
+
+			batch.notifyRequests(err)
+		}
+		log.logger.Info().Msg("Stop flush task when flush-channel closed!")
+	}()
+
+	log.logger.Info().Msg("Started WAL")
+}
+
+func (log *log) triggerFlushing() {
+	for {
+		select {
+		case log.flushChannel <- log.buffer:
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Send buffer to flush-channel. elements: " + strconv.Itoa(log.buffer.count))
+			}
+			log.newBuffer()
+			return
+		default:
+		}
+		time.Sleep(10 * time.Millisecond)
+	}
+}
+
+func (log *log) newBuffer() {
+	log.buffer.timestampMap = make(map[common.SeriesIDV2][]time.Time)
+	log.buffer.valueMap = make(map[common.SeriesIDV2][]byte)
+	log.buffer.callbackMap = make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error))
+	log.buffer.count = 0
+}
+
+func (log *log) flushBuffer(buffer buffer) error {
+	if buffer.count == 0 {
+		return nil
+	}
+
+	defer func() {
+		log.bytesBuffer.Reset()
+		log.timestampsBuffer.Reset()
+	}()
+
+	// placeholder, preset batch length value is 0
+	batchLen := 0
+	if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(batchLen)); err != nil {
+		return errors.Wrap(err, "Write batch length error")
+	}
+	for seriesID, timestamps := range buffer.timestampMap {
+		// Generate seriesID binary
+		seriesIDBytes := seriesID.Marshal()
+		seriesIDBytesLen := len(seriesIDBytes)
+
+		// Generate timestamps compression binary
+		log.timestampsBuffer.Reset()
+		timestampWriter := encoding.NewWriter()
+		timestampEncoder := encoding.NewXOREncoder(timestampWriter)
+		timestampWriter.Reset(log.timestampsBuffer)
+		for _, timestamp := range timestamps {
+			timestampEncoder.Write(timeTouUnixNano(timestamp))
+		}
+		timestampWriter.Flush()
+		timestampsBytes := log.timestampsBuffer.Bytes()
+		timestampsBytesLen := len(timestampsBytes)
+
+		// Generate values compression binary
+		valuesBytes := snappy.Encode(nil, buffer.valueMap[seriesID])
+
+		// Write entry data
+		entryLen := seriesIDLength + seriesIDBytesLen + seriesCountLength + timestampsBinaryLength + timestampsBytesLen + len(valuesBytes)
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(entryLen)); err != nil {
+			return errors.Wrap(err, "Write entry length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int16(seriesIDBytesLen)); err != nil {
+			return errors.Wrap(err, "Write seriesID length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, seriesIDBytes); err != nil {
+			return errors.Wrap(err, "Write seriesID error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int32(len(timestamps))); err != nil {
+			return errors.Wrap(err, "Write series count error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int16(timestampsBytesLen)); err != nil {
+			return errors.Wrap(err, "Write timestamps length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, timestampsBytes); err != nil {
+			return errors.Wrap(err, "Write timestamps error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, valuesBytes); err != nil {
+			return errors.Wrap(err, "Write values error")
+		}
+	}
+	// Rewrite batch length
+	batchBytes := log.bytesBuffer.Bytes()
+	batchLen = len(batchBytes) - batchLength
+	rewriteInt64InBuf(batchBytes, int64(batchLen), 0, binary.LittleEndian)
+
+	return log.writeWorkSegment(batchBytes)
+}
+
+func (log *log) swapWorkSegment() (Segment, error) {
+	log.workSegmentMutex.Lock()
+	defer log.workSegmentMutex.Unlock()
+
+	if err := log.workSegment.file.Close(); err != nil {
+		return nil, errors.Wrap(err, "Close WAL segment error")
+	}
+
+	oldSegment := log.workSegment
+	// Create new segment.
+	segmentID := log.workSegment.segmentID + 1

Review Comment:
   What will happen if the segmentID overflow? In the loading process, the max id will be the work id. Such an assumption will fail if data overflow happens.



##########
pkg/wal/wal.go:
##########
@@ -42,19 +75,759 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// log implements the WAL interface.
+type log struct {
+	entryCloser      *run.Closer
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentMap       map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+	writeWaitGroup   sync.WaitGroup
+	flushWaitGroup   sync.WaitGroup
+	workSegmentMutex sync.Mutex
+	segmentMapMutex  sync.RWMutex
+	closerOnce       sync.Once
+}
+
+type segment struct {
+	file       *os.File
+	path       string
+	logEntries []LogEntry
+	segmentID  SegmentID
+}
+
+type logRequest struct {
+	seriesID  common.SeriesIDV2
+	timestamp time.Time
+	callback  func(common.SeriesIDV2, time.Time, []byte, error)
+	data      []byte
+}
+
+type logEntry struct {
+	timestamps  []time.Time
+	values      *list.List
+	seriesID    common.SeriesIDV2
+	entryLength int64
+	count       int32
+}
+
+type buffer struct {
+	timestampMap map[common.SeriesIDV2][]time.Time
+	valueMap     map[common.SeriesIDV2][]byte
+	callbackMap  map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)
+	count        int
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	walOptions := DefaultOptions
+	if options != nil {
+		fileSize := options.FileSize
+		if fileSize <= 0 {
+			fileSize = DefaultOptions.FileSize
+		}
+		bufferSize := options.BufferSize
+		if bufferSize <= 0 {
+			bufferSize = DefaultOptions.BufferSize
+		}
+		bufferBatchInterval := options.BufferBatchInterval
+		if bufferBatchInterval <= 0 {
+			bufferBatchInterval = DefaultOptions.BufferBatchInterval
+		}
+		walOptions = &Options{
+			FileSize:            fileSize,
+			BufferSize:          bufferSize,
+			BufferBatchInterval: bufferBatchInterval,
+		}
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path: "+path)
+	}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	log := &log{
+		path:             path,
+		options:          *walOptions,
+		logger:           logger.GetLogger(moduleName),
+		writeChannel:     make(chan logRequest),
+		flushChannel:     make(chan buffer, walOptions.FlushQueueSize),
+		bytesBuffer:      bytes.NewBuffer([]byte{}),
+		timestampsBuffer: bytes.NewBuffer([]byte{}),
+		entryCloser:      run.NewCloser(1),
+		buffer: buffer{
+			timestampMap: make(map[common.SeriesIDV2][]time.Time),
+			valueMap:     make(map[common.SeriesIDV2][]byte),
+			callbackMap:  make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)),
+			count:        0,
+		},
+	}
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+
+	log.writeWaitGroup.Add(1)
+	log.flushWaitGroup.Add(1)
+	log.start()
+
+	log.logger.Info().Str("path", path).Msg("WAL has be initialized")
+	return log, nil
+}
+
+// Write a logging entity.
+// It will return immediately when the data is written in the buffer,
+// The callback function will be called when the entity is flushed on the persistent storage.
+func (log *log) Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error)) {
+	if !log.entryCloser.AddRunning() {
+		return
+	}
+	defer log.entryCloser.Done()
+
+	log.writeChannel <- logRequest{
+		seriesID:  seriesID,
+		timestamp: timestamp,
+		data:      data,
+		callback:  callback,
+	}
+}
+
+// Read specified segment by SegmentID.
+func (log *log) Read(segmentID SegmentID) (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segment := log.segmentMap[segmentID]
+	return segment, nil
+}
+
+// ReadAllSegments reads all segments sorted by their creation time in ascending order.
+func (log *log) ReadAllSegments() ([]Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segments := make([]Segment, 0)
+	for _, segment := range log.segmentMap {
+		segments = append(segments, segment)
+	}
+	return segments, nil
+}
+
+// Rotate closes the open segment and opens a new one, returning the closed segment details.
+func (log *log) Rotate() (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	oldWorkSegment, err := log.swapWorkSegment()
+	if err != nil {
+		return nil, err
+	}
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Update segment information.
+	workSegment := log.workSegment
+	log.segmentMap[workSegment.segmentID] = workSegment
+	return oldWorkSegment, nil
+}
+
+// Delete the specified segment.
+func (log *log) Delete(segmentID SegmentID) error {
+	if !log.entryCloser.AddRunning() {
+		return errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Segment which will be deleted must be closed.
+	if segmentID == log.workSegment.segmentID {
+		return errors.New("Can not delete the segment which is working")
+	}
+
+	err := os.Remove(log.segmentMap[segmentID].path)
+	if err != nil {
+		return errors.Wrap(err, "Delete WAL segment error")
+	}
+	delete(log.segmentMap, segmentID)
+	return nil
+}
+
+// Close all of segments and stop WAL work.
+func (log *log) Close() error {
+	log.closerOnce.Do(func() {
+		log.logger.Info().Msg("Closing WAL...")
+
+		log.entryCloser.Done()
+		log.entryCloser.CloseThenWait()
+
+		close(log.writeChannel)
+		log.writeWaitGroup.Wait()
+
+		close(log.flushChannel)
+		log.flushWaitGroup.Wait()
+
+		if err := log.flushBuffer(log.buffer); err != nil {
+			log.logger.Err(err).Msg("Flushing buffer failed")
+		}
+		if err := log.workSegment.file.Close(); err != nil {
+			log.logger.Err(err).Msg("Close work segment file error")
+		}
+		log.logger.Info().Msg("Closed WAL")
+	})
+	return nil
+}
+
+func (log *log) start() {
+	go func() {
+		log.logger.Info().Msg("Start batch task...")
+
+		defer log.writeWaitGroup.Done()
+
+		bufferVolume := 0
+		for {
+			timer := time.NewTimer(log.options.BufferBatchInterval)
+			select {
+			case request, chOpen := <-log.writeChannel:
+				if !chOpen {
+					log.logger.Info().Msg("Stop batch task when write-channel closed!")
+					return
+				}
+
+				log.buffer.write(request)
+				if log.logger.Debug().Enabled() {
+					log.logger.Debug().Msg("Write request to buffer. elements: " + strconv.Itoa(log.buffer.count))
+				}
+
+				bufferVolume += len(request.seriesID.Marshal()) + timestampVolumeLength + len(request.data)
+				if bufferVolume > log.options.BufferSize {
+					log.triggerFlushing()
+					bufferVolume = 0
+				}
+				continue
+			case <-timer.C:
+				if bufferVolume == 0 {
+					continue
+				}
+				log.triggerFlushing()
+				bufferVolume = 0
+				continue

Review Comment:
   ```suggestion
   ```



##########
pkg/wal/wal.go:
##########
@@ -42,19 +75,759 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// log implements the WAL interface.
+type log struct {
+	entryCloser      *run.Closer
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentMap       map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+	writeWaitGroup   sync.WaitGroup
+	flushWaitGroup   sync.WaitGroup
+	workSegmentMutex sync.Mutex
+	segmentMapMutex  sync.RWMutex
+	closerOnce       sync.Once
+}
+
+type segment struct {
+	file       *os.File
+	path       string
+	logEntries []LogEntry
+	segmentID  SegmentID
+}
+
+type logRequest struct {
+	seriesID  common.SeriesIDV2
+	timestamp time.Time
+	callback  func(common.SeriesIDV2, time.Time, []byte, error)
+	data      []byte
+}
+
+type logEntry struct {
+	timestamps  []time.Time
+	values      *list.List
+	seriesID    common.SeriesIDV2
+	entryLength int64
+	count       int32
+}
+
+type buffer struct {
+	timestampMap map[common.SeriesIDV2][]time.Time
+	valueMap     map[common.SeriesIDV2][]byte
+	callbackMap  map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)
+	count        int
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	walOptions := DefaultOptions
+	if options != nil {
+		fileSize := options.FileSize
+		if fileSize <= 0 {
+			fileSize = DefaultOptions.FileSize
+		}
+		bufferSize := options.BufferSize
+		if bufferSize <= 0 {
+			bufferSize = DefaultOptions.BufferSize
+		}
+		bufferBatchInterval := options.BufferBatchInterval
+		if bufferBatchInterval <= 0 {
+			bufferBatchInterval = DefaultOptions.BufferBatchInterval
+		}
+		walOptions = &Options{
+			FileSize:            fileSize,
+			BufferSize:          bufferSize,
+			BufferBatchInterval: bufferBatchInterval,
+		}
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path: "+path)
+	}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	log := &log{
+		path:             path,
+		options:          *walOptions,
+		logger:           logger.GetLogger(moduleName),
+		writeChannel:     make(chan logRequest),
+		flushChannel:     make(chan buffer, walOptions.FlushQueueSize),
+		bytesBuffer:      bytes.NewBuffer([]byte{}),
+		timestampsBuffer: bytes.NewBuffer([]byte{}),
+		entryCloser:      run.NewCloser(1),
+		buffer: buffer{
+			timestampMap: make(map[common.SeriesIDV2][]time.Time),
+			valueMap:     make(map[common.SeriesIDV2][]byte),
+			callbackMap:  make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)),
+			count:        0,
+		},
+	}
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+
+	log.writeWaitGroup.Add(1)
+	log.flushWaitGroup.Add(1)
+	log.start()
+
+	log.logger.Info().Str("path", path).Msg("WAL has be initialized")
+	return log, nil
+}
+
+// Write a logging entity.
+// It will return immediately when the data is written in the buffer,
+// The callback function will be called when the entity is flushed on the persistent storage.
+func (log *log) Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error)) {
+	if !log.entryCloser.AddRunning() {
+		return
+	}
+	defer log.entryCloser.Done()
+
+	log.writeChannel <- logRequest{
+		seriesID:  seriesID,
+		timestamp: timestamp,
+		data:      data,
+		callback:  callback,
+	}
+}
+
+// Read specified segment by SegmentID.
+func (log *log) Read(segmentID SegmentID) (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segment := log.segmentMap[segmentID]
+	return segment, nil
+}
+
+// ReadAllSegments reads all segments sorted by their creation time in ascending order.
+func (log *log) ReadAllSegments() ([]Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segments := make([]Segment, 0)
+	for _, segment := range log.segmentMap {
+		segments = append(segments, segment)
+	}
+	return segments, nil
+}
+
+// Rotate closes the open segment and opens a new one, returning the closed segment details.
+func (log *log) Rotate() (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	oldWorkSegment, err := log.swapWorkSegment()
+	if err != nil {
+		return nil, err
+	}
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Update segment information.
+	workSegment := log.workSegment
+	log.segmentMap[workSegment.segmentID] = workSegment
+	return oldWorkSegment, nil
+}
+
+// Delete the specified segment.
+func (log *log) Delete(segmentID SegmentID) error {
+	if !log.entryCloser.AddRunning() {
+		return errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Segment which will be deleted must be closed.
+	if segmentID == log.workSegment.segmentID {
+		return errors.New("Can not delete the segment which is working")
+	}
+
+	err := os.Remove(log.segmentMap[segmentID].path)
+	if err != nil {
+		return errors.Wrap(err, "Delete WAL segment error")
+	}
+	delete(log.segmentMap, segmentID)
+	return nil
+}
+
+// Close all of segments and stop WAL work.
+func (log *log) Close() error {
+	log.closerOnce.Do(func() {
+		log.logger.Info().Msg("Closing WAL...")
+
+		log.entryCloser.Done()
+		log.entryCloser.CloseThenWait()
+
+		close(log.writeChannel)
+		log.writeWaitGroup.Wait()
+
+		close(log.flushChannel)
+		log.flushWaitGroup.Wait()
+
+		if err := log.flushBuffer(log.buffer); err != nil {
+			log.logger.Err(err).Msg("Flushing buffer failed")
+		}
+		if err := log.workSegment.file.Close(); err != nil {
+			log.logger.Err(err).Msg("Close work segment file error")
+		}
+		log.logger.Info().Msg("Closed WAL")
+	})
+	return nil
+}
+
+func (log *log) start() {
+	go func() {
+		log.logger.Info().Msg("Start batch task...")
+
+		defer log.writeWaitGroup.Done()
+
+		bufferVolume := 0
+		for {
+			timer := time.NewTimer(log.options.BufferBatchInterval)
+			select {
+			case request, chOpen := <-log.writeChannel:
+				if !chOpen {
+					log.logger.Info().Msg("Stop batch task when write-channel closed!")
+					return
+				}
+
+				log.buffer.write(request)
+				if log.logger.Debug().Enabled() {
+					log.logger.Debug().Msg("Write request to buffer. elements: " + strconv.Itoa(log.buffer.count))
+				}
+
+				bufferVolume += len(request.seriesID.Marshal()) + timestampVolumeLength + len(request.data)
+				if bufferVolume > log.options.BufferSize {
+					log.triggerFlushing()
+					bufferVolume = 0
+				}
+				continue
+			case <-timer.C:
+				if bufferVolume == 0 {
+					continue
+				}
+				log.triggerFlushing()
+				bufferVolume = 0
+				continue
+			}
+		}
+	}()
+
+	go func() {
+		log.logger.Info().Msg("Start flush task...")
+
+		defer log.flushWaitGroup.Done()
+
+		for batch := range log.flushChannel {
+			startTime := time.Now()
+
+			var err error
+			for i := 0; i < maxRetries; i++ {
+				if err = log.flushBuffer(batch); err != nil {
+					log.logger.Err(err).Msg("Flushing buffer failed. Retrying...")
+					time.Sleep(100 * time.Millisecond)

Review Comment:
   ```suggestion
   					time.Sleep(time.Second)
   ```



##########
pkg/wal/wal.go:
##########
@@ -42,19 +75,759 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// log implements the WAL interface.
+type log struct {
+	entryCloser      *run.Closer
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentMap       map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+	writeWaitGroup   sync.WaitGroup
+	flushWaitGroup   sync.WaitGroup
+	workSegmentMutex sync.Mutex
+	segmentMapMutex  sync.RWMutex
+	closerOnce       sync.Once
+}
+
+type segment struct {
+	file       *os.File
+	path       string
+	logEntries []LogEntry
+	segmentID  SegmentID
+}
+
+type logRequest struct {
+	seriesID  common.SeriesIDV2
+	timestamp time.Time
+	callback  func(common.SeriesIDV2, time.Time, []byte, error)
+	data      []byte
+}
+
+type logEntry struct {
+	timestamps  []time.Time
+	values      *list.List
+	seriesID    common.SeriesIDV2
+	entryLength int64
+	count       int32
+}
+
+type buffer struct {
+	timestampMap map[common.SeriesIDV2][]time.Time
+	valueMap     map[common.SeriesIDV2][]byte
+	callbackMap  map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)
+	count        int
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	walOptions := DefaultOptions
+	if options != nil {
+		fileSize := options.FileSize
+		if fileSize <= 0 {
+			fileSize = DefaultOptions.FileSize
+		}
+		bufferSize := options.BufferSize
+		if bufferSize <= 0 {
+			bufferSize = DefaultOptions.BufferSize
+		}
+		bufferBatchInterval := options.BufferBatchInterval
+		if bufferBatchInterval <= 0 {
+			bufferBatchInterval = DefaultOptions.BufferBatchInterval
+		}
+		walOptions = &Options{
+			FileSize:            fileSize,
+			BufferSize:          bufferSize,
+			BufferBatchInterval: bufferBatchInterval,
+		}
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path: "+path)
+	}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	log := &log{
+		path:             path,
+		options:          *walOptions,
+		logger:           logger.GetLogger(moduleName),
+		writeChannel:     make(chan logRequest),
+		flushChannel:     make(chan buffer, walOptions.FlushQueueSize),
+		bytesBuffer:      bytes.NewBuffer([]byte{}),
+		timestampsBuffer: bytes.NewBuffer([]byte{}),
+		entryCloser:      run.NewCloser(1),
+		buffer: buffer{
+			timestampMap: make(map[common.SeriesIDV2][]time.Time),
+			valueMap:     make(map[common.SeriesIDV2][]byte),
+			callbackMap:  make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)),
+			count:        0,
+		},
+	}
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+
+	log.writeWaitGroup.Add(1)
+	log.flushWaitGroup.Add(1)
+	log.start()
+
+	log.logger.Info().Str("path", path).Msg("WAL has be initialized")
+	return log, nil
+}
+
+// Write a logging entity.
+// It will return immediately when the data is written in the buffer,
+// The callback function will be called when the entity is flushed on the persistent storage.
+func (log *log) Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error)) {
+	if !log.entryCloser.AddRunning() {
+		return
+	}
+	defer log.entryCloser.Done()
+
+	log.writeChannel <- logRequest{
+		seriesID:  seriesID,
+		timestamp: timestamp,
+		data:      data,
+		callback:  callback,
+	}
+}
+
+// Read specified segment by SegmentID.
+func (log *log) Read(segmentID SegmentID) (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segment := log.segmentMap[segmentID]
+	return segment, nil
+}
+
+// ReadAllSegments reads all segments sorted by their creation time in ascending order.
+func (log *log) ReadAllSegments() ([]Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segments := make([]Segment, 0)
+	for _, segment := range log.segmentMap {
+		segments = append(segments, segment)
+	}
+	return segments, nil
+}
+
+// Rotate closes the open segment and opens a new one, returning the closed segment details.
+func (log *log) Rotate() (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	oldWorkSegment, err := log.swapWorkSegment()
+	if err != nil {
+		return nil, err
+	}
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Update segment information.
+	workSegment := log.workSegment
+	log.segmentMap[workSegment.segmentID] = workSegment
+	return oldWorkSegment, nil
+}
+
+// Delete the specified segment.
+func (log *log) Delete(segmentID SegmentID) error {
+	if !log.entryCloser.AddRunning() {
+		return errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Segment which will be deleted must be closed.
+	if segmentID == log.workSegment.segmentID {
+		return errors.New("Can not delete the segment which is working")
+	}
+
+	err := os.Remove(log.segmentMap[segmentID].path)
+	if err != nil {
+		return errors.Wrap(err, "Delete WAL segment error")
+	}
+	delete(log.segmentMap, segmentID)
+	return nil
+}
+
+// Close all of segments and stop WAL work.
+func (log *log) Close() error {
+	log.closerOnce.Do(func() {
+		log.logger.Info().Msg("Closing WAL...")
+
+		log.entryCloser.Done()
+		log.entryCloser.CloseThenWait()
+
+		close(log.writeChannel)
+		log.writeWaitGroup.Wait()
+
+		close(log.flushChannel)
+		log.flushWaitGroup.Wait()
+
+		if err := log.flushBuffer(log.buffer); err != nil {
+			log.logger.Err(err).Msg("Flushing buffer failed")
+		}
+		if err := log.workSegment.file.Close(); err != nil {
+			log.logger.Err(err).Msg("Close work segment file error")
+		}
+		log.logger.Info().Msg("Closed WAL")
+	})
+	return nil
+}
+
+func (log *log) start() {
+	go func() {
+		log.logger.Info().Msg("Start batch task...")
+
+		defer log.writeWaitGroup.Done()
+
+		bufferVolume := 0
+		for {
+			timer := time.NewTimer(log.options.BufferBatchInterval)
+			select {
+			case request, chOpen := <-log.writeChannel:
+				if !chOpen {
+					log.logger.Info().Msg("Stop batch task when write-channel closed!")
+					return
+				}
+
+				log.buffer.write(request)
+				if log.logger.Debug().Enabled() {
+					log.logger.Debug().Msg("Write request to buffer. elements: " + strconv.Itoa(log.buffer.count))
+				}
+
+				bufferVolume += len(request.seriesID.Marshal()) + timestampVolumeLength + len(request.data)
+				if bufferVolume > log.options.BufferSize {
+					log.triggerFlushing()
+					bufferVolume = 0
+				}
+				continue
+			case <-timer.C:
+				if bufferVolume == 0 {
+					continue
+				}
+				log.triggerFlushing()
+				bufferVolume = 0
+				continue
+			}
+		}
+	}()
+
+	go func() {
+		log.logger.Info().Msg("Start flush task...")
+
+		defer log.flushWaitGroup.Done()
+
+		for batch := range log.flushChannel {
+			startTime := time.Now()
+
+			var err error
+			for i := 0; i < maxRetries; i++ {
+				if err = log.flushBuffer(batch); err != nil {
+					log.logger.Err(err).Msg("Flushing buffer failed. Retrying...")
+					time.Sleep(100 * time.Millisecond)
+					continue
+				}
+				break
+			}
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Flushed buffer to WAL file. elements: " +
+					strconv.Itoa(batch.count) + ", cost: " + time.Since(startTime).String())
+			}
+
+			batch.notifyRequests(err)
+		}
+		log.logger.Info().Msg("Stop flush task when flush-channel closed!")
+	}()
+
+	log.logger.Info().Msg("Started WAL")
+}
+
+func (log *log) triggerFlushing() {
+	for {
+		select {
+		case log.flushChannel <- log.buffer:
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Send buffer to flush-channel. elements: " + strconv.Itoa(log.buffer.count))
+			}
+			log.newBuffer()
+			return
+		default:
+		}
+		time.Sleep(10 * time.Millisecond)
+	}
+}
+
+func (log *log) newBuffer() {
+	log.buffer.timestampMap = make(map[common.SeriesIDV2][]time.Time)
+	log.buffer.valueMap = make(map[common.SeriesIDV2][]byte)
+	log.buffer.callbackMap = make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error))
+	log.buffer.count = 0
+}
+
+func (log *log) flushBuffer(buffer buffer) error {
+	if buffer.count == 0 {
+		return nil
+	}
+
+	defer func() {
+		log.bytesBuffer.Reset()
+		log.timestampsBuffer.Reset()
+	}()
+
+	// placeholder, preset batch length value is 0
+	batchLen := 0
+	if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(batchLen)); err != nil {
+		return errors.Wrap(err, "Write batch length error")
+	}
+	for seriesID, timestamps := range buffer.timestampMap {
+		// Generate seriesID binary
+		seriesIDBytes := seriesID.Marshal()
+		seriesIDBytesLen := len(seriesIDBytes)
+
+		// Generate timestamps compression binary
+		log.timestampsBuffer.Reset()
+		timestampWriter := encoding.NewWriter()
+		timestampEncoder := encoding.NewXOREncoder(timestampWriter)
+		timestampWriter.Reset(log.timestampsBuffer)
+		for _, timestamp := range timestamps {
+			timestampEncoder.Write(timeTouUnixNano(timestamp))
+		}
+		timestampWriter.Flush()
+		timestampsBytes := log.timestampsBuffer.Bytes()
+		timestampsBytesLen := len(timestampsBytes)
+
+		// Generate values compression binary
+		valuesBytes := snappy.Encode(nil, buffer.valueMap[seriesID])
+
+		// Write entry data
+		entryLen := seriesIDLength + seriesIDBytesLen + seriesCountLength + timestampsBinaryLength + timestampsBytesLen + len(valuesBytes)
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(entryLen)); err != nil {
+			return errors.Wrap(err, "Write entry length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int16(seriesIDBytesLen)); err != nil {
+			return errors.Wrap(err, "Write seriesID length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, seriesIDBytes); err != nil {
+			return errors.Wrap(err, "Write seriesID error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int32(len(timestamps))); err != nil {
+			return errors.Wrap(err, "Write series count error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int16(timestampsBytesLen)); err != nil {
+			return errors.Wrap(err, "Write timestamps length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, timestampsBytes); err != nil {
+			return errors.Wrap(err, "Write timestamps error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, valuesBytes); err != nil {
+			return errors.Wrap(err, "Write values error")
+		}
+	}
+	// Rewrite batch length
+	batchBytes := log.bytesBuffer.Bytes()
+	batchLen = len(batchBytes) - batchLength
+	rewriteInt64InBuf(batchBytes, int64(batchLen), 0, binary.LittleEndian)
+
+	return log.writeWorkSegment(batchBytes)
+}
+
+func (log *log) swapWorkSegment() (Segment, error) {
+	log.workSegmentMutex.Lock()
+	defer log.workSegmentMutex.Unlock()
+
+	if err := log.workSegment.file.Close(); err != nil {
+		return nil, errors.Wrap(err, "Close WAL segment error")
+	}
+
+	oldSegment := log.workSegment
+	// Create new segment.
+	segmentID := log.workSegment.segmentID + 1
+	segment := &segment{
+		segmentID: segmentID,
+		path:      filepath.Join(log.path, segmentName(uint64(segmentID))),
+	}
+	var err error
+	segment.file, err = os.OpenFile(segment.path, os.O_CREATE|os.O_RDWR|os.O_TRUNC, os.ModePerm)
+	if err != nil {
+		return nil, errors.Wrap(err, "Open WAL segment error")
+	}
+	log.workSegment = segment
+
+	return oldSegment, nil
+}
+
+func (log *log) writeWorkSegment(data []byte) error {
+	log.workSegmentMutex.Lock()
+	defer log.workSegmentMutex.Unlock()
+
+	// Write batch data to WAL segment file
+	if _, err := log.workSegment.file.Write(data); err != nil {
+		return errors.Wrap(err, "Write WAL segment file error, file: "+log.workSegment.path)
+	}
+	if err := log.workSegment.file.Sync(); err != nil {
+		log.logger.Warn().Msg("Sync WAL segment file to disk failed, file: " + log.workSegment.path)
+	}
+	return nil
+}
+
+func (log *log) load() error {
+	files, err := os.ReadDir(log.path)
+	if err != nil {
+		return errors.Wrap(err, "Can not read dir: "+log.path)
+	}
+	// Load all of WAL segments.
+	var workSegmentID SegmentID
+	log.segmentMap = make(map[SegmentID]*segment)
+	for _, file := range files {
+		name := file.Name()
+		segmentID, parsePathErr := strconv.ParseUint(name[3:19], 10, 64)
+		if parsePathErr != nil {
+			return errors.Wrap(parsePathErr, "Parse file name error, name: "+name)
+		}
+		if segmentID > uint64(workSegmentID) {
+			workSegmentID = SegmentID(segmentID)
+		}
+		segment := &segment{
+			segmentID: SegmentID(segmentID),
+			path:      filepath.Join(log.path, segmentName(segmentID)),
+		}
+		if err = segment.parseLogEntries(); err != nil {
+			return errors.Wrap(err, "Fail to parse log entries")
+		}
+		log.segmentMap[SegmentID(segmentID)] = segment
+
+		if log.logger.Debug().Enabled() {
+			log.logger.Debug().Msg("Loaded segment file: " + segment.path)
+		}
+	}
+
+	// If load first time.
+	if len(log.segmentMap) == 0 {
+		segment := &segment{
+			segmentID: 1,
+			path:      filepath.Join(log.path, segmentName(1)),
+		}
+		log.segmentMap[1] = segment
+		log.workSegment = segment
+	} else {
+		log.workSegment = log.segmentMap[workSegmentID]
+	}
+	log.workSegment.file, err = os.OpenFile(log.workSegment.path, os.O_CREATE|os.O_RDWR|os.O_APPEND, os.ModePerm)
+	if err != nil {
+		return errors.Wrap(err, "Open WAL segment error, file: "+log.workSegment.path)
+	}
+	return nil
+}
+
+func (segment *segment) GetSegmentID() SegmentID {
+	return segment.segmentID
+}
+
+func (segment *segment) GetLogEntries() []LogEntry {
+	return segment.logEntries
+}
+
+func (segment *segment) parseLogEntries() error {
+	segmentBytes, err := os.ReadFile(segment.path)
+	if err != nil {
+		return errors.Wrap(err, "Read WAL segment failed, path: "+segment.path)
+	}
+
+	var logEntries []LogEntry
+	var data []byte
+	var batchLen int64
+	var entryLen int64
+	var seriesIDLen int16
+	var seriesID common.SeriesIDV2
+	var seriesCount int32
+	var timestampsBinaryLen int16
+	var entryEndPos int
+	var bytesBuf *bytes.Buffer
+
+	oldPos := 0
+	pos := 0
+	parseNextBatchFlag := true
+
+	for {
+		if parseNextBatchFlag {
+			if len(segmentBytes) <= batchLength {
+				break
+			}
+			data = segmentBytes[pos : pos+batchLength]
+			bytesBuf = bytes.NewBuffer(data)
+			if err = binary.Read(bytesBuf, binary.LittleEndian, &batchLen); err != nil {
+				return errors.Wrap(err, "Read batch length fail to convert from bytes")
+			}
+			if len(segmentBytes) <= int(batchLen) {
+				break
+			}
+
+			pos += batchLength
+			oldPos = pos
+			parseNextBatchFlag = false
+		}
+
+		// Parse entryLength.
+		data = segmentBytes[pos : pos+entryLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &entryLen); err != nil {
+			return errors.Wrap(err, "Read entry length fail to convert from byte")
+		}
+		pos += entryLength
+
+		// Mark entry end-position
+		entryEndPos = pos + int(entryLen)
+		if len(segmentBytes) < entryEndPos {
+			break
+		}
+
+		// Parse seriesIDLen.
+		data = segmentBytes[pos : pos+seriesIDLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &seriesIDLen); err != nil {
+			return errors.Wrap(err, "Read seriesID length fail to convert from byte")
+		}
+		pos += seriesIDLength
+
+		// Parse seriesID.
+		data = segmentBytes[pos : pos+int(seriesIDLen)]
+		seriesID = common.ParseSeriesIDV2(data)
+		pos += int(seriesIDLen)
+
+		// Parse series count.
+		data = segmentBytes[pos : pos+seriesCountLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &seriesCount); err != nil {
+			return errors.Wrap(err, "Read series count fail to convert from byte")
+		}
+		pos += seriesCountLength
+
+		// Parse timestamps compression binary.
+		data = segmentBytes[pos : pos+timestampsBinaryLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &timestampsBinaryLen); err != nil {
+			return errors.Wrap(err, "Read timestamps compression binary length fail to convert from byte")
+		}
+		pos += timestampsBinaryLength
+		data = segmentBytes[pos : pos+int(timestampsBinaryLen)]
+		timestampReader := encoding.NewReader(bytes.NewReader(data))
+		timestampDecoder := encoding.NewXORDecoder(timestampReader)
+		var timestamps []time.Time
+		for i := 0; i < int(seriesCount); i++ {
+			if !timestampDecoder.Next() {
+				return errors.Wrap(err, "Timestamps length not match series count, index: "+strconv.Itoa(i))
+			}
+			timestamps = append(timestamps, unixNanoToTime(timestampDecoder.Value()))
+		}
+		pos += int(timestampsBinaryLen)
+
+		// Parse values compression binary.
+		data = segmentBytes[pos:entryEndPos]
+		if data, err = snappy.Decode(nil, data); err != nil {
+			return errors.Wrap(err, "Decode values compression binary fail to snappy decode")
+		}
+		values := parseValuesBinary(data)
+		if values.Len() != len(timestamps) {
+			return errors.New("Timestamps length and values length not match: " + strconv.Itoa(len(timestamps)) + " vs " + strconv.Itoa(values.Len()))
+		}
+		pos = entryEndPos
+
+		logEntry := &logEntry{
+			entryLength: entryLen,
+			seriesID:    seriesID,
+			count:       seriesCount,
+			timestamps:  timestamps,
+			values:      values,
+		}
+		logEntries = append(logEntries, logEntry)
+
+		if pos == len(segmentBytes) {
+			break
+		}
+		if pos-oldPos == int(batchLen) {
+			parseNextBatchFlag = true
+		}
+	}
+	segment.logEntries = logEntries
+	return nil
+}
+
+func (logEntry *logEntry) GetSeriesID() common.SeriesIDV2 {
+	return logEntry.seriesID
+}
+
+func (logEntry *logEntry) GetTimestamps() []time.Time {
+	return logEntry.timestamps
+}
+
+func (logEntry *logEntry) GetValues() *list.List {
+	return logEntry.values
+}
+
+func (buffer *buffer) write(request logRequest) {
+	seriesID := request.seriesID
+	buffer.timestampMap[seriesID] = append(buffer.timestampMap[seriesID], request.timestamp)
+
+	// Value item: binary-length(2-bytes) + binary data(n-bytes)
+	binaryLength := int16ToBytes(int16(len(request.data)))
+	buffer.valueMap[seriesID] = append(buffer.valueMap[seriesID], binaryLength...)
+	buffer.valueMap[seriesID] = append(buffer.valueMap[seriesID], request.data...)
+
+	buffer.callbackMap[seriesID] = append(buffer.callbackMap[seriesID], request.callback)
+	buffer.count++
+}
+
+func (buffer *buffer) notifyRequests(err error) {
+	for seriesID, callbacks := range buffer.callbackMap {
+		timestamps := buffer.timestampMap[seriesID]
+		values := buffer.valueMap[seriesID]
+		valuePos := 0
+		var valueItem []byte
+		for index, callback := range callbacks {
+			valuePos, valueItem = readValuesBinary(values, valuePos, valuesBinaryLength)
+			tryCallback(func() {
+				callback(seriesID, timestamps[index], valueItem, err)
+			})
+		}
+	}
+}
+
+func segmentName(index uint64) string {
+	return fmt.Sprintf("%v%016x%v", segmentNamePrefix, index, segmentNameSuffix)
+}
+
+func tryCallback(callback func()) {
+	defer func() {
+		if err := recover(); err != nil {
+			fmt.Printf("Run callback error is %v\n", err)
+		}
+	}()
+	callback()
+}
+
+func parseValuesBinary(binary []byte) *list.List {
+	values := list.New()
+	position := 0
+	for {
+		nextPosition, value := readValuesBinary(binary, position, valuesBinaryLength)
+		if value == nil {
+			break
+		}
+		values.PushBack(value)
+		position = nextPosition
+	}
+	return values
+}
+
+func readValuesBinary(raw []byte, position int, offsetLen int) (int, []byte) {
+	if position == len(raw) {
+		return position, nil
+	}
+
+	data := raw[position : position+offsetLen]
+	binaryLen := bytesToInt16(data)
+	position += offsetLen
+
+	data = raw[position : position+int(binaryLen)]
+	position += int(binaryLen)
+	return position, data
+}
+
+func rewriteInt64InBuf(buf []byte, value int64, offset int, order binary.ByteOrder) {

Review Comment:
   Please use uint64 to store the len.



##########
pkg/wal/wal.go:
##########
@@ -42,19 +75,759 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// log implements the WAL interface.
+type log struct {
+	entryCloser      *run.Closer
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentMap       map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+	writeWaitGroup   sync.WaitGroup
+	flushWaitGroup   sync.WaitGroup
+	workSegmentMutex sync.Mutex
+	segmentMapMutex  sync.RWMutex
+	closerOnce       sync.Once
+}
+
+type segment struct {
+	file       *os.File
+	path       string
+	logEntries []LogEntry
+	segmentID  SegmentID
+}
+
+type logRequest struct {
+	seriesID  common.SeriesIDV2
+	timestamp time.Time
+	callback  func(common.SeriesIDV2, time.Time, []byte, error)
+	data      []byte
+}
+
+type logEntry struct {
+	timestamps  []time.Time
+	values      *list.List
+	seriesID    common.SeriesIDV2
+	entryLength int64
+	count       int32
+}
+
+type buffer struct {
+	timestampMap map[common.SeriesIDV2][]time.Time
+	valueMap     map[common.SeriesIDV2][]byte
+	callbackMap  map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)
+	count        int
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	walOptions := DefaultOptions
+	if options != nil {
+		fileSize := options.FileSize
+		if fileSize <= 0 {
+			fileSize = DefaultOptions.FileSize
+		}
+		bufferSize := options.BufferSize
+		if bufferSize <= 0 {
+			bufferSize = DefaultOptions.BufferSize
+		}
+		bufferBatchInterval := options.BufferBatchInterval
+		if bufferBatchInterval <= 0 {
+			bufferBatchInterval = DefaultOptions.BufferBatchInterval
+		}
+		walOptions = &Options{
+			FileSize:            fileSize,
+			BufferSize:          bufferSize,
+			BufferBatchInterval: bufferBatchInterval,
+		}
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path: "+path)
+	}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	log := &log{
+		path:             path,
+		options:          *walOptions,
+		logger:           logger.GetLogger(moduleName),
+		writeChannel:     make(chan logRequest),
+		flushChannel:     make(chan buffer, walOptions.FlushQueueSize),
+		bytesBuffer:      bytes.NewBuffer([]byte{}),
+		timestampsBuffer: bytes.NewBuffer([]byte{}),
+		entryCloser:      run.NewCloser(1),
+		buffer: buffer{
+			timestampMap: make(map[common.SeriesIDV2][]time.Time),
+			valueMap:     make(map[common.SeriesIDV2][]byte),
+			callbackMap:  make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)),
+			count:        0,
+		},
+	}
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+
+	log.writeWaitGroup.Add(1)
+	log.flushWaitGroup.Add(1)
+	log.start()
+
+	log.logger.Info().Str("path", path).Msg("WAL has be initialized")
+	return log, nil
+}
+
+// Write a logging entity.
+// It will return immediately when the data is written in the buffer,
+// The callback function will be called when the entity is flushed on the persistent storage.
+func (log *log) Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error)) {
+	if !log.entryCloser.AddRunning() {
+		return
+	}
+	defer log.entryCloser.Done()
+
+	log.writeChannel <- logRequest{
+		seriesID:  seriesID,
+		timestamp: timestamp,
+		data:      data,
+		callback:  callback,
+	}
+}
+
+// Read specified segment by SegmentID.
+func (log *log) Read(segmentID SegmentID) (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segment := log.segmentMap[segmentID]
+	return segment, nil
+}
+
+// ReadAllSegments reads all segments sorted by their creation time in ascending order.
+func (log *log) ReadAllSegments() ([]Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segments := make([]Segment, 0)
+	for _, segment := range log.segmentMap {
+		segments = append(segments, segment)
+	}
+	return segments, nil
+}
+
+// Rotate closes the open segment and opens a new one, returning the closed segment details.
+func (log *log) Rotate() (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	oldWorkSegment, err := log.swapWorkSegment()
+	if err != nil {
+		return nil, err
+	}
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Update segment information.
+	workSegment := log.workSegment
+	log.segmentMap[workSegment.segmentID] = workSegment
+	return oldWorkSegment, nil
+}
+
+// Delete the specified segment.
+func (log *log) Delete(segmentID SegmentID) error {
+	if !log.entryCloser.AddRunning() {
+		return errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Segment which will be deleted must be closed.
+	if segmentID == log.workSegment.segmentID {
+		return errors.New("Can not delete the segment which is working")
+	}
+
+	err := os.Remove(log.segmentMap[segmentID].path)
+	if err != nil {
+		return errors.Wrap(err, "Delete WAL segment error")
+	}
+	delete(log.segmentMap, segmentID)
+	return nil
+}
+
+// Close all of segments and stop WAL work.
+func (log *log) Close() error {
+	log.closerOnce.Do(func() {
+		log.logger.Info().Msg("Closing WAL...")
+
+		log.entryCloser.Done()
+		log.entryCloser.CloseThenWait()
+
+		close(log.writeChannel)
+		log.writeWaitGroup.Wait()
+
+		close(log.flushChannel)
+		log.flushWaitGroup.Wait()
+
+		if err := log.flushBuffer(log.buffer); err != nil {
+			log.logger.Err(err).Msg("Flushing buffer failed")
+		}
+		if err := log.workSegment.file.Close(); err != nil {
+			log.logger.Err(err).Msg("Close work segment file error")
+		}
+		log.logger.Info().Msg("Closed WAL")
+	})
+	return nil
+}
+
+func (log *log) start() {
+	go func() {
+		log.logger.Info().Msg("Start batch task...")
+
+		defer log.writeWaitGroup.Done()
+
+		bufferVolume := 0
+		for {
+			timer := time.NewTimer(log.options.BufferBatchInterval)
+			select {
+			case request, chOpen := <-log.writeChannel:
+				if !chOpen {
+					log.logger.Info().Msg("Stop batch task when write-channel closed!")
+					return
+				}
+
+				log.buffer.write(request)
+				if log.logger.Debug().Enabled() {
+					log.logger.Debug().Msg("Write request to buffer. elements: " + strconv.Itoa(log.buffer.count))
+				}
+
+				bufferVolume += len(request.seriesID.Marshal()) + timestampVolumeLength + len(request.data)
+				if bufferVolume > log.options.BufferSize {
+					log.triggerFlushing()
+					bufferVolume = 0
+				}
+				continue
+			case <-timer.C:
+				if bufferVolume == 0 {
+					continue
+				}
+				log.triggerFlushing()
+				bufferVolume = 0
+				continue
+			}
+		}
+	}()
+
+	go func() {
+		log.logger.Info().Msg("Start flush task...")
+
+		defer log.flushWaitGroup.Done()
+
+		for batch := range log.flushChannel {
+			startTime := time.Now()
+
+			var err error
+			for i := 0; i < maxRetries; i++ {
+				if err = log.flushBuffer(batch); err != nil {
+					log.logger.Err(err).Msg("Flushing buffer failed. Retrying...")
+					time.Sleep(100 * time.Millisecond)
+					continue
+				}
+				break
+			}
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Flushed buffer to WAL file. elements: " +
+					strconv.Itoa(batch.count) + ", cost: " + time.Since(startTime).String())
+			}
+
+			batch.notifyRequests(err)
+		}
+		log.logger.Info().Msg("Stop flush task when flush-channel closed!")
+	}()
+
+	log.logger.Info().Msg("Started WAL")
+}
+
+func (log *log) triggerFlushing() {
+	for {
+		select {
+		case log.flushChannel <- log.buffer:
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Send buffer to flush-channel. elements: " + strconv.Itoa(log.buffer.count))
+			}
+			log.newBuffer()
+			return
+		default:
+		}
+		time.Sleep(10 * time.Millisecond)
+	}
+}
+
+func (log *log) newBuffer() {
+	log.buffer.timestampMap = make(map[common.SeriesIDV2][]time.Time)
+	log.buffer.valueMap = make(map[common.SeriesIDV2][]byte)
+	log.buffer.callbackMap = make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error))
+	log.buffer.count = 0
+}
+
+func (log *log) flushBuffer(buffer buffer) error {
+	if buffer.count == 0 {
+		return nil
+	}
+
+	defer func() {
+		log.bytesBuffer.Reset()
+		log.timestampsBuffer.Reset()
+	}()
+
+	// placeholder, preset batch length value is 0
+	batchLen := 0
+	if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(batchLen)); err != nil {
+		return errors.Wrap(err, "Write batch length error")
+	}
+	for seriesID, timestamps := range buffer.timestampMap {
+		// Generate seriesID binary
+		seriesIDBytes := seriesID.Marshal()
+		seriesIDBytesLen := len(seriesIDBytes)
+
+		// Generate timestamps compression binary
+		log.timestampsBuffer.Reset()
+		timestampWriter := encoding.NewWriter()
+		timestampEncoder := encoding.NewXOREncoder(timestampWriter)
+		timestampWriter.Reset(log.timestampsBuffer)
+		for _, timestamp := range timestamps {
+			timestampEncoder.Write(timeTouUnixNano(timestamp))
+		}
+		timestampWriter.Flush()
+		timestampsBytes := log.timestampsBuffer.Bytes()
+		timestampsBytesLen := len(timestampsBytes)
+
+		// Generate values compression binary
+		valuesBytes := snappy.Encode(nil, buffer.valueMap[seriesID])
+
+		// Write entry data
+		entryLen := seriesIDLength + seriesIDBytesLen + seriesCountLength + timestampsBinaryLength + timestampsBytesLen + len(valuesBytes)
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(entryLen)); err != nil {
+			return errors.Wrap(err, "Write entry length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int16(seriesIDBytesLen)); err != nil {
+			return errors.Wrap(err, "Write seriesID length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, seriesIDBytes); err != nil {
+			return errors.Wrap(err, "Write seriesID error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int32(len(timestamps))); err != nil {
+			return errors.Wrap(err, "Write series count error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int16(timestampsBytesLen)); err != nil {
+			return errors.Wrap(err, "Write timestamps length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, timestampsBytes); err != nil {
+			return errors.Wrap(err, "Write timestamps error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, valuesBytes); err != nil {
+			return errors.Wrap(err, "Write values error")
+		}
+	}
+	// Rewrite batch length
+	batchBytes := log.bytesBuffer.Bytes()
+	batchLen = len(batchBytes) - batchLength
+	rewriteInt64InBuf(batchBytes, int64(batchLen), 0, binary.LittleEndian)
+
+	return log.writeWorkSegment(batchBytes)
+}
+
+func (log *log) swapWorkSegment() (Segment, error) {
+	log.workSegmentMutex.Lock()
+	defer log.workSegmentMutex.Unlock()
+
+	if err := log.workSegment.file.Close(); err != nil {
+		return nil, errors.Wrap(err, "Close WAL segment error")
+	}
+
+	oldSegment := log.workSegment
+	// Create new segment.
+	segmentID := log.workSegment.segmentID + 1
+	segment := &segment{
+		segmentID: segmentID,
+		path:      filepath.Join(log.path, segmentName(uint64(segmentID))),
+	}
+	var err error
+	segment.file, err = os.OpenFile(segment.path, os.O_CREATE|os.O_RDWR|os.O_TRUNC, os.ModePerm)
+	if err != nil {
+		return nil, errors.Wrap(err, "Open WAL segment error")
+	}
+	log.workSegment = segment
+
+	return oldSegment, nil
+}
+
+func (log *log) writeWorkSegment(data []byte) error {
+	log.workSegmentMutex.Lock()
+	defer log.workSegmentMutex.Unlock()
+
+	// Write batch data to WAL segment file
+	if _, err := log.workSegment.file.Write(data); err != nil {
+		return errors.Wrap(err, "Write WAL segment file error, file: "+log.workSegment.path)
+	}
+	if err := log.workSegment.file.Sync(); err != nil {
+		log.logger.Warn().Msg("Sync WAL segment file to disk failed, file: " + log.workSegment.path)
+	}
+	return nil
+}
+
+func (log *log) load() error {
+	files, err := os.ReadDir(log.path)
+	if err != nil {
+		return errors.Wrap(err, "Can not read dir: "+log.path)
+	}
+	// Load all of WAL segments.
+	var workSegmentID SegmentID
+	log.segmentMap = make(map[SegmentID]*segment)
+	for _, file := range files {
+		name := file.Name()
+		segmentID, parsePathErr := strconv.ParseUint(name[3:19], 10, 64)
+		if parsePathErr != nil {
+			return errors.Wrap(parsePathErr, "Parse file name error, name: "+name)
+		}
+		if segmentID > uint64(workSegmentID) {
+			workSegmentID = SegmentID(segmentID)
+		}
+		segment := &segment{
+			segmentID: SegmentID(segmentID),
+			path:      filepath.Join(log.path, segmentName(segmentID)),
+		}
+		if err = segment.parseLogEntries(); err != nil {
+			return errors.Wrap(err, "Fail to parse log entries")
+		}
+		log.segmentMap[SegmentID(segmentID)] = segment
+
+		if log.logger.Debug().Enabled() {
+			log.logger.Debug().Msg("Loaded segment file: " + segment.path)
+		}
+	}
+
+	// If load first time.
+	if len(log.segmentMap) == 0 {
+		segment := &segment{
+			segmentID: 1,
+			path:      filepath.Join(log.path, segmentName(1)),
+		}
+		log.segmentMap[1] = segment
+		log.workSegment = segment
+	} else {
+		log.workSegment = log.segmentMap[workSegmentID]
+	}
+	log.workSegment.file, err = os.OpenFile(log.workSegment.path, os.O_CREATE|os.O_RDWR|os.O_APPEND, os.ModePerm)
+	if err != nil {
+		return errors.Wrap(err, "Open WAL segment error, file: "+log.workSegment.path)
+	}
+	return nil
+}
+
+func (segment *segment) GetSegmentID() SegmentID {
+	return segment.segmentID
+}
+
+func (segment *segment) GetLogEntries() []LogEntry {
+	return segment.logEntries
+}
+
+func (segment *segment) parseLogEntries() error {
+	segmentBytes, err := os.ReadFile(segment.path)
+	if err != nil {
+		return errors.Wrap(err, "Read WAL segment failed, path: "+segment.path)
+	}
+
+	var logEntries []LogEntry
+	var data []byte
+	var batchLen int64
+	var entryLen int64
+	var seriesIDLen int16
+	var seriesID common.SeriesIDV2
+	var seriesCount int32
+	var timestampsBinaryLen int16
+	var entryEndPos int
+	var bytesBuf *bytes.Buffer
+
+	oldPos := 0
+	pos := 0
+	parseNextBatchFlag := true
+
+	for {
+		if parseNextBatchFlag {
+			if len(segmentBytes) <= batchLength {
+				break
+			}
+			data = segmentBytes[pos : pos+batchLength]
+			bytesBuf = bytes.NewBuffer(data)
+			if err = binary.Read(bytesBuf, binary.LittleEndian, &batchLen); err != nil {
+				return errors.Wrap(err, "Read batch length fail to convert from bytes")
+			}
+			if len(segmentBytes) <= int(batchLen) {
+				break
+			}
+
+			pos += batchLength
+			oldPos = pos
+			parseNextBatchFlag = false
+		}
+
+		// Parse entryLength.
+		data = segmentBytes[pos : pos+entryLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &entryLen); err != nil {
+			return errors.Wrap(err, "Read entry length fail to convert from byte")
+		}
+		pos += entryLength
+
+		// Mark entry end-position
+		entryEndPos = pos + int(entryLen)
+		if len(segmentBytes) < entryEndPos {
+			break
+		}
+
+		// Parse seriesIDLen.
+		data = segmentBytes[pos : pos+seriesIDLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &seriesIDLen); err != nil {
+			return errors.Wrap(err, "Read seriesID length fail to convert from byte")
+		}
+		pos += seriesIDLength
+
+		// Parse seriesID.
+		data = segmentBytes[pos : pos+int(seriesIDLen)]
+		seriesID = common.ParseSeriesIDV2(data)
+		pos += int(seriesIDLen)
+
+		// Parse series count.
+		data = segmentBytes[pos : pos+seriesCountLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &seriesCount); err != nil {
+			return errors.Wrap(err, "Read series count fail to convert from byte")
+		}
+		pos += seriesCountLength
+
+		// Parse timestamps compression binary.
+		data = segmentBytes[pos : pos+timestampsBinaryLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &timestampsBinaryLen); err != nil {
+			return errors.Wrap(err, "Read timestamps compression binary length fail to convert from byte")
+		}
+		pos += timestampsBinaryLength
+		data = segmentBytes[pos : pos+int(timestampsBinaryLen)]
+		timestampReader := encoding.NewReader(bytes.NewReader(data))
+		timestampDecoder := encoding.NewXORDecoder(timestampReader)
+		var timestamps []time.Time
+		for i := 0; i < int(seriesCount); i++ {
+			if !timestampDecoder.Next() {
+				return errors.Wrap(err, "Timestamps length not match series count, index: "+strconv.Itoa(i))
+			}
+			timestamps = append(timestamps, unixNanoToTime(timestampDecoder.Value()))
+		}
+		pos += int(timestampsBinaryLen)
+
+		// Parse values compression binary.
+		data = segmentBytes[pos:entryEndPos]
+		if data, err = snappy.Decode(nil, data); err != nil {
+			return errors.Wrap(err, "Decode values compression binary fail to snappy decode")
+		}
+		values := parseValuesBinary(data)
+		if values.Len() != len(timestamps) {
+			return errors.New("Timestamps length and values length not match: " + strconv.Itoa(len(timestamps)) + " vs " + strconv.Itoa(values.Len()))
+		}
+		pos = entryEndPos
+
+		logEntry := &logEntry{
+			entryLength: entryLen,
+			seriesID:    seriesID,
+			count:       seriesCount,
+			timestamps:  timestamps,
+			values:      values,
+		}
+		logEntries = append(logEntries, logEntry)
+
+		if pos == len(segmentBytes) {
+			break
+		}
+		if pos-oldPos == int(batchLen) {
+			parseNextBatchFlag = true
+		}
+	}
+	segment.logEntries = logEntries
+	return nil
+}
+
+func (logEntry *logEntry) GetSeriesID() common.SeriesIDV2 {
+	return logEntry.seriesID
+}
+
+func (logEntry *logEntry) GetTimestamps() []time.Time {
+	return logEntry.timestamps
+}
+
+func (logEntry *logEntry) GetValues() *list.List {
+	return logEntry.values
+}
+
+func (buffer *buffer) write(request logRequest) {
+	seriesID := request.seriesID
+	buffer.timestampMap[seriesID] = append(buffer.timestampMap[seriesID], request.timestamp)
+
+	// Value item: binary-length(2-bytes) + binary data(n-bytes)
+	binaryLength := int16ToBytes(int16(len(request.data)))
+	buffer.valueMap[seriesID] = append(buffer.valueMap[seriesID], binaryLength...)
+	buffer.valueMap[seriesID] = append(buffer.valueMap[seriesID], request.data...)
+
+	buffer.callbackMap[seriesID] = append(buffer.callbackMap[seriesID], request.callback)
+	buffer.count++
+}
+
+func (buffer *buffer) notifyRequests(err error) {
+	for seriesID, callbacks := range buffer.callbackMap {
+		timestamps := buffer.timestampMap[seriesID]
+		values := buffer.valueMap[seriesID]
+		valuePos := 0
+		var valueItem []byte
+		for index, callback := range callbacks {
+			valuePos, valueItem = readValuesBinary(values, valuePos, valuesBinaryLength)
+			tryCallback(func() {
+				callback(seriesID, timestamps[index], valueItem, err)
+			})
+		}
+	}
+}
+
+func segmentName(index uint64) string {
+	return fmt.Sprintf("%v%016x%v", segmentNamePrefix, index, segmentNameSuffix)
+}
+
+func tryCallback(callback func()) {
+	defer func() {
+		if err := recover(); err != nil {
+			fmt.Printf("Run callback error is %v\n", err)
+		}
+	}()
+	callback()
+}
+
+func parseValuesBinary(binary []byte) *list.List {
+	values := list.New()
+	position := 0
+	for {
+		nextPosition, value := readValuesBinary(binary, position, valuesBinaryLength)
+		if value == nil {
+			break
+		}
+		values.PushBack(value)
+		position = nextPosition
+	}
+	return values
+}
+
+func readValuesBinary(raw []byte, position int, offsetLen int) (int, []byte) {
+	if position == len(raw) {
+		return position, nil
+	}
+
+	data := raw[position : position+offsetLen]
+	binaryLen := bytesToInt16(data)
+	position += offsetLen
+
+	data = raw[position : position+int(binaryLen)]
+	position += int(binaryLen)
+	return position, data
+}
+
+func rewriteInt64InBuf(buf []byte, value int64, offset int, order binary.ByteOrder) {

Review Comment:
   the `offset` is always 0



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hailin0 commented on pull request #261: Implement Write-ahead Logging

Posted by "hailin0 (via GitHub)" <gi...@apache.org>.
hailin0 commented on PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#issuecomment-1605893844

   Format reference after buffer is flush to file
   
   <img width="422" alt="image" src="https://github.com/apache/skywalking-banyandb/assets/14371345/a54c026c-e037-4d6b-9477-26745ce73c46">
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hailin0 commented on a diff in pull request #261: Implement Write-ahead Logging

Posted by "hailin0 (via GitHub)" <gi...@apache.org>.
hailin0 commented on code in PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#discussion_r1249552014


##########
pkg/wal/wal.go:
##########
@@ -42,19 +75,759 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// log implements the WAL interface.
+type log struct {
+	entryCloser      *run.Closer
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentMap       map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+	writeWaitGroup   sync.WaitGroup
+	flushWaitGroup   sync.WaitGroup
+	workSegmentMutex sync.Mutex
+	segmentMapMutex  sync.RWMutex
+	closerOnce       sync.Once
+}
+
+type segment struct {
+	file       *os.File
+	path       string
+	logEntries []LogEntry
+	segmentID  SegmentID
+}
+
+type logRequest struct {
+	seriesID  common.SeriesIDV2
+	timestamp time.Time
+	callback  func(common.SeriesIDV2, time.Time, []byte, error)
+	data      []byte
+}
+
+type logEntry struct {
+	timestamps  []time.Time
+	values      *list.List
+	seriesID    common.SeriesIDV2
+	entryLength int64
+	count       int32
+}
+
+type buffer struct {
+	timestampMap map[common.SeriesIDV2][]time.Time
+	valueMap     map[common.SeriesIDV2][]byte
+	callbackMap  map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)
+	count        int
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	walOptions := DefaultOptions
+	if options != nil {
+		fileSize := options.FileSize
+		if fileSize <= 0 {
+			fileSize = DefaultOptions.FileSize
+		}
+		bufferSize := options.BufferSize
+		if bufferSize <= 0 {
+			bufferSize = DefaultOptions.BufferSize
+		}
+		bufferBatchInterval := options.BufferBatchInterval
+		if bufferBatchInterval <= 0 {
+			bufferBatchInterval = DefaultOptions.BufferBatchInterval
+		}
+		walOptions = &Options{
+			FileSize:            fileSize,
+			BufferSize:          bufferSize,
+			BufferBatchInterval: bufferBatchInterval,
+		}
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path: "+path)
+	}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	log := &log{
+		path:             path,
+		options:          *walOptions,
+		logger:           logger.GetLogger(moduleName),
+		writeChannel:     make(chan logRequest),
+		flushChannel:     make(chan buffer, walOptions.FlushQueueSize),
+		bytesBuffer:      bytes.NewBuffer([]byte{}),
+		timestampsBuffer: bytes.NewBuffer([]byte{}),
+		entryCloser:      run.NewCloser(1),
+		buffer: buffer{
+			timestampMap: make(map[common.SeriesIDV2][]time.Time),
+			valueMap:     make(map[common.SeriesIDV2][]byte),
+			callbackMap:  make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)),
+			count:        0,
+		},
+	}
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+
+	log.writeWaitGroup.Add(1)
+	log.flushWaitGroup.Add(1)
+	log.start()
+
+	log.logger.Info().Str("path", path).Msg("WAL has be initialized")
+	return log, nil
+}
+
+// Write a logging entity.
+// It will return immediately when the data is written in the buffer,
+// The callback function will be called when the entity is flushed on the persistent storage.
+func (log *log) Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error)) {
+	if !log.entryCloser.AddRunning() {
+		return
+	}
+	defer log.entryCloser.Done()
+
+	log.writeChannel <- logRequest{
+		seriesID:  seriesID,
+		timestamp: timestamp,
+		data:      data,
+		callback:  callback,
+	}
+}
+
+// Read specified segment by SegmentID.
+func (log *log) Read(segmentID SegmentID) (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segment := log.segmentMap[segmentID]
+	return segment, nil
+}
+
+// ReadAllSegments reads all segments sorted by their creation time in ascending order.
+func (log *log) ReadAllSegments() ([]Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segments := make([]Segment, 0)
+	for _, segment := range log.segmentMap {
+		segments = append(segments, segment)
+	}
+	return segments, nil
+}
+
+// Rotate closes the open segment and opens a new one, returning the closed segment details.
+func (log *log) Rotate() (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	oldWorkSegment, err := log.swapWorkSegment()
+	if err != nil {
+		return nil, err
+	}
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Update segment information.
+	workSegment := log.workSegment
+	log.segmentMap[workSegment.segmentID] = workSegment
+	return oldWorkSegment, nil
+}
+
+// Delete the specified segment.
+func (log *log) Delete(segmentID SegmentID) error {
+	if !log.entryCloser.AddRunning() {
+		return errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Segment which will be deleted must be closed.
+	if segmentID == log.workSegment.segmentID {
+		return errors.New("Can not delete the segment which is working")
+	}
+
+	err := os.Remove(log.segmentMap[segmentID].path)
+	if err != nil {
+		return errors.Wrap(err, "Delete WAL segment error")
+	}
+	delete(log.segmentMap, segmentID)
+	return nil
+}
+
+// Close all of segments and stop WAL work.
+func (log *log) Close() error {
+	log.closerOnce.Do(func() {
+		log.logger.Info().Msg("Closing WAL...")
+
+		log.entryCloser.Done()
+		log.entryCloser.CloseThenWait()
+
+		close(log.writeChannel)
+		log.writeWaitGroup.Wait()
+
+		close(log.flushChannel)
+		log.flushWaitGroup.Wait()
+
+		if err := log.flushBuffer(log.buffer); err != nil {
+			log.logger.Err(err).Msg("Flushing buffer failed")
+		}
+		if err := log.workSegment.file.Close(); err != nil {
+			log.logger.Err(err).Msg("Close work segment file error")
+		}
+		log.logger.Info().Msg("Closed WAL")
+	})
+	return nil
+}
+
+func (log *log) start() {
+	go func() {
+		log.logger.Info().Msg("Start batch task...")
+
+		defer log.writeWaitGroup.Done()
+
+		bufferVolume := 0
+		for {
+			timer := time.NewTimer(log.options.BufferBatchInterval)
+			select {
+			case request, chOpen := <-log.writeChannel:
+				if !chOpen {
+					log.logger.Info().Msg("Stop batch task when write-channel closed!")
+					return
+				}
+
+				log.buffer.write(request)
+				if log.logger.Debug().Enabled() {
+					log.logger.Debug().Msg("Write request to buffer. elements: " + strconv.Itoa(log.buffer.count))
+				}
+
+				bufferVolume += len(request.seriesID.Marshal()) + timestampVolumeLength + len(request.data)
+				if bufferVolume > log.options.BufferSize {
+					log.triggerFlushing()
+					bufferVolume = 0
+				}
+				continue
+			case <-timer.C:
+				if bufferVolume == 0 {
+					continue
+				}
+				log.triggerFlushing()
+				bufferVolume = 0
+				continue
+			}
+		}
+	}()
+
+	go func() {
+		log.logger.Info().Msg("Start flush task...")
+
+		defer log.flushWaitGroup.Done()
+
+		for batch := range log.flushChannel {
+			startTime := time.Now()
+
+			var err error
+			for i := 0; i < maxRetries; i++ {
+				if err = log.flushBuffer(batch); err != nil {
+					log.logger.Err(err).Msg("Flushing buffer failed. Retrying...")
+					time.Sleep(100 * time.Millisecond)
+					continue
+				}
+				break
+			}
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Flushed buffer to WAL file. elements: " +
+					strconv.Itoa(batch.count) + ", cost: " + time.Since(startTime).String())
+			}
+
+			batch.notifyRequests(err)
+		}
+		log.logger.Info().Msg("Stop flush task when flush-channel closed!")
+	}()
+
+	log.logger.Info().Msg("Started WAL")
+}
+
+func (log *log) triggerFlushing() {
+	for {
+		select {
+		case log.flushChannel <- log.buffer:
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Send buffer to flush-channel. elements: " + strconv.Itoa(log.buffer.count))
+			}
+			log.newBuffer()
+			return
+		default:
+		}
+		time.Sleep(10 * time.Millisecond)
+	}
+}
+
+func (log *log) newBuffer() {
+	log.buffer.timestampMap = make(map[common.SeriesIDV2][]time.Time)
+	log.buffer.valueMap = make(map[common.SeriesIDV2][]byte)
+	log.buffer.callbackMap = make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error))
+	log.buffer.count = 0
+}
+
+func (log *log) flushBuffer(buffer buffer) error {
+	if buffer.count == 0 {
+		return nil
+	}
+
+	defer func() {
+		log.bytesBuffer.Reset()
+		log.timestampsBuffer.Reset()
+	}()
+
+	// placeholder, preset batch length value is 0
+	batchLen := 0
+	if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(batchLen)); err != nil {
+		return errors.Wrap(err, "Write batch length error")
+	}
+	for seriesID, timestamps := range buffer.timestampMap {
+		// Generate seriesID binary
+		seriesIDBytes := seriesID.Marshal()
+		seriesIDBytesLen := len(seriesIDBytes)
+
+		// Generate timestamps compression binary
+		log.timestampsBuffer.Reset()
+		timestampWriter := encoding.NewWriter()
+		timestampEncoder := encoding.NewXOREncoder(timestampWriter)
+		timestampWriter.Reset(log.timestampsBuffer)
+		for _, timestamp := range timestamps {
+			timestampEncoder.Write(timeTouUnixNano(timestamp))
+		}
+		timestampWriter.Flush()
+		timestampsBytes := log.timestampsBuffer.Bytes()
+		timestampsBytesLen := len(timestampsBytes)
+
+		// Generate values compression binary
+		valuesBytes := snappy.Encode(nil, buffer.valueMap[seriesID])
+
+		// Write entry data
+		entryLen := seriesIDLength + seriesIDBytesLen + seriesCountLength + timestampsBinaryLength + timestampsBytesLen + len(valuesBytes)
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(entryLen)); err != nil {
+			return errors.Wrap(err, "Write entry length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int16(seriesIDBytesLen)); err != nil {
+			return errors.Wrap(err, "Write seriesID length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, seriesIDBytes); err != nil {
+			return errors.Wrap(err, "Write seriesID error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int32(len(timestamps))); err != nil {
+			return errors.Wrap(err, "Write series count error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int16(timestampsBytesLen)); err != nil {
+			return errors.Wrap(err, "Write timestamps length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, timestampsBytes); err != nil {
+			return errors.Wrap(err, "Write timestamps error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, valuesBytes); err != nil {
+			return errors.Wrap(err, "Write values error")
+		}
+	}
+	// Rewrite batch length
+	batchBytes := log.bytesBuffer.Bytes()
+	batchLen = len(batchBytes) - batchLength
+	rewriteInt64InBuf(batchBytes, int64(batchLen), 0, binary.LittleEndian)
+
+	return log.writeWorkSegment(batchBytes)
+}
+
+func (log *log) swapWorkSegment() (Segment, error) {
+	log.workSegmentMutex.Lock()
+	defer log.workSegmentMutex.Unlock()
+
+	if err := log.workSegment.file.Close(); err != nil {
+		return nil, errors.Wrap(err, "Close WAL segment error")
+	}
+
+	oldSegment := log.workSegment
+	// Create new segment.
+	segmentID := log.workSegment.segmentID + 1

Review Comment:
   fixed



##########
pkg/wal/wal.go:
##########
@@ -42,19 +75,759 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// log implements the WAL interface.
+type log struct {
+	entryCloser      *run.Closer
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentMap       map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+	writeWaitGroup   sync.WaitGroup
+	flushWaitGroup   sync.WaitGroup
+	workSegmentMutex sync.Mutex
+	segmentMapMutex  sync.RWMutex
+	closerOnce       sync.Once
+}
+
+type segment struct {
+	file       *os.File
+	path       string
+	logEntries []LogEntry
+	segmentID  SegmentID
+}
+
+type logRequest struct {
+	seriesID  common.SeriesIDV2
+	timestamp time.Time
+	callback  func(common.SeriesIDV2, time.Time, []byte, error)
+	data      []byte
+}
+
+type logEntry struct {
+	timestamps  []time.Time
+	values      *list.List
+	seriesID    common.SeriesIDV2
+	entryLength int64
+	count       int32
+}
+
+type buffer struct {
+	timestampMap map[common.SeriesIDV2][]time.Time
+	valueMap     map[common.SeriesIDV2][]byte
+	callbackMap  map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)
+	count        int
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	walOptions := DefaultOptions
+	if options != nil {
+		fileSize := options.FileSize
+		if fileSize <= 0 {
+			fileSize = DefaultOptions.FileSize
+		}
+		bufferSize := options.BufferSize
+		if bufferSize <= 0 {
+			bufferSize = DefaultOptions.BufferSize
+		}
+		bufferBatchInterval := options.BufferBatchInterval
+		if bufferBatchInterval <= 0 {
+			bufferBatchInterval = DefaultOptions.BufferBatchInterval
+		}
+		walOptions = &Options{
+			FileSize:            fileSize,
+			BufferSize:          bufferSize,
+			BufferBatchInterval: bufferBatchInterval,
+		}
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path: "+path)
+	}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	log := &log{
+		path:             path,
+		options:          *walOptions,
+		logger:           logger.GetLogger(moduleName),
+		writeChannel:     make(chan logRequest),
+		flushChannel:     make(chan buffer, walOptions.FlushQueueSize),
+		bytesBuffer:      bytes.NewBuffer([]byte{}),
+		timestampsBuffer: bytes.NewBuffer([]byte{}),
+		entryCloser:      run.NewCloser(1),
+		buffer: buffer{
+			timestampMap: make(map[common.SeriesIDV2][]time.Time),
+			valueMap:     make(map[common.SeriesIDV2][]byte),
+			callbackMap:  make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)),
+			count:        0,
+		},
+	}
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+
+	log.writeWaitGroup.Add(1)
+	log.flushWaitGroup.Add(1)
+	log.start()
+
+	log.logger.Info().Str("path", path).Msg("WAL has be initialized")
+	return log, nil
+}
+
+// Write a logging entity.
+// It will return immediately when the data is written in the buffer,
+// The callback function will be called when the entity is flushed on the persistent storage.
+func (log *log) Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error)) {
+	if !log.entryCloser.AddRunning() {
+		return
+	}
+	defer log.entryCloser.Done()
+
+	log.writeChannel <- logRequest{
+		seriesID:  seriesID,
+		timestamp: timestamp,
+		data:      data,
+		callback:  callback,
+	}
+}
+
+// Read specified segment by SegmentID.
+func (log *log) Read(segmentID SegmentID) (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segment := log.segmentMap[segmentID]
+	return segment, nil
+}
+
+// ReadAllSegments reads all segments sorted by their creation time in ascending order.
+func (log *log) ReadAllSegments() ([]Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segments := make([]Segment, 0)
+	for _, segment := range log.segmentMap {
+		segments = append(segments, segment)
+	}
+	return segments, nil
+}
+
+// Rotate closes the open segment and opens a new one, returning the closed segment details.
+func (log *log) Rotate() (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	oldWorkSegment, err := log.swapWorkSegment()
+	if err != nil {
+		return nil, err
+	}
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Update segment information.
+	workSegment := log.workSegment
+	log.segmentMap[workSegment.segmentID] = workSegment
+	return oldWorkSegment, nil
+}
+
+// Delete the specified segment.
+func (log *log) Delete(segmentID SegmentID) error {
+	if !log.entryCloser.AddRunning() {
+		return errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Segment which will be deleted must be closed.
+	if segmentID == log.workSegment.segmentID {
+		return errors.New("Can not delete the segment which is working")
+	}
+
+	err := os.Remove(log.segmentMap[segmentID].path)
+	if err != nil {
+		return errors.Wrap(err, "Delete WAL segment error")
+	}
+	delete(log.segmentMap, segmentID)
+	return nil
+}
+
+// Close all of segments and stop WAL work.
+func (log *log) Close() error {
+	log.closerOnce.Do(func() {
+		log.logger.Info().Msg("Closing WAL...")
+
+		log.entryCloser.Done()
+		log.entryCloser.CloseThenWait()
+
+		close(log.writeChannel)
+		log.writeWaitGroup.Wait()
+
+		close(log.flushChannel)
+		log.flushWaitGroup.Wait()
+
+		if err := log.flushBuffer(log.buffer); err != nil {
+			log.logger.Err(err).Msg("Flushing buffer failed")
+		}
+		if err := log.workSegment.file.Close(); err != nil {
+			log.logger.Err(err).Msg("Close work segment file error")
+		}
+		log.logger.Info().Msg("Closed WAL")
+	})
+	return nil
+}
+
+func (log *log) start() {
+	go func() {
+		log.logger.Info().Msg("Start batch task...")
+
+		defer log.writeWaitGroup.Done()
+
+		bufferVolume := 0
+		for {
+			timer := time.NewTimer(log.options.BufferBatchInterval)
+			select {
+			case request, chOpen := <-log.writeChannel:
+				if !chOpen {
+					log.logger.Info().Msg("Stop batch task when write-channel closed!")
+					return
+				}
+
+				log.buffer.write(request)
+				if log.logger.Debug().Enabled() {
+					log.logger.Debug().Msg("Write request to buffer. elements: " + strconv.Itoa(log.buffer.count))
+				}
+
+				bufferVolume += len(request.seriesID.Marshal()) + timestampVolumeLength + len(request.data)
+				if bufferVolume > log.options.BufferSize {
+					log.triggerFlushing()
+					bufferVolume = 0
+				}
+				continue
+			case <-timer.C:
+				if bufferVolume == 0 {
+					continue
+				}
+				log.triggerFlushing()
+				bufferVolume = 0
+				continue
+			}
+		}
+	}()
+
+	go func() {
+		log.logger.Info().Msg("Start flush task...")
+
+		defer log.flushWaitGroup.Done()
+
+		for batch := range log.flushChannel {
+			startTime := time.Now()
+
+			var err error
+			for i := 0; i < maxRetries; i++ {
+				if err = log.flushBuffer(batch); err != nil {
+					log.logger.Err(err).Msg("Flushing buffer failed. Retrying...")
+					time.Sleep(100 * time.Millisecond)
+					continue
+				}
+				break
+			}
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Flushed buffer to WAL file. elements: " +
+					strconv.Itoa(batch.count) + ", cost: " + time.Since(startTime).String())
+			}
+
+			batch.notifyRequests(err)
+		}
+		log.logger.Info().Msg("Stop flush task when flush-channel closed!")
+	}()
+
+	log.logger.Info().Msg("Started WAL")
+}
+
+func (log *log) triggerFlushing() {
+	for {
+		select {
+		case log.flushChannel <- log.buffer:
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Send buffer to flush-channel. elements: " + strconv.Itoa(log.buffer.count))
+			}
+			log.newBuffer()
+			return
+		default:
+		}
+		time.Sleep(10 * time.Millisecond)
+	}
+}
+
+func (log *log) newBuffer() {
+	log.buffer.timestampMap = make(map[common.SeriesIDV2][]time.Time)
+	log.buffer.valueMap = make(map[common.SeriesIDV2][]byte)
+	log.buffer.callbackMap = make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error))
+	log.buffer.count = 0
+}
+
+func (log *log) flushBuffer(buffer buffer) error {
+	if buffer.count == 0 {
+		return nil
+	}
+
+	defer func() {
+		log.bytesBuffer.Reset()
+		log.timestampsBuffer.Reset()
+	}()
+
+	// placeholder, preset batch length value is 0
+	batchLen := 0
+	if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(batchLen)); err != nil {
+		return errors.Wrap(err, "Write batch length error")
+	}
+	for seriesID, timestamps := range buffer.timestampMap {
+		// Generate seriesID binary
+		seriesIDBytes := seriesID.Marshal()
+		seriesIDBytesLen := len(seriesIDBytes)
+
+		// Generate timestamps compression binary
+		log.timestampsBuffer.Reset()
+		timestampWriter := encoding.NewWriter()
+		timestampEncoder := encoding.NewXOREncoder(timestampWriter)
+		timestampWriter.Reset(log.timestampsBuffer)
+		for _, timestamp := range timestamps {
+			timestampEncoder.Write(timeTouUnixNano(timestamp))
+		}
+		timestampWriter.Flush()
+		timestampsBytes := log.timestampsBuffer.Bytes()
+		timestampsBytesLen := len(timestampsBytes)
+
+		// Generate values compression binary
+		valuesBytes := snappy.Encode(nil, buffer.valueMap[seriesID])
+
+		// Write entry data
+		entryLen := seriesIDLength + seriesIDBytesLen + seriesCountLength + timestampsBinaryLength + timestampsBytesLen + len(valuesBytes)
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(entryLen)); err != nil {

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hailin0 commented on pull request #261: Implement Write-ahead Logging

Posted by "hailin0 (via GitHub)" <gi...@apache.org>.
hailin0 commented on PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#issuecomment-1637136436

   <img width="1186" alt="image" src="https://github.com/apache/skywalking-banyandb/assets/14371345/2f755d76-db1e-451d-adb2-f296c41e21ff">
   <img width="570" alt="image" src="https://github.com/apache/skywalking-banyandb/assets/14371345/3f2f5745-04fc-4692-9e98-080c01faa9f8">
   <img width="1511" alt="image" src="https://github.com/apache/skywalking-banyandb/assets/14371345/0382f1fb-0519-4285-b938-15dcf71cf8a1">
   <img width="1511" alt="image" src="https://github.com/apache/skywalking-banyandb/assets/14371345/54c0dd5b-4eeb-4dd3-918a-ebd4ffb6fa26">
   <img width="1511" alt="image" src="https://github.com/apache/skywalking-banyandb/assets/14371345/271ec812-18f3-4fe0-b17e-cecabfa1723c">
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hailin0 commented on a diff in pull request #261: Implement Write-ahead Logging

Posted by "hailin0 (via GitHub)" <gi...@apache.org>.
hailin0 commented on code in PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#discussion_r1254490712


##########
api/common/id.go:
##########
@@ -40,6 +40,27 @@ func (s SeriesID) Marshal() []byte {
 	return convert.Uint64ToBytes(uint64(s))
 }
 
+// SeriesIDV2 identities a series in a shard.

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] codecov-commenter commented on pull request #261: Implement Write-ahead Logging

Posted by "codecov-commenter (via GitHub)" <gi...@apache.org>.
codecov-commenter commented on PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#issuecomment-1475098051

   ## [Codecov](https://codecov.io/gh/apache/skywalking-banyandb/pull/261?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#261](https://codecov.io/gh/apache/skywalking-banyandb/pull/261?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (d6030fe) into [main](https://codecov.io/gh/apache/skywalking-banyandb/commit/82905ba08c9987bf13eb241bcd3dbca5f70cd134?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (82905ba) will **decrease** coverage by `0.61%`.
   > The diff coverage is `24.32%`.
   
   ```diff
   @@            Coverage Diff             @@
   ##             main     #261      +/-   ##
   ==========================================
   - Coverage   45.87%   45.26%   -0.61%     
   ==========================================
     Files          87       88       +1     
     Lines        8898     9157     +259     
   ==========================================
   + Hits         4082     4145      +63     
   - Misses       4432     4615     +183     
   - Partials      384      397      +13     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking-banyandb/pull/261?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [pkg/wal/wal.go](https://codecov.io/gh/apache/skywalking-banyandb/pull/261?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGtnL3dhbC93YWwuZ28=) | `24.32% <24.32%> (ø)` | |
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hailin0 commented on pull request #261: Implement Write-ahead Logging

Posted by "hailin0 (via GitHub)" <gi...@apache.org>.
hailin0 commented on PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#issuecomment-1556175241

   thanks, i am doing


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] HHoflittlefish777 commented on a diff in pull request #261: Implement Write-ahead Logging

Posted by "HHoflittlefish777 (via GitHub)" <gi...@apache.org>.
HHoflittlefish777 commented on code in PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#discussion_r1147426826


##########
pkg/wal/wal.go:
##########
@@ -43,18 +118,364 @@ type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
 	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) error

Review Comment:
   I solve this problem, please review again.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hailin0 commented on a diff in pull request #261: Implement Write-ahead Logging

Posted by "hailin0 (via GitHub)" <gi...@apache.org>.
hailin0 commented on code in PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#discussion_r1246012021


##########
pkg/wal/wal.go:
##########
@@ -42,19 +72,642 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// Log implements the WAL interface.
+type Log struct {
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentIndexMap  map[SegmentID]*segment
+	workSegment      *segment

Review Comment:
   fixed



##########
pkg/wal/wal.go:
##########
@@ -42,19 +72,642 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// Log implements the WAL interface.
+type Log struct {
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentIndexMap  map[SegmentID]*segment

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] wu-sheng commented on pull request #261: Implement Write-ahead Logging

Posted by "wu-sheng (via GitHub)" <gi...@apache.org>.
wu-sheng commented on PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#issuecomment-1537263799

   This PR seems not to update for weeks. Is this still being processed?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] HHoflittlefish777 commented on pull request #261: Implement Write-ahead Logging

Posted by "HHoflittlefish777 (via GitHub)" <gi...@apache.org>.
HHoflittlefish777 commented on PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#issuecomment-1553173877

   That is fine, Looking forward to your further contributions to BanyanDB @hailin0.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hanahmily commented on a diff in pull request #261: Implement Write-ahead Logging

Posted by "hanahmily (via GitHub)" <gi...@apache.org>.
hanahmily commented on code in PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#discussion_r1145552462


##########
pkg/wal/wal.go:
##########
@@ -43,18 +118,364 @@ type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
 	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) error

Review Comment:
   I found a gap in implementing the Buffer. The seriesID's type should be `[]byte` instead of `common.SeriesID`. 
   
   The reason is that the key in underlying storage comprises the seriesID and tagFamily as https://github.com/apache/skywalking-banyandb/blob/main/banyand/tsdb/series_write.go#L194 indicates.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hanahmily commented on pull request #261: Implement Write-ahead Logging

Posted by "hanahmily (via GitHub)" <gi...@apache.org>.
hanahmily commented on PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#issuecomment-1579819646

   @hailin0 any update?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hailin0 commented on a diff in pull request #261: Implement Write-ahead Logging

Posted by "hailin0 (via GitHub)" <gi...@apache.org>.
hailin0 commented on code in PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#discussion_r1249551940


##########
pkg/wal/wal.go:
##########
@@ -42,19 +75,759 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// log implements the WAL interface.
+type log struct {
+	entryCloser      *run.Closer
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentMap       map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+	writeWaitGroup   sync.WaitGroup
+	flushWaitGroup   sync.WaitGroup
+	workSegmentMutex sync.Mutex
+	segmentMapMutex  sync.RWMutex
+	closerOnce       sync.Once
+}
+
+type segment struct {
+	file       *os.File
+	path       string
+	logEntries []LogEntry
+	segmentID  SegmentID
+}
+
+type logRequest struct {
+	seriesID  common.SeriesIDV2
+	timestamp time.Time
+	callback  func(common.SeriesIDV2, time.Time, []byte, error)
+	data      []byte
+}
+
+type logEntry struct {
+	timestamps  []time.Time
+	values      *list.List
+	seriesID    common.SeriesIDV2
+	entryLength int64
+	count       int32
+}
+
+type buffer struct {
+	timestampMap map[common.SeriesIDV2][]time.Time
+	valueMap     map[common.SeriesIDV2][]byte
+	callbackMap  map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)
+	count        int
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	walOptions := DefaultOptions
+	if options != nil {
+		fileSize := options.FileSize
+		if fileSize <= 0 {
+			fileSize = DefaultOptions.FileSize
+		}
+		bufferSize := options.BufferSize
+		if bufferSize <= 0 {
+			bufferSize = DefaultOptions.BufferSize
+		}
+		bufferBatchInterval := options.BufferBatchInterval
+		if bufferBatchInterval <= 0 {
+			bufferBatchInterval = DefaultOptions.BufferBatchInterval
+		}
+		walOptions = &Options{
+			FileSize:            fileSize,
+			BufferSize:          bufferSize,
+			BufferBatchInterval: bufferBatchInterval,
+		}
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path: "+path)
+	}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	log := &log{
+		path:             path,
+		options:          *walOptions,
+		logger:           logger.GetLogger(moduleName),
+		writeChannel:     make(chan logRequest),
+		flushChannel:     make(chan buffer, walOptions.FlushQueueSize),
+		bytesBuffer:      bytes.NewBuffer([]byte{}),
+		timestampsBuffer: bytes.NewBuffer([]byte{}),
+		entryCloser:      run.NewCloser(1),
+		buffer: buffer{
+			timestampMap: make(map[common.SeriesIDV2][]time.Time),
+			valueMap:     make(map[common.SeriesIDV2][]byte),
+			callbackMap:  make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)),
+			count:        0,
+		},
+	}
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+
+	log.writeWaitGroup.Add(1)
+	log.flushWaitGroup.Add(1)
+	log.start()
+
+	log.logger.Info().Str("path", path).Msg("WAL has be initialized")
+	return log, nil
+}
+
+// Write a logging entity.
+// It will return immediately when the data is written in the buffer,
+// The callback function will be called when the entity is flushed on the persistent storage.
+func (log *log) Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error)) {
+	if !log.entryCloser.AddRunning() {
+		return
+	}
+	defer log.entryCloser.Done()
+
+	log.writeChannel <- logRequest{
+		seriesID:  seriesID,
+		timestamp: timestamp,
+		data:      data,
+		callback:  callback,
+	}
+}
+
+// Read specified segment by SegmentID.
+func (log *log) Read(segmentID SegmentID) (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segment := log.segmentMap[segmentID]
+	return segment, nil
+}
+
+// ReadAllSegments reads all segments sorted by their creation time in ascending order.
+func (log *log) ReadAllSegments() ([]Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segments := make([]Segment, 0)
+	for _, segment := range log.segmentMap {
+		segments = append(segments, segment)
+	}
+	return segments, nil
+}
+
+// Rotate closes the open segment and opens a new one, returning the closed segment details.
+func (log *log) Rotate() (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	oldWorkSegment, err := log.swapWorkSegment()
+	if err != nil {
+		return nil, err
+	}
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Update segment information.
+	workSegment := log.workSegment
+	log.segmentMap[workSegment.segmentID] = workSegment
+	return oldWorkSegment, nil
+}
+
+// Delete the specified segment.
+func (log *log) Delete(segmentID SegmentID) error {
+	if !log.entryCloser.AddRunning() {
+		return errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Segment which will be deleted must be closed.
+	if segmentID == log.workSegment.segmentID {
+		return errors.New("Can not delete the segment which is working")
+	}
+
+	err := os.Remove(log.segmentMap[segmentID].path)
+	if err != nil {
+		return errors.Wrap(err, "Delete WAL segment error")
+	}
+	delete(log.segmentMap, segmentID)
+	return nil
+}
+
+// Close all of segments and stop WAL work.
+func (log *log) Close() error {
+	log.closerOnce.Do(func() {
+		log.logger.Info().Msg("Closing WAL...")
+
+		log.entryCloser.Done()
+		log.entryCloser.CloseThenWait()
+
+		close(log.writeChannel)
+		log.writeWaitGroup.Wait()
+
+		close(log.flushChannel)
+		log.flushWaitGroup.Wait()
+
+		if err := log.flushBuffer(log.buffer); err != nil {
+			log.logger.Err(err).Msg("Flushing buffer failed")
+		}
+		if err := log.workSegment.file.Close(); err != nil {
+			log.logger.Err(err).Msg("Close work segment file error")
+		}
+		log.logger.Info().Msg("Closed WAL")
+	})
+	return nil
+}
+
+func (log *log) start() {
+	go func() {
+		log.logger.Info().Msg("Start batch task...")
+
+		defer log.writeWaitGroup.Done()
+
+		bufferVolume := 0
+		for {
+			timer := time.NewTimer(log.options.BufferBatchInterval)
+			select {
+			case request, chOpen := <-log.writeChannel:
+				if !chOpen {
+					log.logger.Info().Msg("Stop batch task when write-channel closed!")
+					return
+				}
+
+				log.buffer.write(request)
+				if log.logger.Debug().Enabled() {
+					log.logger.Debug().Msg("Write request to buffer. elements: " + strconv.Itoa(log.buffer.count))
+				}
+
+				bufferVolume += len(request.seriesID.Marshal()) + timestampVolumeLength + len(request.data)
+				if bufferVolume > log.options.BufferSize {
+					log.triggerFlushing()
+					bufferVolume = 0
+				}
+				continue
+			case <-timer.C:
+				if bufferVolume == 0 {
+					continue
+				}
+				log.triggerFlushing()
+				bufferVolume = 0
+				continue

Review Comment:
   fixed



##########
pkg/wal/wal.go:
##########
@@ -42,19 +75,759 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// log implements the WAL interface.
+type log struct {
+	entryCloser      *run.Closer
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentMap       map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+	writeWaitGroup   sync.WaitGroup
+	flushWaitGroup   sync.WaitGroup
+	workSegmentMutex sync.Mutex
+	segmentMapMutex  sync.RWMutex
+	closerOnce       sync.Once
+}
+
+type segment struct {
+	file       *os.File
+	path       string
+	logEntries []LogEntry
+	segmentID  SegmentID
+}
+
+type logRequest struct {
+	seriesID  common.SeriesIDV2
+	timestamp time.Time
+	callback  func(common.SeriesIDV2, time.Time, []byte, error)
+	data      []byte
+}
+
+type logEntry struct {
+	timestamps  []time.Time
+	values      *list.List
+	seriesID    common.SeriesIDV2
+	entryLength int64
+	count       int32
+}
+
+type buffer struct {
+	timestampMap map[common.SeriesIDV2][]time.Time
+	valueMap     map[common.SeriesIDV2][]byte
+	callbackMap  map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)
+	count        int
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	walOptions := DefaultOptions
+	if options != nil {
+		fileSize := options.FileSize
+		if fileSize <= 0 {
+			fileSize = DefaultOptions.FileSize
+		}
+		bufferSize := options.BufferSize
+		if bufferSize <= 0 {
+			bufferSize = DefaultOptions.BufferSize
+		}
+		bufferBatchInterval := options.BufferBatchInterval
+		if bufferBatchInterval <= 0 {
+			bufferBatchInterval = DefaultOptions.BufferBatchInterval
+		}
+		walOptions = &Options{
+			FileSize:            fileSize,
+			BufferSize:          bufferSize,
+			BufferBatchInterval: bufferBatchInterval,
+		}
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path: "+path)
+	}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	log := &log{
+		path:             path,
+		options:          *walOptions,
+		logger:           logger.GetLogger(moduleName),
+		writeChannel:     make(chan logRequest),
+		flushChannel:     make(chan buffer, walOptions.FlushQueueSize),
+		bytesBuffer:      bytes.NewBuffer([]byte{}),
+		timestampsBuffer: bytes.NewBuffer([]byte{}),
+		entryCloser:      run.NewCloser(1),
+		buffer: buffer{
+			timestampMap: make(map[common.SeriesIDV2][]time.Time),
+			valueMap:     make(map[common.SeriesIDV2][]byte),
+			callbackMap:  make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)),
+			count:        0,
+		},
+	}
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+
+	log.writeWaitGroup.Add(1)
+	log.flushWaitGroup.Add(1)
+	log.start()
+
+	log.logger.Info().Str("path", path).Msg("WAL has be initialized")
+	return log, nil
+}
+
+// Write a logging entity.
+// It will return immediately when the data is written in the buffer,
+// The callback function will be called when the entity is flushed on the persistent storage.
+func (log *log) Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error)) {
+	if !log.entryCloser.AddRunning() {
+		return
+	}
+	defer log.entryCloser.Done()
+
+	log.writeChannel <- logRequest{
+		seriesID:  seriesID,
+		timestamp: timestamp,
+		data:      data,
+		callback:  callback,
+	}
+}
+
+// Read specified segment by SegmentID.
+func (log *log) Read(segmentID SegmentID) (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segment := log.segmentMap[segmentID]
+	return segment, nil
+}
+
+// ReadAllSegments reads all segments sorted by their creation time in ascending order.
+func (log *log) ReadAllSegments() ([]Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segments := make([]Segment, 0)
+	for _, segment := range log.segmentMap {
+		segments = append(segments, segment)
+	}
+	return segments, nil
+}
+
+// Rotate closes the open segment and opens a new one, returning the closed segment details.
+func (log *log) Rotate() (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	oldWorkSegment, err := log.swapWorkSegment()
+	if err != nil {
+		return nil, err
+	}
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Update segment information.
+	workSegment := log.workSegment
+	log.segmentMap[workSegment.segmentID] = workSegment
+	return oldWorkSegment, nil
+}
+
+// Delete the specified segment.
+func (log *log) Delete(segmentID SegmentID) error {
+	if !log.entryCloser.AddRunning() {
+		return errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Segment which will be deleted must be closed.
+	if segmentID == log.workSegment.segmentID {
+		return errors.New("Can not delete the segment which is working")
+	}
+
+	err := os.Remove(log.segmentMap[segmentID].path)
+	if err != nil {
+		return errors.Wrap(err, "Delete WAL segment error")
+	}
+	delete(log.segmentMap, segmentID)
+	return nil
+}
+
+// Close all of segments and stop WAL work.
+func (log *log) Close() error {
+	log.closerOnce.Do(func() {
+		log.logger.Info().Msg("Closing WAL...")
+
+		log.entryCloser.Done()
+		log.entryCloser.CloseThenWait()
+
+		close(log.writeChannel)
+		log.writeWaitGroup.Wait()
+
+		close(log.flushChannel)
+		log.flushWaitGroup.Wait()
+
+		if err := log.flushBuffer(log.buffer); err != nil {
+			log.logger.Err(err).Msg("Flushing buffer failed")
+		}
+		if err := log.workSegment.file.Close(); err != nil {
+			log.logger.Err(err).Msg("Close work segment file error")
+		}
+		log.logger.Info().Msg("Closed WAL")
+	})
+	return nil
+}
+
+func (log *log) start() {
+	go func() {
+		log.logger.Info().Msg("Start batch task...")
+
+		defer log.writeWaitGroup.Done()
+
+		bufferVolume := 0
+		for {
+			timer := time.NewTimer(log.options.BufferBatchInterval)
+			select {
+			case request, chOpen := <-log.writeChannel:
+				if !chOpen {
+					log.logger.Info().Msg("Stop batch task when write-channel closed!")
+					return
+				}
+
+				log.buffer.write(request)
+				if log.logger.Debug().Enabled() {
+					log.logger.Debug().Msg("Write request to buffer. elements: " + strconv.Itoa(log.buffer.count))
+				}
+
+				bufferVolume += len(request.seriesID.Marshal()) + timestampVolumeLength + len(request.data)
+				if bufferVolume > log.options.BufferSize {
+					log.triggerFlushing()
+					bufferVolume = 0
+				}
+				continue
+			case <-timer.C:
+				if bufferVolume == 0 {
+					continue
+				}
+				log.triggerFlushing()
+				bufferVolume = 0
+				continue
+			}
+		}
+	}()
+
+	go func() {
+		log.logger.Info().Msg("Start flush task...")
+
+		defer log.flushWaitGroup.Done()
+
+		for batch := range log.flushChannel {
+			startTime := time.Now()
+
+			var err error
+			for i := 0; i < maxRetries; i++ {
+				if err = log.flushBuffer(batch); err != nil {
+					log.logger.Err(err).Msg("Flushing buffer failed. Retrying...")
+					time.Sleep(100 * time.Millisecond)

Review Comment:
   fixed



##########
pkg/wal/wal.go:
##########
@@ -42,19 +75,759 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// log implements the WAL interface.
+type log struct {
+	entryCloser      *run.Closer
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentMap       map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+	writeWaitGroup   sync.WaitGroup
+	flushWaitGroup   sync.WaitGroup
+	workSegmentMutex sync.Mutex
+	segmentMapMutex  sync.RWMutex
+	closerOnce       sync.Once
+}
+
+type segment struct {
+	file       *os.File
+	path       string
+	logEntries []LogEntry
+	segmentID  SegmentID
+}
+
+type logRequest struct {
+	seriesID  common.SeriesIDV2
+	timestamp time.Time
+	callback  func(common.SeriesIDV2, time.Time, []byte, error)
+	data      []byte
+}
+
+type logEntry struct {
+	timestamps  []time.Time
+	values      *list.List
+	seriesID    common.SeriesIDV2
+	entryLength int64
+	count       int32
+}
+
+type buffer struct {
+	timestampMap map[common.SeriesIDV2][]time.Time
+	valueMap     map[common.SeriesIDV2][]byte
+	callbackMap  map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)
+	count        int
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	walOptions := DefaultOptions
+	if options != nil {
+		fileSize := options.FileSize
+		if fileSize <= 0 {
+			fileSize = DefaultOptions.FileSize
+		}
+		bufferSize := options.BufferSize
+		if bufferSize <= 0 {
+			bufferSize = DefaultOptions.BufferSize
+		}
+		bufferBatchInterval := options.BufferBatchInterval
+		if bufferBatchInterval <= 0 {
+			bufferBatchInterval = DefaultOptions.BufferBatchInterval
+		}
+		walOptions = &Options{
+			FileSize:            fileSize,
+			BufferSize:          bufferSize,
+			BufferBatchInterval: bufferBatchInterval,
+		}
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path: "+path)
+	}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	log := &log{
+		path:             path,
+		options:          *walOptions,
+		logger:           logger.GetLogger(moduleName),
+		writeChannel:     make(chan logRequest),
+		flushChannel:     make(chan buffer, walOptions.FlushQueueSize),
+		bytesBuffer:      bytes.NewBuffer([]byte{}),
+		timestampsBuffer: bytes.NewBuffer([]byte{}),
+		entryCloser:      run.NewCloser(1),
+		buffer: buffer{
+			timestampMap: make(map[common.SeriesIDV2][]time.Time),
+			valueMap:     make(map[common.SeriesIDV2][]byte),
+			callbackMap:  make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)),
+			count:        0,
+		},
+	}
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+
+	log.writeWaitGroup.Add(1)
+	log.flushWaitGroup.Add(1)
+	log.start()
+
+	log.logger.Info().Str("path", path).Msg("WAL has be initialized")
+	return log, nil
+}
+
+// Write a logging entity.
+// It will return immediately when the data is written in the buffer,
+// The callback function will be called when the entity is flushed on the persistent storage.
+func (log *log) Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error)) {
+	if !log.entryCloser.AddRunning() {
+		return
+	}
+	defer log.entryCloser.Done()
+
+	log.writeChannel <- logRequest{
+		seriesID:  seriesID,
+		timestamp: timestamp,
+		data:      data,
+		callback:  callback,
+	}
+}
+
+// Read specified segment by SegmentID.
+func (log *log) Read(segmentID SegmentID) (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segment := log.segmentMap[segmentID]
+	return segment, nil
+}
+
+// ReadAllSegments reads all segments sorted by their creation time in ascending order.
+func (log *log) ReadAllSegments() ([]Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segments := make([]Segment, 0)
+	for _, segment := range log.segmentMap {
+		segments = append(segments, segment)
+	}
+	return segments, nil
+}
+
+// Rotate closes the open segment and opens a new one, returning the closed segment details.
+func (log *log) Rotate() (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	oldWorkSegment, err := log.swapWorkSegment()
+	if err != nil {
+		return nil, err
+	}
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Update segment information.
+	workSegment := log.workSegment
+	log.segmentMap[workSegment.segmentID] = workSegment
+	return oldWorkSegment, nil
+}
+
+// Delete the specified segment.
+func (log *log) Delete(segmentID SegmentID) error {
+	if !log.entryCloser.AddRunning() {
+		return errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Segment which will be deleted must be closed.
+	if segmentID == log.workSegment.segmentID {
+		return errors.New("Can not delete the segment which is working")
+	}
+
+	err := os.Remove(log.segmentMap[segmentID].path)
+	if err != nil {
+		return errors.Wrap(err, "Delete WAL segment error")
+	}
+	delete(log.segmentMap, segmentID)
+	return nil
+}
+
+// Close all of segments and stop WAL work.
+func (log *log) Close() error {
+	log.closerOnce.Do(func() {
+		log.logger.Info().Msg("Closing WAL...")
+
+		log.entryCloser.Done()
+		log.entryCloser.CloseThenWait()
+
+		close(log.writeChannel)
+		log.writeWaitGroup.Wait()
+
+		close(log.flushChannel)
+		log.flushWaitGroup.Wait()
+
+		if err := log.flushBuffer(log.buffer); err != nil {
+			log.logger.Err(err).Msg("Flushing buffer failed")
+		}
+		if err := log.workSegment.file.Close(); err != nil {
+			log.logger.Err(err).Msg("Close work segment file error")
+		}
+		log.logger.Info().Msg("Closed WAL")
+	})
+	return nil
+}
+
+func (log *log) start() {
+	go func() {
+		log.logger.Info().Msg("Start batch task...")
+
+		defer log.writeWaitGroup.Done()
+
+		bufferVolume := 0
+		for {
+			timer := time.NewTimer(log.options.BufferBatchInterval)
+			select {
+			case request, chOpen := <-log.writeChannel:
+				if !chOpen {
+					log.logger.Info().Msg("Stop batch task when write-channel closed!")

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hailin0 commented on a diff in pull request #261: Implement Write-ahead Logging

Posted by "hailin0 (via GitHub)" <gi...@apache.org>.
hailin0 commented on code in PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#discussion_r1250708005


##########
pkg/run/channel_closer.go:
##########
@@ -0,0 +1,116 @@
+// Licensed to 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. Apache Software Foundation (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 run
+
+import (
+	"context"
+	"sync"
+)
+
+var dummyChannelCloserChan <-chan struct{}
+
+// ChannelCloser can close a goroutine then wait for it to stop.
+type ChannelCloser struct {
+	ctx           context.Context
+	cancel        context.CancelFunc
+	running       sync.WaitGroup
+	waiting       sync.WaitGroup

Review Comment:
   fixed



##########
pkg/run/channel_closer_test.go:
##########
@@ -0,0 +1,262 @@
+// Licensed to 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. Apache Software Foundation (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 run

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hanahmily commented on a diff in pull request #261: Implement Write-ahead Logging

Posted by "hanahmily (via GitHub)" <gi...@apache.org>.
hanahmily commented on code in PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#discussion_r1241173838


##########
pkg/wal/wal.go:
##########
@@ -42,19 +72,642 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// Log implements the WAL interface.
+type Log struct {
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentIndexMap  map[SegmentID]*segment

Review Comment:
   A data race occurs when multi goroutine accesses it. Add a `mutex` to protect it.



##########
pkg/wal/wal.go:
##########
@@ -42,19 +72,642 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// Log implements the WAL interface.
+type Log struct {
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentIndexMap  map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+}
+
+type segment struct {
+	file       *os.File
+	path       string
+	logEntries []LogEntry
+	segmentID  SegmentID
+}
+
+type logRequest struct {
+	seriesID  common.SeriesIDV2
+	timestamp time.Time
+	callback  func(common.SeriesIDV2, time.Time, []byte, error)
+	data      []byte
+}
+
+type logEntry struct {
+	timestamps  []time.Time
+	values      *list.List
+	seriesID    common.SeriesIDV2
+	entryLength int64
+	count       int32
+}
+
+type buffer struct {
+	timestampMap map[common.SeriesIDV2][]time.Time
+	valueMap     map[common.SeriesIDV2][]byte
+	callbackMap  map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)
+	count        int
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	if options == nil {
+		options = DefaultOptions
+	}
+	if options.FileSize <= 0 {
+		options.FileSize = DefaultOptions.FileSize

Review Comment:
   Instead of modifying the parameter, kindly create a local `options`.



##########
pkg/wal/wal.go:
##########
@@ -42,19 +72,642 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// Log implements the WAL interface.
+type Log struct {

Review Comment:
   Convert it into an unexported `log`. As you have already defined an interface `WAL` to carry out necessary operations, there is no need to export this struct.



##########
pkg/wal/wal.go:
##########
@@ -42,19 +72,642 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// Log implements the WAL interface.
+type Log struct {
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentIndexMap  map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+}
+
+type segment struct {
+	file       *os.File
+	path       string
+	logEntries []LogEntry
+	segmentID  SegmentID
+}
+
+type logRequest struct {
+	seriesID  common.SeriesIDV2
+	timestamp time.Time
+	callback  func(common.SeriesIDV2, time.Time, []byte, error)
+	data      []byte
+}
+
+type logEntry struct {
+	timestamps  []time.Time
+	values      *list.List
+	seriesID    common.SeriesIDV2
+	entryLength int64
+	count       int32
+}
+
+type buffer struct {
+	timestampMap map[common.SeriesIDV2][]time.Time
+	valueMap     map[common.SeriesIDV2][]byte
+	callbackMap  map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)
+	count        int
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	if options == nil {
+		options = DefaultOptions
+	}
+	if options.FileSize <= 0 {
+		options.FileSize = DefaultOptions.FileSize
+	}
+	if options.BufferSize <= 0 {
+		options.BufferSize = DefaultOptions.BufferSize
+	}
+	if options.BufferBatchInterval <= 0 {
+		options.BufferBatchInterval = DefaultOptions.BufferBatchInterval
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path: "+path)
+	}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	log := &Log{path: path, options: *options, logger: logger.GetLogger(moduleName)}
+
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+	log.startAsyncFlushTask()
+
+	log.logger.Info().Msgf("WAL initialized at %s", path)
+	return log, nil
+}
+
+// Write a logging entity.
+// It will return immediately when the data is written in the buffer,
+// The callback function will be called when the entity is flushed on the persistent storage.
+func (log *Log) Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error)) {
+	log.writeChannel <- logRequest{
+		seriesID:  seriesID,
+		timestamp: timestamp,
+		data:      data,
+		callback:  callback,
+	}
+}
+
+// Read specified segment by SegmentID.
+func (log *Log) Read(segmentID SegmentID) (Segment, error) {
+	segment := log.segmentIndexMap[segmentID]
+	return segment, nil
+}
+
+// ReadAllSegments reads all segments sorted by their creation time in ascending order.
+func (log *Log) ReadAllSegments() ([]Segment, error) {
+	segments := make([]Segment, 0)
+	for _, segment := range log.segmentIndexMap {
+		segments = append(segments, segment)
+	}
+	return segments, nil
+}
+
+// Rotate closes the open segment and opens a new one, returning the closed segment details.
+func (log *Log) Rotate() (Segment, error) {
+	if err := log.workSegment.file.Close(); err != nil {
+		return nil, errors.Wrap(err, "Close WAL segment error")
+	}
+	oldSegment := log.workSegment
+	// Create new segment.
+	segmentID := log.workSegment.segmentID + 1
+	segment := &segment{
+		segmentID: segmentID,
+		path:      filepath.Join(log.path, segmentName(uint64(segmentID))),
+	}
+	var err error
+	segment.file, err = os.OpenFile(segment.path, os.O_CREATE|os.O_RDWR|os.O_TRUNC, os.ModePerm)
+	if err != nil {
+		return nil, errors.Wrap(err, "Open WAL segment error")
+	}
+
+	// Update segment information.
+	log.segmentIndexMap[segmentID] = segment
+	log.workSegment = segment
+	return oldSegment, nil
+}
+
+// Delete the specified segment.
+func (log *Log) Delete(segmentID SegmentID) error {
+	// Segment which will be deleted must be closed.
+	err := os.Remove(log.segmentIndexMap[segmentID].path)
+	if err != nil {
+		return errors.Wrap(err, "Delete WAL segment error")
+	}
+	delete(log.segmentIndexMap, segmentID)
+	return nil
+}
+
+// Close all of segments and stop WAL work.
+func (log *Log) Close() error {
+	log.logger.Info().Msg("Closing WAL...")
+	close(log.writeChannel)
+	close(log.flushChannel)
+	err := log.workSegment.file.Close()
+	if err != nil {
+		return errors.Wrap(err, "Close WAL error")
+	}
+	log.logger.Info().Msg("Closed WAL")
+	return nil
+}
+
+func (log *Log) startAsyncFlushTask() {
+	go func() {
+		log.logger.Info().Msg("Start batch task...")
+		bufferSize := 0
+		for {
+			timer := time.NewTimer(log.options.BufferBatchInterval)
+			select {
+			case request, ok := <-log.writeChannel:
+				if !ok {
+					log.logger.Info().Msg("Exit selector when write-channel closed!")
+					return
+				}
+				log.buffer.write(request)
+				if log.logger.Debug().Enabled() {
+					log.logger.Debug().Msg("Write request to buffer. elements: " + strconv.Itoa(log.buffer.count))
+				}
+
+				bufferSize += len(request.seriesID.Marshal()) + timestampLength + len(request.data)
+				if bufferSize > log.options.BufferSize {
+					// Clone buffer,avoiding to block write.
+					buf := log.buffer
+					// Clear buffer to receive Log request.
+					log.newBuffer()
+					bufferSize = 0
+					// Send buffer to flushBuffer channel.
+					log.flushChannel <- buf
+					if log.logger.Debug().Enabled() {
+						log.logger.Debug().Msg("Send buffer to flush-channel. elements: " + strconv.Itoa(buf.count))
+					}
+				}
+			case <-timer.C:
+				if bufferSize == 0 {
+					continue
+				}
+				// Clone buffer,avoiding to block write.
+				buf := log.buffer
+				// Clear buffer to receive Log request.
+				log.newBuffer()
+				bufferSize = 0
+				// Send buffer to flushBuffer channel.
+				log.flushChannel <- buf
+				if log.logger.Debug().Enabled() {
+					log.logger.Debug().Msg("Send buffer to flush-channel. elements: " + strconv.Itoa(buf.count))
+				}
+			}
+		}
+	}()
+
+	go func() {
+		log.logger.Info().Msg("Start flush task...")
+
+		for batch := range log.flushChannel {
+			log.flushBuffer(batch)
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Flushed buffer to WAL file. elements: " + strconv.Itoa(batch.count))
+			}
+		}
+
+		log.logger.Info().Msg("Exit flush task when flush-channel closed!")
+	}()
+
+	log.logger.Info().Msg("Started WAL async flush task.")
+}
+
+func (log *Log) flushBuffer(buffer buffer) {
+	log.bytesBuffer.Reset()
+	// placeholder, preset batch length value is 0
+	batchLength := 0
+	log.writeToBytesBuffer(log.bytesBuffer, binary.LittleEndian, int64(batchLength))
+
+	for seriesID, timestamps := range buffer.timestampMap {
+		// Generate seriesID binary
+		seriesIDBytes := seriesID.Marshal()
+		seriesIDBytesLen := len(seriesIDBytes)
+
+		// Generate timestamps compression binary
+		log.timestampsBuffer.Reset()
+		timestampWriter := encoding.NewWriter()
+		timestampEncoder := encoding.NewXOREncoder(timestampWriter)
+		timestampWriter.Reset(log.timestampsBuffer)
+		for _, timestamp := range timestamps {
+			timestampEncoder.Write(timeTouUnixNano(timestamp))
+		}
+		timestampWriter.Flush()
+		timestampsBytes := log.timestampsBuffer.Bytes()
+		timestampsBytesLen := len(timestampsBytes)
+
+		// Generate values compression binary
+		valuesBytes := snappy.Encode(nil, buffer.valueMap[seriesID])
+
+		// Write entry data
+		entryLength := seriesIDLength + seriesIDBytesLen + seriesCountLength + timestampsBinaryLength + timestampsBytesLen + len(valuesBytes)
+		log.writeToBytesBuffer(log.bytesBuffer, binary.LittleEndian, int64(entryLength))
+		log.writeToBytesBuffer(log.bytesBuffer, binary.LittleEndian, int16(seriesIDBytesLen))
+		log.writeToBytesBuffer(log.bytesBuffer, binary.LittleEndian, seriesIDBytes)
+		log.writeToBytesBuffer(log.bytesBuffer, binary.LittleEndian, int32(len(timestamps)))
+		log.writeToBytesBuffer(log.bytesBuffer, binary.LittleEndian, int16(timestampsBytesLen))
+		log.writeToBytesBuffer(log.bytesBuffer, binary.LittleEndian, timestampsBytes)
+		log.writeToBytesBuffer(log.bytesBuffer, binary.LittleEndian, valuesBytes)
+	}
+	// Rewrite batch length
+	batchBytes := log.bytesBuffer.Bytes()
+	batchLength = len(batchBytes) - batchWriteLength
+	rewriteInt64InBuf(batchBytes, int64(batchLength), 0, binary.LittleEndian)
+	log.bytesBuffer.Reset()
+
+	// Flush
+	_, err := log.workSegment.file.Write(batchBytes)
+	if err != nil {
+		log.logger.Error().Err(err).Msg("Write WAL segment file error, file: " + log.workSegment.path)
+		buffer.notifyRequests(err)
+		return
+	}
+	err = log.workSegment.file.Sync()
+	if err != nil {
+		log.logger.Error().Err(err).Msg("Sync WAL segment file to disk error, file: " + log.workSegment.path)
+		buffer.notifyRequests(err)
+		return
+	}
+	buffer.notifyRequests(nil)
+	if log.logger.Debug().Enabled() {
+		log.logger.Debug().Msg("Flushed buffer to WAL. file: " + log.workSegment.path +
+			", elements: " + strconv.Itoa(buffer.count) +
+			", bytes: " + strconv.Itoa(len(batchBytes)))
+	}
+}
+
+func (log *Log) newBuffer() {
+	log.buffer.timestampMap = make(map[common.SeriesIDV2][]time.Time)
+	log.buffer.valueMap = make(map[common.SeriesIDV2][]byte)
+	log.buffer.callbackMap = make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error))
+	log.buffer.count = 0
+}
+
+func (log *Log) load() error {
+	files, err := os.ReadDir(log.path)
+	if err != nil {
+		return errors.Wrap(err, "Can not read dir: "+log.path)
+	}
+	// Load all of WAL segments.
+	var workSegmentID SegmentID
+	log.segmentIndexMap = make(map[SegmentID]*segment)
+	for _, file := range files {

Review Comment:
   During the booting process, it is necessary to remove all files except for the `workSegment`. 
   
   To create a new `workSegment`, the user should rotate the WAL and delete the previous one. In case the deletion fails due to a process crash, the `load` function will take care of cleaning up all non-work segments.



##########
pkg/wal/wal.go:
##########
@@ -42,19 +72,642 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// Log implements the WAL interface.
+type Log struct {
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentIndexMap  map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+}
+
+type segment struct {
+	file       *os.File
+	path       string
+	logEntries []LogEntry
+	segmentID  SegmentID
+}
+
+type logRequest struct {
+	seriesID  common.SeriesIDV2
+	timestamp time.Time
+	callback  func(common.SeriesIDV2, time.Time, []byte, error)
+	data      []byte
+}
+
+type logEntry struct {
+	timestamps  []time.Time
+	values      *list.List
+	seriesID    common.SeriesIDV2
+	entryLength int64
+	count       int32
+}
+
+type buffer struct {
+	timestampMap map[common.SeriesIDV2][]time.Time
+	valueMap     map[common.SeriesIDV2][]byte
+	callbackMap  map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)
+	count        int
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	if options == nil {
+		options = DefaultOptions
+	}
+	if options.FileSize <= 0 {
+		options.FileSize = DefaultOptions.FileSize
+	}
+	if options.BufferSize <= 0 {
+		options.BufferSize = DefaultOptions.BufferSize
+	}
+	if options.BufferBatchInterval <= 0 {
+		options.BufferBatchInterval = DefaultOptions.BufferBatchInterval
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path: "+path)
+	}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	log := &Log{path: path, options: *options, logger: logger.GetLogger(moduleName)}
+
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+	log.startAsyncFlushTask()
+
+	log.logger.Info().Msgf("WAL initialized at %s", path)

Review Comment:
   ```suggestion
   	log.logger.Info().Str("path", path).Msg("WAL has be initialized")
   ```



##########
pkg/wal/wal.go:
##########
@@ -42,19 +72,642 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// Log implements the WAL interface.
+type Log struct {
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentIndexMap  map[SegmentID]*segment
+	workSegment      *segment

Review Comment:
   Please use a separate `mutex` to safeguard the field.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] HHoflittlefish777 commented on pull request #261: Implement Write-ahead Logging

Posted by "HHoflittlefish777 (via GitHub)" <gi...@apache.org>.
HHoflittlefish777 commented on PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#issuecomment-1475097801

   There are some things that need help
   
   - During disk flushing, in order to avoid blocking writes, I cloned the buffer so that I can continue to accept write requests while flushing the disk. However, during asynchronous callbacks, all channels will be notified. At this time, write requests that only write to the buffer should not receive this request, but  they receive it.how can I resolve this issue?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hailin0 commented on pull request #261: Implement Write-ahead Logging

Posted by "hailin0 (via GitHub)" <gi...@apache.org>.
hailin0 commented on PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#issuecomment-1550610896

   I want to try to continue, Thanks


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] HHoflittlefish777 commented on pull request #261: Implement Write-ahead Logging

Posted by "HHoflittlefish777 (via GitHub)" <gi...@apache.org>.
HHoflittlefish777 commented on PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#issuecomment-1475729014

   > > There are some things that need help:
   > > 
   > > * During disk flushing, in order to avoid blocking writes, I cloned the buffer so that I can continue to accept write requests while flushing the disk. However, during asynchronous callbacks, all channels will be notified. At this time, write requests that only write to the buffer should not receive this request, but  they receive it.how can I resolve this issue?
   > 
   > You should have switched to a new flushChannel on switching the buffer. Theoretically, a flush channel sends a notice which indicates its corresponding buffer gets flushed. Based on that, the flush channel seems a field in the `buffer` structure instead.
   
   I have two channel field in the log struct, should I move them into buffer?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] HHoflittlefish777 commented on pull request #261: Implement Write-ahead Logging

Posted by "HHoflittlefish777 (via GitHub)" <gi...@apache.org>.
HHoflittlefish777 commented on PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#issuecomment-1475736958

   > @HHoflittlefish777 Please fix test failures.
   
   
   
   > @HHoflittlefish777 Please fix test failures.
   
   
   
   > @HHoflittlefish777 Please fix test failures.
   
   `Failed to compile cmd:
   
   # github.com/apache/skywalking-banyandb/banyand/query
   ../../../banyand/query/processor_topn.go:385:4: unknown field Entity in struct literal of type "github.com/apache/skywalking-banyandb/api/proto/banyandb/measure/v1".TopNList_Item
   ../../../banyand/query/processor_topn.go:441:5: unknown field Entity in struct literal of type "github.com/apache/skywalking-banyandb/api/proto/banyandb/measure/v1".TopNList_Item`
   The error like it,what's the problem?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hanahmily commented on a diff in pull request #261: Implement Write-ahead Logging

Posted by "hanahmily (via GitHub)" <gi...@apache.org>.
hanahmily commented on code in PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#discussion_r1155916253


##########
pkg/wal/wal.go:
##########
@@ -43,18 +128,450 @@ type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
 	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	Write(seriesID []byte, timestamp time.Time, data []byte) error
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	if options == nil {
+		options = DefaultOptions
+	}
+	if options.FileSize <= 0 {
+		options.FileSize = DefaultOptions.FileSize
+	}
+	if options.BufferSize <= 0 {
+		options.BufferSize = DefaultOptions.BufferSize
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path")
+	}
+	log := &Log{path: path, options: *options}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {

Review Comment:
   `FileMode` should be `0700` 



##########
pkg/wal/wal.go:
##########
@@ -19,20 +19,105 @@
 package wal
 
 import (
+	"bytes"
+	"encoding/binary"
+	"fmt"
+	"os"
+	"path/filepath"
+	"strconv"
 	"time"
 
+	"github.com/golang/snappy"
+	"github.com/pkg/errors"
+
 	"github.com/apache/skywalking-banyandb/api/common"
+	"github.com/apache/skywalking-banyandb/pkg/logger"
+)
+
+const (
+	segmentNamePrefix   = "seg"
+	segmentNameSuffix   = ".wal"
+	batchWriteLength    = 8
+	entryLength         = 8
+	seriesIDLength      = 8
+	countLength         = 4
+	timestampLength     = 8
+	binaryLength        = 2
+	flushSuccessFlag    = "success"
+	flushFailFlag       = "fail"
+	bufferBatchInterval = 500
+	parseTimeStr        = "2006-01-02 15:04:05"
 )
 
 // SegmentID identities a segment in a WAL.
 type SegmentID uint64
 
+// Log implements the WAL interface.
+type Log struct {

Review Comment:
   `Log` should be unexported. 



##########
pkg/wal/wal.go:
##########
@@ -43,18 +128,450 @@ type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
 	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	Write(seriesID []byte, timestamp time.Time, data []byte) error
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	if options == nil {
+		options = DefaultOptions
+	}
+	if options.FileSize <= 0 {
+		options.FileSize = DefaultOptions.FileSize
+	}
+	if options.BufferSize <= 0 {
+		options.BufferSize = DefaultOptions.BufferSize
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path")
+	}
+	log := &Log{path: path, options: *options}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+	log.runFlushTask()
+	return log, nil
+}
+
+func (log *Log) runFlushTask() {
+	go func() {
+		bufferSize := 0
+		for {
+			timer := time.NewTimer(bufferBatchInterval * time.Millisecond)
+			select {
+			case request := <-log.writeChannel:
+				bufferSize += seriesIDLength + timestampLength + len(request.data)

Review Comment:
   The length of `seriesID` is not fixed. 



##########
pkg/wal/wal.go:
##########
@@ -43,18 +128,450 @@ type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
 	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	Write(seriesID []byte, timestamp time.Time, data []byte) error
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	if options == nil {
+		options = DefaultOptions
+	}
+	if options.FileSize <= 0 {
+		options.FileSize = DefaultOptions.FileSize
+	}
+	if options.BufferSize <= 0 {
+		options.BufferSize = DefaultOptions.BufferSize
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path")
+	}
+	log := &Log{path: path, options: *options}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+	log.runFlushTask()
+	return log, nil
+}
+
+func (log *Log) runFlushTask() {
+	go func() {
+		bufferSize := 0
+		for {
+			timer := time.NewTimer(bufferBatchInterval * time.Millisecond)
+			select {
+			case request := <-log.writeChannel:
+				bufferSize += seriesIDLength + timestampLength + len(request.data)
+				err := log.buffer.write(request)
+				if err != nil {
+					log.l.Error().Err(err).Msg("Fail to write to buffer")
+				}
+				if bufferSize > log.options.BufferSize {
+					// Clone buffer,avoiding to block write.
+					buf := log.buffer
+					// Clear buffer to receive Log request.
+					log.newBuffer()
+					if log.closeFlag {

Review Comment:
   You can't archive synchronization through a local varable. To safely share data between goroutines is by using channel or context.



##########
pkg/wal/wal.go:
##########
@@ -43,18 +128,450 @@ type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
 	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	Write(seriesID []byte, timestamp time.Time, data []byte) error

Review Comment:
   Why do you drop off the returned function to get flushed event?



##########
pkg/wal/wal.go:
##########
@@ -43,18 +128,450 @@ type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
 	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	Write(seriesID []byte, timestamp time.Time, data []byte) error
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	if options == nil {
+		options = DefaultOptions
+	}
+	if options.FileSize <= 0 {
+		options.FileSize = DefaultOptions.FileSize
+	}
+	if options.BufferSize <= 0 {
+		options.BufferSize = DefaultOptions.BufferSize
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path")
+	}
+	log := &Log{path: path, options: *options}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+	log.runFlushTask()
+	return log, nil
+}
+
+func (log *Log) runFlushTask() {
+	go func() {
+		bufferSize := 0
+		for {
+			timer := time.NewTimer(bufferBatchInterval * time.Millisecond)
+			select {
+			case request := <-log.writeChannel:
+				bufferSize += seriesIDLength + timestampLength + len(request.data)
+				err := log.buffer.write(request)
+				if err != nil {
+					log.l.Error().Err(err).Msg("Fail to write to buffer")
+				}
+				if bufferSize > log.options.BufferSize {
+					// Clone buffer,avoiding to block write.
+					buf := log.buffer
+					// Clear buffer to receive Log request.
+					log.newBuffer()
+					if log.closeFlag {
+						return
+					}
+					log.asyncBatchflush(buf, log.flushChannel)
+				}
+			case <-timer.C:
+				buf := log.buffer
+				log.newBuffer()
+				if log.closeFlag {
+					return
+				}
+				log.asyncBatchflush(buf, log.flushChannel)
+			}
+		}
+	}()
+}
+
+func (log *Log) asyncBatchflush(buffer buffer, flushCh chan string) {
+	go func() {
+		// Convert to byte.
+		var err error
+		bytesBuffer := bytes.NewBuffer([]byte{})
+		err = binary.Write(bytesBuffer, binary.LittleEndian, int64(log.buffer.size))
+		if err != nil {
+			log.l.Error().Err(err).Msg("EntryLength fail to convert to byte")
+		}
+		for seriesID, timestamp := range buffer.timestampMap {
+			count := 0
+			var timeBytes []byte
+			for _, timestamp := range timestamp {
+				time := []byte(timestamp.String())
+				count += len(time)
+				timeBytes = append(timeBytes, time...)
+			}
+			count /= timestampLength
+			entryLength := seriesIDLength + count + len(timeBytes) + binaryLength + len(buffer.valueMap[seriesID])
+			err = binary.Write(bytesBuffer, binary.LittleEndian, int64(entryLength))
+			if err != nil {
+				log.l.Error().Err(err).Msg("EntryLength fail to convert to byte")
+			}
+			err = binary.Write(bytesBuffer, binary.LittleEndian, uint64(seriesID))
+			if err != nil {
+				log.l.Error().Err(err).Msg("SeriesID fail to convert to byte")
+			}
+			err = binary.Write(bytesBuffer, binary.LittleEndian, int32(count))
+			if err != nil {
+				log.l.Error().Err(err).Msg("Count fail to convert to byte")
+			}
+			err = binary.Write(bytesBuffer, binary.LittleEndian, timeBytes)
+			if err != nil {
+				log.l.Error().Err(err).Msg("Timestamp fail to convert to byte")
+			}
+			err = binary.Write(bytesBuffer, binary.LittleEndian, int16(len(buffer.valueMap[seriesID])))
+			if err != nil {
+				log.l.Error().Err(err).Msg("Binary Length fail to convert to byte")
+			}
+			err = binary.Write(bytesBuffer, binary.LittleEndian, buffer.valueMap[seriesID])
+			if err != nil {
+				log.l.Error().Err(err).Msg("Value fail to convert to byte")
+			}
+		}
+
+		// Compression and flush.
+		compressionData := snappy.Encode(nil, bytesBuffer.Bytes())

Review Comment:
   Instead of compressing the entire segment, it is recommended to separately encode/compress timestamps and values.
   
   The `pkg/encoding/xor` package offers methods for encoding uint64. These can be used to encode/decode the UNIX epoch (nanosecond) of a timestamp.
   
   For quick compression of values, snappy can be utilized.
   
   By following this approach, there is no need for `bytesBuffer` during flushing. Instead, data can be written directly to `workSegment.file`.



##########
pkg/wal/wal_test.go:
##########
@@ -0,0 +1,191 @@
+// Licensed to 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. Apache Software Foundation (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 version can be used to implement embedding versioning details from
+// git branches and tags into the binary importing this package.
+package wal_test
+
+import (
+	"bytes"
+	"encoding/binary"
+	"fmt"
+	"os"
+	"path/filepath"
+	"sync"
+	"time"
+
+	"github.com/onsi/ginkgo/v2"
+	"github.com/onsi/gomega"
+	"github.com/onsi/gomega/gleak"
+
+	"github.com/apache/skywalking-banyandb/api/common"
+	"github.com/apache/skywalking-banyandb/pkg/test/flags"
+	"github.com/apache/skywalking-banyandb/pkg/wal"
+)
+
+var _ = ginkgo.Describe("WAL", func() {
+	var (
+		log     wal.WAL
+		options *wal.Options
+		goods   []gleak.Goroutine
+	)
+	ginkgo.BeforeEach(func() {
+		options = &wal.Options{
+			Compression: true,
+			FileSize:    67108864, // 20MB
+			BufferSize:  16,       // 16B
+		}
+		goods = gleak.Goroutines()
+	})
+	ginkgo.AfterEach(func() {
+		gomega.Eventually(gleak.Goroutines, flags.EventuallyTimeout).ShouldNot(gleak.HaveLeaked(goods))
+	})
+
+	ginkgo.Context("Write and Read", func() {
+		ginkgo.BeforeEach(func() {
+			var err error
+			log, err = wal.New("test", options)
+			gomega.Expect(err).ToNot(gomega.HaveOccurred())
+		})
+
+		ginkgo.AfterEach(func() {
+			path, err := filepath.Abs("test")
+			os.RemoveAll(path)
+			gomega.Expect(err).ToNot(gomega.HaveOccurred())
+			log.Close()
+		})
+
+		ginkgo.It("should write and read data correctly", func() {
+			var err error
+			var wg sync.WaitGroup
+			wg.Add(100)
+			for i := 0; i < 100; i++ {
+				go func(idx int) {
+					defer wg.Done()
+					id := int64(idx)
+					bytesBuffer := bytes.NewBuffer([]byte{})
+					binary.Write(bytesBuffer, binary.LittleEndian, &id)
+					seriesID := bytesBuffer.Bytes()
+					timestamp := time.Now()
+					value := []byte(fmt.Sprintf("value-%d", idx))
+					err = log.Write(seriesID, timestamp, value)
+					gomega.Expect(err).ToNot(gomega.HaveOccurred())
+				}(i)
+			}
+			wg.Wait()
+
+			err = log.Close()
+			gomega.Expect(err).ToNot(gomega.HaveOccurred())
+			log, err = wal.New("test", options)
+			gomega.Expect(err).ToNot(gomega.HaveOccurred())
+
+			segments, err := log.ReadAllSegments()
+			gomega.Expect(err).ToNot(gomega.HaveOccurred())
+			for _, segment := range segments {
+				entries := segment.GetLogEntries()
+				for index, entity := range entries {
+					seriesID := entity.GetSeriesID()
+					gomega.Expect(seriesID == common.SeriesID(index)).To(gomega.BeTrue())
+					value := entity.GetBinary()
+					gomega.Expect(bytes.Equal(value, []byte(fmt.Sprintf("value-%d", index)))).To(gomega.BeTrue())
+				}
+			}
+		})
+	})
+
+	ginkgo.Context("Rotate", func() {
+		ginkgo.BeforeEach(func() {
+			var err error
+			log, err = wal.New("test", options)
+			gomega.Expect(err).ToNot(gomega.HaveOccurred())
+		})
+
+		ginkgo.AfterEach(func() {
+			path, err := filepath.Abs("test")
+			os.RemoveAll(path)
+			gomega.Expect(err).ToNot(gomega.HaveOccurred())
+			log.Close()
+		})
+
+		ginkgo.It("should rotate correctly", func() {
+			var err error
+			var wg sync.WaitGroup
+			wg.Add(100)
+			for i := 0; i < 100; i++ {
+				go func(idx int) {
+					defer wg.Done()
+					id := int64(idx)
+					bytesBuffer := bytes.NewBuffer([]byte{})
+					binary.Write(bytesBuffer, binary.LittleEndian, &id)
+					seriesID := bytesBuffer.Bytes()
+					timestamp := time.Now()
+					value := []byte(fmt.Sprintf("value-%d", idx))
+					err = log.Write(seriesID, timestamp, value)
+					gomega.Expect(err).ToNot(gomega.HaveOccurred())
+				}(i)
+			}
+			wg.Wait()
+
+			segment, err := log.Rotate()

Review Comment:
   Could you verify entries in the segment?



##########
pkg/wal/wal.go:
##########
@@ -43,18 +128,450 @@ type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
 	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	Write(seriesID []byte, timestamp time.Time, data []byte) error
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	if options == nil {
+		options = DefaultOptions
+	}
+	if options.FileSize <= 0 {
+		options.FileSize = DefaultOptions.FileSize
+	}
+	if options.BufferSize <= 0 {
+		options.BufferSize = DefaultOptions.BufferSize
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path")
+	}
+	log := &Log{path: path, options: *options}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+	log.runFlushTask()
+	return log, nil
+}
+
+func (log *Log) runFlushTask() {
+	go func() {
+		bufferSize := 0
+		for {
+			timer := time.NewTimer(bufferBatchInterval * time.Millisecond)
+			select {
+			case request := <-log.writeChannel:
+				bufferSize += seriesIDLength + timestampLength + len(request.data)
+				err := log.buffer.write(request)
+				if err != nil {
+					log.l.Error().Err(err).Msg("Fail to write to buffer")
+				}
+				if bufferSize > log.options.BufferSize {
+					// Clone buffer,avoiding to block write.
+					buf := log.buffer
+					// Clear buffer to receive Log request.
+					log.newBuffer()
+					if log.closeFlag {
+						return
+					}
+					log.asyncBatchflush(buf, log.flushChannel)
+				}
+			case <-timer.C:
+				buf := log.buffer
+				log.newBuffer()
+				if log.closeFlag {
+					return
+				}
+				log.asyncBatchflush(buf, log.flushChannel)
+			}
+		}
+	}()
+}
+
+func (log *Log) asyncBatchflush(buffer buffer, flushCh chan string) {

Review Comment:
   
   ```suggestion
   func (log *Log) asyncBatchFlush(buffer buffer, flushCh chan string) {
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hailin0 commented on pull request #261: Implement Write-ahead Logging

Posted by "hailin0 (via GitHub)" <gi...@apache.org>.
hailin0 commented on PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#issuecomment-1623815069

   > Would you add benchmark to the write and rotate operations?
   
   Yes,  I would add benchmark testing


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hailin0 commented on a diff in pull request #261: Implement Write-ahead Logging

Posted by "hailin0 (via GitHub)" <gi...@apache.org>.
hailin0 commented on code in PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#discussion_r1246011887


##########
pkg/wal/wal.go:
##########
@@ -42,19 +72,642 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// Log implements the WAL interface.
+type Log struct {
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentIndexMap  map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+}
+
+type segment struct {
+	file       *os.File
+	path       string
+	logEntries []LogEntry
+	segmentID  SegmentID
+}
+
+type logRequest struct {
+	seriesID  common.SeriesIDV2
+	timestamp time.Time
+	callback  func(common.SeriesIDV2, time.Time, []byte, error)
+	data      []byte
+}
+
+type logEntry struct {
+	timestamps  []time.Time
+	values      *list.List
+	seriesID    common.SeriesIDV2
+	entryLength int64
+	count       int32
+}
+
+type buffer struct {
+	timestampMap map[common.SeriesIDV2][]time.Time
+	valueMap     map[common.SeriesIDV2][]byte
+	callbackMap  map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)
+	count        int
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	if options == nil {
+		options = DefaultOptions
+	}
+	if options.FileSize <= 0 {
+		options.FileSize = DefaultOptions.FileSize
+	}
+	if options.BufferSize <= 0 {
+		options.BufferSize = DefaultOptions.BufferSize
+	}
+	if options.BufferBatchInterval <= 0 {
+		options.BufferBatchInterval = DefaultOptions.BufferBatchInterval
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path: "+path)
+	}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	log := &Log{path: path, options: *options, logger: logger.GetLogger(moduleName)}
+
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+	log.startAsyncFlushTask()
+
+	log.logger.Info().Msgf("WAL initialized at %s", path)

Review Comment:
   fixed



##########
pkg/wal/wal.go:
##########
@@ -42,19 +72,642 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// Log implements the WAL interface.
+type Log struct {

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] HHoflittlefish777 commented on pull request #261: Implement Write-ahead Logging

Posted by "HHoflittlefish777 (via GitHub)" <gi...@apache.org>.
HHoflittlefish777 commented on PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#issuecomment-1552339490

   @hailin0 Thanks for you interest in it, but WAL doesn't have much work to continue with, just fix the review. There is still a lot of work to be done in BanyanDB, and you can do some other work you like.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hanahmily commented on pull request #261: Implement Write-ahead Logging

Posted by "hanahmily (via GitHub)" <gi...@apache.org>.
hanahmily commented on PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#issuecomment-1552834487

   @hailin0 Please address the issues outlined in the review. If you encounter any difficulties or have questions, feel free to reach out for assistance.
   
   @HHoflittlefish777 
   The remaining tasks will be assigned to @hailin0, who is interested in contributing to the project. Completing this PR will serve as an excellent introduction for him to become familiar with the project.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hailin0 commented on a diff in pull request #261: Implement Write-ahead Logging

Posted by "hailin0 (via GitHub)" <gi...@apache.org>.
hailin0 commented on code in PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#discussion_r1241045251


##########
pkg/wal/wal.go:
##########
@@ -153,147 +157,258 @@ func New(path string, options *Options) (WAL, error) {
 	if options.BufferSize <= 0 {
 		options.BufferSize = DefaultOptions.BufferSize
 	}
+	if options.BufferBatchInterval <= 0 {
+		options.BufferBatchInterval = DefaultOptions.BufferBatchInterval
+	}
 
 	// Initial WAL path.
 	path, err := filepath.Abs(path)
 	if err != nil {
-		return nil, errors.Wrap(err, "Can not get absolute path")
+		return nil, errors.Wrap(err, "Can not get absolute path: "+path)
 	}
-	log := &Log{path: path, options: *options}
 	if err := os.MkdirAll(path, os.ModePerm); err != nil {
 		return nil, err
 	}
 
+	log := &Log{path: path, options: *options, logger: logger.GetLogger(moduleName)}
+
 	if err := log.load(); err != nil {
 		return nil, err
 	}
-	log.runFlushTask()
+	log.startAsyncFlushTask()
+
+	log.logger.Info().Msgf("WAL initialized at %s", path)
 	return log, nil
 }
 
-func (log *Log) runFlushTask() {
+// Write a logging entity.
+// It will return immediately when the data is written in the buffer,
+// The callback function will be called when the entity is flushed on the persistent storage.
+func (log *Log) Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte,
+	callback func(common.SeriesIDV2, time.Time, []byte, error)) {
+	log.writeChannel <- logRequest{
+		seriesID:  seriesID,
+		timestamp: timestamp,
+		data:      data,
+		callback:  callback,
+	}
+}
+
+// Read specified segment by SegmentID.
+func (log *Log) Read(segmentID SegmentID) (Segment, error) {
+	segment := log.segmentIndexMap[segmentID]
+	return segment, nil
+}
+
+// ReadAllSegments reads all segments sorted by their creation time in ascending order.
+func (log *Log) ReadAllSegments() ([]Segment, error) {
+	segments := make([]Segment, 0)
+	for _, segment := range log.segmentIndexMap {
+		segments = append(segments, segment)
+	}
+	return segments, nil
+}
+
+// Rotate closes the open segment and opens a new one, returning the closed segment details.
+func (log *Log) Rotate() (Segment, error) {
+	// 需要锁定

Review Comment:
   thanks, in this commit fixed
   
   <img width="930" alt="image" src="https://github.com/apache/skywalking-banyandb/assets/14371345/f427c9d3-a958-4162-90d5-dd360bdf267c">
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hailin0 commented on a diff in pull request #261: Implement Write-ahead Logging

Posted by "hailin0 (via GitHub)" <gi...@apache.org>.
hailin0 commented on code in PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#discussion_r1246012171


##########
pkg/wal/wal.go:
##########
@@ -42,19 +72,642 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// Log implements the WAL interface.
+type Log struct {
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentIndexMap  map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+}
+
+type segment struct {
+	file       *os.File
+	path       string
+	logEntries []LogEntry
+	segmentID  SegmentID
+}
+
+type logRequest struct {
+	seriesID  common.SeriesIDV2
+	timestamp time.Time
+	callback  func(common.SeriesIDV2, time.Time, []byte, error)
+	data      []byte
+}
+
+type logEntry struct {
+	timestamps  []time.Time
+	values      *list.List
+	seriesID    common.SeriesIDV2
+	entryLength int64
+	count       int32
+}
+
+type buffer struct {
+	timestampMap map[common.SeriesIDV2][]time.Time
+	valueMap     map[common.SeriesIDV2][]byte
+	callbackMap  map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)
+	count        int
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	if options == nil {
+		options = DefaultOptions
+	}
+	if options.FileSize <= 0 {
+		options.FileSize = DefaultOptions.FileSize

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hanahmily commented on a diff in pull request #261: Implement Write-ahead Logging

Posted by "hanahmily (via GitHub)" <gi...@apache.org>.
hanahmily commented on code in PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#discussion_r1250208142


##########
pkg/run/channel_closer.go:
##########
@@ -0,0 +1,116 @@
+// Licensed to 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. Apache Software Foundation (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 run
+
+import (
+	"context"
+	"sync"
+)
+
+var dummyChannelCloserChan <-chan struct{}
+
+// ChannelCloser can close a goroutine then wait for it to stop.
+type ChannelCloser struct {
+	ctx           context.Context
+	cancel        context.CancelFunc
+	running       sync.WaitGroup
+	waiting       sync.WaitGroup

Review Comment:
   Could you rename them to "sender" and "receiver" since the closer is dedicated to chan?



##########
pkg/run/channel_closer_test.go:
##########
@@ -0,0 +1,262 @@
+// Licensed to 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. Apache Software Foundation (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 run

Review Comment:
   There is an example: https://github.com/apache/skywalking-banyandb/pull/261/files#diff-9617c3f66e33e2226c2bd11de0d88c6f5662dea0aa62b535b519c242d738ca22R51 



##########
pkg/run/channel_closer_test.go:
##########
@@ -0,0 +1,262 @@
+// Licensed to 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. Apache Software Foundation (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 run

Review Comment:
   Can you please check if there are any goroutines leaking?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hailin0 commented on a diff in pull request #261: Implement Write-ahead Logging

Posted by "hailin0 (via GitHub)" <gi...@apache.org>.
hailin0 commented on code in PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#discussion_r1249551554


##########
pkg/wal/wal.go:
##########
@@ -42,19 +75,759 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// log implements the WAL interface.
+type log struct {
+	entryCloser      *run.Closer
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentMap       map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+	writeWaitGroup   sync.WaitGroup
+	flushWaitGroup   sync.WaitGroup
+	workSegmentMutex sync.Mutex
+	segmentMapMutex  sync.RWMutex
+	closerOnce       sync.Once
+}
+
+type segment struct {
+	file       *os.File
+	path       string
+	logEntries []LogEntry
+	segmentID  SegmentID
+}
+
+type logRequest struct {
+	seriesID  common.SeriesIDV2
+	timestamp time.Time
+	callback  func(common.SeriesIDV2, time.Time, []byte, error)
+	data      []byte
+}
+
+type logEntry struct {
+	timestamps  []time.Time
+	values      *list.List
+	seriesID    common.SeriesIDV2
+	entryLength int64
+	count       int32
+}
+
+type buffer struct {
+	timestampMap map[common.SeriesIDV2][]time.Time
+	valueMap     map[common.SeriesIDV2][]byte
+	callbackMap  map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)
+	count        int
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	walOptions := DefaultOptions
+	if options != nil {
+		fileSize := options.FileSize
+		if fileSize <= 0 {
+			fileSize = DefaultOptions.FileSize
+		}
+		bufferSize := options.BufferSize
+		if bufferSize <= 0 {
+			bufferSize = DefaultOptions.BufferSize
+		}
+		bufferBatchInterval := options.BufferBatchInterval
+		if bufferBatchInterval <= 0 {
+			bufferBatchInterval = DefaultOptions.BufferBatchInterval
+		}
+		walOptions = &Options{
+			FileSize:            fileSize,
+			BufferSize:          bufferSize,
+			BufferBatchInterval: bufferBatchInterval,
+		}
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path: "+path)
+	}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	log := &log{
+		path:             path,
+		options:          *walOptions,
+		logger:           logger.GetLogger(moduleName),
+		writeChannel:     make(chan logRequest),
+		flushChannel:     make(chan buffer, walOptions.FlushQueueSize),
+		bytesBuffer:      bytes.NewBuffer([]byte{}),
+		timestampsBuffer: bytes.NewBuffer([]byte{}),
+		entryCloser:      run.NewCloser(1),
+		buffer: buffer{
+			timestampMap: make(map[common.SeriesIDV2][]time.Time),
+			valueMap:     make(map[common.SeriesIDV2][]byte),
+			callbackMap:  make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)),
+			count:        0,
+		},
+	}
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+
+	log.writeWaitGroup.Add(1)
+	log.flushWaitGroup.Add(1)
+	log.start()
+
+	log.logger.Info().Str("path", path).Msg("WAL has be initialized")
+	return log, nil
+}
+
+// Write a logging entity.
+// It will return immediately when the data is written in the buffer,
+// The callback function will be called when the entity is flushed on the persistent storage.
+func (log *log) Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error)) {
+	if !log.entryCloser.AddRunning() {
+		return
+	}
+	defer log.entryCloser.Done()
+
+	log.writeChannel <- logRequest{
+		seriesID:  seriesID,
+		timestamp: timestamp,
+		data:      data,
+		callback:  callback,
+	}
+}
+
+// Read specified segment by SegmentID.
+func (log *log) Read(segmentID SegmentID) (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segment := log.segmentMap[segmentID]
+	return segment, nil
+}
+
+// ReadAllSegments reads all segments sorted by their creation time in ascending order.
+func (log *log) ReadAllSegments() ([]Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segments := make([]Segment, 0)
+	for _, segment := range log.segmentMap {
+		segments = append(segments, segment)
+	}
+	return segments, nil
+}
+
+// Rotate closes the open segment and opens a new one, returning the closed segment details.
+func (log *log) Rotate() (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	oldWorkSegment, err := log.swapWorkSegment()
+	if err != nil {
+		return nil, err
+	}
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Update segment information.
+	workSegment := log.workSegment
+	log.segmentMap[workSegment.segmentID] = workSegment
+	return oldWorkSegment, nil
+}
+
+// Delete the specified segment.
+func (log *log) Delete(segmentID SegmentID) error {
+	if !log.entryCloser.AddRunning() {
+		return errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Segment which will be deleted must be closed.
+	if segmentID == log.workSegment.segmentID {
+		return errors.New("Can not delete the segment which is working")
+	}
+
+	err := os.Remove(log.segmentMap[segmentID].path)
+	if err != nil {
+		return errors.Wrap(err, "Delete WAL segment error")
+	}
+	delete(log.segmentMap, segmentID)
+	return nil
+}
+
+// Close all of segments and stop WAL work.
+func (log *log) Close() error {
+	log.closerOnce.Do(func() {
+		log.logger.Info().Msg("Closing WAL...")
+
+		log.entryCloser.Done()
+		log.entryCloser.CloseThenWait()
+
+		close(log.writeChannel)
+		log.writeWaitGroup.Wait()
+
+		close(log.flushChannel)
+		log.flushWaitGroup.Wait()
+
+		if err := log.flushBuffer(log.buffer); err != nil {
+			log.logger.Err(err).Msg("Flushing buffer failed")
+		}
+		if err := log.workSegment.file.Close(); err != nil {
+			log.logger.Err(err).Msg("Close work segment file error")
+		}
+		log.logger.Info().Msg("Closed WAL")
+	})
+	return nil
+}
+
+func (log *log) start() {
+	go func() {
+		log.logger.Info().Msg("Start batch task...")
+
+		defer log.writeWaitGroup.Done()
+
+		bufferVolume := 0
+		for {
+			timer := time.NewTimer(log.options.BufferBatchInterval)
+			select {
+			case request, chOpen := <-log.writeChannel:
+				if !chOpen {
+					log.logger.Info().Msg("Stop batch task when write-channel closed!")
+					return
+				}
+
+				log.buffer.write(request)
+				if log.logger.Debug().Enabled() {
+					log.logger.Debug().Msg("Write request to buffer. elements: " + strconv.Itoa(log.buffer.count))
+				}
+
+				bufferVolume += len(request.seriesID.Marshal()) + timestampVolumeLength + len(request.data)
+				if bufferVolume > log.options.BufferSize {
+					log.triggerFlushing()
+					bufferVolume = 0
+				}
+				continue
+			case <-timer.C:
+				if bufferVolume == 0 {
+					continue
+				}
+				log.triggerFlushing()
+				bufferVolume = 0
+				continue
+			}
+		}
+	}()
+
+	go func() {
+		log.logger.Info().Msg("Start flush task...")
+
+		defer log.flushWaitGroup.Done()
+
+		for batch := range log.flushChannel {
+			startTime := time.Now()
+
+			var err error
+			for i := 0; i < maxRetries; i++ {
+				if err = log.flushBuffer(batch); err != nil {
+					log.logger.Err(err).Msg("Flushing buffer failed. Retrying...")
+					time.Sleep(100 * time.Millisecond)
+					continue
+				}
+				break
+			}
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Flushed buffer to WAL file. elements: " +
+					strconv.Itoa(batch.count) + ", cost: " + time.Since(startTime).String())
+			}
+
+			batch.notifyRequests(err)
+		}
+		log.logger.Info().Msg("Stop flush task when flush-channel closed!")
+	}()
+
+	log.logger.Info().Msg("Started WAL")
+}
+
+func (log *log) triggerFlushing() {
+	for {
+		select {
+		case log.flushChannel <- log.buffer:
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Send buffer to flush-channel. elements: " + strconv.Itoa(log.buffer.count))
+			}
+			log.newBuffer()
+			return
+		default:
+		}
+		time.Sleep(10 * time.Millisecond)
+	}
+}
+
+func (log *log) newBuffer() {
+	log.buffer.timestampMap = make(map[common.SeriesIDV2][]time.Time)
+	log.buffer.valueMap = make(map[common.SeriesIDV2][]byte)
+	log.buffer.callbackMap = make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error))
+	log.buffer.count = 0
+}
+
+func (log *log) flushBuffer(buffer buffer) error {
+	if buffer.count == 0 {
+		return nil
+	}
+
+	defer func() {
+		log.bytesBuffer.Reset()
+		log.timestampsBuffer.Reset()
+	}()
+
+	// placeholder, preset batch length value is 0
+	batchLen := 0
+	if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(batchLen)); err != nil {
+		return errors.Wrap(err, "Write batch length error")
+	}
+	for seriesID, timestamps := range buffer.timestampMap {
+		// Generate seriesID binary
+		seriesIDBytes := seriesID.Marshal()
+		seriesIDBytesLen := len(seriesIDBytes)
+
+		// Generate timestamps compression binary
+		log.timestampsBuffer.Reset()
+		timestampWriter := encoding.NewWriter()
+		timestampEncoder := encoding.NewXOREncoder(timestampWriter)
+		timestampWriter.Reset(log.timestampsBuffer)
+		for _, timestamp := range timestamps {
+			timestampEncoder.Write(timeTouUnixNano(timestamp))
+		}
+		timestampWriter.Flush()
+		timestampsBytes := log.timestampsBuffer.Bytes()
+		timestampsBytesLen := len(timestampsBytes)
+
+		// Generate values compression binary
+		valuesBytes := snappy.Encode(nil, buffer.valueMap[seriesID])
+
+		// Write entry data
+		entryLen := seriesIDLength + seriesIDBytesLen + seriesCountLength + timestampsBinaryLength + timestampsBytesLen + len(valuesBytes)
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(entryLen)); err != nil {
+			return errors.Wrap(err, "Write entry length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int16(seriesIDBytesLen)); err != nil {
+			return errors.Wrap(err, "Write seriesID length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, seriesIDBytes); err != nil {
+			return errors.Wrap(err, "Write seriesID error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int32(len(timestamps))); err != nil {
+			return errors.Wrap(err, "Write series count error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int16(timestampsBytesLen)); err != nil {
+			return errors.Wrap(err, "Write timestamps length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, timestampsBytes); err != nil {
+			return errors.Wrap(err, "Write timestamps error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, valuesBytes); err != nil {
+			return errors.Wrap(err, "Write values error")
+		}
+	}
+	// Rewrite batch length
+	batchBytes := log.bytesBuffer.Bytes()
+	batchLen = len(batchBytes) - batchLength
+	rewriteInt64InBuf(batchBytes, int64(batchLen), 0, binary.LittleEndian)
+
+	return log.writeWorkSegment(batchBytes)
+}
+
+func (log *log) swapWorkSegment() (Segment, error) {
+	log.workSegmentMutex.Lock()
+	defer log.workSegmentMutex.Unlock()
+
+	if err := log.workSegment.file.Close(); err != nil {
+		return nil, errors.Wrap(err, "Close WAL segment error")
+	}
+
+	oldSegment := log.workSegment
+	// Create new segment.
+	segmentID := log.workSegment.segmentID + 1
+	segment := &segment{
+		segmentID: segmentID,
+		path:      filepath.Join(log.path, segmentName(uint64(segmentID))),
+	}
+	var err error
+	segment.file, err = os.OpenFile(segment.path, os.O_CREATE|os.O_RDWR|os.O_TRUNC, os.ModePerm)
+	if err != nil {
+		return nil, errors.Wrap(err, "Open WAL segment error")
+	}
+	log.workSegment = segment
+
+	return oldSegment, nil
+}
+
+func (log *log) writeWorkSegment(data []byte) error {
+	log.workSegmentMutex.Lock()
+	defer log.workSegmentMutex.Unlock()
+
+	// Write batch data to WAL segment file
+	if _, err := log.workSegment.file.Write(data); err != nil {
+		return errors.Wrap(err, "Write WAL segment file error, file: "+log.workSegment.path)
+	}
+	if err := log.workSegment.file.Sync(); err != nil {
+		log.logger.Warn().Msg("Sync WAL segment file to disk failed, file: " + log.workSegment.path)
+	}
+	return nil
+}
+
+func (log *log) load() error {
+	files, err := os.ReadDir(log.path)
+	if err != nil {
+		return errors.Wrap(err, "Can not read dir: "+log.path)
+	}
+	// Load all of WAL segments.
+	var workSegmentID SegmentID
+	log.segmentMap = make(map[SegmentID]*segment)
+	for _, file := range files {
+		name := file.Name()
+		segmentID, parsePathErr := strconv.ParseUint(name[3:19], 10, 64)
+		if parsePathErr != nil {
+			return errors.Wrap(parsePathErr, "Parse file name error, name: "+name)
+		}
+		if segmentID > uint64(workSegmentID) {
+			workSegmentID = SegmentID(segmentID)
+		}
+		segment := &segment{
+			segmentID: SegmentID(segmentID),
+			path:      filepath.Join(log.path, segmentName(segmentID)),
+		}
+		if err = segment.parseLogEntries(); err != nil {
+			return errors.Wrap(err, "Fail to parse log entries")
+		}
+		log.segmentMap[SegmentID(segmentID)] = segment
+
+		if log.logger.Debug().Enabled() {
+			log.logger.Debug().Msg("Loaded segment file: " + segment.path)
+		}
+	}
+
+	// If load first time.
+	if len(log.segmentMap) == 0 {
+		segment := &segment{
+			segmentID: 1,
+			path:      filepath.Join(log.path, segmentName(1)),
+		}
+		log.segmentMap[1] = segment
+		log.workSegment = segment
+	} else {
+		log.workSegment = log.segmentMap[workSegmentID]
+	}
+	log.workSegment.file, err = os.OpenFile(log.workSegment.path, os.O_CREATE|os.O_RDWR|os.O_APPEND, os.ModePerm)
+	if err != nil {
+		return errors.Wrap(err, "Open WAL segment error, file: "+log.workSegment.path)
+	}
+	return nil
+}
+
+func (segment *segment) GetSegmentID() SegmentID {
+	return segment.segmentID
+}
+
+func (segment *segment) GetLogEntries() []LogEntry {
+	return segment.logEntries
+}
+
+func (segment *segment) parseLogEntries() error {
+	segmentBytes, err := os.ReadFile(segment.path)
+	if err != nil {
+		return errors.Wrap(err, "Read WAL segment failed, path: "+segment.path)
+	}
+
+	var logEntries []LogEntry
+	var data []byte
+	var batchLen int64
+	var entryLen int64
+	var seriesIDLen int16
+	var seriesID common.SeriesIDV2
+	var seriesCount int32
+	var timestampsBinaryLen int16
+	var entryEndPos int
+	var bytesBuf *bytes.Buffer
+
+	oldPos := 0
+	pos := 0
+	parseNextBatchFlag := true
+
+	for {
+		if parseNextBatchFlag {
+			if len(segmentBytes) <= batchLength {
+				break
+			}
+			data = segmentBytes[pos : pos+batchLength]
+			bytesBuf = bytes.NewBuffer(data)
+			if err = binary.Read(bytesBuf, binary.LittleEndian, &batchLen); err != nil {
+				return errors.Wrap(err, "Read batch length fail to convert from bytes")
+			}
+			if len(segmentBytes) <= int(batchLen) {
+				break
+			}
+
+			pos += batchLength
+			oldPos = pos
+			parseNextBatchFlag = false
+		}
+
+		// Parse entryLength.
+		data = segmentBytes[pos : pos+entryLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &entryLen); err != nil {
+			return errors.Wrap(err, "Read entry length fail to convert from byte")
+		}
+		pos += entryLength
+
+		// Mark entry end-position
+		entryEndPos = pos + int(entryLen)
+		if len(segmentBytes) < entryEndPos {
+			break
+		}
+
+		// Parse seriesIDLen.
+		data = segmentBytes[pos : pos+seriesIDLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &seriesIDLen); err != nil {
+			return errors.Wrap(err, "Read seriesID length fail to convert from byte")
+		}
+		pos += seriesIDLength
+
+		// Parse seriesID.
+		data = segmentBytes[pos : pos+int(seriesIDLen)]
+		seriesID = common.ParseSeriesIDV2(data)
+		pos += int(seriesIDLen)
+
+		// Parse series count.
+		data = segmentBytes[pos : pos+seriesCountLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &seriesCount); err != nil {
+			return errors.Wrap(err, "Read series count fail to convert from byte")
+		}
+		pos += seriesCountLength
+
+		// Parse timestamps compression binary.
+		data = segmentBytes[pos : pos+timestampsBinaryLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &timestampsBinaryLen); err != nil {
+			return errors.Wrap(err, "Read timestamps compression binary length fail to convert from byte")
+		}
+		pos += timestampsBinaryLength
+		data = segmentBytes[pos : pos+int(timestampsBinaryLen)]
+		timestampReader := encoding.NewReader(bytes.NewReader(data))
+		timestampDecoder := encoding.NewXORDecoder(timestampReader)
+		var timestamps []time.Time
+		for i := 0; i < int(seriesCount); i++ {
+			if !timestampDecoder.Next() {
+				return errors.Wrap(err, "Timestamps length not match series count, index: "+strconv.Itoa(i))
+			}
+			timestamps = append(timestamps, unixNanoToTime(timestampDecoder.Value()))
+		}
+		pos += int(timestampsBinaryLen)
+
+		// Parse values compression binary.
+		data = segmentBytes[pos:entryEndPos]
+		if data, err = snappy.Decode(nil, data); err != nil {
+			return errors.Wrap(err, "Decode values compression binary fail to snappy decode")
+		}
+		values := parseValuesBinary(data)
+		if values.Len() != len(timestamps) {
+			return errors.New("Timestamps length and values length not match: " + strconv.Itoa(len(timestamps)) + " vs " + strconv.Itoa(values.Len()))
+		}
+		pos = entryEndPos
+
+		logEntry := &logEntry{
+			entryLength: entryLen,
+			seriesID:    seriesID,
+			count:       seriesCount,
+			timestamps:  timestamps,
+			values:      values,
+		}
+		logEntries = append(logEntries, logEntry)
+
+		if pos == len(segmentBytes) {
+			break
+		}
+		if pos-oldPos == int(batchLen) {
+			parseNextBatchFlag = true
+		}
+	}
+	segment.logEntries = logEntries
+	return nil
+}
+
+func (logEntry *logEntry) GetSeriesID() common.SeriesIDV2 {
+	return logEntry.seriesID
+}
+
+func (logEntry *logEntry) GetTimestamps() []time.Time {
+	return logEntry.timestamps
+}
+
+func (logEntry *logEntry) GetValues() *list.List {
+	return logEntry.values
+}
+
+func (buffer *buffer) write(request logRequest) {
+	seriesID := request.seriesID
+	buffer.timestampMap[seriesID] = append(buffer.timestampMap[seriesID], request.timestamp)
+
+	// Value item: binary-length(2-bytes) + binary data(n-bytes)
+	binaryLength := int16ToBytes(int16(len(request.data)))
+	buffer.valueMap[seriesID] = append(buffer.valueMap[seriesID], binaryLength...)
+	buffer.valueMap[seriesID] = append(buffer.valueMap[seriesID], request.data...)
+
+	buffer.callbackMap[seriesID] = append(buffer.callbackMap[seriesID], request.callback)
+	buffer.count++
+}
+
+func (buffer *buffer) notifyRequests(err error) {
+	for seriesID, callbacks := range buffer.callbackMap {
+		timestamps := buffer.timestampMap[seriesID]
+		values := buffer.valueMap[seriesID]
+		valuePos := 0
+		var valueItem []byte
+		for index, callback := range callbacks {
+			valuePos, valueItem = readValuesBinary(values, valuePos, valuesBinaryLength)
+			tryCallback(func() {
+				callback(seriesID, timestamps[index], valueItem, err)
+			})
+		}
+	}
+}
+
+func segmentName(index uint64) string {
+	return fmt.Sprintf("%v%016x%v", segmentNamePrefix, index, segmentNameSuffix)
+}
+
+func tryCallback(callback func()) {
+	defer func() {
+		if err := recover(); err != nil {
+			fmt.Printf("Run callback error is %v\n", err)
+		}
+	}()
+	callback()
+}
+
+func parseValuesBinary(binary []byte) *list.List {
+	values := list.New()
+	position := 0
+	for {
+		nextPosition, value := readValuesBinary(binary, position, valuesBinaryLength)
+		if value == nil {
+			break
+		}
+		values.PushBack(value)
+		position = nextPosition
+	}
+	return values
+}
+
+func readValuesBinary(raw []byte, position int, offsetLen int) (int, []byte) {
+	if position == len(raw) {
+		return position, nil
+	}
+
+	data := raw[position : position+offsetLen]
+	binaryLen := bytesToInt16(data)
+	position += offsetLen
+
+	data = raw[position : position+int(binaryLen)]
+	position += int(binaryLen)
+	return position, data
+}
+
+func rewriteInt64InBuf(buf []byte, value int64, offset int, order binary.ByteOrder) {
+	_ = buf[offset+7] // early bounds check to guarantee safety of writes below
+	if order == binary.LittleEndian {
+		buf[offset+0] = byte(value)
+		buf[offset+1] = byte(value >> 8)
+		buf[offset+2] = byte(value >> 16)
+		buf[offset+3] = byte(value >> 24)
+		buf[offset+4] = byte(value >> 32)
+		buf[offset+5] = byte(value >> 40)
+		buf[offset+6] = byte(value >> 48)
+		buf[offset+7] = byte(value >> 56)
+	} else {
+		buf[offset+0] = byte(value >> 56)
+		buf[offset+1] = byte(value >> 48)
+		buf[offset+2] = byte(value >> 40)
+		buf[offset+3] = byte(value >> 32)
+		buf[offset+4] = byte(value >> 24)
+		buf[offset+5] = byte(value >> 16)
+		buf[offset+6] = byte(value >> 8)
+		buf[offset+7] = byte(value)
+	}
+}
+
+func int16ToBytes(i int16) []byte {

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hailin0 commented on a diff in pull request #261: Implement Write-ahead Logging

Posted by "hailin0 (via GitHub)" <gi...@apache.org>.
hailin0 commented on code in PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#discussion_r1249551580


##########
pkg/wal/wal.go:
##########
@@ -42,19 +75,759 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// log implements the WAL interface.
+type log struct {
+	entryCloser      *run.Closer
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentMap       map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+	writeWaitGroup   sync.WaitGroup
+	flushWaitGroup   sync.WaitGroup
+	workSegmentMutex sync.Mutex
+	segmentMapMutex  sync.RWMutex
+	closerOnce       sync.Once
+}
+
+type segment struct {
+	file       *os.File
+	path       string
+	logEntries []LogEntry
+	segmentID  SegmentID
+}
+
+type logRequest struct {
+	seriesID  common.SeriesIDV2
+	timestamp time.Time
+	callback  func(common.SeriesIDV2, time.Time, []byte, error)
+	data      []byte
+}
+
+type logEntry struct {
+	timestamps  []time.Time
+	values      *list.List
+	seriesID    common.SeriesIDV2
+	entryLength int64
+	count       int32
+}
+
+type buffer struct {
+	timestampMap map[common.SeriesIDV2][]time.Time
+	valueMap     map[common.SeriesIDV2][]byte
+	callbackMap  map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)
+	count        int
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	walOptions := DefaultOptions
+	if options != nil {
+		fileSize := options.FileSize
+		if fileSize <= 0 {
+			fileSize = DefaultOptions.FileSize
+		}
+		bufferSize := options.BufferSize
+		if bufferSize <= 0 {
+			bufferSize = DefaultOptions.BufferSize
+		}
+		bufferBatchInterval := options.BufferBatchInterval
+		if bufferBatchInterval <= 0 {
+			bufferBatchInterval = DefaultOptions.BufferBatchInterval
+		}
+		walOptions = &Options{
+			FileSize:            fileSize,
+			BufferSize:          bufferSize,
+			BufferBatchInterval: bufferBatchInterval,
+		}
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path: "+path)
+	}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	log := &log{
+		path:             path,
+		options:          *walOptions,
+		logger:           logger.GetLogger(moduleName),
+		writeChannel:     make(chan logRequest),
+		flushChannel:     make(chan buffer, walOptions.FlushQueueSize),
+		bytesBuffer:      bytes.NewBuffer([]byte{}),
+		timestampsBuffer: bytes.NewBuffer([]byte{}),
+		entryCloser:      run.NewCloser(1),
+		buffer: buffer{
+			timestampMap: make(map[common.SeriesIDV2][]time.Time),
+			valueMap:     make(map[common.SeriesIDV2][]byte),
+			callbackMap:  make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)),
+			count:        0,
+		},
+	}
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+
+	log.writeWaitGroup.Add(1)
+	log.flushWaitGroup.Add(1)
+	log.start()
+
+	log.logger.Info().Str("path", path).Msg("WAL has be initialized")
+	return log, nil
+}
+
+// Write a logging entity.
+// It will return immediately when the data is written in the buffer,
+// The callback function will be called when the entity is flushed on the persistent storage.
+func (log *log) Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error)) {
+	if !log.entryCloser.AddRunning() {
+		return
+	}
+	defer log.entryCloser.Done()
+
+	log.writeChannel <- logRequest{
+		seriesID:  seriesID,
+		timestamp: timestamp,
+		data:      data,
+		callback:  callback,
+	}
+}
+
+// Read specified segment by SegmentID.
+func (log *log) Read(segmentID SegmentID) (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segment := log.segmentMap[segmentID]
+	return segment, nil
+}
+
+// ReadAllSegments reads all segments sorted by their creation time in ascending order.
+func (log *log) ReadAllSegments() ([]Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segments := make([]Segment, 0)
+	for _, segment := range log.segmentMap {
+		segments = append(segments, segment)
+	}
+	return segments, nil
+}
+
+// Rotate closes the open segment and opens a new one, returning the closed segment details.
+func (log *log) Rotate() (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	oldWorkSegment, err := log.swapWorkSegment()
+	if err != nil {
+		return nil, err
+	}
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Update segment information.
+	workSegment := log.workSegment
+	log.segmentMap[workSegment.segmentID] = workSegment
+	return oldWorkSegment, nil
+}
+
+// Delete the specified segment.
+func (log *log) Delete(segmentID SegmentID) error {
+	if !log.entryCloser.AddRunning() {
+		return errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Segment which will be deleted must be closed.
+	if segmentID == log.workSegment.segmentID {
+		return errors.New("Can not delete the segment which is working")
+	}
+
+	err := os.Remove(log.segmentMap[segmentID].path)
+	if err != nil {
+		return errors.Wrap(err, "Delete WAL segment error")
+	}
+	delete(log.segmentMap, segmentID)
+	return nil
+}
+
+// Close all of segments and stop WAL work.
+func (log *log) Close() error {
+	log.closerOnce.Do(func() {
+		log.logger.Info().Msg("Closing WAL...")
+
+		log.entryCloser.Done()
+		log.entryCloser.CloseThenWait()
+
+		close(log.writeChannel)
+		log.writeWaitGroup.Wait()
+
+		close(log.flushChannel)
+		log.flushWaitGroup.Wait()
+
+		if err := log.flushBuffer(log.buffer); err != nil {
+			log.logger.Err(err).Msg("Flushing buffer failed")
+		}
+		if err := log.workSegment.file.Close(); err != nil {
+			log.logger.Err(err).Msg("Close work segment file error")
+		}
+		log.logger.Info().Msg("Closed WAL")
+	})
+	return nil
+}
+
+func (log *log) start() {
+	go func() {
+		log.logger.Info().Msg("Start batch task...")
+
+		defer log.writeWaitGroup.Done()
+
+		bufferVolume := 0
+		for {
+			timer := time.NewTimer(log.options.BufferBatchInterval)
+			select {
+			case request, chOpen := <-log.writeChannel:
+				if !chOpen {
+					log.logger.Info().Msg("Stop batch task when write-channel closed!")
+					return
+				}
+
+				log.buffer.write(request)
+				if log.logger.Debug().Enabled() {
+					log.logger.Debug().Msg("Write request to buffer. elements: " + strconv.Itoa(log.buffer.count))
+				}
+
+				bufferVolume += len(request.seriesID.Marshal()) + timestampVolumeLength + len(request.data)
+				if bufferVolume > log.options.BufferSize {
+					log.triggerFlushing()
+					bufferVolume = 0
+				}
+				continue
+			case <-timer.C:
+				if bufferVolume == 0 {
+					continue
+				}
+				log.triggerFlushing()
+				bufferVolume = 0
+				continue
+			}
+		}
+	}()
+
+	go func() {
+		log.logger.Info().Msg("Start flush task...")
+
+		defer log.flushWaitGroup.Done()
+
+		for batch := range log.flushChannel {
+			startTime := time.Now()
+
+			var err error
+			for i := 0; i < maxRetries; i++ {
+				if err = log.flushBuffer(batch); err != nil {
+					log.logger.Err(err).Msg("Flushing buffer failed. Retrying...")
+					time.Sleep(100 * time.Millisecond)
+					continue
+				}
+				break
+			}
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Flushed buffer to WAL file. elements: " +
+					strconv.Itoa(batch.count) + ", cost: " + time.Since(startTime).String())
+			}
+
+			batch.notifyRequests(err)
+		}
+		log.logger.Info().Msg("Stop flush task when flush-channel closed!")
+	}()
+
+	log.logger.Info().Msg("Started WAL")
+}
+
+func (log *log) triggerFlushing() {
+	for {
+		select {
+		case log.flushChannel <- log.buffer:
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Send buffer to flush-channel. elements: " + strconv.Itoa(log.buffer.count))
+			}
+			log.newBuffer()
+			return
+		default:
+		}
+		time.Sleep(10 * time.Millisecond)
+	}
+}
+
+func (log *log) newBuffer() {
+	log.buffer.timestampMap = make(map[common.SeriesIDV2][]time.Time)
+	log.buffer.valueMap = make(map[common.SeriesIDV2][]byte)
+	log.buffer.callbackMap = make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error))
+	log.buffer.count = 0
+}
+
+func (log *log) flushBuffer(buffer buffer) error {
+	if buffer.count == 0 {
+		return nil
+	}
+
+	defer func() {
+		log.bytesBuffer.Reset()
+		log.timestampsBuffer.Reset()
+	}()
+
+	// placeholder, preset batch length value is 0
+	batchLen := 0
+	if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(batchLen)); err != nil {
+		return errors.Wrap(err, "Write batch length error")
+	}
+	for seriesID, timestamps := range buffer.timestampMap {
+		// Generate seriesID binary
+		seriesIDBytes := seriesID.Marshal()
+		seriesIDBytesLen := len(seriesIDBytes)
+
+		// Generate timestamps compression binary
+		log.timestampsBuffer.Reset()
+		timestampWriter := encoding.NewWriter()
+		timestampEncoder := encoding.NewXOREncoder(timestampWriter)
+		timestampWriter.Reset(log.timestampsBuffer)
+		for _, timestamp := range timestamps {
+			timestampEncoder.Write(timeTouUnixNano(timestamp))
+		}
+		timestampWriter.Flush()
+		timestampsBytes := log.timestampsBuffer.Bytes()
+		timestampsBytesLen := len(timestampsBytes)
+
+		// Generate values compression binary
+		valuesBytes := snappy.Encode(nil, buffer.valueMap[seriesID])
+
+		// Write entry data
+		entryLen := seriesIDLength + seriesIDBytesLen + seriesCountLength + timestampsBinaryLength + timestampsBytesLen + len(valuesBytes)
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(entryLen)); err != nil {
+			return errors.Wrap(err, "Write entry length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int16(seriesIDBytesLen)); err != nil {
+			return errors.Wrap(err, "Write seriesID length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, seriesIDBytes); err != nil {
+			return errors.Wrap(err, "Write seriesID error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int32(len(timestamps))); err != nil {
+			return errors.Wrap(err, "Write series count error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int16(timestampsBytesLen)); err != nil {
+			return errors.Wrap(err, "Write timestamps length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, timestampsBytes); err != nil {
+			return errors.Wrap(err, "Write timestamps error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, valuesBytes); err != nil {
+			return errors.Wrap(err, "Write values error")
+		}
+	}
+	// Rewrite batch length
+	batchBytes := log.bytesBuffer.Bytes()
+	batchLen = len(batchBytes) - batchLength
+	rewriteInt64InBuf(batchBytes, int64(batchLen), 0, binary.LittleEndian)
+
+	return log.writeWorkSegment(batchBytes)
+}
+
+func (log *log) swapWorkSegment() (Segment, error) {
+	log.workSegmentMutex.Lock()
+	defer log.workSegmentMutex.Unlock()
+
+	if err := log.workSegment.file.Close(); err != nil {
+		return nil, errors.Wrap(err, "Close WAL segment error")
+	}
+
+	oldSegment := log.workSegment
+	// Create new segment.
+	segmentID := log.workSegment.segmentID + 1
+	segment := &segment{
+		segmentID: segmentID,
+		path:      filepath.Join(log.path, segmentName(uint64(segmentID))),
+	}
+	var err error
+	segment.file, err = os.OpenFile(segment.path, os.O_CREATE|os.O_RDWR|os.O_TRUNC, os.ModePerm)
+	if err != nil {
+		return nil, errors.Wrap(err, "Open WAL segment error")
+	}
+	log.workSegment = segment
+
+	return oldSegment, nil
+}
+
+func (log *log) writeWorkSegment(data []byte) error {
+	log.workSegmentMutex.Lock()
+	defer log.workSegmentMutex.Unlock()
+
+	// Write batch data to WAL segment file
+	if _, err := log.workSegment.file.Write(data); err != nil {
+		return errors.Wrap(err, "Write WAL segment file error, file: "+log.workSegment.path)
+	}
+	if err := log.workSegment.file.Sync(); err != nil {
+		log.logger.Warn().Msg("Sync WAL segment file to disk failed, file: " + log.workSegment.path)
+	}
+	return nil
+}
+
+func (log *log) load() error {
+	files, err := os.ReadDir(log.path)
+	if err != nil {
+		return errors.Wrap(err, "Can not read dir: "+log.path)
+	}
+	// Load all of WAL segments.
+	var workSegmentID SegmentID
+	log.segmentMap = make(map[SegmentID]*segment)
+	for _, file := range files {
+		name := file.Name()
+		segmentID, parsePathErr := strconv.ParseUint(name[3:19], 10, 64)
+		if parsePathErr != nil {
+			return errors.Wrap(parsePathErr, "Parse file name error, name: "+name)
+		}
+		if segmentID > uint64(workSegmentID) {
+			workSegmentID = SegmentID(segmentID)
+		}
+		segment := &segment{
+			segmentID: SegmentID(segmentID),
+			path:      filepath.Join(log.path, segmentName(segmentID)),
+		}
+		if err = segment.parseLogEntries(); err != nil {
+			return errors.Wrap(err, "Fail to parse log entries")
+		}
+		log.segmentMap[SegmentID(segmentID)] = segment
+
+		if log.logger.Debug().Enabled() {
+			log.logger.Debug().Msg("Loaded segment file: " + segment.path)
+		}
+	}
+
+	// If load first time.
+	if len(log.segmentMap) == 0 {
+		segment := &segment{
+			segmentID: 1,
+			path:      filepath.Join(log.path, segmentName(1)),
+		}
+		log.segmentMap[1] = segment
+		log.workSegment = segment
+	} else {
+		log.workSegment = log.segmentMap[workSegmentID]
+	}
+	log.workSegment.file, err = os.OpenFile(log.workSegment.path, os.O_CREATE|os.O_RDWR|os.O_APPEND, os.ModePerm)
+	if err != nil {
+		return errors.Wrap(err, "Open WAL segment error, file: "+log.workSegment.path)
+	}
+	return nil
+}
+
+func (segment *segment) GetSegmentID() SegmentID {
+	return segment.segmentID
+}
+
+func (segment *segment) GetLogEntries() []LogEntry {
+	return segment.logEntries
+}
+
+func (segment *segment) parseLogEntries() error {
+	segmentBytes, err := os.ReadFile(segment.path)
+	if err != nil {
+		return errors.Wrap(err, "Read WAL segment failed, path: "+segment.path)
+	}
+
+	var logEntries []LogEntry
+	var data []byte
+	var batchLen int64
+	var entryLen int64
+	var seriesIDLen int16
+	var seriesID common.SeriesIDV2
+	var seriesCount int32
+	var timestampsBinaryLen int16
+	var entryEndPos int
+	var bytesBuf *bytes.Buffer
+
+	oldPos := 0
+	pos := 0
+	parseNextBatchFlag := true
+
+	for {
+		if parseNextBatchFlag {
+			if len(segmentBytes) <= batchLength {
+				break
+			}
+			data = segmentBytes[pos : pos+batchLength]
+			bytesBuf = bytes.NewBuffer(data)
+			if err = binary.Read(bytesBuf, binary.LittleEndian, &batchLen); err != nil {
+				return errors.Wrap(err, "Read batch length fail to convert from bytes")
+			}
+			if len(segmentBytes) <= int(batchLen) {
+				break
+			}
+
+			pos += batchLength
+			oldPos = pos
+			parseNextBatchFlag = false
+		}
+
+		// Parse entryLength.
+		data = segmentBytes[pos : pos+entryLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &entryLen); err != nil {
+			return errors.Wrap(err, "Read entry length fail to convert from byte")
+		}
+		pos += entryLength
+
+		// Mark entry end-position
+		entryEndPos = pos + int(entryLen)
+		if len(segmentBytes) < entryEndPos {
+			break
+		}
+
+		// Parse seriesIDLen.
+		data = segmentBytes[pos : pos+seriesIDLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &seriesIDLen); err != nil {
+			return errors.Wrap(err, "Read seriesID length fail to convert from byte")
+		}
+		pos += seriesIDLength
+
+		// Parse seriesID.
+		data = segmentBytes[pos : pos+int(seriesIDLen)]
+		seriesID = common.ParseSeriesIDV2(data)
+		pos += int(seriesIDLen)
+
+		// Parse series count.
+		data = segmentBytes[pos : pos+seriesCountLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &seriesCount); err != nil {
+			return errors.Wrap(err, "Read series count fail to convert from byte")
+		}
+		pos += seriesCountLength
+
+		// Parse timestamps compression binary.
+		data = segmentBytes[pos : pos+timestampsBinaryLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &timestampsBinaryLen); err != nil {
+			return errors.Wrap(err, "Read timestamps compression binary length fail to convert from byte")
+		}
+		pos += timestampsBinaryLength
+		data = segmentBytes[pos : pos+int(timestampsBinaryLen)]
+		timestampReader := encoding.NewReader(bytes.NewReader(data))
+		timestampDecoder := encoding.NewXORDecoder(timestampReader)
+		var timestamps []time.Time
+		for i := 0; i < int(seriesCount); i++ {
+			if !timestampDecoder.Next() {
+				return errors.Wrap(err, "Timestamps length not match series count, index: "+strconv.Itoa(i))
+			}
+			timestamps = append(timestamps, unixNanoToTime(timestampDecoder.Value()))
+		}
+		pos += int(timestampsBinaryLen)
+
+		// Parse values compression binary.
+		data = segmentBytes[pos:entryEndPos]
+		if data, err = snappy.Decode(nil, data); err != nil {
+			return errors.Wrap(err, "Decode values compression binary fail to snappy decode")
+		}
+		values := parseValuesBinary(data)
+		if values.Len() != len(timestamps) {
+			return errors.New("Timestamps length and values length not match: " + strconv.Itoa(len(timestamps)) + " vs " + strconv.Itoa(values.Len()))
+		}
+		pos = entryEndPos
+
+		logEntry := &logEntry{
+			entryLength: entryLen,
+			seriesID:    seriesID,
+			count:       seriesCount,
+			timestamps:  timestamps,
+			values:      values,
+		}
+		logEntries = append(logEntries, logEntry)
+
+		if pos == len(segmentBytes) {
+			break
+		}
+		if pos-oldPos == int(batchLen) {
+			parseNextBatchFlag = true
+		}
+	}
+	segment.logEntries = logEntries
+	return nil
+}
+
+func (logEntry *logEntry) GetSeriesID() common.SeriesIDV2 {
+	return logEntry.seriesID
+}
+
+func (logEntry *logEntry) GetTimestamps() []time.Time {
+	return logEntry.timestamps
+}
+
+func (logEntry *logEntry) GetValues() *list.List {
+	return logEntry.values
+}
+
+func (buffer *buffer) write(request logRequest) {
+	seriesID := request.seriesID
+	buffer.timestampMap[seriesID] = append(buffer.timestampMap[seriesID], request.timestamp)
+
+	// Value item: binary-length(2-bytes) + binary data(n-bytes)
+	binaryLength := int16ToBytes(int16(len(request.data)))
+	buffer.valueMap[seriesID] = append(buffer.valueMap[seriesID], binaryLength...)
+	buffer.valueMap[seriesID] = append(buffer.valueMap[seriesID], request.data...)
+
+	buffer.callbackMap[seriesID] = append(buffer.callbackMap[seriesID], request.callback)
+	buffer.count++
+}
+
+func (buffer *buffer) notifyRequests(err error) {
+	for seriesID, callbacks := range buffer.callbackMap {
+		timestamps := buffer.timestampMap[seriesID]
+		values := buffer.valueMap[seriesID]
+		valuePos := 0
+		var valueItem []byte
+		for index, callback := range callbacks {
+			valuePos, valueItem = readValuesBinary(values, valuePos, valuesBinaryLength)
+			tryCallback(func() {
+				callback(seriesID, timestamps[index], valueItem, err)
+			})
+		}
+	}
+}
+
+func segmentName(index uint64) string {
+	return fmt.Sprintf("%v%016x%v", segmentNamePrefix, index, segmentNameSuffix)
+}
+
+func tryCallback(callback func()) {
+	defer func() {
+		if err := recover(); err != nil {
+			fmt.Printf("Run callback error is %v\n", err)
+		}
+	}()
+	callback()
+}
+
+func parseValuesBinary(binary []byte) *list.List {
+	values := list.New()
+	position := 0
+	for {
+		nextPosition, value := readValuesBinary(binary, position, valuesBinaryLength)
+		if value == nil {
+			break
+		}
+		values.PushBack(value)
+		position = nextPosition
+	}
+	return values
+}
+
+func readValuesBinary(raw []byte, position int, offsetLen int) (int, []byte) {
+	if position == len(raw) {
+		return position, nil
+	}
+
+	data := raw[position : position+offsetLen]
+	binaryLen := bytesToInt16(data)
+	position += offsetLen
+
+	data = raw[position : position+int(binaryLen)]
+	position += int(binaryLen)
+	return position, data
+}
+
+func rewriteInt64InBuf(buf []byte, value int64, offset int, order binary.ByteOrder) {

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hailin0 commented on a diff in pull request #261: Implement Write-ahead Logging

Posted by "hailin0 (via GitHub)" <gi...@apache.org>.
hailin0 commented on code in PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#discussion_r1249551617


##########
pkg/wal/wal.go:
##########
@@ -42,19 +75,759 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// log implements the WAL interface.
+type log struct {
+	entryCloser      *run.Closer
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentMap       map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+	writeWaitGroup   sync.WaitGroup
+	flushWaitGroup   sync.WaitGroup
+	workSegmentMutex sync.Mutex
+	segmentMapMutex  sync.RWMutex
+	closerOnce       sync.Once
+}
+
+type segment struct {
+	file       *os.File
+	path       string
+	logEntries []LogEntry
+	segmentID  SegmentID
+}
+
+type logRequest struct {
+	seriesID  common.SeriesIDV2
+	timestamp time.Time
+	callback  func(common.SeriesIDV2, time.Time, []byte, error)
+	data      []byte
+}
+
+type logEntry struct {
+	timestamps  []time.Time
+	values      *list.List
+	seriesID    common.SeriesIDV2
+	entryLength int64
+	count       int32
+}
+
+type buffer struct {
+	timestampMap map[common.SeriesIDV2][]time.Time
+	valueMap     map[common.SeriesIDV2][]byte
+	callbackMap  map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)
+	count        int
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	walOptions := DefaultOptions
+	if options != nil {
+		fileSize := options.FileSize
+		if fileSize <= 0 {
+			fileSize = DefaultOptions.FileSize
+		}
+		bufferSize := options.BufferSize
+		if bufferSize <= 0 {
+			bufferSize = DefaultOptions.BufferSize
+		}
+		bufferBatchInterval := options.BufferBatchInterval
+		if bufferBatchInterval <= 0 {
+			bufferBatchInterval = DefaultOptions.BufferBatchInterval
+		}
+		walOptions = &Options{
+			FileSize:            fileSize,
+			BufferSize:          bufferSize,
+			BufferBatchInterval: bufferBatchInterval,
+		}
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path: "+path)
+	}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	log := &log{
+		path:             path,
+		options:          *walOptions,
+		logger:           logger.GetLogger(moduleName),
+		writeChannel:     make(chan logRequest),
+		flushChannel:     make(chan buffer, walOptions.FlushQueueSize),
+		bytesBuffer:      bytes.NewBuffer([]byte{}),
+		timestampsBuffer: bytes.NewBuffer([]byte{}),
+		entryCloser:      run.NewCloser(1),
+		buffer: buffer{
+			timestampMap: make(map[common.SeriesIDV2][]time.Time),
+			valueMap:     make(map[common.SeriesIDV2][]byte),
+			callbackMap:  make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)),
+			count:        0,
+		},
+	}
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+
+	log.writeWaitGroup.Add(1)
+	log.flushWaitGroup.Add(1)
+	log.start()
+
+	log.logger.Info().Str("path", path).Msg("WAL has be initialized")
+	return log, nil
+}
+
+// Write a logging entity.
+// It will return immediately when the data is written in the buffer,
+// The callback function will be called when the entity is flushed on the persistent storage.
+func (log *log) Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error)) {
+	if !log.entryCloser.AddRunning() {
+		return
+	}
+	defer log.entryCloser.Done()
+
+	log.writeChannel <- logRequest{
+		seriesID:  seriesID,
+		timestamp: timestamp,
+		data:      data,
+		callback:  callback,
+	}
+}
+
+// Read specified segment by SegmentID.
+func (log *log) Read(segmentID SegmentID) (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segment := log.segmentMap[segmentID]
+	return segment, nil
+}
+
+// ReadAllSegments reads all segments sorted by their creation time in ascending order.
+func (log *log) ReadAllSegments() ([]Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segments := make([]Segment, 0)
+	for _, segment := range log.segmentMap {
+		segments = append(segments, segment)
+	}
+	return segments, nil
+}
+
+// Rotate closes the open segment and opens a new one, returning the closed segment details.
+func (log *log) Rotate() (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	oldWorkSegment, err := log.swapWorkSegment()
+	if err != nil {
+		return nil, err
+	}
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Update segment information.
+	workSegment := log.workSegment
+	log.segmentMap[workSegment.segmentID] = workSegment
+	return oldWorkSegment, nil
+}
+
+// Delete the specified segment.
+func (log *log) Delete(segmentID SegmentID) error {
+	if !log.entryCloser.AddRunning() {
+		return errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Segment which will be deleted must be closed.
+	if segmentID == log.workSegment.segmentID {
+		return errors.New("Can not delete the segment which is working")
+	}
+
+	err := os.Remove(log.segmentMap[segmentID].path)
+	if err != nil {
+		return errors.Wrap(err, "Delete WAL segment error")
+	}
+	delete(log.segmentMap, segmentID)
+	return nil
+}
+
+// Close all of segments and stop WAL work.
+func (log *log) Close() error {
+	log.closerOnce.Do(func() {
+		log.logger.Info().Msg("Closing WAL...")
+
+		log.entryCloser.Done()
+		log.entryCloser.CloseThenWait()
+
+		close(log.writeChannel)
+		log.writeWaitGroup.Wait()
+
+		close(log.flushChannel)
+		log.flushWaitGroup.Wait()
+
+		if err := log.flushBuffer(log.buffer); err != nil {
+			log.logger.Err(err).Msg("Flushing buffer failed")
+		}
+		if err := log.workSegment.file.Close(); err != nil {
+			log.logger.Err(err).Msg("Close work segment file error")
+		}
+		log.logger.Info().Msg("Closed WAL")
+	})
+	return nil
+}
+
+func (log *log) start() {
+	go func() {
+		log.logger.Info().Msg("Start batch task...")
+
+		defer log.writeWaitGroup.Done()
+
+		bufferVolume := 0
+		for {
+			timer := time.NewTimer(log.options.BufferBatchInterval)
+			select {
+			case request, chOpen := <-log.writeChannel:
+				if !chOpen {
+					log.logger.Info().Msg("Stop batch task when write-channel closed!")
+					return
+				}
+
+				log.buffer.write(request)
+				if log.logger.Debug().Enabled() {
+					log.logger.Debug().Msg("Write request to buffer. elements: " + strconv.Itoa(log.buffer.count))
+				}
+
+				bufferVolume += len(request.seriesID.Marshal()) + timestampVolumeLength + len(request.data)
+				if bufferVolume > log.options.BufferSize {
+					log.triggerFlushing()
+					bufferVolume = 0
+				}
+				continue
+			case <-timer.C:
+				if bufferVolume == 0 {
+					continue
+				}
+				log.triggerFlushing()
+				bufferVolume = 0
+				continue
+			}
+		}
+	}()
+
+	go func() {
+		log.logger.Info().Msg("Start flush task...")
+
+		defer log.flushWaitGroup.Done()
+
+		for batch := range log.flushChannel {
+			startTime := time.Now()
+
+			var err error
+			for i := 0; i < maxRetries; i++ {
+				if err = log.flushBuffer(batch); err != nil {
+					log.logger.Err(err).Msg("Flushing buffer failed. Retrying...")
+					time.Sleep(100 * time.Millisecond)
+					continue
+				}
+				break
+			}
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Flushed buffer to WAL file. elements: " +
+					strconv.Itoa(batch.count) + ", cost: " + time.Since(startTime).String())
+			}
+
+			batch.notifyRequests(err)
+		}
+		log.logger.Info().Msg("Stop flush task when flush-channel closed!")
+	}()
+
+	log.logger.Info().Msg("Started WAL")
+}
+
+func (log *log) triggerFlushing() {
+	for {
+		select {
+		case log.flushChannel <- log.buffer:
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Send buffer to flush-channel. elements: " + strconv.Itoa(log.buffer.count))
+			}
+			log.newBuffer()
+			return
+		default:
+		}
+		time.Sleep(10 * time.Millisecond)
+	}
+}
+
+func (log *log) newBuffer() {
+	log.buffer.timestampMap = make(map[common.SeriesIDV2][]time.Time)
+	log.buffer.valueMap = make(map[common.SeriesIDV2][]byte)
+	log.buffer.callbackMap = make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error))
+	log.buffer.count = 0
+}
+
+func (log *log) flushBuffer(buffer buffer) error {
+	if buffer.count == 0 {
+		return nil
+	}
+
+	defer func() {
+		log.bytesBuffer.Reset()
+		log.timestampsBuffer.Reset()
+	}()
+
+	// placeholder, preset batch length value is 0
+	batchLen := 0
+	if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(batchLen)); err != nil {
+		return errors.Wrap(err, "Write batch length error")
+	}
+	for seriesID, timestamps := range buffer.timestampMap {
+		// Generate seriesID binary
+		seriesIDBytes := seriesID.Marshal()
+		seriesIDBytesLen := len(seriesIDBytes)
+
+		// Generate timestamps compression binary
+		log.timestampsBuffer.Reset()
+		timestampWriter := encoding.NewWriter()
+		timestampEncoder := encoding.NewXOREncoder(timestampWriter)
+		timestampWriter.Reset(log.timestampsBuffer)
+		for _, timestamp := range timestamps {
+			timestampEncoder.Write(timeTouUnixNano(timestamp))
+		}
+		timestampWriter.Flush()
+		timestampsBytes := log.timestampsBuffer.Bytes()
+		timestampsBytesLen := len(timestampsBytes)
+
+		// Generate values compression binary
+		valuesBytes := snappy.Encode(nil, buffer.valueMap[seriesID])
+
+		// Write entry data
+		entryLen := seriesIDLength + seriesIDBytesLen + seriesCountLength + timestampsBinaryLength + timestampsBytesLen + len(valuesBytes)
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(entryLen)); err != nil {
+			return errors.Wrap(err, "Write entry length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int16(seriesIDBytesLen)); err != nil {
+			return errors.Wrap(err, "Write seriesID length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, seriesIDBytes); err != nil {
+			return errors.Wrap(err, "Write seriesID error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int32(len(timestamps))); err != nil {
+			return errors.Wrap(err, "Write series count error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int16(timestampsBytesLen)); err != nil {
+			return errors.Wrap(err, "Write timestamps length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, timestampsBytes); err != nil {
+			return errors.Wrap(err, "Write timestamps error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, valuesBytes); err != nil {
+			return errors.Wrap(err, "Write values error")
+		}
+	}
+	// Rewrite batch length
+	batchBytes := log.bytesBuffer.Bytes()
+	batchLen = len(batchBytes) - batchLength
+	rewriteInt64InBuf(batchBytes, int64(batchLen), 0, binary.LittleEndian)
+
+	return log.writeWorkSegment(batchBytes)
+}
+
+func (log *log) swapWorkSegment() (Segment, error) {
+	log.workSegmentMutex.Lock()
+	defer log.workSegmentMutex.Unlock()
+
+	if err := log.workSegment.file.Close(); err != nil {
+		return nil, errors.Wrap(err, "Close WAL segment error")
+	}
+
+	oldSegment := log.workSegment
+	// Create new segment.
+	segmentID := log.workSegment.segmentID + 1
+	segment := &segment{
+		segmentID: segmentID,
+		path:      filepath.Join(log.path, segmentName(uint64(segmentID))),
+	}
+	var err error
+	segment.file, err = os.OpenFile(segment.path, os.O_CREATE|os.O_RDWR|os.O_TRUNC, os.ModePerm)
+	if err != nil {
+		return nil, errors.Wrap(err, "Open WAL segment error")
+	}
+	log.workSegment = segment
+
+	return oldSegment, nil
+}
+
+func (log *log) writeWorkSegment(data []byte) error {
+	log.workSegmentMutex.Lock()
+	defer log.workSegmentMutex.Unlock()
+
+	// Write batch data to WAL segment file
+	if _, err := log.workSegment.file.Write(data); err != nil {
+		return errors.Wrap(err, "Write WAL segment file error, file: "+log.workSegment.path)
+	}
+	if err := log.workSegment.file.Sync(); err != nil {
+		log.logger.Warn().Msg("Sync WAL segment file to disk failed, file: " + log.workSegment.path)
+	}
+	return nil
+}
+
+func (log *log) load() error {
+	files, err := os.ReadDir(log.path)
+	if err != nil {
+		return errors.Wrap(err, "Can not read dir: "+log.path)
+	}
+	// Load all of WAL segments.
+	var workSegmentID SegmentID
+	log.segmentMap = make(map[SegmentID]*segment)
+	for _, file := range files {
+		name := file.Name()
+		segmentID, parsePathErr := strconv.ParseUint(name[3:19], 10, 64)
+		if parsePathErr != nil {
+			return errors.Wrap(parsePathErr, "Parse file name error, name: "+name)
+		}
+		if segmentID > uint64(workSegmentID) {
+			workSegmentID = SegmentID(segmentID)
+		}
+		segment := &segment{
+			segmentID: SegmentID(segmentID),
+			path:      filepath.Join(log.path, segmentName(segmentID)),
+		}
+		if err = segment.parseLogEntries(); err != nil {
+			return errors.Wrap(err, "Fail to parse log entries")
+		}
+		log.segmentMap[SegmentID(segmentID)] = segment
+
+		if log.logger.Debug().Enabled() {
+			log.logger.Debug().Msg("Loaded segment file: " + segment.path)
+		}
+	}
+
+	// If load first time.
+	if len(log.segmentMap) == 0 {
+		segment := &segment{
+			segmentID: 1,
+			path:      filepath.Join(log.path, segmentName(1)),
+		}
+		log.segmentMap[1] = segment
+		log.workSegment = segment
+	} else {
+		log.workSegment = log.segmentMap[workSegmentID]
+	}
+	log.workSegment.file, err = os.OpenFile(log.workSegment.path, os.O_CREATE|os.O_RDWR|os.O_APPEND, os.ModePerm)
+	if err != nil {
+		return errors.Wrap(err, "Open WAL segment error, file: "+log.workSegment.path)
+	}
+	return nil
+}
+
+func (segment *segment) GetSegmentID() SegmentID {
+	return segment.segmentID
+}
+
+func (segment *segment) GetLogEntries() []LogEntry {
+	return segment.logEntries
+}
+
+func (segment *segment) parseLogEntries() error {
+	segmentBytes, err := os.ReadFile(segment.path)
+	if err != nil {
+		return errors.Wrap(err, "Read WAL segment failed, path: "+segment.path)
+	}
+
+	var logEntries []LogEntry
+	var data []byte
+	var batchLen int64
+	var entryLen int64
+	var seriesIDLen int16
+	var seriesID common.SeriesIDV2
+	var seriesCount int32
+	var timestampsBinaryLen int16
+	var entryEndPos int
+	var bytesBuf *bytes.Buffer
+
+	oldPos := 0
+	pos := 0
+	parseNextBatchFlag := true
+
+	for {
+		if parseNextBatchFlag {
+			if len(segmentBytes) <= batchLength {
+				break
+			}
+			data = segmentBytes[pos : pos+batchLength]
+			bytesBuf = bytes.NewBuffer(data)
+			if err = binary.Read(bytesBuf, binary.LittleEndian, &batchLen); err != nil {
+				return errors.Wrap(err, "Read batch length fail to convert from bytes")
+			}
+			if len(segmentBytes) <= int(batchLen) {
+				break
+			}
+
+			pos += batchLength
+			oldPos = pos
+			parseNextBatchFlag = false
+		}
+
+		// Parse entryLength.
+		data = segmentBytes[pos : pos+entryLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &entryLen); err != nil {
+			return errors.Wrap(err, "Read entry length fail to convert from byte")
+		}
+		pos += entryLength
+
+		// Mark entry end-position
+		entryEndPos = pos + int(entryLen)
+		if len(segmentBytes) < entryEndPos {
+			break
+		}
+
+		// Parse seriesIDLen.
+		data = segmentBytes[pos : pos+seriesIDLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &seriesIDLen); err != nil {
+			return errors.Wrap(err, "Read seriesID length fail to convert from byte")
+		}
+		pos += seriesIDLength
+
+		// Parse seriesID.
+		data = segmentBytes[pos : pos+int(seriesIDLen)]
+		seriesID = common.ParseSeriesIDV2(data)
+		pos += int(seriesIDLen)
+
+		// Parse series count.
+		data = segmentBytes[pos : pos+seriesCountLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &seriesCount); err != nil {
+			return errors.Wrap(err, "Read series count fail to convert from byte")
+		}
+		pos += seriesCountLength
+
+		// Parse timestamps compression binary.
+		data = segmentBytes[pos : pos+timestampsBinaryLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &timestampsBinaryLen); err != nil {
+			return errors.Wrap(err, "Read timestamps compression binary length fail to convert from byte")
+		}
+		pos += timestampsBinaryLength
+		data = segmentBytes[pos : pos+int(timestampsBinaryLen)]
+		timestampReader := encoding.NewReader(bytes.NewReader(data))
+		timestampDecoder := encoding.NewXORDecoder(timestampReader)
+		var timestamps []time.Time
+		for i := 0; i < int(seriesCount); i++ {
+			if !timestampDecoder.Next() {
+				return errors.Wrap(err, "Timestamps length not match series count, index: "+strconv.Itoa(i))
+			}
+			timestamps = append(timestamps, unixNanoToTime(timestampDecoder.Value()))
+		}
+		pos += int(timestampsBinaryLen)
+
+		// Parse values compression binary.
+		data = segmentBytes[pos:entryEndPos]
+		if data, err = snappy.Decode(nil, data); err != nil {
+			return errors.Wrap(err, "Decode values compression binary fail to snappy decode")
+		}
+		values := parseValuesBinary(data)
+		if values.Len() != len(timestamps) {
+			return errors.New("Timestamps length and values length not match: " + strconv.Itoa(len(timestamps)) + " vs " + strconv.Itoa(values.Len()))
+		}
+		pos = entryEndPos
+
+		logEntry := &logEntry{
+			entryLength: entryLen,
+			seriesID:    seriesID,
+			count:       seriesCount,
+			timestamps:  timestamps,
+			values:      values,
+		}
+		logEntries = append(logEntries, logEntry)
+
+		if pos == len(segmentBytes) {
+			break
+		}
+		if pos-oldPos == int(batchLen) {
+			parseNextBatchFlag = true
+		}
+	}
+	segment.logEntries = logEntries
+	return nil
+}
+
+func (logEntry *logEntry) GetSeriesID() common.SeriesIDV2 {
+	return logEntry.seriesID
+}
+
+func (logEntry *logEntry) GetTimestamps() []time.Time {
+	return logEntry.timestamps
+}
+
+func (logEntry *logEntry) GetValues() *list.List {
+	return logEntry.values
+}
+
+func (buffer *buffer) write(request logRequest) {
+	seriesID := request.seriesID
+	buffer.timestampMap[seriesID] = append(buffer.timestampMap[seriesID], request.timestamp)
+
+	// Value item: binary-length(2-bytes) + binary data(n-bytes)
+	binaryLength := int16ToBytes(int16(len(request.data)))
+	buffer.valueMap[seriesID] = append(buffer.valueMap[seriesID], binaryLength...)
+	buffer.valueMap[seriesID] = append(buffer.valueMap[seriesID], request.data...)
+
+	buffer.callbackMap[seriesID] = append(buffer.callbackMap[seriesID], request.callback)
+	buffer.count++
+}
+
+func (buffer *buffer) notifyRequests(err error) {
+	for seriesID, callbacks := range buffer.callbackMap {
+		timestamps := buffer.timestampMap[seriesID]
+		values := buffer.valueMap[seriesID]
+		valuePos := 0
+		var valueItem []byte
+		for index, callback := range callbacks {
+			valuePos, valueItem = readValuesBinary(values, valuePos, valuesBinaryLength)
+			tryCallback(func() {
+				callback(seriesID, timestamps[index], valueItem, err)
+			})
+		}
+	}
+}
+
+func segmentName(index uint64) string {
+	return fmt.Sprintf("%v%016x%v", segmentNamePrefix, index, segmentNameSuffix)
+}
+
+func tryCallback(callback func()) {
+	defer func() {
+		if err := recover(); err != nil {
+			fmt.Printf("Run callback error is %v\n", err)
+		}
+	}()
+	callback()
+}
+
+func parseValuesBinary(binary []byte) *list.List {
+	values := list.New()
+	position := 0
+	for {
+		nextPosition, value := readValuesBinary(binary, position, valuesBinaryLength)
+		if value == nil {
+			break
+		}
+		values.PushBack(value)
+		position = nextPosition
+	}
+	return values
+}
+
+func readValuesBinary(raw []byte, position int, offsetLen int) (int, []byte) {
+	if position == len(raw) {
+		return position, nil
+	}
+
+	data := raw[position : position+offsetLen]
+	binaryLen := bytesToInt16(data)
+	position += offsetLen
+
+	data = raw[position : position+int(binaryLen)]
+	position += int(binaryLen)
+	return position, data
+}
+
+func rewriteInt64InBuf(buf []byte, value int64, offset int, order binary.ByteOrder) {
+	_ = buf[offset+7] // early bounds check to guarantee safety of writes below
+	if order == binary.LittleEndian {
+		buf[offset+0] = byte(value)
+		buf[offset+1] = byte(value >> 8)
+		buf[offset+2] = byte(value >> 16)
+		buf[offset+3] = byte(value >> 24)
+		buf[offset+4] = byte(value >> 32)
+		buf[offset+5] = byte(value >> 40)
+		buf[offset+6] = byte(value >> 48)
+		buf[offset+7] = byte(value >> 56)
+	} else {
+		buf[offset+0] = byte(value >> 56)
+		buf[offset+1] = byte(value >> 48)
+		buf[offset+2] = byte(value >> 40)
+		buf[offset+3] = byte(value >> 32)
+		buf[offset+4] = byte(value >> 24)
+		buf[offset+5] = byte(value >> 16)
+		buf[offset+6] = byte(value >> 8)
+		buf[offset+7] = byte(value)
+	}
+}
+
+func int16ToBytes(i int16) []byte {
+	buf := make([]byte, 2)
+	binary.LittleEndian.PutUint16(buf, uint16(i))
+	return buf
+}
+
+func bytesToInt16(buf []byte) int16 {
+	return int16(binary.LittleEndian.Uint16(buf))
+}
+
+func timeTouUnixNano(time time.Time) uint64 {

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hanahmily commented on a diff in pull request #261: Implement Write-ahead Logging

Posted by "hanahmily (via GitHub)" <gi...@apache.org>.
hanahmily commented on code in PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#discussion_r1241182783


##########
pkg/wal/wal.go:
##########
@@ -42,19 +72,642 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// Log implements the WAL interface.
+type Log struct {
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentIndexMap  map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+}
+
+type segment struct {
+	file       *os.File
+	path       string
+	logEntries []LogEntry
+	segmentID  SegmentID
+}
+
+type logRequest struct {
+	seriesID  common.SeriesIDV2
+	timestamp time.Time
+	callback  func(common.SeriesIDV2, time.Time, []byte, error)
+	data      []byte
+}
+
+type logEntry struct {
+	timestamps  []time.Time
+	values      *list.List
+	seriesID    common.SeriesIDV2
+	entryLength int64
+	count       int32
+}
+
+type buffer struct {
+	timestampMap map[common.SeriesIDV2][]time.Time
+	valueMap     map[common.SeriesIDV2][]byte
+	callbackMap  map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)
+	count        int
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	if options == nil {
+		options = DefaultOptions
+	}
+	if options.FileSize <= 0 {
+		options.FileSize = DefaultOptions.FileSize
+	}
+	if options.BufferSize <= 0 {
+		options.BufferSize = DefaultOptions.BufferSize
+	}
+	if options.BufferBatchInterval <= 0 {
+		options.BufferBatchInterval = DefaultOptions.BufferBatchInterval
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path: "+path)
+	}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	log := &Log{path: path, options: *options, logger: logger.GetLogger(moduleName)}
+
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+	log.startAsyncFlushTask()
+
+	log.logger.Info().Msgf("WAL initialized at %s", path)
+	return log, nil
+}
+
+// Write a logging entity.
+// It will return immediately when the data is written in the buffer,
+// The callback function will be called when the entity is flushed on the persistent storage.
+func (log *Log) Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error)) {
+	log.writeChannel <- logRequest{
+		seriesID:  seriesID,
+		timestamp: timestamp,
+		data:      data,
+		callback:  callback,
+	}
+}
+
+// Read specified segment by SegmentID.
+func (log *Log) Read(segmentID SegmentID) (Segment, error) {
+	segment := log.segmentIndexMap[segmentID]
+	return segment, nil
+}
+
+// ReadAllSegments reads all segments sorted by their creation time in ascending order.
+func (log *Log) ReadAllSegments() ([]Segment, error) {
+	segments := make([]Segment, 0)
+	for _, segment := range log.segmentIndexMap {
+		segments = append(segments, segment)
+	}
+	return segments, nil
+}
+
+// Rotate closes the open segment and opens a new one, returning the closed segment details.
+func (log *Log) Rotate() (Segment, error) {
+	if err := log.workSegment.file.Close(); err != nil {
+		return nil, errors.Wrap(err, "Close WAL segment error")
+	}
+	oldSegment := log.workSegment
+	// Create new segment.
+	segmentID := log.workSegment.segmentID + 1
+	segment := &segment{
+		segmentID: segmentID,
+		path:      filepath.Join(log.path, segmentName(uint64(segmentID))),
+	}
+	var err error
+	segment.file, err = os.OpenFile(segment.path, os.O_CREATE|os.O_RDWR|os.O_TRUNC, os.ModePerm)
+	if err != nil {
+		return nil, errors.Wrap(err, "Open WAL segment error")
+	}
+
+	// Update segment information.
+	log.segmentIndexMap[segmentID] = segment
+	log.workSegment = segment
+	return oldSegment, nil
+}
+
+// Delete the specified segment.
+func (log *Log) Delete(segmentID SegmentID) error {
+	// Segment which will be deleted must be closed.
+	err := os.Remove(log.segmentIndexMap[segmentID].path)
+	if err != nil {
+		return errors.Wrap(err, "Delete WAL segment error")
+	}
+	delete(log.segmentIndexMap, segmentID)
+	return nil
+}
+
+// Close all of segments and stop WAL work.
+func (log *Log) Close() error {
+	log.logger.Info().Msg("Closing WAL...")
+	close(log.writeChannel)
+	close(log.flushChannel)
+	err := log.workSegment.file.Close()
+	if err != nil {
+		return errors.Wrap(err, "Close WAL error")
+	}
+	log.logger.Info().Msg("Closed WAL")
+	return nil
+}
+
+func (log *Log) startAsyncFlushTask() {
+	go func() {
+		log.logger.Info().Msg("Start batch task...")
+		bufferSize := 0
+		for {
+			timer := time.NewTimer(log.options.BufferBatchInterval)
+			select {
+			case request, ok := <-log.writeChannel:
+				if !ok {
+					log.logger.Info().Msg("Exit selector when write-channel closed!")
+					return
+				}
+				log.buffer.write(request)
+				if log.logger.Debug().Enabled() {
+					log.logger.Debug().Msg("Write request to buffer. elements: " + strconv.Itoa(log.buffer.count))
+				}
+
+				bufferSize += len(request.seriesID.Marshal()) + timestampLength + len(request.data)
+				if bufferSize > log.options.BufferSize {
+					// Clone buffer,avoiding to block write.
+					buf := log.buffer
+					// Clear buffer to receive Log request.
+					log.newBuffer()
+					bufferSize = 0
+					// Send buffer to flushBuffer channel.
+					log.flushChannel <- buf
+					if log.logger.Debug().Enabled() {
+						log.logger.Debug().Msg("Send buffer to flush-channel. elements: " + strconv.Itoa(buf.count))
+					}
+				}
+			case <-timer.C:
+				if bufferSize == 0 {
+					continue
+				}
+				// Clone buffer,avoiding to block write.
+				buf := log.buffer
+				// Clear buffer to receive Log request.
+				log.newBuffer()
+				bufferSize = 0
+				// Send buffer to flushBuffer channel.
+				log.flushChannel <- buf
+				if log.logger.Debug().Enabled() {
+					log.logger.Debug().Msg("Send buffer to flush-channel. elements: " + strconv.Itoa(buf.count))
+				}
+			}
+		}
+	}()
+
+	go func() {
+		log.logger.Info().Msg("Start flush task...")
+
+		for batch := range log.flushChannel {
+			log.flushBuffer(batch)
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Flushed buffer to WAL file. elements: " + strconv.Itoa(batch.count))
+			}
+		}
+
+		log.logger.Info().Msg("Exit flush task when flush-channel closed!")
+	}()
+
+	log.logger.Info().Msg("Started WAL async flush task.")
+}
+
+func (log *Log) flushBuffer(buffer buffer) {
+	log.bytesBuffer.Reset()
+	// placeholder, preset batch length value is 0
+	batchLength := 0
+	log.writeToBytesBuffer(log.bytesBuffer, binary.LittleEndian, int64(batchLength))
+
+	for seriesID, timestamps := range buffer.timestampMap {
+		// Generate seriesID binary
+		seriesIDBytes := seriesID.Marshal()
+		seriesIDBytesLen := len(seriesIDBytes)
+
+		// Generate timestamps compression binary
+		log.timestampsBuffer.Reset()
+		timestampWriter := encoding.NewWriter()
+		timestampEncoder := encoding.NewXOREncoder(timestampWriter)
+		timestampWriter.Reset(log.timestampsBuffer)
+		for _, timestamp := range timestamps {
+			timestampEncoder.Write(timeTouUnixNano(timestamp))
+		}
+		timestampWriter.Flush()
+		timestampsBytes := log.timestampsBuffer.Bytes()
+		timestampsBytesLen := len(timestampsBytes)
+
+		// Generate values compression binary
+		valuesBytes := snappy.Encode(nil, buffer.valueMap[seriesID])
+
+		// Write entry data
+		entryLength := seriesIDLength + seriesIDBytesLen + seriesCountLength + timestampsBinaryLength + timestampsBytesLen + len(valuesBytes)
+		log.writeToBytesBuffer(log.bytesBuffer, binary.LittleEndian, int64(entryLength))
+		log.writeToBytesBuffer(log.bytesBuffer, binary.LittleEndian, int16(seriesIDBytesLen))
+		log.writeToBytesBuffer(log.bytesBuffer, binary.LittleEndian, seriesIDBytes)
+		log.writeToBytesBuffer(log.bytesBuffer, binary.LittleEndian, int32(len(timestamps)))
+		log.writeToBytesBuffer(log.bytesBuffer, binary.LittleEndian, int16(timestampsBytesLen))
+		log.writeToBytesBuffer(log.bytesBuffer, binary.LittleEndian, timestampsBytes)
+		log.writeToBytesBuffer(log.bytesBuffer, binary.LittleEndian, valuesBytes)
+	}
+	// Rewrite batch length
+	batchBytes := log.bytesBuffer.Bytes()
+	batchLength = len(batchBytes) - batchWriteLength
+	rewriteInt64InBuf(batchBytes, int64(batchLength), 0, binary.LittleEndian)
+	log.bytesBuffer.Reset()
+
+	// Flush
+	_, err := log.workSegment.file.Write(batchBytes)
+	if err != nil {
+		log.logger.Error().Err(err).Msg("Write WAL segment file error, file: " + log.workSegment.path)
+		buffer.notifyRequests(err)
+		return
+	}
+	err = log.workSegment.file.Sync()
+	if err != nil {
+		log.logger.Error().Err(err).Msg("Sync WAL segment file to disk error, file: " + log.workSegment.path)
+		buffer.notifyRequests(err)
+		return
+	}
+	buffer.notifyRequests(nil)
+	if log.logger.Debug().Enabled() {
+		log.logger.Debug().Msg("Flushed buffer to WAL. file: " + log.workSegment.path +
+			", elements: " + strconv.Itoa(buffer.count) +
+			", bytes: " + strconv.Itoa(len(batchBytes)))
+	}
+}
+
+func (log *Log) newBuffer() {
+	log.buffer.timestampMap = make(map[common.SeriesIDV2][]time.Time)
+	log.buffer.valueMap = make(map[common.SeriesIDV2][]byte)
+	log.buffer.callbackMap = make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error))
+	log.buffer.count = 0
+}
+
+func (log *Log) load() error {
+	files, err := os.ReadDir(log.path)
+	if err != nil {
+		return errors.Wrap(err, "Can not read dir: "+log.path)
+	}
+	// Load all of WAL segments.
+	var workSegmentID SegmentID
+	log.segmentIndexMap = make(map[SegmentID]*segment)
+	for _, file := range files {

Review Comment:
   During the booting process, it is necessary to remove all files except for the `workSegment`. 
   
   To create a new `workSegment`, the user should rotate the WAL and delete the previous one. In case the deletion fails due to a process crash, the `load` function will take care of cleaning up all non-work segments.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hanahmily commented on pull request #261: Implement Write-ahead Logging

Posted by "hanahmily (via GitHub)" <gi...@apache.org>.
hanahmily commented on PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#issuecomment-1480423776

   > I have two channel field in the` Log` struct, should I move them into `buffer` struct?
   
   Make sense to me. I submitted the `Buffer` patch(#262), in which you could find a similar structure. I hope you could get some inspiration from it.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hanahmily commented on pull request #261: Implement Write-ahead Logging

Posted by "hanahmily (via GitHub)" <gi...@apache.org>.
hanahmily commented on PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#issuecomment-1475448566

   > There are some things that need help:
   > 
   > * During disk flushing, in order to avoid blocking writes, I cloned the buffer so that I can continue to accept write requests while flushing the disk. However, during asynchronous callbacks, all channels will be notified. At this time, write requests that only write to the buffer should not receive this request, but  they receive it.how can I resolve this issue?
   
   You should have switched to a new flushChannel on switching the buffer. Theoretically, a flush channel sends a notice which indicates its corresponding buffer gets flushed. Based on that, the flush channel seems a field in the `buffer` structure instead.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hanahmily commented on pull request #261: Implement Write-ahead Logging

Posted by "hanahmily (via GitHub)" <gi...@apache.org>.
hanahmily commented on PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#issuecomment-1475448718

   @HHoflittlefish777 Please fix test failures.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hanahmily commented on a diff in pull request #261: Implement Write-ahead Logging

Posted by "hanahmily (via GitHub)" <gi...@apache.org>.
hanahmily commented on code in PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#discussion_r1252874619


##########
CHANGES.md:
##########
@@ -27,6 +27,7 @@ Release Notes.
 - [UI] Add measure CRUD operations.
 - [UI] Add indexRule CRUD operations.
 - [UI] Add indexRuleBinding CRUD operations.
+- Implement Write-ahead Logging

Review Comment:
   Move it to 0.5.0



##########
api/common/id.go:
##########
@@ -40,6 +40,27 @@ func (s SeriesID) Marshal() []byte {
 	return convert.Uint64ToBytes(uint64(s))
 }
 
+// SeriesIDV2 identities a series in a shard.

Review Comment:
   We should name it to GlobalSeriesID



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hailin0 commented on a diff in pull request #261: Implement Write-ahead Logging

Posted by "hailin0 (via GitHub)" <gi...@apache.org>.
hailin0 commented on code in PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#discussion_r1249551683


##########
pkg/wal/wal.go:
##########
@@ -42,19 +75,759 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// log implements the WAL interface.
+type log struct {
+	entryCloser      *run.Closer
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentMap       map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+	writeWaitGroup   sync.WaitGroup
+	flushWaitGroup   sync.WaitGroup
+	workSegmentMutex sync.Mutex
+	segmentMapMutex  sync.RWMutex
+	closerOnce       sync.Once
+}
+
+type segment struct {
+	file       *os.File
+	path       string
+	logEntries []LogEntry
+	segmentID  SegmentID
+}
+
+type logRequest struct {
+	seriesID  common.SeriesIDV2
+	timestamp time.Time
+	callback  func(common.SeriesIDV2, time.Time, []byte, error)
+	data      []byte
+}
+
+type logEntry struct {
+	timestamps  []time.Time
+	values      *list.List
+	seriesID    common.SeriesIDV2
+	entryLength int64
+	count       int32
+}
+
+type buffer struct {
+	timestampMap map[common.SeriesIDV2][]time.Time
+	valueMap     map[common.SeriesIDV2][]byte
+	callbackMap  map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)
+	count        int
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	walOptions := DefaultOptions
+	if options != nil {
+		fileSize := options.FileSize
+		if fileSize <= 0 {
+			fileSize = DefaultOptions.FileSize
+		}
+		bufferSize := options.BufferSize
+		if bufferSize <= 0 {
+			bufferSize = DefaultOptions.BufferSize
+		}
+		bufferBatchInterval := options.BufferBatchInterval
+		if bufferBatchInterval <= 0 {
+			bufferBatchInterval = DefaultOptions.BufferBatchInterval
+		}
+		walOptions = &Options{
+			FileSize:            fileSize,
+			BufferSize:          bufferSize,
+			BufferBatchInterval: bufferBatchInterval,
+		}
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path: "+path)
+	}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	log := &log{
+		path:             path,
+		options:          *walOptions,
+		logger:           logger.GetLogger(moduleName),
+		writeChannel:     make(chan logRequest),
+		flushChannel:     make(chan buffer, walOptions.FlushQueueSize),
+		bytesBuffer:      bytes.NewBuffer([]byte{}),
+		timestampsBuffer: bytes.NewBuffer([]byte{}),
+		entryCloser:      run.NewCloser(1),
+		buffer: buffer{
+			timestampMap: make(map[common.SeriesIDV2][]time.Time),
+			valueMap:     make(map[common.SeriesIDV2][]byte),
+			callbackMap:  make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)),
+			count:        0,
+		},
+	}
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+
+	log.writeWaitGroup.Add(1)
+	log.flushWaitGroup.Add(1)
+	log.start()
+
+	log.logger.Info().Str("path", path).Msg("WAL has be initialized")
+	return log, nil
+}
+
+// Write a logging entity.
+// It will return immediately when the data is written in the buffer,
+// The callback function will be called when the entity is flushed on the persistent storage.
+func (log *log) Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error)) {
+	if !log.entryCloser.AddRunning() {
+		return
+	}
+	defer log.entryCloser.Done()
+
+	log.writeChannel <- logRequest{
+		seriesID:  seriesID,
+		timestamp: timestamp,
+		data:      data,
+		callback:  callback,
+	}
+}
+
+// Read specified segment by SegmentID.
+func (log *log) Read(segmentID SegmentID) (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segment := log.segmentMap[segmentID]
+	return segment, nil
+}
+
+// ReadAllSegments reads all segments sorted by their creation time in ascending order.
+func (log *log) ReadAllSegments() ([]Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segments := make([]Segment, 0)
+	for _, segment := range log.segmentMap {
+		segments = append(segments, segment)
+	}
+	return segments, nil
+}
+
+// Rotate closes the open segment and opens a new one, returning the closed segment details.
+func (log *log) Rotate() (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	oldWorkSegment, err := log.swapWorkSegment()
+	if err != nil {
+		return nil, err
+	}
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Update segment information.
+	workSegment := log.workSegment
+	log.segmentMap[workSegment.segmentID] = workSegment
+	return oldWorkSegment, nil
+}
+
+// Delete the specified segment.
+func (log *log) Delete(segmentID SegmentID) error {
+	if !log.entryCloser.AddRunning() {
+		return errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Segment which will be deleted must be closed.
+	if segmentID == log.workSegment.segmentID {
+		return errors.New("Can not delete the segment which is working")
+	}
+
+	err := os.Remove(log.segmentMap[segmentID].path)
+	if err != nil {
+		return errors.Wrap(err, "Delete WAL segment error")
+	}
+	delete(log.segmentMap, segmentID)
+	return nil
+}
+
+// Close all of segments and stop WAL work.
+func (log *log) Close() error {
+	log.closerOnce.Do(func() {
+		log.logger.Info().Msg("Closing WAL...")
+
+		log.entryCloser.Done()
+		log.entryCloser.CloseThenWait()
+
+		close(log.writeChannel)
+		log.writeWaitGroup.Wait()
+
+		close(log.flushChannel)
+		log.flushWaitGroup.Wait()
+
+		if err := log.flushBuffer(log.buffer); err != nil {
+			log.logger.Err(err).Msg("Flushing buffer failed")
+		}
+		if err := log.workSegment.file.Close(); err != nil {
+			log.logger.Err(err).Msg("Close work segment file error")
+		}
+		log.logger.Info().Msg("Closed WAL")
+	})
+	return nil
+}
+
+func (log *log) start() {
+	go func() {
+		log.logger.Info().Msg("Start batch task...")
+
+		defer log.writeWaitGroup.Done()
+
+		bufferVolume := 0
+		for {
+			timer := time.NewTimer(log.options.BufferBatchInterval)
+			select {
+			case request, chOpen := <-log.writeChannel:
+				if !chOpen {
+					log.logger.Info().Msg("Stop batch task when write-channel closed!")
+					return
+				}
+
+				log.buffer.write(request)
+				if log.logger.Debug().Enabled() {
+					log.logger.Debug().Msg("Write request to buffer. elements: " + strconv.Itoa(log.buffer.count))
+				}
+
+				bufferVolume += len(request.seriesID.Marshal()) + timestampVolumeLength + len(request.data)
+				if bufferVolume > log.options.BufferSize {
+					log.triggerFlushing()
+					bufferVolume = 0
+				}
+				continue
+			case <-timer.C:
+				if bufferVolume == 0 {
+					continue
+				}
+				log.triggerFlushing()
+				bufferVolume = 0
+				continue
+			}
+		}
+	}()
+
+	go func() {
+		log.logger.Info().Msg("Start flush task...")
+
+		defer log.flushWaitGroup.Done()
+
+		for batch := range log.flushChannel {
+			startTime := time.Now()
+
+			var err error
+			for i := 0; i < maxRetries; i++ {
+				if err = log.flushBuffer(batch); err != nil {
+					log.logger.Err(err).Msg("Flushing buffer failed. Retrying...")
+					time.Sleep(100 * time.Millisecond)
+					continue
+				}
+				break
+			}
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Flushed buffer to WAL file. elements: " +
+					strconv.Itoa(batch.count) + ", cost: " + time.Since(startTime).String())
+			}
+
+			batch.notifyRequests(err)
+		}
+		log.logger.Info().Msg("Stop flush task when flush-channel closed!")
+	}()
+
+	log.logger.Info().Msg("Started WAL")
+}
+
+func (log *log) triggerFlushing() {
+	for {
+		select {
+		case log.flushChannel <- log.buffer:
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Send buffer to flush-channel. elements: " + strconv.Itoa(log.buffer.count))
+			}
+			log.newBuffer()
+			return
+		default:
+		}
+		time.Sleep(10 * time.Millisecond)
+	}
+}
+
+func (log *log) newBuffer() {
+	log.buffer.timestampMap = make(map[common.SeriesIDV2][]time.Time)
+	log.buffer.valueMap = make(map[common.SeriesIDV2][]byte)
+	log.buffer.callbackMap = make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error))
+	log.buffer.count = 0
+}
+
+func (log *log) flushBuffer(buffer buffer) error {
+	if buffer.count == 0 {
+		return nil
+	}
+
+	defer func() {
+		log.bytesBuffer.Reset()
+		log.timestampsBuffer.Reset()
+	}()
+
+	// placeholder, preset batch length value is 0
+	batchLen := 0
+	if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(batchLen)); err != nil {
+		return errors.Wrap(err, "Write batch length error")
+	}
+	for seriesID, timestamps := range buffer.timestampMap {
+		// Generate seriesID binary
+		seriesIDBytes := seriesID.Marshal()
+		seriesIDBytesLen := len(seriesIDBytes)
+
+		// Generate timestamps compression binary
+		log.timestampsBuffer.Reset()
+		timestampWriter := encoding.NewWriter()
+		timestampEncoder := encoding.NewXOREncoder(timestampWriter)
+		timestampWriter.Reset(log.timestampsBuffer)
+		for _, timestamp := range timestamps {
+			timestampEncoder.Write(timeTouUnixNano(timestamp))
+		}
+		timestampWriter.Flush()
+		timestampsBytes := log.timestampsBuffer.Bytes()
+		timestampsBytesLen := len(timestampsBytes)
+
+		// Generate values compression binary
+		valuesBytes := snappy.Encode(nil, buffer.valueMap[seriesID])
+
+		// Write entry data
+		entryLen := seriesIDLength + seriesIDBytesLen + seriesCountLength + timestampsBinaryLength + timestampsBytesLen + len(valuesBytes)
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(entryLen)); err != nil {
+			return errors.Wrap(err, "Write entry length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int16(seriesIDBytesLen)); err != nil {
+			return errors.Wrap(err, "Write seriesID length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, seriesIDBytes); err != nil {
+			return errors.Wrap(err, "Write seriesID error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int32(len(timestamps))); err != nil {
+			return errors.Wrap(err, "Write series count error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int16(timestampsBytesLen)); err != nil {
+			return errors.Wrap(err, "Write timestamps length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, timestampsBytes); err != nil {
+			return errors.Wrap(err, "Write timestamps error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, valuesBytes); err != nil {
+			return errors.Wrap(err, "Write values error")
+		}
+	}
+	// Rewrite batch length
+	batchBytes := log.bytesBuffer.Bytes()
+	batchLen = len(batchBytes) - batchLength
+	rewriteInt64InBuf(batchBytes, int64(batchLen), 0, binary.LittleEndian)
+
+	return log.writeWorkSegment(batchBytes)
+}
+
+func (log *log) swapWorkSegment() (Segment, error) {
+	log.workSegmentMutex.Lock()
+	defer log.workSegmentMutex.Unlock()
+
+	if err := log.workSegment.file.Close(); err != nil {
+		return nil, errors.Wrap(err, "Close WAL segment error")
+	}
+
+	oldSegment := log.workSegment
+	// Create new segment.
+	segmentID := log.workSegment.segmentID + 1
+	segment := &segment{
+		segmentID: segmentID,
+		path:      filepath.Join(log.path, segmentName(uint64(segmentID))),
+	}
+	var err error
+	segment.file, err = os.OpenFile(segment.path, os.O_CREATE|os.O_RDWR|os.O_TRUNC, os.ModePerm)
+	if err != nil {
+		return nil, errors.Wrap(err, "Open WAL segment error")
+	}
+	log.workSegment = segment
+
+	return oldSegment, nil
+}
+
+func (log *log) writeWorkSegment(data []byte) error {
+	log.workSegmentMutex.Lock()
+	defer log.workSegmentMutex.Unlock()
+
+	// Write batch data to WAL segment file
+	if _, err := log.workSegment.file.Write(data); err != nil {
+		return errors.Wrap(err, "Write WAL segment file error, file: "+log.workSegment.path)
+	}
+	if err := log.workSegment.file.Sync(); err != nil {
+		log.logger.Warn().Msg("Sync WAL segment file to disk failed, file: " + log.workSegment.path)
+	}
+	return nil
+}
+
+func (log *log) load() error {
+	files, err := os.ReadDir(log.path)
+	if err != nil {
+		return errors.Wrap(err, "Can not read dir: "+log.path)
+	}
+	// Load all of WAL segments.
+	var workSegmentID SegmentID
+	log.segmentMap = make(map[SegmentID]*segment)
+	for _, file := range files {
+		name := file.Name()
+		segmentID, parsePathErr := strconv.ParseUint(name[3:19], 10, 64)
+		if parsePathErr != nil {
+			return errors.Wrap(parsePathErr, "Parse file name error, name: "+name)
+		}
+		if segmentID > uint64(workSegmentID) {
+			workSegmentID = SegmentID(segmentID)
+		}
+		segment := &segment{
+			segmentID: SegmentID(segmentID),
+			path:      filepath.Join(log.path, segmentName(segmentID)),
+		}
+		if err = segment.parseLogEntries(); err != nil {
+			return errors.Wrap(err, "Fail to parse log entries")
+		}
+		log.segmentMap[SegmentID(segmentID)] = segment
+
+		if log.logger.Debug().Enabled() {
+			log.logger.Debug().Msg("Loaded segment file: " + segment.path)
+		}
+	}
+
+	// If load first time.
+	if len(log.segmentMap) == 0 {
+		segment := &segment{
+			segmentID: 1,
+			path:      filepath.Join(log.path, segmentName(1)),
+		}
+		log.segmentMap[1] = segment
+		log.workSegment = segment
+	} else {
+		log.workSegment = log.segmentMap[workSegmentID]
+	}
+	log.workSegment.file, err = os.OpenFile(log.workSegment.path, os.O_CREATE|os.O_RDWR|os.O_APPEND, os.ModePerm)
+	if err != nil {
+		return errors.Wrap(err, "Open WAL segment error, file: "+log.workSegment.path)
+	}
+	return nil
+}
+
+func (segment *segment) GetSegmentID() SegmentID {
+	return segment.segmentID
+}
+
+func (segment *segment) GetLogEntries() []LogEntry {
+	return segment.logEntries
+}
+
+func (segment *segment) parseLogEntries() error {
+	segmentBytes, err := os.ReadFile(segment.path)
+	if err != nil {
+		return errors.Wrap(err, "Read WAL segment failed, path: "+segment.path)
+	}
+
+	var logEntries []LogEntry
+	var data []byte
+	var batchLen int64
+	var entryLen int64
+	var seriesIDLen int16
+	var seriesID common.SeriesIDV2
+	var seriesCount int32
+	var timestampsBinaryLen int16
+	var entryEndPos int
+	var bytesBuf *bytes.Buffer
+
+	oldPos := 0
+	pos := 0
+	parseNextBatchFlag := true
+
+	for {
+		if parseNextBatchFlag {
+			if len(segmentBytes) <= batchLength {
+				break
+			}
+			data = segmentBytes[pos : pos+batchLength]
+			bytesBuf = bytes.NewBuffer(data)
+			if err = binary.Read(bytesBuf, binary.LittleEndian, &batchLen); err != nil {
+				return errors.Wrap(err, "Read batch length fail to convert from bytes")
+			}
+			if len(segmentBytes) <= int(batchLen) {
+				break
+			}
+
+			pos += batchLength
+			oldPos = pos
+			parseNextBatchFlag = false
+		}
+
+		// Parse entryLength.
+		data = segmentBytes[pos : pos+entryLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &entryLen); err != nil {
+			return errors.Wrap(err, "Read entry length fail to convert from byte")
+		}
+		pos += entryLength
+
+		// Mark entry end-position
+		entryEndPos = pos + int(entryLen)
+		if len(segmentBytes) < entryEndPos {
+			break
+		}
+
+		// Parse seriesIDLen.
+		data = segmentBytes[pos : pos+seriesIDLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &seriesIDLen); err != nil {
+			return errors.Wrap(err, "Read seriesID length fail to convert from byte")
+		}
+		pos += seriesIDLength
+
+		// Parse seriesID.
+		data = segmentBytes[pos : pos+int(seriesIDLen)]
+		seriesID = common.ParseSeriesIDV2(data)
+		pos += int(seriesIDLen)
+
+		// Parse series count.
+		data = segmentBytes[pos : pos+seriesCountLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &seriesCount); err != nil {
+			return errors.Wrap(err, "Read series count fail to convert from byte")
+		}
+		pos += seriesCountLength
+
+		// Parse timestamps compression binary.
+		data = segmentBytes[pos : pos+timestampsBinaryLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &timestampsBinaryLen); err != nil {
+			return errors.Wrap(err, "Read timestamps compression binary length fail to convert from byte")
+		}
+		pos += timestampsBinaryLength
+		data = segmentBytes[pos : pos+int(timestampsBinaryLen)]
+		timestampReader := encoding.NewReader(bytes.NewReader(data))
+		timestampDecoder := encoding.NewXORDecoder(timestampReader)
+		var timestamps []time.Time
+		for i := 0; i < int(seriesCount); i++ {
+			if !timestampDecoder.Next() {
+				return errors.Wrap(err, "Timestamps length not match series count, index: "+strconv.Itoa(i))
+			}
+			timestamps = append(timestamps, unixNanoToTime(timestampDecoder.Value()))
+		}
+		pos += int(timestampsBinaryLen)
+
+		// Parse values compression binary.
+		data = segmentBytes[pos:entryEndPos]
+		if data, err = snappy.Decode(nil, data); err != nil {
+			return errors.Wrap(err, "Decode values compression binary fail to snappy decode")
+		}
+		values := parseValuesBinary(data)
+		if values.Len() != len(timestamps) {
+			return errors.New("Timestamps length and values length not match: " + strconv.Itoa(len(timestamps)) + " vs " + strconv.Itoa(values.Len()))
+		}
+		pos = entryEndPos
+
+		logEntry := &logEntry{
+			entryLength: entryLen,
+			seriesID:    seriesID,
+			count:       seriesCount,
+			timestamps:  timestamps,
+			values:      values,
+		}
+		logEntries = append(logEntries, logEntry)
+
+		if pos == len(segmentBytes) {
+			break
+		}
+		if pos-oldPos == int(batchLen) {
+			parseNextBatchFlag = true
+		}
+	}
+	segment.logEntries = logEntries
+	return nil
+}
+
+func (logEntry *logEntry) GetSeriesID() common.SeriesIDV2 {
+	return logEntry.seriesID
+}
+
+func (logEntry *logEntry) GetTimestamps() []time.Time {
+	return logEntry.timestamps
+}
+
+func (logEntry *logEntry) GetValues() *list.List {
+	return logEntry.values
+}
+
+func (buffer *buffer) write(request logRequest) {
+	seriesID := request.seriesID
+	buffer.timestampMap[seriesID] = append(buffer.timestampMap[seriesID], request.timestamp)
+
+	// Value item: binary-length(2-bytes) + binary data(n-bytes)
+	binaryLength := int16ToBytes(int16(len(request.data)))
+	buffer.valueMap[seriesID] = append(buffer.valueMap[seriesID], binaryLength...)
+	buffer.valueMap[seriesID] = append(buffer.valueMap[seriesID], request.data...)
+
+	buffer.callbackMap[seriesID] = append(buffer.callbackMap[seriesID], request.callback)
+	buffer.count++
+}
+
+func (buffer *buffer) notifyRequests(err error) {
+	for seriesID, callbacks := range buffer.callbackMap {
+		timestamps := buffer.timestampMap[seriesID]
+		values := buffer.valueMap[seriesID]
+		valuePos := 0
+		var valueItem []byte
+		for index, callback := range callbacks {
+			valuePos, valueItem = readValuesBinary(values, valuePos, valuesBinaryLength)
+			tryCallback(func() {
+				callback(seriesID, timestamps[index], valueItem, err)
+			})
+		}
+	}
+}
+
+func segmentName(index uint64) string {
+	return fmt.Sprintf("%v%016x%v", segmentNamePrefix, index, segmentNameSuffix)
+}
+
+func tryCallback(callback func()) {
+	defer func() {
+		if err := recover(); err != nil {
+			fmt.Printf("Run callback error is %v\n", err)
+		}
+	}()
+	callback()
+}
+
+func parseValuesBinary(binary []byte) *list.List {
+	values := list.New()
+	position := 0
+	for {
+		nextPosition, value := readValuesBinary(binary, position, valuesBinaryLength)
+		if value == nil {
+			break
+		}
+		values.PushBack(value)
+		position = nextPosition
+	}
+	return values
+}
+
+func readValuesBinary(raw []byte, position int, offsetLen int) (int, []byte) {
+	if position == len(raw) {
+		return position, nil
+	}
+
+	data := raw[position : position+offsetLen]
+	binaryLen := bytesToInt16(data)
+	position += offsetLen
+
+	data = raw[position : position+int(binaryLen)]
+	position += int(binaryLen)
+	return position, data
+}
+
+func rewriteInt64InBuf(buf []byte, value int64, offset int, order binary.ByteOrder) {

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hailin0 commented on a diff in pull request #261: Implement Write-ahead Logging

Posted by "hailin0 (via GitHub)" <gi...@apache.org>.
hailin0 commented on code in PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#discussion_r1249551844


##########
pkg/wal/wal.go:
##########
@@ -42,19 +75,759 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// log implements the WAL interface.
+type log struct {
+	entryCloser      *run.Closer
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentMap       map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+	writeWaitGroup   sync.WaitGroup
+	flushWaitGroup   sync.WaitGroup
+	workSegmentMutex sync.Mutex
+	segmentMapMutex  sync.RWMutex
+	closerOnce       sync.Once
+}
+
+type segment struct {
+	file       *os.File
+	path       string
+	logEntries []LogEntry
+	segmentID  SegmentID
+}
+
+type logRequest struct {
+	seriesID  common.SeriesIDV2
+	timestamp time.Time
+	callback  func(common.SeriesIDV2, time.Time, []byte, error)
+	data      []byte
+}
+
+type logEntry struct {
+	timestamps  []time.Time
+	values      *list.List
+	seriesID    common.SeriesIDV2
+	entryLength int64
+	count       int32
+}
+
+type buffer struct {
+	timestampMap map[common.SeriesIDV2][]time.Time
+	valueMap     map[common.SeriesIDV2][]byte
+	callbackMap  map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)
+	count        int
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	walOptions := DefaultOptions
+	if options != nil {
+		fileSize := options.FileSize
+		if fileSize <= 0 {
+			fileSize = DefaultOptions.FileSize
+		}
+		bufferSize := options.BufferSize
+		if bufferSize <= 0 {
+			bufferSize = DefaultOptions.BufferSize
+		}
+		bufferBatchInterval := options.BufferBatchInterval
+		if bufferBatchInterval <= 0 {
+			bufferBatchInterval = DefaultOptions.BufferBatchInterval
+		}
+		walOptions = &Options{
+			FileSize:            fileSize,
+			BufferSize:          bufferSize,
+			BufferBatchInterval: bufferBatchInterval,
+		}
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path: "+path)
+	}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	log := &log{
+		path:             path,
+		options:          *walOptions,
+		logger:           logger.GetLogger(moduleName),
+		writeChannel:     make(chan logRequest),
+		flushChannel:     make(chan buffer, walOptions.FlushQueueSize),
+		bytesBuffer:      bytes.NewBuffer([]byte{}),
+		timestampsBuffer: bytes.NewBuffer([]byte{}),
+		entryCloser:      run.NewCloser(1),
+		buffer: buffer{
+			timestampMap: make(map[common.SeriesIDV2][]time.Time),
+			valueMap:     make(map[common.SeriesIDV2][]byte),
+			callbackMap:  make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)),
+			count:        0,
+		},
+	}
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+
+	log.writeWaitGroup.Add(1)
+	log.flushWaitGroup.Add(1)
+	log.start()
+
+	log.logger.Info().Str("path", path).Msg("WAL has be initialized")
+	return log, nil
+}
+
+// Write a logging entity.
+// It will return immediately when the data is written in the buffer,
+// The callback function will be called when the entity is flushed on the persistent storage.
+func (log *log) Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error)) {
+	if !log.entryCloser.AddRunning() {
+		return
+	}
+	defer log.entryCloser.Done()
+
+	log.writeChannel <- logRequest{
+		seriesID:  seriesID,
+		timestamp: timestamp,
+		data:      data,
+		callback:  callback,
+	}
+}
+
+// Read specified segment by SegmentID.
+func (log *log) Read(segmentID SegmentID) (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segment := log.segmentMap[segmentID]
+	return segment, nil
+}
+
+// ReadAllSegments reads all segments sorted by their creation time in ascending order.
+func (log *log) ReadAllSegments() ([]Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segments := make([]Segment, 0)
+	for _, segment := range log.segmentMap {
+		segments = append(segments, segment)
+	}
+	return segments, nil
+}
+
+// Rotate closes the open segment and opens a new one, returning the closed segment details.
+func (log *log) Rotate() (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	oldWorkSegment, err := log.swapWorkSegment()
+	if err != nil {
+		return nil, err
+	}
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Update segment information.
+	workSegment := log.workSegment
+	log.segmentMap[workSegment.segmentID] = workSegment
+	return oldWorkSegment, nil
+}
+
+// Delete the specified segment.
+func (log *log) Delete(segmentID SegmentID) error {
+	if !log.entryCloser.AddRunning() {
+		return errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Segment which will be deleted must be closed.
+	if segmentID == log.workSegment.segmentID {
+		return errors.New("Can not delete the segment which is working")
+	}
+
+	err := os.Remove(log.segmentMap[segmentID].path)
+	if err != nil {
+		return errors.Wrap(err, "Delete WAL segment error")
+	}
+	delete(log.segmentMap, segmentID)
+	return nil
+}
+
+// Close all of segments and stop WAL work.
+func (log *log) Close() error {
+	log.closerOnce.Do(func() {
+		log.logger.Info().Msg("Closing WAL...")
+
+		log.entryCloser.Done()
+		log.entryCloser.CloseThenWait()
+
+		close(log.writeChannel)
+		log.writeWaitGroup.Wait()
+
+		close(log.flushChannel)
+		log.flushWaitGroup.Wait()
+
+		if err := log.flushBuffer(log.buffer); err != nil {
+			log.logger.Err(err).Msg("Flushing buffer failed")

Review Comment:
   fixed



##########
pkg/wal/wal.go:
##########
@@ -42,19 +75,759 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// log implements the WAL interface.
+type log struct {
+	entryCloser      *run.Closer
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentMap       map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+	writeWaitGroup   sync.WaitGroup
+	flushWaitGroup   sync.WaitGroup
+	workSegmentMutex sync.Mutex
+	segmentMapMutex  sync.RWMutex
+	closerOnce       sync.Once
+}
+
+type segment struct {
+	file       *os.File
+	path       string
+	logEntries []LogEntry
+	segmentID  SegmentID
+}
+
+type logRequest struct {
+	seriesID  common.SeriesIDV2
+	timestamp time.Time
+	callback  func(common.SeriesIDV2, time.Time, []byte, error)
+	data      []byte
+}
+
+type logEntry struct {
+	timestamps  []time.Time
+	values      *list.List
+	seriesID    common.SeriesIDV2
+	entryLength int64
+	count       int32
+}
+
+type buffer struct {
+	timestampMap map[common.SeriesIDV2][]time.Time
+	valueMap     map[common.SeriesIDV2][]byte
+	callbackMap  map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)
+	count        int
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	walOptions := DefaultOptions
+	if options != nil {
+		fileSize := options.FileSize
+		if fileSize <= 0 {
+			fileSize = DefaultOptions.FileSize
+		}
+		bufferSize := options.BufferSize
+		if bufferSize <= 0 {
+			bufferSize = DefaultOptions.BufferSize
+		}
+		bufferBatchInterval := options.BufferBatchInterval
+		if bufferBatchInterval <= 0 {
+			bufferBatchInterval = DefaultOptions.BufferBatchInterval
+		}
+		walOptions = &Options{
+			FileSize:            fileSize,
+			BufferSize:          bufferSize,
+			BufferBatchInterval: bufferBatchInterval,
+		}
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path: "+path)
+	}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	log := &log{
+		path:             path,
+		options:          *walOptions,
+		logger:           logger.GetLogger(moduleName),
+		writeChannel:     make(chan logRequest),
+		flushChannel:     make(chan buffer, walOptions.FlushQueueSize),
+		bytesBuffer:      bytes.NewBuffer([]byte{}),
+		timestampsBuffer: bytes.NewBuffer([]byte{}),
+		entryCloser:      run.NewCloser(1),
+		buffer: buffer{
+			timestampMap: make(map[common.SeriesIDV2][]time.Time),
+			valueMap:     make(map[common.SeriesIDV2][]byte),
+			callbackMap:  make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)),
+			count:        0,
+		},
+	}
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+
+	log.writeWaitGroup.Add(1)
+	log.flushWaitGroup.Add(1)
+	log.start()
+
+	log.logger.Info().Str("path", path).Msg("WAL has be initialized")
+	return log, nil
+}
+
+// Write a logging entity.
+// It will return immediately when the data is written in the buffer,
+// The callback function will be called when the entity is flushed on the persistent storage.
+func (log *log) Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error)) {
+	if !log.entryCloser.AddRunning() {
+		return
+	}
+	defer log.entryCloser.Done()
+
+	log.writeChannel <- logRequest{
+		seriesID:  seriesID,
+		timestamp: timestamp,
+		data:      data,
+		callback:  callback,
+	}
+}
+
+// Read specified segment by SegmentID.
+func (log *log) Read(segmentID SegmentID) (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segment := log.segmentMap[segmentID]
+	return segment, nil
+}
+
+// ReadAllSegments reads all segments sorted by their creation time in ascending order.
+func (log *log) ReadAllSegments() ([]Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segments := make([]Segment, 0)
+	for _, segment := range log.segmentMap {
+		segments = append(segments, segment)
+	}
+	return segments, nil
+}
+
+// Rotate closes the open segment and opens a new one, returning the closed segment details.
+func (log *log) Rotate() (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	oldWorkSegment, err := log.swapWorkSegment()
+	if err != nil {
+		return nil, err
+	}
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Update segment information.
+	workSegment := log.workSegment
+	log.segmentMap[workSegment.segmentID] = workSegment
+	return oldWorkSegment, nil
+}
+
+// Delete the specified segment.
+func (log *log) Delete(segmentID SegmentID) error {
+	if !log.entryCloser.AddRunning() {
+		return errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Segment which will be deleted must be closed.
+	if segmentID == log.workSegment.segmentID {
+		return errors.New("Can not delete the segment which is working")
+	}
+
+	err := os.Remove(log.segmentMap[segmentID].path)
+	if err != nil {
+		return errors.Wrap(err, "Delete WAL segment error")
+	}
+	delete(log.segmentMap, segmentID)
+	return nil
+}
+
+// Close all of segments and stop WAL work.
+func (log *log) Close() error {
+	log.closerOnce.Do(func() {
+		log.logger.Info().Msg("Closing WAL...")
+
+		log.entryCloser.Done()
+		log.entryCloser.CloseThenWait()
+
+		close(log.writeChannel)
+		log.writeWaitGroup.Wait()
+
+		close(log.flushChannel)
+		log.flushWaitGroup.Wait()
+
+		if err := log.flushBuffer(log.buffer); err != nil {
+			log.logger.Err(err).Msg("Flushing buffer failed")
+		}
+		if err := log.workSegment.file.Close(); err != nil {
+			log.logger.Err(err).Msg("Close work segment file error")
+		}
+		log.logger.Info().Msg("Closed WAL")
+	})
+	return nil
+}
+
+func (log *log) start() {
+	go func() {
+		log.logger.Info().Msg("Start batch task...")
+
+		defer log.writeWaitGroup.Done()
+
+		bufferVolume := 0
+		for {
+			timer := time.NewTimer(log.options.BufferBatchInterval)
+			select {
+			case request, chOpen := <-log.writeChannel:
+				if !chOpen {
+					log.logger.Info().Msg("Stop batch task when write-channel closed!")
+					return
+				}
+
+				log.buffer.write(request)
+				if log.logger.Debug().Enabled() {
+					log.logger.Debug().Msg("Write request to buffer. elements: " + strconv.Itoa(log.buffer.count))
+				}
+
+				bufferVolume += len(request.seriesID.Marshal()) + timestampVolumeLength + len(request.data)
+				if bufferVolume > log.options.BufferSize {
+					log.triggerFlushing()
+					bufferVolume = 0
+				}
+				continue

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hailin0 commented on a diff in pull request #261: Implement Write-ahead Logging

Posted by "hailin0 (via GitHub)" <gi...@apache.org>.
hailin0 commented on code in PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#discussion_r1249551769


##########
pkg/wal/wal.go:
##########
@@ -42,19 +75,759 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// log implements the WAL interface.
+type log struct {
+	entryCloser      *run.Closer

Review Comment:
   fixed



##########
pkg/wal/wal.go:
##########
@@ -42,19 +75,759 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// log implements the WAL interface.
+type log struct {
+	entryCloser      *run.Closer
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentMap       map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+	writeWaitGroup   sync.WaitGroup
+	flushWaitGroup   sync.WaitGroup
+	workSegmentMutex sync.Mutex
+	segmentMapMutex  sync.RWMutex

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hailin0 commented on a diff in pull request #261: Implement Write-ahead Logging

Posted by "hailin0 (via GitHub)" <gi...@apache.org>.
hailin0 commented on code in PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#discussion_r1249551976


##########
pkg/wal/wal.go:
##########
@@ -42,19 +75,759 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// log implements the WAL interface.
+type log struct {
+	entryCloser      *run.Closer
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentMap       map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+	writeWaitGroup   sync.WaitGroup
+	flushWaitGroup   sync.WaitGroup
+	workSegmentMutex sync.Mutex
+	segmentMapMutex  sync.RWMutex
+	closerOnce       sync.Once
+}
+
+type segment struct {
+	file       *os.File
+	path       string
+	logEntries []LogEntry
+	segmentID  SegmentID
+}
+
+type logRequest struct {
+	seriesID  common.SeriesIDV2
+	timestamp time.Time
+	callback  func(common.SeriesIDV2, time.Time, []byte, error)
+	data      []byte
+}
+
+type logEntry struct {
+	timestamps  []time.Time
+	values      *list.List
+	seriesID    common.SeriesIDV2
+	entryLength int64
+	count       int32
+}
+
+type buffer struct {
+	timestampMap map[common.SeriesIDV2][]time.Time
+	valueMap     map[common.SeriesIDV2][]byte
+	callbackMap  map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)
+	count        int
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	walOptions := DefaultOptions
+	if options != nil {
+		fileSize := options.FileSize
+		if fileSize <= 0 {
+			fileSize = DefaultOptions.FileSize
+		}
+		bufferSize := options.BufferSize
+		if bufferSize <= 0 {
+			bufferSize = DefaultOptions.BufferSize
+		}
+		bufferBatchInterval := options.BufferBatchInterval
+		if bufferBatchInterval <= 0 {
+			bufferBatchInterval = DefaultOptions.BufferBatchInterval
+		}
+		walOptions = &Options{
+			FileSize:            fileSize,
+			BufferSize:          bufferSize,
+			BufferBatchInterval: bufferBatchInterval,
+		}
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path: "+path)
+	}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	log := &log{
+		path:             path,
+		options:          *walOptions,
+		logger:           logger.GetLogger(moduleName),
+		writeChannel:     make(chan logRequest),
+		flushChannel:     make(chan buffer, walOptions.FlushQueueSize),
+		bytesBuffer:      bytes.NewBuffer([]byte{}),
+		timestampsBuffer: bytes.NewBuffer([]byte{}),
+		entryCloser:      run.NewCloser(1),
+		buffer: buffer{
+			timestampMap: make(map[common.SeriesIDV2][]time.Time),
+			valueMap:     make(map[common.SeriesIDV2][]byte),
+			callbackMap:  make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)),
+			count:        0,
+		},
+	}
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+
+	log.writeWaitGroup.Add(1)
+	log.flushWaitGroup.Add(1)
+	log.start()
+
+	log.logger.Info().Str("path", path).Msg("WAL has be initialized")
+	return log, nil
+}
+
+// Write a logging entity.
+// It will return immediately when the data is written in the buffer,
+// The callback function will be called when the entity is flushed on the persistent storage.
+func (log *log) Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error)) {
+	if !log.entryCloser.AddRunning() {
+		return
+	}
+	defer log.entryCloser.Done()
+
+	log.writeChannel <- logRequest{
+		seriesID:  seriesID,
+		timestamp: timestamp,
+		data:      data,
+		callback:  callback,
+	}
+}
+
+// Read specified segment by SegmentID.
+func (log *log) Read(segmentID SegmentID) (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segment := log.segmentMap[segmentID]
+	return segment, nil
+}
+
+// ReadAllSegments reads all segments sorted by their creation time in ascending order.
+func (log *log) ReadAllSegments() ([]Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segments := make([]Segment, 0)
+	for _, segment := range log.segmentMap {
+		segments = append(segments, segment)
+	}
+	return segments, nil
+}
+
+// Rotate closes the open segment and opens a new one, returning the closed segment details.
+func (log *log) Rotate() (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	oldWorkSegment, err := log.swapWorkSegment()
+	if err != nil {
+		return nil, err
+	}
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Update segment information.
+	workSegment := log.workSegment
+	log.segmentMap[workSegment.segmentID] = workSegment
+	return oldWorkSegment, nil
+}
+
+// Delete the specified segment.
+func (log *log) Delete(segmentID SegmentID) error {
+	if !log.entryCloser.AddRunning() {
+		return errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Segment which will be deleted must be closed.
+	if segmentID == log.workSegment.segmentID {
+		return errors.New("Can not delete the segment which is working")
+	}
+
+	err := os.Remove(log.segmentMap[segmentID].path)
+	if err != nil {
+		return errors.Wrap(err, "Delete WAL segment error")
+	}
+	delete(log.segmentMap, segmentID)
+	return nil
+}
+
+// Close all of segments and stop WAL work.
+func (log *log) Close() error {
+	log.closerOnce.Do(func() {
+		log.logger.Info().Msg("Closing WAL...")
+
+		log.entryCloser.Done()
+		log.entryCloser.CloseThenWait()
+
+		close(log.writeChannel)
+		log.writeWaitGroup.Wait()
+
+		close(log.flushChannel)
+		log.flushWaitGroup.Wait()
+
+		if err := log.flushBuffer(log.buffer); err != nil {
+			log.logger.Err(err).Msg("Flushing buffer failed")
+		}
+		if err := log.workSegment.file.Close(); err != nil {
+			log.logger.Err(err).Msg("Close work segment file error")
+		}
+		log.logger.Info().Msg("Closed WAL")
+	})
+	return nil
+}
+
+func (log *log) start() {
+	go func() {
+		log.logger.Info().Msg("Start batch task...")
+
+		defer log.writeWaitGroup.Done()
+
+		bufferVolume := 0
+		for {
+			timer := time.NewTimer(log.options.BufferBatchInterval)
+			select {
+			case request, chOpen := <-log.writeChannel:
+				if !chOpen {
+					log.logger.Info().Msg("Stop batch task when write-channel closed!")
+					return
+				}
+
+				log.buffer.write(request)
+				if log.logger.Debug().Enabled() {
+					log.logger.Debug().Msg("Write request to buffer. elements: " + strconv.Itoa(log.buffer.count))
+				}
+
+				bufferVolume += len(request.seriesID.Marshal()) + timestampVolumeLength + len(request.data)
+				if bufferVolume > log.options.BufferSize {
+					log.triggerFlushing()
+					bufferVolume = 0
+				}
+				continue
+			case <-timer.C:
+				if bufferVolume == 0 {
+					continue
+				}
+				log.triggerFlushing()
+				bufferVolume = 0
+				continue
+			}
+		}
+	}()
+
+	go func() {
+		log.logger.Info().Msg("Start flush task...")
+
+		defer log.flushWaitGroup.Done()
+
+		for batch := range log.flushChannel {
+			startTime := time.Now()
+
+			var err error
+			for i := 0; i < maxRetries; i++ {
+				if err = log.flushBuffer(batch); err != nil {
+					log.logger.Err(err).Msg("Flushing buffer failed. Retrying...")
+					time.Sleep(100 * time.Millisecond)
+					continue
+				}
+				break
+			}
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Flushed buffer to WAL file. elements: " +
+					strconv.Itoa(batch.count) + ", cost: " + time.Since(startTime).String())
+			}
+
+			batch.notifyRequests(err)
+		}
+		log.logger.Info().Msg("Stop flush task when flush-channel closed!")
+	}()
+
+	log.logger.Info().Msg("Started WAL")
+}
+
+func (log *log) triggerFlushing() {
+	for {
+		select {
+		case log.flushChannel <- log.buffer:
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Send buffer to flush-channel. elements: " + strconv.Itoa(log.buffer.count))
+			}
+			log.newBuffer()
+			return
+		default:
+		}
+		time.Sleep(10 * time.Millisecond)
+	}
+}
+
+func (log *log) newBuffer() {
+	log.buffer.timestampMap = make(map[common.SeriesIDV2][]time.Time)
+	log.buffer.valueMap = make(map[common.SeriesIDV2][]byte)
+	log.buffer.callbackMap = make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error))
+	log.buffer.count = 0
+}
+
+func (log *log) flushBuffer(buffer buffer) error {
+	if buffer.count == 0 {
+		return nil
+	}
+
+	defer func() {
+		log.bytesBuffer.Reset()
+		log.timestampsBuffer.Reset()
+	}()
+
+	// placeholder, preset batch length value is 0
+	batchLen := 0
+	if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(batchLen)); err != nil {
+		return errors.Wrap(err, "Write batch length error")
+	}
+	for seriesID, timestamps := range buffer.timestampMap {
+		// Generate seriesID binary
+		seriesIDBytes := seriesID.Marshal()
+		seriesIDBytesLen := len(seriesIDBytes)
+
+		// Generate timestamps compression binary
+		log.timestampsBuffer.Reset()
+		timestampWriter := encoding.NewWriter()
+		timestampEncoder := encoding.NewXOREncoder(timestampWriter)
+		timestampWriter.Reset(log.timestampsBuffer)
+		for _, timestamp := range timestamps {
+			timestampEncoder.Write(timeTouUnixNano(timestamp))
+		}
+		timestampWriter.Flush()
+		timestampsBytes := log.timestampsBuffer.Bytes()
+		timestampsBytesLen := len(timestampsBytes)
+
+		// Generate values compression binary
+		valuesBytes := snappy.Encode(nil, buffer.valueMap[seriesID])
+
+		// Write entry data
+		entryLen := seriesIDLength + seriesIDBytesLen + seriesCountLength + timestampsBinaryLength + timestampsBytesLen + len(valuesBytes)
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(entryLen)); err != nil {
+			return errors.Wrap(err, "Write entry length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int16(seriesIDBytesLen)); err != nil {
+			return errors.Wrap(err, "Write seriesID length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, seriesIDBytes); err != nil {
+			return errors.Wrap(err, "Write seriesID error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int32(len(timestamps))); err != nil {
+			return errors.Wrap(err, "Write series count error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int16(timestampsBytesLen)); err != nil {
+			return errors.Wrap(err, "Write timestamps length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, timestampsBytes); err != nil {
+			return errors.Wrap(err, "Write timestamps error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, valuesBytes); err != nil {
+			return errors.Wrap(err, "Write values error")
+		}
+	}
+	// Rewrite batch length
+	batchBytes := log.bytesBuffer.Bytes()
+	batchLen = len(batchBytes) - batchLength
+	rewriteInt64InBuf(batchBytes, int64(batchLen), 0, binary.LittleEndian)
+
+	return log.writeWorkSegment(batchBytes)
+}
+
+func (log *log) swapWorkSegment() (Segment, error) {
+	log.workSegmentMutex.Lock()
+	defer log.workSegmentMutex.Unlock()
+
+	if err := log.workSegment.file.Close(); err != nil {
+		return nil, errors.Wrap(err, "Close WAL segment error")
+	}
+
+	oldSegment := log.workSegment
+	// Create new segment.
+	segmentID := log.workSegment.segmentID + 1
+	segment := &segment{
+		segmentID: segmentID,
+		path:      filepath.Join(log.path, segmentName(uint64(segmentID))),
+	}
+	var err error
+	segment.file, err = os.OpenFile(segment.path, os.O_CREATE|os.O_RDWR|os.O_TRUNC, os.ModePerm)

Review Comment:
   fixed



##########
pkg/wal/wal.go:
##########
@@ -42,19 +75,759 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// log implements the WAL interface.
+type log struct {
+	entryCloser      *run.Closer
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentMap       map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+	writeWaitGroup   sync.WaitGroup
+	flushWaitGroup   sync.WaitGroup
+	workSegmentMutex sync.Mutex
+	segmentMapMutex  sync.RWMutex
+	closerOnce       sync.Once
+}
+
+type segment struct {
+	file       *os.File
+	path       string
+	logEntries []LogEntry
+	segmentID  SegmentID
+}
+
+type logRequest struct {
+	seriesID  common.SeriesIDV2
+	timestamp time.Time
+	callback  func(common.SeriesIDV2, time.Time, []byte, error)
+	data      []byte
+}
+
+type logEntry struct {
+	timestamps  []time.Time
+	values      *list.List
+	seriesID    common.SeriesIDV2
+	entryLength int64
+	count       int32
+}
+
+type buffer struct {
+	timestampMap map[common.SeriesIDV2][]time.Time
+	valueMap     map[common.SeriesIDV2][]byte
+	callbackMap  map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)
+	count        int
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	walOptions := DefaultOptions
+	if options != nil {
+		fileSize := options.FileSize
+		if fileSize <= 0 {
+			fileSize = DefaultOptions.FileSize
+		}
+		bufferSize := options.BufferSize
+		if bufferSize <= 0 {
+			bufferSize = DefaultOptions.BufferSize
+		}
+		bufferBatchInterval := options.BufferBatchInterval
+		if bufferBatchInterval <= 0 {
+			bufferBatchInterval = DefaultOptions.BufferBatchInterval
+		}
+		walOptions = &Options{
+			FileSize:            fileSize,
+			BufferSize:          bufferSize,
+			BufferBatchInterval: bufferBatchInterval,
+		}
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path: "+path)
+	}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	log := &log{
+		path:             path,
+		options:          *walOptions,
+		logger:           logger.GetLogger(moduleName),
+		writeChannel:     make(chan logRequest),
+		flushChannel:     make(chan buffer, walOptions.FlushQueueSize),
+		bytesBuffer:      bytes.NewBuffer([]byte{}),
+		timestampsBuffer: bytes.NewBuffer([]byte{}),
+		entryCloser:      run.NewCloser(1),
+		buffer: buffer{
+			timestampMap: make(map[common.SeriesIDV2][]time.Time),
+			valueMap:     make(map[common.SeriesIDV2][]byte),
+			callbackMap:  make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)),
+			count:        0,
+		},
+	}
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+
+	log.writeWaitGroup.Add(1)
+	log.flushWaitGroup.Add(1)
+	log.start()
+
+	log.logger.Info().Str("path", path).Msg("WAL has be initialized")
+	return log, nil
+}
+
+// Write a logging entity.
+// It will return immediately when the data is written in the buffer,
+// The callback function will be called when the entity is flushed on the persistent storage.
+func (log *log) Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error)) {
+	if !log.entryCloser.AddRunning() {
+		return
+	}
+	defer log.entryCloser.Done()
+
+	log.writeChannel <- logRequest{
+		seriesID:  seriesID,
+		timestamp: timestamp,
+		data:      data,
+		callback:  callback,
+	}
+}
+
+// Read specified segment by SegmentID.
+func (log *log) Read(segmentID SegmentID) (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segment := log.segmentMap[segmentID]
+	return segment, nil
+}
+
+// ReadAllSegments reads all segments sorted by their creation time in ascending order.
+func (log *log) ReadAllSegments() ([]Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segments := make([]Segment, 0)
+	for _, segment := range log.segmentMap {
+		segments = append(segments, segment)
+	}
+	return segments, nil
+}
+
+// Rotate closes the open segment and opens a new one, returning the closed segment details.
+func (log *log) Rotate() (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	oldWorkSegment, err := log.swapWorkSegment()
+	if err != nil {
+		return nil, err
+	}
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Update segment information.
+	workSegment := log.workSegment
+	log.segmentMap[workSegment.segmentID] = workSegment
+	return oldWorkSegment, nil
+}
+
+// Delete the specified segment.
+func (log *log) Delete(segmentID SegmentID) error {
+	if !log.entryCloser.AddRunning() {
+		return errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Segment which will be deleted must be closed.
+	if segmentID == log.workSegment.segmentID {
+		return errors.New("Can not delete the segment which is working")
+	}
+
+	err := os.Remove(log.segmentMap[segmentID].path)
+	if err != nil {
+		return errors.Wrap(err, "Delete WAL segment error")
+	}
+	delete(log.segmentMap, segmentID)
+	return nil
+}
+
+// Close all of segments and stop WAL work.
+func (log *log) Close() error {
+	log.closerOnce.Do(func() {
+		log.logger.Info().Msg("Closing WAL...")
+
+		log.entryCloser.Done()
+		log.entryCloser.CloseThenWait()
+
+		close(log.writeChannel)
+		log.writeWaitGroup.Wait()
+
+		close(log.flushChannel)
+		log.flushWaitGroup.Wait()
+
+		if err := log.flushBuffer(log.buffer); err != nil {
+			log.logger.Err(err).Msg("Flushing buffer failed")
+		}
+		if err := log.workSegment.file.Close(); err != nil {
+			log.logger.Err(err).Msg("Close work segment file error")
+		}
+		log.logger.Info().Msg("Closed WAL")
+	})
+	return nil
+}
+
+func (log *log) start() {
+	go func() {
+		log.logger.Info().Msg("Start batch task...")
+
+		defer log.writeWaitGroup.Done()
+
+		bufferVolume := 0
+		for {
+			timer := time.NewTimer(log.options.BufferBatchInterval)
+			select {
+			case request, chOpen := <-log.writeChannel:
+				if !chOpen {
+					log.logger.Info().Msg("Stop batch task when write-channel closed!")
+					return
+				}
+
+				log.buffer.write(request)
+				if log.logger.Debug().Enabled() {
+					log.logger.Debug().Msg("Write request to buffer. elements: " + strconv.Itoa(log.buffer.count))
+				}
+
+				bufferVolume += len(request.seriesID.Marshal()) + timestampVolumeLength + len(request.data)
+				if bufferVolume > log.options.BufferSize {
+					log.triggerFlushing()
+					bufferVolume = 0
+				}
+				continue
+			case <-timer.C:
+				if bufferVolume == 0 {
+					continue
+				}
+				log.triggerFlushing()
+				bufferVolume = 0
+				continue
+			}
+		}
+	}()
+
+	go func() {
+		log.logger.Info().Msg("Start flush task...")
+
+		defer log.flushWaitGroup.Done()
+
+		for batch := range log.flushChannel {
+			startTime := time.Now()
+
+			var err error
+			for i := 0; i < maxRetries; i++ {
+				if err = log.flushBuffer(batch); err != nil {
+					log.logger.Err(err).Msg("Flushing buffer failed. Retrying...")
+					time.Sleep(100 * time.Millisecond)
+					continue
+				}
+				break
+			}
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Flushed buffer to WAL file. elements: " +
+					strconv.Itoa(batch.count) + ", cost: " + time.Since(startTime).String())
+			}
+
+			batch.notifyRequests(err)
+		}
+		log.logger.Info().Msg("Stop flush task when flush-channel closed!")
+	}()
+
+	log.logger.Info().Msg("Started WAL")
+}
+
+func (log *log) triggerFlushing() {
+	for {
+		select {
+		case log.flushChannel <- log.buffer:
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Send buffer to flush-channel. elements: " + strconv.Itoa(log.buffer.count))
+			}
+			log.newBuffer()
+			return
+		default:
+		}
+		time.Sleep(10 * time.Millisecond)
+	}
+}
+
+func (log *log) newBuffer() {
+	log.buffer.timestampMap = make(map[common.SeriesIDV2][]time.Time)
+	log.buffer.valueMap = make(map[common.SeriesIDV2][]byte)
+	log.buffer.callbackMap = make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error))
+	log.buffer.count = 0
+}
+
+func (log *log) flushBuffer(buffer buffer) error {
+	if buffer.count == 0 {
+		return nil
+	}
+
+	defer func() {
+		log.bytesBuffer.Reset()
+		log.timestampsBuffer.Reset()
+	}()
+
+	// placeholder, preset batch length value is 0
+	batchLen := 0
+	if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(batchLen)); err != nil {
+		return errors.Wrap(err, "Write batch length error")
+	}
+	for seriesID, timestamps := range buffer.timestampMap {
+		// Generate seriesID binary
+		seriesIDBytes := seriesID.Marshal()
+		seriesIDBytesLen := len(seriesIDBytes)
+
+		// Generate timestamps compression binary
+		log.timestampsBuffer.Reset()
+		timestampWriter := encoding.NewWriter()
+		timestampEncoder := encoding.NewXOREncoder(timestampWriter)
+		timestampWriter.Reset(log.timestampsBuffer)
+		for _, timestamp := range timestamps {
+			timestampEncoder.Write(timeTouUnixNano(timestamp))
+		}
+		timestampWriter.Flush()
+		timestampsBytes := log.timestampsBuffer.Bytes()
+		timestampsBytesLen := len(timestampsBytes)
+
+		// Generate values compression binary
+		valuesBytes := snappy.Encode(nil, buffer.valueMap[seriesID])
+
+		// Write entry data
+		entryLen := seriesIDLength + seriesIDBytesLen + seriesCountLength + timestampsBinaryLength + timestampsBytesLen + len(valuesBytes)
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(entryLen)); err != nil {
+			return errors.Wrap(err, "Write entry length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int16(seriesIDBytesLen)); err != nil {
+			return errors.Wrap(err, "Write seriesID length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, seriesIDBytes); err != nil {
+			return errors.Wrap(err, "Write seriesID error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int32(len(timestamps))); err != nil {
+			return errors.Wrap(err, "Write series count error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int16(timestampsBytesLen)); err != nil {
+			return errors.Wrap(err, "Write timestamps length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, timestampsBytes); err != nil {
+			return errors.Wrap(err, "Write timestamps error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, valuesBytes); err != nil {
+			return errors.Wrap(err, "Write values error")
+		}
+	}
+	// Rewrite batch length
+	batchBytes := log.bytesBuffer.Bytes()
+	batchLen = len(batchBytes) - batchLength
+	rewriteInt64InBuf(batchBytes, int64(batchLen), 0, binary.LittleEndian)
+
+	return log.writeWorkSegment(batchBytes)
+}
+
+func (log *log) swapWorkSegment() (Segment, error) {
+	log.workSegmentMutex.Lock()
+	defer log.workSegmentMutex.Unlock()
+
+	if err := log.workSegment.file.Close(); err != nil {
+		return nil, errors.Wrap(err, "Close WAL segment error")
+	}
+
+	oldSegment := log.workSegment
+	// Create new segment.
+	segmentID := log.workSegment.segmentID + 1
+	segment := &segment{
+		segmentID: segmentID,
+		path:      filepath.Join(log.path, segmentName(uint64(segmentID))),
+	}
+	var err error
+	segment.file, err = os.OpenFile(segment.path, os.O_CREATE|os.O_RDWR|os.O_TRUNC, os.ModePerm)
+	if err != nil {
+		return nil, errors.Wrap(err, "Open WAL segment error")
+	}
+	log.workSegment = segment
+
+	return oldSegment, nil
+}
+
+func (log *log) writeWorkSegment(data []byte) error {
+	log.workSegmentMutex.Lock()
+	defer log.workSegmentMutex.Unlock()
+
+	// Write batch data to WAL segment file
+	if _, err := log.workSegment.file.Write(data); err != nil {
+		return errors.Wrap(err, "Write WAL segment file error, file: "+log.workSegment.path)
+	}
+	if err := log.workSegment.file.Sync(); err != nil {
+		log.logger.Warn().Msg("Sync WAL segment file to disk failed, file: " + log.workSegment.path)
+	}
+	return nil
+}
+
+func (log *log) load() error {
+	files, err := os.ReadDir(log.path)
+	if err != nil {
+		return errors.Wrap(err, "Can not read dir: "+log.path)
+	}
+	// Load all of WAL segments.
+	var workSegmentID SegmentID
+	log.segmentMap = make(map[SegmentID]*segment)
+	for _, file := range files {
+		name := file.Name()
+		segmentID, parsePathErr := strconv.ParseUint(name[3:19], 10, 64)

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] hailin0 commented on a diff in pull request #261: Implement Write-ahead Logging

Posted by "hailin0 (via GitHub)" <gi...@apache.org>.
hailin0 commented on code in PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#discussion_r1249551921


##########
pkg/wal/wal.go:
##########
@@ -42,19 +75,759 @@ type Segment interface {
 type WAL interface {
 	// Write a logging entity.
 	// It will return immediately when the data is written in the buffer,
-	// The returned function will be called when the entity is flushed on the persistent storage.
-	Write(seriesID common.SeriesID, timestamp time.Time, data []byte) (func(), error)
+	// The callback function will be called when the entity is flushed on the persistent storage.
+	Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error))
 	// Read specified segment by SegmentID.
-	Read(segmentID SegmentID) (*Segment, error)
+	Read(segmentID SegmentID) (Segment, error)
 	// ReadAllSegments reads all segments sorted by their creation time in ascending order.
-	ReadAllSegments() ([]*Segment, error)
+	ReadAllSegments() ([]Segment, error)
 	// Rotate closes the open segment and opens a new one, returning the closed segment details.
-	Rotate() (*Segment, error)
+	Rotate() (Segment, error)
 	// Delete the specified segment.
 	Delete(segmentID SegmentID) error
+	// Close all of segments and stop WAL work.
+	Close() error
+}
+
+// SegmentID identities a segment in a WAL.
+type SegmentID uint64
+
+// Segment allows reading underlying segments that hold WAl entities.
+type Segment interface {
+	GetSegmentID() SegmentID
+	GetLogEntries() []LogEntry
+}
+
+// LogEntry used for attain detail value of WAL entry.
+type LogEntry interface {
+	GetSeriesID() common.SeriesIDV2
+	GetTimestamps() []time.Time
+	GetValues() *list.List
+}
+
+// log implements the WAL interface.
+type log struct {
+	entryCloser      *run.Closer
+	buffer           buffer
+	logger           *logger.Logger
+	bytesBuffer      *bytes.Buffer
+	timestampsBuffer *bytes.Buffer
+	segmentMap       map[SegmentID]*segment
+	workSegment      *segment
+	writeChannel     chan logRequest
+	flushChannel     chan buffer
+	path             string
+	options          Options
+	writeWaitGroup   sync.WaitGroup
+	flushWaitGroup   sync.WaitGroup
+	workSegmentMutex sync.Mutex
+	segmentMapMutex  sync.RWMutex
+	closerOnce       sync.Once
+}
+
+type segment struct {
+	file       *os.File
+	path       string
+	logEntries []LogEntry
+	segmentID  SegmentID
+}
+
+type logRequest struct {
+	seriesID  common.SeriesIDV2
+	timestamp time.Time
+	callback  func(common.SeriesIDV2, time.Time, []byte, error)
+	data      []byte
+}
+
+type logEntry struct {
+	timestamps  []time.Time
+	values      *list.List
+	seriesID    common.SeriesIDV2
+	entryLength int64
+	count       int32
+}
+
+type buffer struct {
+	timestampMap map[common.SeriesIDV2][]time.Time
+	valueMap     map[common.SeriesIDV2][]byte
+	callbackMap  map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)
+	count        int
 }
 
 // New creates a WAL instance in the specified path.
-func New(_ string, _ Options) (WAL, error) {
-	return nil, nil
+func New(path string, options *Options) (WAL, error) {
+	//  Check configuration options.
+	walOptions := DefaultOptions
+	if options != nil {
+		fileSize := options.FileSize
+		if fileSize <= 0 {
+			fileSize = DefaultOptions.FileSize
+		}
+		bufferSize := options.BufferSize
+		if bufferSize <= 0 {
+			bufferSize = DefaultOptions.BufferSize
+		}
+		bufferBatchInterval := options.BufferBatchInterval
+		if bufferBatchInterval <= 0 {
+			bufferBatchInterval = DefaultOptions.BufferBatchInterval
+		}
+		walOptions = &Options{
+			FileSize:            fileSize,
+			BufferSize:          bufferSize,
+			BufferBatchInterval: bufferBatchInterval,
+		}
+	}
+
+	// Initial WAL path.
+	path, err := filepath.Abs(path)
+	if err != nil {
+		return nil, errors.Wrap(err, "Can not get absolute path: "+path)
+	}
+	if err := os.MkdirAll(path, os.ModePerm); err != nil {
+		return nil, err
+	}
+
+	log := &log{
+		path:             path,
+		options:          *walOptions,
+		logger:           logger.GetLogger(moduleName),
+		writeChannel:     make(chan logRequest),
+		flushChannel:     make(chan buffer, walOptions.FlushQueueSize),
+		bytesBuffer:      bytes.NewBuffer([]byte{}),
+		timestampsBuffer: bytes.NewBuffer([]byte{}),
+		entryCloser:      run.NewCloser(1),
+		buffer: buffer{
+			timestampMap: make(map[common.SeriesIDV2][]time.Time),
+			valueMap:     make(map[common.SeriesIDV2][]byte),
+			callbackMap:  make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error)),
+			count:        0,
+		},
+	}
+	if err := log.load(); err != nil {
+		return nil, err
+	}
+
+	log.writeWaitGroup.Add(1)
+	log.flushWaitGroup.Add(1)
+	log.start()
+
+	log.logger.Info().Str("path", path).Msg("WAL has be initialized")
+	return log, nil
+}
+
+// Write a logging entity.
+// It will return immediately when the data is written in the buffer,
+// The callback function will be called when the entity is flushed on the persistent storage.
+func (log *log) Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte, callback func(common.SeriesIDV2, time.Time, []byte, error)) {
+	if !log.entryCloser.AddRunning() {
+		return
+	}
+	defer log.entryCloser.Done()
+
+	log.writeChannel <- logRequest{
+		seriesID:  seriesID,
+		timestamp: timestamp,
+		data:      data,
+		callback:  callback,
+	}
+}
+
+// Read specified segment by SegmentID.
+func (log *log) Read(segmentID SegmentID) (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segment := log.segmentMap[segmentID]
+	return segment, nil
+}
+
+// ReadAllSegments reads all segments sorted by their creation time in ascending order.
+func (log *log) ReadAllSegments() ([]Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.RLock()
+	defer log.segmentMapMutex.RUnlock()
+
+	segments := make([]Segment, 0)
+	for _, segment := range log.segmentMap {
+		segments = append(segments, segment)
+	}
+	return segments, nil
+}
+
+// Rotate closes the open segment and opens a new one, returning the closed segment details.
+func (log *log) Rotate() (Segment, error) {
+	if !log.entryCloser.AddRunning() {
+		return nil, errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	oldWorkSegment, err := log.swapWorkSegment()
+	if err != nil {
+		return nil, err
+	}
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Update segment information.
+	workSegment := log.workSegment
+	log.segmentMap[workSegment.segmentID] = workSegment
+	return oldWorkSegment, nil
+}
+
+// Delete the specified segment.
+func (log *log) Delete(segmentID SegmentID) error {
+	if !log.entryCloser.AddRunning() {
+		return errors.New("WAL is closed")
+	}
+	defer log.entryCloser.Done()
+
+	log.segmentMapMutex.Lock()
+	defer log.segmentMapMutex.Unlock()
+
+	// Segment which will be deleted must be closed.
+	if segmentID == log.workSegment.segmentID {
+		return errors.New("Can not delete the segment which is working")
+	}
+
+	err := os.Remove(log.segmentMap[segmentID].path)
+	if err != nil {
+		return errors.Wrap(err, "Delete WAL segment error")
+	}
+	delete(log.segmentMap, segmentID)
+	return nil
+}
+
+// Close all of segments and stop WAL work.
+func (log *log) Close() error {
+	log.closerOnce.Do(func() {
+		log.logger.Info().Msg("Closing WAL...")
+
+		log.entryCloser.Done()
+		log.entryCloser.CloseThenWait()
+
+		close(log.writeChannel)
+		log.writeWaitGroup.Wait()
+
+		close(log.flushChannel)
+		log.flushWaitGroup.Wait()
+
+		if err := log.flushBuffer(log.buffer); err != nil {
+			log.logger.Err(err).Msg("Flushing buffer failed")
+		}
+		if err := log.workSegment.file.Close(); err != nil {
+			log.logger.Err(err).Msg("Close work segment file error")
+		}
+		log.logger.Info().Msg("Closed WAL")
+	})
+	return nil
+}
+
+func (log *log) start() {
+	go func() {
+		log.logger.Info().Msg("Start batch task...")
+
+		defer log.writeWaitGroup.Done()
+
+		bufferVolume := 0
+		for {
+			timer := time.NewTimer(log.options.BufferBatchInterval)
+			select {
+			case request, chOpen := <-log.writeChannel:
+				if !chOpen {
+					log.logger.Info().Msg("Stop batch task when write-channel closed!")
+					return
+				}
+
+				log.buffer.write(request)
+				if log.logger.Debug().Enabled() {
+					log.logger.Debug().Msg("Write request to buffer. elements: " + strconv.Itoa(log.buffer.count))
+				}
+
+				bufferVolume += len(request.seriesID.Marshal()) + timestampVolumeLength + len(request.data)
+				if bufferVolume > log.options.BufferSize {
+					log.triggerFlushing()
+					bufferVolume = 0
+				}
+				continue
+			case <-timer.C:
+				if bufferVolume == 0 {
+					continue
+				}
+				log.triggerFlushing()
+				bufferVolume = 0
+				continue
+			}
+		}
+	}()
+
+	go func() {
+		log.logger.Info().Msg("Start flush task...")
+
+		defer log.flushWaitGroup.Done()
+
+		for batch := range log.flushChannel {
+			startTime := time.Now()
+
+			var err error
+			for i := 0; i < maxRetries; i++ {
+				if err = log.flushBuffer(batch); err != nil {
+					log.logger.Err(err).Msg("Flushing buffer failed. Retrying...")
+					time.Sleep(100 * time.Millisecond)
+					continue
+				}
+				break
+			}
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Flushed buffer to WAL file. elements: " +
+					strconv.Itoa(batch.count) + ", cost: " + time.Since(startTime).String())
+			}
+
+			batch.notifyRequests(err)
+		}
+		log.logger.Info().Msg("Stop flush task when flush-channel closed!")
+	}()
+
+	log.logger.Info().Msg("Started WAL")
+}
+
+func (log *log) triggerFlushing() {
+	for {
+		select {
+		case log.flushChannel <- log.buffer:
+			if log.logger.Debug().Enabled() {
+				log.logger.Debug().Msg("Send buffer to flush-channel. elements: " + strconv.Itoa(log.buffer.count))
+			}
+			log.newBuffer()
+			return
+		default:
+		}
+		time.Sleep(10 * time.Millisecond)
+	}
+}
+
+func (log *log) newBuffer() {
+	log.buffer.timestampMap = make(map[common.SeriesIDV2][]time.Time)
+	log.buffer.valueMap = make(map[common.SeriesIDV2][]byte)
+	log.buffer.callbackMap = make(map[common.SeriesIDV2][]func(common.SeriesIDV2, time.Time, []byte, error))
+	log.buffer.count = 0
+}
+
+func (log *log) flushBuffer(buffer buffer) error {
+	if buffer.count == 0 {
+		return nil
+	}
+
+	defer func() {
+		log.bytesBuffer.Reset()
+		log.timestampsBuffer.Reset()
+	}()
+
+	// placeholder, preset batch length value is 0
+	batchLen := 0
+	if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(batchLen)); err != nil {
+		return errors.Wrap(err, "Write batch length error")
+	}
+	for seriesID, timestamps := range buffer.timestampMap {
+		// Generate seriesID binary
+		seriesIDBytes := seriesID.Marshal()
+		seriesIDBytesLen := len(seriesIDBytes)
+
+		// Generate timestamps compression binary
+		log.timestampsBuffer.Reset()
+		timestampWriter := encoding.NewWriter()
+		timestampEncoder := encoding.NewXOREncoder(timestampWriter)
+		timestampWriter.Reset(log.timestampsBuffer)
+		for _, timestamp := range timestamps {
+			timestampEncoder.Write(timeTouUnixNano(timestamp))
+		}
+		timestampWriter.Flush()
+		timestampsBytes := log.timestampsBuffer.Bytes()
+		timestampsBytesLen := len(timestampsBytes)
+
+		// Generate values compression binary
+		valuesBytes := snappy.Encode(nil, buffer.valueMap[seriesID])
+
+		// Write entry data
+		entryLen := seriesIDLength + seriesIDBytesLen + seriesCountLength + timestampsBinaryLength + timestampsBytesLen + len(valuesBytes)
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int64(entryLen)); err != nil {
+			return errors.Wrap(err, "Write entry length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int16(seriesIDBytesLen)); err != nil {
+			return errors.Wrap(err, "Write seriesID length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, seriesIDBytes); err != nil {
+			return errors.Wrap(err, "Write seriesID error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int32(len(timestamps))); err != nil {
+			return errors.Wrap(err, "Write series count error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, int16(timestampsBytesLen)); err != nil {
+			return errors.Wrap(err, "Write timestamps length error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, timestampsBytes); err != nil {
+			return errors.Wrap(err, "Write timestamps error")
+		}
+		if err := binary.Write(log.bytesBuffer, binary.LittleEndian, valuesBytes); err != nil {
+			return errors.Wrap(err, "Write values error")
+		}
+	}
+	// Rewrite batch length
+	batchBytes := log.bytesBuffer.Bytes()
+	batchLen = len(batchBytes) - batchLength
+	rewriteInt64InBuf(batchBytes, int64(batchLen), 0, binary.LittleEndian)
+
+	return log.writeWorkSegment(batchBytes)
+}
+
+func (log *log) swapWorkSegment() (Segment, error) {
+	log.workSegmentMutex.Lock()
+	defer log.workSegmentMutex.Unlock()
+
+	if err := log.workSegment.file.Close(); err != nil {
+		return nil, errors.Wrap(err, "Close WAL segment error")
+	}
+
+	oldSegment := log.workSegment
+	// Create new segment.
+	segmentID := log.workSegment.segmentID + 1
+	segment := &segment{
+		segmentID: segmentID,
+		path:      filepath.Join(log.path, segmentName(uint64(segmentID))),
+	}
+	var err error
+	segment.file, err = os.OpenFile(segment.path, os.O_CREATE|os.O_RDWR|os.O_TRUNC, os.ModePerm)
+	if err != nil {
+		return nil, errors.Wrap(err, "Open WAL segment error")
+	}
+	log.workSegment = segment
+
+	return oldSegment, nil
+}
+
+func (log *log) writeWorkSegment(data []byte) error {
+	log.workSegmentMutex.Lock()
+	defer log.workSegmentMutex.Unlock()
+
+	// Write batch data to WAL segment file
+	if _, err := log.workSegment.file.Write(data); err != nil {
+		return errors.Wrap(err, "Write WAL segment file error, file: "+log.workSegment.path)
+	}
+	if err := log.workSegment.file.Sync(); err != nil {
+		log.logger.Warn().Msg("Sync WAL segment file to disk failed, file: " + log.workSegment.path)
+	}
+	return nil
+}
+
+func (log *log) load() error {
+	files, err := os.ReadDir(log.path)
+	if err != nil {
+		return errors.Wrap(err, "Can not read dir: "+log.path)
+	}
+	// Load all of WAL segments.
+	var workSegmentID SegmentID
+	log.segmentMap = make(map[SegmentID]*segment)
+	for _, file := range files {
+		name := file.Name()
+		segmentID, parsePathErr := strconv.ParseUint(name[3:19], 10, 64)
+		if parsePathErr != nil {
+			return errors.Wrap(parsePathErr, "Parse file name error, name: "+name)
+		}
+		if segmentID > uint64(workSegmentID) {
+			workSegmentID = SegmentID(segmentID)
+		}
+		segment := &segment{
+			segmentID: SegmentID(segmentID),
+			path:      filepath.Join(log.path, segmentName(segmentID)),
+		}
+		if err = segment.parseLogEntries(); err != nil {
+			return errors.Wrap(err, "Fail to parse log entries")
+		}
+		log.segmentMap[SegmentID(segmentID)] = segment
+
+		if log.logger.Debug().Enabled() {
+			log.logger.Debug().Msg("Loaded segment file: " + segment.path)
+		}
+	}
+
+	// If load first time.
+	if len(log.segmentMap) == 0 {
+		segment := &segment{
+			segmentID: 1,
+			path:      filepath.Join(log.path, segmentName(1)),
+		}
+		log.segmentMap[1] = segment
+		log.workSegment = segment
+	} else {
+		log.workSegment = log.segmentMap[workSegmentID]
+	}
+	log.workSegment.file, err = os.OpenFile(log.workSegment.path, os.O_CREATE|os.O_RDWR|os.O_APPEND, os.ModePerm)
+	if err != nil {
+		return errors.Wrap(err, "Open WAL segment error, file: "+log.workSegment.path)
+	}
+	return nil
+}
+
+func (segment *segment) GetSegmentID() SegmentID {
+	return segment.segmentID
+}
+
+func (segment *segment) GetLogEntries() []LogEntry {
+	return segment.logEntries
+}
+
+func (segment *segment) parseLogEntries() error {
+	segmentBytes, err := os.ReadFile(segment.path)
+	if err != nil {
+		return errors.Wrap(err, "Read WAL segment failed, path: "+segment.path)
+	}
+
+	var logEntries []LogEntry
+	var data []byte
+	var batchLen int64
+	var entryLen int64
+	var seriesIDLen int16
+	var seriesID common.SeriesIDV2
+	var seriesCount int32
+	var timestampsBinaryLen int16
+	var entryEndPos int
+	var bytesBuf *bytes.Buffer
+
+	oldPos := 0
+	pos := 0
+	parseNextBatchFlag := true
+
+	for {
+		if parseNextBatchFlag {
+			if len(segmentBytes) <= batchLength {
+				break
+			}
+			data = segmentBytes[pos : pos+batchLength]
+			bytesBuf = bytes.NewBuffer(data)
+			if err = binary.Read(bytesBuf, binary.LittleEndian, &batchLen); err != nil {
+				return errors.Wrap(err, "Read batch length fail to convert from bytes")
+			}
+			if len(segmentBytes) <= int(batchLen) {
+				break
+			}
+
+			pos += batchLength
+			oldPos = pos
+			parseNextBatchFlag = false
+		}
+
+		// Parse entryLength.
+		data = segmentBytes[pos : pos+entryLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &entryLen); err != nil {
+			return errors.Wrap(err, "Read entry length fail to convert from byte")
+		}
+		pos += entryLength
+
+		// Mark entry end-position
+		entryEndPos = pos + int(entryLen)
+		if len(segmentBytes) < entryEndPos {
+			break
+		}
+
+		// Parse seriesIDLen.
+		data = segmentBytes[pos : pos+seriesIDLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &seriesIDLen); err != nil {
+			return errors.Wrap(err, "Read seriesID length fail to convert from byte")
+		}
+		pos += seriesIDLength
+
+		// Parse seriesID.
+		data = segmentBytes[pos : pos+int(seriesIDLen)]
+		seriesID = common.ParseSeriesIDV2(data)
+		pos += int(seriesIDLen)
+
+		// Parse series count.
+		data = segmentBytes[pos : pos+seriesCountLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &seriesCount); err != nil {
+			return errors.Wrap(err, "Read series count fail to convert from byte")
+		}
+		pos += seriesCountLength
+
+		// Parse timestamps compression binary.
+		data = segmentBytes[pos : pos+timestampsBinaryLength]
+		bytesBuf = bytes.NewBuffer(data)
+		if err = binary.Read(bytesBuf, binary.LittleEndian, &timestampsBinaryLen); err != nil {
+			return errors.Wrap(err, "Read timestamps compression binary length fail to convert from byte")
+		}
+		pos += timestampsBinaryLength
+		data = segmentBytes[pos : pos+int(timestampsBinaryLen)]
+		timestampReader := encoding.NewReader(bytes.NewReader(data))
+		timestampDecoder := encoding.NewXORDecoder(timestampReader)
+		var timestamps []time.Time
+		for i := 0; i < int(seriesCount); i++ {
+			if !timestampDecoder.Next() {
+				return errors.Wrap(err, "Timestamps length not match series count, index: "+strconv.Itoa(i))
+			}
+			timestamps = append(timestamps, unixNanoToTime(timestampDecoder.Value()))
+		}
+		pos += int(timestampsBinaryLen)
+
+		// Parse values compression binary.
+		data = segmentBytes[pos:entryEndPos]
+		if data, err = snappy.Decode(nil, data); err != nil {
+			return errors.Wrap(err, "Decode values compression binary fail to snappy decode")
+		}
+		values := parseValuesBinary(data)
+		if values.Len() != len(timestamps) {
+			return errors.New("Timestamps length and values length not match: " + strconv.Itoa(len(timestamps)) + " vs " + strconv.Itoa(values.Len()))
+		}
+		pos = entryEndPos
+
+		logEntry := &logEntry{
+			entryLength: entryLen,
+			seriesID:    seriesID,
+			count:       seriesCount,
+			timestamps:  timestamps,
+			values:      values,
+		}
+		logEntries = append(logEntries, logEntry)
+
+		if pos == len(segmentBytes) {
+			break
+		}
+		if pos-oldPos == int(batchLen) {
+			parseNextBatchFlag = true
+		}
+	}
+	segment.logEntries = logEntries
+	return nil
+}
+
+func (logEntry *logEntry) GetSeriesID() common.SeriesIDV2 {
+	return logEntry.seriesID
+}
+
+func (logEntry *logEntry) GetTimestamps() []time.Time {
+	return logEntry.timestamps
+}
+
+func (logEntry *logEntry) GetValues() *list.List {
+	return logEntry.values
+}
+
+func (buffer *buffer) write(request logRequest) {
+	seriesID := request.seriesID
+	buffer.timestampMap[seriesID] = append(buffer.timestampMap[seriesID], request.timestamp)
+
+	// Value item: binary-length(2-bytes) + binary data(n-bytes)
+	binaryLength := int16ToBytes(int16(len(request.data)))
+	buffer.valueMap[seriesID] = append(buffer.valueMap[seriesID], binaryLength...)
+	buffer.valueMap[seriesID] = append(buffer.valueMap[seriesID], request.data...)
+
+	buffer.callbackMap[seriesID] = append(buffer.callbackMap[seriesID], request.callback)
+	buffer.count++
+}
+
+func (buffer *buffer) notifyRequests(err error) {
+	for seriesID, callbacks := range buffer.callbackMap {
+		timestamps := buffer.timestampMap[seriesID]
+		values := buffer.valueMap[seriesID]
+		valuePos := 0
+		var valueItem []byte
+		for index, callback := range callbacks {
+			valuePos, valueItem = readValuesBinary(values, valuePos, valuesBinaryLength)
+			tryCallback(func() {
+				callback(seriesID, timestamps[index], valueItem, err)
+			})
+		}
+	}
+}
+
+func segmentName(index uint64) string {
+	return fmt.Sprintf("%v%016x%v", segmentNamePrefix, index, segmentNameSuffix)
+}
+
+func tryCallback(callback func()) {
+	defer func() {
+		if err := recover(); err != nil {
+			fmt.Printf("Run callback error is %v\n", err)
+		}
+	}()
+	callback()
+}
+
+func parseValuesBinary(binary []byte) *list.List {
+	values := list.New()
+	position := 0
+	for {
+		nextPosition, value := readValuesBinary(binary, position, valuesBinaryLength)
+		if value == nil {
+			break
+		}
+		values.PushBack(value)
+		position = nextPosition
+	}
+	return values
+}
+
+func readValuesBinary(raw []byte, position int, offsetLen int) (int, []byte) {
+	if position == len(raw) {
+		return position, nil
+	}
+
+	data := raw[position : position+offsetLen]
+	binaryLen := bytesToInt16(data)
+	position += offsetLen
+
+	data = raw[position : position+int(binaryLen)]
+	position += int(binaryLen)
+	return position, data
+}
+
+func rewriteInt64InBuf(buf []byte, value int64, offset int, order binary.ByteOrder) {
+	_ = buf[offset+7] // early bounds check to guarantee safety of writes below
+	if order == binary.LittleEndian {

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [skywalking-banyandb] HHoflittlefish777 commented on a diff in pull request #261: Implement Write-ahead Logging

Posted by "HHoflittlefish777 (via GitHub)" <gi...@apache.org>.
HHoflittlefish777 commented on code in PR #261:
URL: https://github.com/apache/skywalking-banyandb/pull/261#discussion_r1241037492


##########
pkg/wal/wal.go:
##########
@@ -153,147 +157,258 @@ func New(path string, options *Options) (WAL, error) {
 	if options.BufferSize <= 0 {
 		options.BufferSize = DefaultOptions.BufferSize
 	}
+	if options.BufferBatchInterval <= 0 {
+		options.BufferBatchInterval = DefaultOptions.BufferBatchInterval
+	}
 
 	// Initial WAL path.
 	path, err := filepath.Abs(path)
 	if err != nil {
-		return nil, errors.Wrap(err, "Can not get absolute path")
+		return nil, errors.Wrap(err, "Can not get absolute path: "+path)
 	}
-	log := &Log{path: path, options: *options}
 	if err := os.MkdirAll(path, os.ModePerm); err != nil {
 		return nil, err
 	}
 
+	log := &Log{path: path, options: *options, logger: logger.GetLogger(moduleName)}
+
 	if err := log.load(); err != nil {
 		return nil, err
 	}
-	log.runFlushTask()
+	log.startAsyncFlushTask()
+
+	log.logger.Info().Msgf("WAL initialized at %s", path)
 	return log, nil
 }
 
-func (log *Log) runFlushTask() {
+// Write a logging entity.
+// It will return immediately when the data is written in the buffer,
+// The callback function will be called when the entity is flushed on the persistent storage.
+func (log *Log) Write(seriesID common.SeriesIDV2, timestamp time.Time, data []byte,
+	callback func(common.SeriesIDV2, time.Time, []byte, error)) {
+	log.writeChannel <- logRequest{
+		seriesID:  seriesID,
+		timestamp: timestamp,
+		data:      data,
+		callback:  callback,
+	}
+}
+
+// Read specified segment by SegmentID.
+func (log *Log) Read(segmentID SegmentID) (Segment, error) {
+	segment := log.segmentIndexMap[segmentID]
+	return segment, nil
+}
+
+// ReadAllSegments reads all segments sorted by their creation time in ascending order.
+func (log *Log) ReadAllSegments() ([]Segment, error) {
+	segments := make([]Segment, 0)
+	for _, segment := range log.segmentIndexMap {
+		segments = append(segments, segment)
+	}
+	return segments, nil
+}
+
+// Rotate closes the open segment and opens a new one, returning the closed segment details.
+func (log *Log) Rotate() (Segment, error) {
+	// 需要锁定

Review Comment:
   Please use English



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org