Make Future generic (#22411)

Signed-off-by: yah01 <yang.cen@zilliz.com>
pull/22412/head
yah01 2023-02-27 12:07:47 +08:00 committed by GitHub
parent 3d6bded115
commit b758c305a7
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 94 additions and 21 deletions

View File

@ -203,7 +203,7 @@ func (dsService *dataSyncService) initNodes(vchanInfo *datapb.VchannelInfo, tick
return err
}
futures := make([]*concurrency.Future, 0, len(unflushedSegmentInfos)+len(flushedSegmentInfos))
futures := make([]*concurrency.Future[any], 0, len(unflushedSegmentInfos)+len(flushedSegmentInfos))
for _, us := range unflushedSegmentInfos {
if us.CollectionID != dsService.collectionID ||

View File

@ -23,7 +23,7 @@ func Test_getOrCreateIOPool(t *testing.T) {
go func() {
defer wg.Done()
p := getOrCreateIOPool()
futures := make([]*concurrency.Future, 0, nTask)
futures := make([]*concurrency.Future[any], 0, nTask)
for j := 0; j < nTask; j++ {
future := p.Submit(func() (interface{}, error) {
return nil, nil

View File

@ -319,7 +319,7 @@ func (loader *segmentLoader) loadGrowingSegmentFields(ctx context.Context, segme
iCodec := storage.InsertCodec{}
// change all field bin log loading into concurrent
loadFutures := make([]*concurrency.Future, 0, len(fieldBinlogs))
loadFutures := make([]*concurrency.Future[any], 0, len(fieldBinlogs))
for _, fieldBinlog := range fieldBinlogs {
futures := loader.loadFieldBinlogsAsync(ctx, fieldBinlog)
loadFutures = append(loadFutures, futures...)
@ -427,8 +427,8 @@ func (loader *segmentLoader) loadSealedField(ctx context.Context, segment *Segme
}
// Load binlogs concurrently into memory from KV storage asyncly
func (loader *segmentLoader) loadFieldBinlogsAsync(ctx context.Context, field *datapb.FieldBinlog) []*concurrency.Future {
futures := make([]*concurrency.Future, 0, len(field.Binlogs))
func (loader *segmentLoader) loadFieldBinlogsAsync(ctx context.Context, field *datapb.FieldBinlog) []*concurrency.Future[any] {
futures := make([]*concurrency.Future[any], 0, len(field.Binlogs))
for i := range field.Binlogs {
path := field.Binlogs[i].GetLogPath()
future := loader.ioPool.Submit(func() (interface{}, error) {
@ -473,7 +473,7 @@ func (loader *segmentLoader) loadFieldIndexData(ctx context.Context, segment *Se
log := log.With(zap.Int64("segment", segment.ID()))
indexBuffer := make([][]byte, 0, len(indexInfo.IndexFilePaths))
filteredPaths := make([]string, 0, len(indexInfo.IndexFilePaths))
futures := make([]*concurrency.Future, 0, len(indexInfo.IndexFilePaths))
futures := make([]*concurrency.Future[any], 0, len(indexInfo.IndexFilePaths))
indexCodec := storage.NewIndexFileBinlogCodec()
// TODO, remove the load index info froam

View File

@ -16,31 +16,40 @@
package concurrency
type future interface {
wait()
OK() bool
Err() error
}
// Future is a result type of async-await style.
// It contains the result (or error) of an async task.
// Trying to obtain the result (or error) blocks until the async task completes.
type Future struct {
type Future[T any] struct {
ch chan struct{}
value interface{}
value T
err error
}
func newFuture() *Future {
return &Future{
func newFuture[T any]() *Future[T] {
return &Future[T]{
ch: make(chan struct{}),
}
}
// Return the result and error of the async task.
func (future *Future) Await() (interface{}, error) {
func (future *Future[T]) wait() {
<-future.ch
}
// Return the result and error of the async task.
func (future *Future[T]) Await() (T, error) {
future.wait()
return future.value, future.err
}
// Return the result of the async task,
// nil if no result or error occurred.
func (future *Future) Value() interface{} {
func (future *Future[T]) Value() T {
<-future.ch
return future.value
@ -48,7 +57,7 @@ func (future *Future) Value() interface{} {
// False if error occurred,
// true otherwise.
func (future *Future) OK() bool {
func (future *Future[T]) OK() bool {
<-future.ch
return future.err == nil
@ -56,7 +65,7 @@ func (future *Future) OK() bool {
// Return the error of the async task,
// nil if no error.
func (future *Future) Err() error {
func (future *Future[T]) Err() error {
<-future.ch
return future.err
@ -65,17 +74,29 @@ func (future *Future) Err() error {
// Return a read-only channel,
// which will be closed if the async task completes.
// Use this if you need to wait the async task in a select statement.
func (future *Future) Inner() <-chan struct{} {
func (future *Future[T]) Inner() <-chan struct{} {
return future.ch
}
// Go spawns a goroutine to execute fn,
// returns a future that contains the result of fn.
// NOTE: use Pool if you need limited goroutines.
func Go[T any](fn func() (T, error)) *Future[T] {
future := newFuture[T]()
go func() {
future.value, future.err = fn()
close(future.ch)
}()
return future
}
// Await for multiple futures,
// Return nil if no future returns error,
// or return the first error in these futures.
func AwaitAll(futures ...*Future) error {
func AwaitAll[T future](futures ...T) error {
for i := range futures {
if !futures[i].OK() {
return futures[i].err
return futures[i].Err()
}
}

View File

@ -0,0 +1,51 @@
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package concurrency
import (
"testing"
"time"
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/suite"
)
type FutureSuite struct {
suite.Suite
}
func (s *FutureSuite) TestFuture() {
const sleepDuration = 200 * time.Millisecond
errFuture := Go(func() (any, error) {
time.Sleep(sleepDuration)
return nil, errors.New("errFuture")
})
resultFuture := Go(func() (int, error) {
time.Sleep(sleepDuration)
return 10, nil
})
s.False(errFuture.OK())
s.True(resultFuture.OK())
s.Error(errFuture.Err())
s.Equal(10, resultFuture.Value())
}
func TestFuture(t *testing.T) {
suite.Run(t, new(FutureSuite))
}

View File

@ -39,8 +39,9 @@ func NewPool(cap int, opts ...ants.Option) (*Pool, error) {
// Submit a task into the pool,
// executes it asynchronously.
// This will block if the pool has finite workers and no idle worker.
func (pool *Pool) Submit(method func() (interface{}, error)) *Future {
future := newFuture()
// NOTE: As now golang doesn't support the member method being generic, we use Future[any]
func (pool *Pool) Submit(method func() (any, error)) *Future[any] {
future := newFuture[any]()
err := pool.inner.Submit(func() {
defer close(future.ch)
res, err := method()

View File

@ -29,7 +29,7 @@ func TestPool(t *testing.T) {
assert.NoError(t, err)
taskNum := pool.Cap() * 2
futures := make([]*Future, 0, taskNum)
futures := make([]*Future[any], 0, taskNum)
for i := 0; i < taskNum; i++ {
res := i
future := pool.Submit(func() (interface{}, error) {