Use log id instead of log path of the binlog in metastore (#19123)

Signed-off-by: yun.zhang <yun.zhang@zilliz.com>

Signed-off-by: yun.zhang <yun.zhang@zilliz.com>
pull/19428/head
jaime 2022-09-25 15:56:51 +08:00 committed by GitHub
parent 29d2c0bd7b
commit 91df8f2d6a
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
21 changed files with 1351 additions and 696 deletions

View File

@ -44,10 +44,10 @@ type meta struct {
}
// NewMeta creates meta from provided `kv.TxnKV`
func newMeta(ctx context.Context, kv kv.TxnKV) (*meta, error) {
func newMeta(ctx context.Context, kv kv.TxnKV, chunkManagerRootPath string) (*meta, error) {
mt := &meta{
ctx: ctx,
catalog: &datacoord.Catalog{Txn: kv},
catalog: &datacoord.Catalog{Txn: kv, ChunkManagerRootPath: chunkManagerRootPath},
collections: make(map[UniqueID]*datapb.CollectionInfo),
segments: NewSegmentsInfo(),
}
@ -504,8 +504,6 @@ func (m *meta) mergeDropSegment(seg2Drop *SegmentInfo) *SegmentInfo {
// since the flag is not marked so DataNode can re-consume the drop collection msg
// 2. when failure occurs between save meta and unwatch channel, the removal flag shall be check before let datanode watch this channel
func (m *meta) batchSaveDropSegments(channel string, modSegments map[int64]*SegmentInfo) error {
// TODO: RootCoord supports read-write prohibit when dropping collection
// divides two api calls: save dropped segments & mark channel deleted
segments := make([]*datapb.SegmentInfo, 0)
for _, seg := range modSegments {
segments = append(segments, seg.SegmentInfo)
@ -817,12 +815,17 @@ func (m *meta) CompleteMergeCompaction(compactionLogs []*datapb.CompactionSegmen
zap.Int64("NumOfRows", segmentInfo.NumOfRows),
zap.Any("compactionFrom", segmentInfo.CompactionFrom))
modSegments := make([]*datapb.SegmentInfo, 0, len(segments))
modSegments := make([]*datapb.SegmentInfo, 0)
for _, s := range segments {
modSegments = append(modSegments, s.SegmentInfo)
}
if err := m.catalog.AlterSegmentsAndAddNewSegment(m.ctx, modSegments, segment.SegmentInfo); err != nil {
var newSegment *datapb.SegmentInfo
if segment.SegmentInfo.NumOfRows > 0 {
newSegment = segment.SegmentInfo
}
if err := m.catalog.AlterSegmentsAndAddNewSegment(m.ctx, modSegments, newSegment); err != nil {
return err
}

View File

@ -179,55 +179,55 @@ func (mek *mockKvIllegalStatslog) LoadWithPrefix(key string) ([]string, []string
func TestMetaReloadFromKV(t *testing.T) {
t.Run("Test ReloadFromKV success", func(t *testing.T) {
fkv := &mockEtcdKv{}
_, err := newMeta(context.TODO(), fkv)
_, err := newMeta(context.TODO(), fkv, "")
assert.Nil(t, err)
})
// load segment error
t.Run("Test ReloadFromKV load segment fails", func(t *testing.T) {
fkv := &mockKvLoadSegmentError{}
_, err := newMeta(context.TODO(), fkv)
_, err := newMeta(context.TODO(), fkv, "")
assert.NotNil(t, err)
})
// illegal segment info
t.Run("Test ReloadFromKV unmarshal segment fails", func(t *testing.T) {
fkv := &mockKvIllegalSegment{}
_, err := newMeta(context.TODO(), fkv)
_, err := newMeta(context.TODO(), fkv, "")
assert.NotNil(t, err)
})
// load binlog/deltalog/statslog error
t.Run("Test ReloadFromKV load binlog fails", func(t *testing.T) {
fkv := &mockKvLoadBinlogError{}
_, err := newMeta(context.TODO(), fkv)
_, err := newMeta(context.TODO(), fkv, "")
assert.NotNil(t, err)
})
t.Run("Test ReloadFromKV load deltalog fails", func(t *testing.T) {
fkv := &mockKvLoadDeltaBinlogError{}
_, err := newMeta(context.TODO(), fkv)
_, err := newMeta(context.TODO(), fkv, "")
assert.NotNil(t, err)
})
t.Run("Test ReloadFromKV load statslog fails", func(t *testing.T) {
fkv := &mockKvLoadStatsBinlogError{}
_, err := newMeta(context.TODO(), fkv)
_, err := newMeta(context.TODO(), fkv, "")
assert.NotNil(t, err)
})
// illegal binlog/deltalog/statslog info
t.Run("Test ReloadFromKV unmarshal binlog fails", func(t *testing.T) {
fkv := &mockKvIllegalBinlog{}
_, err := newMeta(context.TODO(), fkv)
_, err := newMeta(context.TODO(), fkv, "")
assert.NotNil(t, err)
})
t.Run("Test ReloadFromKV unmarshal deltalog fails", func(t *testing.T) {
fkv := &mockKvIllegalDeltalog{}
_, err := newMeta(context.TODO(), fkv)
_, err := newMeta(context.TODO(), fkv, "")
assert.NotNil(t, err)
})
t.Run("Test ReloadFromKV unmarshal statslog fails", func(t *testing.T) {
fkv := &mockKvIllegalStatslog{}
_, err := newMeta(context.TODO(), fkv)
_, err := newMeta(context.TODO(), fkv, "")
assert.NotNil(t, err)
})
}
@ -335,14 +335,14 @@ func TestMeta_Basic(t *testing.T) {
// inject error for `Save`
memoryKV := memkv.NewMemoryKV()
fkv := &saveFailKV{TxnKV: memoryKV}
meta, err := newMeta(context.TODO(), fkv)
meta, err := newMeta(context.TODO(), fkv, "")
assert.Nil(t, err)
err = meta.AddSegment(NewSegmentInfo(&datapb.SegmentInfo{}))
assert.NotNil(t, err)
fkv2 := &removeFailKV{TxnKV: memoryKV}
meta, err = newMeta(context.TODO(), fkv2)
meta, err = newMeta(context.TODO(), fkv2, "")
assert.Nil(t, err)
// nil, since no segment yet
err = meta.DropSegment(0)
@ -434,7 +434,7 @@ func TestGetUnFlushedSegments(t *testing.T) {
func TestUpdateFlushSegmentsInfo(t *testing.T) {
t.Run("normal", func(t *testing.T) {
meta, err := newMeta(context.TODO(), memkv.NewMemoryKV())
meta, err := newMeta(context.TODO(), memkv.NewMemoryKV(), "")
assert.Nil(t, err)
segment1 := &SegmentInfo{SegmentInfo: &datapb.SegmentInfo{ID: 1, State: commonpb.SegmentState_Growing, Binlogs: []*datapb.FieldBinlog{getFieldBinlogPaths(1, "binlog0")},
@ -460,7 +460,7 @@ func TestUpdateFlushSegmentsInfo(t *testing.T) {
})
t.Run("update non-existed segment", func(t *testing.T) {
meta, err := newMeta(context.TODO(), memkv.NewMemoryKV())
meta, err := newMeta(context.TODO(), memkv.NewMemoryKV(), "")
assert.Nil(t, err)
err = meta.UpdateFlushSegmentsInfo(1, false, false, false, nil, nil, nil, nil, nil)
@ -468,7 +468,7 @@ func TestUpdateFlushSegmentsInfo(t *testing.T) {
})
t.Run("update checkpoints and start position of non existed segment", func(t *testing.T) {
meta, err := newMeta(context.TODO(), memkv.NewMemoryKV())
meta, err := newMeta(context.TODO(), memkv.NewMemoryKV(), "")
assert.Nil(t, err)
segment1 := &SegmentInfo{SegmentInfo: &datapb.SegmentInfo{ID: 1, State: commonpb.SegmentState_Growing}}
@ -485,7 +485,7 @@ func TestUpdateFlushSegmentsInfo(t *testing.T) {
t.Run("test save etcd failed", func(t *testing.T) {
kv := memkv.NewMemoryKV()
failedKv := &saveFailKV{kv}
meta, err := newMeta(context.TODO(), failedKv)
meta, err := newMeta(context.TODO(), failedKv, "")
assert.Nil(t, err)
segmentInfo := &SegmentInfo{
@ -513,7 +513,7 @@ func TestUpdateFlushSegmentsInfo(t *testing.T) {
func TestSaveHandoffMeta(t *testing.T) {
kvClient := memkv.NewMemoryKV()
meta, err := newMeta(context.TODO(), kvClient)
meta, err := newMeta(context.TODO(), kvClient, "")
assert.Nil(t, err)
info := &datapb.SegmentInfo{

View File

@ -38,7 +38,7 @@ import (
func newMemoryMeta(allocator allocator) (*meta, error) {
memoryKV := memkv.NewMemoryKV()
return newMeta(context.TODO(), memoryKV)
return newMeta(context.TODO(), memoryKV, "")
}
var _ allocator = (*MockAllocator)(nil)

View File

@ -492,7 +492,7 @@ func TestTryToSealSegment(t *testing.T) {
mockAllocator := newMockAllocator()
memoryKV := memkv.NewMemoryKV()
fkv := &saveFailKV{TxnKV: memoryKV}
meta, err := newMeta(context.TODO(), memoryKV)
meta, err := newMeta(context.TODO(), memoryKV, "")
assert.Nil(t, err)
@ -518,7 +518,7 @@ func TestTryToSealSegment(t *testing.T) {
mockAllocator := newMockAllocator()
memoryKV := memkv.NewMemoryKV()
fkv := &saveFailKV{TxnKV: memoryKV}
meta, err := newMeta(context.TODO(), memoryKV)
meta, err := newMeta(context.TODO(), memoryKV, "")
assert.Nil(t, err)

View File

@ -274,7 +274,12 @@ func (s *Server) Start() error {
return err
}
if err = s.initMeta(); err != nil {
storageCli, err := s.newChunkManagerFactory()
if err != nil {
return err
}
if err = s.initMeta(storageCli.RootPath()); err != nil {
return err
}
@ -296,9 +301,7 @@ func (s *Server) Start() error {
}
s.startSegmentManager()
if err = s.initGarbageCollection(); err != nil {
return err
}
s.initGarbageCollection(storageCli)
s.startServerLoop()
Params.DataCoordCfg.CreatedTime = time.Now()
@ -358,17 +361,17 @@ func (s *Server) stopCompactionTrigger() {
s.compactionTrigger.stop()
}
func (s *Server) initGarbageCollection() error {
var cli storage.ChunkManager
var err error
func (s *Server) newChunkManagerFactory() (storage.ChunkManager, error) {
chunkManagerFactory := storage.NewChunkManagerFactoryWithParam(&Params)
cli, err = chunkManagerFactory.NewPersistentStorageChunkManager(s.ctx)
cli, err := chunkManagerFactory.NewPersistentStorageChunkManager(s.ctx)
if err != nil {
log.Error("chunk manager init failed", zap.Error(err))
return err
return nil, err
}
log.Info("Datacoord garbage collector chunk manager init success")
return cli, err
}
func (s *Server) initGarbageCollection(cli storage.ChunkManager) {
s.garbageCollector = newGarbageCollector(s.meta, s.segReferManager, s.indexCoord, GcOption{
cli: cli,
enabled: Params.DataCoordCfg.EnableGarbageCollection,
@ -376,7 +379,6 @@ func (s *Server) initGarbageCollection() error {
missingTolerance: Params.DataCoordCfg.GCMissingTolerance,
dropTolerance: Params.DataCoordCfg.GCDropTolerance,
})
return nil
}
// here we use variable for test convenience
@ -461,12 +463,12 @@ func (s *Server) startSegmentManager() {
}
}
func (s *Server) initMeta() error {
func (s *Server) initMeta(chunkManagerRootPath string) error {
etcdKV := etcdkv.NewEtcdKV(s.etcdCli, Params.EtcdCfg.MetaRootPath)
s.kvClient = etcdKV
reloadEtcdFn := func() error {
var err error
s.meta, err = newMeta(s.ctx, s.kvClient)
s.meta, err = newMeta(s.ctx, s.kvClient, chunkManagerRootPath)
if err != nil {
return err
}

View File

@ -2678,7 +2678,7 @@ func TestDataCoordServer_SetSegmentState(t *testing.T) {
svr.meta.Lock()
func() {
defer svr.meta.Unlock()
svr.meta, _ = newMeta(context.TODO(), &mockTxnKVext{})
svr.meta, _ = newMeta(context.TODO(), &mockTxnKVext{}, "")
}()
defer closeTestServer(t, svr)
segment := &datapb.SegmentInfo{
@ -3083,13 +3083,14 @@ func Test_initServiceDiscovery(t *testing.T) {
closeTestServer(t, server)
}
func Test_initGarbageCollection(t *testing.T) {
func Test_newChunkManagerFactory(t *testing.T) {
server := newTestServer2(t, nil)
Params.DataCoordCfg.EnableGarbageCollection = true
t.Run("err_minio_bad_address", func(t *testing.T) {
Params.MinioCfg.Address = "host:9000:bad"
err := server.initGarbageCollection()
storageCli, err := server.newChunkManagerFactory()
assert.Nil(t, storageCli)
assert.Error(t, err)
assert.Contains(t, err.Error(), "too many colons in address")
})
@ -3103,19 +3104,51 @@ func Test_initGarbageCollection(t *testing.T) {
getCheckBucketFn = getCheckBucketFnBak
}()
Params.MinioCfg.Address = "minio:9000"
t.Run("ok", func(t *testing.T) {
err := server.initGarbageCollection()
storageCli, err := server.newChunkManagerFactory()
assert.NotNil(t, storageCli)
assert.NoError(t, err)
})
t.Run("iam_ok", func(t *testing.T) {
Params.MinioCfg.UseIAM = true
err := server.initGarbageCollection()
storageCli, err := server.newChunkManagerFactory()
assert.Nil(t, storageCli)
assert.Error(t, err)
assert.Contains(t, err.Error(), "404 Not Found")
})
t.Run("local storage init", func(t *testing.T) {
Params.CommonCfg.StorageType = "local"
err := server.initGarbageCollection()
storageCli, err := server.newChunkManagerFactory()
assert.NotNil(t, storageCli)
assert.NoError(t, err)
})
t.Run("bad storage type", func(t *testing.T) {
Params.CommonCfg.StorageType = "bad"
storageCli, err := server.newChunkManagerFactory()
assert.Nil(t, storageCli)
assert.Error(t, err)
})
}
func Test_initGarbageCollection(t *testing.T) {
server := newTestServer2(t, nil)
Params.DataCoordCfg.EnableGarbageCollection = true
// mock CheckBucketFn
getCheckBucketFnBak := getCheckBucketFn
getCheckBucketFn = func(cli *minio.Client) func() error {
return func() error { return nil }
}
defer func() {
getCheckBucketFn = getCheckBucketFnBak
}()
Params.MinioCfg.Address = "minio:9000"
t.Run("ok", func(t *testing.T) {
storageCli, err := server.newChunkManagerFactory()
assert.NotNil(t, storageCli)
assert.NoError(t, err)
server.initGarbageCollection(storageCli)
})
}

View File

@ -19,8 +19,8 @@ package datanode
import (
"context"
"errors"
"path"
"strconv"
"github.com/milvus-io/milvus/internal/util/metautil"
"github.com/milvus-io/milvus/api/commonpb"
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
@ -98,14 +98,5 @@ func (alloc *allocator) genKey(ids ...UniqueID) (string, error) {
return "", err
}
ids = append(ids, idx)
return JoinIDPath(ids...), nil
}
// JoinIDPath joins ids to path format.
func JoinIDPath(ids ...UniqueID) string {
idStr := make([]string, 0, len(ids))
for _, id := range ids {
idStr = append(idStr, strconv.FormatInt(id, 10))
}
return path.Join(idStr...)
return metautil.JoinIDPath(ids...), nil
}

View File

@ -23,6 +23,8 @@ import (
"strconv"
"time"
"github.com/milvus-io/milvus/internal/util/metautil"
"github.com/milvus-io/milvus/internal/common"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/proto/datapb"
@ -165,7 +167,7 @@ func (b *binlogIO) upload(
if err != nil {
return nil, err
}
k := JoinIDPath(meta.GetID(), partID, segID, pkID, logID)
k := metautil.JoinIDPath(meta.GetID(), partID, segID, pkID, logID)
key := path.Join(b.ChunkManager.RootPath(), common.SegmentStatslogPath, k)
fileLen := len(segStats)
@ -270,7 +272,7 @@ func (b *binlogIO) genInsertBlobs(data *InsertData, partID, segID UniqueID, meta
for _, blob := range inlogs {
// Blob Key is generated by Serialize from int64 fieldID in collection schema, which won't raise error in ParseInt
fID, _ := strconv.ParseInt(blob.GetKey(), 10, 64)
k := JoinIDPath(meta.GetID(), partID, segID, fID, <-generator)
k := metautil.JoinIDPath(meta.GetID(), partID, segID, fID, <-generator)
key := path.Join(b.ChunkManager.RootPath(), common.SegmentInsertLogPath, k)
value := blob.GetValue()

View File

@ -34,6 +34,8 @@ import (
"syscall"
"time"
"github.com/milvus-io/milvus/internal/util/metautil"
"github.com/golang/protobuf/proto"
v3rpc "go.etcd.io/etcd/api/v3/v3rpc/rpctypes"
clientv3 "go.etcd.io/etcd/client/v3"
@ -1193,7 +1195,7 @@ func importFlushReqFunc(node *DataNode, req *datapb.ImportTaskRequest, res *root
logidx := start + int64(idx)
// no error raise if alloc=false
k := JoinIDPath(req.GetImportTask().GetCollectionId(), req.GetImportTask().GetPartitionId(), segmentID, fieldID, logidx)
k := metautil.JoinIDPath(req.GetImportTask().GetCollectionId(), req.GetImportTask().GetPartitionId(), segmentID, fieldID, logidx)
key := path.Join(node.chunkManager.RootPath(), common.SegmentInsertLogPath, k)
kvs[key] = blob.Value[:]
@ -1219,7 +1221,7 @@ func importFlushReqFunc(node *DataNode, req *datapb.ImportTaskRequest, res *root
logidx := field2Logidx[fieldID]
// no error raise if alloc=false
k := JoinIDPath(req.GetImportTask().GetCollectionId(), req.GetImportTask().GetPartitionId(), segmentID, fieldID, logidx)
k := metautil.JoinIDPath(req.GetImportTask().GetCollectionId(), req.GetImportTask().GetPartitionId(), segmentID, fieldID, logidx)
key := path.Join(node.chunkManager.RootPath(), common.SegmentStatslogPath, k)
kvs[key] = blob.Value

View File

@ -35,6 +35,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/etcdpb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/util/metautil"
"github.com/milvus-io/milvus/internal/util/retry"
"github.com/milvus-io/milvus/internal/util/timerecord"
)
@ -385,7 +386,7 @@ func (m *rendezvousFlushManager) flushBufferData(data *BufferData, segStats []by
logidx := start + int64(idx)
// no error raise if alloc=false
k := JoinIDPath(collID, partID, segmentID, fieldID, logidx)
k := metautil.JoinIDPath(collID, partID, segmentID, fieldID, logidx)
// [rootPath]/[insert_log]/key
key := path.Join(m.ChunkManager.RootPath(), common.SegmentInsertLogPath, k)
@ -409,7 +410,7 @@ func (m *rendezvousFlushManager) flushBufferData(data *BufferData, segStats []by
}
logidx := start + int64(len(binLogs))
k := JoinIDPath(collID, partID, segmentID, pkID, logidx)
k := metautil.JoinIDPath(collID, partID, segmentID, pkID, logidx)
key := path.Join(m.ChunkManager.RootPath(), common.SegmentStatslogPath, k)
kvs[key] = segStats
field2Stats[pkID] = &datapb.Binlog{
@ -458,7 +459,7 @@ func (m *rendezvousFlushManager) flushDelData(data *DelDataBuf, segmentID Unique
return err
}
blobKey := JoinIDPath(collID, partID, segmentID, logID)
blobKey := metautil.JoinIDPath(collID, partID, segmentID, logID)
blobPath := path.Join(m.ChunkManager.RootPath(), common.SegmentDeltaLogPath, blobKey)
kvs := map[string][]byte{blobPath: blob.Value[:]}
data.LogSize = int64(len(blob.Value))

View File

@ -27,6 +27,8 @@ import (
"sync"
"time"
"github.com/milvus-io/milvus/internal/util/metautil"
"go.uber.org/zap"
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
@ -864,7 +866,7 @@ func (alloc *AllocatorFactory) genKey(ids ...UniqueID) (string, error) {
return "", err
}
ids = append(ids, idx)
return JoinIDPath(ids...), nil
return metautil.JoinIDPath(ids...), nil
}
// If id == 0, AllocID will return not successful status

View File

@ -25,7 +25,4 @@ const (
ChannelRemovePrefix = MetaPrefix + "/channel-removal"
RemoveFlagTomestone = "removed"
maxOperationsPerTxn = 64
maxBytesPerTxn = 1024 * 1024
)

View File

@ -20,6 +20,13 @@ import (
"context"
"fmt"
"strconv"
"strings"
"github.com/milvus-io/milvus/internal/util/metautil"
"github.com/milvus-io/milvus/internal/util/etcd"
"golang.org/x/exp/maps"
"github.com/golang/protobuf/proto"
"github.com/milvus-io/milvus/api/commonpb"
@ -33,7 +40,8 @@ import (
)
type Catalog struct {
Txn kv.TxnKV
Txn kv.TxnKV
ChunkManagerRootPath string
}
func (kc *Catalog) ListSegments(ctx context.Context) ([]*datapb.SegmentInfo, error) {
@ -54,53 +62,41 @@ func (kc *Catalog) ListSegments(ctx context.Context) ([]*datapb.SegmentInfo, err
segments = append(segments, segmentInfo)
}
// get binlogs from segment meta first for compatible
var binlogs, deltalogs, statslogs []*datapb.FieldBinlog
for _, segmentInfo := range segments {
if len(segmentInfo.Binlogs) == 0 {
binlogs, err = kc.unmarshalBinlog(storage.InsertBinlog, segmentInfo.CollectionID, segmentInfo.PartitionID, segmentInfo.ID)
binlogs, err := kc.unmarshalBinlog(storage.InsertBinlog, segmentInfo.CollectionID, segmentInfo.PartitionID, segmentInfo.ID)
if err != nil {
return nil, err
}
segmentInfo.Binlogs = binlogs
}
if len(segmentInfo.Deltalogs) == 0 {
deltalogs, err = kc.unmarshalBinlog(storage.DeleteBinlog, segmentInfo.CollectionID, segmentInfo.PartitionID, segmentInfo.CollectionID)
deltalogs, err := kc.unmarshalBinlog(storage.DeleteBinlog, segmentInfo.CollectionID, segmentInfo.PartitionID, segmentInfo.ID)
if err != nil {
return nil, err
}
segmentInfo.Deltalogs = deltalogs
}
if len(segmentInfo.Statslogs) == 0 {
statslogs, err = kc.unmarshalBinlog(storage.StatsBinlog, segmentInfo.CollectionID, segmentInfo.PartitionID, segmentInfo.PartitionID)
statslogs, err := kc.unmarshalBinlog(storage.StatsBinlog, segmentInfo.CollectionID, segmentInfo.PartitionID, segmentInfo.ID)
if err != nil {
return nil, err
}
segmentInfo.Statslogs = statslogs
}
segmentInfo.Binlogs = binlogs
segmentInfo.Deltalogs = deltalogs
segmentInfo.Statslogs = statslogs
}
return segments, nil
}
func (kc *Catalog) AddSegment(ctx context.Context, segment *datapb.SegmentInfo) error {
// save binlogs separately
kvs, err := buildBinlogKvPair(segment)
kvs, err := buildSegmentAndBinlogsKvs(segment)
if err != nil {
return err
}
// save segment info
k, v, err := buildSegmentKeyValuePair(segment)
if err != nil {
return err
}
kvs[k] = v
// save handoff req if segment is flushed
if segment.State == commonpb.SegmentState_Flushed {
flushSegKey := buildFlushedSegmentPath(segment.GetCollectionID(), segment.GetPartitionID(), segment.GetID())
@ -111,100 +107,119 @@ func (kc *Catalog) AddSegment(ctx context.Context, segment *datapb.SegmentInfo)
}
func (kc *Catalog) AlterSegments(ctx context.Context, modSegments []*datapb.SegmentInfo) error {
kv := make(map[string]string)
if len(modSegments) == 0 {
return nil
}
kvs := make(map[string]string)
for _, segment := range modSegments {
// save binlogs separately
binlogKvs, err := buildBinlogKvPair(segment)
segmentKvs, err := buildSegmentAndBinlogsKvs(segment)
if err != nil {
return err
}
kv = typeutil.MergeMap(binlogKvs, kv)
// save segment info
k, v, err := buildSegmentKeyValuePair(segment)
if err != nil {
return err
}
kv[k] = v
maps.Copy(kvs, segmentKvs)
// save handoff req if segment is flushed
if segment.State == commonpb.SegmentState_Flushed {
flushSegKey := buildFlushedSegmentPath(segment.GetCollectionID(), segment.GetPartitionID(), segment.GetID())
kv[flushSegKey] = strconv.FormatInt(segment.GetID(), 10)
kvs[flushSegKey] = strconv.FormatInt(segment.GetID(), 10)
}
}
return kc.Txn.MultiSave(kv)
return kc.Txn.MultiSave(kvs)
}
func (kc *Catalog) hasBinlogPrefix(segment *datapb.SegmentInfo) (bool, error) {
binlogsKey, _, err := kc.getBinlogsWithPrefix(storage.InsertBinlog, segment.CollectionID, segment.PartitionID, segment.ID)
if err != nil {
return false, err
}
deltalogsKey, _, err := kc.getBinlogsWithPrefix(storage.DeleteBinlog, segment.CollectionID, segment.PartitionID, segment.ID)
if err != nil {
return false, err
}
statslogsKey, _, err := kc.getBinlogsWithPrefix(storage.StatsBinlog, segment.CollectionID, segment.PartitionID, segment.ID)
if err != nil {
return false, err
}
if len(binlogsKey) == 0 && len(deltalogsKey) == 0 && len(statslogsKey) == 0 {
return false, nil
}
return true, nil
}
func (kc *Catalog) AlterSegmentsAndAddNewSegment(ctx context.Context, segments []*datapb.SegmentInfo, newSegment *datapb.SegmentInfo) error {
data := make(map[string]string)
kvs := make(map[string]string)
for _, s := range segments {
k, v, err := buildSegmentKeyValuePair(s)
noBinlogsSegment, binlogs, deltalogs, statslogs := CloneSegmentWithExcludeBinlogs(s)
// for compacted segments
if noBinlogsSegment.State == commonpb.SegmentState_Dropped {
hasBinlogkeys, err := kc.hasBinlogPrefix(s)
if err != nil {
return err
}
// In order to guarantee back compatibility, the old format segments need
// convert to new format that include segment key and three binlog keys,
// or GC can not find data path on the storage.
if !hasBinlogkeys {
binlogsKvs, err := buildBinlogKvsWithLogID(noBinlogsSegment.CollectionID, noBinlogsSegment.PartitionID, noBinlogsSegment.ID, binlogs, deltalogs, statslogs)
if err != nil {
return err
}
maps.Copy(kvs, binlogsKvs)
}
}
k, v, err := buildSegmentKv(noBinlogsSegment)
if err != nil {
return err
}
data[k] = v
kvs[k] = v
}
if newSegment.NumOfRows > 0 {
// save binlogs separately
binlogKvs, err := buildBinlogKvPair(newSegment)
if newSegment != nil {
segmentKvs, err := buildSegmentAndBinlogsKvs(newSegment)
if err != nil {
return err
}
data = typeutil.MergeMap(binlogKvs, data)
// save segment info
k, v, err := buildSegmentKeyValuePair(newSegment)
if err != nil {
return err
}
data[k] = v
maps.Copy(kvs, segmentKvs)
}
err := kc.Txn.MultiSave(data)
if err != nil {
log.Error("batch save segments failed", zap.Error(err))
return err
}
return nil
return kc.Txn.MultiSave(kvs)
}
// RevertAlterSegmentsAndAddNewSegment reverts the metastore operation of AtlerSegmentsAndAddNewSegment
func (kc *Catalog) RevertAlterSegmentsAndAddNewSegment(ctx context.Context, oldSegments []*datapb.SegmentInfo, removeSegment *datapb.SegmentInfo) error {
var (
data = make(map[string]string)
kvs = make(map[string]string)
removals []string
)
for _, s := range oldSegments {
k, v, err := buildSegmentKeyValuePair(s)
segmentKvs, err := buildSegmentAndBinlogsKvs(s)
if err != nil {
return err
}
data[k] = v
maps.Copy(kvs, segmentKvs)
}
if removeSegment.NumOfRows > 0 {
// get all binlog keys
binlogKvs, err := buildBinlogKvPair(removeSegment)
if err != nil {
return err
}
binlogKeys := typeutil.GetMapKeys(binlogKvs)
if removeSegment != nil {
segKey := buildSegmentPath(removeSegment.GetCollectionID(), removeSegment.GetPartitionID(), removeSegment.GetID())
removals = append(removals, segKey)
binlogKeys := buildBinlogKeys(removeSegment)
removals = append(removals, binlogKeys...)
// get segment key
k, _, err := buildSegmentKeyValuePair(removeSegment)
if err != nil {
return err
}
removals = append(removals, k)
}
err := kc.Txn.MultiSaveAndRemove(data, removals)
err := kc.Txn.MultiSaveAndRemove(kvs, removals)
if err != nil {
log.Warn("batch save and remove segments failed", zap.Error(err))
return err
@ -214,65 +229,36 @@ func (kc *Catalog) RevertAlterSegmentsAndAddNewSegment(ctx context.Context, oldS
}
func (kc *Catalog) SaveDroppedSegmentsInBatch(ctx context.Context, segments []*datapb.SegmentInfo) error {
kvs := make(map[string]string)
batchIDs := make([]int64, 0, maxOperationsPerTxn)
multiSave := func() error {
if len(kvs) == 0 {
return nil
}
if err := kc.Txn.MultiSave(kvs); err != nil {
log.Error("Failed to save segments in batch for DropChannel",
zap.Any("segmentIDs", batchIDs),
zap.Error(err))
return err
}
if len(segments) == 0 {
return nil
}
// the limitation of etcd operations number per transaction is 128,
// since segment number might be enormous, so we shall split all save operations into batches
splitCount := 0
kvs := make(map[string]string)
for _, s := range segments {
key := buildSegmentPath(s.GetCollectionID(), s.GetPartitionID(), s.GetID())
segBytes, err := proto.Marshal(s)
noBinlogsSegment, _, _, _ := CloneSegmentWithExcludeBinlogs(s)
segBytes, err := proto.Marshal(noBinlogsSegment)
if err != nil {
return fmt.Errorf("failed to marshal segment: %d, err: %w", s.GetID(), err)
}
kvSize := len(key) + len(segBytes)
splitCount += kvSize
if len(kvs) == maxOperationsPerTxn || (len(kvs) > 0 && splitCount >= maxBytesPerTxn) {
if err := multiSave(); err != nil {
return err
}
kvs = make(map[string]string)
batchIDs = make([]int64, 0, maxOperationsPerTxn)
if splitCount >= maxBytesPerTxn {
splitCount = kvSize
}
}
kvs[key] = string(segBytes)
batchIDs = append(batchIDs, s.ID)
}
return multiSave()
saveFn := func(partialKvs map[string]string) error {
return kc.Txn.MultiSave(partialKvs)
}
if err := etcd.SaveByBatch(kvs, saveFn); err != nil {
return err
}
return nil
}
func (kc *Catalog) DropSegment(ctx context.Context, segment *datapb.SegmentInfo) error {
segKey := buildSegmentPath(segment.GetCollectionID(), segment.GetPartitionID(), segment.GetID())
keys := []string{segKey}
binlogKvs, err := buildBinlogKvPair(segment)
if err != nil {
return err
}
binlogKeys := typeutil.GetMapKeys(binlogKvs)
binlogKeys := buildBinlogKeys(segment)
keys = append(keys, binlogKeys...)
return kc.Txn.MultiRemove(keys)
}
@ -302,8 +288,8 @@ func (kc *Catalog) DropChannel(ctx context.Context, channel string) error {
return kc.Txn.Remove(key)
}
func (kc *Catalog) unmarshalBinlog(binlogType storage.BinlogType, collectionID, partitionID, segmentID typeutil.UniqueID) ([]*datapb.FieldBinlog, error) {
// unmarshal binlog/deltalog/statslog
func (kc *Catalog) getBinlogsWithPrefix(binlogType storage.BinlogType, collectionID, partitionID,
segmentID typeutil.UniqueID) ([]string, []string, error) {
var binlogPrefix string
switch binlogType {
case storage.InsertBinlog:
@ -313,12 +299,23 @@ func (kc *Catalog) unmarshalBinlog(binlogType storage.BinlogType, collectionID,
case storage.StatsBinlog:
binlogPrefix = buildFieldStatslogPathPrefix(collectionID, partitionID, segmentID)
default:
return nil, fmt.Errorf("invalid binlog type: %d", binlogType)
return nil, nil, fmt.Errorf("invalid binlog type: %d", binlogType)
}
_, values, err := kc.Txn.LoadWithPrefix(binlogPrefix)
keys, values, err := kc.Txn.LoadWithPrefix(binlogPrefix)
if err != nil {
return nil, nil, err
}
return keys, values, nil
}
// unmarshal binlog/deltalog/statslog
func (kc *Catalog) unmarshalBinlog(binlogType storage.BinlogType, collectionID, partitionID, segmentID typeutil.UniqueID) ([]*datapb.FieldBinlog, error) {
_, values, err := kc.getBinlogsWithPrefix(binlogType, collectionID, partitionID, segmentID)
if err != nil {
return nil, err
}
result := make([]*datapb.FieldBinlog, len(values))
for i, value := range values {
fieldBinlog := &datapb.FieldBinlog{}
@ -326,57 +323,176 @@ func (kc *Catalog) unmarshalBinlog(binlogType storage.BinlogType, collectionID,
if err != nil {
return nil, fmt.Errorf("failed to unmarshal datapb.FieldBinlog: %d, err:%w", fieldBinlog.FieldID, err)
}
fillLogPathByLogID(kc.ChunkManagerRootPath, binlogType, collectionID, partitionID, segmentID, fieldBinlog)
result[i] = fieldBinlog
}
return result, nil
}
func buildBinlogKvPair(segment *datapb.SegmentInfo) (map[string]string, error) {
func fillLogPathByLogID(chunkManagerRootPath string, binlogType storage.BinlogType, collectionID, partitionID,
segmentID typeutil.UniqueID, fieldBinlog *datapb.FieldBinlog) error {
for _, binlog := range fieldBinlog.Binlogs {
path, err := buildLogPath(chunkManagerRootPath, binlogType, collectionID, partitionID,
segmentID, fieldBinlog.GetFieldID(), binlog.GetLogID())
if err != nil {
return err
}
binlog.LogPath = path
}
return nil
}
func fillLogIDByLogPath(multiFieldBinlogs ...[]*datapb.FieldBinlog) error {
for _, fieldBinlogs := range multiFieldBinlogs {
for _, fieldBinlog := range fieldBinlogs {
for _, binlog := range fieldBinlog.Binlogs {
logPath := binlog.LogPath
idx := strings.LastIndex(logPath, "/")
if idx == -1 {
return fmt.Errorf("invailed binlog path: %s", logPath)
}
logPathStr := logPath[(idx + 1):]
logID, err := strconv.ParseInt(logPathStr, 10, 64)
if err != nil {
return err
}
// set log path to empty and only store log id
binlog.LogPath = ""
binlog.LogID = logID
}
}
}
return nil
}
// build a binlog path on the storage by metadata
func buildLogPath(chunkManagerRootPath string, binlogType storage.BinlogType, collectionID, partitionID, segmentID, filedID, logID typeutil.UniqueID) (string, error) {
switch binlogType {
case storage.InsertBinlog:
path := metautil.BuildInsertLogPath(chunkManagerRootPath, collectionID, partitionID, segmentID, filedID, logID)
return path, nil
case storage.DeleteBinlog:
path := metautil.BuildDeltaLogPath(chunkManagerRootPath, collectionID, partitionID, segmentID, logID)
return path, nil
case storage.StatsBinlog:
path := metautil.BuildStatsLogPath(chunkManagerRootPath, collectionID, partitionID, segmentID, filedID, logID)
return path, nil
default:
return "", fmt.Errorf("invalid binlog type: %d", binlogType)
}
}
func buildBinlogKvsWithLogID(collectionID, partitionID, segmentID typeutil.UniqueID,
binlogs, deltalogs, statslogs []*datapb.FieldBinlog) (map[string]string, error) {
fillLogIDByLogPath(binlogs, deltalogs, statslogs)
kvs, err := buildBinlogKvs(collectionID, partitionID, segmentID, binlogs, deltalogs, statslogs)
if err != nil {
return nil, err
}
return kvs, nil
}
func buildSegmentAndBinlogsKvs(segment *datapb.SegmentInfo) (map[string]string, error) {
noBinlogsSegment, binlogs, deltalogs, statslogs := CloneSegmentWithExcludeBinlogs(segment)
// save binlogs separately
kvs, err := buildBinlogKvsWithLogID(noBinlogsSegment.CollectionID, noBinlogsSegment.PartitionID, noBinlogsSegment.ID, binlogs, deltalogs, statslogs)
if err != nil {
return nil, err
}
// save segment info
k, v, err := buildSegmentKv(noBinlogsSegment)
if err != nil {
return nil, err
}
kvs[k] = v
return kvs, nil
}
func buildBinlogKeys(segment *datapb.SegmentInfo) []string {
var keys []string
// binlog
for _, binlog := range segment.Binlogs {
key := buildFieldBinlogPath(segment.CollectionID, segment.PartitionID, segment.ID, binlog.FieldID)
keys = append(keys, key)
}
// deltalog
for _, deltalog := range segment.Deltalogs {
key := buildFieldDeltalogPath(segment.CollectionID, segment.PartitionID, segment.ID, deltalog.FieldID)
keys = append(keys, key)
}
// statslog
for _, statslog := range segment.Statslogs {
key := buildFieldStatslogPath(segment.CollectionID, segment.PartitionID, segment.ID, statslog.FieldID)
keys = append(keys, key)
}
return keys
}
func buildBinlogKvs(collectionID, partitionID, segmentID typeutil.UniqueID, binlogs, deltalogs, statslogs []*datapb.FieldBinlog) (map[string]string, error) {
kv := make(map[string]string)
// binlog kv
for _, binlog := range segment.Binlogs {
for _, binlog := range binlogs {
binlogBytes, err := proto.Marshal(binlog)
if err != nil {
return nil, fmt.Errorf("marshal binlogs failed, collectionID:%d, segmentID:%d, fieldID:%d, error:%w", segment.CollectionID, segment.GetID(), binlog.FieldID, err)
return nil, fmt.Errorf("marshal binlogs failed, collectionID:%d, segmentID:%d, fieldID:%d, error:%w", collectionID, segmentID, binlog.FieldID, err)
}
key := buildFieldBinlogPath(segment.CollectionID, segment.PartitionID, segment.ID, binlog.FieldID)
key := buildFieldBinlogPath(collectionID, partitionID, segmentID, binlog.FieldID)
kv[key] = string(binlogBytes)
}
// deltalog etcd kv
for _, deltalog := range segment.Deltalogs {
// deltalog
for _, deltalog := range deltalogs {
binlogBytes, err := proto.Marshal(deltalog)
if err != nil {
return nil, fmt.Errorf("marshal deltalogs failed, collectionID:%d, segmentID:%d, fieldID:%d, error:%w", segment.CollectionID, segment.GetID(), deltalog.FieldID, err)
return nil, fmt.Errorf("marshal deltalogs failed, collectionID:%d, segmentID:%d, fieldID:%d, error:%w", collectionID, segmentID, deltalog.FieldID, err)
}
key := buildFieldDeltalogPath(segment.CollectionID, segment.PartitionID, segment.ID, deltalog.FieldID)
key := buildFieldDeltalogPath(collectionID, partitionID, segmentID, deltalog.FieldID)
kv[key] = string(binlogBytes)
}
// statslog etcd kv
for _, statslog := range segment.Statslogs {
// statslog
for _, statslog := range statslogs {
binlogBytes, err := proto.Marshal(statslog)
if err != nil {
return nil, fmt.Errorf("marshal statslogs failed, collectionID:%d, segmentID:%d, fieldID:%d, error:%w", segment.CollectionID, segment.GetID(), statslog.FieldID, err)
return nil, fmt.Errorf("marshal statslogs failed, collectionID:%d, segmentID:%d, fieldID:%d, error:%w", collectionID, segmentID, statslog.FieldID, err)
}
key := buildFieldStatslogPath(segment.CollectionID, segment.PartitionID, segment.ID, statslog.FieldID)
key := buildFieldStatslogPath(collectionID, partitionID, segmentID, statslog.FieldID)
kv[key] = string(binlogBytes)
}
return kv, nil
}
func buildSegmentKeyValuePair(segment *datapb.SegmentInfo) (string, string, error) {
func CloneSegmentWithExcludeBinlogs(segment *datapb.SegmentInfo) (*datapb.SegmentInfo, []*datapb.FieldBinlog, []*datapb.FieldBinlog, []*datapb.FieldBinlog) {
clonedSegment := proto.Clone(segment).(*datapb.SegmentInfo)
binlogs := clonedSegment.Binlogs
deltalogs := clonedSegment.Deltalogs
statlogs := clonedSegment.Statslogs
clonedSegment.Binlogs = nil
clonedSegment.Deltalogs = nil
clonedSegment.Statslogs = nil
segBytes, err := proto.Marshal(clonedSegment)
return clonedSegment, binlogs, deltalogs, statlogs
}
func buildSegmentKv(segment *datapb.SegmentInfo) (string, string, error) {
segBytes, err := proto.Marshal(segment)
if err != nil {
return "", "", fmt.Errorf("failed to marshal segment: %d, err: %w", segment.ID, err)
}
key := buildSegmentPath(clonedSegment.GetCollectionID(), clonedSegment.GetPartitionID(), clonedSegment.GetID())
key := buildSegmentPath(segment.GetCollectionID(), segment.GetPartitionID(), segment.GetID())
return key, string(segBytes), nil
}

View File

@ -3,121 +3,414 @@ package datacoord
import (
"context"
"errors"
"math/rand"
"strings"
"testing"
"github.com/golang/protobuf/proto"
"github.com/milvus-io/milvus/api/commonpb"
"github.com/milvus-io/milvus/internal/kv"
"github.com/milvus-io/milvus/internal/kv/mocks"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/util/metautil"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"golang.org/x/exp/maps"
)
type MockedTxnKV struct {
kv.TxnKV
multiSave func(kvs map[string]string) error
save func(key, value string) error
loadWithPrefix func(key string) ([]string, []string, error)
multiRemove func(keys []string) error
}
var (
segments = []*datapb.SegmentInfo{
{
ID: 1,
CollectionID: 1000,
logID = int64(99)
collectionID = int64(2)
partitionID = int64(1)
segmentID = int64(11)
segmentID2 = int64(1)
fieldID = int64(1)
binlogPath = metautil.BuildInsertLogPath("a", collectionID, partitionID, segmentID, fieldID, logID)
deltalogPath = metautil.BuildDeltaLogPath("a", collectionID, partitionID, segmentID, logID)
statslogPath = metautil.BuildStatsLogPath("a", collectionID, partitionID, segmentID, fieldID, logID)
k1 = buildFieldBinlogPath(collectionID, partitionID, segmentID, fieldID)
k2 = buildFieldDeltalogPath(collectionID, partitionID, segmentID, fieldID)
k3 = buildFieldStatslogPath(collectionID, partitionID, segmentID, fieldID)
k4 = buildSegmentPath(collectionID, partitionID, segmentID2)
k5 = buildSegmentPath(collectionID, partitionID, segmentID)
k6 = buildFlushedSegmentPath(collectionID, partitionID, segmentID)
k7 = buildFieldBinlogPath(collectionID, partitionID, segmentID2, fieldID)
k8 = buildFieldDeltalogPath(collectionID, partitionID, segmentID2, fieldID)
k9 = buildFieldStatslogPath(collectionID, partitionID, segmentID2, fieldID)
keys = map[string]struct{}{
k1: {},
k2: {},
k3: {},
k4: {},
k5: {},
k6: {},
k7: {},
k8: {},
k9: {},
}
invalidSegment = &datapb.SegmentInfo{
ID: segmentID,
CollectionID: collectionID,
PartitionID: partitionID,
NumOfRows: 100,
State: commonpb.SegmentState_Flushed,
Binlogs: []*datapb.FieldBinlog{
{
FieldID: 1,
Binlogs: []*datapb.Binlog{
{
EntriesNum: 5,
LogPath: "badpath",
},
},
},
},
}
newSegment = &datapb.SegmentInfo{
ID: 2,
CollectionID: 1000,
binlogs = []*datapb.FieldBinlog{
{
FieldID: 1,
Binlogs: []*datapb.Binlog{
{
EntriesNum: 5,
LogPath: binlogPath,
},
},
},
}
deltalogs = []*datapb.FieldBinlog{
{
FieldID: 1,
Binlogs: []*datapb.Binlog{
{
EntriesNum: 5,
LogPath: deltalogPath,
}},
},
}
statslogs = []*datapb.FieldBinlog{
{
FieldID: 1,
Binlogs: []*datapb.Binlog{
{
EntriesNum: 5,
LogPath: statslogPath,
},
},
},
}
segment1 = &datapb.SegmentInfo{
ID: segmentID,
CollectionID: collectionID,
PartitionID: partitionID,
NumOfRows: 100,
State: commonpb.SegmentState_Flushed,
Binlogs: binlogs,
Deltalogs: deltalogs,
Statslogs: statslogs,
}
droppedSegment = &datapb.SegmentInfo{
ID: segmentID2,
CollectionID: collectionID,
PartitionID: partitionID,
NumOfRows: 100,
State: commonpb.SegmentState_Dropped,
Binlogs: binlogs,
Deltalogs: deltalogs,
Statslogs: statslogs,
}
)
func TestCatalog_AlterSegmentsAndAddNewSegment(t *testing.T) {
t.Run("save error", func(t *testing.T) {
txn := &mocks.TxnKV{}
txn.EXPECT().MultiSave(mock.Anything).Return(errors.New("mock error"))
func (mc *MockedTxnKV) MultiSave(kvs map[string]string) error {
return mc.multiSave(kvs)
}
catalog := &Catalog{txn}
err := catalog.AlterSegmentsAndAddNewSegment(context.TODO(), segments, newSegment)
func (mc *MockedTxnKV) Save(key, value string) error {
return mc.save(key, value)
}
func (mc *MockedTxnKV) LoadWithPrefix(key string) ([]string, []string, error) {
return mc.loadWithPrefix(key)
}
func (mc *MockedTxnKV) MultiRemove(keys []string) error {
return mc.multiRemove(keys)
}
func Test_ListSegments(t *testing.T) {
t.Run("load failed", func(t *testing.T) {
txn := &MockedTxnKV{}
txn.loadWithPrefix = func(key string) ([]string, []string, error) {
return nil, nil, errors.New("error")
}
catalog := &Catalog{txn, "a"}
ret, err := catalog.ListSegments(context.TODO())
assert.Nil(t, ret)
assert.Error(t, err)
})
t.Run("numRow>0", func(t *testing.T) {
txn := &mocks.TxnKV{}
txn.EXPECT().MultiSave(mock.Anything).Return(nil)
verifySegments := func(t *testing.T, logID int64, ret []*datapb.SegmentInfo) {
assert.Equal(t, 1, len(ret))
segment := ret[0]
assert.Equal(t, segmentID, segment.ID)
assert.Equal(t, collectionID, segment.CollectionID)
assert.Equal(t, partitionID, segment.PartitionID)
toAlter := []*datapb.SegmentInfo{
{
CollectionID: 100,
PartitionID: 10,
ID: 1,
},
}
assert.Equal(t, 1, len(segment.Binlogs))
assert.Equal(t, fieldID, segment.Binlogs[0].FieldID)
assert.Equal(t, 1, len(segment.Binlogs[0].Binlogs))
assert.Equal(t, logID, segment.Binlogs[0].Binlogs[0].LogID)
assert.Equal(t, binlogPath, segment.Binlogs[0].Binlogs[0].LogPath)
newSeg := &datapb.SegmentInfo{
Binlogs: []*datapb.FieldBinlog{
{
FieldID: 101,
Binlogs: []*datapb.Binlog{},
},
},
Deltalogs: []*datapb.FieldBinlog{
{
FieldID: 101,
Binlogs: []*datapb.Binlog{},
},
},
CollectionID: 100,
PartitionID: 10,
ID: 2,
NumOfRows: 15,
}
assert.Equal(t, 1, len(segment.Deltalogs))
assert.Equal(t, fieldID, segment.Deltalogs[0].FieldID)
assert.Equal(t, 1, len(segment.Deltalogs[0].Binlogs))
assert.Equal(t, logID, segment.Deltalogs[0].Binlogs[0].LogID)
assert.Equal(t, deltalogPath, segment.Deltalogs[0].Binlogs[0].LogPath)
catalog := &Catalog{txn}
err := catalog.AlterSegmentsAndAddNewSegment(context.Background(), toAlter, newSeg)
assert.Equal(t, 1, len(segment.Statslogs))
assert.Equal(t, fieldID, segment.Statslogs[0].FieldID)
assert.Equal(t, 1, len(segment.Statslogs[0].Binlogs))
assert.Equal(t, logID, segment.Statslogs[0].Binlogs[0].LogID)
assert.Equal(t, statslogPath, segment.Statslogs[0].Binlogs[0].LogPath)
}
t.Run("test compatibility", func(t *testing.T) {
txn := &MockedTxnKV{}
segBytes, err := proto.Marshal(segment1)
assert.NoError(t, err)
txn.loadWithPrefix = func(key string) ([]string, []string, error) {
return []string{k5}, []string{string(segBytes)}, nil
}
catalog := &Catalog{txn, "a"}
ret, err := catalog.ListSegments(context.TODO())
assert.NotNil(t, ret)
assert.NoError(t, err)
verifySegments(t, int64(0), ret)
})
t.Run("list successfully", func(t *testing.T) {
txn := &MockedTxnKV{}
var savedKvs map[string]string
txn.multiSave = func(kvs map[string]string) error {
savedKvs = kvs
return nil
}
catalog := &Catalog{txn, "a"}
err := catalog.AddSegment(context.TODO(), segment1)
assert.Nil(t, err)
txn.loadWithPrefix = func(key string) ([]string, []string, error) {
if strings.HasPrefix(k5, key) {
return []string{k5}, []string{savedKvs[k5]}, nil
}
if strings.HasPrefix(k1, key) {
return []string{k1}, []string{savedKvs[k1]}, nil
}
if strings.HasPrefix(k2, key) {
return []string{k2}, []string{savedKvs[k2]}, nil
}
if strings.HasPrefix(k3, key) {
return []string{k3}, []string{savedKvs[k3]}, nil
}
return nil, nil, errors.New("should not reach here")
}
ret, err := catalog.ListSegments(context.TODO())
assert.NotNil(t, ret)
assert.Nil(t, err)
verifySegments(t, logID, ret)
})
}
func TestCatalog_RevertAlterSegmentsAndAddNewSegment(t *testing.T) {
t.Run("save error", func(t *testing.T) {
txn := &mocks.TxnKV{}
txn.EXPECT().MultiSaveAndRemove(mock.Anything, mock.Anything).Return(errors.New("mock error"))
func Test_AddSegments(t *testing.T) {
t.Run("generate binlog kvs failed", func(t *testing.T) {
txn := &MockedTxnKV{}
txn.multiSave = func(kvs map[string]string) error {
return errors.New("error")
}
catalog := &Catalog{txn}
err := catalog.RevertAlterSegmentsAndAddNewSegment(context.TODO(), segments, newSegment)
catalog := &Catalog{txn, "a"}
err := catalog.AddSegment(context.TODO(), invalidSegment)
assert.Error(t, err)
})
t.Run("numRow>0", func(t *testing.T) {
txn := &mocks.TxnKV{}
txn.EXPECT().MultiSaveAndRemove(mock.Anything, mock.Anything).Return(nil)
toAlter := []*datapb.SegmentInfo{
{
CollectionID: 100,
PartitionID: 10,
ID: 1,
},
t.Run("save error", func(t *testing.T) {
txn := &MockedTxnKV{}
txn.multiSave = func(kvs map[string]string) error {
return errors.New("error")
}
newSeg := &datapb.SegmentInfo{
Binlogs: []*datapb.FieldBinlog{
{
FieldID: 101,
Binlogs: []*datapb.Binlog{},
},
},
Deltalogs: []*datapb.FieldBinlog{
{
FieldID: 101,
Binlogs: []*datapb.Binlog{},
},
},
CollectionID: 100,
PartitionID: 10,
ID: 2,
NumOfRows: 15,
catalog := &Catalog{txn, "a"}
err := catalog.AddSegment(context.TODO(), segment1)
assert.Error(t, err)
})
t.Run("save successfully", func(t *testing.T) {
txn := &MockedTxnKV{}
var savedKvs map[string]string
txn.multiSave = func(kvs map[string]string) error {
savedKvs = kvs
return nil
}
catalog := &Catalog{txn}
err := catalog.RevertAlterSegmentsAndAddNewSegment(context.Background(), toAlter, newSeg)
catalog := &Catalog{txn, "a"}
err := catalog.AddSegment(context.TODO(), segment1)
assert.Nil(t, err)
_, ok := savedKvs[k4]
assert.False(t, ok)
assert.Equal(t, 5, len(savedKvs))
verifySavedKvsForSegment(t, savedKvs)
})
}
func Test_AlterSegments(t *testing.T) {
t.Run("generate binlog kvs failed", func(t *testing.T) {
txn := &MockedTxnKV{}
txn.multiSave = func(kvs map[string]string) error {
return errors.New("error")
}
catalog := &Catalog{txn, "a"}
err := catalog.AlterSegments(context.TODO(), []*datapb.SegmentInfo{invalidSegment})
assert.Error(t, err)
})
t.Run("save error", func(t *testing.T) {
txn := &MockedTxnKV{}
txn.multiSave = func(kvs map[string]string) error {
return errors.New("error")
}
catalog := &Catalog{txn, "a"}
err := catalog.AlterSegments(context.TODO(), []*datapb.SegmentInfo{segment1})
assert.Error(t, err)
})
t.Run("save successfully", func(t *testing.T) {
txn := &MockedTxnKV{}
var savedKvs map[string]string
txn.multiSave = func(kvs map[string]string) error {
savedKvs = kvs
return nil
}
catalog := &Catalog{txn, "a"}
err := catalog.AlterSegments(context.TODO(), []*datapb.SegmentInfo{})
assert.Nil(t, err)
err = catalog.AlterSegments(context.TODO(), []*datapb.SegmentInfo{segment1})
assert.Nil(t, err)
_, ok := savedKvs[k4]
assert.False(t, ok)
assert.Equal(t, 5, len(savedKvs))
verifySavedKvsForSegment(t, savedKvs)
})
}
func Test_AlterSegmentsAndAddNewSegment(t *testing.T) {
t.Run("save error", func(t *testing.T) {
txn := &MockedTxnKV{}
txn.multiSave = func(kvs map[string]string) error {
return errors.New("error")
}
catalog := &Catalog{txn, "a"}
err := catalog.AlterSegmentsAndAddNewSegment(context.TODO(), []*datapb.SegmentInfo{}, segment1)
assert.Error(t, err)
})
t.Run("get prefix fail", func(t *testing.T) {
txn := &MockedTxnKV{}
txn.loadWithPrefix = func(key string) ([]string, []string, error) {
return nil, nil, errors.New("error")
}
catalog := &Catalog{txn, "a"}
err := catalog.AlterSegmentsAndAddNewSegment(context.TODO(), []*datapb.SegmentInfo{droppedSegment}, nil)
assert.Error(t, err)
})
t.Run("save successfully", func(t *testing.T) {
txn := &MockedTxnKV{}
savedKvs := make(map[string]string, 0)
txn.multiSave = func(kvs map[string]string) error {
maps.Copy(savedKvs, kvs)
return nil
}
txn.loadWithPrefix = func(key string) ([]string, []string, error) {
return []string{}, []string{}, nil
}
catalog := &Catalog{txn, "a"}
err := catalog.AlterSegmentsAndAddNewSegment(context.TODO(), []*datapb.SegmentInfo{droppedSegment}, segment1)
assert.NoError(t, err)
assert.Equal(t, 8, len(savedKvs))
verifySavedKvsForDroppedSegment(t, savedKvs)
verifySavedKvsForSegment(t, savedKvs)
})
}
func Test_DropSegment(t *testing.T) {
t.Run("remove failed", func(t *testing.T) {
txn := &MockedTxnKV{}
txn.multiRemove = func(keys []string) error {
return errors.New("error")
}
catalog := &Catalog{txn, "a"}
err := catalog.DropSegment(context.TODO(), segment1)
assert.Error(t, err)
})
t.Run("remove successfully", func(t *testing.T) {
txn := &MockedTxnKV{}
removedKvs := make(map[string]struct{}, 0)
txn.multiRemove = func(keys []string) error {
for _, key := range keys {
removedKvs[key] = struct{}{}
}
return nil
}
catalog := &Catalog{txn, "a"}
err := catalog.DropSegment(context.TODO(), segment1)
assert.NoError(t, err)
assert.Equal(t, 4, len(removedKvs))
for _, k := range []string{k1, k2, k3, k5} {
_, ok := removedKvs[k]
assert.True(t, ok)
}
})
}
@ -125,7 +418,7 @@ func Test_SaveDroppedSegmentsInBatch_SaveError(t *testing.T) {
txn := &mocks.TxnKV{}
txn.EXPECT().MultiSave(mock.Anything).Return(errors.New("mock error"))
catalog := &Catalog{txn}
catalog := &Catalog{txn, ""}
segments := []*datapb.SegmentInfo{
{
ID: 1,
@ -150,7 +443,16 @@ func Test_SaveDroppedSegmentsInBatch_MultiSave(t *testing.T) {
}).
Return(nil)
catalog := &Catalog{txn}
catalog := &Catalog{txn, ""}
// no segments
{
var segments []*datapb.SegmentInfo
err := catalog.SaveDroppedSegmentsInBatch(context.TODO(), segments)
assert.Nil(t, err)
assert.Equal(t, 0, count)
assert.Equal(t, 0, kvSize)
}
// testing for no splitting
{
@ -186,46 +488,99 @@ func Test_SaveDroppedSegmentsInBatch_MultiSave(t *testing.T) {
assert.Equal(t, 2, count)
assert.Equal(t, 65, kvSize)
}
// testing for reaching max bytes size
{
segments3 := []*datapb.SegmentInfo{
{
ID: int64(1),
CollectionID: 1000,
PartitionID: 100,
InsertChannel: randomString(1024 * 1024 * 2),
},
{
ID: int64(2),
CollectionID: 1000,
PartitionID: 100,
InsertChannel: randomString(1024),
},
}
count = 0
kvSize = 0
err := catalog.SaveDroppedSegmentsInBatch(context.TODO(), segments3)
assert.Nil(t, err)
assert.Equal(t, 2, count)
assert.Equal(t, 2, kvSize)
}
}
func randomString(len int) string {
bytes := make([]byte, len)
for i := 0; i < len; i++ {
bytes[i] = byte(65 + rand.Intn(25))
}
return string(bytes)
func TestCatalog_RevertAlterSegmentsAndAddNewSegment(t *testing.T) {
t.Run("save error", func(t *testing.T) {
txn := &mocks.TxnKV{}
txn.EXPECT().MultiSaveAndRemove(mock.Anything, mock.Anything).Return(errors.New("mock error"))
catalog := &Catalog{txn, ""}
err := catalog.RevertAlterSegmentsAndAddNewSegment(context.TODO(), []*datapb.SegmentInfo{segment1}, droppedSegment)
assert.Error(t, err)
})
t.Run("revert successfully", func(t *testing.T) {
txn := &mocks.TxnKV{}
txn.EXPECT().MultiSaveAndRemove(mock.Anything, mock.Anything).Return(nil)
catalog := &Catalog{txn, ""}
err := catalog.RevertAlterSegmentsAndAddNewSegment(context.TODO(), []*datapb.SegmentInfo{segment1}, droppedSegment)
assert.NoError(t, err)
})
}
func Test_MarkChannelDeleted_SaveError(t *testing.T) {
txn := &mocks.TxnKV{}
txn.EXPECT().Save(mock.Anything, mock.Anything).Return(errors.New("mock error"))
txn.EXPECT().
Save(mock.Anything, mock.Anything).
Return(errors.New("mock error"))
catalog := &Catalog{txn}
catalog := &Catalog{txn, ""}
err := catalog.MarkChannelDeleted(context.TODO(), "test_channel_1")
assert.Error(t, err)
}
func verifyBinlogs(t *testing.T, binlogBytes []byte) {
binlogs := &datapb.FieldBinlog{}
err := proto.Unmarshal([]byte(binlogBytes), binlogs)
assert.NoError(t, err)
assert.Equal(t, 1, len(binlogs.Binlogs))
assert.Equal(t, int64(99), binlogs.Binlogs[0].GetLogID())
assert.Equal(t, "", binlogs.Binlogs[0].GetLogPath())
}
func verifySegmentInfo(t *testing.T, segmentInfoBytes []byte) {
segmentInfo := &datapb.SegmentInfo{}
err := proto.Unmarshal(segmentInfoBytes, segmentInfo)
assert.NoError(t, err)
assert.Nil(t, segmentInfo.Binlogs)
assert.Nil(t, segmentInfo.Binlogs)
assert.Nil(t, segmentInfo.Deltalogs)
assert.Equal(t, collectionID, segmentInfo.CollectionID)
assert.Equal(t, segmentID, segmentInfo.ID)
}
func verifySavedKvsForSegment(t *testing.T, savedKvs map[string]string) {
for k := range savedKvs {
_, ok := keys[k]
assert.True(t, ok)
}
for _, k := range []string{k1, k2, k3} {
ret, ok := savedKvs[k]
assert.True(t, ok)
verifyBinlogs(t, []byte(ret))
}
ret, ok := savedKvs[k5]
assert.True(t, ok)
verifySegmentInfo(t, []byte(ret))
}
func verifySegmentInfo2(t *testing.T, segmentInfoBytes []byte) {
segmentInfo := &datapb.SegmentInfo{}
err := proto.Unmarshal(segmentInfoBytes, segmentInfo)
assert.NoError(t, err)
assert.Nil(t, segmentInfo.Binlogs)
assert.Nil(t, segmentInfo.Binlogs)
assert.Nil(t, segmentInfo.Deltalogs)
assert.Equal(t, collectionID, segmentInfo.CollectionID)
assert.Equal(t, segmentID2, segmentInfo.ID)
}
func verifySavedKvsForDroppedSegment(t *testing.T, savedKvs map[string]string) {
for k := range savedKvs {
_, ok := keys[k]
assert.True(t, ok)
}
for _, k := range []string{k7, k8, k9} {
ret, ok := savedKvs[k]
assert.True(t, ok)
verifyBinlogs(t, []byte(ret))
}
ret, ok := savedKvs[k4]
assert.True(t, ok)
verifySegmentInfo2(t, []byte(ret))
}

View File

@ -6,7 +6,9 @@ import (
"fmt"
"github.com/milvus-io/milvus/internal/metastore"
"github.com/milvus-io/milvus/internal/util/crypto"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util"
@ -62,60 +64,18 @@ func buildAliasKey(aliasName string) string {
return fmt.Sprintf("%s/%s", AliasMetaPrefix, aliasName)
}
func buildKvs(keys, values []string) (map[string]string, error) {
if len(keys) != len(values) {
return nil, fmt.Errorf("length of keys (%d) and values (%d) are not equal", len(keys), len(values))
}
ret := make(map[string]string, len(keys))
for i, k := range keys {
_, ok := ret[k]
if ok {
return nil, fmt.Errorf("duplicated key was found: %s", k)
}
ret[k] = values[i]
}
return ret, nil
}
func min(a, b int) int {
if a < b {
return a
}
return b
}
func batchSave(snapshot kv.SnapShotKV, maxTxnNum int, kvs map[string]string, ts typeutil.Timestamp) error {
keys := make([]string, 0, len(kvs))
values := make([]string, 0, len(kvs))
for k, v := range kvs {
keys = append(keys, k)
values = append(values, v)
}
for i := 0; i < len(kvs); i = i + maxTxnNum {
end := min(i+maxTxnNum, len(keys))
batch, err := buildKvs(keys[i:end], values[i:end])
if err != nil {
return err
}
if err := snapshot.MultiSave(batch, ts); err != nil {
return err
}
}
return nil
}
func batchMultiSaveAndRemoveWithPrefix(snapshot kv.SnapShotKV, maxTxnNum int, saves map[string]string, removals []string, ts typeutil.Timestamp) error {
if err := batchSave(snapshot, maxTxnNum, saves, ts); err != nil {
saveFn := func(partialKvs map[string]string) error {
return snapshot.MultiSave(partialKvs, ts)
}
if err := etcd.SaveByBatch(saves, saveFn); err != nil {
return err
}
for i := 0; i < len(removals); i = i + maxTxnNum {
end := min(i+maxTxnNum, len(removals))
batch := removals[i:end]
if err := snapshot.MultiSaveAndRemoveWithPrefix(nil, batch, ts); err != nil {
return err
}
removeFn := func(partialKeys []string) error {
return snapshot.MultiSaveAndRemoveWithPrefix(nil, partialKeys, ts)
}
return nil
return etcd.RemoveByBatch(removals, removeFn)
}
func (kc *Catalog) CreateCollection(ctx context.Context, coll *model.Collection, ts typeutil.Timestamp) error {
@ -168,7 +128,9 @@ func (kc *Catalog) CreateCollection(ctx context.Context, coll *model.Collection,
// Though batchSave is not atomic enough, we can promise the atomicity outside.
// Recovering from failure, if we found collection is creating, we should removing all these related meta.
return batchSave(kc.Snapshot, maxTxnNum, kvs, ts)
return etcd.SaveByBatch(kvs, func(partialKvs map[string]string) error {
return kc.Snapshot.MultiSave(partialKvs, ts)
})
}
func (kc *Catalog) loadCollection(ctx context.Context, collectionID typeutil.UniqueID, ts typeutil.Timestamp) (*pb.CollectionInfo, error) {

View File

@ -703,94 +703,6 @@ func TestCatalog_ListAliasesV2(t *testing.T) {
})
}
func Test_buildKvs(t *testing.T) {
t.Run("length not equal", func(t *testing.T) {
keys := []string{"k1", "k2"}
values := []string{"v1"}
_, err := buildKvs(keys, values)
assert.Error(t, err)
})
t.Run("duplicate", func(t *testing.T) {
keys := []string{"k1", "k1"}
values := []string{"v1", "v2"}
_, err := buildKvs(keys, values)
assert.Error(t, err)
})
t.Run("normal case", func(t *testing.T) {
keys := []string{"k1", "k2"}
values := []string{"v1", "v2"}
kvs, err := buildKvs(keys, values)
assert.NoError(t, err)
for i, k := range keys {
v, ok := kvs[k]
assert.True(t, ok)
assert.Equal(t, values[i], v)
}
})
}
func Test_batchSave(t *testing.T) {
t.Run("normal case", func(t *testing.T) {
snapshot := kv.NewMockSnapshotKV()
snapshot.MultiSaveFunc = func(kvs map[string]string, ts typeutil.Timestamp) error {
return nil
}
kvs := map[string]string{
"k1": "v1",
"k2": "v2",
"k3": "v3",
}
maxTxnNum := 2
err := batchSave(snapshot, maxTxnNum, kvs, 100)
assert.NoError(t, err)
})
t.Run("multi save failed", func(t *testing.T) {
snapshot := kv.NewMockSnapshotKV()
snapshot.MultiSaveFunc = func(kvs map[string]string, ts typeutil.Timestamp) error {
return errors.New("mock")
}
kvs := map[string]string{
"k1": "v1",
"k2": "v2",
"k3": "v3",
}
maxTxnNum := 2
err := batchSave(snapshot, maxTxnNum, kvs, 100)
assert.Error(t, err)
})
}
func Test_min(t *testing.T) {
type args struct {
a int
b int
}
tests := []struct {
name string
args args
want int
}{
{
args: args{a: 1, b: 2},
want: 1,
},
{
args: args{a: 4, b: 3},
want: 3,
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
if got := min(tt.args.a, tt.args.b); got != tt.want {
t.Errorf("min() = %v, want %v", got, tt.want)
}
})
}
}
func Test_batchMultiSaveAndRemoveWithPrefix(t *testing.T) {
t.Run("failed to save", func(t *testing.T) {
snapshot := kv.NewMockSnapshotKV()

View File

@ -353,8 +353,10 @@ message Binlog {
int64 entries_num = 1;
uint64 timestamp_from = 2;
uint64 timestamp_to = 3;
// deprecated
string log_path = 4;
int64 log_size = 5;
int64 logID = 6;
}
message GetRecoveryInfoResponse {

View File

@ -2421,11 +2421,13 @@ func (m *FieldBinlog) GetBinlogs() []*Binlog {
}
type Binlog struct {
EntriesNum int64 `protobuf:"varint,1,opt,name=entries_num,json=entriesNum,proto3" json:"entries_num,omitempty"`
TimestampFrom uint64 `protobuf:"varint,2,opt,name=timestamp_from,json=timestampFrom,proto3" json:"timestamp_from,omitempty"`
TimestampTo uint64 `protobuf:"varint,3,opt,name=timestamp_to,json=timestampTo,proto3" json:"timestamp_to,omitempty"`
EntriesNum int64 `protobuf:"varint,1,opt,name=entries_num,json=entriesNum,proto3" json:"entries_num,omitempty"`
TimestampFrom uint64 `protobuf:"varint,2,opt,name=timestamp_from,json=timestampFrom,proto3" json:"timestamp_from,omitempty"`
TimestampTo uint64 `protobuf:"varint,3,opt,name=timestamp_to,json=timestampTo,proto3" json:"timestamp_to,omitempty"`
// deprecated
LogPath string `protobuf:"bytes,4,opt,name=log_path,json=logPath,proto3" json:"log_path,omitempty"`
LogSize int64 `protobuf:"varint,5,opt,name=log_size,json=logSize,proto3" json:"log_size,omitempty"`
LogID int64 `protobuf:"varint,6,opt,name=logID,proto3" json:"logID,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
@ -2491,6 +2493,13 @@ func (m *Binlog) GetLogSize() int64 {
return 0
}
func (m *Binlog) GetLogID() int64 {
if m != nil {
return m.LogID
}
return 0
}
type GetRecoveryInfoResponse struct {
Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"`
Channels []*VchannelInfo `protobuf:"bytes,2,rep,name=channels,proto3" json:"channels,omitempty"`
@ -4429,253 +4438,254 @@ func init() {
func init() { proto.RegisterFile("data_coord.proto", fileDescriptor_82cd95f524594f49) }
var fileDescriptor_82cd95f524594f49 = []byte{
// 3935 bytes of a gzipped FileDescriptorProto
// 3947 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xdc, 0x3c, 0x5b, 0x6f, 0x1b, 0x47,
0x77, 0x5e, 0xde, 0x44, 0x1e, 0x5e, 0x44, 0x8d, 0x1d, 0x89, 0xa6, 0xef, 0xeb, 0xd8, 0x91, 0xfd,
0x39, 0x76, 0x22, 0x37, 0xf8, 0x82, 0xfa, 0xfb, 0xf2, 0xc1, 0xb2, 0x2c, 0x99, 0xad, 0xe4, 0xc8,
0x2b, 0x39, 0x06, 0x9a, 0x02, 0xc4, 0x8a, 0x3b, 0xa2, 0x36, 0xe2, 0xee, 0xd2, 0xbb, 0x4b, 0xcb,
0x4a, 0x1f, 0x12, 0x34, 0x40, 0x81, 0x14, 0x45, 0xd3, 0x0b, 0x8a, 0xb6, 0x0f, 0x05, 0x8a, 0x3e,
0xa5, 0x05, 0x0a, 0x04, 0x08, 0xfa, 0xd0, 0x16, 0x79, 0x0f, 0xda, 0x87, 0xa2, 0x7f, 0xa2, 0xed,
0xaf, 0x28, 0x8a, 0xb9, 0xec, 0xec, 0x9d, 0x5c, 0x91, 0x76, 0x5c, 0x7c, 0x6f, 0x9a, 0xb3, 0xe7,
0x9c, 0x39, 0x33, 0x73, 0xee, 0x33, 0x14, 0x34, 0x35, 0xd5, 0x55, 0xbb, 0x3d, 0xcb, 0xb2, 0xb5,
0xdb, 0x43, 0xdb, 0x72, 0x2d, 0xb4, 0x60, 0xe8, 0x83, 0x17, 0x23, 0x87, 0x8d, 0x6e, 0x93, 0xcf,
0xed, 0x5a, 0xcf, 0x32, 0x0c, 0xcb, 0x64, 0xa0, 0x76, 0x43, 0x37, 0x5d, 0x6c, 0x9b, 0xea, 0x80,
0x8f, 0x6b, 0x41, 0x82, 0x76, 0xcd, 0xe9, 0x1d, 0x60, 0x43, 0x65, 0x23, 0x79, 0x0e, 0x8a, 0x0f,
0x8d, 0xa1, 0x7b, 0x2c, 0xff, 0x95, 0x04, 0xb5, 0xf5, 0xc1, 0xc8, 0x39, 0x50, 0xf0, 0xf3, 0x11,
0x76, 0x5c, 0xf4, 0x1e, 0x14, 0xf6, 0x54, 0x07, 0xb7, 0xa4, 0xcb, 0xd2, 0x72, 0x75, 0xe5, 0xfc,
0xed, 0xd0, 0xac, 0x7c, 0xbe, 0x2d, 0xa7, 0xbf, 0xaa, 0x3a, 0x58, 0xa1, 0x98, 0x08, 0x41, 0x41,
0xdb, 0xeb, 0xac, 0xb5, 0x72, 0x97, 0xa5, 0xe5, 0xbc, 0x42, 0xff, 0x46, 0x17, 0x01, 0x1c, 0xdc,
0x37, 0xb0, 0xe9, 0x76, 0xd6, 0x9c, 0x56, 0xfe, 0x72, 0x7e, 0x39, 0xaf, 0x04, 0x20, 0x48, 0x86,
0x5a, 0xcf, 0x1a, 0x0c, 0x70, 0xcf, 0xd5, 0x2d, 0xb3, 0xb3, 0xd6, 0x2a, 0x50, 0xda, 0x10, 0x4c,
0xfe, 0x2f, 0x09, 0xea, 0x5c, 0x34, 0x67, 0x68, 0x99, 0x0e, 0x46, 0x77, 0xa1, 0xe4, 0xb8, 0xaa,
0x3b, 0x72, 0xb8, 0x74, 0xe7, 0x12, 0xa5, 0xdb, 0xa1, 0x28, 0x0a, 0x47, 0x4d, 0x14, 0x2f, 0x3a,
0x7d, 0x3e, 0x3e, 0x7d, 0x64, 0x09, 0x85, 0xd8, 0x12, 0x96, 0x61, 0x7e, 0x9f, 0x48, 0xb7, 0xe3,
0x23, 0x15, 0x29, 0x52, 0x14, 0x4c, 0x38, 0xb9, 0xba, 0x81, 0x3f, 0xde, 0xdf, 0xc1, 0xea, 0xa0,
0x55, 0xa2, 0x73, 0x05, 0x20, 0xf2, 0x7f, 0x4a, 0xd0, 0x14, 0xe8, 0xde, 0x39, 0x9c, 0x81, 0x62,
0xcf, 0x1a, 0x99, 0x2e, 0x5d, 0x6a, 0x5d, 0x61, 0x03, 0x74, 0x05, 0x6a, 0xbd, 0x03, 0xd5, 0x34,
0xf1, 0xa0, 0x6b, 0xaa, 0x06, 0xa6, 0x8b, 0xaa, 0x28, 0x55, 0x0e, 0x7b, 0xac, 0x1a, 0x38, 0xd3,
0xda, 0x2e, 0x43, 0x75, 0xa8, 0xda, 0xae, 0x1e, 0xda, 0xfd, 0x20, 0x08, 0xb5, 0xa1, 0xac, 0x3b,
0x1d, 0x63, 0x68, 0xd9, 0x6e, 0xab, 0x78, 0x59, 0x5a, 0x2e, 0x2b, 0x62, 0x4c, 0x66, 0xd0, 0xe9,
0x5f, 0xbb, 0xaa, 0x73, 0xd8, 0x59, 0xe3, 0x2b, 0x0a, 0xc1, 0xe4, 0xbf, 0x95, 0x60, 0xf1, 0xbe,
0xe3, 0xe8, 0x7d, 0x33, 0xb6, 0xb2, 0x45, 0x28, 0x99, 0x96, 0x86, 0x3b, 0x6b, 0x74, 0x69, 0x79,
0x85, 0x8f, 0xd0, 0x39, 0xa8, 0x0c, 0x31, 0xb6, 0xbb, 0xb6, 0x35, 0xf0, 0x16, 0x56, 0x26, 0x00,
0xc5, 0x1a, 0x60, 0xf4, 0x04, 0x16, 0x9c, 0x08, 0x23, 0xa6, 0x57, 0xd5, 0x95, 0xab, 0xb7, 0x63,
0x96, 0x71, 0x3b, 0x3a, 0xa9, 0x12, 0xa7, 0x96, 0xbf, 0xcc, 0xc1, 0x69, 0x81, 0xc7, 0x64, 0x25,
0x7f, 0x93, 0x9d, 0x77, 0x70, 0x5f, 0x88, 0xc7, 0x06, 0x59, 0x76, 0x5e, 0x1c, 0x59, 0x3e, 0x78,
0x64, 0x19, 0x54, 0x3d, 0x7a, 0x1e, 0xc5, 0xf8, 0x79, 0x5c, 0x82, 0x2a, 0x7e, 0x39, 0xd4, 0x6d,
0xdc, 0x25, 0x8a, 0x43, 0xb7, 0xbc, 0xa0, 0x00, 0x03, 0xed, 0xea, 0x46, 0xd0, 0x36, 0xe6, 0x32,
0xdb, 0x86, 0xfc, 0x77, 0x12, 0x2c, 0xc5, 0x4e, 0x89, 0x1b, 0x9b, 0x02, 0x4d, 0xba, 0x72, 0x7f,
0x67, 0x88, 0xd9, 0x91, 0x0d, 0xbf, 0x3e, 0x6e, 0xc3, 0x7d, 0x74, 0x25, 0x46, 0x1f, 0x10, 0x32,
0x97, 0x5d, 0xc8, 0x43, 0x58, 0xda, 0xc0, 0x2e, 0x9f, 0x80, 0x7c, 0xc3, 0xce, 0xf4, 0xce, 0x2a,
0x6c, 0xd5, 0xb9, 0xa8, 0x55, 0xcb, 0xdf, 0xe5, 0x84, 0x2d, 0xd2, 0xa9, 0x3a, 0xe6, 0xbe, 0x85,
0xce, 0x43, 0x45, 0xa0, 0x70, 0xad, 0xf0, 0x01, 0xe8, 0xe7, 0x50, 0x24, 0x92, 0x32, 0x95, 0x68,
0xac, 0x5c, 0x49, 0x5e, 0x53, 0x80, 0xa7, 0xc2, 0xf0, 0x51, 0x07, 0x1a, 0x8e, 0xab, 0xda, 0x6e,
0x77, 0x68, 0x39, 0xf4, 0x9c, 0xa9, 0xe2, 0x54, 0x57, 0xe4, 0x30, 0x07, 0xe1, 0xd6, 0xb7, 0x9c,
0xfe, 0x36, 0xc7, 0x54, 0xea, 0x94, 0xd2, 0x1b, 0xa2, 0x87, 0x50, 0xc3, 0xa6, 0xe6, 0x33, 0x2a,
0x64, 0x66, 0x54, 0xc5, 0xa6, 0x26, 0xd8, 0xf8, 0xe7, 0x53, 0xcc, 0x7e, 0x3e, 0x7f, 0x24, 0x41,
0x2b, 0x7e, 0x40, 0xb3, 0xb8, 0xec, 0x7b, 0x8c, 0x08, 0xb3, 0x03, 0x1a, 0x6b, 0xe1, 0xe2, 0x90,
0x14, 0x4e, 0x22, 0xff, 0x85, 0x04, 0x6f, 0xf9, 0xe2, 0xd0, 0x4f, 0xaf, 0x4b, 0x5b, 0xd0, 0x4d,
0x68, 0xea, 0x66, 0x6f, 0x30, 0xd2, 0xf0, 0x53, 0xf3, 0x11, 0x56, 0x07, 0xee, 0xc1, 0x31, 0x3d,
0xc3, 0xb2, 0x12, 0x83, 0xcb, 0x5f, 0x49, 0xb0, 0x18, 0x95, 0x6b, 0x96, 0x4d, 0xfa, 0x0d, 0x28,
0xea, 0xe6, 0xbe, 0xe5, 0xed, 0xd1, 0xc5, 0x31, 0x46, 0x49, 0xe6, 0x62, 0xc8, 0xb2, 0x01, 0xe7,
0x36, 0xb0, 0xdb, 0x31, 0x1d, 0x6c, 0xbb, 0xab, 0xba, 0x39, 0xb0, 0xfa, 0xdb, 0xaa, 0x7b, 0x30,
0x83, 0x41, 0x85, 0x6c, 0x23, 0x17, 0xb1, 0x0d, 0xf9, 0x5b, 0x09, 0xce, 0x27, 0xcf, 0xc7, 0x97,
0xde, 0x86, 0xf2, 0xbe, 0x8e, 0x07, 0x1a, 0xd9, 0x5f, 0x89, 0xee, 0xaf, 0x18, 0x13, 0xc3, 0x1a,
0x12, 0x64, 0xbe, 0xc2, 0x2b, 0x29, 0xda, 0xbc, 0xe3, 0xda, 0xba, 0xd9, 0xdf, 0xd4, 0x1d, 0x57,
0x61, 0xf8, 0x81, 0xfd, 0xcc, 0x67, 0x57, 0xe3, 0x3f, 0x94, 0xe0, 0xe2, 0x06, 0x76, 0x1f, 0x08,
0xbf, 0x4c, 0xbe, 0xeb, 0x8e, 0xab, 0xf7, 0x9c, 0x57, 0x9b, 0x1b, 0x65, 0x08, 0xd0, 0xf2, 0x37,
0x12, 0x5c, 0x4a, 0x15, 0x86, 0x6f, 0x1d, 0xf7, 0x3b, 0x9e, 0x57, 0x4e, 0xf6, 0x3b, 0xbf, 0x8d,
0x8f, 0x3f, 0x51, 0x07, 0x23, 0xbc, 0xad, 0xea, 0x36, 0xf3, 0x3b, 0x53, 0x7a, 0xe1, 0x7f, 0x94,
0xe0, 0xc2, 0x06, 0x76, 0xb7, 0xbd, 0x98, 0xf4, 0x06, 0x77, 0x87, 0xe0, 0x04, 0x62, 0xa3, 0x97,
0x9c, 0x85, 0x60, 0xf2, 0x1f, 0xb3, 0xe3, 0x4c, 0x94, 0xf7, 0x8d, 0x6c, 0xe0, 0x45, 0x6a, 0x09,
0x01, 0x93, 0x7c, 0xc0, 0x52, 0x07, 0xbe, 0x7d, 0xf2, 0xdf, 0x48, 0x70, 0xf6, 0x7e, 0xef, 0xf9,
0x48, 0xb7, 0x31, 0x47, 0xda, 0xb4, 0x7a, 0x87, 0xd3, 0x6f, 0xae, 0x9f, 0x66, 0xe5, 0x42, 0x69,
0xd6, 0xa4, 0xd4, 0x7c, 0x11, 0x4a, 0x2e, 0xcb, 0xeb, 0x58, 0xa6, 0xc2, 0x47, 0x54, 0x3e, 0x05,
0x0f, 0xb0, 0xea, 0xfc, 0xff, 0x94, 0xef, 0x9b, 0x02, 0xd4, 0x3e, 0xe1, 0xe9, 0x18, 0x8d, 0xda,
0x51, 0x4d, 0x92, 0x92, 0x13, 0xaf, 0x40, 0x06, 0x97, 0x94, 0xd4, 0x6d, 0x40, 0xdd, 0xc1, 0xf8,
0x70, 0x9a, 0x18, 0x5d, 0x23, 0x84, 0x22, 0xb6, 0x6e, 0xc2, 0xc2, 0xc8, 0xa4, 0xa5, 0x01, 0xd6,
0xf8, 0x06, 0x32, 0xcd, 0x9d, 0xec, 0xbb, 0xe3, 0x84, 0xe8, 0x11, 0xaf, 0x3e, 0x02, 0xbc, 0x8a,
0x99, 0x78, 0x45, 0xc9, 0x50, 0x07, 0x9a, 0x9a, 0x6d, 0x0d, 0x87, 0x58, 0xeb, 0x3a, 0x1e, 0xab,
0x52, 0x36, 0x56, 0x9c, 0x4e, 0xb0, 0x7a, 0x0f, 0x4e, 0x47, 0x25, 0xed, 0x68, 0x24, 0x21, 0x25,
0x67, 0x98, 0xf4, 0x09, 0xdd, 0x82, 0x85, 0x38, 0x7e, 0x99, 0xe2, 0xc7, 0x3f, 0xa0, 0x77, 0x01,
0x45, 0x44, 0x25, 0xe8, 0x15, 0x86, 0x1e, 0x16, 0xa6, 0xa3, 0x39, 0xf2, 0xd7, 0x12, 0x2c, 0x3e,
0xa5, 0x05, 0x0a, 0x04, 0x08, 0xfa, 0xd0, 0x16, 0x41, 0x5f, 0x83, 0xf6, 0xa1, 0xe8, 0x9f, 0x68,
0xfb, 0x2b, 0x8a, 0x62, 0x2e, 0x3b, 0x7b, 0x27, 0x57, 0xa4, 0x1d, 0x17, 0x7d, 0xd3, 0x9c, 0x3d,
0x67, 0xe6, 0xcc, 0x9c, 0xfb, 0x99, 0xa1, 0xa0, 0xa9, 0xa9, 0xae, 0xda, 0xed, 0x59, 0x96, 0xad,
0xdd, 0x1e, 0xda, 0x96, 0x6b, 0xa1, 0x05, 0x43, 0x1f, 0xbc, 0x18, 0x39, 0x6c, 0x74, 0x9b, 0x7c,
0x6e, 0xd7, 0x7a, 0x96, 0x61, 0x58, 0x26, 0x03, 0xb5, 0x1b, 0xba, 0xe9, 0x62, 0xdb, 0x54, 0x07,
0x7c, 0x5c, 0x0b, 0x12, 0xb4, 0x6b, 0x4e, 0xef, 0x00, 0x1b, 0x2a, 0x1b, 0xc9, 0x73, 0x50, 0x7c,
0x68, 0x0c, 0xdd, 0x63, 0xf9, 0x2f, 0x24, 0xa8, 0xad, 0x0f, 0x46, 0xce, 0x81, 0x82, 0x9f, 0x8f,
0xb0, 0xe3, 0xa2, 0xf7, 0xa0, 0xb0, 0xa7, 0x3a, 0xb8, 0x25, 0x5d, 0x96, 0x96, 0xab, 0x2b, 0xe7,
0x6f, 0x87, 0x56, 0xe5, 0xeb, 0x6d, 0x39, 0xfd, 0x55, 0xd5, 0xc1, 0x0a, 0xc5, 0x44, 0x08, 0x0a,
0xda, 0x5e, 0x67, 0xad, 0x95, 0xbb, 0x2c, 0x2d, 0xe7, 0x15, 0xfa, 0x37, 0xba, 0x08, 0xe0, 0xe0,
0xbe, 0x81, 0x4d, 0xb7, 0xb3, 0xe6, 0xb4, 0xf2, 0x97, 0xf3, 0xcb, 0x79, 0x25, 0x00, 0x41, 0x32,
0xd4, 0x7a, 0xd6, 0x60, 0x80, 0x7b, 0xae, 0x6e, 0x99, 0x9d, 0xb5, 0x56, 0x81, 0xd2, 0x86, 0x60,
0xf2, 0x7f, 0x4a, 0x50, 0xe7, 0xac, 0x39, 0x43, 0xcb, 0x74, 0x30, 0xba, 0x0b, 0x25, 0xc7, 0x55,
0xdd, 0x91, 0xc3, 0xb9, 0x3b, 0x97, 0xc8, 0xdd, 0x0e, 0x45, 0x51, 0x38, 0x6a, 0x22, 0x7b, 0xd1,
0xe5, 0xf3, 0xf1, 0xe5, 0x23, 0x5b, 0x28, 0xc4, 0xb6, 0xb0, 0x0c, 0xf3, 0xfb, 0x84, 0xbb, 0x1d,
0x1f, 0xa9, 0x48, 0x91, 0xa2, 0x60, 0x32, 0x93, 0xab, 0x1b, 0xf8, 0xe3, 0xfd, 0x1d, 0xac, 0x0e,
0x5a, 0x25, 0xba, 0x56, 0x00, 0x22, 0xff, 0x87, 0x04, 0x4d, 0x81, 0xee, 0xc9, 0xe1, 0x0c, 0x14,
0x7b, 0xd6, 0xc8, 0x74, 0xe9, 0x56, 0xeb, 0x0a, 0x1b, 0xa0, 0x2b, 0x50, 0xeb, 0x1d, 0xa8, 0xa6,
0x89, 0x07, 0x5d, 0x53, 0x35, 0x30, 0xdd, 0x54, 0x45, 0xa9, 0x72, 0xd8, 0x63, 0xd5, 0xc0, 0x99,
0xf6, 0x76, 0x19, 0xaa, 0x43, 0xd5, 0x76, 0xf5, 0xd0, 0xe9, 0x07, 0x41, 0xa8, 0x0d, 0x65, 0xdd,
0xe9, 0x18, 0x43, 0xcb, 0x76, 0x5b, 0xc5, 0xcb, 0xd2, 0x72, 0x59, 0x11, 0x63, 0xb2, 0x82, 0x4e,
0xff, 0xda, 0x55, 0x9d, 0xc3, 0xce, 0x1a, 0xdf, 0x51, 0x08, 0x26, 0xff, 0xb5, 0x04, 0x8b, 0xf7,
0x1d, 0x47, 0xef, 0x9b, 0xb1, 0x9d, 0x2d, 0x42, 0xc9, 0xb4, 0x34, 0xdc, 0x59, 0xa3, 0x5b, 0xcb,
0x2b, 0x7c, 0x84, 0xce, 0x41, 0x65, 0x88, 0xb1, 0xdd, 0xb5, 0xad, 0x81, 0xb7, 0xb1, 0x32, 0x01,
0x28, 0xd6, 0x00, 0xa3, 0x27, 0xb0, 0xe0, 0x44, 0x26, 0x62, 0x7a, 0x55, 0x5d, 0xb9, 0x7a, 0x3b,
0x66, 0x19, 0xb7, 0xa3, 0x8b, 0x2a, 0x71, 0x6a, 0xf9, 0xcb, 0x1c, 0x9c, 0x16, 0x78, 0x8c, 0x57,
0xf2, 0x37, 0x39, 0x79, 0x07, 0xf7, 0x05, 0x7b, 0x6c, 0x90, 0xe5, 0xe4, 0x85, 0xc8, 0xf2, 0x41,
0x91, 0x65, 0x50, 0xf5, 0xa8, 0x3c, 0x8a, 0x71, 0x79, 0x5c, 0x82, 0x2a, 0x7e, 0x39, 0xd4, 0x6d,
0xdc, 0x25, 0x8a, 0x43, 0x8f, 0xbc, 0xa0, 0x00, 0x03, 0xed, 0xea, 0x46, 0xd0, 0x36, 0xe6, 0x32,
0xdb, 0x86, 0xfc, 0x37, 0x12, 0x2c, 0xc5, 0xa4, 0xc4, 0x8d, 0x4d, 0x81, 0x26, 0xdd, 0xb9, 0x7f,
0x32, 0xc4, 0xec, 0xc8, 0x81, 0x5f, 0x1f, 0x77, 0xe0, 0x3e, 0xba, 0x12, 0xa3, 0x0f, 0x30, 0x99,
0xcb, 0xce, 0xe4, 0x21, 0x2c, 0x6d, 0x60, 0x97, 0x2f, 0x40, 0xbe, 0x61, 0x67, 0x7a, 0x67, 0x15,
0xb6, 0xea, 0x5c, 0xd4, 0xaa, 0xe5, 0xef, 0x72, 0xc2, 0x16, 0xe9, 0x52, 0x1d, 0x73, 0xdf, 0x42,
0xe7, 0xa1, 0x22, 0x50, 0xb8, 0x56, 0xf8, 0x00, 0xf4, 0x73, 0x28, 0x12, 0x4e, 0x99, 0x4a, 0x34,
0x56, 0xae, 0x24, 0xef, 0x29, 0x30, 0xa7, 0xc2, 0xf0, 0x51, 0x07, 0x1a, 0x8e, 0xab, 0xda, 0x6e,
0x77, 0x68, 0x39, 0x54, 0xce, 0x54, 0x71, 0xaa, 0x2b, 0x72, 0x78, 0x06, 0xe1, 0xd6, 0xb7, 0x9c,
0xfe, 0x36, 0xc7, 0x54, 0xea, 0x94, 0xd2, 0x1b, 0xa2, 0x87, 0x50, 0xc3, 0xa6, 0xe6, 0x4f, 0x54,
0xc8, 0x3c, 0x51, 0x15, 0x9b, 0x9a, 0x98, 0xc6, 0x97, 0x4f, 0x31, 0xbb, 0x7c, 0xfe, 0x40, 0x82,
0x56, 0x5c, 0x40, 0xb3, 0xb8, 0xec, 0x7b, 0x8c, 0x08, 0x33, 0x01, 0x8d, 0xb5, 0x70, 0x21, 0x24,
0x85, 0x93, 0xc8, 0x7f, 0x26, 0xc1, 0x5b, 0x3e, 0x3b, 0xf4, 0xd3, 0xeb, 0xd2, 0x16, 0x74, 0x13,
0x9a, 0xba, 0xd9, 0x1b, 0x8c, 0x34, 0xfc, 0xd4, 0x7c, 0x84, 0xd5, 0x81, 0x7b, 0x70, 0x4c, 0x65,
0x58, 0x56, 0x62, 0x70, 0xf9, 0x2b, 0x09, 0x16, 0xa3, 0x7c, 0xcd, 0x72, 0x48, 0xbf, 0x06, 0x45,
0xdd, 0xdc, 0xb7, 0xbc, 0x33, 0xba, 0x38, 0xc6, 0x28, 0xc9, 0x5a, 0x0c, 0x59, 0x36, 0xe0, 0xdc,
0x06, 0x76, 0x3b, 0xa6, 0x83, 0x6d, 0x77, 0x55, 0x37, 0x07, 0x56, 0x7f, 0x5b, 0x75, 0x0f, 0x66,
0x30, 0xa8, 0x90, 0x6d, 0xe4, 0x22, 0xb6, 0x21, 0x7f, 0x2b, 0xc1, 0xf9, 0xe4, 0xf5, 0xf8, 0xd6,
0xdb, 0x50, 0xde, 0xd7, 0xf1, 0x40, 0x23, 0xe7, 0x2b, 0xd1, 0xf3, 0x15, 0x63, 0x62, 0x58, 0x43,
0x82, 0xcc, 0x77, 0x78, 0x25, 0x45, 0x9b, 0x77, 0x5c, 0x5b, 0x37, 0xfb, 0x9b, 0xba, 0xe3, 0x2a,
0x0c, 0x3f, 0x70, 0x9e, 0xf9, 0xec, 0x6a, 0xfc, 0xfb, 0x12, 0x5c, 0xdc, 0xc0, 0xee, 0x03, 0xe1,
0x97, 0xc9, 0x77, 0xdd, 0x71, 0xf5, 0x9e, 0xf3, 0x6a, 0x73, 0xa3, 0x0c, 0x01, 0x5a, 0xfe, 0x46,
0x82, 0x4b, 0xa9, 0xcc, 0xf0, 0xa3, 0xe3, 0x7e, 0xc7, 0xf3, 0xca, 0xc9, 0x7e, 0xe7, 0x37, 0xf1,
0xf1, 0x27, 0xea, 0x60, 0x84, 0xb7, 0x55, 0xdd, 0x66, 0x7e, 0x67, 0x4a, 0x2f, 0xfc, 0xf7, 0x12,
0x5c, 0xd8, 0xc0, 0xee, 0xb6, 0x17, 0x93, 0xde, 0xe0, 0xe9, 0x10, 0x9c, 0x40, 0x6c, 0xf4, 0x92,
0xb3, 0x10, 0x4c, 0xfe, 0x43, 0x26, 0xce, 0x44, 0x7e, 0xdf, 0xc8, 0x01, 0x5e, 0xa4, 0x96, 0x10,
0x30, 0xc9, 0x07, 0x2c, 0x75, 0xe0, 0xc7, 0x27, 0xff, 0x95, 0x04, 0x67, 0xef, 0xf7, 0x9e, 0x8f,
0x74, 0x1b, 0x73, 0xa4, 0x4d, 0xab, 0x77, 0x38, 0xfd, 0xe1, 0xfa, 0x69, 0x56, 0x2e, 0x94, 0x66,
0x4d, 0x4a, 0xcd, 0x17, 0xa1, 0xe4, 0xb2, 0xbc, 0x8e, 0x65, 0x2a, 0x7c, 0x44, 0xf9, 0x53, 0xf0,
0x00, 0xab, 0xce, 0xff, 0x4d, 0xfe, 0xbe, 0x29, 0x40, 0xed, 0x13, 0x9e, 0x8e, 0xd1, 0xa8, 0x1d,
0xd5, 0x24, 0x29, 0x39, 0xf1, 0x0a, 0x64, 0x70, 0x49, 0x49, 0xdd, 0x06, 0xd4, 0x1d, 0x8c, 0x0f,
0xa7, 0x89, 0xd1, 0x35, 0x42, 0x28, 0x62, 0xeb, 0x26, 0x2c, 0x8c, 0x4c, 0x5a, 0x1a, 0x60, 0x8d,
0x1f, 0x20, 0xd3, 0xdc, 0xc9, 0xbe, 0x3b, 0x4e, 0x88, 0x1e, 0xf1, 0xea, 0x23, 0x30, 0x57, 0x31,
0xd3, 0x5c, 0x51, 0x32, 0xd4, 0x81, 0xa6, 0x66, 0x5b, 0xc3, 0x21, 0xd6, 0xba, 0x8e, 0x37, 0x55,
0x29, 0xdb, 0x54, 0x9c, 0x4e, 0x4c, 0xf5, 0x1e, 0x9c, 0x8e, 0x72, 0xda, 0xd1, 0x48, 0x42, 0x4a,
0x64, 0x98, 0xf4, 0x09, 0xdd, 0x82, 0x85, 0x38, 0x7e, 0x99, 0xe2, 0xc7, 0x3f, 0xa0, 0x77, 0x01,
0x45, 0x58, 0x25, 0xe8, 0x15, 0x86, 0x1e, 0x66, 0xa6, 0xa3, 0x39, 0xf2, 0xd7, 0x12, 0x2c, 0x3e,
0x53, 0xdd, 0xde, 0xc1, 0x9a, 0xc1, 0x6d, 0x6d, 0x06, 0x5f, 0xf5, 0x4b, 0xa8, 0xbc, 0xe0, 0x7a,
0xe1, 0x05, 0xa4, 0x4b, 0x09, 0xfb, 0x13, 0xd4, 0x40, 0xc5, 0xa7, 0x90, 0x7f, 0x94, 0xe0, 0xcc,
0xe1, 0x05, 0xa4, 0x4b, 0x09, 0xe7, 0x13, 0xd4, 0x40, 0xc5, 0xa7, 0x90, 0x7f, 0x94, 0xe0, 0xcc,
0x7a, 0xa0, 0x2e, 0x7c, 0x03, 0x5e, 0x73, 0x52, 0x41, 0x7b, 0x1d, 0x1a, 0x86, 0x6a, 0x1f, 0xc6,
0xea, 0xd9, 0x08, 0x54, 0x7e, 0x09, 0xc0, 0x47, 0x5b, 0x4e, 0x7f, 0x0a, 0xf9, 0x3f, 0x84, 0x39,
0x3e, 0x2b, 0x77, 0x9f, 0x93, 0xf4, 0xcc, 0x43, 0x97, 0xff, 0x4d, 0x82, 0x86, 0x1f, 0x12, 0xa9,
0x91, 0x37, 0x20, 0x27, 0x4c, 0x3b, 0xd7, 0x59, 0x43, 0xbf, 0x84, 0x12, 0x6b, 0x74, 0x70, 0xde,
0xd7, 0xc2, 0xbc, 0x79, 0x13, 0x24, 0x10, 0x57, 0x29, 0x40, 0xe1, 0x44, 0x64, 0x8f, 0x44, 0x14,
0x11, 0xce, 0xc7, 0x87, 0xa0, 0x0e, 0xcc, 0x87, 0x53, 0x76, 0xcf, 0x84, 0x2f, 0xa7, 0x05, 0x8f,
0x35, 0xd5, 0x55, 0x69, 0xec, 0x68, 0x84, 0x32, 0x76, 0x47, 0xfe, 0xd3, 0x12, 0x54, 0x03, 0xab,
0x8c, 0xad, 0x24, 0x7a, 0xa4, 0xb9, 0xc9, 0x75, 0x63, 0x3e, 0x5e, 0x37, 0x5e, 0x83, 0x86, 0x4e,
0x93, 0xaf, 0x2e, 0x57, 0x45, 0xea, 0x35, 0x2b, 0x4a, 0x9d, 0x41, 0xb9, 0x5d, 0xa0, 0x8b, 0x50,
0x35, 0x47, 0x46, 0xd7, 0xda, 0xef, 0xda, 0xd6, 0x91, 0xc3, 0x0b, 0xd0, 0x8a, 0x39, 0x32, 0x3e,
0xde, 0x57, 0xac, 0x23, 0xc7, 0xaf, 0x71, 0x4a, 0x27, 0xac, 0x71, 0x2e, 0x42, 0xd5, 0x50, 0x5f,
0x12, 0xae, 0x5d, 0x73, 0x64, 0xd0, 0xda, 0x34, 0xaf, 0x54, 0x0c, 0xf5, 0xa5, 0x62, 0x1d, 0x3d,
0x1e, 0x19, 0x68, 0x19, 0x9a, 0x03, 0xd5, 0x71, 0xbb, 0xc1, 0xe2, 0xb6, 0x4c, 0x8b, 0xdb, 0x06,
0x81, 0x3f, 0xf4, 0x0b, 0xdc, 0x78, 0xb5, 0x54, 0x99, 0xa1, 0x5a, 0xd2, 0x8c, 0x81, 0xcf, 0x08,
0xb2, 0x57, 0x4b, 0x9a, 0x31, 0x10, 0x6c, 0x3e, 0x84, 0xb9, 0x3d, 0x9a, 0xd2, 0x3a, 0xad, 0x6a,
0xaa, 0xc3, 0x5c, 0x27, 0xd9, 0x2c, 0xcb, 0x7c, 0x15, 0x0f, 0x1d, 0xfd, 0x02, 0x2a, 0x34, 0x93,
0xa0, 0xb4, 0xb5, 0x4c, 0xb4, 0x3e, 0x01, 0xa1, 0xd6, 0xf0, 0xc0, 0x55, 0x29, 0x75, 0x3d, 0x1b,
0xb5, 0x20, 0x20, 0x4e, 0xba, 0x67, 0x63, 0xd5, 0xc5, 0xda, 0xea, 0xf1, 0x03, 0xcb, 0x18, 0xaa,
0x54, 0x99, 0x5a, 0x0d, 0x5a, 0xb6, 0x24, 0x7d, 0x22, 0x8e, 0xa1, 0x27, 0x46, 0xeb, 0xb6, 0x65,
0xb4, 0xe6, 0x99, 0x63, 0x08, 0x43, 0xd1, 0x05, 0x00, 0xcf, 0x3d, 0xab, 0x6e, 0xab, 0x49, 0x4f,
0xb1, 0xc2, 0x21, 0xf7, 0x69, 0xef, 0x4a, 0x77, 0xba, 0xac, 0x4b, 0xa4, 0x9b, 0xfd, 0xd6, 0x02,
0x9d, 0xb1, 0xea, 0xb5, 0x95, 0x74, 0xb3, 0x2f, 0x7f, 0x01, 0x67, 0x7c, 0x25, 0x0a, 0x1c, 0x58,
0xfc, 0xec, 0xa5, 0x69, 0xcf, 0x7e, 0x7c, 0xbd, 0xf2, 0x1f, 0x05, 0x58, 0xdc, 0x51, 0x5f, 0xe0,
0xd7, 0x5f, 0x1a, 0x65, 0x72, 0xd9, 0x9b, 0xb0, 0x40, 0xab, 0xa1, 0x95, 0x80, 0x3c, 0x63, 0x72,
0x86, 0xe0, 0x89, 0xc7, 0x09, 0xd1, 0xaf, 0x48, 0xb2, 0x83, 0x7b, 0x87, 0xdb, 0x96, 0xee, 0xe7,
0x0b, 0x17, 0x12, 0xf8, 0x3c, 0x10, 0x58, 0x4a, 0x90, 0x02, 0x6d, 0xc7, 0xbd, 0x1f, 0xcb, 0x14,
0xde, 0x19, 0x5b, 0xa0, 0xfb, 0xbb, 0x1f, 0x75, 0x82, 0xa8, 0x05, 0x73, 0x3c, 0xcc, 0x53, 0xd7,
0x50, 0x56, 0xbc, 0x21, 0xda, 0x86, 0xd3, 0x6c, 0x05, 0x3b, 0x5c, 0xef, 0xd9, 0xe2, 0xcb, 0x99,
0x16, 0x9f, 0x44, 0x1a, 0x36, 0x9b, 0xca, 0x49, 0xcd, 0xa6, 0x05, 0x73, 0x5c, 0x95, 0xa9, 0xbb,
0x28, 0x2b, 0xde, 0x90, 0x1c, 0xb3, 0xaf, 0xd4, 0x55, 0xfa, 0xcd, 0x07, 0x90, 0xb2, 0x12, 0xfc,
0xfd, 0x9c, 0xd0, 0x4a, 0xfa, 0x08, 0xca, 0x42, 0xc3, 0x73, 0x99, 0x35, 0x5c, 0xd0, 0x44, 0xdd,
0x78, 0x3e, 0xe2, 0xc6, 0xe5, 0x7f, 0x97, 0xa0, 0xb6, 0x46, 0x96, 0xb4, 0x69, 0xf5, 0x69, 0xd0,
0xb9, 0x06, 0x0d, 0x1b, 0xf7, 0x2c, 0x5b, 0xeb, 0x62, 0xd3, 0xb5, 0x75, 0xcc, 0x3a, 0x10, 0x05,
0xa5, 0xce, 0xa0, 0x0f, 0x19, 0x90, 0xa0, 0x11, 0xcf, 0xec, 0xb8, 0xaa, 0x31, 0xec, 0xee, 0x13,
0x0f, 0x90, 0x63, 0x68, 0x02, 0x4a, 0x1d, 0xc0, 0x15, 0xa8, 0xf9, 0x68, 0xae, 0x45, 0xe7, 0x2f,
0x28, 0x55, 0x01, 0xdb, 0xb5, 0xd0, 0xdb, 0xd0, 0xa0, 0x7b, 0xda, 0x1d, 0x58, 0xfd, 0x2e, 0xa9,
0xd6, 0x79, 0x3c, 0xaa, 0x69, 0x5c, 0x2c, 0x72, 0x56, 0x61, 0x2c, 0x47, 0xff, 0x1c, 0xf3, 0x88,
0x24, 0xb0, 0x76, 0xf4, 0xcf, 0x31, 0x49, 0x07, 0xea, 0x24, 0xbc, 0x3e, 0xb6, 0x34, 0xbc, 0x3b,
0x65, 0x32, 0x92, 0xa1, 0xad, 0x7b, 0x1e, 0x2a, 0x62, 0x05, 0x7c, 0x49, 0x3e, 0x00, 0xad, 0x43,
0xc3, 0x4b, 0x9b, 0xbb, 0xac, 0x9a, 0x2c, 0xa4, 0x26, 0x87, 0x81, 0x00, 0xe9, 0x28, 0x75, 0x8f,
0x8c, 0x0e, 0xe5, 0x75, 0xa8, 0x05, 0x3f, 0x93, 0x59, 0x77, 0xa2, 0x8a, 0x22, 0x00, 0x44, 0x1b,
0x1f, 0x8f, 0x0c, 0x72, 0xa6, 0xdc, 0xb1, 0x78, 0x43, 0xf9, 0x2b, 0x09, 0xea, 0x3c, 0xaa, 0xef,
0x88, 0x0b, 0x10, 0xba, 0x34, 0x89, 0x2e, 0x8d, 0xfe, 0x8d, 0x7e, 0x33, 0xdc, 0xb3, 0x7c, 0x3b,
0xd1, 0x09, 0x50, 0x26, 0x34, 0x81, 0x0e, 0x85, 0xf4, 0x2c, 0xfd, 0x8b, 0x2f, 0x89, 0xa2, 0xf1,
0xa3, 0xa1, 0x8a, 0xd6, 0x82, 0x39, 0x55, 0xd3, 0x6c, 0xec, 0x38, 0x5c, 0x0e, 0x6f, 0x48, 0xbe,
0xbc, 0xc0, 0xb6, 0xe3, 0xa9, 0x7c, 0x5e, 0xf1, 0x86, 0xe8, 0x17, 0x50, 0x16, 0x19, 0x77, 0x3e,
0x29, 0xcb, 0x0a, 0xca, 0xc9, 0xab, 0x6d, 0x41, 0x21, 0xff, 0x53, 0x0e, 0x1a, 0x7c, 0xc3, 0x56,
0x79, 0xd8, 0x1d, 0x6f, 0x7c, 0xab, 0x50, 0xdb, 0xf7, 0x6d, 0x7f, 0x5c, 0x5f, 0x2d, 0xe8, 0x22,
0x42, 0x34, 0x93, 0x0c, 0x30, 0x1c, 0xf8, 0x0b, 0x33, 0x05, 0xfe, 0xe2, 0x49, 0x3d, 0x58, 0x3c,
0x15, 0x2c, 0x25, 0xa4, 0x82, 0xf2, 0xef, 0x42, 0x35, 0xc0, 0x80, 0x7a, 0x68, 0xd6, 0x90, 0xe3,
0x3b, 0xe6, 0x0d, 0xd1, 0x5d, 0x3f, 0xfd, 0x61, 0x5b, 0x75, 0x36, 0x41, 0x96, 0x48, 0xe6, 0x23,
0xff, 0xbd, 0x04, 0x25, 0xce, 0xf9, 0x12, 0x54, 0xb9, 0xd3, 0xa1, 0xa9, 0x21, 0xe3, 0x0e, 0x1c,
0xea, 0xd9, 0x08, 0x54, 0x7e, 0x09, 0xc0, 0x47, 0x5b, 0x4e, 0x7f, 0x0a, 0xfe, 0x3f, 0x84, 0x39,
0xbe, 0x2a, 0x77, 0x9f, 0x93, 0xf4, 0xcc, 0x43, 0x97, 0xff, 0x55, 0x82, 0x86, 0x1f, 0x12, 0xa9,
0x91, 0x37, 0x20, 0x27, 0x4c, 0x3b, 0xd7, 0x59, 0x43, 0xbf, 0x84, 0x12, 0x6b, 0x74, 0xf0, 0xb9,
0xaf, 0x85, 0xe7, 0xe6, 0x4d, 0x90, 0x40, 0x5c, 0xa5, 0x00, 0x85, 0x13, 0x91, 0x33, 0x12, 0x51,
0x44, 0x38, 0x1f, 0x1f, 0x82, 0x3a, 0x30, 0x1f, 0x4e, 0xd9, 0x3d, 0x13, 0xbe, 0x9c, 0x16, 0x3c,
0xd6, 0x54, 0x57, 0xa5, 0xb1, 0xa3, 0x11, 0xca, 0xd8, 0x1d, 0xf9, 0x8f, 0x4b, 0x50, 0x0d, 0xec,
0x32, 0xb6, 0x93, 0xa8, 0x48, 0x73, 0x93, 0xeb, 0xc6, 0x7c, 0xbc, 0x6e, 0xbc, 0x06, 0x0d, 0x9d,
0x26, 0x5f, 0x5d, 0xae, 0x8a, 0xd4, 0x6b, 0x56, 0x94, 0x3a, 0x83, 0x72, 0xbb, 0x40, 0x17, 0xa1,
0x6a, 0x8e, 0x8c, 0xae, 0xb5, 0xdf, 0xb5, 0xad, 0x23, 0x87, 0x17, 0xa0, 0x15, 0x73, 0x64, 0x7c,
0xbc, 0xaf, 0x58, 0x47, 0x8e, 0x5f, 0xe3, 0x94, 0x4e, 0x58, 0xe3, 0x5c, 0x84, 0xaa, 0xa1, 0xbe,
0x24, 0xb3, 0x76, 0xcd, 0x91, 0x41, 0x6b, 0xd3, 0xbc, 0x52, 0x31, 0xd4, 0x97, 0x8a, 0x75, 0xf4,
0x78, 0x64, 0xa0, 0x65, 0x68, 0x0e, 0x54, 0xc7, 0xed, 0x06, 0x8b, 0xdb, 0x32, 0x2d, 0x6e, 0x1b,
0x04, 0xfe, 0xd0, 0x2f, 0x70, 0xe3, 0xd5, 0x52, 0x65, 0x86, 0x6a, 0x49, 0x33, 0x06, 0xfe, 0x44,
0x90, 0xbd, 0x5a, 0xd2, 0x8c, 0x81, 0x98, 0xe6, 0x43, 0x98, 0xdb, 0xa3, 0x29, 0xad, 0xd3, 0xaa,
0xa6, 0x3a, 0xcc, 0x75, 0x92, 0xcd, 0xb2, 0xcc, 0x57, 0xf1, 0xd0, 0xd1, 0x2f, 0xa0, 0x42, 0x33,
0x09, 0x4a, 0x5b, 0xcb, 0x44, 0xeb, 0x13, 0x10, 0x6a, 0x0d, 0x0f, 0x5c, 0x95, 0x52, 0xd7, 0xb3,
0x51, 0x0b, 0x02, 0xe2, 0xa4, 0x7b, 0x36, 0x56, 0x5d, 0xac, 0xad, 0x1e, 0x3f, 0xb0, 0x8c, 0xa1,
0x4a, 0x95, 0xa9, 0xd5, 0xa0, 0x65, 0x4b, 0xd2, 0x27, 0xe2, 0x18, 0x7a, 0x62, 0xb4, 0x6e, 0x5b,
0x46, 0x6b, 0x9e, 0x39, 0x86, 0x30, 0x14, 0x5d, 0x00, 0xf0, 0xdc, 0xb3, 0xea, 0xb6, 0x9a, 0x54,
0x8a, 0x15, 0x0e, 0xb9, 0x4f, 0x7b, 0x57, 0xba, 0xd3, 0x65, 0x5d, 0x22, 0xdd, 0xec, 0xb7, 0x16,
0xe8, 0x8a, 0x55, 0xaf, 0xad, 0xa4, 0x9b, 0x7d, 0xf9, 0x0b, 0x38, 0xe3, 0x2b, 0x51, 0x40, 0x60,
0x71, 0xd9, 0x4b, 0xd3, 0xca, 0x7e, 0x7c, 0xbd, 0xf2, 0xef, 0x05, 0x58, 0xdc, 0x51, 0x5f, 0xe0,
0xd7, 0x5f, 0x1a, 0x65, 0x72, 0xd9, 0x9b, 0xb0, 0x40, 0xab, 0xa1, 0x95, 0x00, 0x3f, 0x63, 0x72,
0x86, 0xa0, 0xc4, 0xe3, 0x84, 0xe8, 0x57, 0x24, 0xd9, 0xc1, 0xbd, 0xc3, 0x6d, 0x4b, 0xf7, 0xf3,
0x85, 0x0b, 0x09, 0xf3, 0x3c, 0x10, 0x58, 0x4a, 0x90, 0x02, 0x6d, 0xc7, 0xbd, 0x1f, 0xcb, 0x14,
0xde, 0x19, 0x5b, 0xa0, 0xfb, 0xa7, 0x1f, 0x75, 0x82, 0xa8, 0x05, 0x73, 0x3c, 0xcc, 0x53, 0xd7,
0x50, 0x56, 0xbc, 0x21, 0xda, 0x86, 0xd3, 0x6c, 0x07, 0x3b, 0x5c, 0xef, 0xd9, 0xe6, 0xcb, 0x99,
0x36, 0x9f, 0x44, 0x1a, 0x36, 0x9b, 0xca, 0x49, 0xcd, 0xa6, 0x05, 0x73, 0x5c, 0x95, 0xa9, 0xbb,
0x28, 0x2b, 0xde, 0x90, 0x88, 0xd9, 0x57, 0xea, 0x2a, 0xfd, 0xe6, 0x03, 0x48, 0x59, 0x09, 0xfe,
0x79, 0x4e, 0x68, 0x25, 0x7d, 0x04, 0x65, 0xa1, 0xe1, 0xb9, 0xcc, 0x1a, 0x2e, 0x68, 0xa2, 0x6e,
0x3c, 0x1f, 0x71, 0xe3, 0xf2, 0xbf, 0x49, 0x50, 0x5b, 0x23, 0x5b, 0xda, 0xb4, 0xfa, 0x34, 0xe8,
0x5c, 0x83, 0x86, 0x8d, 0x7b, 0x96, 0xad, 0x75, 0xb1, 0xe9, 0xda, 0x3a, 0x66, 0x1d, 0x88, 0x82,
0x52, 0x67, 0xd0, 0x87, 0x0c, 0x48, 0xd0, 0x88, 0x67, 0x76, 0x5c, 0xd5, 0x18, 0x76, 0xf7, 0x89,
0x07, 0xc8, 0x31, 0x34, 0x01, 0xa5, 0x0e, 0xe0, 0x0a, 0xd4, 0x7c, 0x34, 0xd7, 0xa2, 0xeb, 0x17,
0x94, 0xaa, 0x80, 0xed, 0x5a, 0xe8, 0x6d, 0x68, 0xd0, 0x33, 0xed, 0x0e, 0xac, 0x7e, 0x97, 0x54,
0xeb, 0x3c, 0x1e, 0xd5, 0x34, 0xce, 0x16, 0x91, 0x55, 0x18, 0xcb, 0xd1, 0x3f, 0xc7, 0x3c, 0x22,
0x09, 0xac, 0x1d, 0xfd, 0x73, 0x4c, 0xd2, 0x81, 0x3a, 0x09, 0xaf, 0x8f, 0x2d, 0x0d, 0xef, 0x4e,
0x99, 0x8c, 0x64, 0x68, 0xeb, 0x9e, 0x87, 0x8a, 0xd8, 0x01, 0xdf, 0x92, 0x0f, 0x40, 0xeb, 0xd0,
0xf0, 0xd2, 0xe6, 0x2e, 0xab, 0x26, 0x0b, 0xa9, 0xc9, 0x61, 0x20, 0x40, 0x3a, 0x4a, 0xdd, 0x23,
0xa3, 0x43, 0x79, 0x1d, 0x6a, 0xc1, 0xcf, 0x64, 0xd5, 0x9d, 0xa8, 0xa2, 0x08, 0x00, 0xd1, 0xc6,
0xc7, 0x23, 0x83, 0xc8, 0x94, 0x3b, 0x16, 0x6f, 0x28, 0x7f, 0x25, 0x41, 0x9d, 0x47, 0xf5, 0x1d,
0x71, 0x01, 0x42, 0xb7, 0x26, 0xd1, 0xad, 0xd1, 0xbf, 0xd1, 0xaf, 0x87, 0x7b, 0x96, 0x6f, 0x27,
0x3a, 0x01, 0x3a, 0x09, 0x4d, 0xa0, 0x43, 0x21, 0x3d, 0x4b, 0xff, 0xe2, 0x4b, 0xa2, 0x68, 0x5c,
0x34, 0x54, 0xd1, 0x5a, 0x30, 0xa7, 0x6a, 0x9a, 0x8d, 0x1d, 0x87, 0xf3, 0xe1, 0x0d, 0xc9, 0x97,
0x17, 0xd8, 0x76, 0x3c, 0x95, 0xcf, 0x2b, 0xde, 0x10, 0xfd, 0x02, 0xca, 0x22, 0xe3, 0xce, 0x27,
0x65, 0x59, 0x41, 0x3e, 0x79, 0xb5, 0x2d, 0x28, 0xe4, 0x7f, 0xc8, 0x41, 0x83, 0x1f, 0xd8, 0x2a,
0x0f, 0xbb, 0xe3, 0x8d, 0x6f, 0x15, 0x6a, 0xfb, 0xbe, 0xed, 0x8f, 0xeb, 0xab, 0x05, 0x5d, 0x44,
0x88, 0x66, 0x92, 0x01, 0x86, 0x03, 0x7f, 0x61, 0xa6, 0xc0, 0x5f, 0x3c, 0xa9, 0x07, 0x8b, 0xa7,
0x82, 0xa5, 0x84, 0x54, 0x50, 0xfe, 0x6d, 0xa8, 0x06, 0x26, 0xa0, 0x1e, 0x9a, 0x35, 0xe4, 0xf8,
0x89, 0x79, 0x43, 0x74, 0xd7, 0x4f, 0x7f, 0xd8, 0x51, 0x9d, 0x4d, 0xe0, 0x25, 0x92, 0xf9, 0xc8,
0xff, 0x22, 0x41, 0x89, 0xcf, 0x7c, 0x09, 0xaa, 0xdc, 0xe9, 0xd0, 0xd4, 0x90, 0xcd, 0x0e, 0x1c,
0x44, 0x72, 0xc3, 0x57, 0xe7, 0x75, 0xce, 0x42, 0x39, 0xe2, 0x6f, 0xe6, 0x78, 0x58, 0xf0, 0x3e,
0x05, 0x9c, 0x0c, 0xf9, 0x44, 0xfd, 0xcb, 0x8f, 0x12, 0xbd, 0x79, 0x50, 0x70, 0xcf, 0x7a, 0x81,
0xed, 0xe3, 0xd9, 0x5b, 0xb6, 0xf7, 0x02, 0x0a, 0x9d, 0xb1, 0x84, 0x14, 0x04, 0xe8, 0x9e, 0xbf,
0xdd, 0xf9, 0xa4, 0x7e, 0x55, 0xd0, 0xc3, 0x70, 0x75, 0xf4, 0xb7, 0xfd, 0x4f, 0x58, 0xf3, 0x39,
0xbc, 0x94, 0x69, 0xf3, 0x9a, 0x57, 0x52, 0x99, 0xc8, 0x7f, 0x2e, 0xc1, 0xd9, 0x0d, 0xec, 0xae,
0x87, 0xdb, 0x11, 0x6f, 0x5a, 0x2a, 0x03, 0xda, 0x49, 0x42, 0xcd, 0x72, 0xea, 0x6d, 0x28, 0x8b,
0xc6, 0x0a, 0xbb, 0x42, 0x10, 0x63, 0xf9, 0x0f, 0x24, 0x68, 0xf1, 0x59, 0xe8, 0x9c, 0x24, 0xeb,
0x1e, 0x60, 0x17, 0x6b, 0x3f, 0x75, 0x69, 0xfd, 0x83, 0x04, 0xcd, 0xa0, 0xc7, 0xa7, 0x4e, 0xfb,
0x03, 0x28, 0xd2, 0x0e, 0x06, 0x97, 0x60, 0xa2, 0xb2, 0x32, 0x6c, 0xe2, 0x32, 0x68, 0x9a, 0xb7,
0x2b, 0x82, 0x13, 0x1f, 0xfa, 0x61, 0x27, 0x7f, 0xf2, 0xb0, 0xc3, 0xc3, 0xb0, 0x35, 0x22, 0x7c,
0x59, 0xeb, 0xcf, 0x07, 0xc8, 0xbf, 0x05, 0x8b, 0x7e, 0xc5, 0xc2, 0xe8, 0xa6, 0xd5, 0x24, 0xf9,
0x2f, 0x25, 0x38, 0xbd, 0x73, 0x6c, 0xf6, 0xa2, 0x3a, 0xb9, 0x08, 0xa5, 0xe1, 0x40, 0xf5, 0x5b,
0x89, 0x7c, 0x44, 0x73, 0x08, 0x36, 0x37, 0xd6, 0x88, 0x03, 0x62, 0x8b, 0xae, 0x0a, 0xd8, 0xae,
0x35, 0x31, 0x2e, 0x5c, 0x13, 0x25, 0x16, 0xd6, 0x98, 0xab, 0x63, 0xfd, 0x99, 0xba, 0x80, 0x12,
0x57, 0x27, 0x7f, 0x9f, 0x83, 0x56, 0x60, 0x99, 0x3f, 0x75, 0x74, 0x4b, 0xc9, 0xc9, 0xf3, 0xaf,
0x28, 0x27, 0x2f, 0xcc, 0x1e, 0xd1, 0x8a, 0x49, 0x11, 0xed, 0x5f, 0x73, 0xd0, 0xf0, 0x77, 0x6d,
0x7b, 0xa0, 0x9a, 0xa9, 0x47, 0xb9, 0x23, 0xb2, 0xb9, 0xf0, 0x3e, 0xfd, 0x2c, 0x49, 0x53, 0x53,
0x0e, 0x42, 0x89, 0xb0, 0x20, 0x75, 0x31, 0x2b, 0x9b, 0x68, 0x77, 0x83, 0x67, 0x90, 0xcc, 0x24,
0x74, 0x03, 0xa3, 0x5b, 0x80, 0xb8, 0x1e, 0x77, 0x75, 0xb3, 0xeb, 0xe0, 0x9e, 0x65, 0x6a, 0x4c,
0xc3, 0x8b, 0x4a, 0x93, 0x7f, 0xe9, 0x98, 0x3b, 0x0c, 0x8e, 0x3e, 0x80, 0x82, 0x7b, 0x3c, 0x64,
0xb1, 0xaa, 0x91, 0x18, 0x03, 0x7c, 0xb9, 0x76, 0x8f, 0x87, 0x58, 0xa1, 0xe8, 0xde, 0x1b, 0x14,
0xd7, 0x56, 0x5f, 0xf0, 0xc0, 0x5f, 0x50, 0x02, 0x10, 0x62, 0xb3, 0xde, 0x1e, 0xce, 0xb1, 0x00,
0xc9, 0x87, 0xf2, 0x3f, 0xe7, 0xa0, 0xe9, 0xb3, 0x54, 0xb0, 0x33, 0x1a, 0xa4, 0x9b, 0xc2, 0xf8,
0x92, 0x77, 0x92, 0x15, 0xfc, 0x0a, 0xaa, 0xfc, 0x3c, 0x4f, 0xa0, 0x0f, 0xc0, 0x48, 0x36, 0xc7,
0x28, 0x68, 0xf1, 0x15, 0x29, 0x68, 0xe9, 0x84, 0x0a, 0x2a, 0x7f, 0x2b, 0xc1, 0x5b, 0x31, 0xb7,
0x34, 0x76, 0x03, 0xc7, 0x27, 0xe6, 0xdc, 0x5d, 0x45, 0x59, 0x72, 0x0f, 0x79, 0x0f, 0x4a, 0x36,
0xe5, 0xce, 0xef, 0x28, 0xae, 0x8e, 0x55, 0x0e, 0x26, 0x88, 0xc2, 0x49, 0xe4, 0x3f, 0x93, 0x60,
0x29, 0x2e, 0xea, 0x0c, 0x61, 0x6f, 0x15, 0xe6, 0x18, 0x6b, 0xcf, 0x86, 0x96, 0xc7, 0xdb, 0x90,
0xbf, 0x39, 0x8a, 0x47, 0x28, 0xef, 0xc0, 0xa2, 0x17, 0x1d, 0xfd, 0x0d, 0xde, 0xc2, 0xae, 0x3a,
0x26, 0x2d, 0xbd, 0x04, 0x55, 0x96, 0xf5, 0xb0, 0x74, 0x8f, 0x15, 0x74, 0xb0, 0x27, 0xfa, 0x20,
0x24, 0x05, 0x3d, 0x43, 0xc3, 0x4b, 0xf4, 0x52, 0x20, 0xcb, 0x85, 0x91, 0x2c, 0xea, 0x45, 0x52,
0x1b, 0xb2, 0xa5, 0x55, 0x94, 0x10, 0x2c, 0xa9, 0x49, 0x9c, 0x9f, 0xb2, 0x49, 0xbc, 0x09, 0x6f,
0x45, 0x44, 0x9d, 0xe1, 0x48, 0xc8, 0xca, 0x17, 0x77, 0xc2, 0x2f, 0x35, 0xa6, 0xcf, 0xb7, 0x2e,
0x88, 0xeb, 0x84, 0xae, 0xae, 0x45, 0x6d, 0x5d, 0x43, 0x1f, 0x41, 0xc5, 0xc4, 0x47, 0xdd, 0x60,
0xb8, 0xcf, 0xd0, 0x35, 0x2e, 0x9b, 0xf8, 0x88, 0xfe, 0x25, 0x3f, 0x86, 0xa5, 0x98, 0xa8, 0xb3,
0xac, 0xfd, 0x5f, 0x24, 0x38, 0xbb, 0x66, 0x5b, 0xc3, 0x4f, 0x74, 0xdb, 0x1d, 0xa9, 0x83, 0xf0,
0xe5, 0xeb, 0xeb, 0x69, 0x1c, 0x3c, 0x0a, 0x24, 0x7e, 0x4c, 0x01, 0x6e, 0x25, 0x98, 0x40, 0x5c,
0x28, 0xbe, 0xe8, 0x40, 0x9a, 0xf8, 0xdf, 0xf9, 0x24, 0xe1, 0x39, 0xde, 0x84, 0xc0, 0x9f, 0x25,
0x2f, 0x4e, 0xec, 0x33, 0xe6, 0xa7, 0xed, 0x33, 0xa6, 0x78, 0xe1, 0xc2, 0x2b, 0xf2, 0xc2, 0x27,
0x2e, 0x7c, 0x1f, 0x41, 0xb8, 0x07, 0x4c, 0xc3, 0xdf, 0x54, 0xcd, 0xe3, 0x55, 0x00, 0xbf, 0x1f,
0xca, 0x1f, 0xda, 0x65, 0x61, 0x13, 0xa0, 0x22, 0xa7, 0x25, 0x22, 0x1e, 0xbd, 0xea, 0x08, 0x75,
0xe8, 0x9e, 0x40, 0x3b, 0x49, 0x4b, 0x67, 0xd1, 0xfc, 0xef, 0x73, 0x00, 0x1d, 0xf1, 0x36, 0x73,
0x3a, 0x67, 0x7e, 0x15, 0xea, 0xbe, 0xc2, 0xf8, 0xf6, 0x1e, 0xd4, 0x22, 0x8d, 0x98, 0x84, 0x28,
0xa5, 0x08, 0x4e, 0xac, 0xbc, 0xd2, 0x28, 0x9f, 0x80, 0xd5, 0x30, 0xa5, 0x88, 0xfa, 0xcf, 0x73,
0x50, 0xb1, 0xad, 0xa3, 0x2e, 0x31, 0x33, 0xcd, 0x7b, 0x7c, 0x6a, 0x5b, 0x47, 0xc4, 0xf8, 0x34,
0xb4, 0x04, 0x73, 0xae, 0xea, 0x1c, 0x12, 0xfe, 0xa5, 0xc0, 0xfd, 0xbf, 0x86, 0xce, 0x40, 0x71,
0x5f, 0x1f, 0x60, 0x76, 0xdd, 0x5c, 0x51, 0xd8, 0x00, 0xfd, 0xdc, 0x7b, 0x25, 0x55, 0xce, 0xfc,
0xc6, 0x83, 0x3d, 0x94, 0xfa, 0x51, 0x82, 0x79, 0x7f, 0xd7, 0xa8, 0x03, 0x22, 0x3e, 0x8d, 0xfa,
0xb3, 0x07, 0x96, 0xc6, 0x5c, 0x45, 0x23, 0xc5, 0xa5, 0x33, 0x42, 0xe6, 0xb5, 0x7c, 0x92, 0x71,
0x95, 0x20, 0x59, 0x17, 0x59, 0xb4, 0xae, 0x79, 0xd7, 0x8e, 0x25, 0xdb, 0x3a, 0xea, 0x68, 0x62,
0x37, 0xd8, 0xcb, 0x52, 0x56, 0xf7, 0x90, 0xdd, 0x78, 0x40, 0x1f, 0x97, 0x5e, 0x85, 0x3a, 0xb6,
0x6d, 0xcb, 0xee, 0x1a, 0xd8, 0x71, 0xd4, 0x3e, 0xe6, 0x09, 0x70, 0x8d, 0x02, 0xb7, 0x18, 0x4c,
0xfe, 0x21, 0x0f, 0x0d, 0x7f, 0x29, 0xde, 0x65, 0xa3, 0xae, 0x79, 0x97, 0x8d, 0x3a, 0x39, 0x3a,
0xb0, 0x99, 0x2b, 0x14, 0x87, 0xbb, 0x9a, 0x6b, 0x49, 0x4a, 0x85, 0x43, 0x3b, 0x1a, 0x89, 0xab,
0xc4, 0xc8, 0x4c, 0x4b, 0xc3, 0xfe, 0xe1, 0x82, 0x07, 0xe2, 0x67, 0x1b, 0xd2, 0x91, 0x42, 0x06,
0x1d, 0x29, 0x66, 0xd0, 0x91, 0x52, 0x82, 0x8e, 0x2c, 0x42, 0x69, 0x6f, 0xd4, 0x3b, 0xc4, 0x2e,
0x4f, 0x57, 0xf9, 0x28, 0xac, 0x3b, 0xe5, 0x88, 0xee, 0x08, 0x15, 0xa9, 0x04, 0x55, 0xe4, 0x1c,
0x54, 0xd8, 0xad, 0x57, 0xd7, 0x75, 0x68, 0x6f, 0x3f, 0xaf, 0x94, 0x19, 0x60, 0xd7, 0x41, 0x1f,
0x7a, 0xf9, 0x58, 0x35, 0xc9, 0xd8, 0xa9, 0xd7, 0x89, 0x68, 0x89, 0x97, 0x8d, 0x5d, 0x83, 0x06,
0x7d, 0xc3, 0xff, 0x7c, 0x84, 0xed, 0x63, 0x75, 0x6f, 0x80, 0x5b, 0x35, 0x2a, 0x4e, 0x9d, 0x40,
0x9f, 0x78, 0x40, 0xb2, 0x21, 0x14, 0x4d, 0x37, 0x35, 0xfc, 0x12, 0x6b, 0xad, 0x3a, 0xbb, 0x15,
0x23, 0xb0, 0x0e, 0x03, 0xc9, 0x9f, 0x01, 0xf2, 0xe7, 0x98, 0x2d, 0x29, 0x8b, 0x1c, 0x62, 0x2e,
0x7a, 0x88, 0xf2, 0x3f, 0x48, 0xb0, 0x10, 0x9c, 0x6c, 0xda, 0xf0, 0xf8, 0x11, 0x54, 0xd9, 0x1d,
0x48, 0x97, 0x98, 0x27, 0xef, 0x46, 0x5c, 0x18, 0xbb, 0x7b, 0x0a, 0xf8, 0x2f, 0xc8, 0x89, 0x12,
0x1c, 0x59, 0xf6, 0xa1, 0x6e, 0xf6, 0xbb, 0x44, 0x32, 0xcf, 0x28, 0x6a, 0x1c, 0xf8, 0x98, 0xc0,
0xe4, 0xaf, 0x25, 0xb8, 0xf8, 0x74, 0xa8, 0xa9, 0x2e, 0x0e, 0xe4, 0x09, 0xb3, 0x3e, 0x4a, 0xfb,
0xc0, 0x7b, 0x15, 0x96, 0xcb, 0xd6, 0xc7, 0x67, 0xd8, 0xf2, 0x16, 0x9c, 0x55, 0xb0, 0x83, 0x4d,
0x2d, 0xf4, 0x71, 0xea, 0x1e, 0xc4, 0x10, 0xda, 0x49, 0xec, 0x66, 0x39, 0x7b, 0x96, 0xb0, 0x75,
0x6d, 0xc2, 0xd6, 0xe5, 0xfe, 0x87, 0xe4, 0x09, 0x74, 0x1e, 0x57, 0xfe, 0x1f, 0x09, 0x16, 0xee,
0x6b, 0xde, 0x7c, 0xaf, 0x2d, 0x2f, 0x8c, 0xe6, 0x4d, 0xf9, 0x78, 0xde, 0xf4, 0xaa, 0x1c, 0x09,
0x77, 0xa9, 0xe6, 0xc8, 0xf0, 0x42, 0x85, 0x4d, 0x1f, 0x1d, 0xc8, 0xfb, 0xe2, 0x9a, 0x59, 0xc1,
0xfb, 0xd8, 0xc6, 0x66, 0x0f, 0x6f, 0x5a, 0xbd, 0xc3, 0xc0, 0xd3, 0x32, 0x29, 0xf8, 0xb4, 0x6c,
0xda, 0xa7, 0x6a, 0x37, 0xff, 0x5a, 0x82, 0x85, 0x58, 0x3f, 0x0b, 0x35, 0x00, 0x9e, 0x9a, 0x3d,
0xde, 0xe8, 0x6b, 0x9e, 0x42, 0x35, 0x28, 0x7b, 0x6d, 0xbf, 0xa6, 0x84, 0xaa, 0x30, 0xb7, 0x6b,
0x51, 0xec, 0x66, 0x0e, 0x35, 0xa1, 0xc6, 0x08, 0x47, 0xbd, 0x1e, 0x76, 0x9c, 0x66, 0x5e, 0x40,
0xd6, 0x55, 0x7d, 0x30, 0xb2, 0x71, 0xb3, 0x80, 0xea, 0x50, 0xd9, 0xb5, 0xf8, 0xc3, 0xbc, 0x66,
0x11, 0x21, 0x68, 0x78, 0xaf, 0xf4, 0x38, 0x51, 0x29, 0x00, 0xf3, 0xc8, 0xe6, 0x6e, 0x3e, 0x0b,
0xf6, 0x44, 0x76, 0x8f, 0x87, 0x18, 0x2d, 0xc1, 0xe9, 0xa7, 0xa6, 0x86, 0xf7, 0x75, 0x13, 0x6b,
0xfe, 0xa7, 0xe6, 0x29, 0x74, 0x1a, 0xe6, 0xb7, 0xb0, 0xdd, 0xc7, 0x01, 0x60, 0x0e, 0x2d, 0x40,
0x7d, 0x4b, 0x7f, 0x19, 0x00, 0xe5, 0xe5, 0x42, 0x59, 0x6a, 0x4a, 0x2b, 0xff, 0xbb, 0x04, 0x15,
0x52, 0xcf, 0x3c, 0xb0, 0x2c, 0x5b, 0x43, 0x43, 0x40, 0xf4, 0x1d, 0xab, 0x31, 0xb4, 0x4c, 0xf1,
0x3a, 0x1c, 0xbd, 0x97, 0x92, 0x33, 0xc5, 0x51, 0xb9, 0x1e, 0xb6, 0xaf, 0xa7, 0x50, 0x44, 0xd0,
0xe5, 0x53, 0xc8, 0xa0, 0x33, 0xee, 0xea, 0x06, 0xde, 0xd5, 0x7b, 0x87, 0xde, 0x03, 0x97, 0x31,
0x33, 0x46, 0x50, 0xbd, 0x19, 0x23, 0xd5, 0x33, 0x1f, 0xb0, 0xc7, 0xc6, 0x9e, 0x21, 0xca, 0xa7,
0xd0, 0x73, 0x38, 0xb3, 0x81, 0x03, 0x8e, 0xc7, 0x9b, 0x70, 0x25, 0x7d, 0xc2, 0x18, 0xf2, 0x09,
0xa7, 0xdc, 0x84, 0x22, 0x6d, 0x16, 0xa3, 0x24, 0xdf, 0x14, 0xfc, 0x31, 0x57, 0xfb, 0x72, 0x3a,
0x82, 0xe0, 0xf6, 0x19, 0xcc, 0x47, 0x7e, 0x02, 0x82, 0x6e, 0x24, 0x90, 0x25, 0xff, 0x98, 0xa7,
0x7d, 0x33, 0x0b, 0xaa, 0x98, 0xab, 0x0f, 0x8d, 0xf0, 0x1b, 0x58, 0x94, 0xd4, 0x14, 0x48, 0x7c,
0xbd, 0xdf, 0xbe, 0x91, 0x01, 0x53, 0x4c, 0x64, 0x40, 0x33, 0xfa, 0x93, 0x04, 0x74, 0x73, 0x2c,
0x83, 0xb0, 0xba, 0xfd, 0x2c, 0x13, 0xae, 0x98, 0xee, 0x98, 0x2a, 0x41, 0xec, 0x95, 0x3b, 0xba,
0x9d, 0xcc, 0x26, 0xed, 0xf9, 0x7d, 0xfb, 0x4e, 0x66, 0x7c, 0x31, 0xf5, 0xef, 0xb3, 0x4b, 0xaa,
0xa4, 0x97, 0xe2, 0xe8, 0xfd, 0x64, 0x76, 0x63, 0x9e, 0xb8, 0xb7, 0x57, 0x4e, 0x42, 0x22, 0x84,
0xf8, 0x82, 0xde, 0x2e, 0x25, 0xbc, 0xb5, 0x8e, 0xda, 0x9d, 0xc7, 0x2f, 0xfd, 0x19, 0x79, 0xfb,
0xfd, 0x13, 0x50, 0x08, 0x01, 0xac, 0xe8, 0x6f, 0x3e, 0x3c, 0x33, 0xbc, 0x33, 0x51, 0x6b, 0xa6,
0xb3, 0xc1, 0x4f, 0x61, 0x3e, 0xf2, 0x4e, 0x28, 0xd1, 0x6a, 0x92, 0xdf, 0x12, 0xb5, 0xc7, 0xc5,
0x6b, 0x66, 0x92, 0x91, 0xcb, 0x3a, 0x94, 0xa2, 0xfd, 0x09, 0x17, 0x7a, 0xed, 0x9b, 0x59, 0x50,
0xc5, 0x42, 0x1c, 0xea, 0x2e, 0x23, 0x17, 0x5e, 0xe8, 0x56, 0x32, 0x8f, 0xe4, 0xcb, 0xba, 0xf6,
0xbb, 0x19, 0xb1, 0xc5, 0xa4, 0xbf, 0x07, 0x68, 0xe7, 0x80, 0xd4, 0x30, 0xe6, 0xbe, 0xde, 0x1f,
0xd9, 0x2a, 0x7b, 0x0c, 0x94, 0xe6, 0xa3, 0xe3, 0xa8, 0x29, 0xba, 0x32, 0x96, 0x42, 0x4c, 0xde,
0x05, 0xd8, 0xc0, 0xee, 0x16, 0x76, 0x6d, 0xa2, 0xa0, 0xd7, 0x13, 0xcf, 0xdb, 0x47, 0xf0, 0xa6,
0x7a, 0x67, 0x22, 0x5e, 0x20, 0x24, 0x34, 0xb7, 0x54, 0x93, 0x94, 0xef, 0xfe, 0x1b, 0xba, 0x5b,
0x89, 0xe4, 0x51, 0xb4, 0x94, 0x0d, 0x4d, 0xc5, 0x16, 0x53, 0x1e, 0x89, 0x30, 0x1b, 0xe8, 0xa6,
0x46, 0xdd, 0x8f, 0x2f, 0x73, 0xf2, 0x55, 0x59, 0xd4, 0xfd, 0x8c, 0xc1, 0x17, 0x13, 0x7f, 0x29,
0xd1, 0x1f, 0x14, 0x45, 0x10, 0x9e, 0xe9, 0xee, 0xc1, 0xf6, 0x40, 0x35, 0x9d, 0x2c, 0x22, 0x50,
0xc4, 0x13, 0x88, 0xc0, 0xf1, 0x85, 0x08, 0x1a, 0xd4, 0x43, 0x3d, 0x52, 0x94, 0xf4, 0x1a, 0x2d,
0xa9, 0xe1, 0xdb, 0x5e, 0x9e, 0x8c, 0x28, 0x66, 0x39, 0x80, 0xba, 0xa7, 0xd2, 0x6c, 0x73, 0x6f,
0xa4, 0x49, 0xea, 0xe3, 0xa4, 0x58, 0x64, 0x32, 0x6a, 0xd0, 0x22, 0xe3, 0x2d, 0x20, 0x94, 0xad,
0x75, 0x38, 0xce, 0x22, 0xd3, 0xfb, 0x4a, 0xcc, 0xe5, 0x44, 0xda, 0xad, 0xc9, 0xfe, 0x2c, 0xb1,
0x7b, 0x9c, 0xe8, 0x72, 0x52, 0xba, 0xb7, 0xf2, 0x29, 0xf4, 0x0c, 0x4a, 0xfc, 0x97, 0xc4, 0x6f,
0x8f, 0x2f, 0x08, 0x39, 0xf7, 0x6b, 0x13, 0xb0, 0x04, 0xe3, 0x43, 0x58, 0x4a, 0x29, 0x07, 0x13,
0x43, 0xe1, 0xf8, 0xd2, 0x71, 0x92, 0x93, 0x56, 0x01, 0xc5, 0x7f, 0xae, 0x93, 0x78, 0x4c, 0xa9,
0xbf, 0xea, 0xc9, 0x30, 0x45, 0xfc, 0x17, 0x37, 0x89, 0x53, 0xa4, 0xfe, 0x30, 0x67, 0xd2, 0x14,
0x4f, 0x00, 0xfc, 0xa2, 0x2f, 0xf1, 0x3c, 0x62, 0x35, 0xe1, 0x04, 0x96, 0x2b, 0xdf, 0x55, 0xa0,
0xec, 0xbd, 0xfd, 0x7a, 0x03, 0xf9, 0xff, 0x1b, 0x48, 0xc8, 0x3f, 0x85, 0xf9, 0xc8, 0xef, 0x4c,
0x12, 0x8d, 0x27, 0xf9, 0xb7, 0x28, 0x93, 0x4e, 0xe8, 0x19, 0xff, 0x2f, 0x08, 0x22, 0x36, 0xbf,
0x93, 0x96, 0xd4, 0x47, 0xc3, 0xf2, 0x04, 0xc6, 0xbf, 0xde, 0x41, 0xf8, 0x31, 0x40, 0x20, 0xfc,
0x8e, 0xbf, 0x27, 0x27, 0x11, 0x65, 0xd2, 0x6e, 0x19, 0x89, 0x11, 0xf6, 0x46, 0x96, 0x3b, 0xcd,
0x74, 0x1f, 0x99, 0x1e, 0x57, 0x9f, 0x42, 0x2d, 0xf8, 0x04, 0x05, 0x25, 0xfe, 0xe6, 0x3e, 0xfe,
0x46, 0x65, 0xd2, 0x2a, 0xb6, 0x4e, 0xe8, 0x7a, 0x27, 0xb0, 0x73, 0x88, 0x83, 0x8a, 0x76, 0xa9,
0x52, 0x1c, 0x54, 0x4a, 0x6f, 0x2c, 0x31, 0x54, 0xa5, 0xb7, 0xbe, 0x5e, 0x8b, 0xcb, 0x5a, 0xbd,
0xfb, 0x3b, 0xef, 0xf7, 0x75, 0xf7, 0x60, 0xb4, 0x47, 0xbe, 0xdc, 0x61, 0xa8, 0xef, 0xea, 0x16,
0xff, 0xeb, 0x8e, 0xa7, 0xd2, 0x77, 0x28, 0xf5, 0x1d, 0x32, 0xc7, 0x70, 0x6f, 0xaf, 0x44, 0x47,
0x77, 0xff, 0x2f, 0x00, 0x00, 0xff, 0xff, 0xcc, 0x22, 0xe3, 0x1e, 0xab, 0x45, 0x00, 0x00,
0x05, 0x9c, 0x0c, 0xf9, 0x44, 0xfc, 0x0b, 0x3a, 0x03, 0xc5, 0x81, 0xd5, 0x17, 0x17, 0x5c, 0x6c,
0x40, 0x2a, 0xb9, 0xa5, 0x0d, 0xec, 0x2a, 0xb8, 0x67, 0xbd, 0xc0, 0xf6, 0xf1, 0xec, 0x8d, 0xdc,
0x7b, 0x01, 0x35, 0xcf, 0x58, 0x58, 0x0a, 0x02, 0x74, 0xcf, 0x17, 0x42, 0x3e, 0xa9, 0x8b, 0x15,
0xf4, 0x3b, 0x5c, 0x49, 0x7d, 0x61, 0xfc, 0x11, 0x6b, 0x49, 0x87, 0xb7, 0x32, 0x6d, 0xb6, 0xf3,
0x4a, 0xea, 0x15, 0xf9, 0x4f, 0x25, 0x38, 0xbb, 0x81, 0xdd, 0xf5, 0x70, 0x93, 0xe2, 0x4d, 0x73,
0x65, 0x40, 0x3b, 0x89, 0xa9, 0x59, 0xa4, 0xde, 0x86, 0xb2, 0x68, 0xb7, 0xb0, 0x8b, 0x05, 0x31,
0x96, 0x7f, 0x4f, 0x82, 0x16, 0x5f, 0x85, 0xae, 0x49, 0x72, 0xf1, 0x01, 0x76, 0xb1, 0xf6, 0x53,
0x17, 0xdc, 0x3f, 0x48, 0xd0, 0x0c, 0xc6, 0x01, 0xea, 0xca, 0x3f, 0x80, 0x22, 0xed, 0x6b, 0x70,
0x0e, 0x26, 0x2a, 0x2b, 0xc3, 0x26, 0x8e, 0x84, 0x26, 0x7f, 0xbb, 0x22, 0x64, 0xf1, 0xa1, 0x1f,
0x8c, 0xf2, 0x27, 0x0f, 0x46, 0x3c, 0x38, 0x5b, 0x23, 0x32, 0x2f, 0x6b, 0x08, 0xfa, 0x00, 0xf9,
0x37, 0x60, 0xd1, 0xaf, 0x63, 0x18, 0xdd, 0xb4, 0x9a, 0x24, 0xff, 0xb9, 0x04, 0xa7, 0x77, 0x8e,
0xcd, 0x5e, 0x54, 0x27, 0x17, 0xa1, 0x34, 0x1c, 0xa8, 0x7e, 0x83, 0x91, 0x8f, 0x68, 0x66, 0xc1,
0xd6, 0xc6, 0x1a, 0x71, 0x4b, 0x6c, 0xd3, 0x55, 0x01, 0xdb, 0xb5, 0x26, 0x46, 0x8b, 0x6b, 0xa2,
0xf0, 0xc2, 0x1a, 0x73, 0x80, 0xac, 0x6b, 0x53, 0x17, 0x50, 0xe2, 0x00, 0xe5, 0xef, 0x73, 0xd0,
0x0a, 0x6c, 0xf3, 0xa7, 0x8e, 0x79, 0x29, 0x99, 0x7a, 0xfe, 0x15, 0x65, 0xea, 0x85, 0xd9, 0xe3,
0x5c, 0x31, 0x29, 0xce, 0xfd, 0x73, 0x0e, 0x1a, 0xfe, 0xa9, 0x6d, 0x0f, 0x54, 0x33, 0x55, 0x94,
0x3b, 0x22, 0xc7, 0x0b, 0x9f, 0xd3, 0xcf, 0x92, 0x34, 0x35, 0x45, 0x10, 0x4a, 0x64, 0x0a, 0x52,
0x2d, 0xb3, 0x62, 0x8a, 0xf6, 0x3c, 0x78, 0x5e, 0xc9, 0x4c, 0x42, 0x37, 0x30, 0xba, 0x05, 0x88,
0xeb, 0x71, 0x57, 0x37, 0xbb, 0x0e, 0xee, 0x59, 0xa6, 0xc6, 0x34, 0xbc, 0xa8, 0x34, 0xf9, 0x97,
0x8e, 0xb9, 0xc3, 0xe0, 0xe8, 0x03, 0x28, 0xb8, 0xc7, 0x43, 0x16, 0xc1, 0x1a, 0x89, 0x31, 0xc0,
0xe7, 0x6b, 0xf7, 0x78, 0x88, 0x15, 0x8a, 0xee, 0xbd, 0x4c, 0x71, 0x6d, 0xf5, 0x05, 0x4f, 0x07,
0x0a, 0x4a, 0x00, 0x42, 0x6c, 0xd6, 0x3b, 0xc3, 0x39, 0x16, 0x36, 0xf9, 0x50, 0xfe, 0xc7, 0x1c,
0x34, 0xfd, 0x29, 0x15, 0xec, 0x8c, 0x06, 0xe9, 0xa6, 0x30, 0xbe, 0x10, 0x9e, 0x64, 0x05, 0xbf,
0x82, 0x2a, 0x97, 0xe7, 0x09, 0xf4, 0x01, 0x18, 0xc9, 0xe6, 0x18, 0x05, 0x2d, 0xbe, 0x22, 0x05,
0x2d, 0x9d, 0x50, 0x41, 0xe5, 0x6f, 0x25, 0x78, 0x2b, 0xe6, 0x96, 0xc6, 0x1e, 0xe0, 0xf8, 0x74,
0x9d, 0xbb, 0xab, 0xe8, 0x94, 0xdc, 0x43, 0xde, 0x83, 0x92, 0x4d, 0x67, 0xe7, 0x37, 0x17, 0x57,
0xc7, 0x2a, 0x07, 0x63, 0x44, 0xe1, 0x24, 0xf2, 0x9f, 0x48, 0xb0, 0x14, 0x67, 0x75, 0x86, 0xb0,
0xb7, 0x0a, 0x73, 0x6c, 0x6a, 0xcf, 0x86, 0x96, 0xc7, 0xdb, 0x90, 0x7f, 0x38, 0x8a, 0x47, 0x28,
0xef, 0xc0, 0xa2, 0x17, 0x1d, 0xfd, 0x03, 0xde, 0xc2, 0xae, 0x3a, 0x26, 0x59, 0xbd, 0x04, 0x55,
0x96, 0xf5, 0xb0, 0x24, 0x90, 0x95, 0x79, 0xb0, 0x27, 0xba, 0x23, 0xf2, 0xdf, 0x4a, 0x70, 0x86,
0x86, 0x97, 0xe8, 0x55, 0x41, 0x96, 0x6b, 0x24, 0x59, 0x54, 0x91, 0xa4, 0x62, 0x64, 0x5b, 0xab,
0x28, 0x21, 0x58, 0x52, 0xeb, 0x38, 0x3f, 0x65, 0xeb, 0x78, 0x13, 0xde, 0x8a, 0xb0, 0x3a, 0x83,
0x48, 0xc8, 0xce, 0x17, 0x77, 0xc2, 0xef, 0x37, 0xa6, 0xcf, 0xb7, 0x2e, 0x88, 0x4b, 0x86, 0xae,
0xae, 0x45, 0x6d, 0x5d, 0x43, 0x1f, 0x41, 0xc5, 0xc4, 0x47, 0xdd, 0x60, 0xb8, 0xcf, 0xd0, 0x4b,
0x2e, 0x9b, 0xf8, 0x88, 0xfe, 0x25, 0x3f, 0x86, 0xa5, 0x18, 0xab, 0xb3, 0xec, 0xfd, 0x9f, 0x24,
0x38, 0xbb, 0x66, 0x5b, 0xc3, 0x4f, 0x74, 0xdb, 0x1d, 0xa9, 0x83, 0xf0, 0x95, 0xec, 0xeb, 0x69,
0x27, 0x3c, 0x0a, 0x24, 0x7e, 0x4c, 0x01, 0x6e, 0x25, 0x98, 0x40, 0x9c, 0x29, 0xbe, 0xe9, 0x40,
0x9a, 0xf8, 0x5f, 0xf9, 0x24, 0xe6, 0x39, 0xde, 0x84, 0xc0, 0x9f, 0x25, 0x2f, 0x4e, 0xec, 0x3e,
0xe6, 0xa7, 0xed, 0x3e, 0xa6, 0x78, 0xe1, 0xc2, 0x2b, 0xf2, 0xc2, 0x27, 0x2e, 0x87, 0x1f, 0x41,
0xb8, 0x33, 0x4c, 0xc3, 0xdf, 0x54, 0x2d, 0xe5, 0x55, 0x00, 0xbf, 0x4b, 0xca, 0x9f, 0xdf, 0x65,
0x99, 0x26, 0x40, 0x45, 0xa4, 0x25, 0x22, 0x1e, 0xbd, 0x00, 0x09, 0xf5, 0xed, 0x9e, 0x40, 0x3b,
0x49, 0x4b, 0x67, 0xd1, 0xfc, 0xef, 0x73, 0x00, 0x1d, 0xf1, 0x62, 0x73, 0x3a, 0x67, 0x7e, 0x15,
0xea, 0xbe, 0xc2, 0xf8, 0xf6, 0x1e, 0xd4, 0x22, 0x8d, 0x98, 0x84, 0x28, 0xa5, 0x08, 0x4e, 0xac,
0xbc, 0xd2, 0xe8, 0x3c, 0x01, 0xab, 0x61, 0x4a, 0x11, 0xf5, 0x9f, 0xe7, 0xa0, 0x62, 0x5b, 0x47,
0x5d, 0x62, 0x66, 0x9a, 0xf7, 0x24, 0xd5, 0xb6, 0x8e, 0x88, 0xf1, 0x69, 0x68, 0x09, 0xe6, 0x5c,
0xd5, 0x39, 0x24, 0xf3, 0x97, 0x02, 0xaf, 0x02, 0x34, 0x52, 0xc3, 0xef, 0xeb, 0x03, 0xcc, 0x2e,
0xa1, 0x2b, 0x0a, 0x1b, 0xa0, 0x9f, 0x7b, 0x6f, 0xa7, 0xca, 0x99, 0x5f, 0x7e, 0xb0, 0xe7, 0x53,
0x3f, 0x4a, 0x30, 0xef, 0x9f, 0x1a, 0x75, 0x40, 0xc4, 0xa7, 0x51, 0x7f, 0xf6, 0xc0, 0xd2, 0x98,
0xab, 0x68, 0xa4, 0xb8, 0x74, 0x46, 0xc8, 0xbc, 0x96, 0x4f, 0x32, 0xae, 0x12, 0x24, 0xfb, 0x22,
0x9b, 0xd6, 0x35, 0xef, 0x32, 0xb2, 0x64, 0x5b, 0x47, 0x1d, 0x4d, 0x9c, 0x06, 0x7b, 0x6f, 0xca,
0xea, 0x1e, 0x72, 0x1a, 0x0f, 0xe8, 0x93, 0xd3, 0xab, 0x50, 0xc7, 0xb6, 0x6d, 0xd9, 0x5d, 0x03,
0x3b, 0x8e, 0xda, 0xc7, 0x3c, 0x01, 0xae, 0x51, 0xe0, 0x16, 0x83, 0xc9, 0x3f, 0xe4, 0xa1, 0xe1,
0x6f, 0xc5, 0xbb, 0x82, 0xd4, 0x35, 0xef, 0x0a, 0x52, 0x27, 0xa2, 0x03, 0x9b, 0xb9, 0x42, 0x21,
0xdc, 0xd5, 0x5c, 0x4b, 0x52, 0x2a, 0x1c, 0xda, 0xd1, 0x48, 0x5c, 0x25, 0x46, 0x66, 0x5a, 0x1a,
0xf6, 0x85, 0x0b, 0x1e, 0x88, 0xcb, 0x36, 0xa4, 0x23, 0x85, 0x0c, 0x3a, 0x52, 0xcc, 0xa0, 0x23,
0xa5, 0x04, 0x1d, 0x59, 0x84, 0xd2, 0xde, 0xa8, 0x77, 0x88, 0x5d, 0x9e, 0xae, 0xf2, 0x51, 0x58,
0x77, 0xca, 0x11, 0xdd, 0x11, 0x2a, 0x52, 0x09, 0xaa, 0xc8, 0x39, 0xa8, 0xb0, 0xbb, 0xb0, 0xae,
0xeb, 0xd0, 0x8e, 0x7f, 0x5e, 0x29, 0x33, 0xc0, 0xae, 0x83, 0x3e, 0xf4, 0xf2, 0xb1, 0x6a, 0x92,
0xb1, 0x53, 0xaf, 0x13, 0xd1, 0x12, 0x2f, 0x1b, 0xbb, 0x06, 0x0d, 0xfa, 0xb2, 0xff, 0xf9, 0x08,
0xdb, 0xc7, 0xea, 0xde, 0x00, 0xb7, 0x6a, 0x94, 0x9d, 0x3a, 0x81, 0x3e, 0xf1, 0x80, 0xe4, 0x40,
0x28, 0x9a, 0x6e, 0x6a, 0xf8, 0x25, 0xd6, 0x5a, 0x75, 0x76, 0x57, 0x46, 0x60, 0x1d, 0x06, 0x92,
0x3f, 0x03, 0xe4, 0xaf, 0x31, 0x5b, 0x52, 0x16, 0x11, 0x62, 0x2e, 0x2a, 0x44, 0xf9, 0xef, 0x24,
0x58, 0x08, 0x2e, 0x36, 0x6d, 0x78, 0xfc, 0x08, 0xaa, 0xec, 0x66, 0xa4, 0x4b, 0xcc, 0x93, 0x77,
0x23, 0x2e, 0x8c, 0x3d, 0x3d, 0x05, 0xfc, 0x77, 0xe5, 0x44, 0x09, 0x8e, 0x2c, 0xfb, 0x50, 0x37,
0xfb, 0x5d, 0xc2, 0x99, 0x67, 0x14, 0x35, 0x0e, 0x7c, 0x4c, 0x60, 0xf2, 0xd7, 0x12, 0x5c, 0x7c,
0x3a, 0xd4, 0x54, 0x17, 0x07, 0xf2, 0x84, 0x59, 0x9f, 0xaa, 0x7d, 0xe0, 0xbd, 0x15, 0xcb, 0x65,
0xeb, 0xee, 0x33, 0x6c, 0x79, 0x0b, 0xce, 0x2a, 0xd8, 0xc1, 0xa6, 0x16, 0xfa, 0x38, 0x75, 0x0f,
0x62, 0x08, 0xed, 0xa4, 0xe9, 0x66, 0x91, 0x3d, 0x4b, 0xd8, 0xba, 0x36, 0x99, 0xd6, 0xe5, 0xfe,
0x87, 0xe4, 0x09, 0x74, 0x1d, 0x57, 0xfe, 0x6f, 0x09, 0x16, 0xee, 0x6b, 0xde, 0x7a, 0xaf, 0x2d,
0x2f, 0x8c, 0xe6, 0x4d, 0xf9, 0x78, 0xde, 0xf4, 0xaa, 0x1c, 0x09, 0x77, 0xa9, 0xe6, 0xc8, 0xf0,
0x42, 0x85, 0x4d, 0x9f, 0x22, 0xc8, 0xfb, 0xe2, 0xf2, 0x59, 0xc1, 0xfb, 0xd8, 0xc6, 0x66, 0x0f,
0x6f, 0x5a, 0xbd, 0xc3, 0xc0, 0x83, 0x33, 0x29, 0xf8, 0xe0, 0x6c, 0xda, 0x07, 0x6c, 0x37, 0xff,
0x52, 0x82, 0x85, 0x58, 0x3f, 0x0b, 0x35, 0x00, 0x9e, 0x9a, 0x3d, 0xde, 0xe8, 0x6b, 0x9e, 0x42,
0x35, 0x28, 0x7b, 0x6d, 0xbf, 0xa6, 0x84, 0xaa, 0x30, 0xb7, 0x6b, 0x51, 0xec, 0x66, 0x0e, 0x35,
0xa1, 0xc6, 0x08, 0x47, 0xbd, 0x1e, 0x76, 0x9c, 0x66, 0x5e, 0x40, 0xd6, 0x55, 0x7d, 0x30, 0xb2,
0x71, 0xb3, 0x80, 0xea, 0x50, 0xd9, 0xb5, 0xf8, 0x73, 0xbd, 0x66, 0x11, 0x21, 0x68, 0x78, 0x6f,
0xf7, 0x38, 0x51, 0x29, 0x00, 0xf3, 0xc8, 0xe6, 0x6e, 0x3e, 0x0b, 0xf6, 0x44, 0x76, 0x8f, 0x87,
0x18, 0x2d, 0xc1, 0xe9, 0xa7, 0xa6, 0x86, 0xf7, 0x75, 0x13, 0x6b, 0xfe, 0xa7, 0xe6, 0x29, 0x74,
0x1a, 0xe6, 0xb7, 0xb0, 0xdd, 0xc7, 0x01, 0x60, 0x0e, 0x2d, 0x40, 0x7d, 0x4b, 0x7f, 0x19, 0x00,
0xe5, 0xe5, 0x42, 0x59, 0x6a, 0x4a, 0x2b, 0xff, 0xb3, 0x04, 0x15, 0x52, 0xcf, 0x3c, 0xb0, 0x2c,
0x5b, 0x43, 0x43, 0x40, 0xf4, 0x75, 0xab, 0x31, 0xb4, 0x4c, 0xf1, 0x66, 0x1c, 0xbd, 0x97, 0x92,
0x33, 0xc5, 0x51, 0xb9, 0x1e, 0xb6, 0xaf, 0xa7, 0x50, 0x44, 0xd0, 0xe5, 0x53, 0xc8, 0xa0, 0x2b,
0xee, 0xea, 0x06, 0xde, 0xd5, 0x7b, 0x87, 0xde, 0xb3, 0x97, 0x31, 0x2b, 0x46, 0x50, 0xbd, 0x15,
0x23, 0xd5, 0x33, 0x1f, 0xb0, 0x27, 0xc8, 0x9e, 0x21, 0xca, 0xa7, 0xd0, 0x73, 0x38, 0xb3, 0x81,
0x03, 0x8e, 0xc7, 0x5b, 0x70, 0x25, 0x7d, 0xc1, 0x18, 0xf2, 0x09, 0x97, 0xdc, 0x84, 0x22, 0x6d,
0x16, 0xa3, 0x24, 0xdf, 0x14, 0xfc, 0x89, 0x57, 0xfb, 0x72, 0x3a, 0x82, 0x98, 0xed, 0x33, 0x98,
0x8f, 0xfc, 0x30, 0x04, 0xdd, 0x48, 0x20, 0x4b, 0xfe, 0x89, 0x4f, 0xfb, 0x66, 0x16, 0x54, 0xb1,
0x56, 0x1f, 0x1a, 0xe1, 0x97, 0xb1, 0x28, 0xa9, 0x29, 0x90, 0xf8, 0xa6, 0xbf, 0x7d, 0x23, 0x03,
0xa6, 0x58, 0xc8, 0x80, 0x66, 0xf4, 0x87, 0x0a, 0xe8, 0xe6, 0xd8, 0x09, 0xc2, 0xea, 0xf6, 0xb3,
0x4c, 0xb8, 0x62, 0xb9, 0x63, 0xaa, 0x04, 0xb1, 0xb7, 0xef, 0xe8, 0x76, 0xf2, 0x34, 0x69, 0x8f,
0xf2, 0xdb, 0x77, 0x32, 0xe3, 0x8b, 0xa5, 0x7f, 0x97, 0x5d, 0x52, 0x25, 0xbd, 0x1f, 0x47, 0xef,
0x27, 0x4f, 0x37, 0xe6, 0xe1, 0x7b, 0x7b, 0xe5, 0x24, 0x24, 0x82, 0x89, 0x2f, 0xe8, 0xed, 0x52,
0xc2, 0x0b, 0xec, 0xa8, 0xdd, 0x79, 0xf3, 0xa5, 0x3f, 0x2e, 0x6f, 0xbf, 0x7f, 0x02, 0x0a, 0xc1,
0x80, 0x15, 0xfd, 0x25, 0x88, 0x67, 0x86, 0x77, 0x26, 0x6a, 0xcd, 0x74, 0x36, 0xf8, 0x29, 0xcc,
0x47, 0x5e, 0x0f, 0x25, 0x5a, 0x4d, 0xf2, 0x0b, 0xa3, 0xf6, 0xb8, 0x78, 0xcd, 0x4c, 0x32, 0x72,
0x59, 0x87, 0x52, 0xb4, 0x3f, 0xe1, 0x42, 0xaf, 0x7d, 0x33, 0x0b, 0xaa, 0xd8, 0x88, 0x43, 0xdd,
0x65, 0xe4, 0xc2, 0x0b, 0xdd, 0x4a, 0x9e, 0x23, 0xf9, 0xb2, 0xae, 0xfd, 0x6e, 0x46, 0x6c, 0xb1,
0xe8, 0xef, 0x00, 0xda, 0x39, 0x20, 0x35, 0x8c, 0xb9, 0xaf, 0xf7, 0x47, 0xb6, 0xca, 0x9e, 0x08,
0xa5, 0xf9, 0xe8, 0x38, 0x6a, 0x8a, 0xae, 0x8c, 0xa5, 0x10, 0x8b, 0x77, 0x01, 0x36, 0xb0, 0xbb,
0x85, 0x5d, 0x9b, 0x28, 0xe8, 0xf5, 0x44, 0x79, 0xfb, 0x08, 0xde, 0x52, 0xef, 0x4c, 0xc4, 0x0b,
0x84, 0x84, 0xe6, 0x96, 0x6a, 0x92, 0xf2, 0xdd, 0x7f, 0x59, 0x77, 0x2b, 0x91, 0x3c, 0x8a, 0x96,
0x72, 0xa0, 0xa9, 0xd8, 0x62, 0xc9, 0x23, 0x11, 0x66, 0x03, 0xdd, 0xd4, 0xa8, 0xfb, 0xf1, 0x79,
0x4e, 0xbe, 0x2a, 0x8b, 0xba, 0x9f, 0x31, 0xf8, 0x62, 0xe1, 0x2f, 0x25, 0xfa, 0x33, 0xa3, 0x08,
0xc2, 0x33, 0xdd, 0x3d, 0xd8, 0x1e, 0xa8, 0xa6, 0x93, 0x85, 0x05, 0x8a, 0x78, 0x02, 0x16, 0x38,
0xbe, 0x60, 0x41, 0x83, 0x7a, 0xa8, 0x47, 0x8a, 0x92, 0xde, 0xa8, 0x25, 0x35, 0x7c, 0xdb, 0xcb,
0x93, 0x11, 0xc5, 0x2a, 0x07, 0x50, 0xf7, 0x54, 0x9a, 0x1d, 0xee, 0x8d, 0x34, 0x4e, 0x7d, 0x9c,
0x14, 0x8b, 0x4c, 0x46, 0x0d, 0x5a, 0x64, 0xbc, 0x05, 0x84, 0xb2, 0xb5, 0x0e, 0xc7, 0x59, 0x64,
0x7a, 0x5f, 0x89, 0xb9, 0x9c, 0x48, 0xbb, 0x35, 0xd9, 0x9f, 0x25, 0x76, 0x8f, 0x13, 0x5d, 0x4e,
0x4a, 0xf7, 0x56, 0x3e, 0x85, 0x9e, 0x41, 0x89, 0xff, 0xbe, 0xf8, 0xed, 0xf1, 0x05, 0x21, 0x9f,
0xfd, 0xda, 0x04, 0x2c, 0x31, 0xf1, 0x21, 0x2c, 0xa5, 0x94, 0x83, 0x89, 0xa1, 0x70, 0x7c, 0xe9,
0x38, 0xc9, 0x49, 0xab, 0x80, 0xe2, 0x3f, 0xe2, 0x49, 0x14, 0x53, 0xea, 0x6f, 0x7d, 0x32, 0x2c,
0x11, 0xff, 0x1d, 0x4e, 0xe2, 0x12, 0xa9, 0x3f, 0xd7, 0x99, 0xb4, 0xc4, 0x13, 0x00, 0xbf, 0xe8,
0x4b, 0x94, 0x47, 0xac, 0x26, 0x9c, 0x30, 0xe5, 0xca, 0x77, 0x15, 0x28, 0x7b, 0x2f, 0xc2, 0xde,
0x40, 0xfe, 0xff, 0x06, 0x12, 0xf2, 0x4f, 0x61, 0x3e, 0xf2, 0xeb, 0x93, 0x44, 0xe3, 0x49, 0xfe,
0x85, 0xca, 0x24, 0x09, 0x3d, 0xe3, 0xff, 0x1b, 0x41, 0xc4, 0xe6, 0x77, 0xd2, 0x92, 0xfa, 0x68,
0x58, 0x9e, 0x30, 0xf1, 0xff, 0xef, 0x20, 0xfc, 0x18, 0x20, 0x10, 0x7e, 0xc7, 0xdf, 0x93, 0x93,
0x88, 0x32, 0xe9, 0xb4, 0x8c, 0xc4, 0x08, 0x7b, 0x23, 0xcb, 0x9d, 0x66, 0xba, 0x8f, 0x4c, 0x8f,
0xab, 0x4f, 0xa1, 0x16, 0x7c, 0x82, 0x82, 0x12, 0x7f, 0x89, 0x1f, 0x7f, 0xa3, 0x32, 0x69, 0x17,
0x5b, 0x27, 0x74, 0xbd, 0x13, 0xa6, 0x73, 0x88, 0x83, 0x8a, 0x76, 0xa9, 0x52, 0x1c, 0x54, 0x4a,
0x6f, 0x2c, 0x31, 0x54, 0xa5, 0xb7, 0xbe, 0x5e, 0x8b, 0xcb, 0x5a, 0xbd, 0xfb, 0x5b, 0xef, 0xf7,
0x75, 0xf7, 0x60, 0xb4, 0x47, 0xbe, 0xdc, 0x61, 0xa8, 0xef, 0xea, 0x16, 0xff, 0xeb, 0x8e, 0xa7,
0xd2, 0x77, 0x28, 0xf5, 0x1d, 0xb2, 0xc6, 0x70, 0x6f, 0xaf, 0x44, 0x47, 0x77, 0xff, 0x37, 0x00,
0x00, 0xff, 0xff, 0xda, 0x68, 0xef, 0x1f, 0xc1, 0x45, 0x00, 0x00,
}
// Reference imports to suppress errors if they are not otherwise used.

View File

@ -19,6 +19,7 @@ package etcd
import (
"crypto/tls"
"crypto/x509"
"fmt"
"io/ioutil"
"time"
@ -28,6 +29,10 @@ import (
clientv3 "go.etcd.io/etcd/client/v3"
)
var (
maxTxnNum = 64
)
// GetEtcdClient returns etcd client
func GetEtcdClient(cfg *paramtable.EtcdConfig) (*clientv3.Client, error) {
if cfg.UseEmbedEtcd {
@ -88,3 +93,68 @@ func GetRemoteEtcdSSLClient(endpoints []string, certFile string, keyFile string,
return clientv3.New(cfg)
}
func min(a, b int) int {
if a < b {
return a
}
return b
}
//SaveByBatch there will not guarantee atomicity
func SaveByBatch(kvs map[string]string, op func(partialKvs map[string]string) error) error {
if len(kvs) == 0 {
return nil
}
keys := make([]string, 0, len(kvs))
values := make([]string, 0, len(kvs))
for k, v := range kvs {
keys = append(keys, k)
values = append(values, v)
}
for i := 0; i < len(kvs); i = i + maxTxnNum {
end := min(i+maxTxnNum, len(keys))
batch, err := buildKvGroup(keys[i:end], values[i:end])
if err != nil {
return err
}
if err := op(batch); err != nil {
return err
}
}
return nil
}
func RemoveByBatch(removals []string, op func(partialKeys []string) error) error {
if len(removals) == 0 {
return nil
}
for i := 0; i < len(removals); i = i + maxTxnNum {
end := min(i+maxTxnNum, len(removals))
batch := removals[i:end]
if err := op(batch); err != nil {
return err
}
}
return nil
}
func buildKvGroup(keys, values []string) (map[string]string, error) {
if len(keys) != len(values) {
return nil, fmt.Errorf("length of keys (%d) and values (%d) are not equal", len(keys), len(values))
}
ret := make(map[string]string, len(keys))
for i, k := range keys {
_, ok := ret[k]
if ok {
return nil, fmt.Errorf("duplicated key was found: %s", k)
}
ret[k] = values[i]
}
return ret, nil
}

View File

@ -18,6 +18,7 @@ package etcd
import (
"context"
"errors"
"os"
"path"
"testing"
@ -72,3 +73,163 @@ func TestEtcd(t *testing.T) {
assert.NotNil(t, err)
}
func Test_buildKvGroup(t *testing.T) {
t.Run("length not equal", func(t *testing.T) {
keys := []string{"k1", "k2"}
values := []string{"v1"}
_, err := buildKvGroup(keys, values)
assert.Error(t, err)
})
t.Run("duplicate", func(t *testing.T) {
keys := []string{"k1", "k1"}
values := []string{"v1", "v2"}
_, err := buildKvGroup(keys, values)
assert.Error(t, err)
})
t.Run("normal case", func(t *testing.T) {
keys := []string{"k1", "k2"}
values := []string{"v1", "v2"}
kvs, err := buildKvGroup(keys, values)
assert.NoError(t, err)
for i, k := range keys {
v, ok := kvs[k]
assert.True(t, ok)
assert.Equal(t, values[i], v)
}
})
}
func Test_SaveByBatch(t *testing.T) {
t.Run("empty kvs", func(t *testing.T) {
kvs := map[string]string{}
group := 0
count := 0
saveFn := func(partialKvs map[string]string) error {
group++
count += len(partialKvs)
return nil
}
maxTxnNum = 2
err := SaveByBatch(kvs, saveFn)
assert.NoError(t, err)
assert.Equal(t, 0, group)
assert.Equal(t, 0, count)
})
t.Run("normal case", func(t *testing.T) {
kvs := map[string]string{
"k1": "v1",
"k2": "v2",
"k3": "v3",
}
group := 0
count := 0
saveFn := func(partialKvs map[string]string) error {
group++
count += len(partialKvs)
return nil
}
maxTxnNum = 2
err := SaveByBatch(kvs, saveFn)
assert.NoError(t, err)
assert.Equal(t, 2, group)
assert.Equal(t, 3, count)
})
t.Run("multi save failed", func(t *testing.T) {
saveFn := func(partialKvs map[string]string) error {
return errors.New("mock")
}
kvs := map[string]string{
"k1": "v1",
"k2": "v2",
"k3": "v3",
}
maxTxnNum = 2
err := SaveByBatch(kvs, saveFn)
assert.Error(t, err)
})
}
func Test_RemoveByBatch(t *testing.T) {
t.Run("empty kvs case", func(t *testing.T) {
var kvs []string
group := 0
count := 0
removeFn := func(partialKvs []string) error {
group++
count += len(partialKvs)
return nil
}
maxTxnNum = 2
err := RemoveByBatch(kvs, removeFn)
assert.NoError(t, err)
assert.Equal(t, 0, group)
assert.Equal(t, 0, count)
})
t.Run("normal case", func(t *testing.T) {
kvs := []string{"k1", "k2", "k3", "k4", "k5"}
group := 0
count := 0
removeFn := func(partialKvs []string) error {
group++
count += len(partialKvs)
return nil
}
maxTxnNum = 2
err := RemoveByBatch(kvs, removeFn)
assert.NoError(t, err)
assert.Equal(t, 3, group)
assert.Equal(t, 5, count)
})
t.Run("multi remove failed", func(t *testing.T) {
removeFn := func(partialKvs []string) error {
return errors.New("mock")
}
kvs := []string{"k1", "k2", "k3", "k4", "k5"}
maxTxnNum = 2
err := RemoveByBatch(kvs, removeFn)
assert.Error(t, err)
})
}
func Test_min(t *testing.T) {
type args struct {
a int
b int
}
tests := []struct {
name string
args args
want int
}{
{
args: args{a: 1, b: 2},
want: 1,
},
{
args: args{a: 4, b: 3},
want: 3,
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
if got := min(tt.args.a, tt.args.b); got != tt.want {
t.Errorf("min() = %v, want %v", got, tt.want)
}
})
}
}

View File

@ -0,0 +1,34 @@
package metautil
import (
"path"
"strconv"
"github.com/milvus-io/milvus/internal/util/typeutil"
"github.com/milvus-io/milvus/internal/common"
)
func BuildInsertLogPath(rootPath string, collectionID, partitionID, segmentID, fieldID, logID typeutil.UniqueID) string {
k := JoinIDPath(collectionID, partitionID, segmentID, fieldID, logID)
return path.Join(rootPath, common.SegmentInsertLogPath, k)
}
func BuildStatsLogPath(rootPath string, collectionID, partitionID, segmentID, fieldID, logID typeutil.UniqueID) string {
k := JoinIDPath(collectionID, partitionID, segmentID, fieldID, logID)
return path.Join(rootPath, common.SegmentStatslogPath, k)
}
func BuildDeltaLogPath(rootPath string, collectionID, partitionID, segmentID, logID typeutil.UniqueID) string {
k := JoinIDPath(collectionID, partitionID, segmentID, logID)
return path.Join(rootPath, common.SegmentDeltaLogPath, k)
}
// JoinIDPath joins ids to path format.
func JoinIDPath(ids ...typeutil.UniqueID) string {
idStr := make([]string, 0, len(ids))
for _, id := range ids {
idStr = append(idStr, strconv.FormatInt(id, 10))
}
return path.Join(idStr...)
}