diff --git a/pkg/common/moerr/error.go b/pkg/common/moerr/error.go index d4680dd03678..e5fde6faf16b 100644 --- a/pkg/common/moerr/error.go +++ b/pkg/common/moerr/error.go @@ -86,6 +86,7 @@ const ( ErrWrongDatetimeSpec uint16 = 20310 ErrUpgrateError uint16 = 20311 ErrInvalidTz uint16 = 20312 + ErrUnsupportedDML uint16 = 20313 // Group 4: unexpected state and io errors ErrInvalidState uint16 = 20400 @@ -341,6 +342,7 @@ var errorMsgRefer = map[uint16]moErrorMsgItem{ ErrBadFieldError: {ER_BAD_FIELD_ERROR, []string{MySQLDefaultSqlState}, "Unknown column '%s' in '%s'"}, ErrWrongDatetimeSpec: {ER_WRONG_DATETIME_SPEC, []string{MySQLDefaultSqlState}, "wrong date/time format specifier: %s"}, ErrUpgrateError: {ER_UNKNOWN_ERROR, []string{MySQLDefaultSqlState}, "CN upgrade table or view '%s.%s' under tenant '%s:%d' reports error: %s"}, + ErrUnsupportedDML: {ER_UNKNOWN_ERROR, []string{MySQLDefaultSqlState}, "unsupported DML: %s"}, // Group 4: unexpected state or file io error ErrInvalidState: {ER_UNKNOWN_ERROR, []string{MySQLDefaultSqlState}, "invalid state %s"}, @@ -854,6 +856,11 @@ func NewConstraintViolation(ctx context.Context, msg string) *Error { return newError(ctx, ErrConstraintViolation, msg) } +func NewUnsupportedDML(ctx context.Context, msg string, args ...any) *Error { + xmsg := fmt.Sprintf(msg, args...) + return newError(ctx, ErrUnsupportedDML, xmsg) +} + func NewEmptyVector(ctx context.Context) *Error { return newError(ctx, ErrEmptyVector) } diff --git a/pkg/container/batch/compact_batchs.go b/pkg/container/batch/compact_batchs.go index 8ac8ec57db66..ea2d7385c440 100644 --- a/pkg/container/batch/compact_batchs.go +++ b/pkg/container/batch/compact_batchs.go @@ -15,6 +15,8 @@ package batch import ( + "context" + "github.com/matrixorigin/matrixone/pkg/common/mpool" "github.com/matrixorigin/matrixone/pkg/container/vector" ) @@ -49,11 +51,49 @@ func (bats *CompactBatchs) Get(idx int) *Batch { return bats.batchs[idx] } -// Push push one batch to CompactBatchs -// CompactBatchs donot obtain ownership of inBatch +// Push append inBatch to CompactBatchs. +// CompactBatchs will obtain ownership of inBatch func (bats *CompactBatchs) Push(mpool *mpool.MPool, inBatch *Batch) error { batLen := bats.Length() var err error + + // empty input + if inBatch.rowCount == 0 { + return nil + } + + // empty bats + if batLen == 0 { + bats.batchs = append(bats.batchs, inBatch) + return nil + } + + // fast path 1 + lastBatRowCount := bats.batchs[batLen-1].rowCount + if lastBatRowCount == 0 { + bats.batchs = append(bats.batchs, inBatch) + return nil + } + + defer func() { + inBatch.Clean(mpool) + }() + + // fast path 2 + if lastBatRowCount+inBatch.RowCount() <= DefaultBatchMaxRow { + bats.batchs[batLen-1], err = bats.batchs[batLen-1].Append(context.TODO(), mpool, inBatch) + return err + } + + // slow path + return bats.fillData(mpool, inBatch) +} + +// Extend extend one batch'data to CompactBatchs +// CompactBatchs donot obtain ownership of inBatch +func (bats *CompactBatchs) Extend(mpool *mpool.MPool, inBatch *Batch) error { + batLen := bats.Length() + var err error var tmpBat *Batch // empty input @@ -71,12 +111,40 @@ func (bats *CompactBatchs) Push(mpool *mpool.MPool, inBatch *Batch) error { return nil } + return bats.fillData(mpool, inBatch) +} + +func (bats *CompactBatchs) RowCount() int { + rowCount := 0 + for _, bat := range bats.batchs { + rowCount += bat.rowCount + } + return rowCount +} + +func (bats *CompactBatchs) Clean(mpool *mpool.MPool) { + for _, bat := range bats.batchs { + bat.Clean(mpool) + } + bats.batchs = nil +} + +func (bats *CompactBatchs) TakeBatchs() []*Batch { + batchs := bats.batchs + bats.batchs = nil + return batchs +} + +func (bats *CompactBatchs) fillData(mpool *mpool.MPool, inBatch *Batch) error { + batLen := bats.Length() + var tmpBat *Batch + var err error + if len(bats.ufs) == 0 { for i := 0; i < inBatch.VectorCount(); i++ { typ := *inBatch.GetVector(int32(i)).GetType() bats.ufs = append(bats.ufs, vector.GetUnionAllFunction(typ, mpool)) } - } //fill data @@ -124,24 +192,3 @@ func (bats *CompactBatchs) Push(mpool *mpool.MPool, inBatch *Batch) error { return nil } - -func (bats *CompactBatchs) RowCount() int { - rowCount := 0 - for _, bat := range bats.batchs { - rowCount += bat.rowCount - } - return rowCount -} - -func (bats *CompactBatchs) Clean(mpool *mpool.MPool) { - for _, bat := range bats.batchs { - bat.Clean(mpool) - } - bats.batchs = nil -} - -func (bats *CompactBatchs) TakeBatchs() []*Batch { - batchs := bats.batchs - bats.batchs = nil - return batchs -} diff --git a/pkg/container/batch/compact_batchs_test.go b/pkg/container/batch/compact_batchs_test.go index 064876b9e035..257769a8e099 100644 --- a/pkg/container/batch/compact_batchs_test.go +++ b/pkg/container/batch/compact_batchs_test.go @@ -23,7 +23,7 @@ import ( "github.com/stretchr/testify/require" ) -func TestCompactBatchsFo(t *testing.T) { +func TestCompactBatchsPush(t *testing.T) { var err error var bat1, bat2 *Batch mp := mpool.MustNewZero() @@ -32,7 +32,6 @@ func TestCompactBatchsFo(t *testing.T) { //empty input bat1 = NewWithSize(1) err = bats.Push(mp, bat1) - bat1.Clean(mp) require.NoError(t, err) require.Nil(t, bats.Get(0)) bats.Clean(mp) @@ -41,7 +40,6 @@ func TestCompactBatchsFo(t *testing.T) { //simple test bat1 = makeTestBatch(10, mp) err = bats.Push(mp, bat1) - bat1.Clean(mp) require.NoError(t, err) require.Equal(t, 1, bats.Length()) require.Equal(t, 10, bats.RowCount()) @@ -55,8 +53,6 @@ func TestCompactBatchsFo(t *testing.T) { _ = bats.Push(mp, bat1) err = bats.Push(mp, bat2) require.NoError(t, err) - bat1.Clean(mp) - bat2.Clean(mp) require.Equal(t, 1, bats.Length()) require.Equal(t, 20, bats.RowCount()) require.Equal(t, 20, bats.Get(0).rowCount) @@ -70,8 +66,6 @@ func TestCompactBatchsFo(t *testing.T) { _ = bats.Push(mp, bat1) err = bats.Push(mp, bat2) require.NoError(t, err) - bat1.Clean(mp) - bat2.Clean(mp) require.Equal(t, 2, bats.Length()) require.Equal(t, 8195, bats.RowCount()) require.Equal(t, 8192, bats.Get(0).rowCount) @@ -86,6 +80,78 @@ func TestCompactBatchsFo(t *testing.T) { _ = bats.Push(mp, bat1) err = bats.Push(mp, bat2) require.NoError(t, err) + require.Equal(t, 3, bats.Length()) + require.Equal(t, 8192*2+1+3, bats.RowCount()) + require.Equal(t, 8192, bats.Get(0).rowCount) + require.Equal(t, 8192, bats.Get(1).rowCount) + require.Equal(t, 4, bats.Get(2).rowCount) + bats.Clean(mp) + require.Equal(t, int64(0), mp.CurrNB()) +} + +func TestCompactBatchsExtend(t *testing.T) { + var err error + var bat1, bat2 *Batch + mp := mpool.MustNewZero() + bats := NewCompactBatchs() + + //empty input + bat1 = NewWithSize(1) + err = bats.Extend(mp, bat1) + bat1.Clean(mp) + require.NoError(t, err) + require.Nil(t, bats.Get(0)) + bats.Clean(mp) + require.Equal(t, int64(0), mp.CurrNB()) + + //simple test + bat1 = makeTestBatch(10, mp) + err = bats.Extend(mp, bat1) + bat1.Clean(mp) + require.NoError(t, err) + require.Equal(t, 1, bats.Length()) + require.Equal(t, 10, bats.RowCount()) + require.Equal(t, 10, bats.Get(0).rowCount) + bats.Clean(mp) + require.Equal(t, int64(0), mp.CurrNB()) + + // bat1.rowCount + bat2.rowCount < DefaultBatchMaxRow + bat1 = makeTestBatch(10, mp) + bat2 = makeTestBatch(10, mp) + _ = bats.Extend(mp, bat1) + err = bats.Extend(mp, bat2) + require.NoError(t, err) + bat1.Clean(mp) + bat2.Clean(mp) + require.Equal(t, 1, bats.Length()) + require.Equal(t, 20, bats.RowCount()) + require.Equal(t, 20, bats.Get(0).rowCount) + bats.Clean(mp) + require.Equal(t, int64(0), mp.CurrNB()) + + // bat1.rowCount + bat2.rowCount > DefaultBatchMaxRow + // but bat1.rowCount + bat2.rowCount - DefaultBatchMaxRow < DefaultBatchMaxRow + bat1 = makeTestBatch(3, mp) + bat2 = makeTestBatch(8192, mp) + _ = bats.Extend(mp, bat1) + err = bats.Extend(mp, bat2) + require.NoError(t, err) + bat1.Clean(mp) + bat2.Clean(mp) + require.Equal(t, 2, bats.Length()) + require.Equal(t, 8195, bats.RowCount()) + require.Equal(t, 8192, bats.Get(0).rowCount) + require.Equal(t, 3, bats.Get(1).rowCount) + bats.Clean(mp) + require.Equal(t, int64(0), mp.CurrNB()) + + // bat1.rowCount + bat2.rowCount > DefaultBatchMaxRow + // but bat1.rowCount + bat2.rowCount - DefaultBatchMaxRow > DefaultBatchMaxRow + bat1 = makeTestBatch(3, mp) + bat2 = makeTestBatch(8192*2+1, mp) + _ = bats.Extend(mp, bat1) + err = bats.Extend(mp, bat2) + require.NoError(t, err) bat1.Clean(mp) bat2.Clean(mp) require.Equal(t, 3, bats.Length()) diff --git a/pkg/container/types/tuple.go b/pkg/container/types/tuple.go index 426c28405041..551e30da87e7 100644 --- a/pkg/container/types/tuple.go +++ b/pkg/container/types/tuple.go @@ -33,9 +33,11 @@ import ( "fmt" "strconv" "strings" + "time" "unsafe" "github.com/matrixorigin/matrixone/pkg/common/moerr" + "github.com/matrixorigin/matrixone/pkg/pb/plan" ) /* @@ -544,3 +546,222 @@ func UnpackWithSchema(b []byte) (Tuple, []T, error) { t, _, schema, err := decodeTuple(b) return t, schema, err } + +func StringifyTuple(b []byte, types []plan.Type) ([]string, error) { + items := make([]string, len(types)) + + offset := 0 + for i := 0; i < len(items) && offset < len(b); i++ { + var ( + item string + itemLen int + ) + + switch { + case b[offset] == nilCode: + item = "null" + itemLen = 1 + case b[offset] == int8Code: + item, itemLen = stringifyInt(int8Code, b[offset+1:], types[i].Scale) + itemLen += 1 + case b[offset] == int16Code: + item, itemLen = stringifyInt(int16Code, b[offset+1:], types[i].Scale) + itemLen += 1 + case b[offset] == int32Code: + item, itemLen = stringifyInt(int32Code, b[offset+1:], types[i].Scale) + itemLen += 1 + case b[offset] == int64Code: + item, itemLen = stringifyInt(int64Code, b[offset+1:], types[i].Scale) + itemLen += 1 + case b[offset] == uint8Code: + item, itemLen = stringifyUint(uint8Code, b[offset+1:]) + itemLen += 1 + case b[offset] == uint16Code: + item, itemLen = stringifyUint(uint16Code, b[offset+1:]) + itemLen += 1 + case b[offset] == uint32Code: + item, itemLen = stringifyUint(uint32Code, b[offset+1:]) + itemLen += 1 + case b[offset] == uint64Code: + item, itemLen = stringifyUint(uint64Code, b[offset+1:]) + itemLen += 1 + case b[offset] == trueCode: + item = "true" + itemLen = 1 + case b[offset] == falseCode: + item = "false" + itemLen = 1 + case b[offset] == float32Code: + item, itemLen = stringifyFloat32(b[offset:]) + case b[offset] == float64Code: + item, itemLen = stringifyFloat64(b[offset:]) + case b[offset] == dateCode: + item, itemLen = stringifyInt(dateCode, b[offset+1:], types[i].Scale) + itemLen += 1 + case b[offset] == datetimeCode: + item, itemLen = stringifyInt(datetimeCode, b[offset+1:], types[i].Scale) + itemLen += 1 + case b[offset] == timestampCode: + item, itemLen = stringifyInt(timestampCode, b[offset+1:], types[i].Scale) + itemLen += 1 + case b[offset] == timeCode: + item, itemLen = stringifyInt(timeCode, b[offset+1:], types[i].Scale) + itemLen += 1 + case b[offset] == decimal64Code: + item, itemLen = stringifyDecimal64(b[offset+1:], types[i].Scale) + case b[offset] == decimal128Code: + item, itemLen = stringifyDecimal128(b[offset+1:], types[i].Scale) + case b[offset] == stringTypeCode: + item, itemLen = stringifyBytes(b[offset+1:]) + itemLen += 1 + case b[offset] == bitCode: + item, itemLen = stringifyUint(uint64Code, b[offset+1:]) + itemLen += 1 + case b[offset] == enumCode: + // TODO: need to verify @YANGGMM + item, itemLen = stringifyUint(uint16Code, b[offset+1:]) + itemLen += 1 + case b[offset] == uuidCode: + item, itemLen = stringifyUuid(b[offset:]) + // off += 1 + default: + return nil, moerr.NewInternalErrorNoCtxf("unable to decode tuple element with unknown typecode %02x", b[offset]) + } + + items[i] = item + offset += itemLen + } + + return items, nil +} + +func stringifyBytes(b []byte) (string, int) { + idx := findTerminator(b[1:]) + return string(bytes.ReplaceAll(b[1:idx+1], []byte{0x00, 0xFF}, []byte{0x00})), idx + 2 +} + +func stringifyInt(code byte, b []byte, scale int32) (string, int) { + loc := time.Local + if b[0] == intZeroCode { + switch code { + case dateCode: + return Date(0).String(), 1 + case timeCode: + return Time(0).String2(scale), 1 + case datetimeCode: + return Datetime(0).String2(scale), 1 + case timestampCode: + return Timestamp(0).String2(loc, scale), 1 + default: + return "0", 1 + } + } + + var neg bool + + n := int(b[0]) - intZeroCode + if n < 0 { + n = -n + neg = true + } + + bp := make([]byte, 8) + copy(bp[8-n:], b[1:n+1]) + + var ret int64 + binary.Read(bytes.NewBuffer(bp), binary.BigEndian, &ret) + + if neg { + switch code { + case dateCode: + return Date(ret - int64(sizeLimits[n])).String(), n + 1 + case timeCode: + return Time(ret - int64(sizeLimits[n])).String2(scale), n + 1 + case datetimeCode: + return Datetime(ret - int64(sizeLimits[n])).String2(scale), n + 1 + case timestampCode: + return Timestamp(ret-int64(sizeLimits[n])).String2(loc, scale), n + 1 + default: + return strconv.FormatInt(ret-int64(sizeLimits[n]), 10), n + 1 + } + } + switch code { + case dateCode: + return Date(ret).String(), n + 1 + case timeCode: + return Time(ret).String2(scale), n + 1 + case datetimeCode: + return Datetime(ret).String2(scale), n + 1 + case timestampCode: + return Timestamp(ret).String2(loc, scale), n + 1 + //case enumCode: + // return Enum(ret), n + 1 + default: + return strconv.FormatInt(ret, 10), n + 1 + } +} + +func stringifyUint(code byte, b []byte) (string, int) { + if b[0] == intZeroCode { + return "0", 1 + } + n := int(b[0]) - intZeroCode + + bp := make([]byte, 8) + copy(bp[8-n:], b[1:n+1]) + + var ret uint64 + binary.Read(bytes.NewBuffer(bp), binary.BigEndian, &ret) + + return strconv.FormatUint(ret, 10), n + 1 +} + +func stringifyFloat32(b []byte) (string, int) { + bp := make([]byte, 4) + copy(bp, b[1:]) + adjustFloatBytes(bp, false) + var ret float32 + binary.Read(bytes.NewBuffer(bp), binary.BigEndian, &ret) + return strconv.FormatFloat(float64(ret), 'G', -1, 32), 5 +} + +func stringifyFloat64(b []byte) (string, int) { + bp := make([]byte, 8) + copy(bp, b[1:]) + adjustFloatBytes(bp, false) + var ret float64 + binary.Read(bytes.NewBuffer(bp), binary.BigEndian, &ret) + return strconv.FormatFloat(ret, 'G', -1, 64), 9 +} + +func stringifyDecimal64(b []byte, scale int32) (string, int) { + bp := make([]byte, 8) + copy(bp, b[:]) + bp[0] ^= 0x80 + for i := 0; i < 4; i++ { + bp[i] ^= bp[7-i] + bp[7-i] ^= bp[i] + bp[i] ^= bp[7-i] + } + ret := *(*Decimal64)(unsafe.Pointer(&bp[0])) + return ret.Format(scale), 9 +} + +func stringifyDecimal128(b []byte, scale int32) (string, int) { + bp := make([]byte, 16) + copy(bp, b[:]) + bp[0] ^= 0x80 + for i := 0; i < 8; i++ { + bp[i] ^= bp[15-i] + bp[15-i] ^= bp[i] + bp[i] ^= bp[15-i] + } + ret := *(*Decimal128)(unsafe.Pointer(&bp[0])) + return ret.Format(scale), 17 +} + +func stringifyUuid(b []byte) (string, int) { + var ret Uuid + copy(ret[:], b[1:]) + return ret.String(), 17 +} diff --git a/pkg/container/vector/functionTools.go b/pkg/container/vector/functionTools.go index 20a15b44faed..1658ac4188e2 100644 --- a/pkg/container/vector/functionTools.go +++ b/pkg/container/vector/functionTools.go @@ -559,6 +559,12 @@ func (fr *FunctionResult[T]) PreExtendAndReset(targetSize int) error { return nil } +func (fr *FunctionResult[T]) SetNullResult(length uint64) { + fr.vec.nsp.AddRange(0, length) + fr.vec.SetLength(int(length)) + fr.length = length +} + func (fr *FunctionResult[T]) Append(val T, isnull bool) error { if isnull { // XXX LOW PERF diff --git a/pkg/container/vector/vector.go b/pkg/container/vector/vector.go index 79e2d729230d..cf0dabb5be2d 100644 --- a/pkg/container/vector/vector.go +++ b/pkg/container/vector/vector.go @@ -19,6 +19,7 @@ import ( "fmt" "slices" "sort" + "time" "unsafe" "github.com/matrixorigin/matrixone/pkg/common/bitmap" @@ -2653,6 +2654,174 @@ func (v *Vector) String() string { } } +func implFixedRowToString[T types.FixedSizeT](v *Vector, idx int) string { + if v.IsConstNull() { + return "null" + } + + if v.IsConst() { + if nulls.Contains(&v.nsp, 0) { + return "null" + } else { + return fmt.Sprintf("%v", GetFixedAtNoTypeCheck[T](v, 0)) + } + } + if v.nsp.Contains(uint64(idx)) { + return "null" + } else { + return fmt.Sprintf("%v", GetFixedAtNoTypeCheck[T](v, idx)) + } +} + +func implTimestampRowToString(v *Vector, idx int) string { + if v.IsConstNull() { + return "null" + } + + loc := time.Local + if v.IsConst() { + if nulls.Contains(&v.nsp, 0) { + return "null" + } else { + return GetFixedAtNoTypeCheck[types.Timestamp](v, 0).String2(loc, v.typ.Scale) + } + } + if v.nsp.Contains(uint64(idx)) { + return "null" + } else { + return GetFixedAtNoTypeCheck[types.Timestamp](v, idx).String2(loc, v.typ.Scale) + } +} + +func implDatetimeRowToString(v *Vector, idx int) string { + if v.IsConstNull() { + return "null" + } + + if v.IsConst() { + if nulls.Contains(&v.nsp, 0) { + return "null" + } else { + return GetFixedAtNoTypeCheck[types.Datetime](v, 0).String2(v.typ.Scale) + } + } + if v.nsp.Contains(uint64(idx)) { + return "null" + } else { + return GetFixedAtNoTypeCheck[types.Datetime](v, idx).String2(v.typ.Scale) + } +} + +func implDecimalRowToString[T types.DecimalWithFormat](v *Vector, idx int) string { + if v.IsConstNull() { + return "null" + } + + if v.IsConst() { + if nulls.Contains(&v.nsp, 0) { + return "null" + } else { + return GetFixedAtNoTypeCheck[T](v, 0).Format(v.typ.Scale) + } + } + if v.nsp.Contains(uint64(idx)) { + return "null" + } else { + return GetFixedAtNoTypeCheck[T](v, idx).Format(v.typ.Scale) + } +} + +func implArrayRowToString[T types.RealNumbers](v *Vector, idx int) string { + if v.IsConstNull() { + return "null" + } + + if v.IsConst() { + if nulls.Contains(&v.nsp, 0) { + return "null" + } else { + return types.ArrayToString(GetArrayAt[T](v, 0)) + } + } + if v.nsp.Contains(uint64(idx)) { + return "null" + } else { + return types.ArrayToString(GetArrayAt[T](v, idx)) + } +} + +func (v *Vector) RowToString(idx int) string { + switch v.typ.Oid { + case types.T_bool: + return implFixedRowToString[bool](v, idx) + case types.T_bit: + return implFixedRowToString[uint64](v, idx) + case types.T_int8: + return implFixedRowToString[int8](v, idx) + case types.T_int16: + return implFixedRowToString[int16](v, idx) + case types.T_int32: + return implFixedRowToString[int32](v, idx) + case types.T_int64: + return implFixedRowToString[int64](v, idx) + case types.T_uint8: + return implFixedRowToString[uint8](v, idx) + case types.T_uint16: + return implFixedRowToString[uint16](v, idx) + case types.T_uint32: + return implFixedRowToString[uint32](v, idx) + case types.T_uint64: + return implFixedRowToString[uint64](v, idx) + case types.T_float32: + return implFixedRowToString[float32](v, idx) + case types.T_float64: + return implFixedRowToString[float64](v, idx) + case types.T_date: + return implFixedRowToString[types.Date](v, idx) + case types.T_datetime: + return implDatetimeRowToString(v, idx) + case types.T_time: + return implFixedRowToString[types.Time](v, idx) + case types.T_timestamp: + return implTimestampRowToString(v, idx) + case types.T_enum: + return implFixedRowToString[types.Enum](v, idx) + case types.T_decimal64: + return implDecimalRowToString[types.Decimal64](v, idx) + case types.T_decimal128: + return implDecimalRowToString[types.Decimal128](v, idx) + case types.T_uuid: + return implFixedRowToString[types.Uuid](v, idx) + case types.T_TS: + return implFixedRowToString[types.TS](v, idx) + case types.T_Rowid: + return implFixedRowToString[types.Rowid](v, idx) + case types.T_Blockid: + return implFixedRowToString[types.Blockid](v, idx) + case types.T_char, types.T_varchar, types.T_binary, types.T_varbinary, types.T_json, types.T_blob, types.T_text, types.T_datalink: + col := MustFixedColNoTypeCheck[types.Varlena](v) + if len(col) == 1 { + if nulls.Contains(&v.nsp, 0) { + return "null" + } else { + return col[0].UnsafeGetString(v.area) + } + } + if v.nsp.Contains(uint64(idx)) { + return "null" + } else { + return col[idx].UnsafeGetString(v.area) + } + //return fmt.Sprintf("%v-%s", col, v.nsp.GetBitmap().String()) + case types.T_array_float32: + return implArrayRowToString[float32](v, idx) + case types.T_array_float64: + return implArrayRowToString[float64](v, idx) + default: + panic("vec to string unknown types.") + } +} + func SetConstNull(vec *Vector, length int, mp *mpool.MPool) error { if len(vec.data) > 0 { vec.data = vec.data[:0] diff --git a/pkg/container/vector/vector_test.go b/pkg/container/vector/vector_test.go index 21d937dc9a29..edc3de2e213f 100644 --- a/pkg/container/vector/vector_test.go +++ b/pkg/container/vector/vector_test.go @@ -1607,311 +1607,6 @@ func TestCloneWindowWithMpNil(t *testing.T) { } } -/* -func TestUnionOne(t *testing.T) { - mp := mpool.MustNewZero() - { // bool - v := NewVector(types.T_bool.ToType()) - err := v.PreExtend(10, mp) - require.NoError(t, err) - err = AppendList(v, []bool{true, false, true, false}, nil, mp) - require.NoError(t, err) - w := NewVector(types.T_bool.ToType()) - w.UnionOne(v, 0, false, mp) - require.Equal(t, MustFixedColWithTypeCheck[bool](v)[:1], MustFixedColWithTypeCheck[bool](w)) - w.UnionOne(v, 0, true, mp) - v.Free(mp) - w.Free(mp) - require.Equal(t, int64(0), mp.CurrNB()) - } - { // int8 - v := NewVector(types.T_int8.ToType()) - err := v.PreExtend(10, mp) - require.NoError(t, err) - err = AppendList(v, []int8{1, 2, 3, 4}, nil, mp) - require.NoError(t, err) - w := NewVector(types.T_int8.ToType()) - w.UnionOne(v, 0, false, mp) - require.Equal(t, MustFixedColWithTypeCheck[int8](v)[:1], MustFixedColWithTypeCheck[int8](w)) - w.UnionOne(v, 0, true, mp) - v.Free(mp) - w.Free(mp) - require.Equal(t, int64(0), mp.CurrNB()) - } - { // int16 - v := NewVector(types.T_int16.ToType()) - err := v.PreExtend(10, mp) - require.NoError(t, err) - err = AppendList(v, []int16{1, 2, 3, 4}, nil, mp) - require.NoError(t, err) - w := NewVector(types.T_int16.ToType()) - w.UnionOne(v, 0, false, mp) - require.Equal(t, MustFixedColWithTypeCheck[int16](v)[:1], MustFixedColWithTypeCheck[int16](w)) - w.UnionOne(v, 0, true, mp) - v.Free(mp) - w.Free(mp) - require.Equal(t, int64(0), mp.CurrNB()) - } - { // int32 - v := NewVector(types.T_int32.ToType()) - err := v.PreExtend(10, mp) - require.NoError(t, err) - err = AppendList(v, []int32{1, 2, 3, 4}, nil, mp) - require.NoError(t, err) - w := NewVector(types.T_int32.ToType()) - w.UnionOne(v, 0, false, mp) - require.Equal(t, MustFixedColWithTypeCheck[int32](v)[:1], MustFixedColWithTypeCheck[int32](w)) - w.UnionOne(v, 0, true, mp) - v.Free(mp) - w.Free(mp) - require.Equal(t, int64(0), mp.CurrNB()) - } - { // int64 - v := NewVector(types.T_int64.ToType()) - err := v.PreExtend(10, mp) - require.NoError(t, err) - err = AppendList(v, []int64{1, 2, 3, 4}, nil, mp) - require.NoError(t, err) - w := NewVector(types.T_int64.ToType()) - w.UnionOne(v, 0, false, mp) - require.Equal(t, MustFixedColWithTypeCheck[int64](v)[:1], MustFixedColWithTypeCheck[int64](w)) - w.UnionOne(v, 0, true, mp) - v.Free(mp) - w.Free(mp) - require.Equal(t, int64(0), mp.CurrNB()) - } - { // uint8 - v := NewVector(types.T_uint8.ToType()) - err := v.PreExtend(10, mp) - require.NoError(t, err) - err = AppendList(v, []uint8{1, 2, 3, 4}, nil, mp) - require.NoError(t, err) - w := NewVector(types.T_uint8.ToType()) - w.UnionOne(v, 0, false, mp) - require.Equal(t, MustFixedColWithTypeCheck[uint8](v)[:1], MustFixedColWithTypeCheck[uint8](w)) - w.UnionOne(v, 0, true, mp) - v.Free(mp) - w.Free(mp) - require.Equal(t, int64(0), mp.CurrNB()) - } - { // uint16 - v := NewVector(types.T_uint16.ToType()) - err := v.PreExtend(10, mp) - require.NoError(t, err) - err = AppendList(v, []uint16{1, 2, 3, 4}, nil, mp) - require.NoError(t, err) - w := NewVector(types.T_uint16.ToType()) - w.UnionOne(v, 0, false, mp) - require.Equal(t, MustFixedColWithTypeCheck[uint16](v)[:1], MustFixedColWithTypeCheck[uint16](w)) - w.UnionOne(v, 0, true, mp) - v.Free(mp) - w.Free(mp) - require.Equal(t, int64(0), mp.CurrNB()) - } - { // uint32 - v := NewVector(types.T_uint32.ToType()) - err := v.PreExtend(10, mp) - require.NoError(t, err) - err = AppendList(v, []uint32{1, 2, 3, 4}, nil, mp) - require.NoError(t, err) - w := NewVector(types.T_uint32.ToType()) - w.UnionOne(v, 0, false, mp) - require.Equal(t, MustFixedColWithTypeCheck[uint32](v)[:1], MustFixedColWithTypeCheck[uint32](w)) - w.UnionOne(v, 0, true, mp) - v.Free(mp) - w.Free(mp) - require.Equal(t, int64(0), mp.CurrNB()) - } - { // uint64 - v := NewVector(types.T_uint64.ToType()) - err := v.PreExtend(10, mp) - require.NoError(t, err) - err = AppendList(v, []uint64{1, 2, 3, 4}, nil, mp) - require.NoError(t, err) - w := NewVector(types.T_uint64.ToType()) - w.UnionOne(v, 0, false, mp) - require.Equal(t, MustFixedColWithTypeCheck[uint64](v)[:1], MustFixedColWithTypeCheck[uint64](w)) - w.UnionOne(v, 0, true, mp) - v.Free(mp) - w.Free(mp) - require.Equal(t, int64(0), mp.CurrNB()) - } - { // float32 - v := NewVector(types.T_float32.ToType()) - err := v.PreExtend(10, mp) - require.NoError(t, err) - err = AppendList(v, []float32{1, 2, 3, 4}, nil, mp) - require.NoError(t, err) - w := NewVector(types.T_float32.ToType()) - w.UnionOne(v, 0, false, mp) - require.Equal(t, MustFixedColWithTypeCheck[float32](v)[:1], MustFixedColWithTypeCheck[float32](w)) - w.UnionOne(v, 0, true, mp) - v.Free(mp) - w.Free(mp) - require.Equal(t, int64(0), mp.CurrNB()) - } - { // float64 - v := NewVector(types.T_float64.ToType()) - err := v.PreExtend(10, mp) - require.NoError(t, err) - err = AppendList(v, []float64{1, 2, 3, 4}, nil, mp) - require.NoError(t, err) - w := NewVector(types.T_float64.ToType()) - w.UnionOne(v, 0, false, mp) - require.Equal(t, MustFixedColWithTypeCheck[float64](v)[:1], MustFixedColWithTypeCheck[float64](w)) - w.UnionOne(v, 0, true, mp) - v.Free(mp) - w.Free(mp) - require.Equal(t, int64(0), mp.CurrNB()) - } - { // text - v := NewVector(types.T_text.ToType()) - err := v.PreExtend(10, mp) - require.NoError(t, err) - err = AppendBytesList(v, [][]byte{[]byte("1"), []byte("2"), []byte("3"), []byte("4")}, nil, mp) - require.NoError(t, err) - w := NewVector(types.T_text.ToType()) - w.UnionOne(v, 0, false, mp) - require.Equal(t, MustStrCols(v)[:1], MustStrCols(w)) - w.UnionOne(v, 0, true, mp) - v.Free(mp) - w.Free(mp) - require.Equal(t, int64(0), mp.CurrNB()) - } - { // date - v := NewVector(types.T_date.ToType()) - err := v.PreExtend(10, mp) - require.NoError(t, err) - err = AppendList(v, []types.Date{1, 2, 3, 4}, nil, mp) - require.NoError(t, err) - w := NewVector(types.T_date.ToType()) - w.UnionOne(v, 0, false, mp) - require.Equal(t, MustFixedColWithTypeCheck[types.Date](v)[:1], MustFixedColWithTypeCheck[types.Date](w)) - w.UnionOne(v, 0, true, mp) - v.Free(mp) - w.Free(mp) - require.Equal(t, int64(0), mp.CurrNB()) - } - { // datetime - v := NewVector(types.T_datetime.ToType()) - err := v.PreExtend(10, mp) - require.NoError(t, err) - err = AppendList(v, []types.Datetime{1, 2, 3, 4}, nil, mp) - require.NoError(t, err) - w := NewVector(types.T_datetime.ToType()) - w.UnionOne(v, 0, false, mp) - require.Equal(t, MustFixedColWithTypeCheck[types.Datetime](v)[:1], MustFixedColWithTypeCheck[types.Datetime](w)) - w.UnionOne(v, 0, true, mp) - v.Free(mp) - w.Free(mp) - require.Equal(t, int64(0), mp.CurrNB()) - } - { // time - v := NewVector(types.T_time.ToType()) - err := v.PreExtend(10, mp) - require.NoError(t, err) - err = AppendList(v, []types.Time{1, 2, 3, 4}, nil, mp) - require.NoError(t, err) - w := NewVector(types.T_time.ToType()) - w.UnionOne(v, 0, false, mp) - require.Equal(t, MustFixedColWithTypeCheck[types.Time](v)[:1], MustFixedColWithTypeCheck[types.Time](w)) - w.UnionOne(v, 0, true, mp) - v.Free(mp) - w.Free(mp) - require.Equal(t, int64(0), mp.CurrNB()) - } - { // timestamp - v := NewVector(types.T_timestamp.ToType()) - err := v.PreExtend(10, mp) - require.NoError(t, err) - err = AppendList(v, []types.Timestamp{1, 2, 3, 4}, nil, mp) - require.NoError(t, err) - w := NewVector(types.T_timestamp.ToType()) - w.UnionOne(v, 0, false, mp) - require.Equal(t, MustFixedColWithTypeCheck[types.Timestamp](v)[:1], MustFixedColWithTypeCheck[types.Timestamp](w)) - w.UnionOne(v, 0, true, mp) - v.Free(mp) - w.Free(mp) - require.Equal(t, int64(0), mp.CurrNB()) - } - { // decimal64 - vs := make([]types.Decimal64, 4) - v := NewVector(types.T_decimal64.ToType()) - err := v.PreExtend(10, mp) - require.NoError(t, err) - err = AppendList(v, vs, nil, mp) - require.NoError(t, err) - w := NewVector(types.T_decimal64.ToType()) - w.UnionOne(v, 0, false, mp) - require.Equal(t, MustFixedColWithTypeCheck[types.Decimal64](v)[:1], MustFixedColWithTypeCheck[types.Decimal64](w)) - w.UnionOne(v, 0, true, mp) - v.Free(mp) - w.Free(mp) - require.Equal(t, int64(0), mp.CurrNB()) - } - { // decimal128 - vs := make([]types.Decimal128, 4) - v := NewVector(types.T_decimal128.ToType()) - err := v.PreExtend(10, mp) - require.NoError(t, err) - err = AppendList(v, vs, nil, mp) - require.NoError(t, err) - w := NewVector(types.T_decimal128.ToType()) - w.UnionOne(v, 0, false, mp) - require.Equal(t, MustFixedColWithTypeCheck[types.Decimal128](v)[:1], MustFixedColWithTypeCheck[types.Decimal128](w)) - w.UnionOne(v, 0, true, mp) - v.Free(mp) - w.Free(mp) - require.Equal(t, int64(0), mp.CurrNB()) - } - { // uuid - vs := make([]types.Uuid, 4) - v := NewVector(types.T_uuid.ToType()) - err := v.PreExtend(10, mp) - require.NoError(t, err) - err = AppendList(v, vs, nil, mp) - require.NoError(t, err) - w := NewVector(types.T_uuid.ToType()) - w.UnionOne(v, 0, false, mp) - require.Equal(t, MustFixedColWithTypeCheck[types.Uuid](v)[:1], MustFixedColWithTypeCheck[types.Uuid](w)) - w.UnionOne(v, 0, true, mp) - v.Free(mp) - w.Free(mp) - require.Equal(t, int64(0), mp.CurrNB()) - } - { // ts - vs := make([]types.TS, 4) - v := NewVector(types.T_TS.ToType()) - err := v.PreExtend(10, mp) - require.NoError(t, err) - err = AppendList(v, vs, nil, mp) - require.NoError(t, err) - w := NewVector(types.T_TS.ToType()) - w.UnionOne(v, 0, false, mp) - require.Equal(t, MustFixedColWithTypeCheck[types.TS](v)[:1], MustFixedColWithTypeCheck[types.TS](w)) - w.UnionOne(v, 0, true, mp) - v.Free(mp) - w.Free(mp) - require.Equal(t, int64(0), mp.CurrNB()) - } - { // rowid - vs := make([]types.Rowid, 4) - v := NewVector(types.T_Rowid.ToType()) - err := v.PreExtend(10, mp) - require.NoError(t, err) - err = AppendList(v, vs, nil, mp) - require.NoError(t, err) - w := NewVector(types.T_Rowid.ToType()) - w.UnionOne(v, 0, false, mp) - require.Equal(t, MustFixedColWithTypeCheck[types.Rowid](v)[:1], MustFixedColWithTypeCheck[types.Rowid](w)) - w.UnionOne(v, 0, true, mp) - v.Free(mp) - w.Free(mp) - require.Equal(t, int64(0), mp.CurrNB()) - } -} -*/ - func TestMarshalAndUnMarshal(t *testing.T) { mp := mpool.MustNewZero() v := NewVec(types.T_int8.ToType()) @@ -2993,6 +2688,242 @@ func BenchmarkMustFixedCol(b *testing.B) { } } +func TestRowToString(t *testing.T) { + mp := mpool.MustNewZero() + + { // Array Float32 + v := NewVec(types.T_array_float32.ToType()) + err := AppendArrayList(v, [][]float32{{1, 1}}, nil, mp) + require.NoError(t, err) + require.Equal(t, "[1, 1]", v.RowToString(0)) + err = AppendArrayList(v, [][]float32{{2, 2}, {3, 3}}, nil, mp) + require.NoError(t, err) + require.Equal(t, "[2, 2]", v.RowToString(1)) + v.Free(mp) + require.Equal(t, int64(0), mp.CurrNB()) + } + { // Array Float64 + v := NewVec(types.T_array_float64.ToType()) + err := AppendArrayList(v, [][]float64{{1, 1}}, nil, mp) + require.NoError(t, err) + require.Equal(t, "[1, 1]", v.RowToString(0)) + err = AppendArrayList(v, [][]float64{{2, 2}, {3, 3}}, nil, mp) + require.NoError(t, err) + require.Equal(t, "[2, 2]", v.RowToString(1)) + v.Free(mp) + require.Equal(t, int64(0), mp.CurrNB()) + } + { // bool + v := NewVec(types.T_bool.ToType()) + err := AppendFixed(v, true, false, mp) + require.NoError(t, err) + require.Equal(t, "true", v.RowToString(0)) + err = AppendFixed(v, false, true, mp) + require.NoError(t, err) + require.Equal(t, "null", v.RowToString(1)) + v.Free(mp) + + v0 := NewVec(types.T_bool.ToType()) + err = AppendFixed(v0, false, true, mp) + require.NoError(t, err) + require.Equal(t, "null", v0.RowToString(0)) + err = AppendFixed(v0, true, false, mp) + require.NoError(t, err) + require.Equal(t, "true", v0.RowToString(1)) + v0.Free(mp) + + v1 := NewConstNull(types.T_bool.ToType(), 1, mp) + require.Equal(t, "null", v1.RowToString(1)) + v1.Free(mp) + + require.Equal(t, int64(0), mp.CurrNB()) + } + { // int8 + v := NewVec(types.T_int8.ToType()) + err := AppendFixedList(v, []int8{1, 2, 3, 4}, nil, mp) + require.NoError(t, err) + require.Equal(t, "2", v.RowToString(1)) + v.Free(mp) + require.Equal(t, int64(0), mp.CurrNB()) + } + { // int16 + v := NewVec(types.T_int16.ToType()) + err := AppendFixedList(v, []int16{1, 2, 3, 4}, nil, mp) + require.NoError(t, err) + require.Equal(t, "2", v.RowToString(1)) + v.Free(mp) + require.Equal(t, int64(0), mp.CurrNB()) + } + { // int32 + v := NewVec(types.T_int32.ToType()) + err := AppendFixedList(v, []int32{1, 2, 3, 4}, nil, mp) + require.NoError(t, err) + require.Equal(t, "2", v.RowToString(1)) + v.Free(mp) + require.Equal(t, int64(0), mp.CurrNB()) + } + { // int64 + v := NewVec(types.T_int64.ToType()) + err := AppendFixedList(v, []int64{1, 2, 3, 4}, nil, mp) + require.NoError(t, err) + require.Equal(t, "2", v.RowToString(1)) + v.Free(mp) + require.Equal(t, int64(0), mp.CurrNB()) + } + { // uint8 + v := NewVec(types.T_uint8.ToType()) + err := AppendFixedList(v, []uint8{1, 2, 3, 4}, nil, mp) + require.NoError(t, err) + require.Equal(t, "2", v.RowToString(1)) + v.Free(mp) + require.Equal(t, int64(0), mp.CurrNB()) + } + { // uint16 + v := NewVec(types.T_uint16.ToType()) + err := AppendFixedList(v, []uint16{1, 2, 3, 4}, nil, mp) + require.NoError(t, err) + require.Equal(t, "2", v.RowToString(1)) + v.Free(mp) + require.Equal(t, int64(0), mp.CurrNB()) + } + { // uint32 + v := NewVec(types.T_uint32.ToType()) + err := AppendFixedList(v, []uint32{1, 2, 3, 4}, nil, mp) + require.NoError(t, err) + require.Equal(t, "2", v.RowToString(1)) + v.Free(mp) + require.Equal(t, int64(0), mp.CurrNB()) + } + { // uint64 + v := NewVec(types.T_uint64.ToType()) + err := AppendFixedList(v, []uint64{1, 2, 3, 4}, nil, mp) + require.NoError(t, err) + require.Equal(t, "2", v.RowToString(1)) + v.Free(mp) + require.Equal(t, int64(0), mp.CurrNB()) + } + { // float32 + v := NewVec(types.T_float32.ToType()) + err := AppendFixedList(v, []float32{1, 2, 3, 4}, nil, mp) + require.NoError(t, err) + require.Equal(t, "2", v.RowToString(1)) + v.Free(mp) + require.Equal(t, int64(0), mp.CurrNB()) + } + { // float64 + v := NewVec(types.T_float64.ToType()) + err := AppendFixedList(v, []float64{1, 2, 3, 4}, nil, mp) + require.NoError(t, err) + require.Equal(t, "2", v.RowToString(1)) + v.Free(mp) + require.Equal(t, int64(0), mp.CurrNB()) + } + { // text + v := NewVec(types.T_text.ToType()) + err := AppendBytesList(v, [][]byte{[]byte("1"), []byte("2"), []byte("3"), []byte("4")}, nil, mp) + require.NoError(t, err) + require.Equal(t, "2", v.RowToString(1)) + v.Free(mp) + require.Equal(t, int64(0), mp.CurrNB()) + } + { // date + v := NewVec(types.T_date.ToType()) + err := AppendFixedList(v, []types.Date{1, 2, 3, 4}, nil, mp) + require.NoError(t, err) + require.Equal(t, "0001-01-03", v.RowToString(1)) + v.Free(mp) + require.Equal(t, int64(0), mp.CurrNB()) + } + { // datetime + v := NewVec(types.T_datetime.ToType()) + scale := types.Datetime(types.MicroSecsPerSec * types.SecsPerDay) + err := AppendFixedList(v, []types.Datetime{1 * scale, 2 * scale, 3 * scale, 4 * scale}, nil, mp) + require.NoError(t, err) + require.Equal(t, "0001-01-03 00:00:00", v.RowToString(1)) + v.Free(mp) + require.Equal(t, int64(0), mp.CurrNB()) + } + { // time + v := NewVec(types.T_time.ToType()) + scale := types.Time(types.MicroSecsPerSec) + err := AppendFixedList(v, []types.Time{1 * scale, 2 * scale, 3 * scale, 4 * scale}, nil, mp) + require.NoError(t, err) + require.Equal(t, "00:00:02", v.RowToString(1)) + v.Free(mp) + require.Equal(t, int64(0), mp.CurrNB()) + } + { // timestamp + v := NewVec(types.T_timestamp.ToType()) + err := AppendFixedList(v, []types.Timestamp{1, types.Timestamp(types.DatetimeFromClock(1970, 1, 1, 0, 0, 0, 0)), 3, 4}, nil, mp) + require.NoError(t, err) + require.Equal(t, "1970-01-01 00:00:00", v.RowToString(1)) + v.Free(mp) + require.Equal(t, int64(0), mp.CurrNB()) + } + { // decimal64 + vs := make([]types.Decimal64, 4) + v := NewVec(types.T_decimal64.ToType()) + err := AppendFixedList(v, vs, nil, mp) + require.NoError(t, err) + require.Equal(t, "0", v.RowToString(1)) + v.Free(mp) + require.Equal(t, int64(0), mp.CurrNB()) + } + { // decimal128 + vs := make([]types.Decimal128, 4) + v := NewVec(types.T_decimal128.ToType()) + err := AppendFixedList(v, vs, nil, mp) + require.NoError(t, err) + require.Equal(t, "0", v.RowToString(1)) + v.Free(mp) + require.Equal(t, int64(0), mp.CurrNB()) + } + { // uuid + vs := make([]types.Uuid, 4) + v := NewVec(types.T_uuid.ToType()) + err := AppendFixedList(v, vs, nil, mp) + require.NoError(t, err) + require.Equal(t, "00000000-0000-0000-0000-000000000000", v.RowToString(1)) + v.Free(mp) + require.Equal(t, int64(0), mp.CurrNB()) + } + { // ts + vs := make([]types.TS, 4) + v := NewVec(types.T_TS.ToType()) + err := AppendFixedList(v, vs, nil, mp) + require.NoError(t, err) + require.Equal(t, "[0 0 0 0 0 0 0 0 0 0 0 0]", v.RowToString(1)) + v.Free(mp) + require.Equal(t, int64(0), mp.CurrNB()) + } + { // rowid + vs := make([]types.Rowid, 4) + v := NewVec(types.T_Rowid.ToType()) + err := AppendFixedList(v, vs, nil, mp) + require.NoError(t, err) + require.Equal(t, "00000000-0000-0000-0000-000000000000-0-0-0", v.RowToString(1)) + v.Free(mp) + require.Equal(t, int64(0), mp.CurrNB()) + } + { // blockid + vs := make([]types.Blockid, 4) + v := NewVec(types.T_Blockid.ToType()) + err := AppendFixedList(v, vs, nil, mp) + require.NoError(t, err) + require.Equal(t, "[0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0]", v.RowToString(1)) + v.Free(mp) + require.Equal(t, int64(0), mp.CurrNB()) + } + { // bit + v := NewVec(types.T_bit.ToType()) + err := AppendFixedList(v, []uint64{1, 2, 3, 4}, nil, mp) + require.NoError(t, err) + require.Equal(t, "2", v.RowToString(1)) + v.Free(mp) + require.Equal(t, int64(0), mp.CurrNB()) + } +} + func TestIntersection2VectorOrdered(t *testing.T) { const ll = 10000 const cnt = 100 diff --git a/pkg/frontend/authenticate.go b/pkg/frontend/authenticate.go index 9e0d94575606..d26b0246750e 100644 --- a/pkg/frontend/authenticate.go +++ b/pkg/frontend/authenticate.go @@ -5707,6 +5707,19 @@ func extractPrivilegeTipsFromPlan(p *plan2.Plan) privilegeTipsArray { }) } } + } else if node.NodeType == plan.Node_MULTI_UPDATE { + for _, updateCtx := range node.UpdateCtxList { + if !isIndexTable(updateCtx.ObjRef.GetObjName()) { + isClusterTable := updateCtx.TableDef.TableType == catalog.SystemClusterRel + appendPt(privilegeTips{ + typ: t, + databaseName: updateCtx.ObjRef.GetSchemaName(), + tableName: updateCtx.ObjRef.GetObjName(), + isClusterTable: isClusterTable, + clusterTableOperation: clusterTableModify, + }) + } + } } } } else if p.GetDdl() != nil { diff --git a/pkg/frontend/predefined.go b/pkg/frontend/predefined.go index a6dfde4ef054..806c096590c1 100644 --- a/pkg/frontend/predefined.go +++ b/pkg/frontend/predefined.go @@ -294,21 +294,21 @@ var ( )`, catalog.MO_CATALOG, catalog.MOAutoIncrTable) MoCatalogMoIndexesDDL = fmt.Sprintf(`create table %s.%s ( - id bigint unsigned not null, - table_id bigint unsigned not null, - database_id bigint unsigned not null, - name varchar(64) not null, - type varchar(11) not null, - algo varchar(11), - algo_table_type varchar(11), - algo_params varchar(2048), - is_visible tinyint not null, - hidden tinyint not null, - comment varchar(2048) not null, - column_name varchar(256) not null, - ordinal_position int unsigned not null, - options text, - index_table_name varchar(5000), + id bigint unsigned not null, + table_id bigint unsigned not null, + database_id bigint unsigned not null, + name varchar(64) not null, + type varchar(11) not null, + algo varchar(11), + algo_table_type varchar(11), + algo_params varchar(2048), + is_visible tinyint not null, + hidden tinyint not null, + comment varchar(2048) not null, + column_name varchar(256) not null, + ordinal_position int unsigned not null, + options text, + index_table_name varchar(5000), primary key(id, column_name) )`, catalog.MO_CATALOG, catalog.MO_INDEXES) @@ -348,18 +348,18 @@ var ( )`, catalog.MO_CATALOG, catalog.MOForeignKeys) MoCatalogMoTablePartitionsDDL = fmt.Sprintf(`CREATE TABLE %s.%s ( - table_id bigint unsigned NOT NULL, - database_id bigint unsigned not null, - number smallint unsigned NOT NULL, - name varchar(64) NOT NULL, - partition_type varchar(50) NOT NULL, - partition_expression varchar(2048) NULL, - description_utf8 text, - comment varchar(2048) NOT NULL, - options text, - partition_table_name varchar(1024) NOT NULL, - PRIMARY KEY table_id (table_id, name) - )`, catalog.MO_CATALOG, catalog.MO_TABLE_PARTITIONS) + table_id bigint unsigned NOT NULL, + database_id bigint unsigned not null, + number smallint unsigned NOT NULL, + name varchar(64) NOT NULL, + partition_type varchar(50) NOT NULL, + partition_expression varchar(2048) NULL, + description_utf8 text, + comment varchar(2048) NOT NULL, + options text, + partition_table_name varchar(1024) NOT NULL, + PRIMARY KEY table_id (table_id, name) + )`, catalog.MO_CATALOG, catalog.MO_TABLE_PARTITIONS) ) // step3InitSQLs diff --git a/pkg/pb/pipeline/pipeline.pb.go b/pkg/pb/pipeline/pipeline.pb.go index 50d575748695..ab6e4d7cf3b1 100644 --- a/pkg/pb/pipeline/pipeline.pb.go +++ b/pkg/pb/pipeline/pipeline.pb.go @@ -119,7 +119,7 @@ func (x SampleFunc_SampleType) String() string { } func (SampleFunc_SampleType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_7ac67a7adf3df9c7, []int{44, 0} + return fileDescriptor_7ac67a7adf3df9c7, []int{45, 0} } type SessionLoggerInfo_LogLevel int32 @@ -156,7 +156,7 @@ func (x SessionLoggerInfo_LogLevel) String() string { } func (SessionLoggerInfo_LogLevel) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_7ac67a7adf3df9c7, []int{53, 0} + return fileDescriptor_7ac67a7adf3df9c7, []int{54, 0} } type Pipeline_PipelineType int32 @@ -187,7 +187,7 @@ func (x Pipeline_PipelineType) String() string { } func (Pipeline_PipelineType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_7ac67a7adf3df9c7, []int{54, 0} + return fileDescriptor_7ac67a7adf3df9c7, []int{55, 0} } type Message struct { @@ -1032,11 +1032,11 @@ func (m *Insert) GetTableDef() *plan.TableDef { type MultiUpdate struct { AffectedRows uint64 `protobuf:"varint,1,opt,name=affected_rows,json=affectedRows,proto3" json:"affected_rows,omitempty"` - ToWriteS3 bool `protobuf:"varint,2,opt,name=ToWriteS3,proto3" json:"ToWriteS3,omitempty"` + Action uint32 `protobuf:"varint,2,opt,name=Action,proto3" json:"Action,omitempty"` IBucket uint32 `protobuf:"varint,3,opt,name=IBucket,proto3" json:"IBucket,omitempty"` NBucket uint32 `protobuf:"varint,4,opt,name=NBucket,proto3" json:"NBucket,omitempty"` SegmentMap map[string]int32 `protobuf:"bytes,5,rep,name=SegmentMap,proto3" json:"SegmentMap,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` - UpdateCtx []*plan.UpdateCtx `protobuf:"bytes,6,rep,name=update_ctx,json=updateCtx,proto3" json:"update_ctx,omitempty"` + UpdateCtxList []*plan.UpdateCtx `protobuf:"bytes,6,rep,name=update_ctx_list,json=updateCtxList,proto3" json:"update_ctx_list,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -1082,11 +1082,11 @@ func (m *MultiUpdate) GetAffectedRows() uint64 { return 0 } -func (m *MultiUpdate) GetToWriteS3() bool { +func (m *MultiUpdate) GetAction() uint32 { if m != nil { - return m.ToWriteS3 + return m.Action } - return false + return 0 } func (m *MultiUpdate) GetIBucket() uint32 { @@ -1110,9 +1110,9 @@ func (m *MultiUpdate) GetSegmentMap() map[string]int32 { return nil } -func (m *MultiUpdate) GetUpdateCtx() []*plan.UpdateCtx { +func (m *MultiUpdate) GetUpdateCtxList() []*plan.UpdateCtx { if m != nil { - return m.UpdateCtx + return m.UpdateCtxList } return nil } @@ -3092,6 +3092,117 @@ func (m *MarkJoin) GetShuffleIdx() int32 { return 0 } +type DedupJoin struct { + LeftCond []*plan.Expr `protobuf:"bytes,1,rep,name=left_cond,json=leftCond,proto3" json:"left_cond,omitempty"` + RightCond []*plan.Expr `protobuf:"bytes,2,rep,name=right_cond,json=rightCond,proto3" json:"right_cond,omitempty"` + RuntimeFilterBuildList []*plan.RuntimeFilterSpec `protobuf:"bytes,3,rep,name=runtime_filter_build_list,json=runtimeFilterBuildList,proto3" json:"runtime_filter_build_list,omitempty"` + IsShuffle bool `protobuf:"varint,4,opt,name=is_shuffle,json=isShuffle,proto3" json:"is_shuffle,omitempty"` + JoinMapTag int32 `protobuf:"varint,5,opt,name=join_map_tag,json=joinMapTag,proto3" json:"join_map_tag,omitempty"` + ShuffleIdx int32 `protobuf:"varint,6,opt,name=shuffle_idx,json=shuffleIdx,proto3" json:"shuffle_idx,omitempty"` + OnDuplicateAction plan.Node_OnDuplicateAction `protobuf:"varint,7,opt,name=on_duplicate_action,json=onDuplicateAction,proto3,enum=plan.Node_OnDuplicateAction" json:"on_duplicate_action,omitempty"` + DedupColName string `protobuf:"bytes,8,opt,name=dedup_col_name,json=dedupColName,proto3" json:"dedup_col_name,omitempty"` + DedupColTypes []plan.Type `protobuf:"bytes,9,rep,name=dedup_col_types,json=dedupColTypes,proto3" json:"dedup_col_types"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *DedupJoin) Reset() { *m = DedupJoin{} } +func (m *DedupJoin) String() string { return proto.CompactTextString(m) } +func (*DedupJoin) ProtoMessage() {} +func (*DedupJoin) Descriptor() ([]byte, []int) { + return fileDescriptor_7ac67a7adf3df9c7, []int{30} +} +func (m *DedupJoin) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DedupJoin) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_DedupJoin.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *DedupJoin) XXX_Merge(src proto.Message) { + xxx_messageInfo_DedupJoin.Merge(m, src) +} +func (m *DedupJoin) XXX_Size() int { + return m.ProtoSize() +} +func (m *DedupJoin) XXX_DiscardUnknown() { + xxx_messageInfo_DedupJoin.DiscardUnknown(m) +} + +var xxx_messageInfo_DedupJoin proto.InternalMessageInfo + +func (m *DedupJoin) GetLeftCond() []*plan.Expr { + if m != nil { + return m.LeftCond + } + return nil +} + +func (m *DedupJoin) GetRightCond() []*plan.Expr { + if m != nil { + return m.RightCond + } + return nil +} + +func (m *DedupJoin) GetRuntimeFilterBuildList() []*plan.RuntimeFilterSpec { + if m != nil { + return m.RuntimeFilterBuildList + } + return nil +} + +func (m *DedupJoin) GetIsShuffle() bool { + if m != nil { + return m.IsShuffle + } + return false +} + +func (m *DedupJoin) GetJoinMapTag() int32 { + if m != nil { + return m.JoinMapTag + } + return 0 +} + +func (m *DedupJoin) GetShuffleIdx() int32 { + if m != nil { + return m.ShuffleIdx + } + return 0 +} + +func (m *DedupJoin) GetOnDuplicateAction() plan.Node_OnDuplicateAction { + if m != nil { + return m.OnDuplicateAction + } + return plan.Node_ERROR +} + +func (m *DedupJoin) GetDedupColName() string { + if m != nil { + return m.DedupColName + } + return "" +} + +func (m *DedupJoin) GetDedupColTypes() []plan.Type { + if m != nil { + return m.DedupColTypes + } + return nil +} + type Product struct { RelList []int32 `protobuf:"varint,1,rep,packed,name=rel_list,json=relList,proto3" json:"rel_list,omitempty"` ColList []int32 `protobuf:"varint,2,rep,packed,name=col_list,json=colList,proto3" json:"col_list,omitempty"` @@ -3106,7 +3217,7 @@ func (m *Product) Reset() { *m = Product{} } func (m *Product) String() string { return proto.CompactTextString(m) } func (*Product) ProtoMessage() {} func (*Product) Descriptor() ([]byte, []int) { - return fileDescriptor_7ac67a7adf3df9c7, []int{30} + return fileDescriptor_7ac67a7adf3df9c7, []int{31} } func (m *Product) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3177,7 +3288,7 @@ func (m *ProductL2) Reset() { *m = ProductL2{} } func (m *ProductL2) String() string { return proto.CompactTextString(m) } func (*ProductL2) ProtoMessage() {} func (*ProductL2) Descriptor() ([]byte, []int) { - return fileDescriptor_7ac67a7adf3df9c7, []int{31} + return fileDescriptor_7ac67a7adf3df9c7, []int{32} } func (m *ProductL2) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3246,7 +3357,7 @@ func (m *IndexJoin) Reset() { *m = IndexJoin{} } func (m *IndexJoin) String() string { return proto.CompactTextString(m) } func (*IndexJoin) ProtoMessage() {} func (*IndexJoin) Descriptor() ([]byte, []int) { - return fileDescriptor_7ac67a7adf3df9c7, []int{32} + return fileDescriptor_7ac67a7adf3df9c7, []int{33} } func (m *IndexJoin) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3304,7 +3415,7 @@ func (m *TableFunction) Reset() { *m = TableFunction{} } func (m *TableFunction) String() string { return proto.CompactTextString(m) } func (*TableFunction) ProtoMessage() {} func (*TableFunction) Descriptor() ([]byte, []int) { - return fileDescriptor_7ac67a7adf3df9c7, []int{33} + return fileDescriptor_7ac67a7adf3df9c7, []int{34} } func (m *TableFunction) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3380,7 +3491,7 @@ func (m *ExternalName2ColIndex) Reset() { *m = ExternalName2ColIndex{} } func (m *ExternalName2ColIndex) String() string { return proto.CompactTextString(m) } func (*ExternalName2ColIndex) ProtoMessage() {} func (*ExternalName2ColIndex) Descriptor() ([]byte, []int) { - return fileDescriptor_7ac67a7adf3df9c7, []int{34} + return fileDescriptor_7ac67a7adf3df9c7, []int{35} } func (m *ExternalName2ColIndex) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3434,7 +3545,7 @@ func (m *FileOffset) Reset() { *m = FileOffset{} } func (m *FileOffset) String() string { return proto.CompactTextString(m) } func (*FileOffset) ProtoMessage() {} func (*FileOffset) Descriptor() ([]byte, []int) { - return fileDescriptor_7ac67a7adf3df9c7, []int{35} + return fileDescriptor_7ac67a7adf3df9c7, []int{36} } func (m *FileOffset) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3491,7 +3602,7 @@ func (m *ExternalScan) Reset() { *m = ExternalScan{} } func (m *ExternalScan) String() string { return proto.CompactTextString(m) } func (*ExternalScan) ProtoMessage() {} func (*ExternalScan) Descriptor() ([]byte, []int) { - return fileDescriptor_7ac67a7adf3df9c7, []int{36} + return fileDescriptor_7ac67a7adf3df9c7, []int{37} } func (m *ExternalScan) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3610,7 +3721,7 @@ func (m *StreamScan) Reset() { *m = StreamScan{} } func (m *StreamScan) String() string { return proto.CompactTextString(m) } func (*StreamScan) ProtoMessage() {} func (*StreamScan) Descriptor() ([]byte, []int) { - return fileDescriptor_7ac67a7adf3df9c7, []int{37} + return fileDescriptor_7ac67a7adf3df9c7, []int{38} } func (m *StreamScan) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3671,7 +3782,7 @@ func (m *TableScan) Reset() { *m = TableScan{} } func (m *TableScan) String() string { return proto.CompactTextString(m) } func (*TableScan) ProtoMessage() {} func (*TableScan) Descriptor() ([]byte, []int) { - return fileDescriptor_7ac67a7adf3df9c7, []int{38} + return fileDescriptor_7ac67a7adf3df9c7, []int{39} } func (m *TableScan) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3718,7 +3829,7 @@ func (m *ValueScan) Reset() { *m = ValueScan{} } func (m *ValueScan) String() string { return proto.CompactTextString(m) } func (*ValueScan) ProtoMessage() {} func (*ValueScan) Descriptor() ([]byte, []int) { - return fileDescriptor_7ac67a7adf3df9c7, []int{39} + return fileDescriptor_7ac67a7adf3df9c7, []int{40} } func (m *ValueScan) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3764,7 +3875,7 @@ func (m *UnionAll) Reset() { *m = UnionAll{} } func (m *UnionAll) String() string { return proto.CompactTextString(m) } func (*UnionAll) ProtoMessage() {} func (*UnionAll) Descriptor() ([]byte, []int) { - return fileDescriptor_7ac67a7adf3df9c7, []int{40} + return fileDescriptor_7ac67a7adf3df9c7, []int{41} } func (m *UnionAll) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3794,24 +3905,28 @@ func (m *UnionAll) XXX_DiscardUnknown() { var xxx_messageInfo_UnionAll proto.InternalMessageInfo type HashBuild struct { - NeedHashMap bool `protobuf:"varint,1,opt,name=NeedHashMap,proto3" json:"NeedHashMap,omitempty"` - HashOnPK bool `protobuf:"varint,2,opt,name=HashOnPK,proto3" json:"HashOnPK,omitempty"` - NeedBatches bool `protobuf:"varint,3,opt,name=NeedBatches,proto3" json:"NeedBatches,omitempty"` - NeedAllocateSels bool `protobuf:"varint,4,opt,name=NeedAllocateSels,proto3" json:"NeedAllocateSels,omitempty"` - Conditions []*plan.Expr `protobuf:"bytes,5,rep,name=Conditions,proto3" json:"Conditions,omitempty"` - JoinMapTag int32 `protobuf:"varint,6,opt,name=JoinMapTag,proto3" json:"JoinMapTag,omitempty"` - JoinMapRefCnt int32 `protobuf:"varint,7,opt,name=JoinMapRefCnt,proto3" json:"JoinMapRefCnt,omitempty"` - RuntimeFilterSpec *plan.RuntimeFilterSpec `protobuf:"bytes,8,opt,name=RuntimeFilterSpec,proto3" json:"RuntimeFilterSpec,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + NeedHashMap bool `protobuf:"varint,1,opt,name=NeedHashMap,proto3" json:"NeedHashMap,omitempty"` + HashOnPK bool `protobuf:"varint,2,opt,name=HashOnPK,proto3" json:"HashOnPK,omitempty"` + NeedBatches bool `protobuf:"varint,3,opt,name=NeedBatches,proto3" json:"NeedBatches,omitempty"` + NeedAllocateSels bool `protobuf:"varint,4,opt,name=NeedAllocateSels,proto3" json:"NeedAllocateSels,omitempty"` + Conditions []*plan.Expr `protobuf:"bytes,5,rep,name=Conditions,proto3" json:"Conditions,omitempty"` + JoinMapTag int32 `protobuf:"varint,6,opt,name=JoinMapTag,proto3" json:"JoinMapTag,omitempty"` + JoinMapRefCnt int32 `protobuf:"varint,7,opt,name=JoinMapRefCnt,proto3" json:"JoinMapRefCnt,omitempty"` + RuntimeFilterSpec *plan.RuntimeFilterSpec `protobuf:"bytes,8,opt,name=RuntimeFilterSpec,proto3" json:"RuntimeFilterSpec,omitempty"` + IsDedup bool `protobuf:"varint,9,opt,name=is_dedup,json=isDedup,proto3" json:"is_dedup,omitempty"` + OnDuplicateAction plan.Node_OnDuplicateAction `protobuf:"varint,10,opt,name=on_duplicate_action,json=onDuplicateAction,proto3,enum=plan.Node_OnDuplicateAction" json:"on_duplicate_action,omitempty"` + DedupColName string `protobuf:"bytes,11,opt,name=dedup_col_name,json=dedupColName,proto3" json:"dedup_col_name,omitempty"` + DedupColTypes []plan.Type `protobuf:"bytes,12,rep,name=dedup_col_types,json=dedupColTypes,proto3" json:"dedup_col_types"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *HashBuild) Reset() { *m = HashBuild{} } func (m *HashBuild) String() string { return proto.CompactTextString(m) } func (*HashBuild) ProtoMessage() {} func (*HashBuild) Descriptor() ([]byte, []int) { - return fileDescriptor_7ac67a7adf3df9c7, []int{41} + return fileDescriptor_7ac67a7adf3df9c7, []int{42} } func (m *HashBuild) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3896,24 +4011,56 @@ func (m *HashBuild) GetRuntimeFilterSpec() *plan.RuntimeFilterSpec { return nil } +func (m *HashBuild) GetIsDedup() bool { + if m != nil { + return m.IsDedup + } + return false +} + +func (m *HashBuild) GetOnDuplicateAction() plan.Node_OnDuplicateAction { + if m != nil { + return m.OnDuplicateAction + } + return plan.Node_ERROR +} + +func (m *HashBuild) GetDedupColName() string { + if m != nil { + return m.DedupColName + } + return "" +} + +func (m *HashBuild) GetDedupColTypes() []plan.Type { + if m != nil { + return m.DedupColTypes + } + return nil +} + type Shufflebuild struct { - HashOnPK bool `protobuf:"varint,1,opt,name=HashOnPK,proto3" json:"HashOnPK,omitempty"` - NeedBatches bool `protobuf:"varint,2,opt,name=NeedBatches,proto3" json:"NeedBatches,omitempty"` - NeedAllocateSels bool `protobuf:"varint,3,opt,name=NeedAllocateSels,proto3" json:"NeedAllocateSels,omitempty"` - Conditions []*plan.Expr `protobuf:"bytes,4,rep,name=Conditions,proto3" json:"Conditions,omitempty"` - RuntimeFilterSpec *plan.RuntimeFilterSpec `protobuf:"bytes,5,opt,name=RuntimeFilterSpec,proto3" json:"RuntimeFilterSpec,omitempty"` - JoinMapTag int32 `protobuf:"varint,6,opt,name=JoinMapTag,proto3" json:"JoinMapTag,omitempty"` - ShuffleIdx int32 `protobuf:"varint,7,opt,name=ShuffleIdx,proto3" json:"ShuffleIdx,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + HashOnPK bool `protobuf:"varint,1,opt,name=HashOnPK,proto3" json:"HashOnPK,omitempty"` + NeedBatches bool `protobuf:"varint,2,opt,name=NeedBatches,proto3" json:"NeedBatches,omitempty"` + NeedAllocateSels bool `protobuf:"varint,3,opt,name=NeedAllocateSels,proto3" json:"NeedAllocateSels,omitempty"` + Conditions []*plan.Expr `protobuf:"bytes,4,rep,name=Conditions,proto3" json:"Conditions,omitempty"` + RuntimeFilterSpec *plan.RuntimeFilterSpec `protobuf:"bytes,5,opt,name=RuntimeFilterSpec,proto3" json:"RuntimeFilterSpec,omitempty"` + JoinMapTag int32 `protobuf:"varint,6,opt,name=JoinMapTag,proto3" json:"JoinMapTag,omitempty"` + ShuffleIdx int32 `protobuf:"varint,7,opt,name=ShuffleIdx,proto3" json:"ShuffleIdx,omitempty"` + IsDedup bool `protobuf:"varint,8,opt,name=is_dedup,json=isDedup,proto3" json:"is_dedup,omitempty"` + OnDuplicateAction plan.Node_OnDuplicateAction `protobuf:"varint,9,opt,name=on_duplicate_action,json=onDuplicateAction,proto3,enum=plan.Node_OnDuplicateAction" json:"on_duplicate_action,omitempty"` + DedupColName string `protobuf:"bytes,10,opt,name=dedup_col_name,json=dedupColName,proto3" json:"dedup_col_name,omitempty"` + DedupColTypes []plan.Type `protobuf:"bytes,11,rep,name=dedup_col_types,json=dedupColTypes,proto3" json:"dedup_col_types"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *Shufflebuild) Reset() { *m = Shufflebuild{} } func (m *Shufflebuild) String() string { return proto.CompactTextString(m) } func (*Shufflebuild) ProtoMessage() {} func (*Shufflebuild) Descriptor() ([]byte, []int) { - return fileDescriptor_7ac67a7adf3df9c7, []int{42} + return fileDescriptor_7ac67a7adf3df9c7, []int{43} } func (m *Shufflebuild) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3991,6 +4138,34 @@ func (m *Shufflebuild) GetShuffleIdx() int32 { return 0 } +func (m *Shufflebuild) GetIsDedup() bool { + if m != nil { + return m.IsDedup + } + return false +} + +func (m *Shufflebuild) GetOnDuplicateAction() plan.Node_OnDuplicateAction { + if m != nil { + return m.OnDuplicateAction + } + return plan.Node_ERROR +} + +func (m *Shufflebuild) GetDedupColName() string { + if m != nil { + return m.DedupColName + } + return "" +} + +func (m *Shufflebuild) GetDedupColTypes() []plan.Type { + if m != nil { + return m.DedupColTypes + } + return nil +} + type Indexbuild struct { RuntimeFilterSpec *plan.RuntimeFilterSpec `protobuf:"bytes,1,opt,name=RuntimeFilterSpec,proto3" json:"RuntimeFilterSpec,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` @@ -4002,7 +4177,7 @@ func (m *Indexbuild) Reset() { *m = Indexbuild{} } func (m *Indexbuild) String() string { return proto.CompactTextString(m) } func (*Indexbuild) ProtoMessage() {} func (*Indexbuild) Descriptor() ([]byte, []int) { - return fileDescriptor_7ac67a7adf3df9c7, []int{43} + return fileDescriptor_7ac67a7adf3df9c7, []int{44} } func (m *Indexbuild) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4052,7 +4227,7 @@ func (m *SampleFunc) Reset() { *m = SampleFunc{} } func (m *SampleFunc) String() string { return proto.CompactTextString(m) } func (*SampleFunc) ProtoMessage() {} func (*SampleFunc) Descriptor() ([]byte, []int) { - return fileDescriptor_7ac67a7adf3df9c7, []int{44} + return fileDescriptor_7ac67a7adf3df9c7, []int{45} } func (m *SampleFunc) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4118,11 +4293,12 @@ type Instruction struct { Connect *Connector `protobuf:"bytes,4,opt,name=connect,proto3" json:"connect,omitempty"` Dispatch *Dispatch `protobuf:"bytes,5,opt,name=dispatch,proto3" json:"dispatch,omitempty"` Agg *Group `protobuf:"bytes,6,opt,name=agg,proto3" json:"agg,omitempty"` + Join *Join `protobuf:"bytes,7,opt,name=join,proto3" json:"join,omitempty"` LeftJoin *LeftJoin `protobuf:"bytes,8,opt,name=left_join,json=leftJoin,proto3" json:"left_join,omitempty"` SemiJoin *SemiJoin `protobuf:"bytes,9,opt,name=semi_join,json=semiJoin,proto3" json:"semi_join,omitempty"` SingleJoin *SingleJoin `protobuf:"bytes,10,opt,name=single_join,json=singleJoin,proto3" json:"single_join,omitempty"` MarkJoin *MarkJoin `protobuf:"bytes,11,opt,name=mark_join,json=markJoin,proto3" json:"mark_join,omitempty"` - Join *Join `protobuf:"bytes,12,opt,name=join,proto3" json:"join,omitempty"` + DedupJoin *DedupJoin `protobuf:"bytes,12,opt,name=dedup_join,json=dedupJoin,proto3" json:"dedup_join,omitempty"` Product *Product `protobuf:"bytes,13,opt,name=product,proto3" json:"product,omitempty"` TableFunction *TableFunction `protobuf:"bytes,14,opt,name=table_function,json=tableFunction,proto3" json:"table_function,omitempty"` ExternalScan *ExternalScan `protobuf:"bytes,16,opt,name=external_scan,json=externalScan,proto3" json:"external_scan,omitempty"` @@ -4163,7 +4339,8 @@ type Instruction struct { ShuffleBuild *Shufflebuild `protobuf:"bytes,49,opt,name=shuffle_build,json=shuffleBuild,proto3" json:"shuffle_build,omitempty"` IndexBuild *Indexbuild `protobuf:"bytes,50,opt,name=index_build,json=indexBuild,proto3" json:"index_build,omitempty"` Apply *Apply `protobuf:"bytes,51,opt,name=apply,proto3" json:"apply,omitempty"` - PostDml *PostDml `protobuf:"bytes,52,opt,name=post_dml,json=postDml,proto3" json:"post_dml,omitempty"` + MultiUpdate *MultiUpdate `protobuf:"bytes,52,opt,name=multi_update,json=multiUpdate,proto3" json:"multi_update,omitempty"` + PostDml *PostDml `protobuf:"bytes,53,opt,name=post_dml,json=postDml,proto3" json:"post_dml,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -4173,7 +4350,7 @@ func (m *Instruction) Reset() { *m = Instruction{} } func (m *Instruction) String() string { return proto.CompactTextString(m) } func (*Instruction) ProtoMessage() {} func (*Instruction) Descriptor() ([]byte, []int) { - return fileDescriptor_7ac67a7adf3df9c7, []int{45} + return fileDescriptor_7ac67a7adf3df9c7, []int{46} } func (m *Instruction) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4244,6 +4421,13 @@ func (m *Instruction) GetAgg() *Group { return nil } +func (m *Instruction) GetJoin() *Join { + if m != nil { + return m.Join + } + return nil +} + func (m *Instruction) GetLeftJoin() *LeftJoin { if m != nil { return m.LeftJoin @@ -4272,9 +4456,9 @@ func (m *Instruction) GetMarkJoin() *MarkJoin { return nil } -func (m *Instruction) GetJoin() *Join { +func (m *Instruction) GetDedupJoin() *DedupJoin { if m != nil { - return m.Join + return m.DedupJoin } return nil } @@ -4545,6 +4729,13 @@ func (m *Instruction) GetApply() *Apply { return nil } +func (m *Instruction) GetMultiUpdate() *MultiUpdate { + if m != nil { + return m.MultiUpdate + } + return nil +} + func (m *Instruction) GetPostDml() *PostDml { if m != nil { return m.PostDml @@ -4563,7 +4754,7 @@ func (m *AnalysisList) Reset() { *m = AnalysisList{} } func (m *AnalysisList) String() string { return proto.CompactTextString(m) } func (*AnalysisList) ProtoMessage() {} func (*AnalysisList) Descriptor() ([]byte, []int) { - return fileDescriptor_7ac67a7adf3df9c7, []int{46} + return fileDescriptor_7ac67a7adf3df9c7, []int{47} } func (m *AnalysisList) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4620,7 +4811,7 @@ func (m *Source) Reset() { *m = Source{} } func (m *Source) String() string { return proto.CompactTextString(m) } func (*Source) ProtoMessage() {} func (*Source) Descriptor() ([]byte, []int) { - return fileDescriptor_7ac67a7adf3df9c7, []int{47} + return fileDescriptor_7ac67a7adf3df9c7, []int{48} } func (m *Source) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4740,7 +4931,7 @@ func (m *NodeInfo) Reset() { *m = NodeInfo{} } func (m *NodeInfo) String() string { return proto.CompactTextString(m) } func (*NodeInfo) ProtoMessage() {} func (*NodeInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_7ac67a7adf3df9c7, []int{48} + return fileDescriptor_7ac67a7adf3df9c7, []int{49} } func (m *NodeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4812,7 +5003,7 @@ func (m *ProcessLimitation) Reset() { *m = ProcessLimitation{} } func (m *ProcessLimitation) String() string { return proto.CompactTextString(m) } func (*ProcessLimitation) ProtoMessage() {} func (*ProcessLimitation) Descriptor() ([]byte, []int) { - return fileDescriptor_7ac67a7adf3df9c7, []int{49} + return fileDescriptor_7ac67a7adf3df9c7, []int{50} } func (m *ProcessLimitation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4890,7 +5081,7 @@ func (m *PrepareParamInfo) Reset() { *m = PrepareParamInfo{} } func (m *PrepareParamInfo) String() string { return proto.CompactTextString(m) } func (*PrepareParamInfo) ProtoMessage() {} func (*PrepareParamInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_7ac67a7adf3df9c7, []int{50} + return fileDescriptor_7ac67a7adf3df9c7, []int{51} } func (m *PrepareParamInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4966,7 +5157,7 @@ func (m *ProcessInfo) Reset() { *m = ProcessInfo{} } func (m *ProcessInfo) String() string { return proto.CompactTextString(m) } func (*ProcessInfo) ProtoMessage() {} func (*ProcessInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_7ac67a7adf3df9c7, []int{51} + return fileDescriptor_7ac67a7adf3df9c7, []int{52} } func (m *ProcessInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -5077,7 +5268,7 @@ func (m *SessionInfo) Reset() { *m = SessionInfo{} } func (m *SessionInfo) String() string { return proto.CompactTextString(m) } func (*SessionInfo) ProtoMessage() {} func (*SessionInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_7ac67a7adf3df9c7, []int{52} + return fileDescriptor_7ac67a7adf3df9c7, []int{53} } func (m *SessionInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -5183,7 +5374,7 @@ func (m *SessionLoggerInfo) Reset() { *m = SessionLoggerInfo{} } func (m *SessionLoggerInfo) String() string { return proto.CompactTextString(m) } func (*SessionLoggerInfo) ProtoMessage() {} func (*SessionLoggerInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_7ac67a7adf3df9c7, []int{53} + return fileDescriptor_7ac67a7adf3df9c7, []int{54} } func (m *SessionLoggerInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -5264,7 +5455,7 @@ func (m *Pipeline) Reset() { *m = Pipeline{} } func (m *Pipeline) String() string { return proto.CompactTextString(m) } func (*Pipeline) ProtoMessage() {} func (*Pipeline) Descriptor() ([]byte, []int) { - return fileDescriptor_7ac67a7adf3df9c7, []int{54} + return fileDescriptor_7ac67a7adf3df9c7, []int{55} } func (m *Pipeline) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -5403,7 +5594,7 @@ func (m *WrapNode) Reset() { *m = WrapNode{} } func (m *WrapNode) String() string { return proto.CompactTextString(m) } func (*WrapNode) ProtoMessage() {} func (*WrapNode) Descriptor() ([]byte, []int) { - return fileDescriptor_7ac67a7adf3df9c7, []int{55} + return fileDescriptor_7ac67a7adf3df9c7, []int{56} } func (m *WrapNode) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -5459,7 +5650,7 @@ func (m *UuidToRegIdx) Reset() { *m = UuidToRegIdx{} } func (m *UuidToRegIdx) String() string { return proto.CompactTextString(m) } func (*UuidToRegIdx) ProtoMessage() {} func (*UuidToRegIdx) Descriptor() ([]byte, []int) { - return fileDescriptor_7ac67a7adf3df9c7, []int{56} + return fileDescriptor_7ac67a7adf3df9c7, []int{57} } func (m *UuidToRegIdx) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -5523,7 +5714,7 @@ func (m *Apply) Reset() { *m = Apply{} } func (m *Apply) String() string { return proto.CompactTextString(m) } func (*Apply) ProtoMessage() {} func (*Apply) Descriptor() ([]byte, []int) { - return fileDescriptor_7ac67a7adf3df9c7, []int{57} + return fileDescriptor_7ac67a7adf3df9c7, []int{58} } func (m *Apply) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -5620,6 +5811,7 @@ func init() { proto.RegisterType((*SemiJoin)(nil), "pipeline.SemiJoin") proto.RegisterType((*SingleJoin)(nil), "pipeline.SingleJoin") proto.RegisterType((*MarkJoin)(nil), "pipeline.MarkJoin") + proto.RegisterType((*DedupJoin)(nil), "pipeline.DedupJoin") proto.RegisterType((*Product)(nil), "pipeline.Product") proto.RegisterType((*ProductL2)(nil), "pipeline.ProductL2") proto.RegisterType((*IndexJoin)(nil), "pipeline.IndexJoin") @@ -5654,353 +5846,365 @@ func init() { func init() { proto.RegisterFile("pipeline.proto", fileDescriptor_7ac67a7adf3df9c7) } var fileDescriptor_7ac67a7adf3df9c7 = []byte{ - // 5525 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x7c, 0xcd, 0x6f, 0x1d, 0x47, - 0x72, 0xb8, 0xde, 0xf7, 0x4c, 0xbd, 0x0f, 0x3e, 0xb6, 0xbe, 0x9e, 0x25, 0x5b, 0xa2, 0xc6, 0x96, - 0xcc, 0x95, 0x6d, 0xca, 0xa6, 0xd7, 0xbf, 0x9f, 0x93, 0xcd, 0xae, 0x97, 0x22, 0xa5, 0x5d, 0xae, - 0x45, 0x9a, 0x69, 0x52, 0x31, 0xb2, 0x08, 0x32, 0x18, 0xce, 0xf4, 0x7b, 0x9c, 0xe5, 0xbc, 0xe9, - 0xd1, 0xcc, 0x3c, 0x89, 0xd4, 0x29, 0x40, 0x72, 0xcd, 0x29, 0xff, 0x40, 0xb0, 0x97, 0x20, 0x08, - 0xf2, 0x81, 0xe4, 0x12, 0x24, 0xc8, 0x7d, 0x73, 0xdb, 0x53, 0x0e, 0x39, 0x04, 0xc1, 0xe6, 0x96, - 0x8f, 0x5b, 0x12, 0xe4, 0x12, 0x24, 0xa8, 0xea, 0x9e, 0x8f, 0xf7, 0x21, 0xea, 0xc3, 0x76, 0xb0, - 0x0b, 0xf8, 0xd6, 0x5d, 0x55, 0xdd, 0xd3, 0x5d, 0x55, 0x5d, 0x5d, 0x5d, 0x5d, 0x3d, 0xd0, 0x8b, - 0xfc, 0x48, 0x04, 0x7e, 0x28, 0xd6, 0xa2, 0x58, 0xa6, 0x92, 0x19, 0x59, 0xfd, 0xca, 0x7b, 0x23, - 0x3f, 0x3d, 0x9a, 0x1c, 0xae, 0xb9, 0x72, 0x7c, 0x67, 0x24, 0x47, 0xf2, 0x0e, 0x11, 0x1c, 0x4e, - 0x86, 0x54, 0xa3, 0x0a, 0x95, 0x54, 0xc3, 0x2b, 0x10, 0x05, 0x4e, 0xa8, 0xcb, 0x4b, 0xa9, 0x3f, - 0x16, 0x49, 0xea, 0x8c, 0xa3, 0x0c, 0x19, 0x48, 0xf7, 0x58, 0x97, 0xcd, 0xf4, 0x44, 0xd3, 0x59, - 0xff, 0x53, 0x81, 0xd6, 0x8e, 0x48, 0x12, 0x67, 0x24, 0x98, 0x05, 0xb5, 0xc4, 0xf7, 0x06, 0x95, - 0x95, 0xca, 0x6a, 0x6f, 0xbd, 0xbf, 0x96, 0x0f, 0x6b, 0x3f, 0x75, 0xd2, 0x49, 0xc2, 0x11, 0x89, - 0x34, 0xee, 0xd8, 0x1b, 0x54, 0x67, 0x69, 0x76, 0x44, 0x7a, 0x24, 0x3d, 0x8e, 0x48, 0xd6, 0x87, - 0x9a, 0x88, 0xe3, 0x41, 0x6d, 0xa5, 0xb2, 0xda, 0xe1, 0x58, 0x64, 0x0c, 0xea, 0x9e, 0x93, 0x3a, - 0x83, 0x3a, 0x81, 0xa8, 0xcc, 0xde, 0x82, 0x5e, 0x14, 0x4b, 0xd7, 0xf6, 0xc3, 0xa1, 0xb4, 0x09, - 0xdb, 0x20, 0x6c, 0x07, 0xa1, 0xdb, 0xe1, 0x50, 0x6e, 0x21, 0xd5, 0x00, 0x5a, 0x4e, 0xe8, 0x04, - 0xa7, 0x89, 0x18, 0x34, 0x09, 0x9d, 0x55, 0x59, 0x0f, 0xaa, 0xbe, 0x37, 0x68, 0xad, 0x54, 0x56, - 0xeb, 0xbc, 0xea, 0x7b, 0xf8, 0x8d, 0xc9, 0xc4, 0xf7, 0x06, 0x86, 0xfa, 0x06, 0x96, 0x99, 0x05, - 0x9d, 0x50, 0x08, 0x6f, 0x57, 0xa6, 0x5c, 0x44, 0xc1, 0xe9, 0xc0, 0x5c, 0xa9, 0xac, 0x1a, 0x7c, - 0x0a, 0x66, 0x3d, 0x04, 0x73, 0x53, 0x86, 0xa1, 0x70, 0x53, 0x19, 0xb3, 0xeb, 0xd0, 0xce, 0xa6, - 0x64, 0x6b, 0x56, 0x34, 0x38, 0x64, 0xa0, 0x6d, 0x8f, 0xbd, 0x0d, 0x4b, 0x6e, 0x46, 0x6d, 0xfb, - 0xa1, 0x27, 0x4e, 0x88, 0x17, 0x0d, 0xde, 0xcb, 0xc1, 0xdb, 0x08, 0xb5, 0xfe, 0xb5, 0x0a, 0xad, - 0xfd, 0xa3, 0xc9, 0x70, 0x18, 0x08, 0xf6, 0x16, 0x74, 0x75, 0x71, 0x53, 0x06, 0xdb, 0xde, 0x89, - 0xee, 0x77, 0x1a, 0xc8, 0x56, 0xa0, 0xad, 0x01, 0x07, 0xa7, 0x91, 0xd0, 0xdd, 0x96, 0x41, 0xd3, - 0xfd, 0xec, 0xf8, 0x21, 0xb1, 0xb8, 0xc6, 0xa7, 0x81, 0x33, 0x54, 0xce, 0x09, 0x71, 0x7d, 0x9a, - 0xca, 0xa1, 0xaf, 0x6d, 0x04, 0xfe, 0x63, 0xc1, 0xc5, 0x68, 0x33, 0x4c, 0x89, 0xf7, 0x0d, 0x5e, - 0x06, 0xb1, 0x75, 0xb8, 0x98, 0xa8, 0x26, 0x76, 0xec, 0x84, 0x23, 0x91, 0xd8, 0x13, 0x3f, 0x4c, - 0xff, 0xdf, 0x37, 0x07, 0xcd, 0x95, 0xda, 0x6a, 0x9d, 0x9f, 0xd7, 0x48, 0x4e, 0xb8, 0x87, 0x84, - 0x62, 0xef, 0xc3, 0x85, 0x99, 0x36, 0xaa, 0x49, 0x6b, 0xa5, 0xb6, 0x5a, 0xe3, 0x6c, 0xaa, 0xc9, - 0x36, 0xb5, 0xb8, 0x07, 0xcb, 0xf1, 0x24, 0x44, 0x6d, 0xbd, 0xef, 0x07, 0xa9, 0x88, 0xf7, 0x23, - 0xe1, 0x92, 0x0c, 0xdb, 0xeb, 0x97, 0xd7, 0x48, 0xa1, 0xf9, 0x2c, 0x9a, 0xcf, 0xb7, 0xb0, 0xfe, - 0xab, 0x0a, 0xc6, 0x96, 0x9f, 0x44, 0x4e, 0xea, 0x1e, 0xb1, 0xcb, 0xd0, 0x1a, 0x4e, 0x42, 0xb7, - 0x90, 0x60, 0x13, 0xab, 0xdb, 0x1e, 0xfb, 0x15, 0x58, 0x0a, 0xa4, 0xeb, 0x04, 0x76, 0x2e, 0xac, - 0x41, 0x75, 0xa5, 0xb6, 0xda, 0x5e, 0x3f, 0x5f, 0x68, 0x72, 0xae, 0x0c, 0xbc, 0x47, 0xb4, 0x85, - 0x72, 0x7c, 0x1b, 0xfa, 0xb1, 0x18, 0xcb, 0x54, 0x94, 0x9a, 0xd7, 0xa8, 0x39, 0x2b, 0x9a, 0x7f, - 0x1e, 0x3b, 0xd1, 0xae, 0xf4, 0x04, 0x5f, 0x52, 0xb4, 0x45, 0xf3, 0x0f, 0x4a, 0xfc, 0x14, 0x23, - 0xdb, 0xf7, 0x4e, 0x6c, 0xfa, 0xc0, 0xa0, 0xbe, 0x52, 0x5b, 0x6d, 0x14, 0xcc, 0x11, 0xa3, 0x6d, - 0xef, 0xe4, 0x01, 0x62, 0xd8, 0x87, 0x70, 0x69, 0xb6, 0x89, 0xea, 0x75, 0xd0, 0xa0, 0x36, 0xe7, - 0xa7, 0xda, 0x70, 0x42, 0xb1, 0x1b, 0xd0, 0xc9, 0x1a, 0xa5, 0xa8, 0x48, 0x4d, 0x25, 0xda, 0xa4, - 0xa4, 0x48, 0x97, 0xa1, 0xe5, 0x27, 0x76, 0xe2, 0x87, 0xc7, 0xb4, 0x80, 0x0c, 0xde, 0xf4, 0x93, - 0x7d, 0x3f, 0x3c, 0x66, 0xaf, 0x81, 0x11, 0x0b, 0x57, 0x61, 0x0c, 0xc2, 0xb4, 0x62, 0xe1, 0x12, - 0xea, 0x32, 0x60, 0xd1, 0x76, 0x53, 0xa1, 0x97, 0x51, 0x33, 0x16, 0xee, 0x66, 0x2a, 0xac, 0x04, - 0x1a, 0x3b, 0x22, 0x1e, 0x09, 0x76, 0x05, 0x0c, 0x6c, 0xb8, 0xef, 0x3a, 0x21, 0xf1, 0xdd, 0xe0, - 0x79, 0x1d, 0xd7, 0x71, 0xe4, 0xc4, 0xa9, 0xef, 0x04, 0xa4, 0xd8, 0x06, 0xcf, 0xaa, 0xec, 0x2a, - 0x98, 0x49, 0xea, 0xc4, 0x29, 0xce, 0x8e, 0x14, 0xba, 0xc1, 0x0d, 0x02, 0xe0, 0x9a, 0xb8, 0x0c, - 0x2d, 0x11, 0x7a, 0x84, 0xaa, 0x2b, 0x49, 0x8a, 0xd0, 0xdb, 0xf6, 0x4e, 0xac, 0xbf, 0xa8, 0x40, - 0x77, 0x67, 0x12, 0xa4, 0xfe, 0x46, 0x3c, 0x9a, 0x88, 0x71, 0x98, 0xe2, 0xfa, 0xdf, 0xf2, 0x93, - 0x54, 0x7f, 0x99, 0xca, 0x6c, 0x15, 0xcc, 0xef, 0xc5, 0x72, 0x12, 0xdd, 0x3b, 0x89, 0x32, 0x49, - 0x83, 0x52, 0x2a, 0x84, 0xf0, 0x02, 0xc9, 0xde, 0x85, 0xf6, 0x67, 0xb1, 0x27, 0xe2, 0xbb, 0xa7, - 0x44, 0x5b, 0x9b, 0xa3, 0x2d, 0xa3, 0xd9, 0xeb, 0x60, 0xee, 0x8b, 0xc8, 0x89, 0x1d, 0x54, 0x01, - 0x1c, 0x98, 0xc9, 0x0b, 0x00, 0xce, 0x95, 0x88, 0xb7, 0x3d, 0xbd, 0xac, 0xb2, 0xaa, 0x35, 0x02, - 0x73, 0x63, 0x34, 0x8a, 0xc5, 0xc8, 0x49, 0xc9, 0x80, 0xc9, 0x88, 0x86, 0x5b, 0xe3, 0x55, 0x19, - 0x91, 0x91, 0xc4, 0x09, 0x28, 0xfe, 0x50, 0x99, 0x5d, 0x83, 0xba, 0x58, 0x3c, 0x1e, 0x82, 0xb3, - 0x4b, 0xd0, 0x74, 0x65, 0x38, 0xf4, 0x47, 0xda, 0xb4, 0xea, 0x9a, 0xf5, 0xbb, 0x35, 0x68, 0xd0, - 0xe4, 0x90, 0xbd, 0x68, 0xee, 0x6c, 0xf1, 0xd8, 0x09, 0x32, 0xa9, 0x20, 0xe0, 0xde, 0x63, 0x27, - 0x60, 0x2b, 0xd0, 0xc0, 0x6e, 0x92, 0x05, 0xbc, 0x51, 0x08, 0x76, 0x0b, 0x1a, 0xa8, 0x44, 0xc9, - 0xf4, 0x08, 0x50, 0x89, 0xee, 0xd6, 0x7f, 0xf2, 0x0f, 0xd7, 0xcf, 0x71, 0x85, 0x66, 0x6f, 0x43, - 0xdd, 0x19, 0x8d, 0x12, 0xd2, 0xe5, 0xa9, 0xe5, 0x94, 0xcf, 0x97, 0x13, 0x01, 0xfb, 0x08, 0x4c, - 0x25, 0x37, 0xa4, 0x6e, 0x10, 0xf5, 0xe5, 0xd2, 0x36, 0x52, 0x16, 0x29, 0x2f, 0x28, 0x91, 0xe3, - 0x7e, 0xa2, 0x2d, 0x18, 0x69, 0xb4, 0xc1, 0x0b, 0x00, 0xda, 0xf9, 0x28, 0x16, 0x1b, 0x41, 0x20, - 0xdd, 0x7d, 0xff, 0xa9, 0xd0, 0xbb, 0xc2, 0x14, 0x8c, 0xdd, 0x82, 0xde, 0x9e, 0x52, 0x39, 0x2e, - 0x92, 0x49, 0x90, 0x26, 0x7a, 0xa7, 0x98, 0x81, 0xb2, 0x35, 0x60, 0x53, 0x90, 0x03, 0x9a, 0xbe, - 0xb9, 0x52, 0x5b, 0xed, 0xf2, 0x05, 0x18, 0xf6, 0x26, 0x74, 0x47, 0xc8, 0x69, 0x3f, 0x1c, 0xd9, - 0xc3, 0xc0, 0x19, 0x0d, 0x60, 0xa5, 0x86, 0x9b, 0x4c, 0x06, 0xbc, 0x1f, 0x38, 0x23, 0xeb, 0xdf, - 0xab, 0xd0, 0xdc, 0x0e, 0x13, 0x11, 0xa7, 0xb8, 0x4a, 0x9c, 0xe1, 0x50, 0xb8, 0xa9, 0x50, 0xd6, - 0xa9, 0xce, 0xf3, 0x3a, 0xce, 0xf2, 0x40, 0x7e, 0x1e, 0xfb, 0xa9, 0xd8, 0xff, 0x50, 0xeb, 0x41, - 0x01, 0x60, 0xb7, 0x61, 0xd9, 0xf1, 0x3c, 0x3b, 0xa3, 0xb6, 0x63, 0xf9, 0x24, 0xa1, 0x15, 0x63, - 0xf0, 0x25, 0xc7, 0xf3, 0x36, 0x34, 0x9c, 0xcb, 0x27, 0x09, 0xbb, 0x01, 0xb5, 0x58, 0x0c, 0x49, - 0x2b, 0xda, 0xeb, 0x4b, 0x4a, 0x6a, 0x9f, 0x1d, 0xfe, 0x48, 0xb8, 0x29, 0x17, 0x43, 0x8e, 0x38, - 0x76, 0x01, 0x1a, 0x4e, 0x9a, 0xc6, 0x4a, 0x0a, 0x26, 0x57, 0x15, 0xb6, 0x06, 0xe7, 0x69, 0x65, - 0xa6, 0xbe, 0x0c, 0xed, 0xd4, 0x39, 0x0c, 0x70, 0x23, 0x4c, 0xb4, 0xcd, 0x5f, 0xce, 0x51, 0x07, - 0x88, 0xd9, 0xf6, 0x12, 0xdc, 0x25, 0x66, 0xe9, 0x43, 0x67, 0x2c, 0x12, 0x32, 0xf9, 0x26, 0x3f, - 0x3f, 0xdd, 0x62, 0x17, 0x51, 0xc8, 0xb2, 0xa2, 0x0d, 0xae, 0x6d, 0x83, 0x96, 0x49, 0x27, 0x07, - 0xe2, 0xd2, 0xbf, 0x08, 0x4d, 0x3f, 0xb1, 0x45, 0xe8, 0x69, 0x73, 0xd3, 0xf0, 0x93, 0x7b, 0xa1, - 0xc7, 0xde, 0x01, 0x53, 0x7d, 0xc5, 0x13, 0xc3, 0x01, 0xd0, 0xf4, 0x7a, 0x5a, 0x29, 0x11, 0xbc, - 0x25, 0x86, 0xdc, 0x48, 0x75, 0xc9, 0xfa, 0xab, 0x2a, 0xb4, 0x49, 0x87, 0x1e, 0x46, 0x1e, 0x2e, - 0xb9, 0x37, 0xa1, 0x3b, 0xcd, 0x3d, 0x25, 0x80, 0x8e, 0x53, 0x66, 0xdd, 0xd9, 0x42, 0x18, 0x40, - 0x6b, 0xfb, 0xee, 0xc4, 0x3d, 0x16, 0x29, 0xb1, 0xbe, 0xcb, 0xb3, 0x2a, 0x62, 0x76, 0x35, 0xa6, - 0xae, 0x30, 0xba, 0xca, 0xee, 0x01, 0xec, 0x8b, 0xd1, 0x58, 0x84, 0xe9, 0x8e, 0x13, 0x69, 0xa5, - 0xbf, 0x39, 0xa3, 0xf4, 0x6a, 0x84, 0x6b, 0x05, 0xdd, 0xbd, 0x30, 0x8d, 0x4f, 0x79, 0xa9, 0x21, - 0x5b, 0x03, 0x98, 0x10, 0x95, 0xed, 0xa6, 0x27, 0x24, 0x91, 0x5c, 0xb4, 0xaa, 0xf5, 0x66, 0x7a, - 0xc2, 0xcd, 0x49, 0x56, 0xbc, 0xf2, 0x6d, 0x58, 0x9a, 0xe9, 0x0e, 0x5d, 0xb3, 0x63, 0x71, 0x4a, - 0xd3, 0x36, 0x39, 0x16, 0x51, 0x0b, 0x1e, 0x3b, 0xc1, 0x24, 0xf3, 0x37, 0x54, 0xe5, 0x97, 0xab, - 0x1f, 0x57, 0xac, 0x37, 0xa0, 0xb1, 0x11, 0xc7, 0x0e, 0x91, 0x38, 0x58, 0x18, 0x54, 0x68, 0xd3, - 0x51, 0x15, 0xcb, 0x85, 0x1a, 0x0e, 0xea, 0x26, 0x54, 0xc7, 0x11, 0x61, 0xda, 0xeb, 0x17, 0x4b, - 0x73, 0x72, 0xa2, 0xb5, 0x1d, 0x3d, 0x87, 0xea, 0x38, 0xba, 0xf2, 0x11, 0xb4, 0x76, 0x5e, 0x61, - 0x0c, 0xff, 0x51, 0x07, 0x63, 0x4b, 0x04, 0x02, 0x95, 0x02, 0x57, 0x79, 0x59, 0xc7, 0x33, 0xe1, - 0x4d, 0xe9, 0xbd, 0x05, 0x1d, 0xb5, 0x0d, 0x52, 0x2b, 0xa1, 0xe5, 0x37, 0x05, 0x7b, 0x25, 0x11, - 0xbe, 0x0e, 0x10, 0xcb, 0x27, 0xb6, 0xaf, 0xf6, 0x22, 0x65, 0xd6, 0x8d, 0x58, 0x3e, 0xd9, 0xc6, - 0xdd, 0xe8, 0xff, 0x64, 0xd1, 0xfc, 0x7f, 0x18, 0x94, 0x16, 0x0d, 0xfa, 0x98, 0xb6, 0x1f, 0xda, - 0x87, 0xe8, 0xf0, 0xe8, 0xf5, 0x53, 0xf4, 0x49, 0x2e, 0xe8, 0x76, 0x78, 0x97, 0xbc, 0x21, 0x6d, - 0x0a, 0xcc, 0x33, 0x4c, 0xc1, 0x42, 0xcb, 0x02, 0x8b, 0x2d, 0xcb, 0xdd, 0x29, 0x65, 0x6e, 0x93, - 0xe0, 0xad, 0x42, 0xf0, 0x99, 0xb4, 0xce, 0xd4, 0xe4, 0x1b, 0xd0, 0x71, 0x9d, 0xd0, 0x4e, 0xe3, - 0x49, 0xe8, 0x3a, 0xa9, 0x18, 0x74, 0xe8, 0x53, 0x6d, 0xd7, 0x09, 0x0f, 0x34, 0xa8, 0xb4, 0xfc, - 0xbb, 0xe5, 0xe5, 0x7f, 0x0b, 0x96, 0xa2, 0xd8, 0x1f, 0x3b, 0xf1, 0xa9, 0x7d, 0x2c, 0x4e, 0x49, - 0x18, 0x3d, 0xe5, 0x4c, 0x6b, 0xf0, 0xa7, 0xe2, 0x74, 0xdb, 0xfb, 0xc2, 0xba, 0xff, 0x77, 0x55, - 0x30, 0xf7, 0x62, 0xa1, 0x4d, 0xf6, 0x75, 0x68, 0x27, 0xee, 0x91, 0x18, 0x3b, 0x24, 0x25, 0xdd, - 0x03, 0x28, 0x10, 0x0a, 0x67, 0xda, 0x28, 0x55, 0xcf, 0x36, 0x4a, 0x38, 0x0e, 0xe5, 0xea, 0xe0, - 0x62, 0xc2, 0x62, 0x61, 0x89, 0xeb, 0x65, 0x4b, 0xbc, 0x02, 0x9d, 0x23, 0x27, 0xb1, 0x9d, 0x49, - 0x2a, 0x6d, 0x57, 0x06, 0xa4, 0x74, 0x06, 0x87, 0x23, 0x27, 0xd9, 0x98, 0xa4, 0x72, 0x53, 0x92, - 0xeb, 0xe4, 0x27, 0xb6, 0x5a, 0xf0, 0x7a, 0x53, 0x34, 0xfc, 0x44, 0xdb, 0xba, 0x35, 0x38, 0x2f, - 0x92, 0xd4, 0x1f, 0x3b, 0x5a, 0xa0, 0xb6, 0x2b, 0x27, 0x61, 0x4a, 0x5b, 0x63, 0x8d, 0x2f, 0xe7, - 0x28, 0x2e, 0x9f, 0x6c, 0x22, 0x82, 0xbd, 0x0f, 0x3d, 0x57, 0x8e, 0x23, 0x3b, 0x42, 0xbe, 0x92, - 0xd3, 0xa1, 0xbc, 0xf0, 0xb2, 0x53, 0xd0, 0x41, 0x8a, 0xbd, 0x63, 0xa1, 0xbc, 0xa0, 0x75, 0x58, - 0x72, 0x83, 0x49, 0x92, 0x8a, 0xd8, 0x3e, 0xd4, 0x4d, 0xcc, 0xb9, 0x26, 0x5d, 0x4d, 0xa2, 0x3c, - 0x27, 0x64, 0x6c, 0x6b, 0x4f, 0x26, 0xe9, 0xd6, 0x38, 0xc8, 0x14, 0xb3, 0xf2, 0xb2, 0x8a, 0x59, - 0x5d, 0xac, 0x98, 0x0b, 0x54, 0xa3, 0xb6, 0x40, 0x35, 0xd8, 0x2a, 0xf4, 0xcb, 0x74, 0x24, 0x52, - 0xe5, 0xc3, 0xf5, 0x0a, 0x42, 0x12, 0xab, 0xe2, 0xaf, 0xa7, 0x2c, 0x49, 0x23, 0xe3, 0xaf, 0xb6, - 0x22, 0x0a, 0xe9, 0x93, 0x86, 0x14, 0xcc, 0xd7, 0x1a, 0xf3, 0x4b, 0xf0, 0x5a, 0xde, 0xd2, 0x7e, - 0xe2, 0xa7, 0x47, 0x72, 0x92, 0xda, 0x43, 0x3a, 0xae, 0x24, 0xda, 0xe5, 0xbe, 0x94, 0xf5, 0xf4, - 0xb9, 0x42, 0xab, 0xc3, 0x0c, 0x39, 0x48, 0xc3, 0x49, 0x10, 0xd8, 0xa9, 0x38, 0x49, 0xb5, 0x08, - 0x06, 0x8a, 0x37, 0x9a, 0x6f, 0xf7, 0x27, 0x41, 0x70, 0x20, 0x4e, 0x52, 0xb4, 0xf6, 0xc6, 0x50, - 0x57, 0xac, 0xbf, 0xaf, 0x02, 0x3c, 0x90, 0xee, 0xf1, 0x81, 0x13, 0x8f, 0x44, 0x8a, 0x8e, 0x7c, - 0x66, 0x87, 0xb4, 0x9d, 0x6c, 0xa5, 0xca, 0xfa, 0xb0, 0x75, 0xb8, 0x94, 0xcd, 0xdf, 0x95, 0x01, - 0x1d, 0x2a, 0x94, 0x21, 0xd1, 0xcb, 0x80, 0x69, 0xac, 0x3a, 0x96, 0x92, 0x15, 0x61, 0x1f, 0x17, - 0xbc, 0xc5, 0x36, 0xe9, 0x69, 0x44, 0xbc, 0x5d, 0xe4, 0x10, 0x76, 0x8b, 0xe6, 0x07, 0xa7, 0x11, - 0x7b, 0x1f, 0x2e, 0xc6, 0x62, 0x18, 0x8b, 0xe4, 0xc8, 0x4e, 0x93, 0xf2, 0xc7, 0x94, 0x3f, 0xbf, - 0xac, 0x91, 0x07, 0x49, 0xfe, 0xad, 0xf7, 0xe1, 0xa2, 0xe2, 0xd4, 0xec, 0xf0, 0x94, 0xd5, 0x5d, - 0x56, 0xc8, 0xf2, 0xe8, 0xde, 0x00, 0x8a, 0x6e, 0x28, 0x4b, 0x9a, 0x79, 0x87, 0x01, 0x31, 0xe3, - 0x30, 0x10, 0xb8, 0xa1, 0x6f, 0x1e, 0xe1, 0x91, 0x73, 0x4b, 0x0c, 0x35, 0xf3, 0x0b, 0x00, 0xb3, - 0xa0, 0xbe, 0x23, 0x3d, 0x41, 0xac, 0xee, 0xad, 0xf7, 0xd6, 0x28, 0x4e, 0x82, 0x9c, 0x44, 0x28, - 0x27, 0x9c, 0xf5, 0x31, 0x34, 0x11, 0xf2, 0x19, 0xee, 0xc1, 0xad, 0x94, 0x38, 0x9c, 0xe8, 0x3d, - 0xef, 0x42, 0x61, 0xfa, 0x0a, 0xf6, 0xf3, 0x8c, 0xc8, 0xe2, 0xb0, 0x94, 0xdb, 0x91, 0x87, 0xa1, - 0xff, 0x68, 0x22, 0xd8, 0x27, 0xb0, 0x1c, 0xc5, 0x42, 0x6b, 0x8e, 0x3d, 0x39, 0xa6, 0xdd, 0x5c, - 0x2d, 0x82, 0x0b, 0x5a, 0xd0, 0x79, 0x8b, 0x63, 0x14, 0x72, 0x2f, 0x9a, 0xaa, 0x5b, 0x3f, 0x84, - 0xcb, 0x39, 0xc5, 0xbe, 0x70, 0x65, 0xe8, 0x39, 0xf1, 0x29, 0x99, 0xfc, 0x99, 0xbe, 0x93, 0x97, - 0xe9, 0x7b, 0x9f, 0xfa, 0xfe, 0x71, 0x0d, 0x7a, 0x9f, 0x85, 0x5b, 0x93, 0x28, 0xf0, 0xd1, 0x0c, - 0x7f, 0xaa, 0xac, 0xa4, 0xb2, 0x4e, 0x95, 0xb2, 0x75, 0x5a, 0x85, 0xbe, 0xfe, 0x0a, 0x4a, 0x49, - 0xd9, 0x16, 0x1d, 0x09, 0x51, 0xf0, 0x4d, 0x19, 0x28, 0xc3, 0xf2, 0x6d, 0xb8, 0x38, 0xa1, 0x99, - 0x2b, 0xca, 0x23, 0xe1, 0x1e, 0xdb, 0xcf, 0x38, 0xd4, 0x30, 0x45, 0x88, 0x4d, 0x91, 0x8c, 0xac, - 0xcc, 0x75, 0x68, 0x17, 0xcd, 0x33, 0x13, 0x09, 0x39, 0x21, 0x8d, 0x44, 0x86, 0xb6, 0x97, 0x0d, - 0x59, 0x6f, 0xd0, 0x68, 0x5c, 0x7b, 0xb2, 0x98, 0x09, 0xae, 0xfc, 0x5f, 0x87, 0xe5, 0x29, 0x4a, - 0x1a, 0x85, 0xf2, 0xa3, 0xde, 0x2b, 0xc4, 0x38, 0x3d, 0xfd, 0x72, 0x15, 0xc7, 0xa3, 0x36, 0xb3, - 0x25, 0x39, 0x0d, 0xcd, 0xac, 0xc1, 0x28, 0x94, 0xb1, 0xd0, 0x3a, 0x86, 0xd6, 0x80, 0xea, 0x57, - 0x76, 0xe1, 0xc2, 0xa2, 0x5e, 0x16, 0xec, 0x48, 0x2b, 0xe5, 0x1d, 0x69, 0xe6, 0x40, 0x56, 0xec, - 0x4e, 0x7f, 0x50, 0x81, 0xf6, 0xfd, 0xc9, 0xd3, 0xa7, 0xa7, 0xca, 0x66, 0xb0, 0x0e, 0x54, 0x76, - 0xa9, 0x97, 0x2a, 0xaf, 0xec, 0xe2, 0x99, 0x70, 0xef, 0x18, 0xed, 0x17, 0x75, 0x62, 0x72, 0x5d, - 0xc3, 0xa3, 0xdc, 0xde, 0xf1, 0xc1, 0x19, 0x2b, 0x57, 0xa1, 0xf1, 0x80, 0x72, 0x77, 0xe2, 0x07, - 0xe8, 0xd8, 0xe8, 0x45, 0x9a, 0xd7, 0xf1, 0x70, 0xb4, 0x3d, 0x54, 0xfa, 0x72, 0x3f, 0x96, 0x63, - 0xa5, 0xd1, 0xda, 0x34, 0x2e, 0xc0, 0x58, 0x7f, 0x5b, 0x83, 0xfa, 0x0f, 0xa4, 0x1f, 0xaa, 0xc0, - 0x42, 0x60, 0x07, 0xea, 0x84, 0x8e, 0xc2, 0x69, 0xc5, 0x22, 0x78, 0x80, 0x67, 0xdc, 0xd7, 0xc0, - 0x40, 0xc5, 0x08, 0xd4, 0xd9, 0x97, 0x50, 0xae, 0x54, 0xa8, 0xe2, 0xf8, 0x5b, 0x59, 0x78, 0xfc, - 0xcd, 0x4f, 0xa7, 0xf5, 0xe7, 0x9d, 0x4e, 0xcd, 0x40, 0x0c, 0x51, 0x55, 0x43, 0x4f, 0xfb, 0xdf, - 0xe5, 0xce, 0x0c, 0x44, 0x6e, 0xca, 0xd0, 0x63, 0xdf, 0x00, 0x88, 0xfd, 0xd1, 0x91, 0xa6, 0x6c, - 0xce, 0x47, 0x0c, 0x08, 0x4b, 0xa4, 0x1c, 0x5e, 0xd3, 0x61, 0x28, 0x6d, 0xd8, 0xed, 0x43, 0xe4, - 0x92, 0x9a, 0x47, 0x2b, 0x3b, 0xd8, 0x2e, 0x0e, 0x60, 0x5d, 0x9a, 0x0a, 0x60, 0x11, 0x77, 0x69, - 0xbe, 0xaf, 0x03, 0x6e, 0xef, 0x47, 0xb6, 0x0c, 0xed, 0x28, 0x0b, 0xc0, 0x18, 0x08, 0xf9, 0x2c, - 0xdc, 0x3b, 0x46, 0x33, 0xe7, 0x27, 0xb6, 0x8e, 0xe3, 0xe8, 0x53, 0x51, 0xe9, 0x10, 0xbc, 0x02, - 0x9d, 0x1f, 0x49, 0x3f, 0xb4, 0xc7, 0x4e, 0x64, 0xa7, 0x74, 0x0e, 0xa5, 0xe0, 0x25, 0xc2, 0x76, - 0x9c, 0xe8, 0xc0, 0x19, 0x91, 0x1f, 0xa3, 0x23, 0x43, 0xb8, 0x48, 0xda, 0x8a, 0x40, 0x83, 0x50, - 0xbc, 0x57, 0xc1, 0xa4, 0x2e, 0x28, 0x6e, 0xd4, 0x51, 0xb2, 0x47, 0x00, 0x72, 0xd4, 0xfa, 0xe7, - 0x2a, 0x18, 0x1b, 0x61, 0xea, 0x93, 0x3c, 0x2f, 0x41, 0x33, 0xa6, 0x43, 0xb0, 0x96, 0xa6, 0xae, - 0xe5, 0x12, 0xab, 0x3e, 0x43, 0x62, 0x53, 0x92, 0xa8, 0xbd, 0xb0, 0x24, 0xea, 0x67, 0x49, 0x62, - 0x9a, 0x6b, 0x8d, 0x33, 0xb9, 0x36, 0x17, 0x3a, 0xf8, 0x2a, 0xc4, 0x38, 0x2b, 0x09, 0xe3, 0x79, - 0x92, 0x30, 0x67, 0x25, 0x61, 0xfd, 0x59, 0x0d, 0x8c, 0x07, 0x62, 0x98, 0x7e, 0xbd, 0x78, 0x7e, - 0x51, 0x16, 0x8f, 0xf5, 0x6f, 0x35, 0x30, 0x39, 0xce, 0xf0, 0x2b, 0x94, 0xd9, 0x1d, 0x00, 0x92, - 0xc5, 0xd9, 0x82, 0x23, 0x79, 0xa9, 0xe8, 0xd4, 0x07, 0xd0, 0x56, 0x32, 0x51, 0x2d, 0x1a, 0xcf, - 0x68, 0xa1, 0x04, 0x77, 0x30, 0x2f, 0xef, 0xe6, 0x0b, 0xcb, 0xbb, 0xf5, 0xca, 0xf2, 0x36, 0xbe, - 0x0c, 0x79, 0x9b, 0x67, 0xca, 0x1b, 0x9e, 0x27, 0xef, 0xf6, 0xf3, 0xe4, 0xdd, 0x99, 0x93, 0xf7, - 0x8f, 0x6b, 0xd0, 0x25, 0x79, 0xef, 0x8b, 0xf1, 0x17, 0x33, 0x8a, 0x33, 0x42, 0xaa, 0xbd, 0xac, - 0x90, 0xea, 0x2f, 0x2c, 0xa4, 0xc6, 0x2b, 0x0b, 0xa9, 0xf9, 0x65, 0x08, 0xa9, 0x75, 0xa6, 0x90, - 0x8c, 0xe7, 0x09, 0xc9, 0x7c, 0xf9, 0x45, 0x99, 0x0b, 0xe9, 0x0b, 0xef, 0x5c, 0x5f, 0x0b, 0xe9, - 0x4b, 0x12, 0x12, 0xcc, 0x09, 0x09, 0x3d, 0x8b, 0x2f, 0xbc, 0x88, 0xbe, 0x0a, 0xcf, 0xe2, 0x4c, - 0x66, 0x37, 0xbe, 0x0c, 0x66, 0x37, 0xcf, 0x64, 0x76, 0xeb, 0x79, 0xcc, 0x7e, 0x05, 0xcf, 0xe2, - 0xcf, 0x6b, 0x00, 0xfb, 0x7e, 0x38, 0x0a, 0xc4, 0xd7, 0xbe, 0xc5, 0x2f, 0x8c, 0x6f, 0xf1, 0xd7, - 0x55, 0x30, 0x76, 0x9c, 0xf8, 0xf8, 0xe7, 0x6e, 0x85, 0xbc, 0x09, 0x2d, 0x19, 0x96, 0xd7, 0x43, - 0x99, 0xae, 0x29, 0xc3, 0x9f, 0x0b, 0x95, 0xff, 0xad, 0x0a, 0xb4, 0xf6, 0x62, 0xe9, 0x4d, 0xdc, - 0xf4, 0x15, 0xf5, 0x7d, 0x7a, 0x8c, 0xb5, 0xe7, 0x8d, 0xb1, 0x3e, 0x3b, 0x46, 0xeb, 0xb7, 0x2b, - 0x60, 0xea, 0x21, 0x3c, 0x58, 0xff, 0x8a, 0x16, 0xdd, 0xf3, 0x47, 0xf1, 0x04, 0x4c, 0x8a, 0x13, - 0x9d, 0xa9, 0x46, 0x67, 0xae, 0x9f, 0xea, 0x2b, 0xad, 0x1f, 0xeb, 0xf7, 0x2a, 0xd0, 0xa5, 0x60, - 0xdc, 0xfd, 0x49, 0xe8, 0xd2, 0x65, 0xce, 0xe2, 0xa8, 0xd2, 0x0a, 0xd4, 0x63, 0x91, 0x66, 0xf7, - 0xd1, 0x1d, 0xf5, 0x99, 0x4d, 0x19, 0x6c, 0x89, 0x21, 0x27, 0x0c, 0x32, 0xc1, 0x89, 0x47, 0xc9, - 0xa2, 0x1b, 0x71, 0x84, 0xe3, 0xac, 0x22, 0x27, 0x76, 0xc6, 0x49, 0x76, 0x23, 0xae, 0x6a, 0x8c, - 0x41, 0x9d, 0x22, 0xbd, 0x0d, 0x8a, 0x89, 0x50, 0xd9, 0xda, 0x80, 0x8b, 0xf7, 0x4e, 0x52, 0x11, - 0x87, 0x4e, 0xb0, 0xeb, 0x8c, 0xc5, 0xfa, 0xa6, 0x0c, 0x54, 0x18, 0x2d, 0x23, 0xae, 0x14, 0xc4, - 0x38, 0xe0, 0x72, 0xbe, 0x8f, 0xaa, 0x58, 0x37, 0xa1, 0x3d, 0xf4, 0x03, 0x61, 0xcb, 0xe1, 0x30, - 0x11, 0x29, 0x7e, 0x5d, 0x95, 0x68, 0x5a, 0x35, 0xae, 0x6b, 0xd6, 0xdf, 0xd4, 0xa1, 0x93, 0x7d, - 0x8a, 0xf2, 0x21, 0x16, 0x4f, 0xff, 0x2a, 0x98, 0xd4, 0x5b, 0xe2, 0x3f, 0x15, 0xc4, 0x83, 0x1a, - 0x37, 0x10, 0x40, 0x17, 0xd8, 0x1b, 0xb0, 0x5c, 0xfa, 0x94, 0x9d, 0xca, 0xd4, 0x09, 0x34, 0x1b, - 0x4a, 0x17, 0x6f, 0x25, 0x12, 0xbe, 0x84, 0x95, 0xcf, 0xa8, 0x7c, 0x80, 0xd4, 0xc8, 0xde, 0x3c, - 0x88, 0x36, 0xc7, 0x5e, 0xc4, 0xb0, 0xef, 0xc1, 0x12, 0xce, 0x76, 0x5d, 0xc5, 0x5e, 0x69, 0xbe, - 0x6a, 0x5d, 0x5f, 0x2f, 0x3e, 0xb1, 0x90, 0x67, 0xbc, 0x1b, 0x4e, 0xb1, 0xf0, 0x0d, 0x00, 0x37, - 0x16, 0x4e, 0x2a, 0xec, 0xe4, 0x51, 0x40, 0x6b, 0xde, 0xe4, 0xa6, 0x82, 0xec, 0x3f, 0x0a, 0xf2, - 0x99, 0xe6, 0x46, 0xd9, 0x54, 0x33, 0x25, 0x45, 0x7f, 0x0f, 0xda, 0x32, 0xf6, 0x47, 0x7e, 0xa8, - 0x42, 0x7e, 0xc6, 0x82, 0xd1, 0x82, 0x22, 0xa0, 0x00, 0xa0, 0x05, 0x4d, 0xa5, 0xa8, 0x0b, 0xae, - 0x1f, 0x34, 0x86, 0x71, 0xe8, 0x1d, 0x1c, 0x6e, 0xca, 0xe0, 0x80, 0xd2, 0xca, 0x36, 0x65, 0x40, - 0xd7, 0xf4, 0xed, 0xf5, 0xdb, 0xf3, 0xd3, 0x42, 0xf9, 0xac, 0x4d, 0x13, 0xab, 0xa0, 0xdf, 0x4c, - 0x0f, 0xec, 0x16, 0x2c, 0x25, 0x69, 0xec, 0xbb, 0x29, 0x4e, 0xd1, 0x1e, 0x4b, 0x4f, 0x90, 0xe5, - 0x36, 0x78, 0x57, 0x81, 0xf7, 0x1f, 0x05, 0x3b, 0xd2, 0x13, 0x57, 0x36, 0xe0, 0xfc, 0x82, 0xee, - 0x5e, 0xea, 0x3e, 0xca, 0x05, 0xd8, 0x4f, 0x63, 0xe1, 0x8c, 0x49, 0x79, 0xde, 0x86, 0x56, 0x7a, - 0x18, 0xd0, 0x65, 0x53, 0x65, 0xe1, 0x65, 0x53, 0x33, 0x3d, 0x44, 0x2e, 0x95, 0xd4, 0xb1, 0x4a, - 0xd7, 0x3e, 0xba, 0x86, 0x1f, 0x0a, 0xfc, 0xb1, 0x9f, 0xea, 0x04, 0x32, 0x55, 0xb1, 0x3e, 0x04, - 0x93, 0x7a, 0xa0, 0x6f, 0xe4, 0x3b, 0x78, 0xe5, 0xcc, 0x1d, 0xdc, 0x7a, 0x17, 0xcc, 0x5f, 0xc3, - 0x61, 0x52, 0xa3, 0xeb, 0xd0, 0xa6, 0x0b, 0x49, 0xfb, 0x30, 0x90, 0xee, 0x71, 0x76, 0x51, 0x46, - 0xa0, 0xbb, 0x08, 0xb1, 0x00, 0x8c, 0x87, 0xa1, 0x2f, 0xc3, 0x8d, 0x20, 0xb0, 0x7e, 0x5a, 0x05, - 0xf3, 0xfb, 0x4e, 0x72, 0x44, 0x56, 0x82, 0xad, 0x40, 0x7b, 0x57, 0x08, 0x0f, 0x01, 0x3b, 0x4e, - 0xa4, 0x33, 0x55, 0xca, 0x20, 0x76, 0x05, 0x8c, 0xef, 0xab, 0x3d, 0xe3, 0x53, 0x7d, 0x05, 0x94, - 0xd7, 0xb3, 0xd6, 0x74, 0xe1, 0x29, 0xb2, 0xa4, 0x88, 0x32, 0x88, 0xdd, 0x86, 0x3e, 0x56, 0x29, - 0x1f, 0x04, 0x75, 0x50, 0x04, 0xca, 0x42, 0x18, 0x7c, 0x0e, 0xce, 0x6e, 0x03, 0xe0, 0xe6, 0x46, - 0x57, 0xa9, 0xc9, 0x82, 0x7d, 0xad, 0x84, 0x65, 0xd7, 0x00, 0x7e, 0x90, 0x1b, 0x58, 0x9d, 0x6b, - 0x55, 0x82, 0xb0, 0xb7, 0xa0, 0xab, 0x6b, 0x5c, 0x0c, 0x37, 0xf5, 0x05, 0x5c, 0x83, 0x4f, 0x03, - 0xd9, 0x3d, 0x58, 0xe6, 0x2f, 0x9d, 0x05, 0x37, 0x07, 0xb2, 0xfe, 0xa8, 0x0a, 0x1d, 0xbd, 0x27, - 0x91, 0xcd, 0x9e, 0xe2, 0x59, 0xe5, 0x6c, 0x9e, 0x55, 0x5f, 0x8c, 0x67, 0xb5, 0x17, 0xe2, 0x59, - 0xfd, 0x4c, 0x9e, 0x2d, 0x9c, 0x6d, 0xe3, 0x65, 0x67, 0xfb, 0x5c, 0xd6, 0x5f, 0x03, 0xd8, 0xcf, - 0x9d, 0x00, 0xcd, 0xf7, 0x12, 0xc4, 0xda, 0x07, 0x20, 0x5b, 0xa5, 0x58, 0xb5, 0x70, 0x50, 0x95, - 0x97, 0x16, 0xc1, 0x7f, 0x57, 0x00, 0xf6, 0x9d, 0x71, 0xa4, 0xb6, 0x3a, 0xf6, 0x5d, 0x68, 0x27, - 0x54, 0x53, 0x31, 0x55, 0x95, 0x5b, 0x5b, 0xb2, 0xa5, 0x05, 0xa9, 0x2e, 0xe2, 0x0a, 0xe3, 0x90, - 0xe4, 0x65, 0xf2, 0x6e, 0x54, 0x0f, 0xf9, 0xe5, 0x67, 0x23, 0x23, 0xa0, 0x7b, 0xcf, 0x9b, 0xd0, - 0xd3, 0x04, 0x91, 0x88, 0x5d, 0x11, 0xaa, 0x55, 0x5d, 0xe1, 0x5d, 0x05, 0xdd, 0x53, 0x40, 0xf6, - 0x41, 0x4e, 0xe6, 0xca, 0x60, 0x32, 0x5e, 0x28, 0x24, 0xdd, 0x64, 0x53, 0x11, 0x58, 0xeb, 0xd9, - 0x54, 0x68, 0x20, 0x06, 0xd4, 0xf1, 0x7b, 0xfd, 0x73, 0xac, 0x0d, 0x2d, 0xdd, 0x6b, 0xbf, 0xc2, - 0xba, 0x60, 0x52, 0xfa, 0x1f, 0xe1, 0xaa, 0xd6, 0x5f, 0x32, 0x68, 0x6f, 0x87, 0x49, 0x1a, 0x4f, - 0xd4, 0x3e, 0x5f, 0x64, 0xb9, 0x35, 0x28, 0xcb, 0x4d, 0xdf, 0x7e, 0xab, 0x69, 0xd0, 0xed, 0xf7, - 0x2d, 0xa8, 0x3b, 0x61, 0xea, 0x6b, 0xb7, 0xa6, 0x94, 0x4a, 0x99, 0x1d, 0xd9, 0x39, 0xe1, 0xd9, - 0x7b, 0xd0, 0xd2, 0x79, 0x97, 0x3a, 0xad, 0x69, 0x61, 0xd2, 0x66, 0x46, 0xc3, 0xd6, 0xc0, 0xf0, - 0x74, 0x42, 0xa8, 0xd6, 0xad, 0x52, 0xd7, 0x59, 0xaa, 0x28, 0xcf, 0x69, 0xd8, 0x0d, 0xa8, 0x39, - 0x23, 0xa5, 0x46, 0x74, 0x1b, 0x9d, 0x91, 0x52, 0x1a, 0x1d, 0x47, 0x1c, 0xbb, 0xa3, 0x1d, 0x68, - 0xf4, 0xa8, 0xf4, 0xea, 0x2c, 0xf5, 0x99, 0x85, 0x6b, 0x95, 0x23, 0x4d, 0x2e, 0xd6, 0x1d, 0x30, - 0x13, 0x31, 0xf6, 0x55, 0x03, 0x73, 0xb6, 0x41, 0x76, 0xe4, 0xe5, 0x46, 0x92, 0x1d, 0x7e, 0x3f, - 0x82, 0x76, 0x42, 0x67, 0x33, 0xd5, 0x04, 0xb2, 0x9b, 0xbb, 0xbc, 0x49, 0x7e, 0x70, 0xe3, 0x90, - 0x14, 0x87, 0xb8, 0x3b, 0x60, 0x8e, 0x9d, 0xf8, 0x58, 0x35, 0x6a, 0xcf, 0x7e, 0x27, 0x3b, 0x38, - 0x70, 0x63, 0x9c, 0x1d, 0x21, 0x2c, 0xa8, 0x13, 0x6d, 0x27, 0xdb, 0x3e, 0x32, 0x5a, 0xc5, 0x6f, - 0xc4, 0xb1, 0x77, 0xa0, 0x15, 0x29, 0x8f, 0x95, 0x52, 0x30, 0xda, 0xeb, 0xcb, 0x05, 0x99, 0x76, - 0x65, 0x79, 0x46, 0xc1, 0xbe, 0x03, 0x3d, 0x75, 0xdf, 0x3c, 0xd4, 0x0e, 0x1e, 0xa5, 0x65, 0x4c, - 0xe5, 0xf6, 0x4d, 0xf9, 0x7f, 0xbc, 0x9b, 0x4e, 0xb9, 0x83, 0xdf, 0x82, 0xae, 0xd0, 0xfb, 0xaf, - 0x9d, 0xb8, 0x4e, 0x38, 0xe8, 0x53, 0xf3, 0x4b, 0x8b, 0xb7, 0x67, 0xde, 0x11, 0x65, 0x67, 0x6a, - 0x15, 0x9a, 0xfa, 0x1e, 0x7e, 0x99, 0x5a, 0x95, 0xf2, 0xd2, 0xd5, 0x8d, 0x14, 0xd7, 0x78, 0x76, - 0x77, 0xe6, 0xae, 0x10, 0x37, 0x60, 0x96, 0xdd, 0xb1, 0x2f, 0xbe, 0x00, 0x9c, 0xba, 0x45, 0xfc, - 0x54, 0x9c, 0xb2, 0x75, 0x80, 0xe2, 0x8e, 0x75, 0x70, 0x7e, 0x56, 0x15, 0xf3, 0x0b, 0x56, 0x6e, - 0xe6, 0x77, 0xab, 0x68, 0x5c, 0xca, 0x77, 0xbe, 0xea, 0xda, 0xec, 0x02, 0x35, 0x7d, 0x6d, 0x41, - 0x53, 0x75, 0x7b, 0xc6, 0x97, 0xa2, 0x99, 0xab, 0xe3, 0x77, 0xc1, 0x90, 0xb1, 0x47, 0xf9, 0x16, - 0x83, 0x8b, 0xb4, 0x7a, 0x97, 0x75, 0xda, 0x84, 0x4a, 0x4e, 0x25, 0xa3, 0xd4, 0x92, 0xaa, 0xc2, - 0xde, 0x83, 0x4e, 0x14, 0xcb, 0x1f, 0x09, 0x37, 0x55, 0x6e, 0xd6, 0xa5, 0xf9, 0xa4, 0x56, 0x8d, - 0x27, 0xaf, 0xab, 0x70, 0xa3, 0x2e, 0x3f, 0xd3, 0x8d, 0x5a, 0xc9, 0x1c, 0x87, 0xc1, 0xfc, 0xfd, - 0x24, 0x21, 0xb0, 0x17, 0xed, 0x72, 0xbc, 0x36, 0xdf, 0x8b, 0x76, 0x3f, 0x06, 0xd0, 0xf2, 0x93, - 0xfb, 0x7e, 0x9c, 0xa4, 0x83, 0x2b, 0x2a, 0x19, 0x58, 0x57, 0xd1, 0x61, 0xf1, 0x93, 0x07, 0x4e, - 0x92, 0x0e, 0xae, 0x66, 0x79, 0xc9, 0x58, 0x43, 0x9e, 0xab, 0x13, 0x29, 0x69, 0xed, 0xeb, 0xb3, - 0x3c, 0xcf, 0xc3, 0xee, 0xfa, 0x68, 0x4a, 0x3a, 0xfe, 0x09, 0x2c, 0xa9, 0x36, 0xc5, 0x12, 0x7c, - 0x63, 0x56, 0x27, 0xa7, 0xe2, 0xb7, 0xbc, 0x1b, 0x4f, 0x85, 0x73, 0xf3, 0x0e, 0xd0, 0xfc, 0xa8, - 0x0e, 0xae, 0x2d, 0xec, 0x20, 0x37, 0x54, 0xaa, 0x83, 0x3c, 0xd4, 0x78, 0x1b, 0x9a, 0x3a, 0x79, - 0xe4, 0xfa, 0x9c, 0x01, 0xd2, 0x69, 0x52, 0x5c, 0x53, 0xb0, 0x6f, 0x40, 0x8b, 0x72, 0x18, 0x64, - 0x34, 0x58, 0x99, 0x55, 0x62, 0x95, 0x7b, 0xc0, 0x9b, 0x81, 0xca, 0x41, 0x78, 0x07, 0x5a, 0xd9, - 0x49, 0xf4, 0xc6, 0xec, 0xc2, 0xd4, 0xdb, 0x1b, 0xcf, 0x28, 0xd8, 0x4d, 0x68, 0x8c, 0xd1, 0x3c, - 0x0f, 0xac, 0x59, 0xc3, 0xa6, 0xac, 0xb6, 0xc2, 0x92, 0xe1, 0x21, 0x07, 0x53, 0xad, 0xbe, 0x37, - 0xe7, 0x0c, 0x4f, 0xee, 0x7d, 0x72, 0x48, 0x0a, 0x4f, 0xf4, 0x37, 0xe1, 0x4a, 0x39, 0xdf, 0x20, - 0x4b, 0x46, 0xd0, 0x27, 0x87, 0xb7, 0xa8, 0x97, 0x1b, 0x0b, 0x14, 0x7c, 0x3a, 0x6d, 0x81, 0x5f, - 0x8e, 0x9e, 0x91, 0xcf, 0xf0, 0x51, 0xbe, 0xf9, 0xa1, 0x5d, 0x19, 0xdc, 0x9c, 0x1b, 0x56, 0xbe, - 0x7d, 0x66, 0x5b, 0x22, 0xed, 0xba, 0x1f, 0x43, 0x67, 0x38, 0x79, 0xfa, 0xf4, 0x54, 0x1f, 0x60, - 0x07, 0xb7, 0xa8, 0x5d, 0xe9, 0x94, 0x54, 0xba, 0x3d, 0xe7, 0xed, 0x61, 0xe9, 0x2a, 0xfd, 0x32, - 0xb4, 0xdc, 0xd0, 0x76, 0x3c, 0x2f, 0x1e, 0xbc, 0xad, 0x6e, 0xcf, 0xdd, 0x70, 0xc3, 0xf3, 0x28, - 0x0d, 0x41, 0x46, 0x82, 0x12, 0xbc, 0x6d, 0xdf, 0x1b, 0xac, 0xaa, 0x6d, 0x38, 0x03, 0x6d, 0x7b, - 0xf4, 0x74, 0xc4, 0x89, 0x9d, 0x20, 0x10, 0x01, 0x12, 0x7c, 0x43, 0x3f, 0x1d, 0xd1, 0xa0, 0x6d, - 0x8f, 0xdd, 0x80, 0xce, 0xd8, 0x39, 0xb1, 0x33, 0xc8, 0xe0, 0xb6, 0xca, 0xcb, 0x1f, 0x3b, 0x27, - 0x7b, 0x1a, 0x84, 0x6a, 0xae, 0x32, 0xfb, 0x48, 0xd9, 0xde, 0x99, 0x55, 0xf3, 0xfc, 0xec, 0xce, - 0x4d, 0x3f, 0x3f, 0xc6, 0x93, 0x39, 0x22, 0x23, 0x6c, 0x07, 0xeb, 0x83, 0x77, 0xe7, 0xcd, 0x91, - 0x0e, 0x3a, 0xa0, 0x39, 0xca, 0xe2, 0x0f, 0xeb, 0x00, 0xca, 0x5a, 0x93, 0xb0, 0xdf, 0x9b, 0x6d, - 0x93, 0x9f, 0x02, 0xb8, 0x4a, 0x6b, 0x23, 0x51, 0xaf, 0x03, 0xd0, 0x79, 0x44, 0xb5, 0x59, 0x9b, - 0x6d, 0x93, 0x1f, 0x02, 0xb8, 0xf9, 0x38, 0x3f, 0x0f, 0xdc, 0x01, 0x73, 0x82, 0xee, 0xbe, 0xed, - 0x04, 0xc1, 0xe0, 0xce, 0xec, 0x1a, 0xc8, 0x4e, 0x02, 0xdc, 0x98, 0xe8, 0x12, 0x7e, 0x84, 0x22, - 0x45, 0xe4, 0x92, 0x0d, 0xde, 0x9f, 0xfd, 0x48, 0x7e, 0x5c, 0xe0, 0xe6, 0x51, 0x7e, 0x72, 0xf8, - 0x16, 0x74, 0xb3, 0xf0, 0x8f, 0x6a, 0xf6, 0xc1, 0xec, 0xd6, 0x51, 0x76, 0x89, 0x79, 0xf6, 0x38, - 0x42, 0x35, 0xfe, 0x08, 0xda, 0x8a, 0xe3, 0xaa, 0xe9, 0xfa, 0xac, 0x82, 0x15, 0x0e, 0x22, 0x57, - 0xa2, 0x51, 0xcd, 0x6e, 0x42, 0xc3, 0x89, 0xa2, 0xe0, 0x74, 0xf0, 0xe1, 0xec, 0xaa, 0xda, 0x40, - 0x30, 0x57, 0x58, 0xb4, 0xd7, 0x91, 0x4c, 0x52, 0xdb, 0x1b, 0x07, 0x83, 0x6f, 0xce, 0xed, 0xa1, - 0x2a, 0x9d, 0x8b, 0xb7, 0x22, 0x55, 0xb0, 0x3e, 0x82, 0xce, 0x06, 0x3d, 0x6e, 0xf2, 0x13, 0x32, - 0xc8, 0x37, 0xa1, 0x9e, 0x47, 0x88, 0x72, 0x4b, 0x4f, 0x14, 0x4f, 0xc5, 0x76, 0x38, 0x94, 0x9c, - 0xd0, 0xd6, 0x9f, 0xd4, 0xa0, 0xb9, 0x2f, 0x27, 0xb1, 0x2b, 0x9e, 0x9f, 0xa8, 0xf8, 0x46, 0x26, - 0xf8, 0xb0, 0xc8, 0x0f, 0x51, 0x32, 0x26, 0x74, 0x39, 0xf8, 0x54, 0xa3, 0x43, 0x79, 0x1e, 0x7c, - 0xba, 0x00, 0x0d, 0x75, 0xa8, 0x53, 0xa9, 0x72, 0xaa, 0x42, 0x4a, 0x3f, 0x49, 0x8e, 0x3c, 0xf9, - 0x24, 0x44, 0xa5, 0x6f, 0x50, 0xa6, 0x19, 0x64, 0xa0, 0x6d, 0x8f, 0x52, 0xbd, 0x33, 0x02, 0x5a, - 0x55, 0x2a, 0x12, 0xd0, 0xc9, 0x80, 0xb4, 0xb6, 0xb2, 0xc0, 0x56, 0xeb, 0x19, 0x81, 0xad, 0xdb, - 0x90, 0x67, 0x4f, 0x6a, 0xb7, 0xeb, 0xd9, 0xd9, 0x95, 0xeb, 0x60, 0xe6, 0x4f, 0xdf, 0xb4, 0xcb, - 0x75, 0x61, 0xad, 0x78, 0x0c, 0x77, 0x90, 0x95, 0x78, 0x41, 0xb6, 0x20, 0xe2, 0x15, 0xc5, 0xf2, - 0x50, 0x07, 0x27, 0xe0, 0x65, 0x22, 0x5e, 0x7b, 0xd8, 0x2e, 0x8b, 0xe3, 0xf9, 0x89, 0xed, 0xca, - 0x30, 0x49, 0x75, 0x54, 0xa0, 0xe5, 0x27, 0x9b, 0x58, 0xb5, 0x7e, 0x03, 0x8c, 0x5d, 0xe9, 0x91, - 0x08, 0x19, 0x83, 0xfa, 0xd8, 0x8d, 0x26, 0xda, 0x41, 0xa6, 0xb2, 0x7e, 0xd9, 0xa6, 0x84, 0xa3, - 0x5f, 0xb6, 0x11, 0xeb, 0x6a, 0x2a, 0x1a, 0x85, 0x65, 0xf5, 0x9e, 0xe6, 0x34, 0x90, 0x8e, 0xa7, - 0x05, 0x92, 0x55, 0xad, 0x3f, 0xae, 0xc0, 0xf2, 0x5e, 0x2c, 0x5d, 0x91, 0x24, 0x0f, 0x70, 0x47, - 0x76, 0xc8, 0xbf, 0x62, 0x50, 0xa7, 0xa0, 0x92, 0x7a, 0x6e, 0x42, 0x65, 0x54, 0x06, 0x75, 0x5a, - 0xcf, 0x0f, 0x16, 0x35, 0x6e, 0x12, 0x84, 0xce, 0x15, 0x39, 0x9a, 0x1a, 0xd6, 0x4a, 0x68, 0x0a, - 0x47, 0xdd, 0x84, 0x5e, 0x91, 0x8f, 0x4c, 0x3d, 0xe8, 0x77, 0x66, 0x39, 0x94, 0x7a, 0xb9, 0x0e, - 0xed, 0x58, 0x38, 0xe8, 0xb3, 0x50, 0x37, 0x0d, 0xa2, 0x01, 0x05, 0xc2, 0x7e, 0xac, 0x23, 0xe8, - 0xef, 0xc5, 0x22, 0x72, 0x62, 0x81, 0x66, 0x70, 0x4c, 0x5c, 0xb9, 0x04, 0xcd, 0x40, 0x84, 0xa3, - 0xf4, 0x48, 0x8f, 0x57, 0xd7, 0xf2, 0x77, 0x84, 0xd5, 0xd2, 0x3b, 0x42, 0xe4, 0x4e, 0x2c, 0x1c, - 0xfd, 0xdc, 0x90, 0xca, 0xa8, 0xac, 0xe1, 0x24, 0xd0, 0x81, 0x2e, 0x83, 0xab, 0x8a, 0xf5, 0x87, - 0x35, 0x68, 0x6b, 0xce, 0xd0, 0x57, 0x14, 0x9f, 0x2b, 0x39, 0x9f, 0xfb, 0x50, 0x4b, 0x1e, 0x05, - 0x9a, 0xf1, 0x58, 0x64, 0x1f, 0x42, 0x2d, 0xf0, 0xc7, 0xfa, 0x64, 0x72, 0x75, 0xca, 0xa8, 0x4e, - 0xf3, 0x57, 0xc7, 0x44, 0x90, 0x9a, 0x5d, 0x25, 0xa3, 0x77, 0x62, 0xa3, 0x56, 0x68, 0x9e, 0xa0, - 0x81, 0x3b, 0x41, 0xd5, 0x43, 0xa6, 0x3a, 0x2e, 0x65, 0xcb, 0x65, 0xeb, 0xa5, 0xcb, 0x4d, 0x0d, - 0xd9, 0xf6, 0xd8, 0x37, 0xc1, 0x48, 0x42, 0x27, 0x4a, 0x8e, 0x64, 0xaa, 0x4f, 0x22, 0x6c, 0x2d, - 0x3d, 0x09, 0xd7, 0x36, 0x77, 0x0f, 0x4e, 0xc2, 0x7d, 0x8d, 0xd1, 0x1f, 0xcb, 0x29, 0xd9, 0x77, - 0xa0, 0x93, 0x88, 0x24, 0x51, 0x89, 0xe1, 0x43, 0xa9, 0xd7, 0xd1, 0xc5, 0xf2, 0x49, 0x83, 0xb0, - 0x38, 0x6b, 0xdd, 0xb8, 0x9d, 0x14, 0x20, 0xf6, 0x7d, 0xe8, 0x65, 0xed, 0x03, 0x39, 0x1a, 0x89, - 0x2c, 0xf5, 0xf7, 0xea, 0x5c, 0x0f, 0x0f, 0x08, 0x5d, 0xea, 0xa7, 0x9b, 0x94, 0x11, 0xec, 0x7b, - 0xd0, 0x8b, 0x94, 0x30, 0x6d, 0x1d, 0x85, 0x55, 0x4b, 0xf0, 0xca, 0x94, 0x0f, 0x30, 0x25, 0xec, - 0x22, 0x6d, 0xb4, 0x80, 0x27, 0xd6, 0x7f, 0x56, 0xa0, 0x5d, 0x1a, 0x35, 0xbd, 0xee, 0x4c, 0x44, - 0x9c, 0x45, 0x64, 0xb1, 0x8c, 0xb0, 0x23, 0xa9, 0x1f, 0x4c, 0x99, 0x9c, 0xca, 0x08, 0x8b, 0xa5, - 0x0e, 0xd1, 0x9b, 0x9c, 0xca, 0x68, 0x83, 0xf4, 0xa1, 0x50, 0xbd, 0x37, 0x21, 0xa1, 0xd4, 0x79, - 0xa7, 0x00, 0x6e, 0x53, 0xa4, 0x04, 0xd5, 0xe9, 0xd0, 0x49, 0xb2, 0x18, 0x71, 0x5e, 0xc7, 0xc5, - 0xf6, 0x58, 0xc4, 0x38, 0x16, 0x6d, 0xbe, 0xb2, 0x2a, 0xca, 0x9a, 0xcc, 0xc6, 0x53, 0x19, 0xaa, - 0xab, 0x8b, 0x0e, 0x37, 0x10, 0xf0, 0x43, 0x19, 0x52, 0x33, 0x2d, 0x59, 0xe2, 0xa7, 0xc9, 0xb3, - 0x2a, 0x1a, 0x87, 0x47, 0x13, 0x81, 0x7e, 0x92, 0x47, 0x2f, 0x8b, 0x4c, 0xde, 0xa2, 0xfa, 0xb6, - 0x67, 0xfd, 0x4b, 0x05, 0x96, 0xe7, 0x98, 0x8d, 0x6e, 0x09, 0x32, 0x3a, 0xcb, 0xe6, 0xed, 0xf0, - 0x26, 0x56, 0xb7, 0x3d, 0x42, 0xa4, 0x63, 0x52, 0xa6, 0xaa, 0x46, 0xa4, 0x63, 0xd4, 0xa4, 0x8b, - 0xd0, 0x4c, 0x4f, 0x68, 0xb6, 0x6a, 0x61, 0x34, 0xd2, 0x13, 0x9c, 0xe6, 0x06, 0x98, 0x81, 0x1c, - 0xd9, 0x81, 0x78, 0x2c, 0x02, 0xe2, 0x43, 0x6f, 0xfd, 0xad, 0x33, 0xa4, 0xbc, 0xf6, 0x40, 0x8e, - 0x1e, 0x20, 0x2d, 0x37, 0x02, 0x5d, 0xb2, 0x7e, 0x00, 0x46, 0x06, 0x65, 0x26, 0x34, 0xb6, 0xc4, - 0xe1, 0x64, 0xd4, 0x3f, 0xc7, 0x0c, 0xa8, 0x63, 0x8b, 0x7e, 0x05, 0x4b, 0x9f, 0x3b, 0x71, 0xd8, - 0xaf, 0x22, 0xfa, 0x5e, 0x1c, 0xcb, 0xb8, 0x5f, 0xc3, 0xe2, 0x9e, 0x13, 0xfa, 0x6e, 0xbf, 0x8e, - 0xc5, 0xfb, 0x4e, 0xea, 0x04, 0xfd, 0x86, 0xf5, 0xa7, 0x0d, 0x30, 0xf6, 0xf4, 0xd7, 0xd9, 0x16, - 0x74, 0xf3, 0xc7, 0xb7, 0x8b, 0xa3, 0x25, 0x7b, 0xb3, 0x05, 0x8a, 0x96, 0x74, 0xa2, 0x52, 0x6d, - 0xf6, 0x09, 0x6f, 0x75, 0xee, 0x09, 0xef, 0xeb, 0x50, 0x7b, 0x14, 0x9f, 0x4e, 0xdf, 0xa2, 0xec, - 0x05, 0x4e, 0xc8, 0x11, 0xcc, 0x3e, 0x80, 0x36, 0xca, 0xdd, 0x4e, 0x68, 0x47, 0xd5, 0x91, 0x86, - 0xf2, 0x63, 0x68, 0x82, 0x73, 0x40, 0x22, 0xbd, 0xeb, 0xae, 0x81, 0xe1, 0x1e, 0xf9, 0x81, 0x17, - 0x8b, 0x50, 0x07, 0x0b, 0xd9, 0xfc, 0x90, 0x79, 0x4e, 0xc3, 0xbe, 0x4b, 0xa9, 0xb3, 0x59, 0x84, - 0xa4, 0x7c, 0x73, 0x7f, 0x71, 0xea, 0xe0, 0x9a, 0x51, 0xf0, 0xa5, 0x12, 0x39, 0x6d, 0x2e, 0xc5, - 0xe3, 0x88, 0x56, 0xf9, 0x71, 0x84, 0x7a, 0xd6, 0x49, 0x9b, 0x82, 0x91, 0x1f, 0x9f, 0xa4, 0xe3, - 0xb1, 0x5b, 0x50, 0x0f, 0xa5, 0x27, 0xe6, 0x43, 0x10, 0xd9, 0x3e, 0xc4, 0x09, 0x4f, 0x6f, 0xb2, - 0x27, 0xc9, 0x91, 0xad, 0xf6, 0x73, 0x34, 0x25, 0xa0, 0x5f, 0x66, 0x4d, 0x92, 0xa3, 0x2d, 0xdc, - 0xd1, 0x51, 0x19, 0x6f, 0x42, 0x2f, 0x9b, 0x8b, 0x4e, 0xfc, 0x55, 0x17, 0x96, 0xdd, 0x0c, 0xaa, - 0xf2, 0x7e, 0xd7, 0xe0, 0xbc, 0x7b, 0xe4, 0x84, 0xa1, 0x08, 0xec, 0xc3, 0xc9, 0x70, 0x98, 0xed, - 0x00, 0x1d, 0xba, 0x6c, 0x5a, 0xd6, 0xa8, 0xbb, 0x84, 0xa1, 0x0d, 0xc5, 0x82, 0x6e, 0xe8, 0x07, - 0xea, 0x45, 0x8b, 0xed, 0x86, 0xe9, 0xa0, 0x4b, 0x94, 0xed, 0xd0, 0x0f, 0x28, 0x20, 0xb9, 0x19, - 0xa6, 0xec, 0x13, 0xe8, 0x4f, 0x26, 0xbe, 0x97, 0xd8, 0xa9, 0xcc, 0x5e, 0xc4, 0x0e, 0x7a, 0xc4, - 0xba, 0x92, 0xbb, 0xf7, 0x70, 0xe2, 0x7b, 0x07, 0x52, 0xbf, 0x89, 0xed, 0x12, 0x7d, 0x56, 0xb5, - 0x3e, 0x81, 0x4e, 0x59, 0x77, 0x50, 0x17, 0xe9, 0x1c, 0xd4, 0x3f, 0xc7, 0x00, 0x9a, 0xbb, 0x32, - 0x1e, 0x3b, 0x41, 0xbf, 0x82, 0x65, 0xf5, 0x64, 0xa8, 0x5f, 0x65, 0x1d, 0x30, 0x32, 0x07, 0xbd, - 0x5f, 0xb3, 0xbe, 0x05, 0x46, 0xf6, 0xc4, 0x97, 0xde, 0x56, 0x4a, 0x4f, 0x28, 0xc7, 0x46, 0x59, - 0x26, 0x03, 0x01, 0xe4, 0xd4, 0x64, 0xef, 0xd1, 0xab, 0xc5, 0x7b, 0x74, 0xeb, 0x57, 0xa1, 0x53, - 0x1e, 0x5c, 0x16, 0x0c, 0xab, 0x14, 0xc1, 0xb0, 0x05, 0xad, 0xe8, 0xae, 0x24, 0x96, 0x63, 0xbb, - 0xe4, 0x04, 0x18, 0x08, 0xc0, 0xcf, 0x58, 0xbf, 0x53, 0x81, 0x06, 0xf9, 0x9c, 0xb4, 0xb5, 0x60, - 0xa1, 0x58, 0x3b, 0x0d, 0x6e, 0x12, 0x84, 0x66, 0x5a, 0xbe, 0x73, 0xac, 0x3e, 0xfb, 0xce, 0xb1, - 0x36, 0x7d, 0xe7, 0xf8, 0x82, 0x17, 0xf9, 0xb7, 0x1f, 0x41, 0x53, 0xfd, 0x02, 0x80, 0x2d, 0x43, - 0xf7, 0x61, 0x78, 0x1c, 0xca, 0x27, 0xa1, 0x02, 0xf4, 0xcf, 0xb1, 0xf3, 0xb0, 0x94, 0x31, 0x5d, - 0xff, 0x6b, 0xa0, 0x5f, 0x61, 0x7d, 0xe8, 0x90, 0x58, 0x33, 0x48, 0x95, 0xbd, 0x0e, 0x03, 0xbd, - 0x39, 0x6c, 0xc9, 0x50, 0xec, 0xca, 0xd4, 0x1f, 0x9e, 0x66, 0xd8, 0x1a, 0x5b, 0x82, 0xf6, 0x7e, - 0x2a, 0xa3, 0x7d, 0x11, 0x7a, 0x7e, 0x38, 0xea, 0xd7, 0x6f, 0xdf, 0x87, 0xa6, 0xfa, 0x33, 0x41, - 0xe9, 0x93, 0x0a, 0xd0, 0x3f, 0x87, 0xd4, 0x9f, 0x3b, 0x7e, 0xea, 0x87, 0xa3, 0x5d, 0x71, 0x92, - 0x2a, 0xa3, 0xf4, 0xc0, 0x49, 0xd2, 0x7e, 0x95, 0xf5, 0x00, 0x74, 0xaf, 0xf7, 0x42, 0xaf, 0x5f, - 0xbb, 0xbb, 0xf9, 0x93, 0x9f, 0x5d, 0xab, 0xfc, 0xf4, 0x67, 0xd7, 0x2a, 0xff, 0xf8, 0xb3, 0x6b, - 0xe7, 0x7e, 0xff, 0x9f, 0xae, 0x55, 0x7e, 0xf8, 0x41, 0xe9, 0xbf, 0x0b, 0x63, 0x27, 0x8d, 0xfd, - 0x13, 0x75, 0xdb, 0x94, 0x55, 0x42, 0x71, 0x27, 0x3a, 0x1e, 0xdd, 0x89, 0x0e, 0xef, 0x64, 0x3a, - 0x77, 0xd8, 0xa4, 0xdf, 0x29, 0x7c, 0xf8, 0xbf, 0x01, 0x00, 0x00, 0xff, 0xff, 0x48, 0xa3, 0x23, - 0xca, 0xcd, 0x41, 0x00, 0x00, + // 5719 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x7c, 0x49, 0x70, 0x1d, 0x47, + 0x72, 0x28, 0xdf, 0xde, 0x9d, 0x6f, 0xc1, 0x43, 0x71, 0x7b, 0x22, 0x29, 0x12, 0x6c, 0x89, 0x14, + 0x86, 0x92, 0x40, 0x09, 0x1a, 0xfe, 0xd1, 0xf7, 0x78, 0x46, 0x03, 0x02, 0xe4, 0xcc, 0x93, 0x08, + 0x08, 0x2e, 0x80, 0x56, 0x78, 0xc2, 0xe1, 0x8e, 0x46, 0x77, 0xbd, 0x87, 0x1e, 0xf4, 0xeb, 0x6e, + 0xf6, 0x42, 0x02, 0x3c, 0x39, 0xc2, 0xbe, 0xfa, 0xe4, 0x93, 0x6f, 0x8e, 0x39, 0xd8, 0xe1, 0x83, + 0x97, 0xb0, 0x8f, 0x8e, 0x39, 0xf9, 0x32, 0xbe, 0xf9, 0xe4, 0x83, 0x0f, 0x0e, 0xc7, 0xf8, 0xe6, + 0xe5, 0x36, 0x76, 0xf8, 0xe2, 0xb0, 0x23, 0xb3, 0xaa, 0x97, 0xb7, 0x10, 0x5c, 0x44, 0x39, 0x66, + 0x22, 0x74, 0xab, 0xca, 0xcc, 0xaa, 0xae, 0xca, 0xcc, 0xca, 0xca, 0xca, 0xca, 0x6a, 0xe8, 0x85, + 0x6e, 0x28, 0x3c, 0xd7, 0x17, 0x6b, 0x61, 0x14, 0x24, 0x01, 0xd3, 0xb2, 0xfa, 0xa5, 0xf7, 0xc7, + 0x6e, 0x72, 0x98, 0x1e, 0xac, 0xd9, 0xc1, 0xe4, 0xf6, 0x38, 0x18, 0x07, 0xb7, 0x89, 0xe0, 0x20, + 0x1d, 0x51, 0x8d, 0x2a, 0x54, 0x92, 0x0d, 0x2f, 0x41, 0xe8, 0x59, 0xbe, 0x2a, 0x2f, 0x25, 0xee, + 0x44, 0xc4, 0x89, 0x35, 0x09, 0x33, 0xa4, 0x17, 0xd8, 0x47, 0xaa, 0xac, 0x27, 0xc7, 0x8a, 0xce, + 0xf8, 0x9f, 0x0a, 0xb4, 0xb6, 0x45, 0x1c, 0x5b, 0x63, 0xc1, 0x0c, 0xa8, 0xc5, 0xae, 0x33, 0xa8, + 0xac, 0x54, 0x56, 0x7b, 0xeb, 0xfd, 0xb5, 0x7c, 0x58, 0x7b, 0x89, 0x95, 0xa4, 0x31, 0x47, 0x24, + 0xd2, 0xd8, 0x13, 0x67, 0x50, 0x9d, 0xa5, 0xd9, 0x16, 0xc9, 0x61, 0xe0, 0x70, 0x44, 0xb2, 0x3e, + 0xd4, 0x44, 0x14, 0x0d, 0x6a, 0x2b, 0x95, 0xd5, 0x0e, 0xc7, 0x22, 0x63, 0x50, 0x77, 0xac, 0xc4, + 0x1a, 0xd4, 0x09, 0x44, 0x65, 0xf6, 0x36, 0xf4, 0xc2, 0x28, 0xb0, 0x4d, 0xd7, 0x1f, 0x05, 0x26, + 0x61, 0x1b, 0x84, 0xed, 0x20, 0x74, 0xe8, 0x8f, 0x82, 0x2d, 0xa4, 0x1a, 0x40, 0xcb, 0xf2, 0x2d, + 0xef, 0x24, 0x16, 0x83, 0x26, 0xa1, 0xb3, 0x2a, 0xeb, 0x41, 0xd5, 0x75, 0x06, 0xad, 0x95, 0xca, + 0x6a, 0x9d, 0x57, 0x5d, 0x07, 0xbf, 0x91, 0xa6, 0xae, 0x33, 0xd0, 0xe4, 0x37, 0xb0, 0xcc, 0x0c, + 0xe8, 0xf8, 0x42, 0x38, 0x3b, 0x41, 0xc2, 0x45, 0xe8, 0x9d, 0x0c, 0xf4, 0x95, 0xca, 0xaa, 0xc6, + 0xa7, 0x60, 0xc6, 0x43, 0xd0, 0x37, 0x03, 0xdf, 0x17, 0x76, 0x12, 0x44, 0xec, 0x1a, 0xb4, 0xb3, + 0x29, 0x99, 0x8a, 0x15, 0x0d, 0x0e, 0x19, 0x68, 0xe8, 0xb0, 0x77, 0x60, 0xc9, 0xce, 0xa8, 0x4d, + 0xd7, 0x77, 0xc4, 0x31, 0xf1, 0xa2, 0xc1, 0x7b, 0x39, 0x78, 0x88, 0x50, 0xe3, 0xdf, 0xaa, 0xd0, + 0xda, 0x3b, 0x4c, 0x47, 0x23, 0x4f, 0xb0, 0xb7, 0xa1, 0xab, 0x8a, 0x9b, 0x81, 0x37, 0x74, 0x8e, + 0x55, 0xbf, 0xd3, 0x40, 0xb6, 0x02, 0x6d, 0x05, 0xd8, 0x3f, 0x09, 0x85, 0xea, 0xb6, 0x0c, 0x9a, + 0xee, 0x67, 0xdb, 0xf5, 0x89, 0xc5, 0x35, 0x3e, 0x0d, 0x9c, 0xa1, 0xb2, 0x8e, 0x89, 0xeb, 0xd3, + 0x54, 0x16, 0x7d, 0x6d, 0xc3, 0x73, 0x1f, 0x0b, 0x2e, 0xc6, 0x9b, 0x7e, 0x42, 0xbc, 0x6f, 0xf0, + 0x32, 0x88, 0xad, 0xc3, 0xf9, 0x58, 0x36, 0x31, 0x23, 0xcb, 0x1f, 0x8b, 0xd8, 0x4c, 0x5d, 0x3f, + 0xf9, 0x7f, 0xdf, 0x1c, 0x34, 0x57, 0x6a, 0xab, 0x75, 0x7e, 0x56, 0x21, 0x39, 0xe1, 0x1e, 0x12, + 0x8a, 0x7d, 0x00, 0xe7, 0x66, 0xda, 0xc8, 0x26, 0xad, 0x95, 0xda, 0x6a, 0x8d, 0xb3, 0xa9, 0x26, + 0x43, 0x6a, 0x71, 0x0f, 0x96, 0xa3, 0xd4, 0x47, 0x6d, 0xbd, 0xef, 0x7a, 0x89, 0x88, 0xf6, 0x42, + 0x61, 0x93, 0x0c, 0xdb, 0xeb, 0x17, 0xd7, 0x48, 0xa1, 0xf9, 0x2c, 0x9a, 0xcf, 0xb7, 0x30, 0xfe, + 0xab, 0x0a, 0xda, 0x96, 0x1b, 0x87, 0x56, 0x62, 0x1f, 0xb2, 0x8b, 0xd0, 0x1a, 0xa5, 0xbe, 0x5d, + 0x48, 0xb0, 0x89, 0xd5, 0xa1, 0xc3, 0x7e, 0x15, 0x96, 0xbc, 0xc0, 0xb6, 0x3c, 0x33, 0x17, 0xd6, + 0xa0, 0xba, 0x52, 0x5b, 0x6d, 0xaf, 0x9f, 0x2d, 0x34, 0x39, 0x57, 0x06, 0xde, 0x23, 0xda, 0x42, + 0x39, 0xbe, 0x03, 0xfd, 0x48, 0x4c, 0x82, 0x44, 0x94, 0x9a, 0xd7, 0xa8, 0x39, 0x2b, 0x9a, 0x7f, + 0x11, 0x59, 0xe1, 0x4e, 0xe0, 0x08, 0xbe, 0x24, 0x69, 0x8b, 0xe6, 0x1f, 0x96, 0xf8, 0x29, 0xc6, + 0xa6, 0xeb, 0x1c, 0x9b, 0xf4, 0x81, 0x41, 0x7d, 0xa5, 0xb6, 0xda, 0x28, 0x98, 0x23, 0xc6, 0x43, + 0xe7, 0xf8, 0x01, 0x62, 0xd8, 0x47, 0x70, 0x61, 0xb6, 0x89, 0xec, 0x75, 0xd0, 0xa0, 0x36, 0x67, + 0xa7, 0xda, 0x70, 0x42, 0xb1, 0xeb, 0xd0, 0xc9, 0x1a, 0x25, 0xa8, 0x48, 0x4d, 0x29, 0xda, 0xb8, + 0xa4, 0x48, 0x17, 0xa1, 0xe5, 0xc6, 0x66, 0xec, 0xfa, 0x47, 0xb4, 0x80, 0x34, 0xde, 0x74, 0xe3, + 0x3d, 0xd7, 0x3f, 0x62, 0x6f, 0x80, 0x16, 0x09, 0x5b, 0x62, 0x34, 0xc2, 0xb4, 0x22, 0x61, 0x13, + 0xea, 0x22, 0x60, 0xd1, 0xb4, 0x13, 0xa1, 0x96, 0x51, 0x33, 0x12, 0xf6, 0x66, 0x22, 0x8c, 0x18, + 0x1a, 0xdb, 0x22, 0x1a, 0x0b, 0x76, 0x09, 0x34, 0x6c, 0xb8, 0x67, 0x5b, 0x3e, 0xf1, 0x5d, 0xe3, + 0x79, 0x1d, 0xd7, 0x71, 0x68, 0x45, 0x89, 0x6b, 0x79, 0xa4, 0xd8, 0x1a, 0xcf, 0xaa, 0xec, 0x32, + 0xe8, 0x71, 0x62, 0x45, 0x09, 0xce, 0x8e, 0x14, 0xba, 0xc1, 0x35, 0x02, 0xe0, 0x9a, 0xb8, 0x08, + 0x2d, 0xe1, 0x3b, 0x84, 0xaa, 0x4b, 0x49, 0x0a, 0xdf, 0x19, 0x3a, 0xc7, 0xc6, 0x5f, 0x55, 0xa0, + 0xbb, 0x9d, 0x7a, 0x89, 0xbb, 0x11, 0x8d, 0x53, 0x31, 0xf1, 0x13, 0x5c, 0xff, 0x5b, 0x6e, 0x9c, + 0xa8, 0x2f, 0x53, 0x99, 0xad, 0x82, 0xfe, 0xfd, 0x28, 0x48, 0xc3, 0x7b, 0xc7, 0x61, 0x26, 0x69, + 0x90, 0x4a, 0x85, 0x10, 0x5e, 0x20, 0xd9, 0x7b, 0xd0, 0xfe, 0x3c, 0x72, 0x44, 0x74, 0xf7, 0x84, + 0x68, 0x6b, 0x73, 0xb4, 0x65, 0x34, 0xbb, 0x02, 0xfa, 0x9e, 0x08, 0xad, 0xc8, 0x42, 0x15, 0xc0, + 0x81, 0xe9, 0xbc, 0x00, 0xe0, 0x5c, 0x89, 0x78, 0xe8, 0xa8, 0x65, 0x95, 0x55, 0x8d, 0x31, 0xe8, + 0x1b, 0xe3, 0x71, 0x24, 0xc6, 0x56, 0x42, 0x06, 0x2c, 0x08, 0x69, 0xb8, 0x35, 0x5e, 0x0d, 0x42, + 0x32, 0x92, 0x38, 0x01, 0xc9, 0x1f, 0x2a, 0xb3, 0xab, 0x50, 0x17, 0x8b, 0xc7, 0x43, 0x70, 0x76, + 0x01, 0x9a, 0x76, 0xe0, 0x8f, 0xdc, 0xb1, 0x32, 0xad, 0xaa, 0x66, 0xfc, 0x5e, 0x0d, 0x1a, 0x34, + 0x39, 0x64, 0x2f, 0x9a, 0x3b, 0x53, 0x3c, 0xb6, 0xbc, 0x4c, 0x2a, 0x08, 0xb8, 0xf7, 0xd8, 0xf2, + 0xd8, 0x0a, 0x34, 0xb0, 0x9b, 0x78, 0x01, 0x6f, 0x24, 0x82, 0xdd, 0x84, 0x06, 0x2a, 0x51, 0x3c, + 0x3d, 0x02, 0x54, 0xa2, 0xbb, 0xf5, 0x9f, 0xfe, 0xe3, 0xb5, 0x33, 0x5c, 0xa2, 0xd9, 0x3b, 0x50, + 0xb7, 0xc6, 0xe3, 0x98, 0x74, 0x79, 0x6a, 0x39, 0xe5, 0xf3, 0xe5, 0x44, 0xc0, 0xee, 0x80, 0x2e, + 0xe5, 0x86, 0xd4, 0x0d, 0xa2, 0xbe, 0x58, 0xda, 0x46, 0xca, 0x22, 0xe5, 0x05, 0x25, 0x72, 0xdc, + 0x8d, 0x95, 0x05, 0x23, 0x8d, 0xd6, 0x78, 0x01, 0x40, 0x3b, 0x1f, 0x46, 0x62, 0xc3, 0xf3, 0x02, + 0x7b, 0xcf, 0x7d, 0x2a, 0xd4, 0xae, 0x30, 0x05, 0x63, 0x37, 0xa1, 0xb7, 0x2b, 0x55, 0x8e, 0x8b, + 0x38, 0xf5, 0x92, 0x58, 0xed, 0x14, 0x33, 0x50, 0xb6, 0x06, 0x6c, 0x0a, 0xb2, 0x4f, 0xd3, 0xd7, + 0x57, 0x6a, 0xab, 0x5d, 0xbe, 0x00, 0xc3, 0xde, 0x82, 0xee, 0x18, 0x39, 0xed, 0xfa, 0x63, 0x73, + 0xe4, 0x59, 0xe3, 0x01, 0xac, 0xd4, 0x70, 0x93, 0xc9, 0x80, 0xf7, 0x3d, 0x6b, 0x6c, 0xfc, 0xbc, + 0x0a, 0xcd, 0xa1, 0x1f, 0x8b, 0x28, 0xc1, 0x55, 0x62, 0x8d, 0x46, 0xc2, 0x4e, 0x84, 0xb4, 0x4e, + 0x75, 0x9e, 0xd7, 0x71, 0x96, 0xfb, 0xc1, 0x17, 0x91, 0x9b, 0x88, 0xbd, 0x8f, 0x94, 0x1e, 0x14, + 0x00, 0x76, 0x0b, 0x96, 0x2d, 0xc7, 0x31, 0x33, 0x6a, 0x33, 0x0a, 0x9e, 0xc4, 0xb4, 0x62, 0x34, + 0xbe, 0x64, 0x39, 0xce, 0x86, 0x82, 0xf3, 0xe0, 0x49, 0xcc, 0xae, 0x43, 0x2d, 0x12, 0x23, 0xd2, + 0x8a, 0xf6, 0xfa, 0x92, 0x94, 0xda, 0xe7, 0x07, 0x3f, 0x12, 0x76, 0xc2, 0xc5, 0x88, 0x23, 0x8e, + 0x9d, 0x83, 0x86, 0x95, 0x24, 0x91, 0x94, 0x82, 0xce, 0x65, 0x85, 0xad, 0xc1, 0x59, 0x5a, 0x99, + 0x89, 0x1b, 0xf8, 0x66, 0x62, 0x1d, 0x78, 0xb8, 0x11, 0xc6, 0xca, 0xe6, 0x2f, 0xe7, 0xa8, 0x7d, + 0xc4, 0x0c, 0x9d, 0x18, 0x77, 0x89, 0x59, 0x7a, 0xdf, 0x9a, 0x88, 0x98, 0x4c, 0xbe, 0xce, 0xcf, + 0x4e, 0xb7, 0xd8, 0x41, 0x14, 0xb2, 0xac, 0x68, 0x83, 0x6b, 0x5b, 0xa3, 0x65, 0xd2, 0xc9, 0x81, + 0xb8, 0xf4, 0xcf, 0x43, 0xd3, 0x8d, 0x4d, 0xe1, 0x3b, 0xca, 0xdc, 0x34, 0xdc, 0xf8, 0x9e, 0xef, + 0xb0, 0x77, 0x41, 0x97, 0x5f, 0x71, 0xc4, 0x68, 0x00, 0x34, 0xbd, 0x9e, 0x52, 0x4a, 0x04, 0x6f, + 0x89, 0x11, 0xd7, 0x12, 0x55, 0x32, 0x7e, 0x52, 0x85, 0x36, 0xe9, 0xd0, 0xc3, 0xd0, 0xc1, 0x25, + 0xf7, 0x16, 0x74, 0xa7, 0xb9, 0x27, 0x05, 0xd0, 0xb1, 0xca, 0xac, 0xbb, 0x00, 0xcd, 0x0d, 0x1b, + 0x47, 0x41, 0x12, 0xe8, 0x72, 0x55, 0xc3, 0x65, 0x3d, 0xbc, 0x9b, 0xda, 0x47, 0x22, 0x21, 0xa6, + 0x77, 0x79, 0x56, 0x45, 0xcc, 0x8e, 0xc2, 0xd4, 0x25, 0x46, 0x55, 0xd9, 0x3d, 0x80, 0x3d, 0x31, + 0x9e, 0x08, 0x3f, 0xd9, 0xb6, 0x42, 0xa5, 0xee, 0x37, 0x66, 0xd4, 0x5d, 0x8e, 0x6d, 0xad, 0xa0, + 0xbb, 0xe7, 0x27, 0xd1, 0x09, 0x2f, 0x35, 0x64, 0xdf, 0x82, 0xa5, 0x94, 0xa8, 0x4c, 0x3b, 0x39, + 0x36, 0x3d, 0xb4, 0x12, 0x4d, 0xea, 0x4b, 0x49, 0x56, 0x76, 0xb1, 0x99, 0x1c, 0xf3, 0x6e, 0x9a, + 0x15, 0x1f, 0xb8, 0x71, 0x72, 0xe9, 0x3b, 0xb0, 0x34, 0xd3, 0x2f, 0x7a, 0x67, 0x47, 0xe2, 0x84, + 0x66, 0xae, 0x73, 0x2c, 0xa2, 0x22, 0x3c, 0xb6, 0xbc, 0x34, 0x73, 0x39, 0x64, 0xe5, 0x57, 0xaa, + 0x1f, 0x57, 0x8c, 0x37, 0xa1, 0xb1, 0x11, 0x45, 0x16, 0x91, 0x58, 0x58, 0x18, 0x54, 0x68, 0xdf, + 0x91, 0x15, 0xc3, 0x86, 0x1a, 0x8e, 0xee, 0x06, 0x54, 0x27, 0x21, 0x61, 0xda, 0xeb, 0xe7, 0x4b, + 0x93, 0xb3, 0xc2, 0xb5, 0x6d, 0x35, 0x99, 0xea, 0x24, 0xbc, 0x74, 0x07, 0x5a, 0xdb, 0xaf, 0x30, + 0x86, 0xff, 0xa8, 0x83, 0xb6, 0x25, 0x3c, 0x41, 0x32, 0x30, 0xa0, 0x53, 0x56, 0xf3, 0x4c, 0x7e, + 0x53, 0xaa, 0x6f, 0x40, 0x47, 0xee, 0x84, 0xd4, 0x4a, 0xa8, 0x75, 0x34, 0x05, 0x7b, 0x25, 0x59, + 0x5e, 0x01, 0x88, 0x82, 0x27, 0xa6, 0x2b, 0xb7, 0x23, 0x69, 0xd9, 0xb5, 0x28, 0x78, 0x32, 0xc4, + 0x0d, 0xe9, 0xff, 0x64, 0xdd, 0x7c, 0x0b, 0x06, 0xa5, 0x75, 0x83, 0x6e, 0xa6, 0xe9, 0xfa, 0xe6, + 0x01, 0xfa, 0x3c, 0x6a, 0x09, 0x15, 0x7d, 0x92, 0x17, 0x3a, 0xf4, 0xef, 0x92, 0x43, 0xa4, 0xac, + 0x81, 0x7e, 0x8a, 0x35, 0x58, 0x68, 0x5c, 0x60, 0xb1, 0x71, 0xb9, 0x3b, 0xa5, 0xd5, 0x6d, 0x12, + 0xbc, 0x51, 0x08, 0x3e, 0x93, 0xd6, 0xa9, 0x2a, 0x7d, 0x1d, 0x3a, 0xb6, 0xe5, 0x9b, 0x49, 0x94, + 0xfa, 0xb6, 0x95, 0x88, 0x41, 0x87, 0x3e, 0xd5, 0xb6, 0x2d, 0x7f, 0x5f, 0x81, 0x4a, 0x16, 0xa0, + 0x5b, 0xb6, 0x00, 0x37, 0x61, 0x29, 0x8c, 0xdc, 0x89, 0x15, 0x9d, 0x98, 0x47, 0xe2, 0x84, 0x84, + 0xd1, 0x93, 0xfe, 0xb4, 0x02, 0x7f, 0x26, 0x4e, 0x86, 0xce, 0xf1, 0x97, 0xd5, 0xfd, 0xbf, 0xaf, + 0x82, 0xbe, 0x1b, 0x09, 0x65, 0xb5, 0xaf, 0x41, 0x3b, 0xb6, 0x0f, 0xc5, 0xc4, 0x22, 0x29, 0xa9, + 0x1e, 0x40, 0x82, 0x50, 0x38, 0xd3, 0x76, 0xa9, 0x7a, 0xba, 0x5d, 0xc2, 0x71, 0x48, 0x6f, 0x07, + 0x17, 0x13, 0x16, 0x0b, 0x63, 0x5c, 0x2f, 0x1b, 0xe3, 0x15, 0xe8, 0x1c, 0x5a, 0xb1, 0x69, 0xa5, + 0x49, 0x60, 0xda, 0x81, 0x47, 0x4a, 0xa7, 0x71, 0x38, 0xb4, 0xe2, 0x8d, 0x34, 0x09, 0x36, 0x03, + 0xf2, 0x9e, 0xdc, 0xd8, 0x94, 0x8b, 0x5e, 0xed, 0x8b, 0x9a, 0x1b, 0x2b, 0x73, 0xb7, 0x06, 0x67, + 0x45, 0x9c, 0xb8, 0x13, 0x4b, 0x09, 0xd4, 0xb4, 0x83, 0xd4, 0x4f, 0x68, 0x77, 0xac, 0xf1, 0xe5, + 0x1c, 0xc5, 0x83, 0x27, 0x9b, 0x88, 0x60, 0x1f, 0x40, 0xcf, 0x0e, 0x26, 0xa1, 0x19, 0x22, 0x5f, + 0xc9, 0xef, 0x90, 0x8e, 0x78, 0xd9, 0x2f, 0xe8, 0x20, 0xc5, 0xee, 0x91, 0x90, 0x8e, 0xd0, 0x3a, + 0x2c, 0xd9, 0x5e, 0x1a, 0x27, 0x22, 0x32, 0x0f, 0x54, 0x13, 0x7d, 0xae, 0x49, 0x57, 0x91, 0x48, + 0xe7, 0x09, 0x19, 0xdb, 0xda, 0x0d, 0xe2, 0x64, 0x6b, 0xe2, 0x65, 0x8a, 0x59, 0x79, 0x59, 0xc5, + 0xac, 0x2e, 0x56, 0xcc, 0x05, 0xaa, 0x51, 0x5b, 0xa0, 0x1a, 0x6c, 0x15, 0xfa, 0x65, 0x3a, 0x12, + 0xa9, 0x74, 0xe3, 0x7a, 0x05, 0x21, 0x89, 0x55, 0xf2, 0xd7, 0x91, 0x96, 0xa4, 0x91, 0xf1, 0x57, + 0x59, 0x11, 0x89, 0x74, 0x49, 0x43, 0x0a, 0xe6, 0x2b, 0x8d, 0xf9, 0xff, 0xf0, 0x46, 0xde, 0xd2, + 0x7c, 0xe2, 0x26, 0x87, 0x41, 0x9a, 0x98, 0x23, 0x3a, 0xb1, 0xc4, 0xca, 0xeb, 0xbe, 0x90, 0xf5, + 0xf4, 0x85, 0x44, 0xcb, 0xf3, 0x0c, 0xf9, 0x48, 0xa3, 0xd4, 0xf3, 0xcc, 0x44, 0x1c, 0x27, 0x4a, + 0x04, 0x03, 0xc9, 0x1b, 0xc5, 0xb7, 0xfb, 0xa9, 0xe7, 0xed, 0x8b, 0xe3, 0x04, 0x2d, 0xbe, 0x36, + 0x52, 0x15, 0xe3, 0x1f, 0xaa, 0x00, 0x0f, 0x02, 0xfb, 0x68, 0xdf, 0x8a, 0xc6, 0x22, 0x41, 0x5f, + 0x3e, 0xb3, 0x43, 0xca, 0x4e, 0xb6, 0x12, 0x69, 0x7d, 0xd8, 0x3a, 0x5c, 0xc8, 0xe6, 0x6f, 0x07, + 0x1e, 0x9d, 0x2b, 0xa4, 0x21, 0x51, 0xcb, 0x80, 0x29, 0xac, 0x3c, 0x99, 0x92, 0x15, 0x61, 0x1f, + 0x17, 0xbc, 0xc5, 0x36, 0xc9, 0x49, 0x48, 0xbc, 0x5d, 0xe4, 0x13, 0x76, 0x8b, 0xe6, 0xfb, 0x27, + 0x21, 0xfb, 0x00, 0xce, 0x47, 0x62, 0x14, 0x89, 0xf8, 0xd0, 0x4c, 0xe2, 0xf2, 0xc7, 0xa4, 0x4b, + 0xbf, 0xac, 0x90, 0xfb, 0x71, 0xfe, 0xad, 0x0f, 0xe0, 0xbc, 0xe4, 0xd4, 0xec, 0xf0, 0xa4, 0xd5, + 0x5d, 0x96, 0xc8, 0xf2, 0xe8, 0xde, 0x04, 0x0a, 0x70, 0x48, 0x4b, 0x9a, 0x39, 0x88, 0x1e, 0x31, + 0xe3, 0xc0, 0x13, 0xe8, 0x58, 0x6d, 0x1e, 0xe2, 0xa9, 0x73, 0x4b, 0x8c, 0x14, 0xf3, 0x0b, 0x00, + 0x33, 0xa0, 0xbe, 0x1d, 0x38, 0x82, 0x58, 0xdd, 0x5b, 0xef, 0xad, 0x51, 0xa8, 0x04, 0x39, 0x89, + 0x50, 0x4e, 0x38, 0xe3, 0x63, 0x68, 0x22, 0xe4, 0xf3, 0x90, 0xad, 0x41, 0x2b, 0x21, 0x0e, 0xc7, + 0x6a, 0xcf, 0x3b, 0x57, 0x98, 0xbe, 0x82, 0xfd, 0x3c, 0x23, 0x32, 0x38, 0x2c, 0xe5, 0x76, 0xe4, + 0xa1, 0xef, 0x3e, 0x4a, 0x05, 0xfb, 0x04, 0x96, 0xc3, 0x48, 0x28, 0xcd, 0x31, 0xd3, 0x23, 0xdc, + 0xd6, 0xd5, 0x22, 0x38, 0xa7, 0x04, 0x9d, 0xb7, 0x38, 0x42, 0x21, 0xf7, 0xc2, 0xa9, 0xba, 0xf1, + 0x43, 0xb8, 0x98, 0x53, 0xec, 0x09, 0x3b, 0xf0, 0x1d, 0x2b, 0x3a, 0x21, 0x93, 0x3f, 0xd3, 0x77, + 0xfc, 0x32, 0x7d, 0xef, 0x51, 0xdf, 0x3f, 0xae, 0x41, 0xef, 0x73, 0x7f, 0x2b, 0x0d, 0x3d, 0x17, + 0xcd, 0xf0, 0x67, 0xd2, 0x4a, 0x4a, 0xeb, 0x54, 0x29, 0x5b, 0xa7, 0x55, 0xe8, 0xab, 0xaf, 0xa0, + 0x94, 0xa4, 0x6d, 0x51, 0xc1, 0x10, 0x09, 0xdf, 0x0c, 0x3c, 0x69, 0x58, 0xbe, 0x03, 0xe7, 0x53, + 0x9a, 0xb9, 0xa4, 0x3c, 0x14, 0xf6, 0x91, 0xf9, 0x8c, 0x73, 0x0d, 0x93, 0x84, 0xd8, 0x14, 0xc9, + 0xc8, 0xca, 0x5c, 0x83, 0x76, 0xd1, 0x3c, 0x33, 0x91, 0x90, 0x13, 0xd2, 0x48, 0x02, 0xdf, 0x74, + 0xb2, 0x21, 0xab, 0x0d, 0x1a, 0x8d, 0x6b, 0x2f, 0x28, 0x66, 0x82, 0x2b, 0xff, 0x37, 0x60, 0x79, + 0x8a, 0x92, 0x46, 0x21, 0x7d, 0xa9, 0xf7, 0x0b, 0x31, 0x4e, 0x4f, 0xbf, 0x5c, 0xc5, 0xf1, 0xc8, + 0xcd, 0x6c, 0x29, 0x98, 0x86, 0x66, 0xd6, 0x60, 0xec, 0x07, 0x91, 0x50, 0x3a, 0x86, 0xd6, 0x80, + 0xea, 0x97, 0x76, 0xe0, 0xdc, 0xa2, 0x5e, 0x16, 0xec, 0x48, 0x2b, 0xe5, 0x1d, 0x69, 0xe6, 0x4c, + 0x56, 0xec, 0x4e, 0x7f, 0x5c, 0x81, 0xf6, 0xfd, 0xf4, 0xe9, 0xd3, 0x13, 0x69, 0x33, 0x58, 0x07, + 0x2a, 0x3b, 0xd4, 0x4b, 0x95, 0x57, 0x76, 0xd0, 0x85, 0xdd, 0x3d, 0x42, 0xfb, 0x45, 0x9d, 0xe8, + 0x5c, 0xd5, 0xf0, 0x34, 0xb7, 0x7b, 0xb4, 0x7f, 0xca, 0xca, 0x95, 0x68, 0x3c, 0xa3, 0xdc, 0x4d, + 0x5d, 0x0f, 0x1d, 0x1b, 0xb5, 0x48, 0xf3, 0x3a, 0x9e, 0x8f, 0x86, 0x23, 0xa9, 0x2f, 0xf7, 0xa3, + 0x60, 0x22, 0x35, 0x5a, 0x99, 0xc6, 0x05, 0x18, 0xe3, 0x6f, 0x6b, 0x50, 0xff, 0x34, 0x70, 0x7d, + 0x19, 0x5b, 0xf0, 0xa4, 0xf7, 0x2a, 0xdd, 0xc8, 0x56, 0x24, 0x3c, 0x74, 0x53, 0x11, 0x85, 0x8a, + 0xe1, 0xc9, 0xe3, 0x2f, 0xa1, 0xec, 0x40, 0xa2, 0x8a, 0x13, 0x70, 0x65, 0xe1, 0x09, 0x38, 0x3f, + 0xa0, 0xd6, 0x9f, 0x77, 0x40, 0xd5, 0x3d, 0x31, 0x42, 0x55, 0xf5, 0x1d, 0xe5, 0x88, 0x97, 0x3b, + 0xd3, 0x10, 0xb9, 0x19, 0xf8, 0x0e, 0xfb, 0x06, 0x40, 0xe4, 0x8e, 0x0f, 0x15, 0x65, 0x73, 0x3e, + 0x68, 0x40, 0x58, 0x22, 0xe5, 0xf0, 0x86, 0x8a, 0x44, 0x29, 0xc3, 0x6e, 0x1e, 0x20, 0x97, 0xe4, + 0x3c, 0x5a, 0xd9, 0xd9, 0x76, 0x71, 0x0c, 0xeb, 0xc2, 0x54, 0x0c, 0x8b, 0xb8, 0x4b, 0xf3, 0xbd, + 0x02, 0xb8, 0xbd, 0x1f, 0x9a, 0x81, 0x6f, 0x86, 0x59, 0x0c, 0x46, 0x43, 0xc8, 0xe7, 0xfe, 0xee, + 0x11, 0x9a, 0x39, 0x37, 0x36, 0x55, 0x28, 0x47, 0x1d, 0x8c, 0x4a, 0xe7, 0xe0, 0x15, 0xe8, 0xfc, + 0x28, 0x70, 0x7d, 0x73, 0x62, 0x85, 0x66, 0x42, 0x47, 0x51, 0x8a, 0x5f, 0x22, 0x6c, 0xdb, 0x0a, + 0xf7, 0xad, 0x31, 0xf9, 0x31, 0x2a, 0x38, 0x84, 0x8b, 0xa4, 0x2d, 0x09, 0x14, 0x08, 0xc5, 0x7b, + 0x19, 0x74, 0xea, 0x82, 0x42, 0x47, 0x1d, 0x29, 0x7b, 0x04, 0x20, 0x47, 0x8d, 0x7f, 0xa9, 0x82, + 0xb6, 0xe1, 0x27, 0x2e, 0xc9, 0xf3, 0x02, 0x34, 0x23, 0x3a, 0x07, 0x2b, 0x69, 0xaa, 0x5a, 0x2e, + 0xb1, 0xea, 0x33, 0x24, 0x36, 0x25, 0x89, 0xda, 0x0b, 0x4b, 0xa2, 0x7e, 0x9a, 0x24, 0xa6, 0xb9, + 0xd6, 0x38, 0x95, 0x6b, 0x73, 0xd1, 0x83, 0xaf, 0x42, 0x8c, 0xb3, 0x92, 0xd0, 0x9e, 0x27, 0x09, + 0x7d, 0x56, 0x12, 0xc6, 0x5f, 0xd4, 0x40, 0x7b, 0x20, 0x46, 0xc9, 0xd7, 0x8b, 0xe7, 0x97, 0x65, + 0xf1, 0x18, 0xff, 0x5e, 0x03, 0x9d, 0xe3, 0x0c, 0xbf, 0x42, 0x99, 0xdd, 0x06, 0x20, 0x59, 0x9c, + 0x2e, 0x38, 0x92, 0x97, 0x0c, 0x50, 0x7d, 0x08, 0x6d, 0x29, 0x13, 0xd9, 0xa2, 0xf1, 0x8c, 0x16, + 0x52, 0x70, 0xfb, 0xf3, 0xf2, 0x6e, 0xbe, 0xb0, 0xbc, 0x5b, 0xaf, 0x2c, 0x6f, 0xed, 0x75, 0xc8, + 0x5b, 0x3f, 0x55, 0xde, 0xf0, 0x3c, 0x79, 0xb7, 0x9f, 0x27, 0xef, 0xce, 0x9c, 0xbc, 0x7f, 0x5c, + 0x83, 0x2e, 0xc9, 0x7b, 0x4f, 0x4c, 0xbe, 0x9c, 0x51, 0x9c, 0x11, 0x52, 0xed, 0x65, 0x85, 0x54, + 0x7f, 0x61, 0x21, 0x35, 0x5e, 0x59, 0x48, 0xcd, 0xd7, 0x21, 0xa4, 0xd6, 0xa9, 0x42, 0xd2, 0x9e, + 0x27, 0x24, 0xfd, 0xe5, 0x17, 0x65, 0x2e, 0xa4, 0x2f, 0xbd, 0x73, 0x7d, 0x2d, 0xa4, 0xd7, 0x24, + 0x24, 0x98, 0x13, 0x12, 0x7a, 0x16, 0x5f, 0x7a, 0x11, 0x7d, 0x15, 0x9e, 0xc5, 0xa9, 0xcc, 0x6e, + 0xbc, 0x0e, 0x66, 0x37, 0x4f, 0x65, 0x76, 0xeb, 0x79, 0xcc, 0x7e, 0x05, 0xcf, 0xe2, 0x2f, 0x6b, + 0x00, 0x7b, 0xae, 0x3f, 0xf6, 0xc4, 0xd7, 0xbe, 0xc5, 0x2f, 0x8d, 0x6f, 0xf1, 0xd7, 0x55, 0xd0, + 0xb6, 0xad, 0xe8, 0xe8, 0x17, 0x6e, 0x85, 0xbc, 0x05, 0xad, 0xc0, 0x2f, 0xaf, 0x87, 0x32, 0x5d, + 0x33, 0xf0, 0x7f, 0x21, 0x54, 0xfe, 0x6f, 0x6a, 0xa0, 0x6f, 0x09, 0x27, 0x0d, 0x89, 0x7d, 0x53, + 0x6c, 0xa8, 0xbc, 0x30, 0x1b, 0xaa, 0xaf, 0xac, 0x73, 0xb5, 0x57, 0xd3, 0xb9, 0x69, 0xbe, 0xd4, + 0x9f, 0xc7, 0x97, 0xc6, 0xf3, 0xf8, 0xd2, 0x9c, 0x3b, 0xee, 0x3d, 0x80, 0xb3, 0x53, 0xf1, 0x10, + 0x4b, 0xde, 0x7c, 0xb5, 0x28, 0x12, 0x76, 0x45, 0x8e, 0x77, 0x27, 0x70, 0xa6, 0x42, 0x22, 0xf2, + 0x3e, 0x8c, 0x2f, 0x07, 0xb3, 0x20, 0xf6, 0x36, 0xf4, 0x1c, 0x64, 0x32, 0x85, 0x79, 0x28, 0xaa, + 0xaa, 0x51, 0xfc, 0xa1, 0x43, 0xd0, 0xcd, 0xc0, 0xa3, 0x28, 0xc4, 0xc7, 0xb0, 0x54, 0x50, 0x25, + 0xf9, 0xf5, 0xea, 0xc2, 0x48, 0x62, 0xd6, 0x90, 0x76, 0x53, 0xe3, 0xb7, 0x2b, 0xd0, 0xda, 0x8d, + 0x02, 0x27, 0xb5, 0x93, 0x57, 0xb4, 0x5a, 0xd3, 0x1c, 0xad, 0x3d, 0x8f, 0xa3, 0xf5, 0x59, 0x8e, + 0x1a, 0xbf, 0x53, 0x01, 0x5d, 0x0d, 0xe1, 0xc1, 0xfa, 0x57, 0x64, 0x3a, 0x9f, 0x3f, 0x8a, 0x27, + 0xa0, 0x53, 0xb4, 0xef, 0x54, 0x63, 0x70, 0xaa, 0x46, 0x56, 0x5f, 0x49, 0x23, 0x8d, 0xdf, 0xaf, + 0x40, 0x97, 0x42, 0xaa, 0xf7, 0x53, 0x5f, 0xca, 0x7c, 0x71, 0x6c, 0x70, 0x05, 0xea, 0x91, 0x48, + 0xb2, 0xc4, 0x82, 0x8e, 0xfc, 0xcc, 0x66, 0xe0, 0x6d, 0x89, 0x11, 0x27, 0x0c, 0x32, 0xc1, 0x8a, + 0xc6, 0xf1, 0xa2, 0xd4, 0x06, 0x84, 0xe3, 0xac, 0x42, 0x2b, 0xb2, 0x26, 0x71, 0x96, 0xda, 0x20, + 0x6b, 0x8c, 0x41, 0x9d, 0x34, 0xab, 0x41, 0x9a, 0x45, 0x65, 0x63, 0x03, 0xce, 0xdf, 0x3b, 0x4e, + 0x44, 0xe4, 0x5b, 0xa4, 0x61, 0xeb, 0x9b, 0x81, 0x27, 0x83, 0xa1, 0x19, 0x71, 0xa5, 0x20, 0xc6, + 0x01, 0x97, 0x13, 0xb7, 0x64, 0xc5, 0xb8, 0x01, 0xed, 0x91, 0xeb, 0x09, 0x33, 0x18, 0x8d, 0x62, + 0x91, 0xe0, 0xd7, 0x65, 0x89, 0xa6, 0x55, 0xe3, 0xaa, 0x66, 0xfc, 0xa4, 0x0e, 0x9d, 0xec, 0x53, + 0x94, 0xd8, 0xb2, 0x78, 0xfa, 0x97, 0x41, 0xa7, 0xde, 0x62, 0xf7, 0xa9, 0x20, 0x1e, 0xd4, 0xb8, + 0x86, 0x00, 0xca, 0x44, 0xd8, 0x80, 0xe5, 0xd2, 0xa7, 0xcc, 0x24, 0x48, 0x2c, 0x4f, 0xb1, 0xa1, + 0x74, 0x7d, 0x5a, 0x22, 0xe1, 0x4b, 0x58, 0xf9, 0x9c, 0xca, 0xfb, 0x48, 0x8d, 0xec, 0xcd, 0x43, + 0xa1, 0x73, 0xec, 0x45, 0x0c, 0xfb, 0x3e, 0x2c, 0xe1, 0x6c, 0xd7, 0x65, 0x04, 0x9d, 0xe6, 0x2b, + 0xad, 0xf3, 0xb5, 0xe2, 0x13, 0x0b, 0x79, 0xc6, 0xbb, 0xfe, 0x14, 0x0b, 0xdf, 0x04, 0xb0, 0x23, + 0x81, 0xb6, 0x21, 0x7e, 0xe4, 0x91, 0x05, 0xd1, 0xb9, 0x2e, 0x21, 0x7b, 0x8f, 0xbc, 0x7c, 0xa6, + 0xf9, 0xd6, 0xaa, 0xcb, 0x99, 0x92, 0xa2, 0xbf, 0x0f, 0xed, 0x20, 0x72, 0xc7, 0xae, 0x2f, 0x03, + 0xb7, 0xda, 0x82, 0xd1, 0x82, 0x24, 0xa0, 0x30, 0xae, 0x01, 0x4d, 0xa9, 0xa8, 0x0b, 0x2e, 0x91, + 0x14, 0x86, 0x71, 0xe8, 0xed, 0x1f, 0xa0, 0x41, 0xa0, 0xfc, 0xc0, 0xcd, 0xc0, 0xa3, 0x7c, 0x8b, + 0xf6, 0xfa, 0xad, 0xf9, 0x69, 0xa1, 0x7c, 0xd6, 0xa6, 0x89, 0x65, 0xe8, 0x76, 0xa6, 0x07, 0x76, + 0x13, 0x96, 0xe2, 0x24, 0x72, 0xed, 0x04, 0xa7, 0x68, 0x4e, 0x02, 0x47, 0xd0, 0xfe, 0xab, 0xf1, + 0xae, 0x04, 0xef, 0x3d, 0xf2, 0xb6, 0x03, 0x47, 0x5c, 0xda, 0x80, 0xb3, 0x0b, 0xba, 0x7b, 0xa9, + 0x5b, 0x45, 0x1b, 0x60, 0x2f, 0x89, 0x84, 0x35, 0x21, 0xe5, 0x79, 0x07, 0x5a, 0xc9, 0x81, 0x47, + 0x57, 0x86, 0x95, 0x85, 0x57, 0x86, 0xcd, 0xe4, 0x00, 0xb9, 0x54, 0x52, 0xc7, 0x2a, 0x5d, 0xde, + 0xa9, 0x1a, 0x7e, 0xc8, 0x73, 0x27, 0x6e, 0xa2, 0x32, 0x01, 0x65, 0xc5, 0xf8, 0x08, 0x74, 0xea, + 0x81, 0xbe, 0x91, 0xfb, 0x61, 0x95, 0x53, 0xfd, 0x30, 0xe3, 0x3d, 0xd0, 0x7f, 0x1d, 0x87, 0x49, + 0x8d, 0xae, 0x41, 0x9b, 0xae, 0x95, 0xcd, 0x03, 0x2f, 0xb0, 0x8f, 0xb2, 0xeb, 0x4e, 0x02, 0xdd, + 0x45, 0x88, 0x01, 0xa0, 0x3d, 0xf4, 0xdd, 0xc0, 0xdf, 0xf0, 0x3c, 0xe3, 0x0f, 0xea, 0xa0, 0xff, + 0xc0, 0x8a, 0x0f, 0xc9, 0x4a, 0xb0, 0x15, 0x68, 0xef, 0x08, 0xe1, 0x20, 0x60, 0xdb, 0x0a, 0x55, + 0xca, 0x51, 0x19, 0xc4, 0x2e, 0x81, 0xf6, 0x03, 0xb9, 0xf3, 0x7f, 0xa6, 0x2e, 0xf2, 0xf2, 0x7a, + 0xd6, 0x9a, 0xae, 0xad, 0x45, 0x96, 0xdd, 0x52, 0x06, 0xb1, 0x5b, 0xd0, 0xc7, 0x2a, 0x25, 0xf6, + 0xa0, 0x0e, 0x0a, 0x2f, 0x56, 0x3b, 0xe3, 0x1c, 0x9c, 0xdd, 0x02, 0xc0, 0xbd, 0x99, 0x2e, 0xc4, + 0xe3, 0x05, 0xde, 0x49, 0x09, 0xcb, 0xae, 0x02, 0x7c, 0x9a, 0x1b, 0xd8, 0x6c, 0xa7, 0x2c, 0x20, + 0xec, 0x6d, 0xe8, 0xaa, 0x1a, 0x17, 0xa3, 0x4d, 0x75, 0x8d, 0xda, 0xe0, 0xd3, 0x40, 0x76, 0x0f, + 0x96, 0xf9, 0x4b, 0xa7, 0x33, 0xce, 0x81, 0x70, 0xf3, 0xa0, 0xcb, 0x43, 0x27, 0x0d, 0x95, 0x33, + 0xd9, 0x72, 0x63, 0xf2, 0x5f, 0x9e, 0xb5, 0x63, 0xc3, 0xeb, 0xda, 0xb1, 0xdb, 0x2f, 0xb6, 0x63, + 0x77, 0x5e, 0x6c, 0xc7, 0xfe, 0x79, 0x0d, 0x3a, 0x6a, 0x73, 0xa5, 0xcd, 0x67, 0x4a, 0xf8, 0x95, + 0xd3, 0x85, 0x5f, 0x7d, 0x31, 0xe1, 0xd7, 0x5e, 0x48, 0xf8, 0xf5, 0x53, 0x85, 0xbf, 0x50, 0x6c, + 0x8d, 0x97, 0x16, 0xdb, 0xf3, 0x74, 0xe8, 0x2a, 0xc0, 0x5e, 0xee, 0x7b, 0x29, 0x05, 0x2a, 0x41, + 0xa6, 0xc4, 0xae, 0xbd, 0x90, 0xd8, 0xf5, 0xd7, 0x25, 0x76, 0x78, 0x31, 0xb1, 0xb7, 0x5f, 0x4c, + 0xec, 0x7b, 0x00, 0xb4, 0x7b, 0x48, 0x99, 0x2f, 0xe4, 0x6e, 0xe5, 0x65, 0xb9, 0x6b, 0xfc, 0x77, + 0x05, 0x60, 0xcf, 0x9a, 0x84, 0xd2, 0xf9, 0x60, 0xdf, 0x83, 0x76, 0x4c, 0x35, 0x79, 0x57, 0x21, + 0xd3, 0xd6, 0x4b, 0xbb, 0x5b, 0x41, 0xaa, 0x8a, 0x38, 0x34, 0x0e, 0x71, 0x5e, 0x26, 0xef, 0x58, + 0xf6, 0x90, 0x27, 0x15, 0x34, 0x32, 0x02, 0xca, 0x27, 0xb8, 0x01, 0x3d, 0x45, 0x10, 0x8a, 0xc8, + 0x16, 0xbe, 0xb4, 0xb3, 0x15, 0xde, 0x95, 0xd0, 0x5d, 0x09, 0x64, 0x1f, 0xe6, 0x64, 0x76, 0xe0, + 0xa5, 0x93, 0x85, 0xda, 0xa6, 0x9a, 0x6c, 0x4a, 0x02, 0x63, 0x3d, 0x9b, 0x0a, 0x0d, 0x44, 0x83, + 0x3a, 0x7e, 0xaf, 0x7f, 0x86, 0xb5, 0xa1, 0xa5, 0x7a, 0xed, 0x57, 0x58, 0x17, 0x74, 0xca, 0xac, + 0x25, 0x5c, 0xd5, 0xf8, 0xa3, 0xb3, 0xd0, 0x1e, 0xfa, 0x71, 0x12, 0xa5, 0x52, 0x88, 0x45, 0x02, + 0x69, 0x83, 0x12, 0x48, 0x55, 0x56, 0x89, 0x9c, 0x06, 0x65, 0x95, 0xdc, 0x84, 0xba, 0xe5, 0x27, + 0xae, 0x72, 0x34, 0x4b, 0x59, 0xca, 0x59, 0x28, 0x8c, 0x13, 0x9e, 0xbd, 0x0f, 0x2d, 0x95, 0xd2, + 0xac, 0x32, 0x06, 0x17, 0xe6, 0x43, 0x67, 0x34, 0x6c, 0x0d, 0x34, 0x47, 0xe5, 0x5a, 0xab, 0x45, + 0x52, 0xea, 0x3a, 0xcb, 0xc2, 0xe6, 0x39, 0x0d, 0xbb, 0x0e, 0x35, 0x6b, 0x2c, 0xd7, 0x03, 0x65, + 0x79, 0x64, 0xa4, 0x94, 0xa1, 0xca, 0x11, 0xc7, 0x0c, 0xa8, 0xa3, 0x7b, 0x4b, 0x6b, 0x82, 0xb6, + 0xc1, 0x8c, 0x46, 0x8e, 0x12, 0x71, 0xec, 0xb6, 0x3a, 0xb5, 0x11, 0xa1, 0x36, 0xfb, 0xdd, 0xec, + 0xaa, 0x44, 0x9e, 0xde, 0x3e, 0x55, 0x0d, 0x62, 0x31, 0x71, 0x65, 0x03, 0x7d, 0xb6, 0x41, 0x16, + 0x6e, 0xe2, 0x5a, 0x9c, 0x05, 0x9e, 0xee, 0x40, 0x3b, 0xa6, 0xb8, 0x88, 0x6c, 0x02, 0xd9, 0xad, + 0x79, 0xde, 0x24, 0x0f, 0x9a, 0x70, 0x88, 0x8b, 0x00, 0xca, 0x6d, 0xd0, 0x27, 0x56, 0x74, 0x24, + 0x1b, 0xb5, 0x67, 0xbf, 0x93, 0x1d, 0xda, 0xb9, 0x36, 0xc9, 0x8e, 0xef, 0xeb, 0x00, 0x72, 0x61, + 0x51, 0x8b, 0xce, 0x2c, 0xcb, 0xf3, 0x83, 0x2a, 0xd7, 0x9d, 0xfc, 0xcc, 0xfa, 0x2e, 0xb4, 0x42, + 0x79, 0xee, 0xa0, 0x74, 0xa8, 0xf6, 0xfa, 0x72, 0xd1, 0x40, 0x1d, 0x48, 0x78, 0x46, 0xc1, 0xbe, + 0x0b, 0x3d, 0x99, 0xfb, 0x31, 0x52, 0x6e, 0x3a, 0xa5, 0x48, 0x4d, 0xa5, 0xda, 0x4e, 0x79, 0xf1, + 0xbc, 0x9b, 0x4c, 0x39, 0xf5, 0xdf, 0x86, 0xae, 0x50, 0x5e, 0x94, 0x19, 0xdb, 0x96, 0x3f, 0xe8, + 0x53, 0xf3, 0x0b, 0x8b, 0x9d, 0x2c, 0xde, 0x11, 0x65, 0x97, 0x78, 0x15, 0x9a, 0x2a, 0x27, 0x66, + 0x99, 0x5a, 0x95, 0x9e, 0x89, 0xc8, 0xdb, 0x61, 0xae, 0xf0, 0xec, 0xee, 0xcc, 0xbd, 0x3d, 0xba, + 0x51, 0x2c, 0xcb, 0x77, 0x59, 0x7c, 0x19, 0x3f, 0x75, 0xa3, 0xff, 0x99, 0x38, 0x41, 0x5e, 0x16, + 0xf9, 0x0e, 0x83, 0xb3, 0xb3, 0xbc, 0xcc, 0x93, 0x1d, 0xb8, 0x9e, 0xe7, 0x39, 0xa0, 0x41, 0x2a, + 0xe7, 0x5f, 0xc8, 0x2b, 0xec, 0x73, 0xd4, 0xf4, 0x8d, 0x05, 0x4d, 0xe5, 0x4d, 0x36, 0x5f, 0x0a, + 0x67, 0xd2, 0x38, 0xde, 0x03, 0x2d, 0x88, 0x1c, 0xca, 0x7d, 0x1a, 0x9c, 0xa7, 0x15, 0xbf, 0xac, + 0x52, 0x98, 0x64, 0xae, 0x38, 0x19, 0xb2, 0x56, 0x20, 0x2b, 0xec, 0x7d, 0xe8, 0x84, 0x51, 0xf0, + 0x23, 0x61, 0x27, 0xd2, 0x59, 0xbe, 0x30, 0x9f, 0x63, 0xae, 0xf0, 0xe4, 0x3b, 0x17, 0xce, 0xf0, + 0xc5, 0x67, 0x3a, 0xc3, 0x2b, 0x99, 0xfb, 0x37, 0x98, 0xcf, 0x15, 0x20, 0x04, 0xf6, 0xa2, 0x1c, + 0xc7, 0x37, 0xe6, 0x7b, 0x51, 0x4e, 0xe4, 0x00, 0x5a, 0x6e, 0x7c, 0xdf, 0x8d, 0xe2, 0x64, 0x70, + 0x29, 0xdb, 0x74, 0xa8, 0x8a, 0x6e, 0xa7, 0x1b, 0x3f, 0xb0, 0xe2, 0x64, 0x70, 0x39, 0x7b, 0x26, + 0x80, 0x35, 0xe4, 0xb9, 0x0c, 0x8b, 0x90, 0xfe, 0x5e, 0x99, 0xe5, 0x79, 0x7e, 0x05, 0xa6, 0xe2, + 0x23, 0xa4, 0xbf, 0x9f, 0xc0, 0x92, 0x6c, 0x53, 0x2c, 0xc9, 0x37, 0x67, 0x75, 0x72, 0xea, 0x2e, + 0x85, 0x77, 0xa3, 0xa9, 0xab, 0x95, 0xbc, 0x03, 0x34, 0x59, 0xb2, 0x83, 0xab, 0x0b, 0x3b, 0xc8, + 0x8d, 0x9b, 0xec, 0x20, 0x0f, 0xfb, 0xdf, 0x82, 0xa6, 0x4a, 0xe4, 0xba, 0x36, 0x67, 0xb4, 0x54, + 0xca, 0x22, 0x57, 0x14, 0xec, 0x1b, 0xd0, 0xa2, 0x7c, 0xa2, 0x20, 0x1c, 0xac, 0xcc, 0x2a, 0xb1, + 0xcc, 0x03, 0xe2, 0x4d, 0x4f, 0xe6, 0x03, 0xbd, 0x0b, 0xad, 0x2c, 0x9e, 0x70, 0x7d, 0x76, 0x61, + 0xaa, 0xbd, 0x9d, 0x67, 0x14, 0xec, 0x06, 0x34, 0x26, 0x68, 0xd2, 0x07, 0xc6, 0xac, 0x31, 0x94, + 0x96, 0x5e, 0x62, 0xc9, 0x10, 0xd1, 0x31, 0x41, 0xae, 0xbe, 0xb7, 0xe6, 0x0c, 0x51, 0x7e, 0x86, + 0xe0, 0x10, 0x17, 0xe7, 0x89, 0xdf, 0x82, 0x4b, 0xe5, 0xdc, 0x9f, 0x2c, 0x31, 0x48, 0x9d, 0xff, + 0xde, 0xa6, 0x5e, 0xae, 0x2f, 0x50, 0xf0, 0xe9, 0x14, 0x22, 0x7e, 0x31, 0x7c, 0x46, 0x6e, 0xd1, + 0x9d, 0x7c, 0xc3, 0x44, 0xbb, 0x32, 0xb8, 0x31, 0x37, 0xac, 0x7c, 0xcb, 0xcd, 0xb6, 0x51, 0xda, + 0xa9, 0x3f, 0x86, 0xce, 0x28, 0x7d, 0xfa, 0xf4, 0x44, 0x85, 0x21, 0x06, 0x37, 0xa9, 0x5d, 0xe9, + 0xac, 0x5b, 0xca, 0x64, 0xe1, 0xed, 0x51, 0x29, 0xad, 0xe5, 0x22, 0xb4, 0x6c, 0xdf, 0xb4, 0x1c, + 0x27, 0x1a, 0xbc, 0x23, 0x33, 0x59, 0x6c, 0x7f, 0xc3, 0x71, 0x28, 0x25, 0x28, 0x08, 0x05, 0xbd, + 0xb7, 0x30, 0x5d, 0x67, 0xb0, 0x2a, 0xb7, 0xee, 0x0c, 0x34, 0x74, 0xe8, 0x25, 0x97, 0x15, 0x59, + 0x9e, 0x27, 0x3c, 0x24, 0xf8, 0x86, 0x7a, 0xc9, 0xa5, 0x40, 0x43, 0x87, 0x5d, 0x87, 0xce, 0xc4, + 0x3a, 0x36, 0x33, 0xc8, 0xe0, 0x96, 0x7c, 0x26, 0x33, 0xb1, 0x8e, 0x77, 0x15, 0x08, 0xd5, 0x5c, + 0x66, 0xd9, 0x92, 0xb2, 0xbd, 0x3b, 0xab, 0xe6, 0x79, 0x04, 0x86, 0xeb, 0x6e, 0x1e, 0x8c, 0x21, + 0x73, 0x44, 0x46, 0xd8, 0xf4, 0xd6, 0x07, 0xef, 0xcd, 0x9b, 0x23, 0x15, 0x3a, 0x42, 0x73, 0x94, + 0x45, 0x91, 0xd6, 0x01, 0xa4, 0xb5, 0x26, 0x61, 0xbf, 0x3f, 0xdb, 0x26, 0x3f, 0xcb, 0x71, 0x99, + 0x62, 0x4a, 0xa2, 0x5e, 0x07, 0xa0, 0x53, 0xa5, 0x6c, 0xb3, 0x36, 0xdb, 0x26, 0x3f, 0xca, 0x71, + 0xfd, 0x71, 0x7e, 0xaa, 0xbb, 0x0d, 0x7a, 0x8a, 0x87, 0x36, 0xd3, 0xf2, 0xbc, 0xc1, 0xed, 0xd9, + 0x35, 0x90, 0x9d, 0xe7, 0xb8, 0x96, 0xaa, 0x12, 0x7e, 0x84, 0xa2, 0xb6, 0xe4, 0xc6, 0x0d, 0x3e, + 0x98, 0xfd, 0x48, 0x7e, 0xe8, 0xe3, 0xfa, 0x61, 0x7e, 0xfe, 0xfb, 0x36, 0x74, 0xb3, 0x90, 0xa3, + 0x6c, 0xf6, 0xe1, 0xec, 0xd6, 0x51, 0x3e, 0x0f, 0xf0, 0xec, 0xad, 0x92, 0x6c, 0x7c, 0x07, 0xda, + 0x92, 0xe3, 0xb2, 0xe9, 0xfa, 0xac, 0x82, 0x15, 0x4e, 0x25, 0x97, 0xa2, 0x91, 0xcd, 0x6e, 0x40, + 0xc3, 0x0a, 0x43, 0xef, 0x64, 0xf0, 0xd1, 0xec, 0xaa, 0xda, 0x40, 0x30, 0x97, 0x58, 0xd4, 0xc3, + 0x49, 0xea, 0x25, 0x6e, 0x96, 0x2f, 0xfb, 0xcd, 0x59, 0x3d, 0x2c, 0xe5, 0xe3, 0xf3, 0xf6, 0xa4, + 0xf4, 0x70, 0xe0, 0x3d, 0xd0, 0xc2, 0x20, 0x4e, 0x4c, 0x67, 0xe2, 0x0d, 0xee, 0xcc, 0xed, 0xbe, + 0x32, 0x29, 0x93, 0xb7, 0x42, 0x59, 0x30, 0xee, 0x40, 0x67, 0x83, 0x5e, 0x29, 0xba, 0x31, 0x99, + 0xf2, 0x1b, 0x50, 0xcf, 0x23, 0x84, 0xf9, 0x1e, 0x41, 0x14, 0x4f, 0xc5, 0xd0, 0x1f, 0x05, 0x9c, + 0xd0, 0xc6, 0x9f, 0xd5, 0xa0, 0xb9, 0x17, 0xa4, 0x91, 0x2d, 0x9e, 0x9f, 0x6e, 0xfc, 0x66, 0xa6, + 0x32, 0x7e, 0x91, 0xe5, 0x25, 0xb5, 0x83, 0xd0, 0xe5, 0xe0, 0x63, 0x8d, 0x82, 0x32, 0x79, 0xf0, + 0xf1, 0x1c, 0x34, 0xe4, 0xa1, 0x5e, 0x26, 0xbc, 0xca, 0x0a, 0x2d, 0x97, 0x34, 0x3e, 0x74, 0x82, + 0x27, 0x3e, 0x2e, 0x97, 0x06, 0xe5, 0x8b, 0x42, 0x06, 0x1a, 0x3a, 0xf4, 0x66, 0x23, 0x23, 0xa0, + 0xf5, 0x28, 0x23, 0x41, 0x9d, 0x0c, 0x48, 0xab, 0x32, 0x0b, 0x6c, 0xb6, 0x9e, 0x11, 0xd8, 0xbc, + 0x05, 0x79, 0x0e, 0xb4, 0x72, 0xe0, 0x9e, 0x9d, 0x23, 0xbd, 0x0e, 0x7a, 0xfe, 0x86, 0x55, 0x39, + 0x6f, 0xe7, 0xd6, 0x8a, 0x57, 0xad, 0xfb, 0x59, 0x89, 0x17, 0x64, 0x0b, 0x22, 0x9e, 0x61, 0x14, + 0x1c, 0xa8, 0xe0, 0x14, 0xbc, 0x4c, 0xc4, 0x73, 0x17, 0xdb, 0x65, 0x71, 0x5c, 0x37, 0x36, 0xed, + 0xc0, 0x8f, 0x13, 0x15, 0x15, 0x6a, 0xb9, 0xf1, 0x26, 0x56, 0x8d, 0xdf, 0x04, 0x0d, 0x8f, 0x5c, + 0x28, 0x42, 0xc6, 0xa0, 0x3e, 0xb1, 0xc3, 0x54, 0xb9, 0xe3, 0x54, 0x56, 0x4f, 0x54, 0xa5, 0x70, + 0xd4, 0x13, 0x55, 0x62, 0x5d, 0x4d, 0x46, 0x23, 0xb1, 0x2c, 0x1f, 0xc6, 0x9d, 0x78, 0x81, 0xe5, + 0x28, 0x81, 0x64, 0x55, 0xe3, 0x4f, 0x2b, 0xb0, 0xbc, 0x1b, 0x05, 0xb6, 0x88, 0xe3, 0x07, 0xb8, + 0x97, 0x5b, 0xe4, 0x99, 0x31, 0xa8, 0x53, 0x50, 0x51, 0xbe, 0x1b, 0xa3, 0x32, 0x2a, 0x83, 0x8c, + 0xd6, 0xe4, 0xc7, 0x98, 0x1a, 0xd7, 0x09, 0x42, 0xa7, 0x98, 0x1c, 0x4d, 0x0d, 0x6b, 0x25, 0x34, + 0x85, 0x23, 0x6f, 0x40, 0xaf, 0x78, 0x55, 0x40, 0x3d, 0xa8, 0x07, 0xa3, 0x39, 0x94, 0x7a, 0xb9, + 0x06, 0xed, 0x48, 0x58, 0xe8, 0xed, 0x50, 0x37, 0x0d, 0xa2, 0x01, 0x09, 0xc2, 0x7e, 0x8c, 0x43, + 0xe8, 0xef, 0x46, 0x22, 0xb4, 0x22, 0x81, 0x06, 0x74, 0x42, 0x5c, 0xb9, 0x00, 0x4d, 0x4f, 0xf8, + 0xe3, 0xe4, 0x50, 0x8d, 0x57, 0xd5, 0xf2, 0x07, 0xc1, 0xd5, 0xd2, 0x83, 0x60, 0xe4, 0x4e, 0x24, + 0x2c, 0xf5, 0x6e, 0x98, 0xca, 0xa8, 0xac, 0x7e, 0xea, 0xa9, 0x40, 0xa7, 0xc6, 0x65, 0xc5, 0xf8, + 0x93, 0x1a, 0xb4, 0x15, 0x67, 0xe8, 0x2b, 0x92, 0xcf, 0x95, 0x9c, 0xcf, 0x7d, 0xa8, 0xc5, 0x8f, + 0x3c, 0xc5, 0x78, 0x2c, 0xb2, 0x8f, 0xa0, 0xe6, 0xb9, 0x13, 0x75, 0x0e, 0xba, 0x3c, 0x65, 0x8e, + 0xa7, 0xf9, 0xab, 0x8e, 0xb3, 0x48, 0xcd, 0x2e, 0x93, 0xb9, 0x3c, 0x36, 0x51, 0x2b, 0x14, 0x4f, + 0xd0, 0x34, 0x1e, 0xa3, 0xea, 0x21, 0x53, 0x2d, 0x9b, 0x72, 0x5e, 0xb3, 0xf5, 0xd2, 0xe5, 0xba, + 0x82, 0x0c, 0x1d, 0xf6, 0x4d, 0xd0, 0x62, 0xdf, 0x0a, 0xe3, 0xc3, 0x20, 0x51, 0xe7, 0x1e, 0xb6, + 0x96, 0x1c, 0xfb, 0x6b, 0x9b, 0x3b, 0xfb, 0xc7, 0xfe, 0x9e, 0xc2, 0xa8, 0x8f, 0xe5, 0x94, 0xec, + 0xbb, 0xd0, 0x89, 0x45, 0x1c, 0xcb, 0xe7, 0x1d, 0xa3, 0x40, 0xad, 0xa3, 0xf3, 0xe5, 0x33, 0x0b, + 0x61, 0x71, 0xd6, 0xaa, 0x71, 0x3b, 0x2e, 0x40, 0xec, 0x07, 0xd0, 0xcb, 0xda, 0x7b, 0xc1, 0x78, + 0x2c, 0xb2, 0x04, 0xfe, 0xcb, 0x73, 0x3d, 0x3c, 0x20, 0x74, 0xa9, 0x9f, 0x6e, 0x5c, 0x46, 0xb0, + 0xef, 0x43, 0x2f, 0x94, 0xc2, 0x34, 0x55, 0x14, 0x5e, 0x2e, 0xc1, 0x4b, 0x53, 0xde, 0xc3, 0x94, + 0xb0, 0x8b, 0xe4, 0xef, 0x02, 0x1e, 0x1b, 0xff, 0x59, 0x81, 0x76, 0x69, 0xd4, 0xf4, 0x4c, 0x3b, + 0x16, 0x51, 0x16, 0x91, 0xc7, 0x32, 0xc2, 0x0e, 0x03, 0xf5, 0xf2, 0x51, 0xe7, 0x54, 0x46, 0x58, + 0x14, 0xa8, 0x2b, 0x1a, 0x9d, 0x53, 0x19, 0x6d, 0x90, 0x3a, 0x82, 0xca, 0x87, 0x63, 0x24, 0x94, + 0x3a, 0xef, 0x14, 0xc0, 0x21, 0x05, 0x98, 0x50, 0x9d, 0x0e, 0xac, 0x38, 0xbb, 0x23, 0xc8, 0xeb, + 0xb8, 0xd8, 0x1e, 0x8b, 0x08, 0xc7, 0xa2, 0xcc, 0x57, 0x56, 0x45, 0x59, 0x93, 0xd9, 0x78, 0x1a, + 0xf8, 0xf2, 0x02, 0xb2, 0xc3, 0x35, 0x04, 0xfc, 0x30, 0xf0, 0xa9, 0x99, 0x92, 0xac, 0xba, 0xcf, + 0xca, 0xaa, 0x68, 0x1c, 0x1e, 0xa5, 0x02, 0x3d, 0x2c, 0x87, 0xee, 0xb0, 0x74, 0xde, 0xa2, 0xfa, + 0xd0, 0x31, 0xfe, 0xb5, 0x02, 0xcb, 0x73, 0xcc, 0x46, 0x87, 0x06, 0x19, 0x9d, 0xe5, 0xe4, 0x77, + 0x78, 0x13, 0xab, 0x43, 0x87, 0x10, 0xc9, 0x84, 0x94, 0xa9, 0xaa, 0x10, 0xc9, 0x04, 0x35, 0xe9, + 0x3c, 0x34, 0x93, 0x63, 0x9a, 0xad, 0x5c, 0x18, 0x8d, 0xe4, 0x18, 0xa7, 0xb9, 0x01, 0xba, 0x17, + 0x8c, 0x4d, 0x4f, 0x3c, 0x16, 0x1e, 0xf1, 0xa1, 0xb7, 0xfe, 0xf6, 0x29, 0x52, 0x5e, 0x7b, 0x10, + 0x8c, 0x1f, 0x20, 0x2d, 0xd7, 0x3c, 0x55, 0x32, 0x3e, 0x05, 0x2d, 0x83, 0x32, 0x1d, 0x1a, 0x5b, + 0xe2, 0x20, 0x1d, 0xf7, 0xcf, 0x30, 0x0d, 0xea, 0xd8, 0xa2, 0x5f, 0xc1, 0xd2, 0x17, 0x56, 0xe4, + 0xf7, 0xab, 0x88, 0xbe, 0x17, 0x45, 0x41, 0xd4, 0xaf, 0x61, 0x71, 0xd7, 0xf2, 0x5d, 0xbb, 0x5f, + 0xc7, 0xe2, 0x7d, 0x2b, 0xb1, 0xbc, 0x7e, 0xc3, 0xf8, 0xf3, 0x06, 0x68, 0xbb, 0xea, 0xeb, 0x6c, + 0x0b, 0xba, 0xf9, 0x2b, 0xfa, 0xc5, 0xb1, 0x99, 0xdd, 0xd9, 0x02, 0xc5, 0x66, 0x3a, 0x61, 0xa9, + 0x36, 0xfb, 0x16, 0xbf, 0x3a, 0xf7, 0x16, 0xff, 0x0a, 0xd4, 0x1e, 0x45, 0x27, 0xd3, 0xb7, 0x68, + 0xbb, 0x9e, 0xe5, 0x73, 0x04, 0xb3, 0x0f, 0xa1, 0x8d, 0x72, 0x37, 0x63, 0xda, 0x51, 0x55, 0x5c, + 0xa3, 0xfc, 0x57, 0x03, 0x82, 0x73, 0x40, 0x22, 0xb5, 0xeb, 0xae, 0x81, 0x66, 0x1f, 0xba, 0x9e, + 0x13, 0x09, 0x5f, 0x05, 0x8b, 0xd9, 0xfc, 0x90, 0x79, 0x4e, 0xc3, 0xbe, 0x47, 0x09, 0xf0, 0x59, + 0x3c, 0xa6, 0x9c, 0x7f, 0x73, 0x7e, 0xea, 0xc8, 0x9b, 0x51, 0xf0, 0xa5, 0x12, 0x39, 0x6d, 0x2e, + 0xc5, 0x13, 0xa7, 0x56, 0xf9, 0x89, 0x93, 0x7c, 0x9f, 0x4d, 0x9b, 0x82, 0x96, 0x1f, 0xbc, 0x02, + 0xcb, 0x61, 0x37, 0xa1, 0xee, 0x07, 0x8e, 0x98, 0x0f, 0x66, 0x64, 0xfb, 0x10, 0x27, 0x3c, 0xfd, + 0x5c, 0x21, 0x8d, 0x0f, 0x4d, 0xb9, 0x9f, 0xa3, 0x29, 0x01, 0xf5, 0xc4, 0x32, 0x8d, 0x0f, 0xb7, + 0x70, 0x47, 0x47, 0x65, 0xbc, 0x01, 0xbd, 0x6c, 0x2e, 0x2a, 0x7d, 0x5f, 0xa6, 0x1d, 0x74, 0x33, + 0xa8, 0xcc, 0xde, 0x5f, 0x83, 0xb3, 0xf6, 0xa1, 0xe5, 0xfb, 0xc2, 0x33, 0x0f, 0xd2, 0xd1, 0x28, + 0xdb, 0x01, 0x3a, 0x74, 0xd9, 0xb8, 0xac, 0x50, 0x77, 0x09, 0x43, 0x1b, 0x8a, 0x01, 0x5d, 0xdf, + 0xf5, 0xe4, 0xbb, 0x34, 0xd3, 0xf6, 0x93, 0x41, 0x97, 0x28, 0xdb, 0xbe, 0xeb, 0x51, 0x1c, 0x77, + 0xd3, 0x4f, 0xd8, 0x27, 0xd0, 0x4f, 0x53, 0xd7, 0x89, 0xcd, 0x24, 0xc8, 0x9e, 0xb6, 0x0f, 0x7a, + 0xc4, 0xba, 0x92, 0xa3, 0xf8, 0x30, 0x75, 0x9d, 0xfd, 0x40, 0x3d, 0x6e, 0xef, 0x12, 0x7d, 0x56, + 0x35, 0x3e, 0x81, 0x4e, 0x59, 0x77, 0x50, 0x17, 0xe9, 0x04, 0xd5, 0x3f, 0xc3, 0x00, 0x9a, 0x3b, + 0x41, 0x34, 0xb1, 0xbc, 0x7e, 0x05, 0xcb, 0xf2, 0xe1, 0x5f, 0xbf, 0xca, 0x3a, 0xa0, 0x65, 0xae, + 0x7d, 0xbf, 0x66, 0x7c, 0x1b, 0xb4, 0xec, 0xad, 0x3e, 0x3d, 0x92, 0x0e, 0x1c, 0x21, 0x1d, 0x1b, + 0x69, 0x99, 0x34, 0x04, 0x90, 0x53, 0x93, 0xfd, 0x58, 0xa2, 0x5a, 0xfc, 0x58, 0xc2, 0xf8, 0x35, + 0xe8, 0x94, 0x07, 0x97, 0x85, 0xde, 0x2a, 0x45, 0xe8, 0x6d, 0x41, 0x2b, 0xba, 0x2b, 0x8b, 0x82, + 0x89, 0x59, 0x72, 0x02, 0x34, 0x04, 0xe0, 0x67, 0x8c, 0xdf, 0xad, 0x40, 0x83, 0xbc, 0x55, 0xda, + 0x5a, 0xb0, 0x50, 0xac, 0x9d, 0x06, 0xd7, 0x09, 0x42, 0x33, 0x2d, 0xdf, 0x39, 0x57, 0x9f, 0x7d, + 0xe7, 0x5c, 0x9b, 0xbe, 0x73, 0x7e, 0xc1, 0x74, 0x9c, 0x5b, 0x8f, 0xa0, 0x29, 0xff, 0xe5, 0xc1, + 0x96, 0xa1, 0xfb, 0xd0, 0x3f, 0xf2, 0x83, 0x27, 0xbe, 0x04, 0xf4, 0xcf, 0xb0, 0xb3, 0xb0, 0x94, + 0x31, 0x5d, 0xfd, 0x34, 0xa4, 0x5f, 0x61, 0x7d, 0xe8, 0x90, 0x58, 0x33, 0x48, 0x95, 0x5d, 0x81, + 0x81, 0xda, 0x1c, 0xb6, 0x02, 0x5f, 0xec, 0x04, 0x89, 0x3b, 0x3a, 0xc9, 0xb0, 0x35, 0xb6, 0x04, + 0xed, 0xbd, 0x24, 0x08, 0xf7, 0x84, 0xef, 0xb8, 0xfe, 0xb8, 0x5f, 0xbf, 0x75, 0x1f, 0x9a, 0xf2, + 0x17, 0x23, 0xa5, 0x4f, 0x4a, 0x40, 0xff, 0x0c, 0x52, 0x7f, 0x61, 0xb9, 0x89, 0xeb, 0x8f, 0x77, + 0xc4, 0x71, 0x22, 0x8d, 0xd2, 0x03, 0x2b, 0x4e, 0xfa, 0x55, 0xd6, 0x03, 0x50, 0xbd, 0xde, 0xf3, + 0x9d, 0x7e, 0xed, 0xee, 0xe6, 0x4f, 0x7f, 0x76, 0xb5, 0xf2, 0x77, 0x3f, 0xbb, 0x5a, 0xf9, 0xa7, + 0x9f, 0x5d, 0x3d, 0xf3, 0x87, 0xff, 0x7c, 0xb5, 0xf2, 0xc3, 0x0f, 0x4b, 0x3f, 0x50, 0x99, 0x58, + 0x49, 0xe4, 0x1e, 0xcb, 0xdb, 0xc6, 0xac, 0xe2, 0x8b, 0xdb, 0xe1, 0xd1, 0xf8, 0x76, 0x78, 0x70, + 0x3b, 0xd3, 0xb9, 0x83, 0x26, 0xfd, 0x17, 0xe5, 0xa3, 0xff, 0x0d, 0x00, 0x00, 0xff, 0xff, 0x04, + 0xbd, 0x3a, 0xbe, 0x96, 0x45, 0x00, 0x00, } func (m *Message) Marshal() (dAtA []byte, err error) { @@ -6855,10 +7059,10 @@ func (m *MultiUpdate) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } - if len(m.UpdateCtx) > 0 { - for iNdEx := len(m.UpdateCtx) - 1; iNdEx >= 0; iNdEx-- { + if len(m.UpdateCtxList) > 0 { + for iNdEx := len(m.UpdateCtxList) - 1; iNdEx >= 0; iNdEx-- { { - size, err := m.UpdateCtx[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + size, err := m.UpdateCtxList[iNdEx].MarshalToSizedBuffer(dAtA[:i]) if err != nil { return 0, err } @@ -6896,13 +7100,8 @@ func (m *MultiUpdate) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x18 } - if m.ToWriteS3 { - i-- - if m.ToWriteS3 { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } + if m.Action != 0 { + i = encodeVarintPipeline(dAtA, i, uint64(m.Action)) i-- dAtA[i] = 0x10 } @@ -9117,7 +9316,7 @@ func (m *MarkJoin) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } -func (m *Product) Marshal() (dAtA []byte, err error) { +func (m *DedupJoin) Marshal() (dAtA []byte, err error) { size := m.ProtoSize() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -9127,12 +9326,12 @@ func (m *Product) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *Product) MarshalTo(dAtA []byte) (int, error) { +func (m *DedupJoin) MarshalTo(dAtA []byte) (int, error) { size := m.ProtoSize() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *Product) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *DedupJoin) MarshalToSizedBuffer(dAtA []byte) (int, error) { i := len(dAtA) _ = i var l int @@ -9141,10 +9340,41 @@ func (m *Product) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if len(m.DedupColTypes) > 0 { + for iNdEx := len(m.DedupColTypes) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.DedupColTypes[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintPipeline(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x4a + } + } + if len(m.DedupColName) > 0 { + i -= len(m.DedupColName) + copy(dAtA[i:], m.DedupColName) + i = encodeVarintPipeline(dAtA, i, uint64(len(m.DedupColName))) + i-- + dAtA[i] = 0x42 + } + if m.OnDuplicateAction != 0 { + i = encodeVarintPipeline(dAtA, i, uint64(m.OnDuplicateAction)) + i-- + dAtA[i] = 0x38 + } + if m.ShuffleIdx != 0 { + i = encodeVarintPipeline(dAtA, i, uint64(m.ShuffleIdx)) + i-- + dAtA[i] = 0x30 + } if m.JoinMapTag != 0 { i = encodeVarintPipeline(dAtA, i, uint64(m.JoinMapTag)) i-- - dAtA[i] = 0x20 + dAtA[i] = 0x28 } if m.IsShuffle { i-- @@ -9154,32 +9384,116 @@ func (m *Product) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0 } i-- - dAtA[i] = 0x18 + dAtA[i] = 0x20 } - if len(m.ColList) > 0 { - dAtA71 := make([]byte, len(m.ColList)*10) - var j70 int - for _, num1 := range m.ColList { - num := uint64(num1) - for num >= 1<<7 { - dAtA71[j70] = uint8(uint64(num)&0x7f | 0x80) - num >>= 7 - j70++ + if len(m.RuntimeFilterBuildList) > 0 { + for iNdEx := len(m.RuntimeFilterBuildList) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.RuntimeFilterBuildList[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintPipeline(dAtA, i, uint64(size)) } - dAtA71[j70] = uint8(num) - j70++ + i-- + dAtA[i] = 0x1a } - i -= j70 - copy(dAtA[i:], dAtA71[:j70]) - i = encodeVarintPipeline(dAtA, i, uint64(j70)) - i-- - dAtA[i] = 0x12 } - if len(m.RelList) > 0 { - dAtA73 := make([]byte, len(m.RelList)*10) - var j72 int - for _, num1 := range m.RelList { - num := uint64(num1) + if len(m.RightCond) > 0 { + for iNdEx := len(m.RightCond) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.RightCond[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintPipeline(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + } + if len(m.LeftCond) > 0 { + for iNdEx := len(m.LeftCond) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.LeftCond[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintPipeline(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *Product) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Product) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Product) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + if m.JoinMapTag != 0 { + i = encodeVarintPipeline(dAtA, i, uint64(m.JoinMapTag)) + i-- + dAtA[i] = 0x20 + } + if m.IsShuffle { + i-- + if m.IsShuffle { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x18 + } + if len(m.ColList) > 0 { + dAtA71 := make([]byte, len(m.ColList)*10) + var j70 int + for _, num1 := range m.ColList { + num := uint64(num1) + for num >= 1<<7 { + dAtA71[j70] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j70++ + } + dAtA71[j70] = uint8(num) + j70++ + } + i -= j70 + copy(dAtA[i:], dAtA71[:j70]) + i = encodeVarintPipeline(dAtA, i, uint64(j70)) + i-- + dAtA[i] = 0x12 + } + if len(m.RelList) > 0 { + dAtA73 := make([]byte, len(m.RelList)*10) + var j72 int + for _, num1 := range m.RelList { + num := uint64(num1) for num >= 1<<7 { dAtA73[j72] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 @@ -9843,6 +10157,42 @@ func (m *HashBuild) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if len(m.DedupColTypes) > 0 { + for iNdEx := len(m.DedupColTypes) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.DedupColTypes[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintPipeline(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x62 + } + } + if len(m.DedupColName) > 0 { + i -= len(m.DedupColName) + copy(dAtA[i:], m.DedupColName) + i = encodeVarintPipeline(dAtA, i, uint64(len(m.DedupColName))) + i-- + dAtA[i] = 0x5a + } + if m.OnDuplicateAction != 0 { + i = encodeVarintPipeline(dAtA, i, uint64(m.OnDuplicateAction)) + i-- + dAtA[i] = 0x50 + } + if m.IsDedup { + i-- + if m.IsDedup { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x48 + } if m.RuntimeFilterSpec != nil { { size, err := m.RuntimeFilterSpec.MarshalToSizedBuffer(dAtA[:i]) @@ -9946,6 +10296,42 @@ func (m *Shufflebuild) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if len(m.DedupColTypes) > 0 { + for iNdEx := len(m.DedupColTypes) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.DedupColTypes[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintPipeline(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x5a + } + } + if len(m.DedupColName) > 0 { + i -= len(m.DedupColName) + copy(dAtA[i:], m.DedupColName) + i = encodeVarintPipeline(dAtA, i, uint64(len(m.DedupColName))) + i-- + dAtA[i] = 0x52 + } + if m.OnDuplicateAction != 0 { + i = encodeVarintPipeline(dAtA, i, uint64(m.OnDuplicateAction)) + i-- + dAtA[i] = 0x48 + } + if m.IsDedup { + i-- + if m.IsDedup { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x40 + } if m.ShuffleIdx != 0 { i = encodeVarintPipeline(dAtA, i, uint64(m.ShuffleIdx)) i-- @@ -10147,6 +10533,20 @@ func (m *Instruction) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x3 i-- + dAtA[i] = 0xaa + } + if m.MultiUpdate != nil { + { + size, err := m.MultiUpdate.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintPipeline(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x3 + i-- dAtA[i] = 0xa2 } if m.Apply != nil { @@ -10651,9 +11051,9 @@ func (m *Instruction) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x6a } - if m.Join != nil { + if m.DedupJoin != nil { { - size, err := m.Join.MarshalToSizedBuffer(dAtA[:i]) + size, err := m.DedupJoin.MarshalToSizedBuffer(dAtA[:i]) if err != nil { return 0, err } @@ -10711,6 +11111,18 @@ func (m *Instruction) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x42 } + if m.Join != nil { + { + size, err := m.Join.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintPipeline(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x3a + } if m.Agg != nil { { size, err := m.Agg.MarshalToSizedBuffer(dAtA[:i]) @@ -11389,40 +11801,40 @@ func (m *Pipeline) MarshalToSizedBuffer(dAtA []byte) (int, error) { } } if len(m.NilBatchCnt) > 0 { - dAtA139 := make([]byte, len(m.NilBatchCnt)*10) - var j138 int + dAtA141 := make([]byte, len(m.NilBatchCnt)*10) + var j140 int for _, num1 := range m.NilBatchCnt { num := uint64(num1) for num >= 1<<7 { - dAtA139[j138] = uint8(uint64(num)&0x7f | 0x80) + dAtA141[j140] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j138++ + j140++ } - dAtA139[j138] = uint8(num) - j138++ + dAtA141[j140] = uint8(num) + j140++ } - i -= j138 - copy(dAtA[i:], dAtA139[:j138]) - i = encodeVarintPipeline(dAtA, i, uint64(j138)) + i -= j140 + copy(dAtA[i:], dAtA141[:j140]) + i = encodeVarintPipeline(dAtA, i, uint64(j140)) i-- dAtA[i] = 0x6a } if len(m.ChannelBufferSize) > 0 { - dAtA141 := make([]byte, len(m.ChannelBufferSize)*10) - var j140 int + dAtA143 := make([]byte, len(m.ChannelBufferSize)*10) + var j142 int for _, num1 := range m.ChannelBufferSize { num := uint64(num1) for num >= 1<<7 { - dAtA141[j140] = uint8(uint64(num)&0x7f | 0x80) + dAtA143[j142] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j140++ + j142++ } - dAtA141[j140] = uint8(num) - j140++ + dAtA143[j142] = uint8(num) + j142++ } - i -= j140 - copy(dAtA[i:], dAtA141[:j140]) - i = encodeVarintPipeline(dAtA, i, uint64(j140)) + i -= j142 + copy(dAtA[i:], dAtA143[:j142]) + i = encodeVarintPipeline(dAtA, i, uint64(j142)) i-- dAtA[i] = 0x62 } @@ -11659,40 +12071,40 @@ func (m *Apply) MarshalToSizedBuffer(dAtA []byte) (int, error) { } } if len(m.ColList) > 0 { - dAtA146 := make([]byte, len(m.ColList)*10) - var j145 int + dAtA148 := make([]byte, len(m.ColList)*10) + var j147 int for _, num1 := range m.ColList { num := uint64(num1) for num >= 1<<7 { - dAtA146[j145] = uint8(uint64(num)&0x7f | 0x80) + dAtA148[j147] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j145++ + j147++ } - dAtA146[j145] = uint8(num) - j145++ + dAtA148[j147] = uint8(num) + j147++ } - i -= j145 - copy(dAtA[i:], dAtA146[:j145]) - i = encodeVarintPipeline(dAtA, i, uint64(j145)) + i -= j147 + copy(dAtA[i:], dAtA148[:j147]) + i = encodeVarintPipeline(dAtA, i, uint64(j147)) i-- dAtA[i] = 0x1a } if len(m.RelList) > 0 { - dAtA148 := make([]byte, len(m.RelList)*10) - var j147 int + dAtA150 := make([]byte, len(m.RelList)*10) + var j149 int for _, num1 := range m.RelList { num := uint64(num1) for num >= 1<<7 { - dAtA148[j147] = uint8(uint64(num)&0x7f | 0x80) + dAtA150[j149] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j147++ + j149++ } - dAtA148[j147] = uint8(num) - j147++ + dAtA150[j149] = uint8(num) + j149++ } - i -= j147 - copy(dAtA[i:], dAtA148[:j147]) - i = encodeVarintPipeline(dAtA, i, uint64(j147)) + i -= j149 + copy(dAtA[i:], dAtA150[:j149]) + i = encodeVarintPipeline(dAtA, i, uint64(j149)) i-- dAtA[i] = 0x12 } @@ -12083,8 +12495,8 @@ func (m *MultiUpdate) ProtoSize() (n int) { if m.AffectedRows != 0 { n += 1 + sovPipeline(uint64(m.AffectedRows)) } - if m.ToWriteS3 { - n += 2 + if m.Action != 0 { + n += 1 + sovPipeline(uint64(m.Action)) } if m.IBucket != 0 { n += 1 + sovPipeline(uint64(m.IBucket)) @@ -12100,8 +12512,8 @@ func (m *MultiUpdate) ProtoSize() (n int) { n += mapEntrySize + 1 + sovPipeline(uint64(mapEntrySize)) } } - if len(m.UpdateCtx) > 0 { - for _, e := range m.UpdateCtx { + if len(m.UpdateCtxList) > 0 { + for _, e := range m.UpdateCtxList { l = e.ProtoSize() n += 1 + l + sovPipeline(uint64(l)) } @@ -13025,6 +13437,58 @@ func (m *MarkJoin) ProtoSize() (n int) { return n } +func (m *DedupJoin) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.LeftCond) > 0 { + for _, e := range m.LeftCond { + l = e.ProtoSize() + n += 1 + l + sovPipeline(uint64(l)) + } + } + if len(m.RightCond) > 0 { + for _, e := range m.RightCond { + l = e.ProtoSize() + n += 1 + l + sovPipeline(uint64(l)) + } + } + if len(m.RuntimeFilterBuildList) > 0 { + for _, e := range m.RuntimeFilterBuildList { + l = e.ProtoSize() + n += 1 + l + sovPipeline(uint64(l)) + } + } + if m.IsShuffle { + n += 2 + } + if m.JoinMapTag != 0 { + n += 1 + sovPipeline(uint64(m.JoinMapTag)) + } + if m.ShuffleIdx != 0 { + n += 1 + sovPipeline(uint64(m.ShuffleIdx)) + } + if m.OnDuplicateAction != 0 { + n += 1 + sovPipeline(uint64(m.OnDuplicateAction)) + } + l = len(m.DedupColName) + if l > 0 { + n += 1 + l + sovPipeline(uint64(l)) + } + if len(m.DedupColTypes) > 0 { + for _, e := range m.DedupColTypes { + l = e.ProtoSize() + n += 1 + l + sovPipeline(uint64(l)) + } + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + func (m *Product) ProtoSize() (n int) { if m == nil { return 0 @@ -13367,13 +13831,29 @@ func (m *HashBuild) ProtoSize() (n int) { l = m.RuntimeFilterSpec.ProtoSize() n += 1 + l + sovPipeline(uint64(l)) } - if m.XXX_unrecognized != nil { - n += len(m.XXX_unrecognized) + if m.IsDedup { + n += 2 } - return n -} - -func (m *Shufflebuild) ProtoSize() (n int) { + if m.OnDuplicateAction != 0 { + n += 1 + sovPipeline(uint64(m.OnDuplicateAction)) + } + l = len(m.DedupColName) + if l > 0 { + n += 1 + l + sovPipeline(uint64(l)) + } + if len(m.DedupColTypes) > 0 { + for _, e := range m.DedupColTypes { + l = e.ProtoSize() + n += 1 + l + sovPipeline(uint64(l)) + } + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Shufflebuild) ProtoSize() (n int) { if m == nil { return 0 } @@ -13404,6 +13884,22 @@ func (m *Shufflebuild) ProtoSize() (n int) { if m.ShuffleIdx != 0 { n += 1 + sovPipeline(uint64(m.ShuffleIdx)) } + if m.IsDedup { + n += 2 + } + if m.OnDuplicateAction != 0 { + n += 1 + sovPipeline(uint64(m.OnDuplicateAction)) + } + l = len(m.DedupColName) + if l > 0 { + n += 1 + l + sovPipeline(uint64(l)) + } + if len(m.DedupColTypes) > 0 { + for _, e := range m.DedupColTypes { + l = e.ProtoSize() + n += 1 + l + sovPipeline(uint64(l)) + } + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -13481,6 +13977,10 @@ func (m *Instruction) ProtoSize() (n int) { l = m.Agg.ProtoSize() n += 1 + l + sovPipeline(uint64(l)) } + if m.Join != nil { + l = m.Join.ProtoSize() + n += 1 + l + sovPipeline(uint64(l)) + } if m.LeftJoin != nil { l = m.LeftJoin.ProtoSize() n += 1 + l + sovPipeline(uint64(l)) @@ -13497,8 +13997,8 @@ func (m *Instruction) ProtoSize() (n int) { l = m.MarkJoin.ProtoSize() n += 1 + l + sovPipeline(uint64(l)) } - if m.Join != nil { - l = m.Join.ProtoSize() + if m.DedupJoin != nil { + l = m.DedupJoin.ProtoSize() n += 1 + l + sovPipeline(uint64(l)) } if m.Product != nil { @@ -13652,6 +14152,10 @@ func (m *Instruction) ProtoSize() (n int) { l = m.Apply.ProtoSize() n += 2 + l + sovPipeline(uint64(l)) } + if m.MultiUpdate != nil { + l = m.MultiUpdate.ProtoSize() + n += 2 + l + sovPipeline(uint64(l)) + } if m.PostDml != nil { l = m.PostDml.ProtoSize() n += 2 + l + sovPipeline(uint64(l)) @@ -16492,9 +16996,9 @@ func (m *MultiUpdate) Unmarshal(dAtA []byte) error { } case 2: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field ToWriteS3", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Action", wireType) } - var v int + m.Action = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowPipeline @@ -16504,12 +17008,11 @@ func (m *MultiUpdate) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + m.Action |= uint32(b&0x7F) << shift if b < 0x80 { break } } - m.ToWriteS3 = bool(v != 0) case 3: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field IBucket", wireType) @@ -16663,7 +17166,7 @@ func (m *MultiUpdate) Unmarshal(dAtA []byte) error { iNdEx = postIndex case 6: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field UpdateCtx", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field UpdateCtxList", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -16690,8 +17193,8 @@ func (m *MultiUpdate) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.UpdateCtx = append(m.UpdateCtx, &plan.UpdateCtx{}) - if err := m.UpdateCtx[len(m.UpdateCtx)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + m.UpdateCtxList = append(m.UpdateCtxList, &plan.UpdateCtx{}) + if err := m.UpdateCtxList[len(m.UpdateCtxList)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -22807,7 +23310,7 @@ func (m *MarkJoin) Unmarshal(dAtA []byte) error { } return nil } -func (m *Product) Unmarshal(dAtA []byte) error { +func (m *DedupJoin) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -22830,169 +23333,51 @@ func (m *Product) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: Product: wiretype end group for non-group") + return fmt.Errorf("proto: DedupJoin: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: Product: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: DedupJoin: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType == 0 { - var v int32 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPipeline - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= int32(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.RelList = append(m.RelList, v) - } else if wireType == 2 { - var packedLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPipeline - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - packedLen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if packedLen < 0 { - return ErrInvalidLengthPipeline - } - postIndex := iNdEx + packedLen - if postIndex < 0 { - return ErrInvalidLengthPipeline - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - var elementCount int - var count int - for _, integer := range dAtA[iNdEx:postIndex] { - if integer < 128 { - count++ - } - } - elementCount = count - if elementCount != 0 && len(m.RelList) == 0 { - m.RelList = make([]int32, 0, elementCount) - } - for iNdEx < postIndex { - var v int32 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPipeline - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= int32(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.RelList = append(m.RelList, v) - } - } else { - return fmt.Errorf("proto: wrong wireType = %d for field RelList", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LeftCond", wireType) } - case 2: - if wireType == 0 { - var v int32 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPipeline - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= int32(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.ColList = append(m.ColList, v) - } else if wireType == 2 { - var packedLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPipeline - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - packedLen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if packedLen < 0 { - return ErrInvalidLengthPipeline - } - postIndex := iNdEx + packedLen - if postIndex < 0 { - return ErrInvalidLengthPipeline + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPipeline } - if postIndex > l { + if iNdEx >= l { return io.ErrUnexpectedEOF } - var elementCount int - var count int - for _, integer := range dAtA[iNdEx:postIndex] { - if integer < 128 { - count++ - } - } - elementCount = count - if elementCount != 0 && len(m.ColList) == 0 { - m.ColList = make([]int32, 0, elementCount) - } - for iNdEx < postIndex { - var v int32 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPipeline - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= int32(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.ColList = append(m.ColList, v) + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break } - } else { - return fmt.Errorf("proto: wrong wireType = %d for field ColList", wireType) } - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field IsShuffle", wireType) + if msglen < 0 { + return ErrInvalidLengthPipeline } - var v int + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthPipeline + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.LeftCond = append(m.LeftCond, &plan.Expr{}) + if err := m.LeftCond[len(m.LeftCond)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RightCond", wireType) + } + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowPipeline @@ -23002,17 +23387,31 @@ func (m *Product) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - m.IsShuffle = bool(v != 0) - case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field JoinMapTag", wireType) + if msglen < 0 { + return ErrInvalidLengthPipeline } - m.JoinMapTag = 0 + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthPipeline + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.RightCond = append(m.RightCond, &plan.Expr{}) + if err := m.RightCond[len(m.RightCond)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RuntimeFilterBuildList", wireType) + } + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowPipeline @@ -23022,18 +23421,418 @@ func (m *Product) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.JoinMapTag |= int32(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - default: - iNdEx = preIndex - skippy, err := skipPipeline(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { + if msglen < 0 { + return ErrInvalidLengthPipeline + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthPipeline + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.RuntimeFilterBuildList = append(m.RuntimeFilterBuildList, &plan.RuntimeFilterSpec{}) + if err := m.RuntimeFilterBuildList[len(m.RuntimeFilterBuildList)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field IsShuffle", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPipeline + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.IsShuffle = bool(v != 0) + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field JoinMapTag", wireType) + } + m.JoinMapTag = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPipeline + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.JoinMapTag |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ShuffleIdx", wireType) + } + m.ShuffleIdx = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPipeline + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ShuffleIdx |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 7: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field OnDuplicateAction", wireType) + } + m.OnDuplicateAction = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPipeline + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.OnDuplicateAction |= plan.Node_OnDuplicateAction(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DedupColName", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPipeline + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthPipeline + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthPipeline + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.DedupColName = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DedupColTypes", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPipeline + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthPipeline + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthPipeline + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.DedupColTypes = append(m.DedupColTypes, plan.Type{}) + if err := m.DedupColTypes[len(m.DedupColTypes)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipPipeline(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthPipeline + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Product) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPipeline + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Product: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Product: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType == 0 { + var v int32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPipeline + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.RelList = append(m.RelList, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPipeline + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthPipeline + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthPipeline + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.RelList) == 0 { + m.RelList = make([]int32, 0, elementCount) + } + for iNdEx < postIndex { + var v int32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPipeline + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.RelList = append(m.RelList, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field RelList", wireType) + } + case 2: + if wireType == 0 { + var v int32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPipeline + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.ColList = append(m.ColList, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPipeline + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthPipeline + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthPipeline + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.ColList) == 0 { + m.ColList = make([]int32, 0, elementCount) + } + for iNdEx < postIndex { + var v int32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPipeline + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.ColList = append(m.ColList, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field ColList", wireType) + } + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field IsShuffle", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPipeline + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.IsShuffle = bool(v != 0) + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field JoinMapTag", wireType) + } + m.JoinMapTag = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPipeline + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.JoinMapTag |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipPipeline(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthPipeline } if (iNdEx + skippy) > l { @@ -24854,12 +25653,86 @@ func (m *HashBuild) Unmarshal(dAtA []byte) error { break } } - m.HashOnPK = bool(v != 0) - case 3: + m.HashOnPK = bool(v != 0) + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field NeedBatches", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPipeline + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.NeedBatches = bool(v != 0) + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field NeedAllocateSels", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPipeline + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.NeedAllocateSels = bool(v != 0) + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Conditions", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPipeline + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthPipeline + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthPipeline + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Conditions = append(m.Conditions, &plan.Expr{}) + if err := m.Conditions[len(m.Conditions)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field NeedBatches", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field JoinMapTag", wireType) } - var v int + m.JoinMapTag = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowPipeline @@ -24869,17 +25742,16 @@ func (m *HashBuild) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + m.JoinMapTag |= int32(b&0x7F) << shift if b < 0x80 { break } } - m.NeedBatches = bool(v != 0) - case 4: + case 7: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field NeedAllocateSels", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field JoinMapRefCnt", wireType) } - var v int + m.JoinMapRefCnt = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowPipeline @@ -24889,15 +25761,14 @@ func (m *HashBuild) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + m.JoinMapRefCnt |= int32(b&0x7F) << shift if b < 0x80 { break } } - m.NeedAllocateSels = bool(v != 0) - case 5: + case 8: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Conditions", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field RuntimeFilterSpec", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -24924,16 +25795,18 @@ func (m *HashBuild) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Conditions = append(m.Conditions, &plan.Expr{}) - if err := m.Conditions[len(m.Conditions)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if m.RuntimeFilterSpec == nil { + m.RuntimeFilterSpec = &plan.RuntimeFilterSpec{} + } + if err := m.RuntimeFilterSpec.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 6: + case 9: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field JoinMapTag", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field IsDedup", wireType) } - m.JoinMapTag = 0 + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowPipeline @@ -24943,16 +25816,17 @@ func (m *HashBuild) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.JoinMapTag |= int32(b&0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } } - case 7: + m.IsDedup = bool(v != 0) + case 10: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field JoinMapRefCnt", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field OnDuplicateAction", wireType) } - m.JoinMapRefCnt = 0 + m.OnDuplicateAction = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowPipeline @@ -24962,14 +25836,46 @@ func (m *HashBuild) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.JoinMapRefCnt |= int32(b&0x7F) << shift + m.OnDuplicateAction |= plan.Node_OnDuplicateAction(b&0x7F) << shift if b < 0x80 { break } } - case 8: + case 11: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field RuntimeFilterSpec", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field DedupColName", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPipeline + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthPipeline + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthPipeline + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.DedupColName = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DedupColTypes", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -24996,10 +25902,8 @@ func (m *HashBuild) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.RuntimeFilterSpec == nil { - m.RuntimeFilterSpec = &plan.RuntimeFilterSpec{} - } - if err := m.RuntimeFilterSpec.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + m.DedupColTypes = append(m.DedupColTypes, plan.Type{}) + if err := m.DedupColTypes[len(m.DedupColTypes)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -25222,6 +26126,111 @@ func (m *Shufflebuild) Unmarshal(dAtA []byte) error { break } } + case 8: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field IsDedup", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPipeline + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.IsDedup = bool(v != 0) + case 9: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field OnDuplicateAction", wireType) + } + m.OnDuplicateAction = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPipeline + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.OnDuplicateAction |= plan.Node_OnDuplicateAction(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DedupColName", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPipeline + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthPipeline + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthPipeline + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.DedupColName = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DedupColTypes", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPipeline + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthPipeline + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthPipeline + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.DedupColTypes = append(m.DedupColTypes, plan.Type{}) + if err := m.DedupColTypes[len(m.DedupColTypes)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipPipeline(dAtA[iNdEx:]) @@ -25676,6 +26685,42 @@ func (m *Instruction) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Join", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPipeline + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthPipeline + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthPipeline + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Join == nil { + m.Join = &Join{} + } + if err := m.Join.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex case 8: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field LeftJoin", wireType) @@ -25822,7 +26867,7 @@ func (m *Instruction) Unmarshal(dAtA []byte) error { iNdEx = postIndex case 12: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Join", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field DedupJoin", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -25849,10 +26894,10 @@ func (m *Instruction) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Join == nil { - m.Join = &Join{} + if m.DedupJoin == nil { + m.DedupJoin = &DedupJoin{} } - if err := m.Join.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.DedupJoin.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -27134,6 +28179,42 @@ func (m *Instruction) Unmarshal(dAtA []byte) error { } iNdEx = postIndex case 52: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MultiUpdate", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPipeline + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthPipeline + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthPipeline + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.MultiUpdate == nil { + m.MultiUpdate = &MultiUpdate{} + } + if err := m.MultiUpdate.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 53: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field PostDml", wireType) } diff --git a/pkg/pb/plan/plan.pb.go b/pkg/pb/plan/plan.pb.go index 057ad21fd27d..dba99cacb0be 100644 --- a/pkg/pb/plan/plan.pb.go +++ b/pkg/pb/plan/plan.pb.go @@ -491,167 +491,160 @@ type Node_NodeType int32 const ( Node_UNKNOWN Node_NodeType = 0 + Node_ASSERT Node_NodeType = 100 // Scans - Node_VALUE_SCAN Node_NodeType = 1 - Node_TABLE_SCAN Node_NodeType = 2 - Node_FUNCTION_SCAN Node_NodeType = 3 - Node_EXTERNAL_SCAN Node_NodeType = 4 - Node_MATERIAL_SCAN Node_NodeType = 5 - Node_SOURCE_SCAN Node_NodeType = 6 - // Proj, for convenience - Node_PROJECT Node_NodeType = 10 - // External function call (UDF) - Node_EXTERNAL_FUNCTION Node_NodeType = 11 - // Material, CTE, etc. - Node_MATERIAL Node_NodeType = 20 - Node_RECURSIVE_CTE Node_NodeType = 21 - Node_SINK Node_NodeType = 22 - Node_SINK_SCAN Node_NodeType = 23 - Node_RECURSIVE_SCAN Node_NodeType = 24 + Node_VALUE_SCAN Node_NodeType = 1 + Node_TABLE_SCAN Node_NodeType = 2 + Node_FUNCTION_SCAN Node_NodeType = 3 + Node_EXTERNAL_SCAN Node_NodeType = 4 + Node_MATERIAL_SCAN Node_NodeType = 5 + Node_SOURCE_SCAN Node_NodeType = 6 + Node_EXTERNAL_FUNCTION Node_NodeType = 7 // Proper Relational Operators - Node_AGG Node_NodeType = 30 - Node_DISTINCT Node_NodeType = 31 - Node_FILTER Node_NodeType = 32 - Node_JOIN Node_NodeType = 33 - Node_SAMPLE Node_NodeType = 34 - Node_SORT Node_NodeType = 35 - Node_UNION Node_NodeType = 36 - Node_UNION_ALL Node_NodeType = 37 - Node_UNIQUE Node_NodeType = 38 - Node_WINDOW Node_NodeType = 39 - // Physical tuple mover - Node_BROADCAST Node_NodeType = 40 - Node_SPLIT Node_NodeType = 41 - Node_GATHER Node_NodeType = 42 - // Misc - Node_ASSERT Node_NodeType = 50 - // - Node_INSERT Node_NodeType = 51 - Node_DELETE Node_NodeType = 52 - Node_REPLACE Node_NodeType = 53 - Node_ON_DUPLICATE_KEY Node_NodeType = 54 - Node_PRE_INSERT Node_NodeType = 55 - Node_PRE_DELETE Node_NodeType = 56 + Node_PROJECT Node_NodeType = 10 + Node_AGG Node_NodeType = 11 + Node_DISTINCT Node_NodeType = 12 + Node_FILTER Node_NodeType = 13 + Node_JOIN Node_NodeType = 14 + Node_SAMPLE Node_NodeType = 15 + Node_SORT Node_NodeType = 16 + Node_WINDOW Node_NodeType = 17 + Node_UNION Node_NodeType = 18 + Node_UNION_ALL Node_NodeType = 19 + Node_UNIQUE Node_NodeType = 20 + Node_INTERSECT Node_NodeType = 21 + Node_INTERSECT_ALL Node_NodeType = 22 + Node_MINUS Node_NodeType = 23 + Node_MINUS_ALL Node_NodeType = 24 + // DML + Node_INSERT Node_NodeType = 30 + Node_DELETE Node_NodeType = 31 + Node_REPLACE Node_NodeType = 32 + Node_LOCK_OP Node_NodeType = 33 + Node_ON_DUPLICATE_KEY Node_NodeType = 34 + Node_PRE_INSERT Node_NodeType = 35 + Node_PRE_DELETE Node_NodeType = 36 + Node_FUZZY_FILTER Node_NodeType = 37 // the node which build insert batch for hidden table(unique key) - Node_PRE_INSERT_UK Node_NodeType = 57 - Node_PRE_INSERT_SK Node_NodeType = 58 - Node_MULTI_UPDATE Node_NodeType = 59 - Node_LOCK_OP Node_NodeType = 60 - // - Node_INTERSECT Node_NodeType = 61 - Node_INTERSECT_ALL Node_NodeType = 62 - Node_MINUS Node_NodeType = 63 - Node_MINUS_ALL Node_NodeType = 64 - // - // - Node_TIME_WINDOW Node_NodeType = 65 - Node_FILL Node_NodeType = 66 - Node_PARTITION Node_NodeType = 67 - Node_FUZZY_FILTER Node_NodeType = 68 - Node_APPLY Node_NodeType = 69 - Node_POSTDML Node_NodeType = 70 + Node_PRE_INSERT_UK Node_NodeType = 38 + Node_PRE_INSERT_SK Node_NodeType = 39 + // Material, CTE, etc. + Node_MATERIAL Node_NodeType = 40 + Node_RECURSIVE_CTE Node_NodeType = 41 + Node_SINK Node_NodeType = 42 + Node_SINK_SCAN Node_NodeType = 43 + Node_RECURSIVE_SCAN Node_NodeType = 44 + Node_BROADCAST Node_NodeType = 45 + Node_SPLIT Node_NodeType = 46 + Node_GATHER Node_NodeType = 47 + Node_TIME_WINDOW Node_NodeType = 50 + Node_FILL Node_NodeType = 51 + Node_PARTITION Node_NodeType = 52 + Node_APPLY Node_NodeType = 53 + Node_MULTI_UPDATE Node_NodeType = 54 + Node_POSTDML Node_NodeType = 55 ) var Node_NodeType_name = map[int32]string{ - 0: "UNKNOWN", - 1: "VALUE_SCAN", - 2: "TABLE_SCAN", - 3: "FUNCTION_SCAN", - 4: "EXTERNAL_SCAN", - 5: "MATERIAL_SCAN", - 6: "SOURCE_SCAN", - 10: "PROJECT", - 11: "EXTERNAL_FUNCTION", - 20: "MATERIAL", - 21: "RECURSIVE_CTE", - 22: "SINK", - 23: "SINK_SCAN", - 24: "RECURSIVE_SCAN", - 30: "AGG", - 31: "DISTINCT", - 32: "FILTER", - 33: "JOIN", - 34: "SAMPLE", - 35: "SORT", - 36: "UNION", - 37: "UNION_ALL", - 38: "UNIQUE", - 39: "WINDOW", - 40: "BROADCAST", - 41: "SPLIT", - 42: "GATHER", - 50: "ASSERT", - 51: "INSERT", - 52: "DELETE", - 53: "REPLACE", - 54: "ON_DUPLICATE_KEY", - 55: "PRE_INSERT", - 56: "PRE_DELETE", - 57: "PRE_INSERT_UK", - 58: "PRE_INSERT_SK", - 59: "MULTI_UPDATE", - 60: "LOCK_OP", - 61: "INTERSECT", - 62: "INTERSECT_ALL", - 63: "MINUS", - 64: "MINUS_ALL", - 65: "TIME_WINDOW", - 66: "FILL", - 67: "PARTITION", - 68: "FUZZY_FILTER", - 69: "APPLY", - 70: "POSTDML", + 0: "UNKNOWN", + 100: "ASSERT", + 1: "VALUE_SCAN", + 2: "TABLE_SCAN", + 3: "FUNCTION_SCAN", + 4: "EXTERNAL_SCAN", + 5: "MATERIAL_SCAN", + 6: "SOURCE_SCAN", + 7: "EXTERNAL_FUNCTION", + 10: "PROJECT", + 11: "AGG", + 12: "DISTINCT", + 13: "FILTER", + 14: "JOIN", + 15: "SAMPLE", + 16: "SORT", + 17: "WINDOW", + 18: "UNION", + 19: "UNION_ALL", + 20: "UNIQUE", + 21: "INTERSECT", + 22: "INTERSECT_ALL", + 23: "MINUS", + 24: "MINUS_ALL", + 30: "INSERT", + 31: "DELETE", + 32: "REPLACE", + 33: "LOCK_OP", + 34: "ON_DUPLICATE_KEY", + 35: "PRE_INSERT", + 36: "PRE_DELETE", + 37: "FUZZY_FILTER", + 38: "PRE_INSERT_UK", + 39: "PRE_INSERT_SK", + 40: "MATERIAL", + 41: "RECURSIVE_CTE", + 42: "SINK", + 43: "SINK_SCAN", + 44: "RECURSIVE_SCAN", + 45: "BROADCAST", + 46: "SPLIT", + 47: "GATHER", + 50: "TIME_WINDOW", + 51: "FILL", + 52: "PARTITION", + 53: "APPLY", + 54: "MULTI_UPDATE", + 55: "POSTDML", } var Node_NodeType_value = map[string]int32{ "UNKNOWN": 0, + "ASSERT": 100, "VALUE_SCAN": 1, "TABLE_SCAN": 2, "FUNCTION_SCAN": 3, "EXTERNAL_SCAN": 4, "MATERIAL_SCAN": 5, "SOURCE_SCAN": 6, + "EXTERNAL_FUNCTION": 7, "PROJECT": 10, - "EXTERNAL_FUNCTION": 11, - "MATERIAL": 20, - "RECURSIVE_CTE": 21, - "SINK": 22, - "SINK_SCAN": 23, - "RECURSIVE_SCAN": 24, - "AGG": 30, - "DISTINCT": 31, - "FILTER": 32, - "JOIN": 33, - "SAMPLE": 34, - "SORT": 35, - "UNION": 36, - "UNION_ALL": 37, - "UNIQUE": 38, - "WINDOW": 39, - "BROADCAST": 40, - "SPLIT": 41, - "GATHER": 42, - "ASSERT": 50, - "INSERT": 51, - "DELETE": 52, - "REPLACE": 53, - "ON_DUPLICATE_KEY": 54, - "PRE_INSERT": 55, - "PRE_DELETE": 56, - "PRE_INSERT_UK": 57, - "PRE_INSERT_SK": 58, - "MULTI_UPDATE": 59, - "LOCK_OP": 60, - "INTERSECT": 61, - "INTERSECT_ALL": 62, - "MINUS": 63, - "MINUS_ALL": 64, - "TIME_WINDOW": 65, - "FILL": 66, - "PARTITION": 67, - "FUZZY_FILTER": 68, - "APPLY": 69, - "POSTDML": 70, + "AGG": 11, + "DISTINCT": 12, + "FILTER": 13, + "JOIN": 14, + "SAMPLE": 15, + "SORT": 16, + "WINDOW": 17, + "UNION": 18, + "UNION_ALL": 19, + "UNIQUE": 20, + "INTERSECT": 21, + "INTERSECT_ALL": 22, + "MINUS": 23, + "MINUS_ALL": 24, + "INSERT": 30, + "DELETE": 31, + "REPLACE": 32, + "LOCK_OP": 33, + "ON_DUPLICATE_KEY": 34, + "PRE_INSERT": 35, + "PRE_DELETE": 36, + "FUZZY_FILTER": 37, + "PRE_INSERT_UK": 38, + "PRE_INSERT_SK": 39, + "MATERIAL": 40, + "RECURSIVE_CTE": 41, + "SINK": 42, + "SINK_SCAN": 43, + "RECURSIVE_SCAN": 44, + "BROADCAST": 45, + "SPLIT": 46, + "GATHER": 47, + "TIME_WINDOW": 50, + "FILL": 51, + "PARTITION": 52, + "APPLY": 53, + "MULTI_UPDATE": 54, + "POSTDML": 55, } func (x Node_NodeType) String() string { @@ -675,19 +668,21 @@ const ( Node_MARK Node_JoinType = 7 Node_INDEX Node_JoinType = 8 Node_L2 Node_JoinType = 9 + Node_DEDUP Node_JoinType = 10 ) var Node_JoinType_name = map[int32]string{ - 0: "INNER", - 1: "LEFT", - 2: "RIGHT", - 3: "OUTER", - 4: "SEMI", - 5: "ANTI", - 6: "SINGLE", - 7: "MARK", - 8: "INDEX", - 9: "L2", + 0: "INNER", + 1: "LEFT", + 2: "RIGHT", + 3: "OUTER", + 4: "SEMI", + 5: "ANTI", + 6: "SINGLE", + 7: "MARK", + 8: "INDEX", + 9: "L2", + 10: "DEDUP", } var Node_JoinType_value = map[string]int32{ @@ -701,6 +696,7 @@ var Node_JoinType_value = map[string]int32{ "MARK": 7, "INDEX": 8, "L2": 9, + "DEDUP": 10, } func (x Node_JoinType) String() string { @@ -776,6 +772,34 @@ func (Node_FillType) EnumDescriptor() ([]byte, []int) { return fileDescriptor_2d655ab2f7683c23, []int{58, 3} } +type Node_OnDuplicateAction int32 + +const ( + Node_ERROR Node_OnDuplicateAction = 0 + Node_IGNORE Node_OnDuplicateAction = 1 + Node_UPDATE Node_OnDuplicateAction = 2 +) + +var Node_OnDuplicateAction_name = map[int32]string{ + 0: "ERROR", + 1: "IGNORE", + 2: "UPDATE", +} + +var Node_OnDuplicateAction_value = map[string]int32{ + "ERROR": 0, + "IGNORE": 1, + "UPDATE": 2, +} + +func (x Node_OnDuplicateAction) String() string { + return proto.EnumName(Node_OnDuplicateAction_name, int32(x)) +} + +func (Node_OnDuplicateAction) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_2d655ab2f7683c23, []int{58, 4} +} + type Node_ApplyType int32 const ( @@ -798,7 +822,7 @@ func (x Node_ApplyType) String() string { } func (Node_ApplyType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_2d655ab2f7683c23, []int{58, 4} + return fileDescriptor_2d655ab2f7683c23, []int{58, 5} } type Query_StatementType int32 @@ -5680,11 +5704,12 @@ func (m *OnDuplicateKeyCtx) GetTableVersion() uint32 { type UpdateCtx struct { ObjRef *ObjectRef `protobuf:"bytes,1,opt,name=obj_ref,json=objRef,proto3" json:"obj_ref,omitempty"` TableDef *TableDef `protobuf:"bytes,2,opt,name=table_def,json=tableDef,proto3" json:"table_def,omitempty"` - InsertCols []*Expr `protobuf:"bytes,3,rep,name=insert_cols,json=insertCols,proto3" json:"insert_cols,omitempty"` - DeleteCols []*Expr `protobuf:"bytes,4,rep,name=delete_cols,json=deleteCols,proto3" json:"delete_cols,omitempty"` - PartitionTableIds []uint64 `protobuf:"varint,5,rep,packed,name=partition_table_ids,json=partitionTableIds,proto3" json:"partition_table_ids,omitempty"` - PartitionTableNames []string `protobuf:"bytes,6,rep,name=partition_table_names,json=partitionTableNames,proto3" json:"partition_table_names,omitempty"` - PartitionIdx int32 `protobuf:"varint,7,opt,name=partition_idx,json=partitionIdx,proto3" json:"partition_idx,omitempty"` + PartitionTableIds []uint64 `protobuf:"varint,3,rep,packed,name=partition_table_ids,json=partitionTableIds,proto3" json:"partition_table_ids,omitempty"` + PartitionTableNames []string `protobuf:"bytes,4,rep,name=partition_table_names,json=partitionTableNames,proto3" json:"partition_table_names,omitempty"` + OldPartitionIdx int32 `protobuf:"varint,5,opt,name=old_partition_idx,json=oldPartitionIdx,proto3" json:"old_partition_idx,omitempty"` + NewPartitionIdx int32 `protobuf:"varint,6,opt,name=new_partition_idx,json=newPartitionIdx,proto3" json:"new_partition_idx,omitempty"` + InsertCols []ColRef `protobuf:"bytes,7,rep,name=insert_cols,json=insertCols,proto3" json:"insert_cols"` + DeleteCols []ColRef `protobuf:"bytes,8,rep,name=delete_cols,json=deleteCols,proto3" json:"delete_cols"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -5737,39 +5762,46 @@ func (m *UpdateCtx) GetTableDef() *TableDef { return nil } -func (m *UpdateCtx) GetInsertCols() []*Expr { +func (m *UpdateCtx) GetPartitionTableIds() []uint64 { if m != nil { - return m.InsertCols + return m.PartitionTableIds } return nil } -func (m *UpdateCtx) GetDeleteCols() []*Expr { +func (m *UpdateCtx) GetPartitionTableNames() []string { if m != nil { - return m.DeleteCols + return m.PartitionTableNames } return nil } -func (m *UpdateCtx) GetPartitionTableIds() []uint64 { +func (m *UpdateCtx) GetOldPartitionIdx() int32 { if m != nil { - return m.PartitionTableIds + return m.OldPartitionIdx } - return nil + return 0 } -func (m *UpdateCtx) GetPartitionTableNames() []string { +func (m *UpdateCtx) GetNewPartitionIdx() int32 { if m != nil { - return m.PartitionTableNames + return m.NewPartitionIdx + } + return 0 +} + +func (m *UpdateCtx) GetInsertCols() []ColRef { + if m != nil { + return m.InsertCols } return nil } -func (m *UpdateCtx) GetPartitionIdx() int32 { +func (m *UpdateCtx) GetDeleteCols() []ColRef { if m != nil { - return m.PartitionIdx + return m.DeleteCols } - return 0 + return nil } type InsertCtx struct { @@ -6327,21 +6359,22 @@ type Node struct { // SORT OrderBy []*OrderBySpec `protobuf:"bytes,14,rep,name=order_by,json=orderBy,proto3" json:"order_by,omitempty"` // LIMIT - Limit *Expr `protobuf:"bytes,15,opt,name=limit,proto3" json:"limit,omitempty"` - Offset *Expr `protobuf:"bytes,16,opt,name=offset,proto3" json:"offset,omitempty"` - TableDef *TableDef `protobuf:"bytes,17,opt,name=table_def,json=tableDef,proto3" json:"table_def,omitempty"` - ObjRef *ObjectRef `protobuf:"bytes,18,opt,name=obj_ref,json=objRef,proto3" json:"obj_ref,omitempty"` - ParentObjRef *ObjectRef `protobuf:"bytes,19,opt,name=parent_obj_ref,json=parentObjRef,proto3" json:"parent_obj_ref,omitempty"` - RowsetData *RowsetData `protobuf:"bytes,20,opt,name=rowset_data,json=rowsetData,proto3" json:"rowset_data,omitempty"` - ExtraOptions string `protobuf:"bytes,22,opt,name=extra_options,json=extraOptions,proto3" json:"extra_options,omitempty"` - DeleteCtx *DeleteCtx `protobuf:"bytes,23,opt,name=delete_ctx,json=deleteCtx,proto3" json:"delete_ctx,omitempty"` - BindingTags []int32 `protobuf:"varint,24,rep,packed,name=binding_tags,json=bindingTags,proto3" json:"binding_tags,omitempty"` - AnalyzeInfo *AnalyzeInfo `protobuf:"bytes,25,opt,name=analyze_info,json=analyzeInfo,proto3" json:"analyze_info,omitempty"` - TblFuncExprList []*Expr `protobuf:"bytes,26,rep,name=tbl_func_expr_list,json=tblFuncExprList,proto3" json:"tbl_func_expr_list,omitempty"` - ClusterTable *ClusterTable `protobuf:"bytes,28,opt,name=cluster_table,json=clusterTable,proto3" json:"cluster_table,omitempty"` - NotCacheable bool `protobuf:"varint,29,opt,name=not_cacheable,json=notCacheable,proto3" json:"not_cacheable,omitempty"` - InsertCtx *InsertCtx `protobuf:"bytes,30,opt,name=insert_ctx,json=insertCtx,proto3" json:"insert_ctx,omitempty"` - ReplaceCtx *ReplaceCtx `protobuf:"bytes,31,opt,name=replace_ctx,json=replaceCtx,proto3" json:"replace_ctx,omitempty"` + Limit *Expr `protobuf:"bytes,15,opt,name=limit,proto3" json:"limit,omitempty"` + Offset *Expr `protobuf:"bytes,16,opt,name=offset,proto3" json:"offset,omitempty"` + TableDef *TableDef `protobuf:"bytes,17,opt,name=table_def,json=tableDef,proto3" json:"table_def,omitempty"` + ObjRef *ObjectRef `protobuf:"bytes,18,opt,name=obj_ref,json=objRef,proto3" json:"obj_ref,omitempty"` + ParentObjRef *ObjectRef `protobuf:"bytes,19,opt,name=parent_obj_ref,json=parentObjRef,proto3" json:"parent_obj_ref,omitempty"` + RowsetData *RowsetData `protobuf:"bytes,20,opt,name=rowset_data,json=rowsetData,proto3" json:"rowset_data,omitempty"` + InsertDeleteCols []*Expr `protobuf:"bytes,21,rep,name=insert_delete_cols,json=insertDeleteCols,proto3" json:"insert_delete_cols,omitempty"` + ExtraOptions string `protobuf:"bytes,22,opt,name=extra_options,json=extraOptions,proto3" json:"extra_options,omitempty"` + DeleteCtx *DeleteCtx `protobuf:"bytes,23,opt,name=delete_ctx,json=deleteCtx,proto3" json:"delete_ctx,omitempty"` + BindingTags []int32 `protobuf:"varint,24,rep,packed,name=binding_tags,json=bindingTags,proto3" json:"binding_tags,omitempty"` + AnalyzeInfo *AnalyzeInfo `protobuf:"bytes,25,opt,name=analyze_info,json=analyzeInfo,proto3" json:"analyze_info,omitempty"` + TblFuncExprList []*Expr `protobuf:"bytes,26,rep,name=tbl_func_expr_list,json=tblFuncExprList,proto3" json:"tbl_func_expr_list,omitempty"` + ClusterTable *ClusterTable `protobuf:"bytes,28,opt,name=cluster_table,json=clusterTable,proto3" json:"cluster_table,omitempty"` + NotCacheable bool `protobuf:"varint,29,opt,name=not_cacheable,json=notCacheable,proto3" json:"not_cacheable,omitempty"` + InsertCtx *InsertCtx `protobuf:"bytes,30,opt,name=insert_ctx,json=insertCtx,proto3" json:"insert_ctx,omitempty"` + ReplaceCtx *ReplaceCtx `protobuf:"bytes,31,opt,name=replace_ctx,json=replaceCtx,proto3" json:"replace_ctx,omitempty"` // used to connect two plans[steps] SourceStep []int32 `protobuf:"varint,32,rep,packed,name=source_step,json=sourceStep,proto3" json:"source_step,omitempty"` PreDeleteCtx *PreDeleteCtx `protobuf:"bytes,33,opt,name=pre_delete_ctx,json=preDeleteCtx,proto3" json:"pre_delete_ctx,omitempty"` @@ -6373,16 +6406,19 @@ type Node struct { Fuzzymessage *OriginTableMessageForFuzzy `protobuf:"bytes,56,opt,name=fuzzymessage,proto3" json:"fuzzymessage,omitempty"` IfInsertFromUnique bool `protobuf:"varint,57,opt,name=ifInsertFromUnique,proto3" json:"ifInsertFromUnique,omitempty"` // for message - SendMsgList []*MsgHeader `protobuf:"bytes,58,rep,name=send_msg_list,json=sendMsgList,proto3" json:"send_msg_list,omitempty"` - RecvMsgList []*MsgHeader `protobuf:"bytes,59,rep,name=recv_msg_list,json=recvMsgList,proto3" json:"recv_msg_list,omitempty"` - UpdateCtxList []*UpdateCtx `protobuf:"bytes,61,rep,name=update_ctx_list,json=updateCtxList,proto3" json:"update_ctx_list,omitempty"` - ScanSnapshot *Snapshot `protobuf:"bytes,62,opt,name=scan_snapshot,json=scanSnapshot,proto3" json:"scan_snapshot,omitempty"` - RecursiveCte bool `protobuf:"varint,63,opt,name=recursive_cte,json=recursiveCte,proto3" json:"recursive_cte,omitempty"` - ApplyType Node_ApplyType `protobuf:"varint,64,opt,name=apply_type,json=applyType,proto3,enum=plan.Node_ApplyType" json:"apply_type,omitempty"` - PostDmlCtx *PostDmlCtx `protobuf:"bytes,65,opt,name=post_dml_ctx,json=postDmlCtx,proto3" json:"post_dml_ctx,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + SendMsgList []*MsgHeader `protobuf:"bytes,58,rep,name=send_msg_list,json=sendMsgList,proto3" json:"send_msg_list,omitempty"` + RecvMsgList []*MsgHeader `protobuf:"bytes,59,rep,name=recv_msg_list,json=recvMsgList,proto3" json:"recv_msg_list,omitempty"` + UpdateCtxList []*UpdateCtx `protobuf:"bytes,61,rep,name=update_ctx_list,json=updateCtxList,proto3" json:"update_ctx_list,omitempty"` + ScanSnapshot *Snapshot `protobuf:"bytes,62,opt,name=scan_snapshot,json=scanSnapshot,proto3" json:"scan_snapshot,omitempty"` + RecursiveCte bool `protobuf:"varint,63,opt,name=recursive_cte,json=recursiveCte,proto3" json:"recursive_cte,omitempty"` + ApplyType Node_ApplyType `protobuf:"varint,64,opt,name=apply_type,json=applyType,proto3,enum=plan.Node_ApplyType" json:"apply_type,omitempty"` + PostDmlCtx *PostDmlCtx `protobuf:"bytes,65,opt,name=post_dml_ctx,json=postDmlCtx,proto3" json:"post_dml_ctx,omitempty"` + OnDuplicateAction Node_OnDuplicateAction `protobuf:"varint,66,opt,name=on_duplicate_action,json=onDuplicateAction,proto3,enum=plan.Node_OnDuplicateAction" json:"on_duplicate_action,omitempty"` + DedupColName string `protobuf:"bytes,67,opt,name=dedup_col_name,json=dedupColName,proto3" json:"dedup_col_name,omitempty"` + DedupColTypes []Type `protobuf:"bytes,68,rep,name=dedup_col_types,json=dedupColTypes,proto3" json:"dedup_col_types"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *Node) Reset() { *m = Node{} } @@ -6558,6 +6594,13 @@ func (m *Node) GetRowsetData() *RowsetData { return nil } +func (m *Node) GetInsertDeleteCols() []*Expr { + if m != nil { + return m.InsertDeleteCols + } + return nil +} + func (m *Node) GetExtraOptions() string { if m != nil { return m.ExtraOptions @@ -6852,6 +6895,27 @@ func (m *Node) GetPostDmlCtx() *PostDmlCtx { return nil } +func (m *Node) GetOnDuplicateAction() Node_OnDuplicateAction { + if m != nil { + return m.OnDuplicateAction + } + return Node_ERROR +} + +func (m *Node) GetDedupColName() string { + if m != nil { + return m.DedupColName + } + return "" +} + +func (m *Node) GetDedupColTypes() []Type { + if m != nil { + return m.DedupColTypes + } + return nil +} + // Snapshot Represents a snapshot of the database type Snapshot struct { // The timestamp of the snapshot @@ -7096,6 +7160,8 @@ type LockTarget struct { PartitionTableIds []uint64 `protobuf:"varint,8,rep,packed,name=partition_table_ids,json=partitionTableIds,proto3" json:"partition_table_ids,omitempty"` Block bool `protobuf:"varint,9,opt,name=block,proto3" json:"block,omitempty"` Mode lock.LockMode `protobuf:"varint,10,opt,name=Mode,proto3,enum=lock.LockMode" json:"Mode,omitempty"` + PrimaryColRelPos int32 `protobuf:"varint,11,opt,name=primary_col_rel_pos,json=primaryColRelPos,proto3" json:"primary_col_rel_pos,omitempty"` + FilterColRelPos int32 `protobuf:"varint,12,opt,name=filter_col_rel_pos,json=filterColRelPos,proto3" json:"filter_col_rel_pos,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -7204,6 +7270,20 @@ func (m *LockTarget) GetMode() lock.LockMode { return lock.LockMode_Exclusive } +func (m *LockTarget) GetPrimaryColRelPos() int32 { + if m != nil { + return m.PrimaryColRelPos + } + return 0 +} + +func (m *LockTarget) GetFilterColRelPos() int32 { + if m != nil { + return m.FilterColRelPos + } + return 0 +} + type PreInsertUkCtx struct { // index of columns(parts of unique key) in pre batch Columns []int32 `protobuf:"varint,1,rep,packed,name=columns,proto3" json:"columns,omitempty"` @@ -12112,6 +12192,7 @@ func init() { proto.RegisterEnum("plan.Node_JoinType", Node_JoinType_name, Node_JoinType_value) proto.RegisterEnum("plan.Node_AggMode", Node_AggMode_name, Node_AggMode_value) proto.RegisterEnum("plan.Node_FillType", Node_FillType_name, Node_FillType_value) + proto.RegisterEnum("plan.Node_OnDuplicateAction", Node_OnDuplicateAction_name, Node_OnDuplicateAction_value) proto.RegisterEnum("plan.Node_ApplyType", Node_ApplyType_name, Node_ApplyType_value) proto.RegisterEnum("plan.Query_StatementType", Query_StatementType_name, Query_StatementType_value) proto.RegisterEnum("plan.TransationControl_TclType", TransationControl_TclType_name, TransationControl_TclType_value) @@ -12257,722 +12338,732 @@ func init() { func init() { proto.RegisterFile("plan.proto", fileDescriptor_2d655ab2f7683c23) } var fileDescriptor_2d655ab2f7683c23 = []byte{ - // 11425 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0xbd, 0x5d, 0x8c, 0x1b, 0xc7, - 0x96, 0x18, 0x3c, 0xfc, 0x27, 0x0f, 0x39, 0x54, 0x4f, 0x69, 0x24, 0x53, 0xb2, 0x2c, 0x8d, 0xdb, - 0xb2, 0x2d, 0xcb, 0xbe, 0xb2, 0x2d, 0xf9, 0x47, 0xf6, 0xde, 0xbb, 0x36, 0x87, 0xc3, 0xd1, 0xd0, - 0xe2, 0x90, 0x73, 0x9b, 0x1c, 0xc9, 0xbe, 0x8b, 0xef, 0x6b, 0x34, 0xd9, 0xcd, 0x99, 0xf6, 0x34, - 0xbb, 0xe9, 0xee, 0xa6, 0x66, 0xc6, 0xc0, 0x02, 0x37, 0x59, 0x60, 0x83, 0xcd, 0x6b, 0x80, 0x7d, - 0xca, 0x06, 0x77, 0xf7, 0x29, 0x58, 0x64, 0x81, 0x00, 0x59, 0x60, 0x83, 0x20, 0x6f, 0x49, 0x80, - 0xcd, 0x22, 0x08, 0x12, 0xe4, 0x21, 0x48, 0x02, 0x6c, 0x82, 0x9b, 0x87, 0x3c, 0x25, 0x0b, 0x64, - 0xf3, 0x92, 0xb7, 0xe0, 0x9c, 0xaa, 0xee, 0xae, 0x26, 0x39, 0x96, 0x75, 0xef, 0x5d, 0x64, 0xf3, - 0x32, 0x53, 0x75, 0xce, 0xa9, 0xea, 0xfa, 0x3d, 0x7f, 0x75, 0xaa, 0x08, 0x30, 0x73, 0x0c, 0xf7, - 0xde, 0xcc, 0xf7, 0x42, 0x8f, 0xe5, 0x31, 0x7d, 0xfd, 0x07, 0x47, 0x76, 0x78, 0x3c, 0x1f, 0xdd, - 0x1b, 0x7b, 0xd3, 0x77, 0x8f, 0xbc, 0x23, 0xef, 0x5d, 0x42, 0x8e, 0xe6, 0x13, 0xca, 0x51, 0x86, - 0x52, 0xbc, 0xd0, 0x75, 0x70, 0xbc, 0xf1, 0x89, 0x48, 0x5f, 0x0a, 0xed, 0xa9, 0x15, 0x84, 0xc6, - 0x74, 0xc6, 0x01, 0xea, 0x9f, 0x64, 0x20, 0x3f, 0x3c, 0x9f, 0x59, 0xac, 0x0e, 0x59, 0xdb, 0x6c, - 0x64, 0xb6, 0x32, 0x77, 0x0a, 0x5a, 0xd6, 0x36, 0xd9, 0x16, 0x54, 0x5d, 0x2f, 0xec, 0xcd, 0x1d, - 0xc7, 0x18, 0x39, 0x56, 0x23, 0xbb, 0x95, 0xb9, 0x53, 0xd6, 0x64, 0x10, 0x7b, 0x19, 0x2a, 0xc6, - 0x3c, 0xf4, 0x74, 0xdb, 0x1d, 0xfb, 0x8d, 0x1c, 0xe1, 0xcb, 0x08, 0xe8, 0xb8, 0x63, 0x9f, 0x6d, - 0x42, 0xe1, 0xd4, 0x36, 0xc3, 0xe3, 0x46, 0x9e, 0x6a, 0xe4, 0x19, 0x84, 0x06, 0x63, 0xc3, 0xb1, - 0x1a, 0x05, 0x0e, 0xa5, 0x0c, 0x42, 0x43, 0xfa, 0x48, 0x71, 0x2b, 0x73, 0xa7, 0xa2, 0xf1, 0x0c, - 0xbb, 0x09, 0x60, 0xb9, 0xf3, 0xe9, 0x33, 0xc3, 0x99, 0x5b, 0x41, 0xa3, 0x44, 0x28, 0x09, 0xa2, - 0x7e, 0x06, 0x95, 0x69, 0x70, 0xb4, 0x67, 0x19, 0xa6, 0xe5, 0xb3, 0x97, 0xa0, 0x34, 0x0d, 0x8e, - 0xf4, 0xd0, 0x38, 0x12, 0x5d, 0x28, 0x4e, 0x83, 0xa3, 0xa1, 0x71, 0xc4, 0xae, 0x41, 0x99, 0x10, - 0xe7, 0x33, 0xde, 0x87, 0x82, 0x86, 0x84, 0xd8, 0x63, 0xf5, 0x2f, 0x0a, 0x50, 0xea, 0xda, 0xa1, - 0xe5, 0x1b, 0x0e, 0xbb, 0x0a, 0x45, 0x3b, 0x70, 0xe7, 0x8e, 0x43, 0xc5, 0xcb, 0x9a, 0xc8, 0xb1, - 0xab, 0x50, 0xb0, 0x1f, 0x3e, 0x33, 0x1c, 0x5e, 0x76, 0x6f, 0x4d, 0xe3, 0x59, 0xd6, 0x80, 0xa2, - 0xfd, 0xfe, 0x47, 0x88, 0xc8, 0x09, 0x84, 0xc8, 0x13, 0xe6, 0xc1, 0x7d, 0xc4, 0xe4, 0x63, 0x0c, - 0xe5, 0x09, 0xf3, 0xd1, 0x07, 0x88, 0xc1, 0xde, 0xe7, 0x08, 0x43, 0x79, 0xfc, 0xca, 0x9c, 0xbe, - 0x82, 0x03, 0xb0, 0x8e, 0x5f, 0x99, 0x47, 0x5f, 0x99, 0xf3, 0xaf, 0x94, 0x04, 0x42, 0xe4, 0x09, - 0xc3, 0xbf, 0x52, 0x8e, 0x31, 0xf1, 0x57, 0xe6, 0xfc, 0x2b, 0x95, 0xad, 0xcc, 0x9d, 0x3c, 0x61, - 0xf8, 0x57, 0x36, 0x21, 0x6f, 0x22, 0x1c, 0xb6, 0x32, 0x77, 0x32, 0x7b, 0x6b, 0x1a, 0xe5, 0x10, - 0x1a, 0x20, 0xb4, 0x8a, 0x03, 0x8c, 0xd0, 0x40, 0x40, 0x47, 0x08, 0xad, 0xe1, 0x68, 0x20, 0x74, - 0x24, 0xa0, 0x13, 0x84, 0xae, 0x6f, 0x65, 0xee, 0x64, 0x11, 0x8a, 0x39, 0x76, 0x1d, 0x4a, 0xa6, - 0x11, 0x5a, 0x88, 0xa8, 0x8b, 0x2e, 0x47, 0x00, 0xc4, 0xe1, 0x8a, 0x43, 0xdc, 0x25, 0xd1, 0xe9, - 0x08, 0xc0, 0x54, 0xa8, 0x22, 0x59, 0x84, 0x57, 0x04, 0x5e, 0x06, 0xb2, 0x0f, 0xa1, 0x66, 0x5a, - 0x63, 0x7b, 0x6a, 0x38, 0xbc, 0x4f, 0x1b, 0x5b, 0x99, 0x3b, 0xd5, 0xfb, 0x97, 0xee, 0xd1, 0x9e, - 0x88, 0x31, 0x7b, 0x6b, 0x5a, 0x8a, 0x8c, 0x3d, 0x84, 0x75, 0x91, 0x7f, 0xff, 0x3e, 0x0d, 0x2c, - 0xa3, 0x72, 0x4a, 0xaa, 0xdc, 0xfb, 0xf7, 0x1f, 0xee, 0xad, 0x69, 0x69, 0x42, 0x76, 0x1b, 0x6a, - 0xf1, 0x16, 0xc1, 0x82, 0x97, 0x45, 0xab, 0x52, 0x50, 0xec, 0xd6, 0xd7, 0x81, 0xe7, 0x22, 0xc1, - 0xa6, 0x18, 0xb7, 0x08, 0xc0, 0xb6, 0x00, 0x4c, 0x6b, 0x62, 0xcc, 0x9d, 0x10, 0xd1, 0x57, 0xc4, - 0x00, 0x4a, 0x30, 0x76, 0x13, 0x2a, 0xf3, 0x19, 0xf6, 0xf2, 0x89, 0xe1, 0x34, 0xae, 0x0a, 0x82, - 0x04, 0x84, 0xb5, 0xe3, 0x3a, 0x47, 0xec, 0x4b, 0x62, 0x76, 0x23, 0x00, 0xee, 0x15, 0x3b, 0xd8, - 0xb6, 0xdd, 0x46, 0x83, 0xd6, 0x29, 0xcf, 0xb0, 0x1b, 0x90, 0x0b, 0xfc, 0x71, 0xe3, 0x1a, 0xf5, - 0x12, 0x78, 0x2f, 0xdb, 0x67, 0x33, 0x5f, 0x43, 0xf0, 0x76, 0x09, 0x0a, 0xb4, 0x67, 0xd4, 0x1b, - 0x50, 0x3e, 0x30, 0x7c, 0x63, 0xaa, 0x59, 0x13, 0xa6, 0x40, 0x6e, 0xe6, 0x05, 0x62, 0xb7, 0x60, - 0x52, 0xed, 0x42, 0xf1, 0x89, 0xe1, 0x23, 0x8e, 0x41, 0xde, 0x35, 0xa6, 0x16, 0x21, 0x2b, 0x1a, - 0xa5, 0x71, 0x87, 0x04, 0xe7, 0x41, 0x68, 0x4d, 0x05, 0x2b, 0x10, 0x39, 0x84, 0x1f, 0x39, 0xde, - 0x48, 0xec, 0x84, 0xb2, 0x26, 0x72, 0xea, 0xdf, 0xcc, 0x40, 0xb1, 0xe5, 0x39, 0x58, 0xdd, 0x4b, - 0x50, 0xf2, 0x2d, 0x47, 0x4f, 0x3e, 0x57, 0xf4, 0x2d, 0xe7, 0xc0, 0x0b, 0x10, 0x31, 0xf6, 0x38, - 0x82, 0xef, 0xcd, 0xe2, 0xd8, 0x23, 0x44, 0xd4, 0x80, 0x9c, 0xd4, 0x80, 0x6b, 0x50, 0x0e, 0x47, - 0x8e, 0x4e, 0xf0, 0x3c, 0xc1, 0x4b, 0xe1, 0xc8, 0xe9, 0x21, 0xea, 0x25, 0x28, 0x99, 0x23, 0x8e, - 0x29, 0x10, 0xa6, 0x68, 0x8e, 0x10, 0xa1, 0x7e, 0x02, 0x15, 0xcd, 0x38, 0x15, 0xcd, 0xb8, 0x02, - 0x45, 0xac, 0x40, 0x70, 0xb9, 0xbc, 0x56, 0x08, 0x47, 0x4e, 0xc7, 0x44, 0x30, 0x36, 0xc2, 0x36, - 0xa9, 0x0d, 0x79, 0xad, 0x30, 0xf6, 0x9c, 0x8e, 0xa9, 0x0e, 0x01, 0x5a, 0x9e, 0xef, 0xff, 0xc2, - 0x5d, 0xd8, 0x84, 0x82, 0x69, 0xcd, 0xc2, 0x63, 0xce, 0x20, 0x34, 0x9e, 0x51, 0xef, 0x42, 0x19, - 0xe7, 0xa5, 0x6b, 0x07, 0x21, 0xbb, 0x09, 0x79, 0xc7, 0x0e, 0xc2, 0x46, 0x66, 0x2b, 0xb7, 0x30, - 0x6b, 0x04, 0x57, 0xb7, 0xa0, 0xbc, 0x6f, 0x9c, 0x3d, 0xc1, 0x99, 0xc3, 0xda, 0x68, 0x0a, 0xc5, - 0x94, 0x88, 0xf9, 0xac, 0x01, 0x0c, 0x0d, 0xff, 0xc8, 0x0a, 0x89, 0x9f, 0xfd, 0x65, 0x06, 0xaa, - 0x83, 0xf9, 0xe8, 0x9b, 0xb9, 0xe5, 0x9f, 0x63, 0x9b, 0xef, 0x40, 0x2e, 0x3c, 0x9f, 0x51, 0x89, - 0xfa, 0xfd, 0xab, 0xbc, 0x7a, 0x09, 0x7f, 0x0f, 0x0b, 0x69, 0x48, 0x82, 0x9d, 0x70, 0x3d, 0xd3, - 0x8a, 0xc6, 0xa0, 0xa0, 0x15, 0x31, 0xdb, 0x31, 0x51, 0x28, 0x78, 0x33, 0x31, 0x0b, 0x59, 0x6f, - 0xc6, 0xb6, 0xa0, 0x30, 0x3e, 0xb6, 0x1d, 0x93, 0x26, 0x20, 0xdd, 0x66, 0x8e, 0xc0, 0x59, 0xf2, - 0xbd, 0x53, 0x3d, 0xb0, 0xbf, 0x8d, 0x98, 0x7c, 0xc9, 0xf7, 0x4e, 0x07, 0xf6, 0xb7, 0x96, 0x3a, - 0x14, 0x92, 0x06, 0xa0, 0x38, 0x68, 0x35, 0xbb, 0x4d, 0x4d, 0x59, 0xc3, 0x74, 0xfb, 0xcb, 0xce, - 0x60, 0x38, 0x50, 0x32, 0xac, 0x0e, 0xd0, 0xeb, 0x0f, 0x75, 0x91, 0xcf, 0xb2, 0x22, 0x64, 0x3b, - 0x3d, 0x25, 0x87, 0x34, 0x08, 0xef, 0xf4, 0x94, 0x3c, 0x2b, 0x41, 0xae, 0xd9, 0xfb, 0x4a, 0x29, - 0x50, 0xa2, 0xdb, 0x55, 0x8a, 0xea, 0x1f, 0x66, 0xa1, 0xd2, 0x1f, 0x7d, 0x6d, 0x8d, 0x43, 0xec, - 0x33, 0xae, 0x52, 0xcb, 0x7f, 0x66, 0xf9, 0xd4, 0xed, 0x9c, 0x26, 0x72, 0xd8, 0x11, 0x73, 0x44, - 0x9d, 0xcb, 0x69, 0x59, 0x73, 0x44, 0x74, 0xe3, 0x63, 0x6b, 0x6a, 0x50, 0xe7, 0x90, 0x8e, 0x72, - 0xb8, 0x2b, 0xbc, 0xd1, 0xd7, 0xd4, 0xbd, 0x9c, 0x86, 0x49, 0x76, 0x0b, 0xaa, 0xbc, 0x0e, 0x79, - 0x7d, 0x01, 0x07, 0x2d, 0x2e, 0xbe, 0xa2, 0xbc, 0xf8, 0xa8, 0x24, 0xd5, 0xca, 0x91, 0x42, 0x82, - 0x71, 0x50, 0x4f, 0xac, 0x68, 0x6f, 0xf4, 0x35, 0xc7, 0x96, 0xf9, 0x8a, 0xf6, 0x46, 0x5f, 0x13, - 0xea, 0x6d, 0xd8, 0x08, 0xe6, 0xa3, 0x60, 0xec, 0xdb, 0xb3, 0xd0, 0xf6, 0x5c, 0x4e, 0x53, 0x21, - 0x1a, 0x45, 0x46, 0x10, 0xf1, 0x1d, 0x28, 0xcf, 0xe6, 0x23, 0xdd, 0x76, 0x27, 0x1e, 0x31, 0xf7, - 0xea, 0xfd, 0x75, 0x3e, 0x31, 0x07, 0xf3, 0x51, 0xc7, 0x9d, 0x78, 0x5a, 0x69, 0xc6, 0x13, 0xea, - 0x1b, 0x50, 0x12, 0x30, 0x94, 0xde, 0xa1, 0xe5, 0x1a, 0x6e, 0xa8, 0xc7, 0x62, 0xbf, 0xcc, 0x01, - 0x1d, 0x53, 0xfd, 0xe3, 0x0c, 0x28, 0x03, 0xe9, 0x33, 0xfb, 0x56, 0x68, 0xac, 0xe4, 0x0a, 0xaf, - 0x00, 0x18, 0xe3, 0xb1, 0x37, 0xe7, 0xd5, 0xf0, 0xc5, 0x53, 0x11, 0x90, 0x8e, 0x29, 0x8f, 0x4d, - 0x2e, 0x35, 0x36, 0xaf, 0x42, 0x2d, 0x2a, 0x27, 0x6d, 0xe8, 0xaa, 0x80, 0x45, 0xa3, 0x13, 0xcc, - 0x53, 0xbb, 0xba, 0x14, 0xcc, 0x79, 0xe9, 0xab, 0x50, 0x24, 0x1d, 0x21, 0x88, 0x46, 0x9c, 0xe7, - 0xd4, 0xbf, 0x9d, 0x85, 0xf2, 0xee, 0xdc, 0x1d, 0x63, 0x93, 0xd9, 0x6b, 0x90, 0x9f, 0xcc, 0xdd, - 0x31, 0x35, 0x37, 0x16, 0x19, 0xf1, 0x4a, 0xd1, 0x08, 0x89, 0x7b, 0xd0, 0xf0, 0x8f, 0x70, 0xef, - 0x2e, 0xed, 0x41, 0x84, 0xab, 0xff, 0x38, 0xc3, 0x6b, 0xdc, 0x75, 0x8c, 0x23, 0x56, 0x86, 0x7c, - 0xaf, 0xdf, 0x6b, 0x2b, 0x6b, 0xac, 0x06, 0xe5, 0x4e, 0x6f, 0xd8, 0xd6, 0x7a, 0xcd, 0xae, 0x92, - 0xa1, 0x05, 0x3d, 0x6c, 0x6e, 0x77, 0xdb, 0x4a, 0x16, 0x31, 0x4f, 0xfa, 0xdd, 0xe6, 0xb0, 0xd3, - 0x6d, 0x2b, 0x79, 0x8e, 0xd1, 0x3a, 0xad, 0xa1, 0x52, 0x66, 0x0a, 0xd4, 0x0e, 0xb4, 0xfe, 0xce, - 0x61, 0xab, 0xad, 0xf7, 0x0e, 0xbb, 0x5d, 0x45, 0x61, 0x97, 0xe1, 0x52, 0x0c, 0xe9, 0x73, 0xe0, - 0x16, 0x16, 0x79, 0xd2, 0xd4, 0x9a, 0xda, 0x23, 0xe5, 0x73, 0x56, 0x86, 0x5c, 0xf3, 0xd1, 0x23, - 0xe5, 0xa7, 0xb8, 0x37, 0x2a, 0x4f, 0x3b, 0x3d, 0xfd, 0x49, 0xb3, 0x7b, 0xd8, 0x56, 0x7e, 0x9a, - 0x8d, 0xf2, 0x7d, 0x6d, 0xa7, 0xad, 0x29, 0x3f, 0xcd, 0xb3, 0x0d, 0xa8, 0xfd, 0xa4, 0xdf, 0x6b, - 0xef, 0x37, 0x0f, 0x0e, 0xa8, 0x21, 0x3f, 0x2d, 0xab, 0xff, 0x3d, 0x0f, 0x79, 0xec, 0x09, 0x53, - 0x13, 0x3e, 0x10, 0x77, 0x11, 0x37, 0xe2, 0x76, 0xfe, 0x4f, 0xff, 0xfc, 0xd6, 0x1a, 0xe7, 0x00, - 0xaf, 0x42, 0xce, 0xb1, 0x43, 0x9a, 0xc0, 0x78, 0xf5, 0x08, 0xdd, 0x68, 0x6f, 0x4d, 0x43, 0x1c, - 0xbb, 0x09, 0x19, 0xce, 0x0a, 0xaa, 0xf7, 0xeb, 0x62, 0x79, 0x09, 0x59, 0xb2, 0xb7, 0xa6, 0x65, - 0x66, 0xec, 0x06, 0x64, 0x9e, 0x09, 0xbe, 0x50, 0xe3, 0x78, 0x2e, 0x4d, 0x10, 0xfb, 0x8c, 0x6d, - 0x41, 0x6e, 0xec, 0x71, 0xcd, 0x27, 0xc6, 0x73, 0xde, 0x8a, 0xf5, 0x8f, 0x3d, 0x87, 0xbd, 0x06, - 0x39, 0xdf, 0x38, 0xa5, 0x19, 0x8d, 0xa7, 0x2b, 0x66, 0xde, 0x48, 0xe4, 0x1b, 0xa7, 0xd8, 0x88, - 0x09, 0xed, 0xa4, 0xb8, 0x11, 0xd1, 0x7c, 0xe3, 0x67, 0x26, 0x6c, 0x0b, 0x32, 0xa7, 0xb4, 0x97, - 0x62, 0x61, 0xff, 0xd4, 0x76, 0x4d, 0xef, 0x74, 0x30, 0xb3, 0xc6, 0x48, 0x71, 0xca, 0x5e, 0x87, - 0x5c, 0x30, 0x1f, 0xd1, 0x5e, 0xaa, 0xde, 0xdf, 0x58, 0xe2, 0x8a, 0xf8, 0xa1, 0x60, 0x3e, 0x62, - 0x6f, 0x40, 0x7e, 0xec, 0xf9, 0xbe, 0xd8, 0x4f, 0x4a, 0xd4, 0xe0, 0x48, 0x20, 0xa0, 0xf2, 0x83, - 0x78, 0xfc, 0x60, 0x48, 0xba, 0x53, 0x4c, 0x94, 0x70, 0x64, 0xfc, 0x60, 0xc8, 0x6e, 0x0b, 0x36, - 0x5f, 0x93, 0x5b, 0x1d, 0x09, 0x01, 0xac, 0x07, 0xb1, 0x38, 0x49, 0x53, 0xe3, 0x8c, 0x34, 0xab, - 0x98, 0x28, 0xe2, 0xfe, 0xd8, 0xa6, 0xa9, 0x71, 0xc6, 0x6e, 0x43, 0xee, 0x99, 0x35, 0x26, 0x25, - 0x2b, 0xfe, 0x9a, 0x98, 0xa4, 0x27, 0xd4, 0x3d, 0x44, 0xd3, 0xba, 0xf7, 0x1c, 0x93, 0xf4, 0xad, - 0x78, 0x2e, 0x77, 0x3d, 0xc7, 0x7c, 0x42, 0x73, 0x49, 0x48, 0x14, 0x7a, 0xc6, 0xfc, 0x0c, 0xf7, - 0xac, 0xc2, 0xc5, 0x93, 0x31, 0x3f, 0xeb, 0x98, 0xc8, 0xfe, 0x5c, 0xf3, 0x19, 0x69, 0x59, 0x19, - 0x0d, 0x93, 0x68, 0x06, 0x04, 0x96, 0x63, 0x8d, 0x43, 0xfb, 0x99, 0x1d, 0x9e, 0x93, 0x1e, 0x95, - 0xd1, 0x64, 0xd0, 0x76, 0x11, 0xf2, 0xd6, 0xd9, 0xcc, 0x57, 0xf7, 0xa0, 0x24, 0xbe, 0xb2, 0x64, - 0x4b, 0x5c, 0x83, 0xb2, 0x1d, 0xe8, 0x63, 0xcf, 0x0d, 0x42, 0xa1, 0x3d, 0x94, 0xec, 0xa0, 0x85, - 0x59, 0x64, 0x2a, 0xa6, 0x11, 0x72, 0x36, 0x5c, 0xd3, 0x28, 0xad, 0xde, 0x07, 0x48, 0xba, 0x85, - 0x6d, 0x72, 0x2c, 0x37, 0x52, 0x54, 0x1c, 0xcb, 0x8d, 0xcb, 0x64, 0xa5, 0x32, 0xd7, 0xa0, 0x12, - 0x6b, 0x80, 0xac, 0x06, 0x19, 0x43, 0x08, 0x80, 0x8c, 0xa1, 0xde, 0x41, 0x85, 0x2c, 0xd2, 0xf1, - 0xd2, 0x38, 0xcc, 0x45, 0x62, 0x21, 0x33, 0x52, 0x7f, 0x08, 0x35, 0xcd, 0x0a, 0xe6, 0x4e, 0xd8, - 0xf2, 0x9c, 0x1d, 0x6b, 0xc2, 0xde, 0x01, 0x88, 0xf3, 0x81, 0x90, 0xd3, 0xc9, 0xda, 0xdd, 0xb1, - 0x26, 0x9a, 0x84, 0x57, 0xff, 0x7e, 0x9e, 0x34, 0x9e, 0x1d, 0xae, 0x6a, 0x08, 0x9d, 0x22, 0x23, - 0xe9, 0x14, 0x31, 0x07, 0xcd, 0xa6, 0xf5, 0xaa, 0x63, 0xdb, 0x34, 0x2d, 0x37, 0xd2, 0x9f, 0x78, - 0x0e, 0x27, 0xdb, 0x70, 0x8e, 0x68, 0x43, 0xd5, 0xef, 0xb3, 0xe8, 0xa3, 0xd3, 0x99, 0x6f, 0x05, - 0x01, 0x97, 0xdc, 0x86, 0x73, 0x14, 0xed, 0xed, 0xc2, 0x77, 0xed, 0xed, 0x6b, 0x50, 0x76, 0xbd, - 0x50, 0x27, 0xeb, 0xa6, 0xc8, 0x47, 0x5f, 0x98, 0x71, 0xec, 0x4d, 0x28, 0x09, 0xbd, 0x54, 0x6c, - 0x2a, 0xb1, 0x5c, 0x76, 0x38, 0x50, 0x8b, 0xb0, 0xac, 0x81, 0x6a, 0xce, 0x74, 0x6a, 0xb9, 0x61, - 0x24, 0xa9, 0x44, 0x96, 0xbd, 0x0d, 0x15, 0xcf, 0xd5, 0xb9, 0xf2, 0x2a, 0x76, 0x95, 0x58, 0xbe, - 0x7d, 0xf7, 0x90, 0xa0, 0x5a, 0xd9, 0x13, 0x29, 0x6c, 0x8a, 0xe3, 0x9d, 0xea, 0x63, 0xc3, 0x37, - 0x69, 0x67, 0x95, 0xb5, 0x92, 0xe3, 0x9d, 0xb6, 0x0c, 0xdf, 0xe4, 0x92, 0xfb, 0x1b, 0x77, 0x3e, - 0xa5, 0xdd, 0xb4, 0xae, 0x89, 0x1c, 0xbb, 0x01, 0x95, 0xb1, 0x33, 0x0f, 0x42, 0xcb, 0xdf, 0x3e, - 0xe7, 0xe6, 0x88, 0x96, 0x00, 0xb0, 0x5d, 0x33, 0xdf, 0x9e, 0x1a, 0xfe, 0x39, 0x6d, 0x9d, 0xb2, - 0x16, 0x65, 0x51, 0x63, 0x9a, 0x9d, 0xd8, 0xe6, 0x19, 0xb7, 0x49, 0x34, 0x9e, 0x41, 0xfa, 0x63, - 0xb2, 0x18, 0x03, 0xda, 0x1f, 0x65, 0x2d, 0xca, 0xd2, 0x3c, 0x50, 0x92, 0x76, 0x44, 0x45, 0x13, - 0xb9, 0x94, 0xda, 0xb9, 0x71, 0xa1, 0xda, 0xc9, 0x16, 0x25, 0xbf, 0xe7, 0xdb, 0x47, 0xb6, 0x90, - 0xdb, 0x97, 0xb9, 0xe4, 0xe7, 0x20, 0xd2, 0x4b, 0xbf, 0x81, 0x92, 0x18, 0x62, 0x94, 0x40, 0xb8, - 0x7d, 0xd2, 0xec, 0x99, 0x4b, 0x20, 0x84, 0xb3, 0xd7, 0x60, 0x5d, 0xd4, 0x15, 0x84, 0xbe, 0xed, - 0x1e, 0x89, 0xc5, 0x53, 0xe3, 0xc0, 0x01, 0xc1, 0x50, 0x9c, 0xe2, 0xf4, 0xea, 0xc6, 0xc8, 0x76, - 0x70, 0x9b, 0xe6, 0x84, 0xb5, 0x3e, 0x77, 0x9c, 0x26, 0x07, 0xa9, 0x7d, 0x28, 0x47, 0x13, 0xf2, - 0x2b, 0xf9, 0xa6, 0xfa, 0xdb, 0x19, 0xa8, 0x76, 0x5c, 0xd3, 0x3a, 0xeb, 0x93, 0x8a, 0xc0, 0xde, - 0x01, 0x36, 0xf6, 0x2d, 0x23, 0xb4, 0x74, 0xeb, 0x2c, 0xf4, 0x0d, 0x9d, 0x9b, 0xf4, 0xdc, 0x9c, - 0x56, 0x38, 0xa6, 0x8d, 0x88, 0x21, 0x59, 0xf7, 0xb7, 0xa0, 0x3a, 0x33, 0xfc, 0x20, 0x52, 0xab, - 0xf8, 0x07, 0x80, 0x83, 0x84, 0x52, 0xa3, 0xb8, 0x47, 0xbe, 0x31, 0xd5, 0x43, 0xef, 0xc4, 0x72, - 0xb9, 0x42, 0xc9, 0x55, 0xe9, 0x3a, 0xc1, 0x87, 0x08, 0x26, 0xbd, 0xf2, 0x3f, 0x66, 0x60, 0xfd, - 0x80, 0xcf, 0xfa, 0x63, 0xeb, 0x7c, 0x87, 0xdb, 0x2f, 0xe3, 0x68, 0xc7, 0xe6, 0x35, 0x4a, 0xb3, - 0x9b, 0x50, 0x9d, 0x9d, 0x58, 0xe7, 0x7a, 0x4a, 0xd7, 0xaf, 0x20, 0xa8, 0x45, 0x7b, 0xf3, 0x2d, - 0x28, 0x7a, 0xd4, 0x11, 0x21, 0xe3, 0x84, 0x68, 0x90, 0x7a, 0xa8, 0x09, 0x02, 0xa6, 0xc2, 0x7a, - 0x5c, 0x95, 0xac, 0xbd, 0x88, 0xca, 0xa8, 0xf9, 0x9b, 0x50, 0x40, 0x54, 0xd0, 0x28, 0x6c, 0xe5, - 0x50, 0x61, 0xa7, 0x0c, 0x7b, 0x0f, 0xd6, 0xc7, 0xde, 0x74, 0xa6, 0x47, 0xc5, 0x85, 0xb4, 0x4b, - 0xf3, 0x94, 0x2a, 0x92, 0x1c, 0xf0, 0xba, 0xd4, 0xdf, 0xcd, 0x41, 0x99, 0xda, 0x20, 0xd8, 0x8a, - 0x6d, 0x9e, 0x45, 0x6c, 0xa5, 0xa2, 0x15, 0x6c, 0x13, 0xb9, 0xf6, 0x2b, 0x00, 0x36, 0x92, 0xc8, - 0x43, 0x59, 0x21, 0x48, 0xd4, 0x94, 0x99, 0xe1, 0x87, 0x41, 0x23, 0xc7, 0x9b, 0x42, 0x19, 0x5c, - 0xef, 0x73, 0xd7, 0xfe, 0x66, 0xce, 0x5b, 0x5f, 0xd6, 0x44, 0x0e, 0xc7, 0x9d, 0x57, 0x46, 0xf3, - 0x27, 0xab, 0x5f, 0x75, 0x82, 0xd3, 0xf4, 0x45, 0xab, 0x9c, 0xd3, 0x58, 0x67, 0x28, 0xdf, 0x38, - 0x6b, 0x01, 0x02, 0xb5, 0x11, 0x22, 0x33, 0x8d, 0x52, 0x9a, 0x69, 0x34, 0xa0, 0xf4, 0xcc, 0x0e, - 0x6c, 0x5c, 0x20, 0x65, 0xbe, 0x0d, 0x45, 0x56, 0x9a, 0x86, 0xca, 0xf3, 0xa6, 0x21, 0xee, 0xb6, - 0xe1, 0x1c, 0x71, 0xc5, 0x37, 0xea, 0x76, 0xd3, 0x39, 0xf2, 0xd8, 0xfb, 0x70, 0x25, 0x41, 0x8b, - 0xde, 0x90, 0x1b, 0x88, 0x3c, 0x1d, 0x1a, 0x8b, 0x29, 0xa9, 0x47, 0x64, 0x99, 0xdc, 0x85, 0x0d, - 0xa9, 0xc8, 0x0c, 0xd5, 0x9b, 0x80, 0x78, 0x4e, 0x45, 0xbb, 0x14, 0x93, 0x93, 0xd6, 0x13, 0xa8, - 0xff, 0x32, 0x0b, 0xeb, 0xbb, 0x9e, 0x6f, 0xd9, 0x47, 0x6e, 0xb2, 0xea, 0x96, 0xf4, 0xe3, 0x68, - 0x25, 0x66, 0xa5, 0x95, 0x78, 0x0b, 0xaa, 0x13, 0x5e, 0x50, 0x0f, 0x47, 0xdc, 0x6c, 0xce, 0x6b, - 0x20, 0x40, 0xc3, 0x91, 0x83, 0xbb, 0x39, 0x22, 0xa0, 0xc2, 0x79, 0x2a, 0x1c, 0x15, 0x42, 0x59, - 0xc3, 0x3e, 0x25, 0xae, 0x6b, 0x5a, 0x8e, 0x15, 0xf2, 0xe9, 0xa9, 0xdf, 0x7f, 0x25, 0x92, 0xf4, - 0x52, 0x9b, 0xee, 0x69, 0xd6, 0xa4, 0x49, 0xea, 0x11, 0x32, 0xe1, 0x1d, 0x22, 0x17, 0x65, 0x05, - 0xc7, 0x2e, 0x7e, 0xcf, 0xb2, 0x9c, 0x73, 0xa8, 0x43, 0xa8, 0xc4, 0x60, 0xd4, 0x75, 0xb5, 0xb6, - 0xd0, 0x6f, 0xd7, 0x58, 0x15, 0x4a, 0xad, 0xe6, 0xa0, 0xd5, 0xdc, 0x69, 0x2b, 0x19, 0x44, 0x0d, - 0xda, 0x43, 0xae, 0xd3, 0x66, 0xd9, 0x25, 0xa8, 0x62, 0x6e, 0xa7, 0xbd, 0xdb, 0x3c, 0xec, 0x0e, - 0x95, 0x1c, 0x5b, 0x87, 0x4a, 0xaf, 0xaf, 0x37, 0x5b, 0xc3, 0x4e, 0xbf, 0xa7, 0xe4, 0xd5, 0xcf, - 0xa1, 0xdc, 0x3a, 0xb6, 0xc6, 0x27, 0x17, 0x8d, 0x22, 0x99, 0x9d, 0xd6, 0xf8, 0x44, 0xe8, 0xa7, - 0x0b, 0x66, 0xa7, 0x35, 0x3e, 0x51, 0x9f, 0x40, 0xad, 0x15, 0x09, 0x85, 0x8b, 0x6a, 0xb9, 0x0f, - 0x75, 0xda, 0x7c, 0xe3, 0x51, 0xb4, 0xfb, 0xb2, 0x2b, 0x76, 0x5f, 0x0d, 0x69, 0x5a, 0x23, 0xb1, - 0xfd, 0x3e, 0x84, 0xea, 0x81, 0xef, 0xcd, 0x2c, 0x3f, 0xa4, 0x6a, 0x15, 0xc8, 0x9d, 0x58, 0xe7, - 0xa2, 0x56, 0x4c, 0x26, 0x86, 0x79, 0x56, 0x36, 0xcc, 0xef, 0x43, 0x39, 0x2a, 0xf6, 0xbd, 0xcb, - 0x7c, 0x86, 0x5c, 0x8c, 0xca, 0xd8, 0x56, 0x80, 0x1f, 0xbb, 0x07, 0x30, 0x8b, 0x01, 0x42, 0xfb, - 0x88, 0x34, 0x6f, 0x51, 0xb9, 0x26, 0x51, 0xa8, 0x7f, 0x99, 0x83, 0xfa, 0x81, 0xe1, 0x87, 0x36, - 0x4e, 0x0e, 0x1f, 0x86, 0x37, 0x21, 0x4f, 0x4b, 0x9e, 0xfb, 0x00, 0x2e, 0xc7, 0x6a, 0x3b, 0xa7, - 0x21, 0x35, 0x82, 0x08, 0xd8, 0xa7, 0x50, 0x9f, 0x45, 0x60, 0x9d, 0x64, 0x03, 0x1f, 0x9b, 0xc5, - 0x22, 0x34, 0xe6, 0xeb, 0x33, 0x39, 0xcb, 0x7e, 0x04, 0x9b, 0xe9, 0xb2, 0x56, 0x10, 0x24, 0x7c, - 0x54, 0x9e, 0xac, 0xcb, 0xa9, 0x82, 0x9c, 0x8c, 0xb5, 0x60, 0x23, 0x29, 0x3e, 0xf6, 0x9c, 0xf9, - 0xd4, 0x0d, 0x84, 0x1d, 0x71, 0x75, 0xe1, 0xeb, 0x2d, 0x8e, 0xd5, 0x94, 0xd9, 0x02, 0x84, 0xa9, - 0x50, 0x8b, 0x61, 0xbd, 0xf9, 0x94, 0xb6, 0x44, 0x5e, 0x4b, 0xc1, 0xd8, 0x03, 0x80, 0x38, 0x8f, - 0x96, 0x63, 0x6e, 0x45, 0xff, 0x3a, 0xa1, 0x35, 0xd5, 0x24, 0x32, 0x54, 0x3f, 0x90, 0x19, 0xf8, - 0x76, 0x78, 0x3c, 0x25, 0x2e, 0x96, 0xd3, 0x12, 0x00, 0x31, 0xcb, 0x40, 0x47, 0x33, 0x35, 0x2e, - 0x22, 0x18, 0x5a, 0xdd, 0x0e, 0x06, 0xf3, 0x51, 0x5c, 0x2f, 0x8a, 0xd4, 0xa4, 0x97, 0xd3, 0xe0, - 0x48, 0x18, 0xf3, 0x49, 0x0b, 0xf7, 0x83, 0x23, 0x76, 0x1f, 0xae, 0x24, 0x44, 0x09, 0xff, 0x0d, - 0x1a, 0x40, 0x9c, 0x3b, 0x19, 0xbe, 0x98, 0x09, 0x07, 0xea, 0x17, 0xb0, 0x9e, 0x9a, 0x9d, 0xe7, - 0x0a, 0xf7, 0x6b, 0x50, 0xc6, 0xff, 0x28, 0xda, 0xc5, 0x02, 0x2c, 0x61, 0x7e, 0x10, 0xfa, 0xaa, - 0x05, 0xca, 0xe2, 0x58, 0xb3, 0xdb, 0xe4, 0xe0, 0xa2, 0x49, 0x59, 0x76, 0x54, 0x45, 0x28, 0xf6, - 0xf6, 0xaa, 0x49, 0xcc, 0x52, 0xab, 0x97, 0x26, 0x4b, 0xfd, 0xfd, 0xac, 0xd4, 0x66, 0x1c, 0x71, - 0xf6, 0xba, 0xbc, 0xfc, 0xa4, 0x8d, 0x9b, 0x8c, 0x19, 0x49, 0x9c, 0xb7, 0x40, 0xf1, 0x7c, 0xd3, - 0x76, 0x0d, 0x72, 0xb8, 0xf1, 0xe1, 0xce, 0x92, 0xb6, 0x78, 0x49, 0xc0, 0x0f, 0x04, 0x18, 0xed, - 0x16, 0xd3, 0x8a, 0xfd, 0x17, 0xc2, 0xfb, 0x20, 0x83, 0x64, 0xe9, 0x94, 0x4f, 0x4b, 0xa7, 0x37, - 0xa1, 0xe2, 0x58, 0x41, 0xa0, 0x87, 0xc7, 0x86, 0x4b, 0xf2, 0x3b, 0xdd, 0xe9, 0x32, 0x22, 0x87, - 0xc7, 0x86, 0x8b, 0x84, 0xb6, 0xab, 0x8b, 0x13, 0x8a, 0xe2, 0x32, 0xa1, 0xed, 0x92, 0xfd, 0x86, - 0x72, 0x7f, 0x73, 0xd5, 0xc4, 0x0a, 0xb1, 0xc8, 0x96, 0xe7, 0x55, 0x7d, 0x05, 0x4a, 0x4f, 0x6c, - 0xeb, 0x54, 0xf0, 0xb2, 0x67, 0xb6, 0x75, 0x1a, 0xf1, 0x32, 0x4c, 0xab, 0xff, 0xa3, 0x0c, 0x65, - 0x22, 0xde, 0xb9, 0xd8, 0xb1, 0xf9, 0x22, 0xd6, 0xc6, 0x96, 0x90, 0x53, 0xf9, 0x15, 0x36, 0x0e, - 0x97, 0x5a, 0xaf, 0x00, 0x48, 0x32, 0x94, 0x6b, 0x04, 0x95, 0x30, 0x16, 0x9d, 0xa8, 0xa6, 0x93, - 0x8e, 0x17, 0x7c, 0xe3, 0x08, 0xaf, 0x4c, 0x02, 0x60, 0xf7, 0xb8, 0x12, 0x4d, 0xfe, 0x98, 0x92, - 0xcc, 0x58, 0xa8, 0x0f, 0x91, 0x09, 0x4f, 0x9a, 0x35, 0x66, 0x48, 0x3f, 0xb0, 0xfc, 0x20, 0xda, - 0x4e, 0xeb, 0x5a, 0x94, 0x45, 0x8e, 0x86, 0xca, 0x93, 0x30, 0xb9, 0xa3, 0xed, 0x2b, 0x6b, 0x7f, - 0x1a, 0x11, 0xb0, 0x3b, 0x50, 0x22, 0x91, 0x6d, 0xa1, 0x04, 0x97, 0x58, 0x67, 0xa4, 0x4c, 0x69, - 0x11, 0x9a, 0xbd, 0x05, 0x85, 0xc9, 0x89, 0x75, 0x1e, 0x34, 0xd6, 0x65, 0x96, 0x90, 0x92, 0x85, - 0x1a, 0xa7, 0x60, 0xb7, 0xa1, 0xee, 0x5b, 0x13, 0x9d, 0x5c, 0x9d, 0x28, 0xbc, 0x83, 0x46, 0x9d, - 0x64, 0x73, 0xcd, 0xb7, 0x26, 0x2d, 0x04, 0x0e, 0x47, 0x4e, 0xc0, 0xde, 0x80, 0x22, 0x49, 0x25, - 0xb4, 0x31, 0xa4, 0x2f, 0x47, 0x22, 0x4e, 0x13, 0x58, 0x76, 0x1f, 0x2a, 0x09, 0xdb, 0xb8, 0x42, - 0x1d, 0xda, 0x5c, 0xe0, 0x47, 0xc4, 0xc6, 0xb5, 0x84, 0x8c, 0xbd, 0x0f, 0x20, 0xac, 0x1f, 0x7d, - 0x74, 0x4e, 0x87, 0x07, 0xd5, 0xd8, 0x3a, 0x94, 0x04, 0xa0, 0x6c, 0x23, 0xbd, 0x09, 0x05, 0x94, - 0x12, 0x41, 0xe3, 0x25, 0x6a, 0xcd, 0x46, 0x5a, 0x84, 0x50, 0xef, 0x08, 0xcf, 0xee, 0x40, 0x19, - 0x17, 0x97, 0x8e, 0x53, 0xd8, 0x90, 0xcd, 0x41, 0xb1, 0x12, 0x51, 0x4b, 0xb3, 0x4e, 0x07, 0xdf, - 0x38, 0xec, 0x2e, 0xe4, 0x4d, 0x6b, 0x12, 0x34, 0xae, 0x51, 0x8d, 0x57, 0xa5, 0xb9, 0x44, 0xc5, - 0x61, 0xc7, 0x9a, 0x70, 0xd1, 0x82, 0x34, 0x6c, 0x0f, 0xea, 0xb8, 0xf4, 0xee, 0x93, 0xe2, 0x8d, - 0x43, 0xde, 0xb8, 0x4e, 0xa5, 0x5e, 0x5d, 0x28, 0xd5, 0x13, 0x44, 0x34, 0x41, 0x6d, 0x37, 0xf4, - 0xcf, 0xb5, 0x75, 0x57, 0x86, 0xb1, 0xeb, 0x50, 0xb6, 0x83, 0xae, 0x37, 0x3e, 0xb1, 0xcc, 0xc6, - 0xcb, 0xfc, 0xbc, 0x31, 0xca, 0xb3, 0x4f, 0x60, 0x9d, 0x16, 0x23, 0x66, 0xf1, 0xe3, 0x8d, 0x1b, - 0xb2, 0xc8, 0x1b, 0xca, 0x28, 0x2d, 0x4d, 0x89, 0xea, 0x96, 0x1d, 0xe8, 0xa1, 0x35, 0x9d, 0x79, - 0x3e, 0x1a, 0x92, 0xaf, 0x70, 0xe3, 0xc9, 0x0e, 0x86, 0x11, 0x08, 0xf9, 0x7c, 0x7c, 0xd4, 0xa9, - 0x7b, 0x93, 0x49, 0x60, 0x85, 0x8d, 0x9b, 0xb4, 0xd7, 0xea, 0xd1, 0x89, 0x67, 0x9f, 0xa0, 0xa4, - 0x94, 0x06, 0xba, 0x79, 0xee, 0x1a, 0x53, 0x7b, 0xdc, 0xb8, 0xc5, 0xed, 0x55, 0x3b, 0xd8, 0xe1, - 0x00, 0xd9, 0x64, 0xdc, 0x4a, 0x99, 0x8c, 0x97, 0xa1, 0x60, 0x8e, 0x70, 0x0b, 0xbf, 0x4a, 0xd5, - 0xe6, 0xcd, 0x51, 0xc7, 0xbc, 0xfe, 0x88, 0xcc, 0x44, 0x6a, 0xe4, 0x87, 0x0b, 0xca, 0x40, 0x6a, - 0xf5, 0x4b, 0x5a, 0xc3, 0xde, 0x9a, 0xac, 0x13, 0x6c, 0x17, 0x20, 0x67, 0x5a, 0x93, 0xeb, 0x9f, - 0x03, 0x5b, 0x1e, 0xde, 0xe7, 0x69, 0x26, 0x05, 0xa1, 0x99, 0x7c, 0x9a, 0x7d, 0x98, 0x51, 0x3f, - 0x81, 0xf5, 0xd4, 0x5e, 0x5d, 0xa9, 0x61, 0x71, 0x4b, 0xc3, 0x98, 0x0a, 0xcf, 0x0c, 0xcf, 0xa8, - 0xff, 0x3a, 0x07, 0xb5, 0x3d, 0x23, 0x38, 0xde, 0x37, 0x66, 0x83, 0xd0, 0x08, 0x03, 0x1c, 0xf0, - 0x63, 0x23, 0x38, 0x9e, 0x1a, 0x33, 0x6e, 0xd6, 0x65, 0xb8, 0x53, 0x49, 0xc0, 0xd0, 0xa6, 0xc3, - 0xa9, 0xc6, 0x6c, 0xdf, 0x3d, 0x78, 0x2c, 0x3c, 0x46, 0x71, 0x1e, 0x99, 0x43, 0x70, 0x3c, 0x9f, - 0x4c, 0x1c, 0x4b, 0x30, 0xb1, 0x28, 0xcb, 0x6e, 0xc3, 0xba, 0x48, 0x92, 0x4d, 0x77, 0x26, 0x0e, - 0x9f, 0xd3, 0x40, 0xf6, 0x00, 0xaa, 0x02, 0x30, 0x8c, 0x58, 0x59, 0x3d, 0xf6, 0x04, 0x26, 0x08, - 0x4d, 0xa6, 0x62, 0x3f, 0x86, 0x2b, 0x52, 0x76, 0xd7, 0xf3, 0xf7, 0xe7, 0x4e, 0x68, 0xb7, 0x7a, - 0x42, 0x81, 0x7e, 0x79, 0xa9, 0x78, 0x42, 0xa2, 0xad, 0x2e, 0x99, 0x6e, 0xed, 0xbe, 0xed, 0x0a, - 0xf5, 0x22, 0x0d, 0x5c, 0xa0, 0x32, 0xce, 0x88, 0x21, 0xa6, 0xa9, 0x8c, 0x33, 0x5c, 0xfe, 0x02, - 0xb0, 0x6f, 0x85, 0xc7, 0x9e, 0x49, 0xea, 0x45, 0xbc, 0xfc, 0x07, 0x32, 0x4a, 0x4b, 0x53, 0xe2, - 0x70, 0xba, 0x73, 0xc7, 0x19, 0xbb, 0x21, 0xd9, 0x50, 0x39, 0x2d, 0xca, 0xa2, 0xb0, 0xf0, 0x0d, - 0xf7, 0xc8, 0x0a, 0x1a, 0xd5, 0xad, 0xdc, 0x9d, 0x8c, 0x26, 0x72, 0xea, 0xdf, 0xc8, 0x42, 0x81, - 0xcf, 0xe4, 0xcb, 0x50, 0x19, 0x39, 0xde, 0xf8, 0x44, 0x77, 0xe7, 0xd3, 0xe8, 0x10, 0x81, 0x00, - 0xa8, 0x6f, 0x91, 0xed, 0x23, 0x3c, 0x7e, 0x19, 0x8d, 0xd2, 0x58, 0xa5, 0x37, 0x0f, 0xf1, 0x5b, - 0x39, 0x82, 0x8a, 0x1c, 0x36, 0xc2, 0xf7, 0x4e, 0x69, 0x35, 0xe4, 0x09, 0x11, 0x65, 0xe9, 0x9c, - 0x82, 0xe4, 0x0e, 0x16, 0x2a, 0x10, 0xae, 0x4c, 0x80, 0x96, 0x1b, 0x2e, 0x7a, 0x27, 0x8b, 0x4b, - 0xde, 0x49, 0x76, 0x13, 0xd0, 0xb2, 0x1a, 0x5b, 0x7d, 0xd7, 0x6a, 0xf5, 0x68, 0x84, 0xcb, 0x9a, - 0x04, 0x61, 0x1f, 0xc5, 0x6b, 0x91, 0x7a, 0x24, 0x7c, 0xc7, 0x82, 0xa3, 0xca, 0xab, 0x56, 0x4b, - 0xd1, 0xa9, 0x4f, 0x01, 0x34, 0xef, 0x34, 0xb0, 0x42, 0xd2, 0xb9, 0x5e, 0xa2, 0xe6, 0xa7, 0x8e, - 0x07, 0xbd, 0xd3, 0x03, 0x2f, 0x88, 0x4e, 0x59, 0xb3, 0xf1, 0x29, 0x6b, 0xac, 0x9e, 0xe5, 0x56, - 0xab, 0x67, 0xea, 0xbb, 0x50, 0x42, 0xb9, 0x6b, 0x84, 0x06, 0xbb, 0x2d, 0xfc, 0x9c, 0x5c, 0xef, - 0x12, 0x0e, 0xdf, 0xe4, 0xab, 0xc2, 0xf3, 0xd9, 0x8d, 0x5a, 0x42, 0x65, 0x5e, 0x95, 0x5c, 0x1f, - 0x31, 0xff, 0x16, 0x15, 0x0a, 0x49, 0xfe, 0x32, 0x54, 0xb0, 0xb1, 0x74, 0xd2, 0x22, 0x5a, 0x56, - 0xf6, 0xbd, 0xd3, 0x16, 0xe6, 0xd5, 0xff, 0x94, 0x81, 0x6a, 0xdf, 0x37, 0x51, 0x70, 0x0c, 0x66, - 0xd6, 0xf8, 0xb9, 0xda, 0x24, 0xca, 0x7d, 0xcf, 0x71, 0x8c, 0x58, 0x17, 0x43, 0xb9, 0x1f, 0x01, - 0xd8, 0xfb, 0x90, 0x9f, 0x38, 0xc6, 0x11, 0x75, 0x36, 0xb6, 0x32, 0xa5, 0xea, 0xa3, 0xf4, 0xae, - 0x63, 0x1c, 0x69, 0x44, 0xaa, 0xfe, 0x46, 0xfc, 0x7d, 0x3a, 0x73, 0x91, 0x4f, 0x5a, 0xd6, 0xe8, - 0xd4, 0x6f, 0xd0, 0x52, 0x32, 0xac, 0x0c, 0xf9, 0x9d, 0xf6, 0xa0, 0xc5, 0x6d, 0x4b, 0xb4, 0x32, - 0x07, 0xfa, 0x6e, 0x47, 0x1b, 0x0c, 0x95, 0x3c, 0x1d, 0x23, 0x12, 0xa0, 0xdb, 0x1c, 0x0c, 0x95, - 0x32, 0x03, 0x28, 0x1e, 0xf6, 0x3a, 0x3f, 0x3e, 0x6c, 0x2b, 0x8a, 0xfa, 0xef, 0x32, 0x00, 0xc9, - 0x81, 0x00, 0x7b, 0x1b, 0xaa, 0xa7, 0x94, 0xd3, 0xa5, 0x93, 0x22, 0xb9, 0x8f, 0xc0, 0xd1, 0xa4, - 0x93, 0xfc, 0x40, 0x32, 0x31, 0x50, 0xf6, 0x2e, 0x1f, 0x19, 0x55, 0x67, 0x89, 0xd8, 0x66, 0xef, - 0x40, 0xd9, 0xc3, 0x7e, 0x20, 0x69, 0x4e, 0x16, 0xbc, 0x52, 0xf7, 0xb5, 0x92, 0xc7, 0x33, 0x28, - 0xa3, 0x27, 0x7e, 0xe4, 0x4a, 0x8a, 0x49, 0x77, 0x11, 0xd4, 0x72, 0x8c, 0x79, 0x60, 0x69, 0x1c, - 0x1f, 0xb3, 0xdd, 0x42, 0xc2, 0x76, 0xd5, 0x9f, 0x40, 0x7d, 0x60, 0x4c, 0x67, 0x9c, 0x39, 0x53, - 0xc7, 0x18, 0xe4, 0x71, 0x4d, 0x88, 0xc5, 0x48, 0x69, 0xdc, 0x62, 0x07, 0x96, 0x3f, 0xb6, 0xdc, - 0x68, 0x47, 0x46, 0x59, 0x64, 0xb6, 0x87, 0x81, 0xed, 0x1e, 0x69, 0xde, 0x69, 0x14, 0xc7, 0x13, - 0xe5, 0xd5, 0x7f, 0x90, 0x81, 0xaa, 0xd4, 0x0c, 0xf6, 0x6e, 0xca, 0xa2, 0x7c, 0x79, 0xa9, 0x9d, - 0x3c, 0x2d, 0x59, 0x96, 0x6f, 0x40, 0x21, 0x08, 0x0d, 0x3f, 0x3a, 0x5b, 0x52, 0xa4, 0x12, 0xdb, - 0xde, 0xdc, 0x35, 0x35, 0x8e, 0x66, 0x2a, 0xe4, 0x2c, 0xd7, 0x14, 0xdb, 0x62, 0x99, 0x0a, 0x91, - 0xea, 0x16, 0x54, 0xe2, 0xea, 0x71, 0x09, 0x68, 0xfd, 0xa7, 0x03, 0x65, 0x8d, 0x55, 0xa0, 0xa0, - 0x35, 0x7b, 0x8f, 0xda, 0x4a, 0x46, 0xfd, 0xe3, 0x0c, 0x40, 0x52, 0x8a, 0xdd, 0x4b, 0xb5, 0xf6, - 0xfa, 0x62, 0xad, 0xf7, 0xe8, 0xaf, 0xd4, 0xd8, 0x1b, 0x50, 0x99, 0xbb, 0x04, 0xb4, 0x4c, 0x21, - 0x77, 0x12, 0x00, 0xbb, 0x01, 0xb9, 0x28, 0xe2, 0x67, 0x21, 0xca, 0xe2, 0x99, 0xe1, 0xa8, 0x9f, - 0x42, 0x25, 0xae, 0x8e, 0xad, 0x43, 0x65, 0xb7, 0xdf, 0xed, 0xf6, 0x9f, 0x76, 0x7a, 0x8f, 0x94, - 0x35, 0xcc, 0x1e, 0x68, 0xed, 0x56, 0x7b, 0x07, 0xb3, 0x19, 0x5c, 0xb3, 0xad, 0x43, 0x4d, 0x6b, - 0xf7, 0x86, 0xba, 0xd6, 0x7f, 0xaa, 0x64, 0xd5, 0xdf, 0xca, 0xc3, 0x46, 0xdf, 0xdd, 0x99, 0xcf, - 0x1c, 0x7b, 0x6c, 0x84, 0xd6, 0x63, 0xeb, 0xbc, 0x15, 0x9e, 0xa1, 0x38, 0x35, 0xc2, 0xd0, 0xe7, - 0x9b, 0xb9, 0xa2, 0xf1, 0x0c, 0x77, 0xd0, 0x05, 0x96, 0x1f, 0x92, 0xff, 0x51, 0xde, 0xc5, 0x75, - 0x0e, 0x6f, 0x79, 0x0e, 0xed, 0x65, 0xf6, 0x23, 0xb8, 0xc2, 0x9d, 0x7a, 0x9c, 0x12, 0x95, 0x4e, - 0x5d, 0xf0, 0x9e, 0xc5, 0xa5, 0xcb, 0x38, 0x21, 0x16, 0x45, 0x32, 0x62, 0x6a, 0xb7, 0xa0, 0x9a, - 0x14, 0xe7, 0xa6, 0x41, 0x45, 0x83, 0x98, 0x90, 0x5a, 0xe2, 0xb9, 0xba, 0x19, 0xb5, 0x5a, 0xb7, - 0xcd, 0x33, 0x32, 0x97, 0x0a, 0x5a, 0xdd, 0x4b, 0x3a, 0x83, 0x22, 0xf7, 0x4b, 0xd8, 0x48, 0x51, - 0x52, 0x2b, 0xb8, 0xc1, 0xf4, 0x4e, 0x74, 0x58, 0xb0, 0xd0, 0x7b, 0x19, 0x82, 0xcd, 0xe1, 0x1a, - 0xe1, 0x25, 0x2f, 0x0d, 0x45, 0x66, 0x66, 0x07, 0xba, 0x7d, 0xe4, 0x7a, 0xbe, 0x25, 0xd8, 0x7b, - 0xd9, 0x0e, 0x3a, 0x94, 0x4f, 0x6c, 0x16, 0xe9, 0x88, 0x9d, 0x4b, 0x93, 0xe8, 0x84, 0x99, 0xa3, - 0x6d, 0x2e, 0x2f, 0xf3, 0x5a, 0x89, 0xf2, 0x1d, 0x13, 0xcd, 0x75, 0x8e, 0x8a, 0xcc, 0x10, 0x20, - 0x33, 0xa4, 0x46, 0xc0, 0x27, 0x1c, 0x76, 0xbd, 0x07, 0x9b, 0xab, 0x1a, 0xb9, 0x42, 0xaf, 0xda, - 0x92, 0xf5, 0xaa, 0x05, 0x07, 0x56, 0xa2, 0x63, 0xfd, 0x8b, 0x2c, 0x54, 0xb8, 0x9f, 0x0d, 0x67, - 0xff, 0x0e, 0x94, 0xbc, 0xd1, 0xd7, 0xba, 0x6f, 0x4d, 0x2e, 0x3a, 0xc2, 0x2e, 0x7a, 0xa3, 0xaf, - 0x35, 0x6b, 0xc2, 0xde, 0x8e, 0x44, 0xa4, 0x69, 0x4d, 0xc4, 0x17, 0xea, 0x69, 0xe5, 0x5a, 0x88, - 0xcc, 0x1d, 0x22, 0xae, 0x26, 0xcb, 0x27, 0x58, 0xb1, 0x14, 0x20, 0x5e, 0x45, 0x68, 0xd6, 0x57, - 0xb9, 0x8f, 0x51, 0x97, 0xac, 0xc3, 0x14, 0x31, 0x47, 0x13, 0xf1, 0x3d, 0xb8, 0xbc, 0x68, 0xe4, - 0xda, 0x26, 0x77, 0x80, 0xe7, 0xb5, 0x8d, 0xb4, 0x8d, 0xdb, 0x31, 0x83, 0x8b, 0xbd, 0x1d, 0xc5, - 0x0b, 0xbd, 0x1d, 0x69, 0x37, 0x0a, 0xae, 0xb7, 0x12, 0xad, 0xfc, 0x84, 0x33, 0x77, 0xcc, 0x33, - 0xf5, 0x9f, 0x64, 0xa1, 0xd2, 0xe1, 0x9d, 0x08, 0xcf, 0xd8, 0xab, 0x90, 0xfb, 0x8e, 0x31, 0x44, - 0x1c, 0xbb, 0x0b, 0x1b, 0x86, 0x69, 0xea, 0xc6, 0x64, 0x62, 0x8d, 0x43, 0xcb, 0xd4, 0x51, 0xf7, - 0x10, 0xdb, 0xff, 0x92, 0x61, 0x9a, 0x4d, 0x01, 0x27, 0x36, 0xca, 0x5d, 0x3e, 0x91, 0x0d, 0xc6, - 0x0f, 0x39, 0x72, 0x91, 0xcb, 0x47, 0x98, 0x60, 0xfc, 0x88, 0x23, 0x35, 0x2d, 0xf9, 0xe7, 0x4c, - 0xcb, 0x5f, 0x9b, 0xc1, 0xfb, 0x7b, 0x39, 0x00, 0xcd, 0x9a, 0x39, 0xc6, 0xd8, 0xfa, 0x7f, 0x67, - 0xf4, 0x6e, 0x49, 0xeb, 0xd4, 0x35, 0xa3, 0x20, 0x9d, 0x68, 0x6d, 0x92, 0x20, 0x58, 0x39, 0xbc, - 0xc5, 0x17, 0x1e, 0xde, 0xd2, 0x0b, 0x0c, 0x6f, 0x79, 0x79, 0x78, 0xd9, 0xe7, 0xf0, 0x8a, 0x6f, - 0x9d, 0xfa, 0x76, 0x68, 0xe9, 0x13, 0xdf, 0x9b, 0xea, 0x29, 0xb6, 0x88, 0x5c, 0xa3, 0x42, 0xa3, - 0x71, 0x4d, 0x10, 0xed, 0xfa, 0xde, 0x34, 0xcd, 0x1a, 0xd5, 0xff, 0x59, 0x80, 0x6a, 0xd3, 0x35, - 0x9c, 0xf3, 0x6f, 0x2d, 0x0a, 0xe4, 0xa1, 0x63, 0x90, 0xd9, 0x3c, 0xe4, 0xe3, 0xce, 0x4f, 0xb6, - 0x2b, 0x04, 0xa1, 0x11, 0xbf, 0x05, 0x55, 0x6f, 0x1e, 0xc6, 0x78, 0x7e, 0xd6, 0x0d, 0x1c, 0x44, - 0x04, 0x71, 0xf9, 0xf8, 0x88, 0x2d, 0x2a, 0x4f, 0x96, 0x58, 0x52, 0x3e, 0xd6, 0xce, 0xe3, 0xf2, - 0x44, 0x80, 0xac, 0xd2, 0x9e, 0xd2, 0xc8, 0x07, 0xf3, 0xa9, 0xc5, 0x47, 0x3f, 0xc7, 0x03, 0x26, - 0x5b, 0x02, 0x86, 0xb5, 0x4c, 0xad, 0xa9, 0xe7, 0x9f, 0xf3, 0x5a, 0x8a, 0xbc, 0x16, 0x0e, 0xa2, - 0x5a, 0xde, 0x01, 0x76, 0x6a, 0xd8, 0xa1, 0x9e, 0xae, 0x8a, 0x5b, 0x44, 0x0a, 0x62, 0x86, 0x72, - 0x75, 0x57, 0xa1, 0x68, 0xda, 0xc1, 0x49, 0xa7, 0x2f, 0xac, 0x21, 0x91, 0xc3, 0xbe, 0x04, 0x63, - 0x03, 0xd5, 0xb3, 0xd0, 0x0a, 0x68, 0x28, 0x73, 0x5a, 0x05, 0x21, 0xdb, 0x08, 0x40, 0xf1, 0xee, - 0x5a, 0xe1, 0xa9, 0xe7, 0x63, 0x49, 0x6e, 0xec, 0x24, 0x00, 0x54, 0x83, 0x90, 0x14, 0x3f, 0x44, - 0xee, 0xa5, 0x9c, 0x16, 0xe7, 0xd1, 0x8c, 0xe0, 0x6c, 0x91, 0xb0, 0x35, 0xde, 0xfc, 0x04, 0xc2, - 0x6e, 0x43, 0x9d, 0x9a, 0x4f, 0xc6, 0x10, 0xf6, 0x81, 0x8e, 0xa3, 0x73, 0x5a, 0x0d, 0xa1, 0xe4, - 0x69, 0x40, 0xaa, 0x4f, 0xe0, 0x5a, 0xaa, 0x7f, 0xba, 0xe1, 0xfb, 0xc6, 0xb9, 0x3e, 0x35, 0xbe, - 0xf6, 0x7c, 0xf2, 0x24, 0xe5, 0xb4, 0xab, 0xf2, 0xb0, 0x35, 0x11, 0xbd, 0x8f, 0xd8, 0x0b, 0x8b, - 0xda, 0xae, 0xe7, 0x93, 0x9b, 0x69, 0x65, 0x51, 0xc4, 0x92, 0x7f, 0x83, 0x26, 0x98, 0x2c, 0xb3, - 0x80, 0x07, 0xda, 0x6a, 0x55, 0x82, 0x6d, 0x13, 0x08, 0xed, 0x97, 0xe0, 0x81, 0x4e, 0x61, 0x2a, - 0x1b, 0x22, 0x1e, 0xee, 0x01, 0xc5, 0x28, 0x72, 0xc4, 0xb1, 0x65, 0x98, 0x74, 0xc4, 0x4d, 0x88, - 0x3d, 0xcb, 0xa0, 0x00, 0x92, 0xe0, 0x81, 0x3e, 0x9b, 0x87, 0x3c, 0x42, 0x56, 0x2b, 0x04, 0x0f, - 0x0e, 0xe6, 0xa1, 0x00, 0x1f, 0x59, 0x21, 0xc5, 0xc5, 0x12, 0xf8, 0x91, 0x15, 0xa2, 0x94, 0x0e, - 0x1e, 0x44, 0xc7, 0x55, 0x57, 0xc4, 0xd8, 0x3e, 0x10, 0xe7, 0x51, 0x2a, 0xac, 0xc7, 0x48, 0x7d, - 0x3a, 0xe7, 0x21, 0xb1, 0x39, 0xad, 0x1a, 0x11, 0xec, 0xcf, 0x1d, 0xd5, 0x97, 0x8e, 0x36, 0x0e, - 0xfc, 0xb9, 0x6b, 0x71, 0x67, 0x10, 0x25, 0x4d, 0x71, 0xc8, 0x1c, 0xe7, 0xd9, 0x0e, 0x5c, 0xe6, - 0x36, 0xa0, 0x65, 0xea, 0x92, 0xcb, 0x3f, 0x7b, 0xb1, 0xcb, 0x9f, 0x45, 0xf4, 0x31, 0x38, 0x50, - 0x7f, 0x9a, 0x81, 0xeb, 0x7d, 0x3a, 0xf1, 0xa6, 0x4d, 0xbe, 0x6f, 0x05, 0x81, 0x71, 0x84, 0x06, - 0xfc, 0xee, 0xfc, 0xdb, 0x6f, 0xcf, 0xd9, 0x1d, 0xb8, 0x74, 0x60, 0xf8, 0x96, 0x1b, 0xc6, 0x2c, - 0x40, 0x48, 0xfc, 0x45, 0x30, 0x7b, 0x48, 0x6e, 0x75, 0xcb, 0x0d, 0x0f, 0x63, 0xdd, 0x49, 0xb4, - 0x25, 0xed, 0x68, 0x5d, 0xa2, 0x52, 0xff, 0xed, 0x4d, 0xc8, 0xf7, 0x3c, 0xd3, 0x62, 0xef, 0x41, - 0x85, 0x22, 0x34, 0x97, 0x4f, 0x73, 0x10, 0x4d, 0x7f, 0x48, 0x8d, 0x2d, 0xbb, 0x22, 0x75, 0x71, - 0x4c, 0xe7, 0xab, 0xa4, 0x90, 0xd3, 0x71, 0x30, 0x32, 0xd5, 0xaa, 0x70, 0x11, 0x90, 0x8d, 0xcb, - 0x31, 0x38, 0xb6, 0xe4, 0xe2, 0xf4, 0x2d, 0x97, 0x64, 0x7e, 0x41, 0x8b, 0xf3, 0x64, 0x06, 0xf9, - 0x1e, 0x0a, 0x00, 0xbe, 0x5e, 0x0a, 0x2b, 0xcc, 0x20, 0x8e, 0xa7, 0x05, 0xf4, 0x1e, 0x54, 0xbe, - 0xf6, 0x6c, 0x97, 0x37, 0xbc, 0xb8, 0xd4, 0xf0, 0x2f, 0x3c, 0x9b, 0x1f, 0x43, 0x95, 0xbf, 0x16, - 0x29, 0xf6, 0x1a, 0x94, 0x3c, 0x97, 0xd7, 0x5d, 0x5a, 0xaa, 0xbb, 0xe8, 0xb9, 0x5d, 0x1e, 0x2e, - 0xb5, 0x3e, 0x9a, 0xdb, 0x8e, 0x89, 0xfc, 0xd3, 0xb1, 0x26, 0xa1, 0x38, 0x75, 0xa9, 0x12, 0xb0, - 0xef, 0x76, 0xad, 0x49, 0x88, 0xca, 0xcb, 0xc4, 0x76, 0x50, 0xce, 0x50, 0x65, 0x95, 0x65, 0xe5, - 0x85, 0xa3, 0xa9, 0xc2, 0xd7, 0xa1, 0x7c, 0xe4, 0x7b, 0xf3, 0x19, 0x9a, 0x6b, 0xb0, 0x7c, 0xce, - 0x41, 0xb8, 0xed, 0x73, 0x64, 0x76, 0x94, 0xb4, 0xdd, 0x23, 0x9d, 0x2c, 0xdb, 0xea, 0x56, 0xee, - 0x4e, 0x59, 0xab, 0x45, 0x40, 0xb2, 0x59, 0x5f, 0x87, 0xb2, 0x71, 0x74, 0xa4, 0x8b, 0xa8, 0xaf, - 0xa5, 0xba, 0x8c, 0xa3, 0x23, 0xfa, 0xe4, 0x3d, 0x58, 0x3f, 0xb5, 0x5d, 0x3d, 0x98, 0x59, 0x63, - 0x4e, 0xbb, 0xbe, 0x3c, 0x94, 0xa7, 0xb6, 0x8b, 0x06, 0x1d, 0xd1, 0xcb, 0x16, 0x65, 0xfd, 0xb9, - 0x16, 0xe5, 0x16, 0x14, 0x1c, 0x7b, 0x6a, 0x87, 0x22, 0x0e, 0x2c, 0xa5, 0x72, 0x12, 0x82, 0xa9, - 0x50, 0x14, 0xae, 0x4c, 0x65, 0x89, 0x44, 0x60, 0xd2, 0x52, 0x78, 0xe3, 0x39, 0x52, 0x58, 0xd2, - 0x58, 0xd9, 0x77, 0x6b, 0xac, 0x1f, 0xd2, 0x79, 0x8f, 0xe5, 0x86, 0x7a, 0x54, 0xe0, 0xf2, 0xea, - 0x02, 0x35, 0x4e, 0xd6, 0xe7, 0xc5, 0xde, 0x87, 0xaa, 0x4f, 0xae, 0x0e, 0x9d, 0xfc, 0x22, 0x9b, - 0xb2, 0xad, 0x98, 0xf8, 0x40, 0x34, 0xf0, 0x13, 0x7f, 0xc8, 0x6b, 0xb0, 0xce, 0xc3, 0x51, 0x78, - 0xd0, 0x40, 0x40, 0xcc, 0xa5, 0xa2, 0xd5, 0x08, 0xc8, 0x03, 0x0a, 0x50, 0x73, 0x85, 0x48, 0x7d, - 0x08, 0xcf, 0x28, 0xe6, 0x3e, 0x6e, 0x0a, 0x67, 0x41, 0xad, 0xf0, 0x4c, 0xab, 0x98, 0x51, 0x12, - 0x39, 0xea, 0xc8, 0x76, 0x4d, 0x5c, 0x04, 0xa1, 0x71, 0x14, 0x34, 0x1a, 0xb4, 0x47, 0xaa, 0x02, - 0x36, 0x34, 0x8e, 0x02, 0xf6, 0x01, 0xd4, 0x0c, 0x2e, 0xa4, 0x79, 0x5c, 0xee, 0x35, 0xd9, 0xae, - 0x97, 0xc4, 0xb7, 0x56, 0x35, 0x24, 0x59, 0xfe, 0x31, 0xb0, 0xe8, 0x9c, 0x84, 0x6c, 0x24, 0xbe, - 0x2e, 0xae, 0x2f, 0xad, 0x8b, 0x4b, 0xe2, 0xa0, 0x24, 0x8e, 0x25, 0xff, 0x18, 0xd6, 0xd3, 0x4a, - 0xd5, 0x8d, 0x15, 0x27, 0x03, 0x34, 0x65, 0x5a, 0x6d, 0x2c, 0xab, 0x59, 0xaf, 0xc1, 0xba, 0xeb, - 0x85, 0xfa, 0xd8, 0x18, 0x1f, 0x5b, 0x54, 0x90, 0x7b, 0xbf, 0x6b, 0xae, 0x17, 0xb6, 0x22, 0x18, - 0x8e, 0x4f, 0x64, 0x33, 0x84, 0x67, 0xe4, 0xf8, 0x8e, 0xc7, 0x27, 0xd6, 0xb3, 0x51, 0x67, 0x88, - 0x54, 0x6e, 0x9c, 0x27, 0xae, 0x42, 0x52, 0x81, 0x5b, 0xa9, 0x79, 0x8a, 0x75, 0x4b, 0x0d, 0xfc, - 0x44, 0xcf, 0xbc, 0x05, 0xd5, 0xc0, 0x9b, 0xfb, 0x63, 0x4b, 0x0f, 0x42, 0x6b, 0xd6, 0xd8, 0xa2, - 0x11, 0x05, 0x0e, 0x1a, 0x84, 0xd6, 0x8c, 0x3d, 0x84, 0xfa, 0xcc, 0xb7, 0x74, 0x69, 0x9e, 0x5e, - 0x95, 0xbb, 0x78, 0xe0, 0x5b, 0xc9, 0x54, 0xd5, 0x66, 0x52, 0x2e, 0x2a, 0x29, 0xf5, 0x40, 0x5d, - 0x28, 0x99, 0x74, 0x02, 0x4b, 0x26, 0xa6, 0xc3, 0x67, 0xb0, 0x21, 0x95, 0x9c, 0x9f, 0x50, 0xe1, - 0xd7, 0x52, 0x07, 0x35, 0x11, 0xf9, 0xe1, 0x09, 0x16, 0xaf, 0xcf, 0x52, 0x79, 0xd6, 0x5c, 0xb0, - 0x90, 0x51, 0xc1, 0xbb, 0x4d, 0xe5, 0x5f, 0xba, 0xc0, 0xec, 0x4d, 0x99, 0xce, 0x8f, 0xb9, 0x4b, - 0xbe, 0x13, 0xb4, 0x5d, 0xb3, 0xf1, 0x3a, 0xbf, 0xf0, 0x41, 0x19, 0xf6, 0x00, 0x6a, 0x5c, 0xd5, - 0xa0, 0x60, 0xd3, 0xa0, 0xf1, 0x86, 0xec, 0x22, 0x24, 0x7d, 0x83, 0x10, 0x5a, 0xd5, 0x89, 0xd3, - 0x01, 0xfb, 0x08, 0x36, 0xb8, 0xb7, 0x56, 0x66, 0x8b, 0x6f, 0x2e, 0x2f, 0x2e, 0x22, 0xda, 0x4d, - 0x78, 0xa3, 0x06, 0xd7, 0xfc, 0xb9, 0x4b, 0xea, 0x87, 0x28, 0x39, 0xf3, 0xbd, 0x91, 0xc5, 0xcb, - 0xdf, 0xa1, 0xf2, 0xa2, 0x3b, 0x1a, 0x27, 0xe3, 0x65, 0x89, 0x1f, 0x5d, 0xf5, 0x65, 0xd0, 0x01, - 0x96, 0xbb, 0xa0, 0x4e, 0xce, 0xcf, 0xa9, 0xce, 0xb7, 0x5e, 0xa4, 0xce, 0x6d, 0x2c, 0x47, 0x75, - 0x32, 0xc8, 0xcf, 0xe7, 0xb6, 0xd9, 0xb8, 0xcb, 0xe3, 0x42, 0x31, 0xcd, 0x5e, 0x87, 0xba, 0x6f, - 0x8d, 0xe7, 0x7e, 0x60, 0x3f, 0xb3, 0xf4, 0xc0, 0x76, 0x4f, 0x1a, 0x6f, 0xd3, 0x38, 0xae, 0xc7, - 0xd0, 0x81, 0xed, 0x9e, 0xe0, 0x8a, 0xb5, 0xce, 0x42, 0xcb, 0x77, 0x75, 0x54, 0xf9, 0x1a, 0xef, - 0xc8, 0x2b, 0xb6, 0x4d, 0x88, 0xc1, 0xd8, 0x70, 0x35, 0xb0, 0xe2, 0x34, 0xfb, 0x11, 0x5c, 0x4a, - 0xd4, 0xfd, 0x19, 0x2a, 0x1e, 0x8d, 0x1f, 0xac, 0x3c, 0xc3, 0x23, 0xa5, 0x44, 0x4b, 0x0e, 0xb8, - 0xb9, 0xfe, 0x92, 0x5e, 0x5b, 0x01, 0x5f, 0x5b, 0xf7, 0xbe, 0xd7, 0xda, 0x1a, 0xd0, 0xda, 0x7a, - 0x03, 0xca, 0xb6, 0x1b, 0x5a, 0xfe, 0x33, 0xc3, 0x69, 0xbc, 0xbb, 0xc4, 0xc0, 0x63, 0x1c, 0xbb, - 0x0d, 0xa5, 0xc0, 0xb1, 0x91, 0x31, 0x35, 0xde, 0x5b, 0x22, 0x8b, 0x50, 0xec, 0x0e, 0x54, 0xe2, - 0x1b, 0x4e, 0x8d, 0xf7, 0x97, 0xe8, 0x12, 0x24, 0xbb, 0x09, 0xf9, 0x53, 0x5c, 0x8f, 0xf7, 0x97, - 0x3d, 0xc2, 0x08, 0x47, 0x89, 0x3f, 0xb1, 0x1d, 0x87, 0x4b, 0xfc, 0x07, 0x4b, 0x12, 0x7f, 0xd7, - 0x76, 0x1c, 0x2e, 0xf1, 0x27, 0x22, 0x85, 0xf2, 0x92, 0x4a, 0x60, 0x4f, 0x3e, 0x58, 0x96, 0x97, - 0x88, 0x7b, 0x42, 0x77, 0xc1, 0xaa, 0x01, 0xb9, 0x39, 0xb9, 0xb7, 0xf6, 0x43, 0x79, 0xac, 0xd2, - 0xfe, 0x4f, 0x0d, 0x82, 0x38, 0x8f, 0x36, 0x81, 0x70, 0xf2, 0xa2, 0x4d, 0xf6, 0x11, 0xbf, 0xa2, - 0xc0, 0x21, 0x68, 0x90, 0xbd, 0x07, 0xeb, 0x51, 0x74, 0x13, 0x7e, 0x2e, 0x68, 0x7c, 0xbc, 0xd4, - 0x82, 0x34, 0x01, 0xdb, 0x81, 0xda, 0x04, 0x35, 0xc0, 0x29, 0x57, 0x08, 0x1b, 0x0f, 0xa9, 0x21, - 0x5b, 0x91, 0x2c, 0xbe, 0x48, 0x61, 0xd4, 0x52, 0xa5, 0xd8, 0x3d, 0x60, 0xf6, 0x84, 0xcf, 0x27, - 0x1a, 0x79, 0x5c, 0xe9, 0x6b, 0x7c, 0x42, 0x8b, 0x73, 0x05, 0x86, 0x3d, 0x80, 0xf5, 0xc0, 0x72, - 0x4d, 0x7d, 0x1a, 0x08, 0xcd, 0xe2, 0x53, 0x6a, 0xa7, 0x60, 0xc3, 0xf1, 0x4d, 0x48, 0xad, 0x8a, - 0x54, 0xfb, 0x01, 0x57, 0x31, 0x1e, 0x00, 0xae, 0xf3, 0x67, 0x49, 0xa1, 0x5f, 0xbb, 0xa0, 0x10, - 0x52, 0x45, 0x85, 0x3e, 0x86, 0x4b, 0x3c, 0x38, 0x0c, 0x97, 0x24, 0x2f, 0xf6, 0x23, 0xb9, 0x58, - 0xec, 0xa2, 0xd2, 0xd6, 0xe7, 0x51, 0x32, 0xfa, 0x1a, 0x59, 0x5f, 0x81, 0x6b, 0xcc, 0x82, 0x63, - 0x2f, 0x6c, 0xfc, 0xba, 0xac, 0x30, 0x0c, 0x04, 0x54, 0xab, 0x21, 0x51, 0x94, 0x43, 0x01, 0x94, - 0x6c, 0xd0, 0x71, 0x68, 0x35, 0x3e, 0xe3, 0x02, 0x28, 0x06, 0xb6, 0x42, 0xec, 0x3c, 0x18, 0xb3, - 0x99, 0x73, 0xce, 0x17, 0xd5, 0xe7, 0xb4, 0xa8, 0x36, 0xa5, 0x45, 0xd5, 0x44, 0x24, 0xad, 0xaa, - 0x8a, 0x11, 0x25, 0xd9, 0x7d, 0xa8, 0xcd, 0xbc, 0x20, 0xd4, 0xcd, 0xa9, 0x43, 0x9b, 0xab, 0x29, - 0x6f, 0xea, 0x03, 0x2f, 0x08, 0x77, 0xa6, 0x0e, 0x89, 0xa1, 0x59, 0x9c, 0x56, 0xff, 0xac, 0x00, - 0xe5, 0x48, 0x99, 0x66, 0x55, 0x28, 0x1d, 0xf6, 0x1e, 0xf7, 0xfa, 0x4f, 0x7b, 0xca, 0x1a, 0xab, - 0x03, 0xd0, 0xed, 0x0b, 0x7d, 0xd0, 0x6a, 0xf6, 0xf8, 0x6d, 0x25, 0xba, 0xf3, 0xc1, 0xf3, 0x59, - 0xb6, 0x01, 0xeb, 0xbb, 0x87, 0x3d, 0x8a, 0x68, 0xe3, 0xa0, 0x1c, 0x82, 0xda, 0x5f, 0xf2, 0xb3, - 0x0b, 0x0e, 0xca, 0x23, 0x68, 0xbf, 0x39, 0x6c, 0x6b, 0x9d, 0x08, 0x54, 0xa0, 0xe0, 0xb8, 0xfe, - 0xa1, 0xd6, 0x12, 0x35, 0x15, 0xf1, 0xb3, 0x07, 0x5a, 0xff, 0x8b, 0x76, 0x6b, 0xa8, 0x00, 0xbb, - 0x02, 0x1b, 0x71, 0x1d, 0x51, 0xfd, 0x4a, 0x95, 0xd5, 0xa0, 0x1c, 0xd5, 0xa3, 0x6c, 0x62, 0xad, - 0x5a, 0xbb, 0x75, 0xa8, 0x0d, 0x3a, 0x4f, 0xda, 0x7a, 0x6b, 0xd8, 0x56, 0xae, 0xb0, 0x32, 0xe4, - 0x07, 0x9d, 0xde, 0x63, 0xe5, 0x2a, 0x5b, 0x87, 0x0a, 0xa6, 0x78, 0xed, 0x2f, 0x31, 0x06, 0xf5, - 0x84, 0x96, 0x60, 0x0d, 0x3a, 0x56, 0x79, 0xf4, 0x48, 0xb9, 0x89, 0xd5, 0xee, 0x74, 0x06, 0xc3, - 0x4e, 0xaf, 0x35, 0x54, 0x6e, 0x31, 0x80, 0xe2, 0x6e, 0xa7, 0x3b, 0x6c, 0x6b, 0xca, 0x16, 0xd6, - 0xf7, 0x45, 0xbf, 0xd3, 0x53, 0x5e, 0xa5, 0x3b, 0x2d, 0xcd, 0xfd, 0x83, 0x6e, 0x5b, 0x51, 0xe9, - 0x2b, 0x7d, 0x6d, 0xa8, 0xbc, 0xc6, 0x2a, 0x50, 0x38, 0xec, 0x61, 0xdb, 0x6e, 0xe3, 0x07, 0x29, - 0xa9, 0x37, 0xbb, 0x5d, 0xe5, 0x75, 0xe9, 0xfc, 0xe5, 0x0d, 0x4c, 0x3f, 0xed, 0xf4, 0x76, 0xfa, - 0x4f, 0x95, 0x37, 0x91, 0x6c, 0x5b, 0xeb, 0x37, 0x77, 0x5a, 0xcd, 0xc1, 0x50, 0xb9, 0x83, 0x15, - 0x0c, 0x0e, 0xba, 0x9d, 0xa1, 0xf2, 0x16, 0x52, 0x3d, 0x6a, 0x0e, 0xf7, 0xda, 0x9a, 0x72, 0x17, - 0xd3, 0xcd, 0xc1, 0xa0, 0xad, 0x0d, 0x95, 0xfb, 0x98, 0xee, 0xf4, 0x28, 0xfd, 0x00, 0xd3, 0x3b, - 0xed, 0x6e, 0x7b, 0xd8, 0x56, 0x3e, 0xc0, 0x01, 0xd3, 0xda, 0x07, 0xdd, 0x66, 0xab, 0xad, 0x7c, - 0xc8, 0x36, 0x41, 0xe9, 0xf7, 0xf4, 0x9d, 0xc3, 0x83, 0x6e, 0xa7, 0xd5, 0x1c, 0xb6, 0xf5, 0xc7, - 0xed, 0xaf, 0x94, 0x8f, 0x70, 0xb6, 0x0e, 0xb4, 0xb6, 0x2e, 0x8a, 0x7f, 0x1c, 0xe5, 0x45, 0x15, - 0x0f, 0x71, 0x04, 0x13, 0xbc, 0x7e, 0xf8, 0x58, 0xf9, 0x64, 0x01, 0x34, 0x78, 0xac, 0x7c, 0xca, - 0x14, 0xa8, 0xed, 0x1f, 0x76, 0x87, 0x1d, 0xfd, 0xf0, 0x60, 0xa7, 0x39, 0x6c, 0x2b, 0xbf, 0x86, - 0x9f, 0xee, 0xf6, 0x5b, 0x8f, 0xf5, 0xfe, 0x81, 0xf2, 0x43, 0xec, 0x11, 0x9d, 0x55, 0x0d, 0x70, - 0xea, 0x7e, 0x84, 0x15, 0xc4, 0x59, 0x1a, 0x8b, 0x5f, 0xc7, 0x4e, 0xee, 0x77, 0x7a, 0x87, 0x03, - 0xe5, 0x33, 0x24, 0xa6, 0x24, 0x61, 0x3e, 0xc7, 0x55, 0x30, 0xec, 0xec, 0xb7, 0x75, 0x31, 0x3c, - 0x4d, 0x1c, 0xda, 0xdd, 0x4e, 0xb7, 0xab, 0x6c, 0xd3, 0xe1, 0x41, 0x53, 0x1b, 0x76, 0x68, 0xea, - 0x5b, 0xd8, 0x88, 0xdd, 0xc3, 0x9f, 0xfc, 0xe4, 0x2b, 0x5d, 0xcc, 0xcd, 0x0e, 0xd6, 0xda, 0x3c, - 0x38, 0xe8, 0x7e, 0xa5, 0xb4, 0x69, 0xed, 0xf4, 0x07, 0xc3, 0x9d, 0xfd, 0xae, 0xb2, 0xab, 0xfa, - 0x50, 0x8e, 0xec, 0x2b, 0xa4, 0xe9, 0xf4, 0x7a, 0x6d, 0x4d, 0x59, 0xc3, 0x9a, 0xbb, 0xed, 0xdd, - 0xa1, 0x92, 0xa1, 0x83, 0x93, 0xce, 0xa3, 0xbd, 0xa1, 0x92, 0xc5, 0x64, 0xff, 0x10, 0xab, 0xcb, - 0xd1, 0xa4, 0xb6, 0xf7, 0x3b, 0x4a, 0x1e, 0x53, 0xcd, 0xde, 0xb0, 0xa3, 0x14, 0x68, 0xd2, 0x3b, - 0xbd, 0x47, 0xdd, 0xb6, 0x52, 0x44, 0xe8, 0x7e, 0x53, 0x7b, 0xac, 0x94, 0x78, 0xa5, 0x3b, 0xed, - 0x2f, 0x95, 0x32, 0x2b, 0x42, 0xb6, 0x7b, 0x5f, 0xa9, 0xa8, 0x77, 0xa0, 0xd4, 0x3c, 0x3a, 0xda, - 0x47, 0x9b, 0x15, 0x7b, 0x70, 0xd8, 0xed, 0xf2, 0x5b, 0x7e, 0xdb, 0xfd, 0xe1, 0xb0, 0xbf, 0xaf, - 0x64, 0x70, 0xad, 0x0d, 0xfb, 0x07, 0x4a, 0x56, 0xed, 0x40, 0x39, 0x92, 0x05, 0xd2, 0xcd, 0xaa, - 0x32, 0xe4, 0x0f, 0xb4, 0xf6, 0x13, 0x7e, 0xc4, 0xd7, 0x6b, 0x7f, 0x89, 0x6d, 0xc3, 0x14, 0x56, - 0x94, 0xc3, 0x0f, 0xf2, 0x2b, 0x50, 0x74, 0xb5, 0xaa, 0xdb, 0xe9, 0xb5, 0x9b, 0x9a, 0x52, 0x50, - 0xdf, 0x86, 0x4a, 0xcc, 0x01, 0x70, 0x6a, 0x5b, 0x5a, 0x7f, 0x30, 0xe0, 0x43, 0x42, 0x1b, 0x97, - 0x7a, 0xc6, 0xf3, 0x19, 0xf5, 0xff, 0x87, 0x72, 0xcc, 0x7c, 0x6e, 0x43, 0x76, 0x38, 0x10, 0xce, - 0xcd, 0xcd, 0x7b, 0xc9, 0xdd, 0xf8, 0x61, 0x94, 0xd2, 0xb2, 0xc3, 0x01, 0x7b, 0x07, 0x8a, 0xfc, - 0x66, 0x9c, 0x70, 0xae, 0x6f, 0xa6, 0x19, 0xda, 0x90, 0x70, 0x9a, 0xa0, 0x51, 0xbb, 0x50, 0x4f, - 0x63, 0xd8, 0x4d, 0x00, 0x8e, 0x93, 0x9c, 0x04, 0x12, 0x04, 0xcd, 0x6d, 0x71, 0xf3, 0x6e, 0x47, - 0x84, 0xb3, 0xc5, 0x79, 0xf5, 0xef, 0xe6, 0x00, 0x12, 0x05, 0x04, 0x55, 0x9c, 0xd8, 0x05, 0x50, - 0x10, 0x87, 0x56, 0x2f, 0x43, 0xc5, 0xf1, 0x0c, 0x53, 0xbe, 0xe3, 0x5e, 0x46, 0x00, 0x8d, 0x86, - 0x7c, 0xbf, 0xa6, 0xc2, 0x4f, 0x8c, 0xd9, 0x55, 0x28, 0x4e, 0x3c, 0x7f, 0x6a, 0x44, 0x81, 0x6f, - 0x22, 0x87, 0xac, 0x98, 0x1f, 0xa4, 0xa0, 0x1a, 0xe6, 0x52, 0xec, 0x3a, 0x45, 0x51, 0x0a, 0x60, - 0x17, 0x61, 0xa8, 0xa8, 0x5b, 0xee, 0xd8, 0xf1, 0x02, 0xcb, 0x44, 0x43, 0xb4, 0x48, 0xba, 0x16, - 0x44, 0xa0, 0xed, 0x73, 0xde, 0x5b, 0x7f, 0x6a, 0xbb, 0x46, 0x28, 0x3c, 0x78, 0xd4, 0xdb, 0x08, - 0x82, 0xcd, 0xfd, 0x3a, 0xf0, 0x84, 0x47, 0x80, 0x9f, 0xc9, 0x94, 0x11, 0x40, 0xcd, 0x7d, 0x05, - 0xc0, 0x0a, 0xc6, 0xc6, 0x8c, 0x57, 0x5e, 0xa1, 0xca, 0x2b, 0x02, 0xb2, 0x7d, 0xce, 0xba, 0x50, - 0x1f, 0x8e, 0x5a, 0x9e, 0x33, 0xf4, 0xd0, 0xb8, 0x6b, 0x79, 0x8e, 0xb0, 0xd5, 0x6f, 0x2f, 0x6a, - 0x6a, 0xf7, 0xd2, 0x64, 0xfc, 0xf0, 0x68, 0xa1, 0xec, 0xf5, 0x26, 0x5c, 0x5e, 0x41, 0xf6, 0x42, - 0x61, 0x31, 0xff, 0x30, 0x07, 0x90, 0xa8, 0xdb, 0xa9, 0x13, 0xa5, 0x4c, 0xfa, 0x44, 0xe9, 0x3e, - 0x5c, 0x15, 0x57, 0x53, 0xc4, 0x15, 0x84, 0x33, 0xdd, 0x76, 0xf5, 0x91, 0x11, 0x1d, 0xde, 0x31, - 0x81, 0xe5, 0x41, 0x2a, 0x1d, 0x77, 0xdb, 0x08, 0xd9, 0x43, 0xb8, 0x24, 0x97, 0x09, 0xcf, 0x67, - 0xe9, 0xc3, 0x47, 0xe9, 0xa6, 0xcf, 0x7a, 0x52, 0x7c, 0x78, 0x3e, 0x63, 0xef, 0xc1, 0x15, 0xdf, - 0x9a, 0xf8, 0x56, 0x70, 0xac, 0x87, 0x81, 0xfc, 0x31, 0x1e, 0x11, 0xb3, 0x21, 0x90, 0xc3, 0x20, - 0xfe, 0xd6, 0x7b, 0x70, 0x45, 0x28, 0xe2, 0x0b, 0xcd, 0xe3, 0xb7, 0x78, 0x37, 0x38, 0x52, 0x6e, - 0xdd, 0x2b, 0x00, 0xc2, 0x06, 0x89, 0xde, 0x6e, 0x28, 0x6b, 0x15, 0x6e, 0x6f, 0xa0, 0xd1, 0xf8, - 0x0e, 0x30, 0x3b, 0xd0, 0x17, 0xbc, 0xe8, 0xe2, 0x88, 0x4e, 0xb1, 0x83, 0x83, 0x94, 0x07, 0xfd, - 0x22, 0x07, 0x7d, 0xf9, 0x22, 0x07, 0xfd, 0x26, 0x14, 0xc8, 0x4c, 0x11, 0xfe, 0x72, 0x9e, 0x61, - 0x2a, 0xe4, 0x91, 0xf5, 0x90, 0x6f, 0xb7, 0x7e, 0xbf, 0x7e, 0x8f, 0x5e, 0xbe, 0xc0, 0xf9, 0x41, - 0xa8, 0x46, 0x38, 0xf5, 0xf7, 0x32, 0x50, 0x4f, 0xab, 0xd6, 0x3c, 0x14, 0x34, 0x89, 0x71, 0x2d, - 0x24, 0x71, 0xad, 0x2f, 0x43, 0x65, 0x76, 0x22, 0x02, 0x5a, 0xa3, 0xbd, 0x35, 0x3b, 0xe1, 0x81, - 0xac, 0xec, 0x2d, 0x28, 0xcd, 0x4e, 0xf8, 0x3a, 0xbe, 0x68, 0x5a, 0x8a, 0x33, 0x1e, 0x63, 0xf6, - 0x16, 0x94, 0xe6, 0x82, 0x34, 0x7f, 0x11, 0xe9, 0x9c, 0x48, 0xd5, 0x2d, 0xa8, 0xc9, 0xc6, 0x2c, - 0x2e, 0x47, 0x54, 0x5c, 0x79, 0xc3, 0x30, 0xa9, 0xfe, 0x56, 0x96, 0x48, 0x5e, 0xe8, 0x88, 0xeb, - 0x85, 0xce, 0x08, 0xb7, 0x28, 0x28, 0x46, 0xa7, 0x90, 0xb7, 0xb1, 0x17, 0xdd, 0xec, 0x87, 0x63, - 0x23, 0x68, 0xce, 0x43, 0xaf, 0xe5, 0x39, 0xe2, 0xd8, 0x55, 0xdc, 0x21, 0xc8, 0x47, 0xee, 0x57, - 0x71, 0xbd, 0xe8, 0x3d, 0x11, 0x68, 0x4f, 0xb7, 0x5c, 0xe8, 0xa8, 0xb7, 0xb0, 0x64, 0x2b, 0xd4, - 0xa2, 0x4b, 0x2e, 0x74, 0x8a, 0x7b, 0x1f, 0x2e, 0x25, 0x51, 0x8d, 0xd1, 0xe9, 0xf0, 0x62, 0x91, - 0xf5, 0x38, 0xa4, 0x11, 0xb3, 0xea, 0xef, 0x64, 0x60, 0x63, 0xc9, 0x36, 0xc4, 0xd1, 0x4a, 0x1e, - 0x01, 0xc1, 0x24, 0x7b, 0x15, 0x6a, 0x53, 0x23, 0x1c, 0x1f, 0xeb, 0x33, 0xdf, 0x9a, 0xd8, 0x67, - 0xd1, 0x4b, 0x26, 0x04, 0x3b, 0x20, 0x10, 0x9d, 0x74, 0xcf, 0x66, 0x64, 0x11, 0x4f, 0xed, 0x50, - 0x5c, 0x33, 0x02, 0x02, 0x75, 0xc9, 0x55, 0x16, 0x45, 0xc1, 0xe4, 0x2f, 0x08, 0xda, 0xb9, 0x01, - 0xc5, 0x4e, 0x6c, 0x83, 0xc6, 0x97, 0xfa, 0x73, 0xe2, 0x22, 0xbf, 0x07, 0x95, 0x16, 0x3d, 0x0a, - 0xb0, 0x6f, 0xcc, 0xd8, 0x5d, 0xc8, 0x4d, 0x8d, 0x99, 0x88, 0xcf, 0x69, 0xc4, 0xfe, 0x5f, 0x8e, - 0xbd, 0xb7, 0x6f, 0xcc, 0x38, 0xaf, 0x42, 0xa2, 0xeb, 0x1f, 0x41, 0x39, 0x02, 0xbc, 0x10, 0x57, - 0xfa, 0xcf, 0x39, 0xa8, 0xec, 0xc8, 0xde, 0x2a, 0xd4, 0xca, 0x43, 0x7f, 0xee, 0x8e, 0x71, 0xba, - 0xb8, 0xb7, 0xbc, 0x3a, 0x36, 0xdc, 0xa1, 0x00, 0x45, 0x0b, 0x28, 0xfb, 0x1d, 0x0b, 0xe8, 0x06, - 0x80, 0x4f, 0x36, 0x14, 0x99, 0x51, 0xb9, 0x38, 0x6c, 0xa8, 0x63, 0xa2, 0x15, 0xb5, 0xf2, 0x0c, - 0x30, 0xff, 0xfd, 0xcf, 0x00, 0x0b, 0x2b, 0xcf, 0x00, 0xff, 0xda, 0x9c, 0xda, 0xbd, 0x91, 0x30, - 0x62, 0x5c, 0xd3, 0x48, 0x56, 0xe1, 0xa1, 0x85, 0xb3, 0x38, 0xf4, 0x18, 0xe9, 0x3e, 0x85, 0x7a, - 0x34, 0xcc, 0xa2, 0x63, 0x90, 0x8a, 0x76, 0x16, 0x38, 0xee, 0x88, 0x5b, 0x0f, 0xe5, 0x6c, 0x7a, - 0x87, 0x56, 0xbf, 0x7b, 0x87, 0xaa, 0xbf, 0x9f, 0x01, 0x26, 0x4c, 0x98, 0xdd, 0xb9, 0xe3, 0x0c, - 0xad, 0x33, 0x62, 0x04, 0x77, 0x61, 0x43, 0x78, 0xd1, 0xa4, 0xb8, 0x07, 0x71, 0x2a, 0xc1, 0x11, - 0xc9, 0xa9, 0xc4, 0xaa, 0x8b, 0x5e, 0xd9, 0x95, 0x17, 0xbd, 0x56, 0x5f, 0x20, 0xbb, 0x05, 0x55, - 0xf9, 0x9a, 0x14, 0x57, 0x25, 0xc0, 0x48, 0x6e, 0x48, 0xfd, 0x87, 0x2c, 0x40, 0x62, 0x66, 0xfd, - 0xaa, 0x4f, 0x92, 0x57, 0x4c, 0x49, 0x6e, 0xd5, 0x94, 0xdc, 0x01, 0x45, 0xa6, 0x93, 0xee, 0xeb, - 0xd5, 0x13, 0x42, 0xea, 0x26, 0xe7, 0x69, 0xd2, 0x9d, 0x2a, 0xe2, 0x69, 0xe2, 0x90, 0x4a, 0xc4, - 0x99, 0x10, 0xcb, 0x15, 0xb2, 0xae, 0x6c, 0x07, 0x9c, 0x05, 0xb3, 0x4f, 0xe0, 0x5a, 0x5c, 0x52, - 0x3f, 0xb5, 0xc3, 0x63, 0x6f, 0x1e, 0x0a, 0xb7, 0x56, 0x20, 0x24, 0xde, 0xd5, 0xa8, 0xa6, 0xa7, - 0x1c, 0xcd, 0x59, 0x56, 0xc0, 0x3e, 0x84, 0xca, 0x64, 0xee, 0x38, 0x7a, 0x68, 0x9d, 0x85, 0x22, - 0xf8, 0xb0, 0x91, 0xb2, 0x50, 0xa5, 0xe9, 0xd5, 0xca, 0x13, 0x91, 0x51, 0xff, 0x77, 0x16, 0x0a, - 0x3f, 0x9e, 0x5b, 0xfe, 0x39, 0xfb, 0x08, 0x2a, 0x41, 0x38, 0x0d, 0xe5, 0x93, 0xa1, 0x6b, 0xbc, - 0x02, 0xc2, 0xd3, 0xc1, 0x8e, 0x35, 0xb5, 0xdc, 0x90, 0x3b, 0x5d, 0x90, 0x96, 0x24, 0xd2, 0x26, - 0x14, 0x82, 0xd0, 0x9a, 0xf1, 0x93, 0xa8, 0x82, 0xc6, 0x33, 0x6c, 0x0b, 0x0a, 0xae, 0x67, 0x5a, - 0x0b, 0x71, 0x21, 0x68, 0x11, 0x6b, 0x1c, 0xc1, 0x54, 0x28, 0xc6, 0x33, 0xbe, 0x74, 0x3a, 0xc3, - 0x31, 0x14, 0xbd, 0x6b, 0x19, 0xa6, 0xed, 0x1e, 0x45, 0xf7, 0x1f, 0xe3, 0x3c, 0xca, 0x5a, 0x52, - 0x4e, 0x8d, 0xa3, 0xe8, 0x32, 0xb2, 0xc8, 0xb2, 0x2d, 0xa8, 0x62, 0xf2, 0xa9, 0x6f, 0x87, 0xd6, - 0xe0, 0x81, 0x18, 0x37, 0x19, 0x84, 0xaa, 0xa5, 0x69, 0x85, 0xd6, 0x38, 0x1c, 0x7c, 0xe3, 0x70, - 0xdd, 0x80, 0x4e, 0xf9, 0x23, 0x88, 0x6a, 0xc2, 0x7a, 0xaa, 0xbb, 0x69, 0x0b, 0x1e, 0x6d, 0x98, - 0x76, 0x17, 0x6d, 0xb3, 0x8c, 0x64, 0x4a, 0x66, 0x65, 0xf3, 0x31, 0x27, 0xd9, 0x95, 0x64, 0x62, - 0x08, 0x43, 0xaf, 0x40, 0x96, 0x5b, 0x5b, 0x7b, 0xd4, 0x56, 0x8a, 0xea, 0x1f, 0x64, 0x61, 0x63, - 0xe8, 0x1b, 0x6e, 0x60, 0xf0, 0x4b, 0x2d, 0x6e, 0xe8, 0x7b, 0x0e, 0xfb, 0x14, 0xca, 0xe1, 0xd8, - 0x91, 0xa7, 0xe1, 0x56, 0xb4, 0xe9, 0x17, 0x48, 0xef, 0x0d, 0xc7, 0xdc, 0x03, 0x56, 0x0a, 0x79, - 0x82, 0xfd, 0x00, 0x0a, 0x23, 0xeb, 0xc8, 0x76, 0x05, 0x03, 0xbe, 0xb2, 0x58, 0x70, 0x1b, 0x91, - 0x7b, 0x6b, 0x1a, 0xa7, 0x62, 0xef, 0x41, 0x71, 0xec, 0x4d, 0x23, 0x49, 0x95, 0xc4, 0xdf, 0x4b, - 0x1f, 0x42, 0xec, 0xde, 0x9a, 0x26, 0xe8, 0xd8, 0x47, 0x50, 0xf6, 0x3d, 0xc7, 0x19, 0x19, 0xe3, - 0x13, 0x21, 0xc3, 0x1a, 0x8b, 0x65, 0x34, 0x81, 0xdf, 0x5b, 0xd3, 0x62, 0x5a, 0xf5, 0x1e, 0x94, - 0x44, 0x63, 0x71, 0x00, 0xb6, 0xdb, 0x8f, 0x3a, 0x62, 0x20, 0x5b, 0xfd, 0xfd, 0xfd, 0xce, 0x90, - 0x5f, 0xf4, 0xd3, 0xfa, 0xdd, 0xee, 0x76, 0xb3, 0xf5, 0x58, 0xc9, 0x6e, 0x97, 0xa1, 0x68, 0x50, - 0x78, 0xb8, 0xfa, 0xdb, 0x19, 0xb8, 0xb4, 0xd0, 0x01, 0xf6, 0x10, 0xf2, 0x53, 0xd4, 0xce, 0xf8, - 0xf0, 0xdc, 0x5e, 0xd9, 0x4b, 0x29, 0xcf, 0x75, 0x36, 0x2c, 0xa1, 0x7e, 0x02, 0xf5, 0x34, 0x5c, - 0xb2, 0x18, 0xd7, 0xa1, 0xa2, 0xb5, 0x9b, 0x3b, 0x7a, 0xbf, 0x87, 0xe6, 0x1d, 0x9a, 0x7b, 0x94, - 0x7d, 0xaa, 0x75, 0x86, 0x6d, 0x25, 0xab, 0xfe, 0x06, 0x28, 0x8b, 0x03, 0xc3, 0x1e, 0xc1, 0x25, - 0x54, 0x3f, 0x1c, 0x8b, 0x0b, 0x8a, 0x64, 0xca, 0x6e, 0xae, 0x18, 0x49, 0x41, 0x46, 0x33, 0x56, - 0x1f, 0xa7, 0xf2, 0xea, 0xff, 0x07, 0x6c, 0x79, 0x04, 0x7f, 0x75, 0xd5, 0xff, 0xaf, 0x0c, 0xe4, - 0x0f, 0x1c, 0xc3, 0x65, 0xaf, 0x41, 0x81, 0x1e, 0xa8, 0x10, 0xdc, 0xb3, 0x2a, 0x6d, 0x70, 0x5c, - 0x16, 0x84, 0x63, 0x6f, 0x43, 0x2e, 0x1c, 0x47, 0x97, 0x1a, 0x5f, 0xba, 0x60, 0xf1, 0xed, 0xad, - 0x69, 0x48, 0xc5, 0xee, 0x40, 0xce, 0x34, 0xa3, 0x58, 0x46, 0x61, 0xd0, 0xa2, 0x15, 0xb4, 0x63, - 0x4d, 0x6c, 0xd7, 0x16, 0x0f, 0x6a, 0x20, 0x09, 0x7b, 0x1d, 0x72, 0xe6, 0xd8, 0x49, 0x07, 0xa6, - 0x72, 0x7b, 0x29, 0xae, 0xd0, 0x1c, 0x3b, 0x4c, 0x85, 0xf5, 0xd0, 0x3f, 0xd7, 0xfd, 0xb9, 0x4b, - 0xf1, 0x0f, 0x81, 0xb0, 0x1b, 0xaa, 0xa8, 0xcc, 0xcc, 0x29, 0x88, 0x22, 0x10, 0x97, 0x23, 0x66, - 0xbe, 0x35, 0x33, 0xfc, 0xd8, 0x62, 0xb0, 0x83, 0x03, 0x0e, 0xd8, 0x2e, 0x02, 0xbd, 0x6f, 0xa7, - 0xbe, 0x43, 0xaf, 0x27, 0xa0, 0x86, 0xad, 0x46, 0xa9, 0x15, 0x77, 0xcf, 0x04, 0x46, 0xfd, 0xf3, - 0x1c, 0x54, 0xa5, 0xf6, 0xb0, 0x0f, 0xa0, 0x6c, 0xa6, 0x37, 0xe2, 0xb5, 0xa5, 0x46, 0xdf, 0xdb, - 0x89, 0xb6, 0xa0, 0x29, 0x96, 0xf7, 0x27, 0xb0, 0x1e, 0x58, 0xa1, 0xfe, 0xcc, 0xf0, 0x6d, 0xfe, - 0xb2, 0x4c, 0x56, 0x3e, 0x23, 0x1a, 0x58, 0xe1, 0x93, 0x08, 0xb3, 0xb7, 0xa6, 0xd5, 0x02, 0x29, - 0x4f, 0x66, 0x80, 0xe8, 0x52, 0x2e, 0xf5, 0xfa, 0x0e, 0x07, 0xee, 0xad, 0x69, 0x11, 0x1e, 0x49, - 0xad, 0x33, 0x6b, 0x3c, 0x0f, 0x23, 0x33, 0x60, 0x3d, 0xea, 0x10, 0x01, 0xe9, 0xa1, 0x2f, 0x9e, - 0x64, 0xf7, 0x91, 0xd7, 0x19, 0x8e, 0xe3, 0x91, 0xce, 0x56, 0x90, 0x7d, 0x97, 0x3b, 0x31, 0x9c, - 0x3f, 0x2c, 0x16, 0xe5, 0xd8, 0x1b, 0x50, 0xf0, 0xc2, 0x63, 0x2b, 0x52, 0x9e, 0xa3, 0x77, 0x18, - 0x10, 0xb4, 0xd3, 0xea, 0xe2, 0x4a, 0x21, 0xb4, 0xfa, 0xb3, 0x0c, 0x94, 0xc4, 0x08, 0xb0, 0x0d, - 0x58, 0x1f, 0xb4, 0x87, 0xfa, 0x93, 0xa6, 0xd6, 0x69, 0x6e, 0x77, 0xdb, 0x22, 0x9e, 0xf6, 0x91, - 0xd6, 0xec, 0x09, 0x3e, 0xa9, 0xb5, 0x9f, 0xf4, 0x1f, 0xb7, 0xb9, 0x1b, 0x66, 0xa7, 0xdd, 0xfb, - 0x4a, 0xc9, 0x71, 0x0f, 0x65, 0xfb, 0xa0, 0xa9, 0x21, 0x97, 0xac, 0x42, 0xa9, 0xfd, 0x65, 0xbb, - 0x75, 0x48, 0x6c, 0xb2, 0x0e, 0xb0, 0xd3, 0x6e, 0x76, 0xbb, 0xfd, 0x16, 0xb2, 0xcd, 0x22, 0x63, - 0x50, 0x6f, 0x69, 0xed, 0xe6, 0xb0, 0xad, 0x37, 0x5b, 0xad, 0xfe, 0x61, 0x6f, 0xa8, 0x94, 0xf0, - 0x8b, 0xcd, 0xee, 0xb0, 0xad, 0xc5, 0x20, 0x7a, 0x1b, 0x67, 0x47, 0xeb, 0x1f, 0xc4, 0x90, 0xca, - 0x76, 0x05, 0x4d, 0x32, 0x9a, 0x2b, 0xf5, 0xbf, 0xd5, 0xa1, 0x9e, 0x5e, 0x9a, 0xec, 0x63, 0x28, - 0x9b, 0x66, 0x6a, 0x8e, 0x6f, 0xac, 0x5a, 0xc2, 0xf7, 0x76, 0xcc, 0x68, 0x9a, 0x79, 0x82, 0xbd, - 0x1a, 0x6d, 0xa4, 0xec, 0xd2, 0x46, 0x8a, 0xb6, 0xd1, 0x67, 0x70, 0x49, 0xbc, 0x63, 0x60, 0x1a, - 0xa1, 0x31, 0x32, 0x02, 0x2b, 0xbd, 0x4b, 0x5a, 0x84, 0xdc, 0x11, 0xb8, 0xbd, 0x35, 0xad, 0x3e, - 0x4e, 0x41, 0xd8, 0x0f, 0xa1, 0x6e, 0x90, 0x21, 0x1d, 0x97, 0xcf, 0xcb, 0x4a, 0x60, 0x13, 0x71, - 0x52, 0xf1, 0x75, 0x43, 0x06, 0xe0, 0x42, 0x34, 0x7d, 0x6f, 0x96, 0x14, 0x2e, 0xc8, 0x0b, 0x71, - 0xc7, 0xf7, 0x66, 0x52, 0xd9, 0x9a, 0x29, 0xe5, 0xd9, 0x47, 0x50, 0x13, 0x2d, 0x4f, 0x4c, 0xf2, - 0x78, 0xcb, 0xf2, 0x66, 0x93, 0x52, 0xb7, 0xb7, 0xa6, 0x55, 0xc7, 0x49, 0x96, 0x3d, 0x40, 0x4d, - 0x2e, 0xd1, 0xc5, 0x4b, 0xf2, 0x5a, 0xa3, 0xd6, 0x46, 0xa5, 0xc0, 0x88, 0x73, 0xec, 0x3d, 0x00, - 0x6a, 0x27, 0x2f, 0x53, 0x4e, 0x9d, 0x99, 0xfb, 0xde, 0x2c, 0x2a, 0x52, 0x31, 0xa3, 0x8c, 0xd4, - 0x3c, 0x7e, 0x0d, 0xac, 0xb2, 0xdc, 0x3c, 0xba, 0x9c, 0x94, 0x34, 0x8f, 0x5f, 0xfb, 0x8a, 0x9b, - 0xc7, 0x8b, 0xc1, 0x52, 0xf3, 0xa2, 0x52, 0xbc, 0x79, 0xbc, 0x50, 0xd4, 0x3c, 0x5e, 0xa6, 0xba, - 0xd8, 0xbc, 0xa8, 0x08, 0x35, 0x8f, 0x97, 0xf8, 0xe1, 0x92, 0xee, 0x5e, 0xbb, 0x50, 0x77, 0xc7, - 0x69, 0x4b, 0x6b, 0xef, 0x3f, 0x84, 0x7a, 0x70, 0xec, 0x9d, 0x4a, 0x0c, 0x64, 0x5d, 0x2e, 0x3d, - 0x38, 0xf6, 0x4e, 0x65, 0x0e, 0xb2, 0x1e, 0xc8, 0x00, 0x6c, 0x2d, 0xef, 0x22, 0x5d, 0xf4, 0xac, - 0xcb, 0xad, 0xa5, 0x1e, 0x3e, 0xb1, 0xad, 0x53, 0x6c, 0xad, 0x11, 0x65, 0x70, 0x50, 0x12, 0xe7, - 0x4b, 0x20, 0x82, 0x3c, 0x52, 0xe7, 0xbf, 0xe2, 0x4b, 0x10, 0x7b, 0x64, 0x02, 0x5c, 0x5b, 0x73, - 0x57, 0x2e, 0xa6, 0xc8, 0x6b, 0xeb, 0xd0, 0x4d, 0x15, 0xac, 0x71, 0x52, 0x51, 0x34, 0xd9, 0x15, - 0x81, 0xf5, 0xcd, 0xdc, 0x72, 0xc7, 0x96, 0x08, 0x07, 0x49, 0xed, 0x8a, 0x81, 0xc0, 0x25, 0xbb, - 0x22, 0x82, 0xc4, 0xeb, 0x3a, 0x2e, 0xce, 0x16, 0xd7, 0xb5, 0x54, 0x98, 0xd6, 0x75, 0x5c, 0x34, - 0xde, 0x50, 0x71, 0xd9, 0xcb, 0x4b, 0x1b, 0x4a, 0x2a, 0xcc, 0x37, 0x54, 0x5c, 0xfa, 0x01, 0x88, - 0xd5, 0xc4, 0x07, 0x37, 0x15, 0x34, 0xc2, 0x5b, 0x2d, 0x46, 0x17, 0xc6, 0x71, 0x0e, 0xd7, 0xaa, - 0x6f, 0xa1, 0xad, 0x20, 0x96, 0xc2, 0x15, 0x79, 0xad, 0x6a, 0x84, 0x89, 0xb7, 0x92, 0x9f, 0x64, - 0xd5, 0x3f, 0x2a, 0x40, 0x49, 0x30, 0x1d, 0x76, 0x19, 0x2e, 0x09, 0xde, 0xb7, 0xd3, 0x1c, 0x36, - 0xb7, 0x9b, 0x03, 0xd4, 0x56, 0x18, 0xd4, 0x39, 0xf3, 0x8b, 0x61, 0x19, 0x64, 0x88, 0xc4, 0xfd, - 0x62, 0x50, 0x16, 0x19, 0xa2, 0x28, 0xcb, 0x1f, 0x16, 0xcb, 0xb1, 0x4b, 0x50, 0xe5, 0x05, 0x39, - 0x80, 0xee, 0xb9, 0x50, 0x29, 0x9e, 0x2f, 0x48, 0x45, 0xb8, 0x4f, 0xbe, 0x98, 0x14, 0xe1, 0x80, - 0x52, 0x5c, 0x24, 0x72, 0xda, 0x33, 0xa8, 0x0f, 0xb5, 0xc3, 0x5e, 0x2b, 0xf9, 0x4e, 0x85, 0xee, - 0x26, 0xf0, 0x6a, 0x9e, 0x74, 0xda, 0x4f, 0x15, 0xc0, 0x42, 0xbc, 0x16, 0xca, 0x57, 0x51, 0xdf, - 0xa2, 0x4a, 0x28, 0x5b, 0x63, 0x2f, 0xc1, 0xe5, 0xc1, 0x5e, 0xff, 0xa9, 0xce, 0x0b, 0xc5, 0x5d, - 0x58, 0x67, 0x9b, 0xa0, 0x48, 0x08, 0x5e, 0x7d, 0x1d, 0x3f, 0x49, 0xd0, 0x88, 0x70, 0xa0, 0x5c, - 0xa2, 0x13, 0x30, 0x84, 0x0d, 0xb9, 0x00, 0x52, 0xb0, 0x2b, 0xbc, 0x68, 0xbf, 0x7b, 0xb8, 0xdf, - 0x1b, 0x28, 0x1b, 0xd8, 0x08, 0x82, 0xf0, 0x96, 0xb3, 0xb8, 0x9a, 0x44, 0x6c, 0x5d, 0x26, 0x49, - 0x86, 0xb0, 0xa7, 0x4d, 0xad, 0xd7, 0xe9, 0x3d, 0x1a, 0x28, 0x9b, 0x71, 0xcd, 0x6d, 0x4d, 0xeb, - 0x6b, 0x03, 0xe5, 0x4a, 0x0c, 0x18, 0x0c, 0x9b, 0xc3, 0xc3, 0x81, 0x72, 0x35, 0x6e, 0xe5, 0x81, - 0xd6, 0x6f, 0xb5, 0x07, 0x83, 0x6e, 0x67, 0x30, 0x54, 0x5e, 0x62, 0x57, 0x60, 0x23, 0x69, 0x51, - 0x44, 0xdc, 0x90, 0x1a, 0xaa, 0x3d, 0x6a, 0x0f, 0x95, 0x6b, 0x71, 0x33, 0x5a, 0xfd, 0x6e, 0xb7, - 0x49, 0xe7, 0x33, 0xd7, 0x91, 0x88, 0x8e, 0x84, 0x44, 0x6f, 0x5e, 0xc6, 0x76, 0x1d, 0xf6, 0x64, - 0xd0, 0x0d, 0x69, 0x69, 0x0c, 0xda, 0x3f, 0x3e, 0x6c, 0xf7, 0x5a, 0x6d, 0xe5, 0x95, 0x64, 0x69, - 0xc4, 0xb0, 0x9b, 0xf1, 0xd2, 0x88, 0x41, 0xb7, 0xe2, 0x6f, 0x46, 0xa0, 0x81, 0xb2, 0x85, 0xf5, - 0x89, 0x76, 0xf4, 0x7a, 0xed, 0xd6, 0x10, 0xfb, 0xfa, 0x6a, 0x3c, 0x8a, 0x87, 0x07, 0x8f, 0xb4, - 0xe6, 0x4e, 0x5b, 0x51, 0x11, 0xa2, 0xb5, 0x7b, 0xcd, 0xfd, 0x68, 0xb6, 0x5f, 0xdb, 0xae, 0xd1, - 0x63, 0xa5, 0x42, 0x5c, 0xaa, 0x5f, 0x00, 0x93, 0x5f, 0xfd, 0x13, 0x0f, 0xfb, 0x30, 0xc8, 0x4f, - 0x7c, 0x6f, 0x1a, 0xdd, 0xf4, 0xc4, 0x34, 0xda, 0x6a, 0xb3, 0xf9, 0x88, 0xa2, 0x54, 0x92, 0x8b, - 0x5f, 0x32, 0x48, 0xfd, 0xa3, 0x0c, 0xd4, 0xd3, 0xa2, 0x12, 0x55, 0x44, 0x7b, 0xa2, 0xbb, 0x5e, - 0xc8, 0x5f, 0x8c, 0x09, 0x22, 0x4f, 0x94, 0x3d, 0xe9, 0x79, 0x21, 0x3d, 0x19, 0x43, 0xa6, 0x63, - 0x2c, 0xf9, 0x78, 0xad, 0x71, 0x9e, 0x75, 0xe0, 0x72, 0xea, 0x51, 0xc4, 0xd4, 0x7b, 0x3d, 0x8d, - 0xf8, 0x29, 0xb7, 0x85, 0xf6, 0x6b, 0x2c, 0x58, 0xee, 0x93, 0x02, 0xb9, 0xe0, 0x1b, 0x47, 0x38, - 0x02, 0x30, 0xa9, 0xee, 0xc1, 0x7a, 0x4a, 0x32, 0x93, 0xc5, 0x3f, 0x49, 0xb7, 0xb4, 0x6c, 0x4f, - 0x9e, 0xdf, 0x4c, 0xf5, 0x0f, 0x33, 0x50, 0x93, 0xe5, 0xf4, 0x2f, 0x5c, 0x13, 0x85, 0xb5, 0x8b, - 0xb4, 0x6e, 0x9b, 0xd1, 0x4b, 0x31, 0x11, 0xa8, 0x43, 0x8f, 0x34, 0x73, 0x1f, 0xec, 0xee, 0xc9, - 0x20, 0xee, 0x8e, 0x0c, 0x42, 0x93, 0x99, 0x2e, 0xfe, 0xec, 0x3e, 0x46, 0x02, 0x11, 0x18, 0x9f, - 0x40, 0xd4, 0x5b, 0x50, 0xd9, 0x3d, 0x89, 0x1e, 0x2d, 0x92, 0xdf, 0x4d, 0xaa, 0xf0, 0xdb, 0x82, - 0xea, 0x9f, 0x64, 0xa0, 0x9e, 0xdc, 0x85, 0xa7, 0x28, 0x35, 0xfe, 0x98, 0x26, 0x5f, 0x0e, 0x59, - 0x73, 0x94, 0xbc, 0xdf, 0x9c, 0x95, 0xdf, 0x6f, 0x7e, 0x4d, 0x54, 0x96, 0x93, 0xa5, 0x59, 0xfc, - 0x2d, 0x71, 0x17, 0xf1, 0x01, 0xd4, 0xf0, 0xbf, 0x66, 0x4d, 0x2c, 0xdf, 0xb7, 0xa2, 0x77, 0x45, - 0x97, 0x88, 0x53, 0x44, 0x64, 0x91, 0x58, 0x13, 0xa1, 0x18, 0xad, 0xbc, 0xae, 0x8f, 0x78, 0xf5, - 0x9f, 0xe6, 0xa1, 0x2a, 0x69, 0x3d, 0xdf, 0x6b, 0xf9, 0xdd, 0x80, 0x4a, 0x72, 0x11, 0x5c, 0x5c, - 0x00, 0x8b, 0x01, 0xa9, 0xb9, 0xca, 0x2d, 0xcc, 0x55, 0x03, 0x4a, 0x22, 0x9c, 0x4d, 0xf8, 0x3d, - 0xa3, 0x6c, 0xda, 0xb1, 0x57, 0x78, 0x8e, 0xeb, 0xfd, 0x7d, 0xa8, 0x49, 0x5e, 0xb9, 0xe8, 0x55, - 0x89, 0x45, 0xfa, 0x6a, 0xe2, 0xa1, 0x0b, 0xd8, 0x15, 0x28, 0x4e, 0x4e, 0x74, 0x73, 0x14, 0xb9, - 0x39, 0x0b, 0x93, 0x93, 0x9d, 0x11, 0x1d, 0x5d, 0x4c, 0x62, 0x41, 0xcf, 0x7d, 0x25, 0xe5, 0x49, - 0x24, 0xce, 0xef, 0x40, 0x69, 0x72, 0xc2, 0x2f, 0xf5, 0x54, 0xe4, 0xd8, 0x8e, 0x64, 0xc8, 0x8b, - 0x93, 0x13, 0xba, 0xd5, 0xf3, 0x09, 0x28, 0x0b, 0x3e, 0xd5, 0x40, 0x1c, 0xba, 0x2d, 0x36, 0xea, - 0x52, 0xda, 0xbd, 0x1a, 0xb0, 0x77, 0x61, 0x53, 0x48, 0x5e, 0x23, 0xd0, 0x79, 0x80, 0x35, 0xbd, - 0x2d, 0xc0, 0x1f, 0x60, 0xda, 0xe0, 0xb8, 0x66, 0x30, 0x20, 0x0c, 0x2e, 0x56, 0x15, 0x6a, 0xd2, - 0xda, 0xe5, 0x0f, 0x37, 0x54, 0xb4, 0x14, 0x8c, 0x3d, 0x84, 0xda, 0xe4, 0x84, 0xaf, 0x85, 0xa1, - 0xb7, 0x6f, 0x89, 0xa0, 0xd9, 0xcd, 0xc5, 0x55, 0x40, 0xb1, 0x95, 0x29, 0x4a, 0xf6, 0x03, 0x60, - 0xbe, 0x15, 0x5a, 0x2e, 0xf5, 0xc4, 0xb4, 0x0c, 0xd3, 0xb1, 0x5d, 0x8b, 0x94, 0xad, 0x9c, 0xb6, - 0x11, 0x63, 0x76, 0x04, 0x42, 0xfd, 0x67, 0x19, 0xa8, 0x27, 0xda, 0x2f, 0x6e, 0x68, 0x76, 0x57, - 0x7e, 0x51, 0xb7, 0xb1, 0xa8, 0x20, 0x23, 0xc9, 0xbd, 0xe1, 0xf9, 0x8c, 0xbf, 0xba, 0xb7, 0xea, - 0xf5, 0x8d, 0x55, 0x2e, 0xd7, 0xdc, 0x2a, 0x97, 0xab, 0xfa, 0x08, 0x72, 0xc3, 0xf3, 0x19, 0xf7, - 0xb4, 0xa0, 0x0c, 0xe4, 0x56, 0x19, 0x97, 0x7e, 0x74, 0x86, 0xfe, 0xb8, 0xfd, 0x15, 0xbf, 0xfc, - 0x7a, 0xa0, 0x75, 0xf6, 0x9b, 0xda, 0x57, 0x14, 0xe7, 0x40, 0x5a, 0xc2, 0x6e, 0x5f, 0x6b, 0x77, - 0x1e, 0xf5, 0x08, 0x90, 0x27, 0x3f, 0x4c, 0xd2, 0xc4, 0xa6, 0x69, 0xee, 0x9e, 0xc8, 0x8f, 0x10, - 0x64, 0x52, 0x8f, 0x10, 0xa4, 0xef, 0xcb, 0x65, 0x17, 0xef, 0xcb, 0xb1, 0x78, 0x47, 0xc7, 0xec, - 0x81, 0xbd, 0x09, 0xf9, 0xc9, 0x89, 0x75, 0x9e, 0x36, 0x71, 0xd2, 0x9b, 0x91, 0x08, 0xd4, 0x9f, - 0x67, 0x80, 0xa5, 0x1a, 0xc2, 0xb5, 0xee, 0x5f, 0xb4, 0x2d, 0x1f, 0x43, 0x43, 0x3c, 0x51, 0xc7, - 0xa9, 0x24, 0x1f, 0xaf, 0x18, 0xd2, 0x2b, 0x5e, 0x12, 0x8a, 0x95, 0x3c, 0x10, 0xc2, 0xde, 0x05, - 0xfe, 0x46, 0x18, 0x2e, 0x90, 0xb4, 0x53, 0x43, 0xe2, 0x15, 0x5a, 0x42, 0x93, 0x3c, 0x0a, 0x26, - 0x3f, 0x76, 0xc6, 0xdd, 0xc3, 0x97, 0x92, 0x59, 0x23, 0xfe, 0xa1, 0xfe, 0x6e, 0x06, 0x2e, 0xa7, - 0x17, 0xc4, 0x2f, 0xd7, 0xcb, 0xf4, 0xcb, 0x6e, 0xb9, 0xc5, 0x97, 0xdd, 0x56, 0xad, 0xa7, 0xfc, - 0xca, 0xf5, 0xf4, 0xb7, 0x32, 0xb0, 0x29, 0x8d, 0x7e, 0x62, 0x27, 0xfd, 0x15, 0xb5, 0x4c, 0x7a, - 0xe0, 0x2d, 0x9f, 0x7a, 0xe0, 0x4d, 0xfd, 0x83, 0x0c, 0x5c, 0x5d, 0x68, 0x89, 0x66, 0xfd, 0x95, - 0xb6, 0x25, 0xfd, 0x10, 0x1c, 0xb9, 0xa8, 0x79, 0x54, 0x1b, 0xbf, 0xcb, 0xc4, 0xd2, 0x2f, 0xbb, - 0x75, 0x71, 0x1e, 0xff, 0x79, 0xba, 0x91, 0x66, 0x72, 0x2d, 0x84, 0x7d, 0x08, 0xd5, 0x44, 0x63, - 0x8a, 0xee, 0xd9, 0xaf, 0xbc, 0x53, 0x22, 0xd3, 0xad, 0x64, 0xa3, 0xd9, 0xef, 0xc7, 0x46, 0x1f, - 0x42, 0x2d, 0xae, 0x78, 0xc7, 0x9a, 0xa4, 0xbd, 0x11, 0x0b, 0x2f, 0xc5, 0xa4, 0x28, 0xd5, 0x0f, - 0x60, 0x23, 0xe9, 0x45, 0x4b, 0xbc, 0x6e, 0x74, 0x0b, 0xaa, 0xae, 0x75, 0xaa, 0x47, 0x6f, 0x1f, - 0x89, 0x70, 0x14, 0xd7, 0x3a, 0x15, 0x04, 0xea, 0xae, 0xcc, 0xf7, 0xe2, 0x87, 0xaa, 0x1d, 0x53, - 0x9e, 0x99, 0x92, 0xe7, 0x98, 0x11, 0x0a, 0x6b, 0x93, 0x26, 0xa6, 0xe4, 0x5a, 0xa7, 0xb4, 0xe6, - 0x4e, 0x45, 0x3d, 0x4d, 0xd3, 0x14, 0x07, 0xe6, 0xab, 0xde, 0x0c, 0xb9, 0x06, 0xe5, 0x99, 0x9f, - 0x9a, 0xd9, 0xd2, 0xcc, 0xe7, 0x9f, 0xbd, 0x2d, 0x82, 0x5d, 0x2e, 0x3a, 0x5c, 0xe7, 0xe1, 0x2f, - 0xe2, 0x89, 0x85, 0x7c, 0xf2, 0x90, 0xfd, 0x87, 0x82, 0xe5, 0xe1, 0xfe, 0x13, 0x5f, 0x8e, 0x0f, - 0xd1, 0x33, 0x77, 0xd6, 0xe9, 0x10, 0x9d, 0x34, 0x40, 0xeb, 0x1b, 0x11, 0x6f, 0x83, 0x49, 0x75, - 0x1b, 0xaa, 0x92, 0x65, 0x87, 0xaa, 0x89, 0xe4, 0x15, 0x09, 0xd2, 0xaf, 0x30, 0x24, 0x03, 0xa4, - 0x55, 0x13, 0xa7, 0x48, 0xa0, 0xfe, 0x1e, 0x00, 0x24, 0xb8, 0x94, 0xc2, 0x90, 0x59, 0x50, 0x18, - 0x5e, 0xe8, 0x44, 0xfe, 0x03, 0xa8, 0x8f, 0xbd, 0xd9, 0xb9, 0x9e, 0x94, 0xc8, 0xad, 0x2c, 0x51, - 0x43, 0xaa, 0x61, 0x72, 0x21, 0x63, 0xf9, 0xa4, 0x35, 0xbf, 0xf2, 0xa4, 0xf5, 0x7d, 0x28, 0x71, - 0xc7, 0x7d, 0x20, 0x2e, 0xf4, 0xbc, 0xb4, 0xd8, 0xcf, 0x7b, 0xe2, 0x2d, 0xc0, 0x88, 0x8e, 0xb5, - 0xd1, 0x2a, 0x17, 0x0f, 0xa1, 0xc9, 0xd7, 0x7b, 0x6e, 0x2e, 0x97, 0x8c, 0xc8, 0xf8, 0xeb, 0x3b, - 0x86, 0x9c, 0x95, 0x94, 0x84, 0x70, 0x2a, 0xbc, 0x49, 0xa4, 0x24, 0x94, 0x64, 0x25, 0x61, 0x38, - 0xe5, 0x3e, 0x24, 0x54, 0x12, 0x7e, 0x00, 0x97, 0x45, 0xd0, 0x34, 0x16, 0xc0, 0xe1, 0x24, 0x7a, - 0x1e, 0x49, 0x24, 0xae, 0xc6, 0x0f, 0xa7, 0xa4, 0x7d, 0x23, 0xf9, 0x97, 0xb0, 0x39, 0x3e, 0x36, - 0xdc, 0x23, 0x4b, 0x0f, 0x47, 0x8e, 0x4e, 0xef, 0xf0, 0xea, 0x53, 0x63, 0x26, 0xd4, 0x9e, 0x37, - 0x97, 0x1a, 0xdb, 0x22, 0xe2, 0xe1, 0xc8, 0xa1, 0x50, 0x97, 0xf8, 0x3c, 0x7e, 0x63, 0xbc, 0x08, - 0x5f, 0x38, 0x8d, 0x82, 0xc5, 0xd3, 0xa8, 0x25, 0x6d, 0xa6, 0xba, 0xac, 0xcd, 0x5c, 0xff, 0xf7, - 0x79, 0x28, 0x8a, 0xd7, 0x14, 0xef, 0x42, 0xde, 0xf4, 0xbd, 0x59, 0x1c, 0x8d, 0xb6, 0x42, 0xbb, - 0xa0, 0x1f, 0xed, 0x40, 0x45, 0xe4, 0x1e, 0x14, 0x0d, 0xd3, 0xd4, 0x27, 0x27, 0xe9, 0x13, 0xa3, - 0x05, 0x41, 0xbf, 0xb7, 0xa6, 0x15, 0x0c, 0x92, 0xf8, 0x1f, 0x43, 0x05, 0xe9, 0xb9, 0x33, 0x2c, - 0x65, 0x2f, 0x2d, 0x8b, 0xe4, 0xbd, 0x35, 0xad, 0x6c, 0x44, 0xe2, 0xf9, 0x47, 0x69, 0xdf, 0x1b, - 0x97, 0x97, 0xd7, 0x97, 0x8a, 0x5e, 0xe4, 0x85, 0xfb, 0x75, 0xe0, 0xce, 0x98, 0x98, 0xdb, 0x14, - 0xe4, 0xc3, 0x89, 0x25, 0xde, 0xb4, 0xb7, 0xa6, 0xf1, 0x3d, 0x17, 0xf1, 0xaa, 0x0f, 0x23, 0xbf, - 0x58, 0xfc, 0xbc, 0xfe, 0x8a, 0x91, 0x41, 0x5e, 0x11, 0x3b, 0xc7, 0x88, 0x71, 0x60, 0x31, 0xd3, - 0x8c, 0xc2, 0x76, 0x4a, 0x4b, 0xc5, 0x62, 0x8e, 0x44, 0xc5, 0x62, 0xf6, 0xf4, 0x10, 0xaa, 0xe4, - 0xa2, 0x12, 0xe5, 0xca, 0x4b, 0x43, 0x9b, 0x30, 0x14, 0x72, 0xbc, 0x27, 0xec, 0xa5, 0x15, 0xf5, - 0xd3, 0xb7, 0x64, 0xdf, 0xe6, 0x8d, 0x95, 0x03, 0xa5, 0xc5, 0x6e, 0x4e, 0xde, 0x59, 0x8d, 0x97, - 0x61, 0xdb, 0x50, 0x33, 0x24, 0x49, 0x23, 0x1c, 0x9d, 0x37, 0x56, 0xcc, 0x53, 0x4c, 0x43, 0x75, - 0x48, 0xf9, 0xe4, 0x00, 0xee, 0xba, 0x06, 0x57, 0x57, 0x2f, 0x65, 0x39, 0x92, 0x24, 0xcf, 0x23, - 0x49, 0xd4, 0xf4, 0xf3, 0x04, 0xe9, 0x5b, 0x89, 0x52, 0x5c, 0xc9, 0xe7, 0x68, 0x23, 0xcb, 0x9b, - 0xb7, 0x0a, 0xa5, 0xe8, 0x51, 0x4f, 0x0a, 0xdd, 0x6c, 0xf5, 0x0f, 0xbe, 0x52, 0x32, 0x08, 0xee, - 0xf4, 0x06, 0xc3, 0x66, 0x4f, 0x1c, 0xaf, 0x76, 0x7a, 0xe2, 0x78, 0x55, 0xfd, 0x57, 0x39, 0xa8, - 0xc4, 0xee, 0xe1, 0x5f, 0xdc, 0x30, 0x8e, 0x2d, 0xce, 0x9c, 0x6c, 0x71, 0x2e, 0x68, 0x6a, 0x3c, - 0xf4, 0x83, 0x3f, 0x5b, 0x71, 0x29, 0xad, 0x0f, 0x05, 0xcb, 0x17, 0xa6, 0x0a, 0xdf, 0xf3, 0xc2, - 0x94, 0x1c, 0xe2, 0x57, 0x4c, 0x87, 0xf8, 0x2d, 0x3c, 0xec, 0x5a, 0xa2, 0x30, 0x15, 0xf9, 0x61, - 0xd7, 0x0b, 0xe3, 0x53, 0xca, 0x17, 0xc7, 0xa7, 0xd0, 0x2f, 0x13, 0x3d, 0xb1, 0xad, 0x53, 0x11, - 0xe9, 0x26, 0x72, 0x69, 0xf1, 0x01, 0xcf, 0x11, 0x1f, 0xdf, 0x83, 0x15, 0xb1, 0xfb, 0xb0, 0x39, - 0x39, 0x89, 0x1f, 0xb1, 0x4b, 0x0c, 0xac, 0x1a, 0x75, 0x63, 0x25, 0x4e, 0xfd, 0x3b, 0x19, 0x80, - 0xc4, 0x87, 0xfa, 0x4b, 0x3b, 0x78, 0x24, 0x1b, 0x3a, 0xf7, 0x1d, 0x36, 0xf4, 0x73, 0x5e, 0x03, - 0x50, 0xbf, 0x81, 0x4a, 0xec, 0x35, 0xff, 0xc5, 0xd7, 0xd8, 0x0b, 0x7d, 0xf2, 0x37, 0x23, 0x67, - 0x57, 0xec, 0x76, 0xfe, 0x65, 0xc7, 0x22, 0xf5, 0xf9, 0xdc, 0x73, 0x3e, 0x7f, 0xc6, 0x3d, 0x4e, - 0xf1, 0xc7, 0x7f, 0xc5, 0x1b, 0x4b, 0x5e, 0xf3, 0xf9, 0xd4, 0x9a, 0x57, 0xe7, 0xc2, 0x6d, 0xf6, - 0xcb, 0x7f, 0xfa, 0x85, 0x3a, 0xfc, 0x17, 0x99, 0xc8, 0xb7, 0x13, 0x3f, 0x0d, 0x78, 0xa1, 0xa2, - 0xb5, 0xda, 0x3d, 0xf5, 0x22, 0x9f, 0xfb, 0x4e, 0x6b, 0x33, 0xff, 0x5d, 0xd6, 0xe6, 0x9b, 0x50, - 0xe0, 0x02, 0xa1, 0x70, 0x91, 0xa5, 0xc9, 0xf1, 0xcf, 0x7d, 0x4c, 0x5b, 0x55, 0x85, 0x62, 0xc9, - 0xfb, 0xbb, 0x19, 0xd5, 0x1b, 0x3d, 0x04, 0x8e, 0x19, 0x34, 0xf6, 0x2b, 0x89, 0xd1, 0xf9, 0xe2, - 0x63, 0xf2, 0x2b, 0x33, 0x37, 0xff, 0x51, 0x16, 0xd6, 0x53, 0x07, 0x66, 0xbf, 0x40, 0x63, 0x56, - 0x72, 0xf3, 0xdc, 0x6a, 0x6e, 0x7e, 0x21, 0x63, 0xcd, 0x5f, 0xcc, 0x58, 0xff, 0xaf, 0x48, 0x80, - 0x54, 0x8c, 0x59, 0x39, 0x1d, 0x63, 0x86, 0xdc, 0xb4, 0x96, 0xd2, 0xca, 0x57, 0xe9, 0xef, 0x99, - 0x95, 0xfa, 0xfb, 0xcd, 0xf8, 0x77, 0x78, 0x3a, 0x3b, 0xdc, 0xb0, 0x5c, 0xd7, 0x24, 0x08, 0xfb, - 0x04, 0xae, 0x71, 0xad, 0x86, 0x2b, 0x72, 0xba, 0x37, 0xd1, 0xe3, 0x5f, 0xe9, 0x11, 0x71, 0x73, - 0x57, 0x39, 0x01, 0x7f, 0x69, 0x7d, 0xd2, 0x8c, 0xb0, 0x6a, 0x07, 0xd6, 0x53, 0xa7, 0x97, 0xd2, - 0x2f, 0x7e, 0x65, 0xe4, 0x5f, 0xfc, 0x62, 0x5b, 0x50, 0x38, 0x3d, 0xb6, 0x7c, 0x6b, 0xc5, 0xcb, - 0x68, 0x1c, 0xa1, 0xfe, 0x10, 0x6a, 0x72, 0x24, 0x05, 0x7b, 0x07, 0x0a, 0x76, 0x68, 0x4d, 0x23, - 0xdb, 0xea, 0xea, 0x72, 0xb0, 0x05, 0x19, 0xd2, 0x9c, 0x48, 0xfd, 0x59, 0x06, 0x94, 0x45, 0x9c, - 0xf4, 0xb3, 0x64, 0x99, 0x0b, 0x7e, 0x96, 0x2c, 0x9b, 0x6a, 0xe4, 0xaa, 0x5f, 0x16, 0x8b, 0x5f, - 0x67, 0xca, 0x5f, 0xf0, 0x3a, 0x13, 0x7b, 0x03, 0xca, 0xbe, 0x45, 0xbf, 0xf9, 0x64, 0xae, 0x88, - 0x65, 0x8e, 0x71, 0xea, 0xef, 0x64, 0xa0, 0x24, 0xc2, 0x3e, 0x56, 0x1a, 0xbb, 0x6f, 0x41, 0x89, - 0xff, 0xfe, 0x53, 0x64, 0xfc, 0x2f, 0xc5, 0x41, 0x46, 0x78, 0x76, 0x93, 0x07, 0xc3, 0xa4, 0x8d, - 0xdf, 0x03, 0xc7, 0x70, 0x35, 0x82, 0x8b, 0x1f, 0x50, 0x30, 0xb8, 0xe9, 0x15, 0x88, 0x67, 0x1a, - 0x80, 0x40, 0xa8, 0x9a, 0x05, 0xea, 0x8f, 0xa0, 0x24, 0xc2, 0x4a, 0x56, 0x36, 0xe5, 0x79, 0xbf, - 0x7c, 0xb4, 0x05, 0x90, 0xc4, 0x99, 0xac, 0xaa, 0x41, 0xbd, 0x0b, 0xe5, 0x28, 0xb4, 0x04, 0xd7, - 0x5f, 0xf2, 0x69, 0x11, 0xab, 0x2e, 0x37, 0xc6, 0x11, 0xcf, 0x87, 0x76, 0xbd, 0xf1, 0x09, 0x79, - 0xd5, 0xde, 0x85, 0xb2, 0x13, 0x3d, 0xd5, 0x9a, 0xb9, 0xf8, 0xa9, 0xd6, 0x98, 0x88, 0xdd, 0x85, - 0x98, 0x1d, 0x3f, 0xcf, 0x5a, 0x56, 0x9b, 0xd1, 0xa5, 0x0c, 0x5a, 0x65, 0x0f, 0x84, 0xf7, 0xa8, - 0x4b, 0xaf, 0x9f, 0xa4, 0x1c, 0x36, 0xa9, 0x36, 0x69, 0x12, 0x99, 0x5a, 0x87, 0x9a, 0x7c, 0x1e, - 0xae, 0x36, 0x61, 0x63, 0xdf, 0x0a, 0x0d, 0xe4, 0x59, 0x83, 0xb1, 0xe1, 0x22, 0x3d, 0x5f, 0xbf, - 0x98, 0x48, 0xaf, 0xdf, 0x45, 0x3a, 0x8d, 0x13, 0xa9, 0x3f, 0xcb, 0x83, 0xb2, 0x88, 0x43, 0x66, - 0x12, 0xff, 0x8c, 0x44, 0x26, 0x7a, 0x86, 0xda, 0x89, 0x7f, 0x45, 0x84, 0xd6, 0x45, 0xea, 0x27, - 0x32, 0x38, 0x48, 0x0a, 0x58, 0x4d, 0xbd, 0xe7, 0x5c, 0xb6, 0x83, 0x3d, 0xfe, 0xa2, 0xf3, 0x4b, - 0xfc, 0x31, 0x06, 0xc7, 0x1b, 0xd3, 0xb2, 0xae, 0xd1, 0xdb, 0x0b, 0x5d, 0x6f, 0x8c, 0xa5, 0x22, - 0x83, 0x9b, 0x07, 0x69, 0xd5, 0xb4, 0xb2, 0xb0, 0xb2, 0xe9, 0xd0, 0x40, 0x84, 0xb1, 0x86, 0x81, - 0xb8, 0xdb, 0x53, 0xe6, 0x80, 0x61, 0x10, 0xbd, 0x72, 0x39, 0x16, 0xbf, 0xe7, 0x90, 0xa3, 0x57, - 0x2e, 0x5b, 0x2e, 0x5d, 0x7b, 0xa1, 0x9f, 0x1f, 0x19, 0x8b, 0x9f, 0x87, 0x11, 0x6f, 0x88, 0x22, - 0xea, 0x35, 0xfe, 0x8b, 0x17, 0xbe, 0x15, 0x04, 0xfc, 0xe9, 0x1f, 0xfe, 0x28, 0x4f, 0x2d, 0x02, - 0xc6, 0x6f, 0x0c, 0x89, 0xdf, 0x1b, 0x41, 0x12, 0x10, 0x6f, 0x0c, 0xf1, 0x5f, 0x1b, 0x41, 0x82, - 0x6b, 0x50, 0xfe, 0xd6, 0x73, 0x2d, 0x32, 0xdc, 0xab, 0xd4, 0xaa, 0x12, 0xe6, 0xf7, 0x8d, 0x99, - 0xfa, 0x67, 0x19, 0xd8, 0x5c, 0x1c, 0x55, 0x5a, 0x30, 0x35, 0x28, 0xb7, 0xfa, 0x5d, 0xbd, 0xd7, - 0xdc, 0x6f, 0x2b, 0x6b, 0xec, 0x12, 0x54, 0xfb, 0xdb, 0x5f, 0xb4, 0x5b, 0x43, 0x0e, 0xc8, 0xd0, - 0x65, 0xbd, 0x81, 0xbe, 0xd7, 0xd9, 0xd9, 0x69, 0xf7, 0xb8, 0x95, 0xd2, 0xdf, 0xfe, 0x42, 0xef, - 0xf6, 0x5b, 0xfc, 0xe7, 0x09, 0xa2, 0xd3, 0xf7, 0x81, 0x92, 0xa7, 0x13, 0x6f, 0x8a, 0x09, 0xc5, - 0x6c, 0x81, 0x87, 0x3c, 0x3e, 0x1d, 0xe8, 0xad, 0xde, 0x50, 0x29, 0x62, 0xae, 0x77, 0xd8, 0xed, - 0x52, 0x8e, 0x62, 0x9b, 0x5a, 0xfd, 0xfd, 0x03, 0xad, 0x3d, 0x18, 0xe8, 0x83, 0xce, 0x4f, 0xda, - 0x4a, 0x99, 0xbe, 0xac, 0x75, 0x1e, 0x75, 0x7a, 0x1c, 0x50, 0x61, 0x25, 0xc8, 0xed, 0x77, 0x7a, - 0x0a, 0x50, 0xa2, 0xf9, 0xa5, 0x52, 0xc5, 0xc4, 0xe0, 0x70, 0x5f, 0xa9, 0xdd, 0x7d, 0x15, 0x6a, - 0xf2, 0x6f, 0xfc, 0x50, 0x94, 0xa3, 0xe7, 0x5a, 0xfc, 0xe5, 0xcb, 0xee, 0xb7, 0x1f, 0x28, 0x99, - 0xbb, 0xbf, 0x29, 0xbd, 0x9d, 0x4e, 0x34, 0xe2, 0x30, 0x80, 0x2e, 0xba, 0xf1, 0x1b, 0x71, 0xe4, - 0xfa, 0xa7, 0x0b, 0x74, 0x7b, 0xcd, 0xc1, 0x1e, 0x3f, 0x26, 0x10, 0x18, 0x02, 0xe4, 0x92, 0x17, - 0x13, 0xe9, 0x46, 0x2a, 0x25, 0xe3, 0xc3, 0xf6, 0x02, 0x5d, 0x10, 0xec, 0x0c, 0xb0, 0x73, 0x0a, - 0xd4, 0x30, 0x15, 0xe3, 0x4a, 0x77, 0x55, 0xa8, 0x4a, 0x8f, 0xdc, 0xd2, 0x37, 0x8c, 0xe0, 0x58, - 0x3c, 0xc2, 0x88, 0xe6, 0xa6, 0x92, 0xb9, 0xfb, 0x06, 0x4a, 0x0c, 0xf9, 0x89, 0x59, 0x80, 0x62, - 0xcf, 0xf3, 0xa7, 0x86, 0x23, 0xe8, 0xac, 0x79, 0x80, 0x74, 0xef, 0xc2, 0x95, 0x95, 0x0f, 0xe6, - 0x52, 0xa4, 0xae, 0x3d, 0x9d, 0x39, 0x16, 0x0f, 0x36, 0xdd, 0x3b, 0x1f, 0xf9, 0xb6, 0xa9, 0x64, - 0xee, 0x3e, 0x8c, 0xee, 0xc2, 0x45, 0xdf, 0xee, 0xf6, 0x9b, 0x3b, 0x7c, 0x72, 0xe3, 0x4b, 0xb2, - 0xc3, 0x6d, 0xfe, 0xc0, 0xa2, 0xd6, 0x1e, 0x1c, 0x76, 0x87, 0xe2, 0x76, 0xee, 0xdd, 0xcf, 0xa1, - 0x71, 0x51, 0xd4, 0x25, 0xb6, 0xa8, 0xb5, 0xd7, 0xa4, 0xc8, 0x56, 0x9c, 0xcc, 0xbe, 0xce, 0x73, - 0x19, 0x1e, 0x18, 0xdc, 0x6d, 0x53, 0x44, 0xc6, 0xdd, 0x9f, 0x66, 0x24, 0x16, 0x16, 0x45, 0xce, - 0xc5, 0x00, 0x31, 0x4b, 0x32, 0x48, 0xb3, 0x0c, 0x53, 0xc9, 0xb0, 0xab, 0xc0, 0x52, 0xa0, 0xae, - 0x37, 0x36, 0x1c, 0x25, 0x4b, 0xb1, 0x17, 0x11, 0x9c, 0xe2, 0x9b, 0x95, 0x1c, 0x7b, 0x05, 0xae, - 0xc5, 0xb0, 0xae, 0x77, 0x7a, 0xe0, 0xdb, 0x68, 0x6b, 0x9f, 0x73, 0x74, 0x7e, 0xfb, 0xb3, 0x3f, - 0xfd, 0xf9, 0xcd, 0xcc, 0xbf, 0xf9, 0xf9, 0xcd, 0xcc, 0x7f, 0xf9, 0xf9, 0xcd, 0xb5, 0x9f, 0xfd, - 0xd7, 0x9b, 0x99, 0x9f, 0xc8, 0xbf, 0x04, 0x3c, 0x35, 0x42, 0xdf, 0x3e, 0xe3, 0x9b, 0x26, 0xca, - 0xb8, 0xd6, 0xbb, 0xb3, 0x93, 0xa3, 0x77, 0x67, 0xa3, 0x77, 0x91, 0x33, 0x8d, 0x8a, 0xf4, 0x9b, - 0xbf, 0x0f, 0xfe, 0x4f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x72, 0x69, 0xb0, 0x0a, 0x53, 0x78, 0x00, - 0x00, + // 11593 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0xbd, 0x4b, 0x8c, 0x23, 0xc7, + 0x96, 0x18, 0x5a, 0x2c, 0xfe, 0x0f, 0x59, 0xac, 0xac, 0xe8, 0x8f, 0xd8, 0xad, 0x56, 0x77, 0x29, + 0xd5, 0x92, 0x5a, 0x2d, 0xa9, 0x25, 0x75, 0xeb, 0xd3, 0xd2, 0xbb, 0x77, 0xae, 0x58, 0x24, 0xab, + 0x8b, 0x6a, 0x16, 0x59, 0x37, 0xc9, 0xea, 0x96, 0xee, 0xe0, 0xbd, 0x44, 0x92, 0x99, 0xac, 0x4a, + 0x55, 0x32, 0x93, 0xca, 0x4c, 0x76, 0x55, 0x09, 0x18, 0xe0, 0xbe, 0x37, 0xc0, 0x3c, 0x8f, 0xe1, + 0x9d, 0x81, 0x59, 0x79, 0x8c, 0x3b, 0xb3, 0x32, 0x06, 0x9e, 0x95, 0x07, 0x18, 0xc3, 0xf0, 0xce, + 0x5e, 0x8c, 0x07, 0x86, 0x61, 0xc0, 0x0b, 0xc3, 0x36, 0x30, 0x36, 0xae, 0x17, 0x5e, 0xd9, 0x03, + 0x78, 0xbc, 0x31, 0xe0, 0x85, 0x71, 0x4e, 0x44, 0x66, 0x46, 0x92, 0x2c, 0xb5, 0x5a, 0xf7, 0x0e, + 0x6c, 0x6f, 0xaa, 0x32, 0xce, 0x39, 0xf1, 0x8f, 0x38, 0x71, 0x7e, 0x11, 0x04, 0x98, 0x39, 0x86, + 0x7b, 0x6f, 0xe6, 0x7b, 0xa1, 0xc7, 0x72, 0xf8, 0x7d, 0xfd, 0xdd, 0x23, 0x3b, 0x3c, 0x9e, 0x8f, + 0xee, 0x8d, 0xbd, 0xe9, 0x7b, 0x47, 0xde, 0x91, 0xf7, 0x1e, 0x21, 0x47, 0xf3, 0x09, 0xa5, 0x28, + 0x41, 0x5f, 0x3c, 0xd3, 0x75, 0x70, 0xbc, 0xf1, 0x89, 0xf8, 0xde, 0x0c, 0xed, 0xa9, 0x15, 0x84, + 0xc6, 0x74, 0xc6, 0x01, 0xea, 0x9f, 0x66, 0x20, 0x37, 0x3c, 0x9f, 0x59, 0xac, 0x06, 0xeb, 0xb6, + 0x59, 0xcf, 0x6c, 0x67, 0xee, 0xe4, 0xb5, 0x75, 0xdb, 0x64, 0xdb, 0x50, 0x71, 0xbd, 0xb0, 0x37, + 0x77, 0x1c, 0x63, 0xe4, 0x58, 0xf5, 0xf5, 0xed, 0xcc, 0x9d, 0x92, 0x26, 0x83, 0xd8, 0xcb, 0x50, + 0x36, 0xe6, 0xa1, 0xa7, 0xdb, 0xee, 0xd8, 0xaf, 0x67, 0x09, 0x5f, 0x42, 0x40, 0xc7, 0x1d, 0xfb, + 0xec, 0x32, 0xe4, 0x4f, 0x6d, 0x33, 0x3c, 0xae, 0xe7, 0xa8, 0x44, 0x9e, 0x40, 0x68, 0x30, 0x36, + 0x1c, 0xab, 0x9e, 0xe7, 0x50, 0x4a, 0x20, 0x34, 0xa4, 0x4a, 0x0a, 0xdb, 0x99, 0x3b, 0x65, 0x8d, + 0x27, 0xd8, 0x4d, 0x00, 0xcb, 0x9d, 0x4f, 0x9f, 0x19, 0xce, 0xdc, 0x0a, 0xea, 0x45, 0x42, 0x49, + 0x10, 0xf5, 0x27, 0x50, 0x9e, 0x06, 0x47, 0x7b, 0x96, 0x61, 0x5a, 0x3e, 0x7b, 0x09, 0x8a, 0xd3, + 0xe0, 0x48, 0x0f, 0x8d, 0x23, 0xd1, 0x85, 0xc2, 0x34, 0x38, 0x1a, 0x1a, 0x47, 0xec, 0x1a, 0x94, + 0x08, 0x71, 0x3e, 0xe3, 0x7d, 0xc8, 0x6b, 0x48, 0x88, 0x3d, 0x56, 0xff, 0x32, 0x0f, 0xc5, 0xae, + 0x1d, 0x5a, 0xbe, 0xe1, 0xb0, 0xab, 0x50, 0xb0, 0x03, 0x77, 0xee, 0x38, 0x94, 0xbd, 0xa4, 0x89, + 0x14, 0xbb, 0x0a, 0x79, 0xfb, 0xe1, 0x33, 0xc3, 0xe1, 0x79, 0xf7, 0xd6, 0x34, 0x9e, 0x64, 0x75, + 0x28, 0xd8, 0x1f, 0x7c, 0x8c, 0x88, 0xac, 0x40, 0x88, 0x34, 0x61, 0x1e, 0xdc, 0x47, 0x4c, 0x2e, + 0xc6, 0x50, 0x9a, 0x30, 0x1f, 0x7f, 0x88, 0x18, 0xec, 0x7d, 0x96, 0x30, 0x94, 0xc6, 0x5a, 0xe6, + 0x54, 0x0b, 0x0e, 0xc0, 0x06, 0xd6, 0x32, 0x8f, 0x6a, 0x99, 0xf3, 0x5a, 0x8a, 0x02, 0x21, 0xd2, + 0x84, 0xe1, 0xb5, 0x94, 0x62, 0x4c, 0x5c, 0xcb, 0x9c, 0xd7, 0x52, 0xde, 0xce, 0xdc, 0xc9, 0x11, + 0x86, 0xd7, 0x72, 0x19, 0x72, 0x26, 0xc2, 0x61, 0x3b, 0x73, 0x27, 0xb3, 0xb7, 0xa6, 0x51, 0x0a, + 0xa1, 0x01, 0x42, 0x2b, 0x38, 0xc0, 0x08, 0x0d, 0x04, 0x74, 0x84, 0xd0, 0x2a, 0x8e, 0x06, 0x42, + 0x47, 0x02, 0x3a, 0x41, 0xe8, 0xc6, 0x76, 0xe6, 0xce, 0x3a, 0x42, 0x31, 0xc5, 0xae, 0x43, 0xd1, + 0x34, 0x42, 0x0b, 0x11, 0x35, 0xd1, 0xe5, 0x08, 0x80, 0x38, 0x5c, 0x71, 0x88, 0xdb, 0x14, 0x9d, + 0x8e, 0x00, 0x4c, 0x85, 0x0a, 0x92, 0x45, 0x78, 0x45, 0xe0, 0x65, 0x20, 0xfb, 0x08, 0xaa, 0xa6, + 0x35, 0xb6, 0xa7, 0x86, 0xc3, 0xfb, 0xb4, 0xb5, 0x9d, 0xb9, 0x53, 0xb9, 0xbf, 0x79, 0x8f, 0xf6, + 0x44, 0x8c, 0xd9, 0x5b, 0xd3, 0x52, 0x64, 0xec, 0x21, 0x6c, 0x88, 0xf4, 0x07, 0xf7, 0x69, 0x60, + 0x19, 0xe5, 0x53, 0x52, 0xf9, 0x3e, 0xb8, 0xff, 0x70, 0x6f, 0x4d, 0x4b, 0x13, 0xb2, 0xdb, 0x50, + 0x8d, 0xb7, 0x08, 0x66, 0xbc, 0x24, 0x5a, 0x95, 0x82, 0x62, 0xb7, 0xbe, 0x0e, 0x3c, 0x17, 0x09, + 0x2e, 0x8b, 0x71, 0x8b, 0x00, 0x6c, 0x1b, 0xc0, 0xb4, 0x26, 0xc6, 0xdc, 0x09, 0x11, 0x7d, 0x45, + 0x0c, 0xa0, 0x04, 0x63, 0x37, 0xa1, 0x3c, 0x9f, 0x61, 0x2f, 0x9f, 0x18, 0x4e, 0xfd, 0xaa, 0x20, + 0x48, 0x40, 0x58, 0x3a, 0xae, 0x73, 0xc4, 0xbe, 0x24, 0x66, 0x37, 0x02, 0xe0, 0x5e, 0xb1, 0x83, + 0x1d, 0xdb, 0xad, 0xd7, 0x69, 0x9d, 0xf2, 0x04, 0xbb, 0x01, 0xd9, 0xc0, 0x1f, 0xd7, 0xaf, 0x51, + 0x2f, 0x81, 0xf7, 0xb2, 0x7d, 0x36, 0xf3, 0x35, 0x04, 0xef, 0x14, 0x21, 0x4f, 0x7b, 0x46, 0xbd, + 0x01, 0xa5, 0x03, 0xc3, 0x37, 0xa6, 0x9a, 0x35, 0x61, 0x0a, 0x64, 0x67, 0x5e, 0x20, 0x76, 0x0b, + 0x7e, 0xaa, 0x5d, 0x28, 0x3c, 0x31, 0x7c, 0xc4, 0x31, 0xc8, 0xb9, 0xc6, 0xd4, 0x22, 0x64, 0x59, + 0xa3, 0x6f, 0xdc, 0x21, 0xc1, 0x79, 0x10, 0x5a, 0x53, 0xc1, 0x0a, 0x44, 0x0a, 0xe1, 0x47, 0x8e, + 0x37, 0x12, 0x3b, 0xa1, 0xa4, 0x89, 0x94, 0xfa, 0xff, 0x65, 0xa0, 0xd0, 0xf4, 0x1c, 0x2c, 0xee, + 0x25, 0x28, 0xfa, 0x96, 0xa3, 0x27, 0xd5, 0x15, 0x7c, 0xcb, 0x39, 0xf0, 0x02, 0x44, 0x8c, 0x3d, + 0x8e, 0xe0, 0x7b, 0xb3, 0x30, 0xf6, 0x08, 0x11, 0x35, 0x20, 0x2b, 0x35, 0xe0, 0x1a, 0x94, 0xc2, + 0x91, 0xa3, 0x13, 0x3c, 0x47, 0xf0, 0x62, 0x38, 0x72, 0x7a, 0x88, 0x7a, 0x09, 0x8a, 0xe6, 0x88, + 0x63, 0xf2, 0x84, 0x29, 0x98, 0x23, 0x44, 0xa8, 0x9f, 0x42, 0x59, 0x33, 0x4e, 0x45, 0x33, 0xae, + 0x40, 0x01, 0x0b, 0x10, 0x5c, 0x2e, 0xa7, 0xe5, 0xc3, 0x91, 0xd3, 0x31, 0x11, 0x8c, 0x8d, 0xb0, + 0x4d, 0x6a, 0x43, 0x4e, 0xcb, 0x8f, 0x3d, 0xa7, 0x63, 0xaa, 0x43, 0x80, 0xa6, 0xe7, 0xfb, 0x3f, + 0xb8, 0x0b, 0x97, 0x21, 0x6f, 0x5a, 0xb3, 0xf0, 0x98, 0x33, 0x08, 0x8d, 0x27, 0xd4, 0xbb, 0x50, + 0xc2, 0x79, 0xe9, 0xda, 0x41, 0xc8, 0x6e, 0x42, 0xce, 0xb1, 0x83, 0xb0, 0x9e, 0xd9, 0xce, 0x2e, + 0xcc, 0x1a, 0xc1, 0xd5, 0x6d, 0x28, 0xed, 0x1b, 0x67, 0x4f, 0x70, 0xe6, 0xb0, 0x34, 0x9a, 0x42, + 0x31, 0x25, 0x62, 0x3e, 0xab, 0x00, 0x43, 0xc3, 0x3f, 0xb2, 0x42, 0xe2, 0x67, 0x7f, 0x95, 0x81, + 0xca, 0x60, 0x3e, 0xfa, 0x66, 0x6e, 0xf9, 0xe7, 0xd8, 0xe6, 0x3b, 0x90, 0x0d, 0xcf, 0x67, 0x94, + 0xa3, 0x76, 0xff, 0x2a, 0x2f, 0x5e, 0xc2, 0xdf, 0xc3, 0x4c, 0x1a, 0x92, 0x60, 0x27, 0x5c, 0xcf, + 0xb4, 0xa2, 0x31, 0xc8, 0x6b, 0x05, 0x4c, 0x76, 0x4c, 0x3c, 0x14, 0xbc, 0x99, 0x98, 0x85, 0x75, + 0x6f, 0xc6, 0xb6, 0x21, 0x3f, 0x3e, 0xb6, 0x1d, 0x93, 0x26, 0x20, 0xdd, 0x66, 0x8e, 0xc0, 0x59, + 0xf2, 0xbd, 0x53, 0x3d, 0xb0, 0xbf, 0x8d, 0x98, 0x7c, 0xd1, 0xf7, 0x4e, 0x07, 0xf6, 0xb7, 0x96, + 0x3a, 0x14, 0x27, 0x0d, 0x40, 0x61, 0xd0, 0x6c, 0x74, 0x1b, 0x9a, 0xb2, 0x86, 0xdf, 0xed, 0x2f, + 0x3b, 0x83, 0xe1, 0x40, 0xc9, 0xb0, 0x1a, 0x40, 0xaf, 0x3f, 0xd4, 0x45, 0x7a, 0x9d, 0x15, 0x60, + 0xbd, 0xd3, 0x53, 0xb2, 0x48, 0x83, 0xf0, 0x4e, 0x4f, 0xc9, 0xb1, 0x22, 0x64, 0x1b, 0xbd, 0xaf, + 0x94, 0x3c, 0x7d, 0x74, 0xbb, 0x4a, 0x41, 0xfd, 0xa3, 0x75, 0x28, 0xf7, 0x47, 0x5f, 0x5b, 0xe3, + 0x10, 0xfb, 0x8c, 0xab, 0xd4, 0xf2, 0x9f, 0x59, 0x3e, 0x75, 0x3b, 0xab, 0x89, 0x14, 0x76, 0xc4, + 0x1c, 0x51, 0xe7, 0xb2, 0xda, 0xba, 0x39, 0x22, 0xba, 0xf1, 0xb1, 0x35, 0x35, 0xa8, 0x73, 0x48, + 0x47, 0x29, 0xdc, 0x15, 0xde, 0xe8, 0x6b, 0xea, 0x5e, 0x56, 0xc3, 0x4f, 0x76, 0x0b, 0x2a, 0xbc, + 0x0c, 0x79, 0x7d, 0x01, 0x07, 0x2d, 0x2e, 0xbe, 0x82, 0xbc, 0xf8, 0x28, 0x27, 0x95, 0xca, 0x91, + 0xe2, 0x04, 0xe3, 0xa0, 0x9e, 0x58, 0xd1, 0xde, 0xe8, 0x6b, 0x8e, 0x2d, 0xf1, 0x15, 0xed, 0x8d, + 0xbe, 0x26, 0xd4, 0xdb, 0xb0, 0x15, 0xcc, 0x47, 0xc1, 0xd8, 0xb7, 0x67, 0xa1, 0xed, 0xb9, 0x9c, + 0xa6, 0x4c, 0x34, 0x8a, 0x8c, 0x20, 0xe2, 0x3b, 0x50, 0x9a, 0xcd, 0x47, 0xba, 0xed, 0x4e, 0x3c, + 0x62, 0xee, 0x95, 0xfb, 0x1b, 0x7c, 0x62, 0x0e, 0xe6, 0xa3, 0x8e, 0x3b, 0xf1, 0xb4, 0xe2, 0x8c, + 0x7f, 0xa8, 0x6f, 0x40, 0x51, 0xc0, 0xf0, 0xf4, 0x0e, 0x2d, 0xd7, 0x70, 0x43, 0x3d, 0x3e, 0xf6, + 0x4b, 0x1c, 0xd0, 0x31, 0xd5, 0x3f, 0xc9, 0x80, 0x32, 0x90, 0xaa, 0xd9, 0xb7, 0x42, 0x63, 0x25, + 0x57, 0x78, 0x05, 0xc0, 0x18, 0x8f, 0xbd, 0x39, 0x2f, 0x86, 0x2f, 0x9e, 0xb2, 0x80, 0x74, 0x4c, + 0x79, 0x6c, 0xb2, 0xa9, 0xb1, 0x79, 0x15, 0xaa, 0x51, 0x3e, 0x69, 0x43, 0x57, 0x04, 0x2c, 0x1a, + 0x9d, 0x60, 0x9e, 0xda, 0xd5, 0xc5, 0x60, 0xce, 0x73, 0x5f, 0x85, 0x02, 0xc9, 0x08, 0x41, 0x34, + 0xe2, 0x3c, 0xa5, 0xfe, 0xcd, 0x75, 0x28, 0xed, 0xce, 0xdd, 0x31, 0x36, 0x99, 0xbd, 0x06, 0xb9, + 0xc9, 0xdc, 0x1d, 0x53, 0x73, 0xe3, 0x23, 0x23, 0x5e, 0x29, 0x1a, 0x21, 0x71, 0x0f, 0x1a, 0xfe, + 0x11, 0xee, 0xdd, 0xa5, 0x3d, 0x88, 0x70, 0xf5, 0x1f, 0x66, 0x78, 0x89, 0xbb, 0x8e, 0x71, 0xc4, + 0x4a, 0x90, 0xeb, 0xf5, 0x7b, 0x6d, 0x65, 0x8d, 0x55, 0xa1, 0xd4, 0xe9, 0x0d, 0xdb, 0x5a, 0xaf, + 0xd1, 0x55, 0x32, 0xb4, 0xa0, 0x87, 0x8d, 0x9d, 0x6e, 0x5b, 0x59, 0x47, 0xcc, 0x93, 0x7e, 0xb7, + 0x31, 0xec, 0x74, 0xdb, 0x4a, 0x8e, 0x63, 0xb4, 0x4e, 0x73, 0xa8, 0x94, 0x98, 0x02, 0xd5, 0x03, + 0xad, 0xdf, 0x3a, 0x6c, 0xb6, 0xf5, 0xde, 0x61, 0xb7, 0xab, 0x28, 0xec, 0x12, 0x6c, 0xc6, 0x90, + 0x3e, 0x07, 0x6e, 0x63, 0x96, 0x27, 0x0d, 0xad, 0xa1, 0x3d, 0x52, 0x3e, 0x67, 0x25, 0xc8, 0x36, + 0x1e, 0x3d, 0x52, 0x7e, 0x8e, 0x7b, 0xa3, 0xfc, 0xb4, 0xd3, 0xd3, 0x9f, 0x34, 0xba, 0x87, 0x6d, + 0xe5, 0xe7, 0xeb, 0x51, 0xba, 0xaf, 0xb5, 0xda, 0x9a, 0xf2, 0xf3, 0x1c, 0xdb, 0x82, 0xea, 0xcf, + 0xfa, 0xbd, 0xf6, 0x7e, 0xe3, 0xe0, 0x80, 0x1a, 0xf2, 0xf3, 0x92, 0xfa, 0x9f, 0x73, 0x90, 0xc3, + 0x9e, 0x30, 0x35, 0xe1, 0x03, 0x71, 0x17, 0x71, 0x23, 0xee, 0xe4, 0xfe, 0xec, 0x2f, 0x6e, 0xad, + 0x71, 0x0e, 0xf0, 0x2a, 0x64, 0x1d, 0x3b, 0xa4, 0x09, 0x8c, 0x57, 0x8f, 0x90, 0x8d, 0xf6, 0xd6, + 0x34, 0xc4, 0xb1, 0x9b, 0x90, 0xe1, 0xac, 0xa0, 0x72, 0xbf, 0x26, 0x96, 0x97, 0x38, 0x4b, 0xf6, + 0xd6, 0xb4, 0xcc, 0x8c, 0xdd, 0x80, 0xcc, 0x33, 0xc1, 0x17, 0xaa, 0x1c, 0xcf, 0x4f, 0x13, 0xc4, + 0x3e, 0x63, 0xdb, 0x90, 0x1d, 0x7b, 0x5c, 0xf2, 0x89, 0xf1, 0x9c, 0xb7, 0x62, 0xf9, 0x63, 0xcf, + 0x61, 0xaf, 0x41, 0xd6, 0x37, 0x4e, 0x69, 0x46, 0xe3, 0xe9, 0x8a, 0x99, 0x37, 0x12, 0xf9, 0xc6, + 0x29, 0x36, 0x62, 0x42, 0x3b, 0x29, 0x6e, 0x44, 0x34, 0xdf, 0x58, 0xcd, 0x84, 0x6d, 0x43, 0xe6, + 0x94, 0xf6, 0x52, 0x7c, 0xd8, 0x3f, 0xb5, 0x5d, 0xd3, 0x3b, 0x1d, 0xcc, 0xac, 0x31, 0x52, 0x9c, + 0xb2, 0xd7, 0x21, 0x1b, 0xcc, 0x47, 0xb4, 0x97, 0x2a, 0xf7, 0xb7, 0x96, 0xb8, 0x22, 0x56, 0x14, + 0xcc, 0x47, 0xec, 0x0d, 0xc8, 0x8d, 0x3d, 0xdf, 0x17, 0xfb, 0x49, 0x89, 0x1a, 0x1c, 0x1d, 0x08, + 0x28, 0xfc, 0x20, 0x1e, 0x2b, 0x0c, 0x49, 0x76, 0x8a, 0x89, 0x12, 0x8e, 0x8c, 0x15, 0x86, 0xec, + 0xb6, 0x60, 0xf3, 0x55, 0xb9, 0xd5, 0xd1, 0x21, 0x80, 0xe5, 0x20, 0x16, 0x27, 0x69, 0x6a, 0x9c, + 0x91, 0x64, 0x15, 0x13, 0x45, 0xdc, 0x1f, 0xdb, 0x34, 0x35, 0xce, 0xd8, 0x6d, 0xc8, 0x3e, 0xb3, + 0xc6, 0x24, 0x64, 0xc5, 0xb5, 0x89, 0x49, 0x7a, 0x42, 0xdd, 0x43, 0x34, 0xad, 0x7b, 0xcf, 0x31, + 0x49, 0xde, 0x8a, 0xe7, 0x72, 0xd7, 0x73, 0xcc, 0x27, 0x34, 0x97, 0x84, 0xc4, 0x43, 0xcf, 0x98, + 0x9f, 0xe1, 0x9e, 0x55, 0xf8, 0xf1, 0x64, 0xcc, 0xcf, 0x3a, 0x26, 0xb2, 0x3f, 0xd7, 0x7c, 0x46, + 0x52, 0x56, 0x46, 0xc3, 0x4f, 0x54, 0x03, 0x02, 0xcb, 0xb1, 0xc6, 0xa1, 0xfd, 0xcc, 0x0e, 0xcf, + 0x49, 0x8e, 0xca, 0x68, 0x32, 0x68, 0xa7, 0x00, 0x39, 0xeb, 0x6c, 0xe6, 0xab, 0x7b, 0x50, 0x14, + 0xb5, 0x2c, 0xe9, 0x12, 0xd7, 0xa0, 0x64, 0x07, 0xfa, 0xd8, 0x73, 0x83, 0x50, 0x48, 0x0f, 0x45, + 0x3b, 0x68, 0x62, 0x12, 0x99, 0x8a, 0x69, 0x84, 0x9c, 0x0d, 0x57, 0x35, 0xfa, 0x56, 0xef, 0x03, + 0x24, 0xdd, 0xc2, 0x36, 0x39, 0x96, 0x1b, 0x09, 0x2a, 0x8e, 0xe5, 0xc6, 0x79, 0xd6, 0xa5, 0x3c, + 0xd7, 0xa0, 0x1c, 0x4b, 0x80, 0xac, 0x0a, 0x19, 0x43, 0x1c, 0x00, 0x19, 0x43, 0xbd, 0x83, 0x02, + 0x59, 0x24, 0xe3, 0xa5, 0x71, 0x98, 0x8a, 0x8e, 0x85, 0xcc, 0x48, 0xfd, 0x11, 0x54, 0x35, 0x2b, + 0x98, 0x3b, 0x61, 0xd3, 0x73, 0x5a, 0xd6, 0x84, 0xbd, 0x03, 0x10, 0xa7, 0x03, 0x71, 0x4e, 0x27, + 0x6b, 0xb7, 0x65, 0x4d, 0x34, 0x09, 0xaf, 0xfe, 0xbd, 0x1c, 0x49, 0x3c, 0x2d, 0x2e, 0x6a, 0x08, + 0x99, 0x22, 0x23, 0xc9, 0x14, 0x31, 0x07, 0x5d, 0x4f, 0xcb, 0x55, 0xc7, 0xb6, 0x69, 0x5a, 0x6e, + 0x24, 0x3f, 0xf1, 0x14, 0x4e, 0xb6, 0xe1, 0x1c, 0xd1, 0x86, 0xaa, 0xdd, 0x67, 0x51, 0xa5, 0xd3, + 0x99, 0x6f, 0x05, 0x01, 0x3f, 0xb9, 0x0d, 0xe7, 0x28, 0xda, 0xdb, 0xf9, 0xef, 0xda, 0xdb, 0xd7, + 0xa0, 0xe4, 0x7a, 0xa1, 0x4e, 0xda, 0x4d, 0x81, 0x8f, 0xbe, 0x50, 0xe3, 0xd8, 0x9b, 0x50, 0x14, + 0x72, 0xa9, 0xd8, 0x54, 0x62, 0xb9, 0xb4, 0x38, 0x50, 0x8b, 0xb0, 0xac, 0x8e, 0x62, 0xce, 0x74, + 0x6a, 0xb9, 0x61, 0x74, 0x52, 0x89, 0x24, 0x7b, 0x1b, 0xca, 0x9e, 0xab, 0x73, 0xe1, 0x55, 0xec, + 0x2a, 0xb1, 0x7c, 0xfb, 0xee, 0x21, 0x41, 0xb5, 0x92, 0x27, 0xbe, 0xb0, 0x29, 0x8e, 0x77, 0xaa, + 0x8f, 0x0d, 0xdf, 0xa4, 0x9d, 0x55, 0xd2, 0x8a, 0x8e, 0x77, 0xda, 0x34, 0x7c, 0x93, 0x9f, 0xdc, + 0xdf, 0xb8, 0xf3, 0x29, 0xed, 0xa6, 0x0d, 0x4d, 0xa4, 0xd8, 0x0d, 0x28, 0x8f, 0x9d, 0x79, 0x10, + 0x5a, 0xfe, 0xce, 0x39, 0x57, 0x47, 0xb4, 0x04, 0x80, 0xed, 0x9a, 0xf9, 0xf6, 0xd4, 0xf0, 0xcf, + 0x69, 0xeb, 0x94, 0xb4, 0x28, 0x89, 0x12, 0xd3, 0xec, 0xc4, 0x36, 0xcf, 0xb8, 0x4e, 0xa2, 0xf1, + 0x04, 0xd2, 0x1f, 0x93, 0xc6, 0x18, 0xd0, 0xfe, 0x28, 0x69, 0x51, 0x92, 0xe6, 0x81, 0x3e, 0x69, + 0x47, 0x94, 0x35, 0x91, 0x4a, 0x89, 0x9d, 0x5b, 0x17, 0x8a, 0x9d, 0x6c, 0xf1, 0xe4, 0xf7, 0x7c, + 0xfb, 0xc8, 0x16, 0xe7, 0xf6, 0x25, 0x7e, 0xf2, 0x73, 0x10, 0xc9, 0xa5, 0xdf, 0x40, 0x51, 0x0c, + 0x31, 0x9e, 0x40, 0xb8, 0x7d, 0xd2, 0xec, 0x99, 0x9f, 0x40, 0x08, 0x67, 0xaf, 0xc1, 0x86, 0x28, + 0x2b, 0x08, 0x7d, 0xdb, 0x3d, 0x12, 0x8b, 0xa7, 0xca, 0x81, 0x03, 0x82, 0xe1, 0x71, 0x8a, 0xd3, + 0xab, 0x1b, 0x23, 0xdb, 0xc1, 0x6d, 0x9a, 0x15, 0xda, 0xfa, 0xdc, 0x71, 0x1a, 0x1c, 0xa4, 0xf6, + 0xa1, 0x14, 0x4d, 0xc8, 0xaf, 0xa5, 0x4e, 0xf5, 0x77, 0x32, 0x50, 0xe9, 0xb8, 0xa6, 0x75, 0xd6, + 0x27, 0x11, 0x81, 0xbd, 0x03, 0x6c, 0xec, 0x5b, 0x46, 0x68, 0xe9, 0xd6, 0x59, 0xe8, 0x1b, 0x3a, + 0x57, 0xe9, 0xb9, 0x3a, 0xad, 0x70, 0x4c, 0x1b, 0x11, 0x43, 0xd2, 0xee, 0x6f, 0x41, 0x65, 0x66, + 0xf8, 0x41, 0x24, 0x56, 0xf1, 0x0a, 0x80, 0x83, 0x84, 0x50, 0xa3, 0xb8, 0x47, 0xbe, 0x31, 0xd5, + 0x43, 0xef, 0xc4, 0x72, 0xb9, 0x40, 0xc9, 0x45, 0xe9, 0x1a, 0xc1, 0x87, 0x08, 0x26, 0xb9, 0xf2, + 0xdf, 0x66, 0x60, 0xe3, 0x80, 0xcf, 0xfa, 0x63, 0xeb, 0xbc, 0xc5, 0xf5, 0x97, 0x71, 0xb4, 0x63, + 0x73, 0x1a, 0x7d, 0xb3, 0x9b, 0x50, 0x99, 0x9d, 0x58, 0xe7, 0x7a, 0x4a, 0xd6, 0x2f, 0x23, 0xa8, + 0x49, 0x7b, 0xf3, 0x2d, 0x28, 0x78, 0xd4, 0x11, 0x71, 0xc6, 0x89, 0xa3, 0x41, 0xea, 0xa1, 0x26, + 0x08, 0x98, 0x0a, 0x1b, 0x71, 0x51, 0xb2, 0xf4, 0x22, 0x0a, 0xa3, 0xe6, 0x5f, 0x86, 0x3c, 0xa2, + 0x82, 0x7a, 0x7e, 0x3b, 0x8b, 0x02, 0x3b, 0x25, 0xd8, 0xfb, 0xb0, 0x31, 0xf6, 0xa6, 0x33, 0x3d, + 0xca, 0x2e, 0x4e, 0xbb, 0x34, 0x4f, 0xa9, 0x20, 0xc9, 0x01, 0x2f, 0x4b, 0xfd, 0xbd, 0x2c, 0x94, + 0xa8, 0x0d, 0x82, 0xad, 0xd8, 0xe6, 0x59, 0xc4, 0x56, 0xca, 0x5a, 0xde, 0x36, 0x91, 0x6b, 0xbf, + 0x02, 0x60, 0x23, 0x89, 0x3c, 0x94, 0x65, 0x82, 0x44, 0x4d, 0x99, 0x19, 0x7e, 0x18, 0xd4, 0xb3, + 0xbc, 0x29, 0x94, 0xc0, 0xf5, 0x3e, 0x77, 0xed, 0x6f, 0xe6, 0xbc, 0xf5, 0x25, 0x4d, 0xa4, 0x70, + 0xdc, 0x79, 0x61, 0x34, 0x7f, 0xb2, 0xf8, 0x55, 0x23, 0x38, 0x4d, 0x5f, 0xb4, 0xca, 0x39, 0x8d, + 0x75, 0x86, 0xe7, 0x1b, 0x67, 0x2d, 0x40, 0xa0, 0x36, 0x42, 0x64, 0xa6, 0x51, 0x4c, 0x33, 0x8d, + 0x3a, 0x14, 0x9f, 0xd9, 0x81, 0x8d, 0x0b, 0xa4, 0xc4, 0xb7, 0xa1, 0x48, 0x4a, 0xd3, 0x50, 0x7e, + 0xde, 0x34, 0xc4, 0xdd, 0x36, 0x9c, 0x23, 0x2e, 0xf8, 0x46, 0xdd, 0x6e, 0x38, 0x47, 0x1e, 0xfb, + 0x00, 0xae, 0x24, 0x68, 0xd1, 0x1b, 0x32, 0x03, 0x91, 0xa5, 0x43, 0x63, 0x31, 0x25, 0xf5, 0x88, + 0x34, 0x93, 0xbb, 0xb0, 0x25, 0x65, 0x99, 0xa1, 0x78, 0x13, 0x10, 0xcf, 0x29, 0x6b, 0x9b, 0x31, + 0x39, 0x49, 0x3d, 0x81, 0xfa, 0xcf, 0xd6, 0x61, 0x63, 0xd7, 0xf3, 0x2d, 0xfb, 0xc8, 0x4d, 0x56, + 0xdd, 0x92, 0x7c, 0x1c, 0xad, 0xc4, 0x75, 0x69, 0x25, 0xde, 0x82, 0xca, 0x84, 0x67, 0xd4, 0xc3, + 0x11, 0x57, 0x9b, 0x73, 0x1a, 0x08, 0xd0, 0x70, 0xe4, 0xe0, 0x6e, 0x8e, 0x08, 0x28, 0x73, 0x8e, + 0x32, 0x47, 0x99, 0xf0, 0xac, 0x61, 0x9f, 0x11, 0xd7, 0x35, 0x2d, 0xc7, 0x0a, 0xf9, 0xf4, 0xd4, + 0xee, 0xbf, 0x12, 0x9d, 0xf4, 0x52, 0x9b, 0xee, 0x69, 0xd6, 0xa4, 0x41, 0xe2, 0x11, 0x32, 0xe1, + 0x16, 0x91, 0x8b, 0xbc, 0x82, 0x63, 0x17, 0xbe, 0x67, 0x5e, 0xce, 0x39, 0xd4, 0x21, 0x94, 0x63, + 0x30, 0xca, 0xba, 0x5a, 0x5b, 0xc8, 0xb7, 0x6b, 0xac, 0x02, 0xc5, 0x66, 0x63, 0xd0, 0x6c, 0xb4, + 0xda, 0x4a, 0x06, 0x51, 0x83, 0xf6, 0x90, 0xcb, 0xb4, 0xeb, 0x6c, 0x13, 0x2a, 0x98, 0x6a, 0xb5, + 0x77, 0x1b, 0x87, 0xdd, 0xa1, 0x92, 0x65, 0x1b, 0x50, 0xee, 0xf5, 0xf5, 0x46, 0x73, 0xd8, 0xe9, + 0xf7, 0x94, 0x9c, 0xfa, 0x39, 0x94, 0x9a, 0xc7, 0xd6, 0xf8, 0xe4, 0xa2, 0x51, 0x24, 0xb5, 0xd3, + 0x1a, 0x9f, 0x08, 0xf9, 0x74, 0x41, 0xed, 0xb4, 0xc6, 0x27, 0xea, 0x13, 0xa8, 0x36, 0xa3, 0x43, + 0xe1, 0xa2, 0x52, 0xee, 0x43, 0x8d, 0x36, 0xdf, 0x78, 0x14, 0xed, 0xbe, 0xf5, 0x15, 0xbb, 0xaf, + 0x8a, 0x34, 0xcd, 0x91, 0xd8, 0x7e, 0x1f, 0x41, 0xe5, 0xc0, 0xf7, 0x66, 0x96, 0x1f, 0x52, 0xb1, + 0x0a, 0x64, 0x4f, 0xac, 0x73, 0x51, 0x2a, 0x7e, 0x26, 0x8a, 0xf9, 0xba, 0xac, 0x98, 0xdf, 0x87, + 0x52, 0x94, 0xed, 0x7b, 0xe7, 0xf9, 0x09, 0x72, 0x31, 0xca, 0x63, 0x5b, 0x01, 0x56, 0x76, 0x0f, + 0x60, 0x16, 0x03, 0x84, 0xf4, 0x11, 0x49, 0xde, 0xa2, 0x70, 0x4d, 0xa2, 0x50, 0xff, 0x2a, 0x0b, + 0xb5, 0x03, 0xc3, 0x0f, 0x6d, 0x9c, 0x1c, 0x3e, 0x0c, 0x6f, 0x42, 0x8e, 0x96, 0x3c, 0xb7, 0x01, + 0x5c, 0x8a, 0xc5, 0x76, 0x4e, 0x43, 0x62, 0x04, 0x11, 0xb0, 0xcf, 0xa0, 0x36, 0x8b, 0xc0, 0x3a, + 0x9d, 0x0d, 0x7c, 0x6c, 0x16, 0xb3, 0xd0, 0x98, 0x6f, 0xcc, 0xe4, 0x24, 0xfb, 0x31, 0x5c, 0x4e, + 0xe7, 0xb5, 0x82, 0x20, 0xe1, 0xa3, 0xf2, 0x64, 0x5d, 0x4a, 0x65, 0xe4, 0x64, 0xac, 0x09, 0x5b, + 0x49, 0xf6, 0xb1, 0xe7, 0xcc, 0xa7, 0x6e, 0x20, 0xf4, 0x88, 0xab, 0x0b, 0xb5, 0x37, 0x39, 0x56, + 0x53, 0x66, 0x0b, 0x10, 0xa6, 0x42, 0x35, 0x86, 0xf5, 0xe6, 0x53, 0xda, 0x12, 0x39, 0x2d, 0x05, + 0x63, 0x0f, 0x00, 0xe2, 0x34, 0x6a, 0x8e, 0xd9, 0x15, 0xfd, 0xeb, 0x84, 0xd6, 0x54, 0x93, 0xc8, + 0x50, 0xfc, 0x40, 0x66, 0xe0, 0xdb, 0xe1, 0xf1, 0x94, 0xb8, 0x58, 0x56, 0x4b, 0x00, 0xc4, 0x2c, + 0x03, 0x1d, 0xd5, 0xd4, 0x38, 0x8b, 0x60, 0x68, 0x35, 0x3b, 0x18, 0xcc, 0x47, 0x71, 0xb9, 0x78, + 0xa4, 0x26, 0xbd, 0x9c, 0x06, 0x47, 0x42, 0x99, 0x4f, 0x5a, 0xb8, 0x1f, 0x1c, 0xb1, 0xfb, 0x70, + 0x25, 0x21, 0x4a, 0xf8, 0x6f, 0x50, 0x07, 0xe2, 0xdc, 0xc9, 0xf0, 0xc5, 0x4c, 0x38, 0x50, 0xbf, + 0x80, 0x8d, 0xd4, 0xec, 0x3c, 0xf7, 0x70, 0xbf, 0x06, 0x25, 0xfc, 0x8f, 0x47, 0xbb, 0x58, 0x80, + 0x45, 0x4c, 0x0f, 0x42, 0x5f, 0xb5, 0x40, 0x59, 0x1c, 0x6b, 0x76, 0x9b, 0x0c, 0x5c, 0x34, 0x29, + 0xcb, 0x86, 0xaa, 0x08, 0xc5, 0xde, 0x5e, 0x35, 0x89, 0xeb, 0xd4, 0xea, 0xa5, 0xc9, 0x52, 0xff, + 0x60, 0x5d, 0x6a, 0x33, 0x8e, 0x38, 0x7b, 0x5d, 0x5e, 0x7e, 0xd2, 0xc6, 0x4d, 0xc6, 0x8c, 0x4e, + 0x9c, 0xb7, 0x40, 0xf1, 0x7c, 0xd3, 0x76, 0x0d, 0x32, 0xb8, 0xf1, 0xe1, 0x5e, 0x27, 0x69, 0x71, + 0x53, 0xc0, 0x0f, 0x04, 0x18, 0xf5, 0x16, 0xd3, 0x8a, 0xed, 0x17, 0xc2, 0xfa, 0x20, 0x83, 0xe4, + 0xd3, 0x29, 0x97, 0x3e, 0x9d, 0xde, 0x84, 0xb2, 0x63, 0x05, 0x81, 0x1e, 0x1e, 0x1b, 0x2e, 0x9d, + 0xdf, 0xe9, 0x4e, 0x97, 0x10, 0x39, 0x3c, 0x36, 0x5c, 0x24, 0xb4, 0x5d, 0x5d, 0x78, 0x28, 0x0a, + 0xcb, 0x84, 0xb6, 0x4b, 0xfa, 0x1b, 0x9e, 0xfb, 0x97, 0x57, 0x4d, 0xac, 0x38, 0x16, 0xd9, 0xf2, + 0xbc, 0xaa, 0xaf, 0x40, 0xf1, 0x89, 0x6d, 0x9d, 0x0a, 0x5e, 0xf6, 0xcc, 0xb6, 0x4e, 0x23, 0x5e, + 0x86, 0xdf, 0xea, 0x7f, 0x29, 0x41, 0x89, 0x88, 0x5b, 0x17, 0x1b, 0x36, 0x5f, 0x44, 0xdb, 0xd8, + 0x16, 0xe7, 0x54, 0x6e, 0x85, 0x8e, 0xc3, 0x4f, 0xad, 0x57, 0x00, 0xa4, 0x33, 0x94, 0x4b, 0x04, + 0xe5, 0x30, 0x3e, 0x3a, 0x51, 0x4c, 0x27, 0x19, 0x2f, 0xf8, 0xc6, 0x11, 0x56, 0x99, 0x04, 0xc0, + 0xee, 0x71, 0x21, 0x9a, 0xec, 0x31, 0x45, 0x99, 0xb1, 0x50, 0x1f, 0x22, 0x15, 0x9e, 0x24, 0x6b, + 0x4c, 0x90, 0x7c, 0x60, 0xf9, 0x41, 0xb4, 0x9d, 0x36, 0xb4, 0x28, 0x89, 0x1c, 0x0d, 0x85, 0x27, + 0xa1, 0x72, 0x47, 0xdb, 0x57, 0x96, 0xfe, 0x34, 0x22, 0x60, 0x77, 0xa0, 0x48, 0x47, 0xb6, 0x85, + 0x27, 0xb8, 0xc4, 0x3a, 0x23, 0x61, 0x4a, 0x8b, 0xd0, 0xec, 0x2d, 0xc8, 0x4f, 0x4e, 0xac, 0xf3, + 0xa0, 0xbe, 0x21, 0xb3, 0x84, 0xd4, 0x59, 0xa8, 0x71, 0x0a, 0x76, 0x1b, 0x6a, 0xbe, 0x35, 0xd1, + 0xc9, 0xd4, 0x89, 0x87, 0x77, 0x50, 0xaf, 0xd1, 0xd9, 0x5c, 0xf5, 0xad, 0x49, 0x13, 0x81, 0xc3, + 0x91, 0x13, 0xb0, 0x37, 0xa0, 0x40, 0xa7, 0x12, 0xea, 0x18, 0x52, 0xcd, 0xd1, 0x11, 0xa7, 0x09, + 0x2c, 0xbb, 0x0f, 0xe5, 0x84, 0x6d, 0x5c, 0xa1, 0x0e, 0x5d, 0x5e, 0xe0, 0x47, 0xc4, 0xc6, 0xb5, + 0x84, 0x8c, 0x7d, 0x00, 0x20, 0xb4, 0x1f, 0x7d, 0x74, 0x4e, 0xce, 0x83, 0x4a, 0xac, 0x1d, 0x4a, + 0x07, 0xa0, 0xac, 0x23, 0xbd, 0x09, 0x79, 0x3c, 0x25, 0x82, 0xfa, 0x4b, 0xd4, 0x9a, 0xad, 0xf4, + 0x11, 0x42, 0xbd, 0x23, 0x3c, 0xbb, 0x03, 0x25, 0x5c, 0x5c, 0x3a, 0x4e, 0x61, 0x5d, 0x56, 0x07, + 0xc5, 0x4a, 0x44, 0x29, 0xcd, 0x3a, 0x1d, 0x7c, 0xe3, 0xb0, 0xbb, 0x90, 0x33, 0xad, 0x49, 0x50, + 0xbf, 0x46, 0x25, 0x5e, 0x95, 0xe6, 0x12, 0x05, 0x87, 0x96, 0x35, 0xe1, 0x47, 0x0b, 0xd2, 0xb0, + 0x3d, 0xa8, 0xe1, 0xd2, 0xbb, 0x4f, 0x82, 0x37, 0x0e, 0x79, 0xfd, 0x3a, 0xe5, 0x7a, 0x75, 0x21, + 0x57, 0x4f, 0x10, 0xd1, 0x04, 0xb5, 0xdd, 0xd0, 0x3f, 0xd7, 0x36, 0x5c, 0x19, 0xc6, 0xae, 0x43, + 0xc9, 0x0e, 0xba, 0xde, 0xf8, 0xc4, 0x32, 0xeb, 0x2f, 0x73, 0x7f, 0x63, 0x94, 0x66, 0x9f, 0xc2, + 0x06, 0x2d, 0x46, 0x4c, 0x62, 0xe5, 0xf5, 0x1b, 0xf2, 0x91, 0x37, 0x94, 0x51, 0x5a, 0x9a, 0x12, + 0xc5, 0x2d, 0x3b, 0xd0, 0x43, 0x6b, 0x3a, 0xf3, 0x7c, 0x54, 0x24, 0x5f, 0xe1, 0xca, 0x93, 0x1d, + 0x0c, 0x23, 0x10, 0xf2, 0xf9, 0xd8, 0xd5, 0xa9, 0x7b, 0x93, 0x49, 0x60, 0x85, 0xf5, 0x9b, 0xb4, + 0xd7, 0x6a, 0x91, 0xc7, 0xb3, 0x4f, 0x50, 0x12, 0x4a, 0x03, 0xdd, 0x3c, 0x77, 0x8d, 0xa9, 0x3d, + 0xae, 0xdf, 0xe2, 0xfa, 0xaa, 0x1d, 0xb4, 0x38, 0x40, 0x56, 0x19, 0xb7, 0x53, 0x2a, 0xe3, 0x25, + 0xc8, 0x9b, 0x23, 0xdc, 0xc2, 0xaf, 0x52, 0xb1, 0x39, 0x73, 0xd4, 0x31, 0xaf, 0x3f, 0x22, 0x35, + 0x91, 0x1a, 0xf9, 0xd1, 0x82, 0x30, 0x90, 0x5a, 0xfd, 0x92, 0xd4, 0xb0, 0xb7, 0x26, 0xcb, 0x04, + 0x3b, 0x79, 0xc8, 0x9a, 0xd6, 0xe4, 0xfa, 0xe7, 0xc0, 0x96, 0x87, 0xf7, 0x79, 0x92, 0x49, 0x5e, + 0x48, 0x26, 0x9f, 0xad, 0x3f, 0xcc, 0xa8, 0x9f, 0xc2, 0x46, 0x6a, 0xaf, 0xae, 0x94, 0xb0, 0xb8, + 0xa6, 0x61, 0x4c, 0x85, 0x65, 0x86, 0x27, 0xd4, 0x7f, 0x91, 0x85, 0xea, 0x9e, 0x11, 0x1c, 0xef, + 0x1b, 0xb3, 0x41, 0x68, 0x84, 0x01, 0x0e, 0xf8, 0xb1, 0x11, 0x1c, 0x4f, 0x8d, 0x19, 0x57, 0xeb, + 0x32, 0xdc, 0xa8, 0x24, 0x60, 0xa8, 0xd3, 0xe1, 0x54, 0x63, 0xb2, 0xef, 0x1e, 0x3c, 0x16, 0x16, + 0xa3, 0x38, 0x8d, 0xcc, 0x21, 0x38, 0x9e, 0x4f, 0x26, 0x8e, 0x25, 0x98, 0x58, 0x94, 0x64, 0xb7, + 0x61, 0x43, 0x7c, 0x92, 0x4e, 0x77, 0x26, 0x9c, 0xcf, 0x69, 0x20, 0x7b, 0x00, 0x15, 0x01, 0x18, + 0x46, 0xac, 0xac, 0x16, 0x5b, 0x02, 0x13, 0x84, 0x26, 0x53, 0xb1, 0x9f, 0xc2, 0x15, 0x29, 0xb9, + 0xeb, 0xf9, 0xfb, 0x73, 0x27, 0xb4, 0x9b, 0x3d, 0x21, 0x40, 0xbf, 0xbc, 0x94, 0x3d, 0x21, 0xd1, + 0x56, 0xe7, 0x4c, 0xb7, 0x76, 0xdf, 0x76, 0x85, 0x78, 0x91, 0x06, 0x2e, 0x50, 0x19, 0x67, 0xc4, + 0x10, 0xd3, 0x54, 0xc6, 0x19, 0x2e, 0x7f, 0x01, 0xd8, 0xb7, 0xc2, 0x63, 0xcf, 0x24, 0xf1, 0x22, + 0x5e, 0xfe, 0x03, 0x19, 0xa5, 0xa5, 0x29, 0x71, 0x38, 0xdd, 0xb9, 0xe3, 0x8c, 0xdd, 0x90, 0x74, + 0xa8, 0xac, 0x16, 0x25, 0xf1, 0xb0, 0xf0, 0x0d, 0xf7, 0xc8, 0x0a, 0xea, 0x95, 0xed, 0xec, 0x9d, + 0x8c, 0x26, 0x52, 0xea, 0xff, 0xbb, 0x0e, 0x79, 0x3e, 0x93, 0x2f, 0x43, 0x79, 0xe4, 0x78, 0xe3, + 0x13, 0xdd, 0x9d, 0x4f, 0x23, 0x27, 0x02, 0x01, 0x50, 0xde, 0x22, 0xdd, 0x47, 0x58, 0xfc, 0x32, + 0x1a, 0x7d, 0x63, 0x91, 0xde, 0x3c, 0xc4, 0xba, 0xb2, 0x04, 0x15, 0x29, 0x6c, 0x84, 0xef, 0x9d, + 0xd2, 0x6a, 0xc8, 0x11, 0x22, 0x4a, 0x92, 0x9f, 0x82, 0xce, 0x1d, 0xcc, 0x94, 0x27, 0x5c, 0x89, + 0x00, 0x4d, 0x37, 0x5c, 0xb4, 0x4e, 0x16, 0x96, 0xac, 0x93, 0xec, 0x26, 0xa0, 0x66, 0x35, 0xb6, + 0xfa, 0xae, 0xd5, 0xec, 0xd1, 0x08, 0x97, 0x34, 0x09, 0xc2, 0x3e, 0x8e, 0xd7, 0x22, 0xf5, 0x48, + 0xd8, 0x8e, 0x05, 0x47, 0x95, 0x57, 0xad, 0x96, 0xa2, 0x53, 0x9f, 0x02, 0x68, 0xde, 0x69, 0x60, + 0x85, 0x24, 0x73, 0xbd, 0x44, 0xcd, 0x4f, 0xb9, 0x07, 0xbd, 0xd3, 0x03, 0x2f, 0x88, 0xbc, 0xac, + 0xeb, 0xb1, 0x97, 0x35, 0x16, 0xcf, 0xb2, 0xab, 0xc5, 0x33, 0xf5, 0x3d, 0x28, 0xe2, 0xb9, 0x6b, + 0x84, 0x06, 0xbb, 0x2d, 0xec, 0x9c, 0x5c, 0xee, 0x12, 0x06, 0xdf, 0xa4, 0x56, 0x61, 0xf9, 0xec, + 0x46, 0x2d, 0xa1, 0x3c, 0xaf, 0x4a, 0xa6, 0x8f, 0x98, 0x7f, 0x8b, 0x02, 0xc5, 0x49, 0xfe, 0x32, + 0x94, 0xb1, 0xb1, 0xe4, 0x69, 0x11, 0x2d, 0x2b, 0xf9, 0xde, 0x69, 0x13, 0xd3, 0xea, 0xbf, 0xcb, + 0x40, 0xa5, 0xef, 0x9b, 0x78, 0x70, 0x0c, 0x66, 0xd6, 0xf8, 0xb9, 0xd2, 0x24, 0x9e, 0xfb, 0x9e, + 0xe3, 0x18, 0xb1, 0x2c, 0x86, 0xe7, 0x7e, 0x04, 0x60, 0x1f, 0x40, 0x6e, 0xe2, 0x18, 0x47, 0xd4, + 0xd9, 0x58, 0xcb, 0x94, 0x8a, 0x8f, 0xbe, 0x77, 0x1d, 0xe3, 0x48, 0x23, 0x52, 0xf5, 0x37, 0xe3, + 0xfa, 0xc9, 0xe7, 0x22, 0x7b, 0x5a, 0xd6, 0xc8, 0xeb, 0x37, 0x68, 0x2a, 0x19, 0x56, 0x82, 0x5c, + 0xab, 0x3d, 0x68, 0x72, 0xdd, 0x12, 0xb5, 0xcc, 0x81, 0xbe, 0xdb, 0xd1, 0x06, 0x43, 0x25, 0x47, + 0x6e, 0x44, 0x02, 0x74, 0x1b, 0x83, 0xa1, 0x52, 0x62, 0x00, 0x85, 0xc3, 0x5e, 0xe7, 0xa7, 0x87, + 0x6d, 0x45, 0x51, 0xff, 0x55, 0x06, 0x20, 0x71, 0x08, 0xb0, 0xb7, 0xa1, 0x72, 0x4a, 0x29, 0x5d, + 0xf2, 0x14, 0xc9, 0x7d, 0x04, 0x8e, 0x26, 0x99, 0xe4, 0x5d, 0x49, 0xc5, 0xc0, 0xb3, 0x77, 0xd9, + 0x65, 0x54, 0x99, 0x25, 0xc7, 0x36, 0x7b, 0x07, 0x4a, 0x1e, 0xf6, 0x03, 0x49, 0xb3, 0xf2, 0xc1, + 0x2b, 0x75, 0x5f, 0x2b, 0x7a, 0x3c, 0x81, 0x67, 0xf4, 0xc4, 0x8f, 0x4c, 0x49, 0x31, 0xe9, 0x2e, + 0x82, 0x9a, 0x8e, 0x31, 0x0f, 0x2c, 0x8d, 0xe3, 0x63, 0xb6, 0x9b, 0x4f, 0xd8, 0xae, 0xfa, 0x33, + 0xa8, 0x0d, 0x8c, 0xe9, 0x8c, 0x33, 0x67, 0xea, 0x18, 0x83, 0x1c, 0xae, 0x09, 0xb1, 0x18, 0xe9, + 0x1b, 0xb7, 0xd8, 0x81, 0xe5, 0x8f, 0x2d, 0x37, 0xda, 0x91, 0x51, 0x12, 0x99, 0xed, 0x61, 0x60, + 0xbb, 0x47, 0x9a, 0x77, 0x1a, 0xc5, 0xf1, 0x44, 0x69, 0xf5, 0xef, 0x67, 0xa0, 0x22, 0x35, 0x83, + 0xbd, 0x97, 0xd2, 0x28, 0x5f, 0x5e, 0x6a, 0x27, 0xff, 0x96, 0x34, 0xcb, 0x37, 0x20, 0x1f, 0x84, + 0x86, 0x1f, 0xf9, 0x96, 0x14, 0x29, 0xc7, 0x8e, 0x37, 0x77, 0x4d, 0x8d, 0xa3, 0x99, 0x0a, 0x59, + 0xcb, 0x35, 0xc5, 0xb6, 0x58, 0xa6, 0x42, 0xa4, 0xba, 0x0d, 0xe5, 0xb8, 0x78, 0x5c, 0x02, 0x5a, + 0xff, 0xe9, 0x40, 0x59, 0x63, 0x65, 0xc8, 0x6b, 0x8d, 0xde, 0xa3, 0xb6, 0x92, 0x51, 0xff, 0x24, + 0x03, 0x90, 0xe4, 0x62, 0xf7, 0x52, 0xad, 0xbd, 0xbe, 0x58, 0xea, 0x3d, 0xfa, 0x2b, 0x35, 0xf6, + 0x06, 0x94, 0xe7, 0x2e, 0x01, 0x2d, 0x53, 0x9c, 0x3b, 0x09, 0x80, 0xdd, 0x80, 0x6c, 0x14, 0xf1, + 0xb3, 0x10, 0x65, 0xf1, 0xcc, 0x70, 0xd4, 0xcf, 0xa0, 0x1c, 0x17, 0xc7, 0x36, 0xa0, 0xbc, 0xdb, + 0xef, 0x76, 0xfb, 0x4f, 0x3b, 0xbd, 0x47, 0xca, 0x1a, 0x26, 0x0f, 0xb4, 0x76, 0xb3, 0xdd, 0xc2, + 0x64, 0x06, 0xd7, 0x6c, 0xf3, 0x50, 0xd3, 0xda, 0xbd, 0xa1, 0xae, 0xf5, 0x9f, 0x2a, 0xeb, 0xea, + 0x6f, 0xe7, 0x60, 0xab, 0xef, 0xb6, 0xe6, 0x33, 0xc7, 0x1e, 0x1b, 0xa1, 0xf5, 0xd8, 0x3a, 0x6f, + 0x86, 0x67, 0x78, 0x9c, 0x1a, 0x61, 0xe8, 0xf3, 0xcd, 0x5c, 0xd6, 0x78, 0x82, 0x1b, 0xe8, 0x02, + 0xcb, 0x0f, 0xc9, 0xfe, 0x28, 0xef, 0xe2, 0x1a, 0x87, 0x37, 0x3d, 0x87, 0xf6, 0x32, 0xfb, 0x31, + 0x5c, 0xe1, 0x46, 0x3d, 0x4e, 0x89, 0x42, 0xa7, 0x2e, 0x78, 0xcf, 0xe2, 0xd2, 0x65, 0x9c, 0x10, + 0xb3, 0x22, 0x19, 0x31, 0xb5, 0x5b, 0x50, 0x49, 0xb2, 0x73, 0xd5, 0xa0, 0xac, 0x41, 0x4c, 0x48, + 0x2d, 0xf1, 0x5c, 0xdd, 0x8c, 0x5a, 0xad, 0xdb, 0xe6, 0x19, 0xa9, 0x4b, 0x79, 0xad, 0xe6, 0x25, + 0x9d, 0xc1, 0x23, 0xf7, 0x4b, 0xd8, 0x4a, 0x51, 0x52, 0x2b, 0xb8, 0xc2, 0xf4, 0x4e, 0xe4, 0x2c, + 0x58, 0xe8, 0xbd, 0x0c, 0xc1, 0xe6, 0x70, 0x89, 0x70, 0xd3, 0x4b, 0x43, 0x91, 0x99, 0xd9, 0x81, + 0x6e, 0x1f, 0xb9, 0x9e, 0x6f, 0x09, 0xf6, 0x5e, 0xb2, 0x83, 0x0e, 0xa5, 0x13, 0x9d, 0x45, 0x72, + 0xb1, 0xf3, 0xd3, 0x24, 0xf2, 0x30, 0x73, 0xb4, 0xcd, 0xcf, 0xcb, 0x9c, 0x56, 0xa4, 0x74, 0xc7, + 0x44, 0x75, 0x9d, 0xa3, 0x22, 0x35, 0x04, 0x48, 0x0d, 0xa9, 0x12, 0xf0, 0x09, 0x87, 0x5d, 0xef, + 0xc1, 0xe5, 0x55, 0x8d, 0x5c, 0x21, 0x57, 0x6d, 0xcb, 0x72, 0xd5, 0x82, 0x01, 0x2b, 0x91, 0xb1, + 0xfe, 0x46, 0x16, 0xca, 0xdc, 0xce, 0x86, 0xb3, 0x7f, 0x07, 0x8a, 0xde, 0xe8, 0x6b, 0xdd, 0xb7, + 0x26, 0x17, 0xb9, 0xb0, 0x0b, 0xde, 0xe8, 0x6b, 0xcd, 0x9a, 0xb0, 0xb7, 0xa3, 0x23, 0xd2, 0xb4, + 0x26, 0xa2, 0x86, 0x5a, 0x5a, 0xb8, 0x16, 0x47, 0x26, 0xb7, 0x2a, 0x5d, 0x5a, 0x54, 0x45, 0x6d, + 0x93, 0xdb, 0x86, 0x73, 0xda, 0x56, 0x5a, 0x13, 0xed, 0x98, 0xc1, 0xc5, 0x36, 0x89, 0xdc, 0x85, + 0x36, 0x09, 0x76, 0x17, 0xb6, 0x3c, 0xc7, 0x4c, 0x6c, 0x22, 0x62, 0x65, 0xe0, 0x1a, 0xdd, 0xf4, + 0x1c, 0x33, 0xd1, 0xfd, 0xcd, 0x33, 0xa4, 0x75, 0xad, 0xd3, 0x05, 0xda, 0x02, 0xa7, 0x75, 0xad, + 0xd3, 0x14, 0xed, 0x03, 0xa8, 0x24, 0x4b, 0x3f, 0xa8, 0x17, 0x17, 0x94, 0x55, 0xcd, 0x9a, 0x08, + 0xbf, 0x17, 0xc4, 0x3b, 0x21, 0xc0, 0x4c, 0xdc, 0x4e, 0xca, 0x33, 0x95, 0x2e, 0xce, 0xc4, 0xc9, + 0xc8, 0x97, 0xf7, 0x8f, 0xd6, 0xa1, 0xdc, 0xe1, 0x65, 0x84, 0x67, 0xec, 0x55, 0xc8, 0x7e, 0xc7, + 0x34, 0x20, 0x0e, 0xbb, 0x61, 0x98, 0xa6, 0x6e, 0x4c, 0x26, 0xd6, 0x38, 0xb4, 0x4c, 0x1d, 0xc5, + 0x17, 0xc1, 0x41, 0x36, 0x0d, 0xd3, 0x6c, 0x08, 0x38, 0x71, 0x62, 0x6e, 0x35, 0x8a, 0xd4, 0x38, + 0xee, 0x27, 0xc9, 0x46, 0x56, 0x23, 0xa1, 0xc5, 0x71, 0x2f, 0x49, 0x6a, 0x66, 0x73, 0x3f, 0x6c, + 0x66, 0xf3, 0x2f, 0x3c, 0xb3, 0x85, 0x8b, 0x67, 0x36, 0x65, 0xc6, 0xc2, 0x99, 0x2a, 0xd2, 0x4c, + 0x25, 0x27, 0x63, 0xc7, 0x3c, 0x53, 0xff, 0x6e, 0x16, 0x40, 0xb3, 0x66, 0x8e, 0x31, 0xb6, 0xfe, + 0xcf, 0x19, 0xbd, 0x5b, 0xd2, 0x32, 0x71, 0xcd, 0x28, 0xce, 0x27, 0x5a, 0x12, 0x74, 0x96, 0xac, + 0x1c, 0xde, 0xc2, 0x0b, 0x0f, 0x6f, 0xf1, 0x05, 0x86, 0xb7, 0xb4, 0x3c, 0xbc, 0xec, 0x73, 0x78, + 0xc5, 0xb7, 0x4e, 0x7d, 0x3b, 0xb4, 0xf4, 0x89, 0xef, 0x4d, 0xf5, 0x14, 0x67, 0x45, 0xc6, 0x53, + 0xa6, 0xd1, 0xb8, 0x26, 0x88, 0x76, 0x7d, 0x6f, 0x9a, 0xe6, 0xae, 0xea, 0x7f, 0xcd, 0x43, 0xa5, + 0xe1, 0x1a, 0xce, 0xf9, 0xb7, 0x16, 0xc5, 0x02, 0x91, 0x27, 0x65, 0x36, 0x0f, 0xf9, 0xb8, 0x73, + 0xe7, 0x78, 0x99, 0x20, 0x34, 0xe2, 0xb7, 0xa0, 0xe2, 0xcd, 0xc3, 0x18, 0xcf, 0xdd, 0xe5, 0xc0, + 0x41, 0x44, 0x10, 0xe7, 0x8f, 0xbd, 0x74, 0x51, 0x7e, 0x52, 0xe6, 0x92, 0xfc, 0xb1, 0x80, 0x1f, + 0xe7, 0x27, 0x02, 0xe4, 0xb6, 0xf6, 0x94, 0x46, 0x3e, 0x98, 0x4f, 0x2d, 0x3e, 0xfa, 0x59, 0x1e, + 0x73, 0xd9, 0x14, 0x30, 0x2c, 0x65, 0x6a, 0x4d, 0x3d, 0xff, 0x9c, 0x97, 0x52, 0xe0, 0xa5, 0x70, + 0x10, 0x95, 0xf2, 0x0e, 0xb0, 0x53, 0xc3, 0x0e, 0xf5, 0x74, 0x51, 0x5c, 0xa9, 0x52, 0x10, 0x33, + 0x94, 0x8b, 0xbb, 0x0a, 0x05, 0xd3, 0x0e, 0x4e, 0x3a, 0x7d, 0xa1, 0x50, 0x89, 0x14, 0xf6, 0x25, + 0x18, 0x1b, 0x28, 0xe1, 0x85, 0x56, 0x40, 0x43, 0x99, 0xd5, 0xca, 0x08, 0xd9, 0x41, 0x00, 0x4a, + 0x08, 0xae, 0x15, 0x9e, 0x7a, 0x3e, 0xe6, 0xe4, 0xfa, 0x52, 0x02, 0x40, 0x49, 0x0a, 0x49, 0xb1, + 0x22, 0xb2, 0x50, 0x65, 0xb5, 0x38, 0x8d, 0x9a, 0x08, 0xe7, 0x4a, 0x84, 0xad, 0xf2, 0xe6, 0x27, + 0x10, 0x76, 0x1b, 0x6a, 0xd4, 0x7c, 0xd2, 0xa7, 0xb0, 0x0f, 0xe4, 0xd1, 0xce, 0x6a, 0x55, 0x84, + 0x92, 0xb1, 0x02, 0xa9, 0x3e, 0x85, 0x6b, 0xa9, 0xfe, 0xe9, 0x86, 0xef, 0x1b, 0xe7, 0xfa, 0xd4, + 0xf8, 0xda, 0xf3, 0xc9, 0x18, 0x95, 0xd5, 0xae, 0xca, 0xc3, 0xd6, 0x40, 0xf4, 0x3e, 0x62, 0x2f, + 0xcc, 0x6a, 0xbb, 0x9e, 0x4f, 0x96, 0xaa, 0x95, 0x59, 0x11, 0x4b, 0x26, 0x12, 0x9a, 0x60, 0x52, + 0xee, 0x02, 0x1e, 0xab, 0xab, 0x55, 0x08, 0xb6, 0x43, 0x20, 0x54, 0x81, 0x82, 0x07, 0x3a, 0x45, + 0xba, 0x6c, 0x89, 0x90, 0xba, 0x07, 0x14, 0xe6, 0xc8, 0x11, 0xc7, 0x96, 0x61, 0x92, 0x97, 0x9c, + 0x10, 0x7b, 0x96, 0x41, 0x31, 0x28, 0xc1, 0x03, 0x7d, 0x36, 0x0f, 0x79, 0x90, 0xad, 0x96, 0x0f, + 0x1e, 0x1c, 0xcc, 0x43, 0x01, 0x3e, 0xb2, 0x42, 0x0a, 0xad, 0x25, 0xf0, 0x23, 0x2b, 0xc4, 0x83, + 0x3e, 0x78, 0x10, 0x79, 0xbc, 0xae, 0x88, 0xb1, 0x7d, 0x20, 0x5c, 0x5a, 0x2a, 0x6c, 0xc4, 0x48, + 0x7d, 0x3a, 0xe7, 0x51, 0xb5, 0x59, 0xad, 0x12, 0x11, 0xec, 0xcf, 0x1d, 0xd5, 0x97, 0xbc, 0x23, + 0x07, 0xfe, 0xdc, 0xb5, 0xb8, 0x3d, 0x89, 0x3e, 0x4d, 0xe1, 0xa7, 0x8e, 0xd3, 0xac, 0x05, 0x97, + 0xb8, 0x1a, 0x69, 0x49, 0xe7, 0x58, 0x14, 0x27, 0xb6, 0xd2, 0x6b, 0xc0, 0x22, 0xfa, 0x18, 0x1c, + 0xa8, 0x3f, 0xcf, 0xc0, 0xf5, 0x3e, 0x39, 0xcd, 0x69, 0x93, 0xef, 0x5b, 0x41, 0x60, 0x1c, 0x59, + 0xbb, 0x9e, 0xbf, 0x3b, 0xff, 0xf6, 0xdb, 0x73, 0x76, 0x07, 0x36, 0x0f, 0x0c, 0xdf, 0x72, 0xc3, + 0x98, 0x05, 0x08, 0xa1, 0x61, 0x11, 0xcc, 0x1e, 0x92, 0x65, 0xde, 0x72, 0xc3, 0xc3, 0x58, 0xfc, + 0x12, 0x6d, 0x49, 0xdb, 0x6a, 0x97, 0xa8, 0xd4, 0xff, 0xb1, 0x0d, 0xb9, 0x9e, 0x67, 0x5a, 0xec, + 0x7d, 0x28, 0x53, 0x90, 0xe7, 0xb2, 0x43, 0x08, 0xd1, 0xf4, 0x87, 0x24, 0xe1, 0x92, 0x2b, 0xbe, + 0x2e, 0x0e, 0x0b, 0x7d, 0x95, 0x64, 0x7a, 0xf2, 0x28, 0x23, 0x53, 0xad, 0x08, 0x2b, 0x03, 0xa9, + 0xc9, 0x1c, 0x83, 0x63, 0x4b, 0x56, 0x52, 0xdf, 0x72, 0x49, 0x52, 0xc8, 0x6b, 0x71, 0x9a, 0x34, + 0x29, 0xdf, 0xc3, 0x03, 0x80, 0xaf, 0x97, 0xfc, 0x0a, 0x4d, 0x8a, 0xe3, 0x69, 0x01, 0xbd, 0x0f, + 0xe5, 0xaf, 0x3d, 0xdb, 0xe5, 0x0d, 0x2f, 0x2c, 0x35, 0xfc, 0x0b, 0xcf, 0xe6, 0x9e, 0xac, 0xd2, + 0xd7, 0xe2, 0x8b, 0xbd, 0x06, 0x45, 0xcf, 0xe5, 0x65, 0x17, 0x97, 0xca, 0x2e, 0x78, 0x6e, 0x97, + 0x47, 0x5c, 0x6d, 0x8c, 0xe6, 0xb6, 0x63, 0x22, 0xff, 0x74, 0xac, 0x49, 0x28, 0x1c, 0x37, 0x15, + 0x02, 0xf6, 0xdd, 0xae, 0x35, 0x09, 0x51, 0x41, 0x9c, 0xd8, 0x0e, 0x9e, 0x33, 0x54, 0x58, 0x79, + 0xa9, 0x30, 0xe0, 0x68, 0x2a, 0xf0, 0x75, 0x28, 0x1d, 0xf9, 0xde, 0x7c, 0x86, 0x1a, 0x1f, 0x2c, + 0xbb, 0x4a, 0x08, 0xb7, 0x73, 0x8e, 0xcc, 0x8e, 0x3e, 0x6d, 0xf7, 0x48, 0x27, 0xe5, 0xb8, 0xb2, + 0x9d, 0xbd, 0x53, 0xd2, 0xaa, 0x11, 0x90, 0xd4, 0xde, 0xd7, 0xa1, 0x64, 0x1c, 0x1d, 0xe9, 0x22, + 0x70, 0x6c, 0xa9, 0x2c, 0xe3, 0xe8, 0x88, 0xaa, 0xbc, 0x07, 0x1b, 0xa7, 0xb6, 0xab, 0x07, 0x33, + 0x6b, 0xcc, 0x69, 0x37, 0x96, 0x87, 0xf2, 0xd4, 0x76, 0x51, 0x27, 0x24, 0x7a, 0x59, 0x29, 0xad, + 0x3d, 0x57, 0x29, 0xdd, 0x86, 0xbc, 0x63, 0x4f, 0xed, 0x50, 0x84, 0x92, 0xa5, 0xa4, 0x56, 0x42, + 0x30, 0x15, 0x0a, 0xc2, 0x1a, 0xaa, 0x2c, 0x91, 0x08, 0x4c, 0xfa, 0x14, 0xde, 0x7a, 0xce, 0x29, + 0x2c, 0x09, 0xbd, 0xec, 0xbb, 0x85, 0xde, 0x8f, 0xc8, 0x65, 0x64, 0xb9, 0xa1, 0x1e, 0x65, 0xb8, + 0xb4, 0x3a, 0x43, 0x95, 0x93, 0xf5, 0x79, 0xb6, 0x0f, 0xa0, 0xe2, 0x93, 0xb5, 0x44, 0x27, 0xd3, + 0xca, 0x65, 0x59, 0xdd, 0x4c, 0xcc, 0x28, 0x1a, 0xf8, 0x89, 0x49, 0xe5, 0x21, 0x30, 0x21, 0x75, + 0xca, 0x72, 0xe4, 0x95, 0xa5, 0x91, 0x16, 0x6a, 0x59, 0x2b, 0x96, 0x22, 0x71, 0xaa, 0x79, 0x2c, + 0x0c, 0x8f, 0x58, 0x08, 0x88, 0x2d, 0x95, 0xb5, 0x2a, 0x01, 0x79, 0x34, 0x43, 0xc0, 0xee, 0x01, + 0x44, 0xe5, 0x86, 0x67, 0x14, 0xf0, 0x1f, 0x77, 0x42, 0x14, 0x15, 0x9e, 0x69, 0x65, 0x33, 0xfa, + 0x44, 0x5e, 0x3c, 0xb2, 0x5d, 0x13, 0x97, 0x4f, 0x68, 0x1c, 0x05, 0xf5, 0x3a, 0xed, 0xae, 0x8a, + 0x80, 0x0d, 0x8d, 0xa3, 0x80, 0x7d, 0x08, 0x55, 0x83, 0x1f, 0xef, 0x3c, 0x28, 0xf8, 0x9a, 0x6c, + 0x54, 0x90, 0x0e, 0x7e, 0xad, 0x62, 0x48, 0x52, 0xc0, 0x27, 0xc0, 0x22, 0x27, 0x0d, 0x29, 0x68, + 0x7c, 0x45, 0x5d, 0x5f, 0xea, 0xe7, 0xa6, 0xf0, 0xd2, 0xc4, 0x81, 0xec, 0x9f, 0xc0, 0x46, 0x5a, + 0x1c, 0xbb, 0xb1, 0xc2, 0x2d, 0x41, 0x93, 0xad, 0x55, 0xc7, 0xb2, 0x80, 0xf6, 0x1a, 0x6c, 0xb8, + 0x5e, 0xa8, 0x8f, 0x8d, 0xf1, 0xb1, 0x45, 0x19, 0xb9, 0xe9, 0xbd, 0xea, 0x7a, 0x61, 0x33, 0x82, + 0xe1, 0xf8, 0x44, 0x42, 0x7f, 0x78, 0x46, 0x56, 0xf7, 0x78, 0x7c, 0x62, 0x09, 0x1d, 0xa5, 0x8d, + 0x48, 0x58, 0xc7, 0x19, 0xe6, 0xc2, 0x27, 0x65, 0xb8, 0x95, 0x9a, 0xe1, 0x58, 0x2a, 0xd5, 0xc0, + 0x4f, 0x24, 0xd4, 0x5b, 0x50, 0x09, 0xbc, 0xb9, 0x3f, 0xb6, 0xf4, 0x20, 0xb4, 0x66, 0xf5, 0x6d, + 0x1a, 0x51, 0xe0, 0xa0, 0x41, 0x68, 0xcd, 0xd8, 0x43, 0xa8, 0xcd, 0x7c, 0x4b, 0x97, 0xe6, 0xe9, + 0x55, 0xb9, 0x8b, 0x07, 0xbe, 0x95, 0x4c, 0x55, 0x75, 0x26, 0xa5, 0xa2, 0x9c, 0x52, 0x0f, 0xd4, + 0x85, 0x9c, 0x49, 0x27, 0x30, 0x67, 0xa2, 0x74, 0xfc, 0x04, 0xb6, 0xa4, 0x9c, 0xf3, 0x13, 0xca, + 0xfc, 0x5a, 0xca, 0x4b, 0x14, 0x91, 0x1f, 0x9e, 0x60, 0xf6, 0xda, 0x2c, 0x95, 0x66, 0x8d, 0x05, + 0xf5, 0x1c, 0x45, 0xc3, 0xdb, 0x94, 0xff, 0xa5, 0x0b, 0x74, 0xee, 0x94, 0xde, 0xfe, 0x98, 0xfb, + 0x03, 0x3a, 0x41, 0xdb, 0x35, 0xeb, 0xaf, 0xf3, 0xdb, 0x26, 0x94, 0x60, 0x0f, 0xa0, 0xca, 0x85, + 0x14, 0x8a, 0x74, 0x0d, 0xea, 0x6f, 0xc8, 0xf6, 0x49, 0x92, 0x54, 0x08, 0xa1, 0x55, 0x9c, 0xf8, + 0x3b, 0x60, 0x1f, 0xc3, 0x16, 0x37, 0x15, 0xcb, 0x0c, 0xf5, 0xcd, 0xe5, 0xc5, 0x45, 0x44, 0xbb, + 0x09, 0x57, 0xd5, 0xe0, 0x9a, 0x3f, 0x77, 0x49, 0x70, 0x11, 0x39, 0x67, 0xbe, 0x37, 0xb2, 0x78, + 0xfe, 0x3b, 0x94, 0x5f, 0x74, 0x47, 0xe3, 0x64, 0x3c, 0x2f, 0x71, 0xb2, 0xab, 0xbe, 0x0c, 0x3a, + 0xc0, 0x7c, 0x17, 0x94, 0xc9, 0x4f, 0x02, 0x2a, 0xf3, 0xad, 0x17, 0x29, 0x73, 0x07, 0xf3, 0x51, + 0x99, 0x0c, 0x72, 0xf3, 0xb9, 0x6d, 0xd6, 0xef, 0xf2, 0xa0, 0x54, 0xfc, 0x66, 0xaf, 0x43, 0xcd, + 0xb7, 0xc6, 0x73, 0x3f, 0xb0, 0x9f, 0x59, 0x7a, 0x60, 0xbb, 0x27, 0xf5, 0xb7, 0x69, 0x1c, 0x37, + 0x62, 0xe8, 0xc0, 0x76, 0x4f, 0x70, 0xc5, 0x5a, 0x67, 0xa1, 0xe5, 0xbb, 0x3a, 0x0a, 0x8b, 0xf5, + 0x77, 0xe4, 0x15, 0xdb, 0x26, 0xc4, 0x60, 0x6c, 0xb8, 0x1a, 0x58, 0xf1, 0x37, 0xfb, 0x31, 0x6c, + 0x26, 0x8a, 0xc2, 0x0c, 0x45, 0x96, 0xfa, 0xbb, 0x2b, 0x1d, 0x88, 0x24, 0xce, 0x68, 0x89, 0x77, + 0x9d, 0x4b, 0x3e, 0xe9, 0xb5, 0x15, 0xf0, 0xb5, 0x75, 0xef, 0x7b, 0xad, 0xad, 0x01, 0xad, 0xad, + 0x37, 0xa0, 0x64, 0xbb, 0xa1, 0xe5, 0x3f, 0x33, 0x9c, 0xfa, 0x7b, 0x4b, 0xac, 0x3f, 0xc6, 0xb1, + 0xdb, 0x50, 0x0c, 0x1c, 0x1b, 0x19, 0x53, 0xfd, 0xfd, 0x25, 0xb2, 0x08, 0xc5, 0xee, 0x40, 0x39, + 0xbe, 0x5e, 0x55, 0xff, 0x60, 0x89, 0x2e, 0x41, 0xb2, 0x9b, 0x90, 0x3b, 0xc5, 0xf5, 0x78, 0x7f, + 0xd9, 0x1c, 0x8d, 0x70, 0x94, 0x15, 0x26, 0xb6, 0xe3, 0x70, 0x59, 0xe1, 0xc1, 0x92, 0xac, 0xb0, + 0x6b, 0x3b, 0x0e, 0x97, 0x15, 0x26, 0xe2, 0x0b, 0x4f, 0x5a, 0xca, 0x81, 0x3d, 0xf9, 0x70, 0xf9, + 0xa4, 0x45, 0xdc, 0x13, 0xba, 0x88, 0x56, 0x09, 0xc8, 0xc6, 0xca, 0x4d, 0xc5, 0x1f, 0xc9, 0x63, + 0x95, 0x36, 0xbe, 0x6a, 0x10, 0xc4, 0x69, 0xd4, 0x26, 0x84, 0x85, 0x19, 0xb5, 0xb9, 0x8f, 0xf9, + 0xfd, 0x08, 0x0e, 0x41, 0x55, 0xee, 0x7d, 0xd8, 0x88, 0x42, 0xab, 0xb0, 0xba, 0xa0, 0xfe, 0xc9, + 0x52, 0x0b, 0xd2, 0x04, 0xac, 0x05, 0xd5, 0x09, 0xca, 0x8e, 0x53, 0x2e, 0x4a, 0xd6, 0x1f, 0x52, + 0x43, 0xb6, 0xa3, 0x53, 0xfc, 0x22, 0x51, 0x53, 0x4b, 0xe5, 0x62, 0xf7, 0x80, 0xd9, 0x13, 0x3e, + 0x9f, 0xa8, 0x1e, 0x72, 0x71, 0xb1, 0xfe, 0x29, 0x2d, 0xce, 0x15, 0x18, 0xf6, 0x00, 0x36, 0x02, + 0xcb, 0x35, 0xf5, 0x69, 0x20, 0x64, 0x92, 0xcf, 0xa8, 0x9d, 0x82, 0x0d, 0xc7, 0xd7, 0x30, 0xb5, + 0x0a, 0x52, 0xed, 0x07, 0x5c, 0x38, 0x79, 0x00, 0xb8, 0xce, 0x9f, 0x25, 0x99, 0xfe, 0xaf, 0x0b, + 0x32, 0x21, 0x55, 0x94, 0xe9, 0x13, 0xd8, 0xe4, 0x91, 0x69, 0xb8, 0x24, 0x79, 0xb6, 0x1f, 0xcb, + 0xd9, 0x62, 0xfb, 0x98, 0xb6, 0x31, 0x8f, 0x3e, 0xa3, 0xda, 0x48, 0x6f, 0x0b, 0x5c, 0x63, 0x16, + 0x1c, 0x7b, 0x61, 0xfd, 0x37, 0x64, 0x51, 0x63, 0x20, 0xa0, 0x5a, 0x15, 0x89, 0xa2, 0x14, 0x1e, + 0x40, 0xc9, 0x06, 0x1d, 0x87, 0x56, 0xfd, 0x27, 0xfc, 0x00, 0x8a, 0x81, 0xcd, 0x10, 0x3b, 0x0f, + 0xc6, 0x6c, 0xe6, 0x9c, 0xf3, 0x45, 0xf5, 0x39, 0x2d, 0xaa, 0xcb, 0xd2, 0xa2, 0x6a, 0x20, 0x92, + 0x56, 0x55, 0xd9, 0x88, 0x3e, 0xd9, 0x7d, 0xa8, 0xce, 0xbc, 0x20, 0xd4, 0xcd, 0xa9, 0x43, 0x9b, + 0xab, 0x21, 0x6f, 0xea, 0x03, 0x2f, 0x08, 0x5b, 0x53, 0x87, 0x8e, 0xa1, 0x59, 0xfc, 0xcd, 0xba, + 0x70, 0x29, 0xc5, 0xb0, 0x0d, 0xf2, 0xb4, 0xd6, 0x77, 0xa8, 0xc6, 0x1b, 0x52, 0x8d, 0x12, 0xe3, + 0x16, 0x11, 0x7a, 0x5b, 0xde, 0x22, 0x08, 0xf5, 0x49, 0xd3, 0x32, 0xe7, 0xb3, 0x24, 0x4c, 0xb5, + 0xc9, 0xa5, 0x0f, 0x82, 0x46, 0x71, 0xaa, 0x0f, 0x61, 0x33, 0xa1, 0xc2, 0x0e, 0x06, 0xf5, 0x96, + 0xbc, 0x06, 0xa5, 0x60, 0xf2, 0x8d, 0x28, 0x23, 0xc2, 0x02, 0xf5, 0xcf, 0xf3, 0x50, 0x8a, 0x94, + 0x06, 0x56, 0x81, 0xe2, 0x61, 0xef, 0x71, 0xaf, 0xff, 0xb4, 0xc7, 0x2f, 0x75, 0x35, 0x06, 0x83, + 0xb6, 0x36, 0x54, 0x4c, 0x56, 0x03, 0xa0, 0x4b, 0x2b, 0xfa, 0xa0, 0xd9, 0xe8, 0xf1, 0x4b, 0x5e, + 0x74, 0x55, 0x86, 0xa7, 0xd7, 0xd9, 0x16, 0x6c, 0xec, 0x1e, 0xf6, 0x28, 0x10, 0x90, 0x83, 0xb2, + 0x08, 0x6a, 0x7f, 0xc9, 0x5d, 0x3e, 0x1c, 0x94, 0x43, 0xd0, 0x7e, 0x63, 0xd8, 0xd6, 0x3a, 0x11, + 0x28, 0x4f, 0x31, 0x85, 0xfd, 0x43, 0xad, 0x29, 0x4a, 0x2a, 0xb0, 0x2b, 0xb0, 0x15, 0x67, 0x8b, + 0x8a, 0x54, 0x8a, 0xd8, 0xb2, 0x03, 0xad, 0xff, 0x45, 0xbb, 0x39, 0x54, 0x80, 0xfc, 0x47, 0x8f, + 0x1e, 0x29, 0x15, 0x56, 0x85, 0x52, 0xab, 0x33, 0x18, 0x76, 0x7a, 0xcd, 0xa1, 0x52, 0xc5, 0x06, + 0xef, 0x76, 0xba, 0xc3, 0xb6, 0xa6, 0x6c, 0xb0, 0x12, 0xe4, 0xbe, 0xe8, 0x77, 0x7a, 0x4a, 0x8d, + 0x2e, 0xef, 0x34, 0xf6, 0x0f, 0xba, 0x6d, 0x65, 0x13, 0xa1, 0x83, 0xbe, 0x36, 0x54, 0x14, 0x84, + 0x3e, 0xed, 0xf4, 0x5a, 0xfd, 0xa7, 0xca, 0x16, 0x2b, 0x43, 0xfe, 0xb0, 0x87, 0xd5, 0x30, 0xb6, + 0x01, 0x65, 0xfa, 0xd4, 0x1b, 0xdd, 0xae, 0x72, 0x49, 0x72, 0x3a, 0x5d, 0x46, 0x14, 0xb9, 0xb0, + 0x06, 0xd8, 0x86, 0x2b, 0xd8, 0x97, 0x38, 0x49, 0xd4, 0x57, 0xb1, 0x9c, 0xfd, 0x4e, 0xef, 0x70, + 0xa0, 0xbc, 0x84, 0xc4, 0xf4, 0x49, 0x98, 0x3a, 0x96, 0xd3, 0xe9, 0xd1, 0x50, 0xde, 0xc4, 0xef, + 0x56, 0xbb, 0xdb, 0x1e, 0xb6, 0x95, 0x5b, 0xd8, 0x2b, 0xad, 0x7d, 0xd0, 0x6d, 0x34, 0xdb, 0xca, + 0x36, 0x26, 0xba, 0xfd, 0xe6, 0x63, 0xbd, 0x7f, 0xa0, 0xbc, 0xca, 0x2e, 0x83, 0xd2, 0xef, 0xe9, + 0xad, 0xc3, 0x83, 0x6e, 0xa7, 0xd9, 0x18, 0xb6, 0xf5, 0xc7, 0xed, 0xaf, 0x14, 0x15, 0x87, 0xfd, + 0x40, 0x6b, 0xeb, 0xa2, 0xac, 0xd7, 0xa2, 0xb4, 0x28, 0xef, 0x36, 0x53, 0xa0, 0xba, 0x7b, 0xf8, + 0xb3, 0x9f, 0x7d, 0xa5, 0x8b, 0x71, 0x78, 0x1d, 0x9b, 0x99, 0xe4, 0xd0, 0x0f, 0x1f, 0x2b, 0x6f, + 0x2c, 0x80, 0x06, 0x8f, 0x95, 0x37, 0x71, 0x1c, 0xa3, 0x89, 0x51, 0xee, 0x20, 0x81, 0xd6, 0x6e, + 0x1e, 0x6a, 0x83, 0xce, 0x93, 0xb6, 0xde, 0x1c, 0xb6, 0x95, 0xb7, 0x68, 0xe0, 0x3a, 0xbd, 0xc7, + 0xca, 0x5d, 0xec, 0x19, 0x7e, 0xf1, 0xe9, 0x7a, 0x9b, 0x31, 0xa8, 0x25, 0xb4, 0x04, 0x7b, 0x07, + 0x49, 0x76, 0xb4, 0x7e, 0xa3, 0xd5, 0x6c, 0x0c, 0x86, 0xca, 0xbb, 0x38, 0x2c, 0x83, 0x83, 0x6e, + 0x67, 0xa8, 0xdc, 0xc3, 0xbe, 0x3f, 0x6a, 0x0c, 0xf7, 0xda, 0x9a, 0xf2, 0x1e, 0xce, 0xfc, 0xb0, + 0xb3, 0xdf, 0xd6, 0xc5, 0x34, 0xdc, 0xc7, 0x3a, 0x76, 0x3b, 0xdd, 0xae, 0xf2, 0x80, 0xfc, 0x2c, + 0x0d, 0x6d, 0xd8, 0xa1, 0xb9, 0xff, 0x10, 0x0b, 0x68, 0x1c, 0x1c, 0x74, 0xbf, 0x52, 0x3e, 0xc2, + 0x0e, 0xee, 0x1f, 0x76, 0x87, 0x1d, 0xfd, 0xf0, 0xa0, 0xd5, 0x18, 0xb6, 0x95, 0x8f, 0x69, 0x61, + 0xf4, 0x07, 0xc3, 0xd6, 0x7e, 0x57, 0xf9, 0x44, 0xfd, 0x2d, 0x28, 0x45, 0x7a, 0x24, 0xe6, 0xea, + 0xf4, 0x7a, 0x6d, 0x4d, 0x59, 0xc3, 0x92, 0xbb, 0xed, 0xdd, 0xa1, 0x92, 0x21, 0x1f, 0x53, 0xe7, + 0xd1, 0xde, 0x50, 0x59, 0xc7, 0xcf, 0xfe, 0x21, 0x0e, 0x52, 0x96, 0x7a, 0xd7, 0xde, 0xef, 0x28, + 0x39, 0xfc, 0x6a, 0xf4, 0x86, 0x1d, 0x25, 0x4f, 0xcb, 0xa6, 0xd3, 0x7b, 0xd4, 0x6d, 0x2b, 0x05, + 0x84, 0xee, 0x37, 0xb4, 0xc7, 0x4a, 0x91, 0x17, 0xda, 0x6a, 0x7f, 0xa9, 0x94, 0x58, 0x01, 0xd6, + 0xbb, 0xf7, 0x95, 0x32, 0x82, 0x5a, 0xed, 0xd6, 0xe1, 0x81, 0x02, 0xea, 0x1d, 0x28, 0x36, 0x8e, + 0x8e, 0xf6, 0x51, 0x4d, 0xc7, 0xce, 0x1c, 0x76, 0xbb, 0x7c, 0x1b, 0xed, 0xf4, 0x87, 0xc3, 0xfe, + 0xbe, 0x92, 0xc1, 0x85, 0x3b, 0xec, 0x1f, 0x28, 0xeb, 0x6a, 0x07, 0x4a, 0xd1, 0x21, 0x26, 0xdd, + 0x47, 0x2b, 0x41, 0xee, 0x40, 0x6b, 0x3f, 0xe1, 0x8e, 0xd1, 0x5e, 0xfb, 0x4b, 0x6c, 0x26, 0x7e, + 0x61, 0x41, 0x59, 0xac, 0x88, 0x5f, 0x1c, 0xa3, 0x0b, 0x69, 0xdd, 0x4e, 0xaf, 0xdd, 0xd0, 0x94, + 0xbc, 0xfa, 0x71, 0xca, 0xe7, 0x24, 0xb8, 0x46, 0x19, 0xf2, 0x6d, 0x4d, 0xeb, 0x8b, 0xbb, 0x99, + 0x9d, 0x47, 0xbd, 0xbe, 0xd6, 0xe6, 0x57, 0xdc, 0xc4, 0xc0, 0xad, 0xab, 0x6f, 0x43, 0x39, 0x66, + 0x79, 0xb8, 0x90, 0x9a, 0x5a, 0x7f, 0x30, 0xe0, 0xe3, 0xbc, 0x86, 0x69, 0x1a, 0x1c, 0x9e, 0xce, + 0xa8, 0xff, 0x0f, 0x94, 0x62, 0x6e, 0x7b, 0x1b, 0xd6, 0x87, 0x03, 0x61, 0x07, 0xbe, 0x7c, 0x2f, + 0x79, 0x89, 0x60, 0x18, 0x7d, 0x69, 0xeb, 0xc3, 0x01, 0x7b, 0x07, 0x0a, 0xfc, 0x1e, 0xa2, 0x70, + 0x65, 0x5c, 0x4e, 0x73, 0xf0, 0x21, 0xe1, 0x34, 0x41, 0xa3, 0x76, 0xa1, 0x96, 0xc6, 0xb0, 0x9b, + 0x00, 0x1c, 0x27, 0xd9, 0x53, 0x24, 0x08, 0xbb, 0x0e, 0xd1, 0x3d, 0xc7, 0x96, 0x08, 0x1e, 0x8c, + 0xd3, 0xea, 0xdf, 0xc9, 0x02, 0x24, 0x12, 0x17, 0xca, 0x74, 0xb1, 0xb5, 0x24, 0x2f, 0x5c, 0x84, + 0x2f, 0x43, 0xd9, 0xf1, 0x0c, 0x53, 0x7e, 0x51, 0xa0, 0x84, 0x00, 0x1a, 0x0d, 0xf9, 0x36, 0x53, + 0x99, 0xfb, 0xe7, 0xd9, 0x55, 0x28, 0x4c, 0x3c, 0x7f, 0x6a, 0x44, 0x61, 0x86, 0x22, 0x85, 0x67, + 0x0f, 0x77, 0x5b, 0xa1, 0xdc, 0xe9, 0xd2, 0x4d, 0x01, 0x8a, 0x59, 0x15, 0xc0, 0x2e, 0xc2, 0x50, + 0x33, 0xb1, 0xdc, 0xb1, 0xe3, 0x05, 0x96, 0x89, 0x3a, 0x7b, 0x81, 0x84, 0x4b, 0x88, 0x40, 0x3b, + 0xe7, 0xbc, 0xb7, 0xfe, 0xd4, 0x76, 0x8d, 0x50, 0x18, 0x3b, 0xa9, 0xb7, 0x11, 0x04, 0x9b, 0xfb, + 0x75, 0xe0, 0x09, 0xe3, 0x09, 0xf7, 0x80, 0x95, 0x10, 0x40, 0xcd, 0x7d, 0x05, 0xc0, 0x0a, 0xc6, + 0xc6, 0x8c, 0x17, 0x5e, 0xa6, 0xc2, 0xcb, 0x02, 0xb2, 0x73, 0xce, 0xba, 0x50, 0x1b, 0x8e, 0x90, + 0xdf, 0x7b, 0xa8, 0x07, 0x37, 0x3d, 0x47, 0x98, 0x35, 0x6e, 0x2f, 0x8a, 0xa6, 0xf7, 0xd2, 0x64, + 0xdc, 0x55, 0xb7, 0x90, 0xf7, 0x7a, 0x03, 0x2e, 0xad, 0x20, 0x7b, 0xa1, 0x20, 0xa4, 0xbf, 0x95, + 0x03, 0x48, 0xf4, 0x8b, 0x94, 0xff, 0x2e, 0x93, 0xf6, 0xdf, 0xdd, 0x87, 0xab, 0xe2, 0x22, 0x90, + 0xb8, 0xf0, 0x71, 0xa6, 0xdb, 0xae, 0x3e, 0x32, 0x22, 0x57, 0x29, 0x13, 0x58, 0x1e, 0x12, 0xd4, + 0x71, 0x77, 0x8c, 0x10, 0x8f, 0x42, 0x39, 0x4f, 0x78, 0x3e, 0x4b, 0xbb, 0x7a, 0xe5, 0xa3, 0x30, + 0xc9, 0x3e, 0x3c, 0x9f, 0xb1, 0xf7, 0xe1, 0x8a, 0x6f, 0x4d, 0x7c, 0x2b, 0x38, 0xd6, 0xc3, 0x40, + 0xae, 0x8c, 0xc7, 0x1f, 0x6d, 0x09, 0xe4, 0x30, 0x88, 0xeb, 0x7a, 0x1f, 0xae, 0x08, 0xcd, 0x63, + 0xa1, 0x79, 0xdc, 0x4b, 0xb6, 0xc5, 0x91, 0x72, 0xeb, 0x5e, 0x01, 0x10, 0x4a, 0x57, 0xf4, 0x52, + 0x46, 0x49, 0x2b, 0x73, 0x05, 0x0b, 0xb5, 0xe4, 0x77, 0x80, 0xd9, 0x81, 0xbe, 0xe0, 0x70, 0x10, + 0x0e, 0x51, 0xc5, 0x0e, 0x0e, 0x52, 0xce, 0x86, 0x8b, 0x7c, 0x19, 0xa5, 0x8b, 0x7c, 0x19, 0x97, + 0x21, 0x4f, 0x7a, 0x99, 0x70, 0x2d, 0xf0, 0x04, 0x53, 0x21, 0x87, 0x2c, 0x8b, 0xcc, 0xe0, 0xb5, + 0xfb, 0xb5, 0x7b, 0xf4, 0xce, 0x08, 0xce, 0x0f, 0x42, 0x35, 0xc2, 0xb1, 0x77, 0xe1, 0x92, 0x3c, + 0xa8, 0xd1, 0x25, 0xfa, 0x0a, 0x75, 0x53, 0x49, 0x86, 0x51, 0xe3, 0xd7, 0xe9, 0xdf, 0x06, 0x26, + 0x8d, 0x4b, 0x44, 0x5d, 0xe5, 0xee, 0xc0, 0x78, 0x50, 0x38, 0xb1, 0xfa, 0xfb, 0x19, 0xa8, 0xa5, + 0xf5, 0x14, 0x1e, 0xd4, 0x9b, 0x44, 0x2b, 0xe7, 0x93, 0x08, 0xe5, 0x97, 0xa1, 0x3c, 0x3b, 0x11, + 0xa1, 0xc9, 0xd1, 0xbe, 0x9d, 0x9d, 0xf0, 0x90, 0x64, 0xf6, 0x16, 0x14, 0x67, 0x27, 0x7c, 0x8f, + 0x5c, 0x34, 0xe5, 0x85, 0x19, 0x8f, 0x16, 0x7c, 0x0b, 0x8a, 0x73, 0x41, 0x9a, 0xbb, 0x88, 0x74, + 0x4e, 0xa4, 0xea, 0x36, 0x54, 0x65, 0xcb, 0x00, 0x2e, 0x75, 0xd4, 0x02, 0x78, 0xc3, 0xf0, 0x53, + 0xfd, 0xed, 0x75, 0x22, 0x79, 0x21, 0x4f, 0xe3, 0x0b, 0x79, 0x7b, 0xb7, 0x29, 0xbc, 0x49, 0xa7, + 0xe0, 0xc5, 0xb1, 0x17, 0xbd, 0xd1, 0x00, 0xc7, 0x46, 0xd0, 0x98, 0x87, 0x5e, 0xd3, 0x73, 0x84, + 0x03, 0x5d, 0xdc, 0x06, 0xc9, 0x45, 0x56, 0x70, 0x71, 0x51, 0xec, 0x7d, 0x71, 0x65, 0x82, 0xee, + 0x2b, 0x91, 0xd3, 0x3e, 0xbf, 0xa4, 0x78, 0x55, 0xa3, 0xeb, 0x4a, 0xe4, 0x8f, 0xbf, 0x0f, 0x9b, + 0x49, 0x7c, 0x6a, 0xe4, 0xe7, 0x5f, 0xcc, 0xb2, 0x11, 0x07, 0xa7, 0x62, 0x52, 0xfd, 0xdd, 0x0c, + 0x6c, 0x2d, 0x29, 0xda, 0x38, 0x5a, 0xc9, 0x73, 0x2e, 0xf8, 0xc9, 0x5e, 0x85, 0xea, 0xd4, 0x08, + 0xc7, 0xc7, 0xfa, 0xcc, 0xb7, 0x26, 0xf6, 0x59, 0xf4, 0x26, 0x0d, 0xc1, 0x0e, 0x08, 0x44, 0x31, + 0x0b, 0xb3, 0x19, 0x99, 0x17, 0xa6, 0x76, 0x28, 0x2e, 0x8c, 0x01, 0x81, 0xba, 0x64, 0xb1, 0x8c, + 0xe2, 0x99, 0x72, 0x17, 0x84, 0x5f, 0xdd, 0x80, 0x42, 0x27, 0x56, 0xe8, 0xe3, 0xe7, 0x19, 0xb2, + 0xe2, 0x49, 0x06, 0x0f, 0xca, 0x4d, 0x7a, 0xde, 0x61, 0xdf, 0x98, 0xb1, 0xbb, 0x90, 0x9d, 0x1a, + 0x33, 0x11, 0x69, 0x55, 0x8f, 0xcd, 0xf0, 0x1c, 0x7b, 0x6f, 0xdf, 0x98, 0x71, 0x3e, 0x88, 0x44, + 0xd7, 0x3f, 0x86, 0x52, 0x04, 0x78, 0x21, 0x8e, 0xf7, 0xef, 0xb3, 0x50, 0x6e, 0xc9, 0xa6, 0x3f, + 0x54, 0x71, 0x42, 0x7f, 0xee, 0xe2, 0x91, 0x2d, 0x9c, 0x16, 0x95, 0xb1, 0xe1, 0x0e, 0x05, 0x28, + 0x5a, 0x40, 0xeb, 0xdf, 0xb1, 0x80, 0x6e, 0x00, 0xf8, 0xa4, 0x90, 0x92, 0x4e, 0x9a, 0x8d, 0x03, + 0xc0, 0x3a, 0xa6, 0xf0, 0xc7, 0x2f, 0xbb, 0x62, 0x73, 0xdf, 0xdf, 0x15, 0x9b, 0x5f, 0xe9, 0x8a, + 0xfd, 0xdf, 0xc6, 0x79, 0xfa, 0x46, 0xc2, 0xe4, 0x71, 0x4d, 0x23, 0x59, 0x99, 0x07, 0x89, 0xce, + 0xe2, 0x20, 0x72, 0xa4, 0xfb, 0x0c, 0x6a, 0xd1, 0x30, 0x8b, 0x8e, 0x41, 0x2a, 0x6e, 0x5d, 0xe0, + 0xb8, 0x55, 0x73, 0x23, 0x94, 0x93, 0xe9, 0x1d, 0x5a, 0xf9, 0xee, 0x1d, 0xaa, 0xfe, 0x41, 0x06, + 0x98, 0xd0, 0x07, 0x77, 0xe7, 0x8e, 0x33, 0xb4, 0xce, 0x88, 0x11, 0xdc, 0x85, 0x2d, 0x61, 0x92, + 0x94, 0x22, 0x58, 0x84, 0x73, 0x88, 0x23, 0x12, 0xe7, 0xd0, 0xaa, 0x2b, 0x7b, 0xeb, 0x2b, 0xaf, + 0xec, 0xad, 0xbe, 0x0a, 0x78, 0x0b, 0x2a, 0xf2, 0x85, 0x37, 0x2e, 0xa6, 0x80, 0x91, 0xdc, 0x75, + 0xfb, 0x37, 0xeb, 0x00, 0x89, 0xce, 0xfa, 0xeb, 0x76, 0xe8, 0xaf, 0x98, 0x92, 0xec, 0xaa, 0x29, + 0xb9, 0x03, 0x8a, 0x4c, 0x27, 0xdd, 0xbc, 0xac, 0x25, 0x84, 0xd4, 0x4d, 0xce, 0xd3, 0xa4, 0xdb, + 0x71, 0xc4, 0xd3, 0x84, 0xaf, 0x50, 0x44, 0x0c, 0x11, 0xcb, 0x15, 0xe7, 0x68, 0xc9, 0x0e, 0x38, + 0x0b, 0x66, 0x9f, 0xc2, 0xb5, 0x38, 0xa7, 0x7e, 0x6a, 0x87, 0xc7, 0xde, 0x3c, 0x14, 0x36, 0xc2, + 0x40, 0x9c, 0xa6, 0x57, 0xa3, 0x92, 0x9e, 0x72, 0x34, 0x67, 0x59, 0x01, 0xfb, 0x08, 0xca, 0x93, + 0xb9, 0xe3, 0xe8, 0xa1, 0x75, 0x16, 0x8a, 0x30, 0xd2, 0x7a, 0x4a, 0xdd, 0x97, 0xa6, 0x57, 0x2b, + 0x4d, 0x44, 0x42, 0xfd, 0xef, 0xeb, 0x90, 0xff, 0xe9, 0xdc, 0xf2, 0xcf, 0xd9, 0xc7, 0x50, 0x0e, + 0xc2, 0x69, 0x28, 0x3b, 0xe8, 0xae, 0xf1, 0x02, 0x08, 0x4f, 0xfe, 0x35, 0x6b, 0x6a, 0xb9, 0x21, + 0xb7, 0x60, 0x21, 0x2d, 0x9d, 0x48, 0x97, 0x21, 0x1f, 0x84, 0xd6, 0x8c, 0x3b, 0x04, 0xf3, 0x1a, + 0x4f, 0xb0, 0x6d, 0xc8, 0xbb, 0x9e, 0x69, 0x05, 0xe9, 0x60, 0x2f, 0x54, 0xd8, 0x35, 0x8e, 0x60, + 0x2a, 0x14, 0xe2, 0x19, 0x5f, 0x72, 0x92, 0x71, 0x0c, 0xc5, 0x61, 0x5b, 0x86, 0x69, 0xbb, 0x47, + 0xd1, 0x4d, 0xd6, 0x38, 0x8d, 0x67, 0x2d, 0x09, 0xbe, 0xc6, 0x51, 0x74, 0xad, 0x5c, 0x24, 0xd9, + 0x36, 0x54, 0xf0, 0xf3, 0xa9, 0x6f, 0x87, 0xd6, 0xe0, 0x81, 0x18, 0x37, 0x19, 0x84, 0x62, 0xab, + 0x69, 0x85, 0xd6, 0x38, 0x1c, 0x7c, 0x23, 0x62, 0x72, 0x28, 0xd8, 0x22, 0x82, 0xa8, 0x26, 0x6c, + 0xa4, 0xba, 0xbb, 0x64, 0x60, 0x18, 0xb4, 0xbb, 0xa8, 0x4e, 0x67, 0x24, 0x0d, 0x79, 0x5d, 0xd6, + 0x8a, 0xb3, 0x92, 0xba, 0x9c, 0x93, 0xd4, 0x97, 0x3c, 0x29, 0xdb, 0x6d, 0xed, 0x51, 0x5b, 0x29, + 0xa8, 0x7f, 0xb8, 0x0e, 0x5b, 0x43, 0xdf, 0x70, 0x03, 0x83, 0x5f, 0x4f, 0x72, 0x43, 0xdf, 0x73, + 0xd8, 0x67, 0x50, 0x0a, 0xc7, 0x8e, 0x3c, 0x0d, 0xb7, 0xa2, 0x4d, 0xbf, 0x40, 0x7a, 0x6f, 0x38, + 0xe6, 0xe6, 0xc4, 0x62, 0xc8, 0x3f, 0xd8, 0xbb, 0x90, 0x1f, 0x59, 0x47, 0xb6, 0x2b, 0x18, 0xf0, + 0x95, 0xc5, 0x8c, 0x3b, 0x88, 0xdc, 0x5b, 0xd3, 0x38, 0x15, 0x7b, 0x1f, 0x0a, 0x63, 0x6f, 0x1a, + 0x9d, 0x54, 0xc9, 0x4d, 0x0a, 0xa9, 0x22, 0xc4, 0xee, 0xad, 0x69, 0x82, 0x8e, 0x7d, 0x0c, 0x25, + 0xdf, 0x73, 0x9c, 0x91, 0x31, 0x3e, 0x11, 0x67, 0x58, 0x7d, 0x31, 0x8f, 0x26, 0xf0, 0x7b, 0x6b, + 0x5a, 0x4c, 0xab, 0xde, 0x83, 0xa2, 0x68, 0x2c, 0x0e, 0xc0, 0x4e, 0xfb, 0x51, 0x47, 0x0c, 0x64, + 0xb3, 0xbf, 0xbf, 0xdf, 0x19, 0xf2, 0x2b, 0x9b, 0x5a, 0xbf, 0xdb, 0xdd, 0x69, 0x34, 0x1f, 0x2b, + 0xeb, 0x3b, 0x25, 0x28, 0x70, 0xf3, 0x93, 0xfa, 0x3b, 0x19, 0xd8, 0x5c, 0xe8, 0x00, 0x7b, 0x08, + 0xb9, 0x29, 0x4a, 0x7e, 0x7c, 0x78, 0x6e, 0xaf, 0xec, 0xa5, 0x94, 0xe6, 0xf2, 0x20, 0xe6, 0x50, + 0x3f, 0x85, 0x5a, 0x1a, 0x2e, 0x69, 0xb1, 0x1b, 0x50, 0xd6, 0xda, 0x8d, 0x96, 0xde, 0xef, 0xa1, + 0xea, 0x88, 0xaa, 0x24, 0x25, 0x9f, 0x6a, 0x1d, 0xd2, 0x3b, 0x7f, 0x13, 0x94, 0xc5, 0x81, 0x61, + 0x8f, 0x60, 0x13, 0xc5, 0x0f, 0xc7, 0xe2, 0x07, 0x45, 0x32, 0x65, 0x37, 0x57, 0x8c, 0xa4, 0x20, + 0xa3, 0x19, 0xab, 0x8d, 0x53, 0x69, 0xf5, 0xff, 0x06, 0xb6, 0x3c, 0x82, 0xbf, 0xbe, 0xe2, 0xff, + 0x5b, 0x06, 0x72, 0x07, 0x8e, 0xe1, 0xb2, 0xd7, 0x20, 0x4f, 0x4f, 0x8d, 0x08, 0xee, 0x59, 0x91, + 0x36, 0x38, 0x2e, 0x0b, 0xc2, 0xb1, 0xb7, 0x21, 0x1b, 0x8e, 0xa3, 0xeb, 0xa9, 0x2f, 0x5d, 0xb0, + 0xf8, 0xf6, 0xd6, 0x34, 0xa4, 0x62, 0x77, 0x20, 0x6b, 0x9a, 0x51, 0x54, 0xaa, 0x50, 0x96, 0x51, + 0xc3, 0x6a, 0x59, 0x13, 0xdb, 0xb5, 0xc5, 0xd3, 0x28, 0x48, 0xc2, 0x5e, 0x87, 0xac, 0x39, 0x76, + 0xd2, 0x21, 0xc6, 0x5c, 0x17, 0x8b, 0x0b, 0x34, 0xc7, 0x0e, 0x53, 0x61, 0x23, 0xf4, 0xcf, 0x75, + 0x7f, 0xee, 0x52, 0x18, 0x4a, 0x20, 0x74, 0x92, 0x0a, 0x0a, 0x33, 0x73, 0x8a, 0x65, 0x09, 0xc4, + 0x35, 0x97, 0x99, 0x6f, 0xcd, 0x0c, 0x3f, 0xd6, 0x46, 0xec, 0xe0, 0x80, 0x03, 0x76, 0x0a, 0x40, + 0x2f, 0x15, 0xaa, 0xef, 0xd0, 0x3b, 0x18, 0x28, 0x61, 0xab, 0xd1, 0xd7, 0x8a, 0x5b, 0x84, 0x02, + 0xa3, 0xfe, 0x45, 0x16, 0x2a, 0x52, 0x7b, 0xd8, 0x87, 0x50, 0x32, 0xd3, 0x1b, 0xf1, 0xda, 0x52, + 0xa3, 0xef, 0xb5, 0xa2, 0x2d, 0x68, 0x8a, 0xe5, 0xfd, 0x29, 0x6c, 0x04, 0x56, 0xa8, 0x3f, 0x33, + 0x7c, 0x9b, 0xbf, 0x11, 0xb4, 0x2e, 0x3b, 0xdc, 0x06, 0x56, 0xf8, 0x24, 0xc2, 0xec, 0xad, 0x69, + 0xd5, 0x40, 0x4a, 0x93, 0x1a, 0x20, 0xba, 0x94, 0x4d, 0xbd, 0xa3, 0xc4, 0x81, 0x7b, 0x6b, 0x5a, + 0x84, 0x47, 0x52, 0xeb, 0xcc, 0x1a, 0xcf, 0xc3, 0x48, 0x0d, 0xd8, 0x88, 0x3a, 0x44, 0x40, 0x7a, + 0xb2, 0x8d, 0x7f, 0xb2, 0xfb, 0xc8, 0xeb, 0x0c, 0xc7, 0xf1, 0x48, 0x66, 0xcb, 0xcb, 0x86, 0xe0, + 0x56, 0x0c, 0xe7, 0x4f, 0xc4, 0x45, 0x29, 0xf6, 0x06, 0xe4, 0xbd, 0xf0, 0xd8, 0x8a, 0x84, 0xe7, + 0xe8, 0x45, 0x0d, 0x04, 0xb5, 0x9a, 0x5d, 0x5c, 0x29, 0x84, 0x56, 0x7f, 0x91, 0x81, 0xa2, 0x18, + 0x01, 0xb6, 0x05, 0x1b, 0x83, 0xf6, 0x50, 0x7f, 0xd2, 0xd0, 0x3a, 0x8d, 0x9d, 0x6e, 0x5b, 0x44, + 0x46, 0x3f, 0xd2, 0x1a, 0x3d, 0xc1, 0x27, 0xb5, 0xf6, 0x93, 0xfe, 0xe3, 0x36, 0x37, 0x0d, 0xb5, + 0xda, 0xbd, 0xaf, 0x94, 0x2c, 0xb7, 0x8e, 0xb6, 0x0f, 0x1a, 0x1a, 0x72, 0xc9, 0x0a, 0x14, 0xdb, + 0x5f, 0xb6, 0x9b, 0x87, 0xc4, 0x26, 0x6b, 0x00, 0xad, 0x76, 0xa3, 0xdb, 0xed, 0x37, 0x91, 0x6d, + 0x16, 0x18, 0x83, 0x5a, 0x53, 0x6b, 0x37, 0x86, 0x6d, 0xbd, 0xd1, 0x6c, 0xf6, 0x0f, 0x7b, 0x43, + 0xa5, 0x88, 0x35, 0x36, 0xba, 0xc3, 0xb6, 0x16, 0x83, 0xe8, 0x95, 0xa3, 0x96, 0xd6, 0x3f, 0x88, + 0x21, 0xe5, 0x9d, 0x32, 0xaa, 0x64, 0x34, 0x57, 0xea, 0x7f, 0xaa, 0x41, 0x2d, 0xbd, 0x34, 0xd9, + 0x27, 0x50, 0x32, 0xcd, 0xd4, 0x1c, 0xdf, 0x58, 0xb5, 0x84, 0xef, 0xb5, 0xcc, 0x68, 0x9a, 0xf9, + 0x07, 0x7b, 0x35, 0xda, 0x48, 0xeb, 0x4b, 0x1b, 0x29, 0xda, 0x46, 0x3f, 0x81, 0x4d, 0xf1, 0x22, + 0x85, 0x69, 0x84, 0xc6, 0xc8, 0x08, 0xac, 0xf4, 0x2e, 0x69, 0x12, 0xb2, 0x25, 0x70, 0x7b, 0x6b, + 0x5a, 0x6d, 0x9c, 0x82, 0xb0, 0x1f, 0x41, 0xcd, 0x20, 0x65, 0x34, 0xce, 0x9f, 0x93, 0x85, 0xc0, + 0x06, 0xe2, 0xa4, 0xec, 0x1b, 0x86, 0x0c, 0xc0, 0x85, 0x68, 0xfa, 0xde, 0x2c, 0xc9, 0x9c, 0x97, + 0x17, 0x62, 0xcb, 0xf7, 0x66, 0x52, 0xde, 0xaa, 0x29, 0xa5, 0xd9, 0xc7, 0x50, 0x15, 0x2d, 0x4f, + 0xd4, 0xfd, 0x78, 0xcb, 0xf2, 0x66, 0x93, 0x50, 0xb7, 0xb7, 0xa6, 0x55, 0xc6, 0x49, 0x92, 0x3d, + 0x40, 0x49, 0x2e, 0x91, 0xc5, 0x8b, 0xf2, 0x5a, 0xa3, 0xd6, 0x46, 0xb9, 0xc0, 0x88, 0x53, 0xec, + 0x7d, 0x00, 0x6a, 0x27, 0xcf, 0x53, 0x4a, 0x05, 0x20, 0xf8, 0xde, 0x2c, 0xca, 0x52, 0x36, 0xa3, + 0x84, 0xd4, 0x3c, 0x7e, 0xa1, 0xaf, 0xbc, 0xdc, 0x3c, 0xba, 0x66, 0x96, 0x34, 0x8f, 0x5f, 0xe0, + 0x8b, 0x9b, 0xc7, 0xb3, 0xc1, 0x52, 0xf3, 0xa2, 0x5c, 0xbc, 0x79, 0x3c, 0x53, 0xd4, 0x3c, 0x9e, + 0xa7, 0xb2, 0xd8, 0xbc, 0x28, 0x0b, 0x35, 0x8f, 0xe7, 0xf8, 0xd1, 0x92, 0xec, 0x5e, 0xbd, 0x50, + 0x76, 0xc7, 0x69, 0x4b, 0x4b, 0xef, 0x3f, 0x82, 0x5a, 0x70, 0xec, 0x9d, 0x4a, 0x0c, 0x64, 0x43, + 0xce, 0x3d, 0x38, 0xf6, 0x4e, 0x65, 0x0e, 0xb2, 0x11, 0xc8, 0x00, 0x6c, 0x2d, 0xef, 0x22, 0x5d, + 0xd9, 0xad, 0xc9, 0xad, 0xa5, 0x1e, 0x3e, 0xb1, 0xad, 0x53, 0x6c, 0xad, 0x11, 0x25, 0x70, 0x50, + 0x12, 0xc3, 0x4e, 0x20, 0x62, 0x6d, 0x52, 0xce, 0x74, 0x51, 0x13, 0xc4, 0xd6, 0x9e, 0x00, 0xd7, + 0xd6, 0xdc, 0x95, 0xb3, 0x29, 0xf2, 0xda, 0x3a, 0x74, 0x53, 0x19, 0xab, 0x9c, 0x54, 0x64, 0x4d, + 0x76, 0x45, 0x60, 0x7d, 0x33, 0xb7, 0xdc, 0xb1, 0x25, 0xa2, 0x72, 0x52, 0xbb, 0x62, 0x20, 0x70, + 0xc9, 0xae, 0x88, 0x20, 0xf1, 0xba, 0x8e, 0xb3, 0xb3, 0xc5, 0x75, 0x2d, 0x65, 0xa6, 0x75, 0x1d, + 0x67, 0x8d, 0x37, 0x54, 0x9c, 0xf7, 0xd2, 0xd2, 0x86, 0x92, 0x32, 0xf3, 0x0d, 0x15, 0xe7, 0x7e, + 0x00, 0x62, 0x35, 0xf1, 0xc1, 0x4d, 0xc5, 0xee, 0xf0, 0x56, 0x8b, 0xd1, 0x85, 0x71, 0x9c, 0xc2, + 0xb5, 0xea, 0x5b, 0xa8, 0x2b, 0x88, 0xa5, 0x70, 0x45, 0x5e, 0xab, 0x1a, 0x61, 0xe2, 0xad, 0xe4, + 0x27, 0x49, 0xf5, 0x8f, 0xf3, 0x50, 0x14, 0x4c, 0x87, 0x5d, 0x82, 0x4d, 0xc1, 0xfb, 0x5a, 0x8d, + 0x61, 0x63, 0xa7, 0x31, 0x40, 0x69, 0x85, 0x41, 0x8d, 0x33, 0xbf, 0x18, 0x96, 0x41, 0x86, 0x48, + 0xdc, 0x2f, 0x06, 0xad, 0x23, 0x43, 0x14, 0x79, 0xf9, 0x13, 0x71, 0x59, 0xb6, 0x09, 0x15, 0x9e, + 0x91, 0x03, 0xe8, 0xc6, 0x12, 0xe5, 0xe2, 0xe9, 0xbc, 0x94, 0x85, 0xbb, 0x0c, 0x0a, 0x49, 0x16, + 0x0e, 0x28, 0xc6, 0x59, 0x22, 0x9f, 0x02, 0x83, 0xda, 0x50, 0x3b, 0xec, 0x35, 0x93, 0x7a, 0xca, + 0x74, 0xcb, 0x84, 0x17, 0xf3, 0xa4, 0xd3, 0x7e, 0xaa, 0x00, 0x66, 0xe2, 0xa5, 0x50, 0xba, 0x82, + 0xf2, 0x16, 0x15, 0x42, 0xc9, 0x2a, 0x7b, 0x09, 0x2e, 0x0d, 0xf6, 0xfa, 0x4f, 0x75, 0x9e, 0x29, + 0xee, 0xc2, 0x06, 0xbb, 0x0c, 0x8a, 0x84, 0xe0, 0xc5, 0xd7, 0xb0, 0x4a, 0x82, 0x46, 0x84, 0x03, + 0x65, 0x93, 0x9c, 0x72, 0x08, 0x1b, 0xf2, 0x03, 0x48, 0xc1, 0xae, 0xf0, 0xac, 0xfd, 0xee, 0xe1, + 0x7e, 0x6f, 0xa0, 0x6c, 0x61, 0x23, 0x08, 0xc2, 0x5b, 0xce, 0xe2, 0x62, 0x92, 0x63, 0xeb, 0x12, + 0x9d, 0x64, 0x08, 0x7b, 0xda, 0xd0, 0x7a, 0x9d, 0xde, 0xa3, 0x81, 0x72, 0x39, 0x2e, 0x99, 0x9c, + 0x13, 0x03, 0xe5, 0x4a, 0x0c, 0x18, 0x0c, 0x1b, 0xc3, 0xc3, 0x81, 0x72, 0x35, 0x6e, 0xe5, 0x81, + 0xd6, 0x6f, 0xb6, 0x07, 0x83, 0x6e, 0x67, 0x30, 0x54, 0x5e, 0x62, 0x57, 0x60, 0x2b, 0x69, 0x51, + 0x44, 0x5c, 0x97, 0x1a, 0xaa, 0x3d, 0x6a, 0x0f, 0x95, 0x6b, 0x71, 0x33, 0x9a, 0xfd, 0x6e, 0xb7, + 0x41, 0xee, 0xa3, 0xeb, 0x48, 0x44, 0x7e, 0x35, 0xd1, 0x9b, 0x97, 0xb1, 0x5d, 0x87, 0x3d, 0x19, + 0x74, 0x43, 0x5a, 0x1a, 0x83, 0xf6, 0x4f, 0x0f, 0xdb, 0xbd, 0x66, 0x5b, 0x79, 0x25, 0x59, 0x1a, + 0x31, 0xec, 0x66, 0xbc, 0x34, 0x62, 0xd0, 0xad, 0xb8, 0xce, 0x08, 0x34, 0x50, 0xb6, 0xb1, 0x3c, + 0xd1, 0x8e, 0x5e, 0xaf, 0xdd, 0x1c, 0x62, 0x5f, 0x5f, 0x8d, 0x47, 0xf1, 0xf0, 0xe0, 0x91, 0xd6, + 0x68, 0xb5, 0x15, 0x15, 0x21, 0x5a, 0xbb, 0xd7, 0xd8, 0x8f, 0x66, 0xfb, 0xb5, 0x9d, 0x2a, 0x3d, + 0x3b, 0x2b, 0x8e, 0x4b, 0xf5, 0x0b, 0x60, 0xf2, 0xfb, 0x8d, 0xe2, 0x89, 0x26, 0x06, 0xb9, 0x89, + 0xef, 0x4d, 0xa3, 0x3b, 0xbb, 0xf8, 0x8d, 0xba, 0xda, 0x6c, 0x3e, 0x22, 0x1f, 0x50, 0x72, 0x85, + 0x4f, 0x06, 0xa9, 0x7f, 0x9c, 0x81, 0x5a, 0xfa, 0xa8, 0x44, 0x11, 0xd1, 0x9e, 0xe8, 0xae, 0x17, + 0xf2, 0xb7, 0x7f, 0x82, 0xc8, 0x12, 0x65, 0x4f, 0x7a, 0x5e, 0x48, 0x8f, 0xff, 0x90, 0xea, 0x18, + 0x9f, 0x7c, 0xbc, 0xd4, 0x38, 0xcd, 0x3a, 0x70, 0x29, 0xf5, 0xbc, 0x65, 0xea, 0xe5, 0xa5, 0x7a, + 0xfc, 0x28, 0xdf, 0x42, 0xfb, 0x35, 0x16, 0x2c, 0xf7, 0x49, 0x81, 0x6c, 0xf0, 0x8d, 0x23, 0x0c, + 0x01, 0xf8, 0xa9, 0xee, 0xc1, 0x46, 0xea, 0x64, 0x26, 0x8d, 0x7f, 0x92, 0x6e, 0x69, 0xc9, 0x9e, + 0x3c, 0xbf, 0x99, 0xea, 0x1f, 0x65, 0xa0, 0x2a, 0x9f, 0xd3, 0x3f, 0xb8, 0x24, 0xba, 0x5d, 0x20, + 0xbe, 0x75, 0xdb, 0x8c, 0xde, 0xfc, 0x89, 0x40, 0x1d, 0x7a, 0x6e, 0x9b, 0xdb, 0x60, 0x77, 0x4f, + 0x06, 0x71, 0x77, 0x64, 0x10, 0xaa, 0xcc, 0x74, 0x85, 0x6b, 0xf7, 0x31, 0x12, 0x88, 0xfb, 0x09, + 0x09, 0x44, 0xbd, 0x05, 0xe5, 0xdd, 0x93, 0xc8, 0xad, 0x2f, 0xbf, 0x80, 0x55, 0xe6, 0xf7, 0x3e, + 0xd5, 0x3f, 0xcd, 0x40, 0x2d, 0x79, 0xd5, 0x80, 0x42, 0xfe, 0xf8, 0xb3, 0xa8, 0x7c, 0x39, 0xac, + 0x9b, 0xa3, 0xe4, 0x25, 0xee, 0x75, 0xf9, 0x25, 0xee, 0xd7, 0x44, 0x61, 0x59, 0xf9, 0x34, 0x8b, + 0xeb, 0x12, 0xb7, 0x4a, 0x1f, 0x40, 0x15, 0xff, 0x6b, 0xd6, 0xc4, 0xf2, 0x7d, 0x2b, 0x7a, 0x21, + 0x76, 0x89, 0x38, 0x45, 0x44, 0x1a, 0x89, 0x35, 0x11, 0x82, 0xd1, 0xca, 0x87, 0x17, 0x10, 0xaf, + 0xfe, 0xe3, 0x1c, 0x54, 0x24, 0xa9, 0xe7, 0x7b, 0x2d, 0xbf, 0x1b, 0x50, 0x4e, 0xae, 0xf4, 0x8b, + 0xab, 0x7c, 0x31, 0x20, 0x35, 0x57, 0xd9, 0x85, 0xb9, 0xaa, 0x43, 0x51, 0xc4, 0x06, 0x0a, 0xbb, + 0x67, 0x94, 0x4c, 0x1b, 0xf6, 0xf2, 0xcf, 0x31, 0xbd, 0x7f, 0x00, 0x55, 0xc9, 0x2a, 0x17, 0xbd, + 0x0f, 0xb2, 0x48, 0x5f, 0x49, 0x2c, 0x74, 0x01, 0xbb, 0x02, 0x85, 0xc9, 0x89, 0x6e, 0x8e, 0x22, + 0x33, 0x67, 0x7e, 0x72, 0xd2, 0x1a, 0x91, 0xeb, 0x62, 0x12, 0x1f, 0xf4, 0xdc, 0x56, 0x52, 0x9a, + 0x44, 0xc7, 0xf9, 0x1d, 0x28, 0x4e, 0x4e, 0x78, 0x48, 0x6a, 0x59, 0x0e, 0x94, 0x49, 0x86, 0xbc, + 0x30, 0x39, 0xa1, 0x68, 0xd4, 0x4f, 0x41, 0x59, 0xb0, 0xa9, 0x06, 0xc2, 0xa1, 0xb7, 0xd8, 0xa8, + 0xcd, 0xb4, 0x79, 0x35, 0x60, 0xef, 0xc1, 0x65, 0x71, 0xf2, 0x1a, 0x81, 0xce, 0xe3, 0xdc, 0xe9, + 0x95, 0x08, 0xfe, 0x94, 0xd6, 0x16, 0xc7, 0x35, 0x82, 0x01, 0x61, 0x70, 0xb1, 0xaa, 0x50, 0x95, + 0xd6, 0x2e, 0x7f, 0x82, 0xa3, 0xac, 0xa5, 0x60, 0xec, 0x21, 0x54, 0x27, 0x27, 0x7c, 0x2d, 0x0c, + 0xbd, 0x7d, 0x4b, 0xc4, 0x2e, 0x5f, 0x5e, 0x5c, 0x05, 0x14, 0xa8, 0x9a, 0xa2, 0x64, 0xef, 0x02, + 0xf3, 0xad, 0xd0, 0x72, 0xa9, 0x27, 0xa6, 0x65, 0x98, 0x8e, 0xed, 0x5a, 0x24, 0x6c, 0x65, 0xb5, + 0xad, 0x18, 0xd3, 0x12, 0x08, 0xf5, 0x9f, 0x64, 0xa0, 0x96, 0x48, 0xbf, 0xb8, 0xa1, 0xd9, 0x5d, + 0xf9, 0x6d, 0xe4, 0xfa, 0xa2, 0x80, 0x8c, 0x24, 0xf7, 0x86, 0xe7, 0x33, 0xfe, 0x7e, 0xe2, 0xaa, + 0x77, 0x54, 0x56, 0x99, 0x5c, 0xb3, 0xab, 0x4c, 0xae, 0xea, 0x23, 0xc8, 0x0e, 0xcf, 0x67, 0xdc, + 0xd2, 0x82, 0x67, 0x20, 0xd7, 0xca, 0xf8, 0xe9, 0x47, 0x7e, 0xfd, 0xc7, 0xed, 0xaf, 0xf8, 0x35, + 0xe6, 0x03, 0xad, 0xb3, 0xdf, 0xd0, 0xbe, 0xa2, 0x88, 0x0d, 0x92, 0x12, 0x76, 0xfb, 0x5a, 0xbb, + 0xf3, 0xa8, 0x47, 0x80, 0x1c, 0xd9, 0x61, 0x92, 0x26, 0x36, 0x4c, 0x73, 0xf7, 0x44, 0x7e, 0x4e, + 0x22, 0x93, 0x7a, 0x4e, 0x22, 0x7d, 0xf3, 0x71, 0x7d, 0xf1, 0xe6, 0x23, 0x8b, 0x77, 0x74, 0xcc, + 0x1e, 0xd8, 0x9b, 0x90, 0x9b, 0x9c, 0x58, 0xe7, 0x69, 0x15, 0x27, 0xbd, 0x19, 0x89, 0x40, 0xfd, + 0x65, 0x06, 0x58, 0xaa, 0x21, 0x5c, 0xea, 0xfe, 0xa1, 0x6d, 0xf9, 0x04, 0xea, 0xe2, 0xb1, 0x41, + 0x4e, 0x25, 0xd9, 0x78, 0xc5, 0x90, 0x5e, 0xf1, 0x92, 0xb8, 0xb6, 0xe4, 0xa9, 0x17, 0xf6, 0x1e, + 0xf0, 0xd7, 0xde, 0x70, 0x81, 0xa4, 0x8d, 0x1a, 0x12, 0xaf, 0xd0, 0x12, 0x9a, 0xe4, 0x79, 0x37, + 0xf9, 0xd9, 0x3a, 0x6e, 0x1e, 0xde, 0x4c, 0x66, 0x8d, 0xf8, 0x87, 0xfa, 0x7b, 0x19, 0xb8, 0x94, + 0x5e, 0x10, 0xbf, 0x5a, 0x2f, 0xd3, 0x6f, 0xf4, 0x65, 0x17, 0xdf, 0xe8, 0x5b, 0xb5, 0x9e, 0x72, + 0x2b, 0xd7, 0xd3, 0xff, 0x9f, 0x81, 0xcb, 0xd2, 0xe8, 0x27, 0x7a, 0xd2, 0x5f, 0x53, 0xcb, 0xa4, + 0xa7, 0xfa, 0x72, 0xa9, 0xa7, 0xfa, 0xd4, 0x3f, 0xcc, 0xc0, 0xd5, 0x85, 0x96, 0x68, 0xd6, 0x5f, + 0x6b, 0x5b, 0xd2, 0x4f, 0xfa, 0x91, 0x89, 0x9a, 0x87, 0x08, 0xf2, 0x2b, 0x65, 0x2c, 0xfd, 0x46, + 0x5f, 0x17, 0xe7, 0xf1, 0x9f, 0xa6, 0x1b, 0x69, 0x26, 0xb7, 0x73, 0xd8, 0x47, 0x50, 0x49, 0x24, + 0xa6, 0xe8, 0xc5, 0x84, 0x95, 0x57, 0x7b, 0x64, 0xba, 0x95, 0x6c, 0x74, 0xfd, 0xfb, 0xb1, 0xd1, + 0x87, 0x50, 0x8d, 0x0b, 0x6e, 0x59, 0x93, 0xb4, 0x35, 0x62, 0xe1, 0xcd, 0x9f, 0x14, 0xa5, 0xfa, + 0x21, 0x6c, 0x25, 0xbd, 0x68, 0x8a, 0x77, 0xaa, 0x6e, 0x41, 0xc5, 0xb5, 0x4e, 0xf5, 0xe8, 0x15, + 0x2b, 0x11, 0xea, 0xe2, 0x5a, 0xa7, 0x82, 0x40, 0xdd, 0x95, 0xf9, 0x5e, 0xfc, 0xe4, 0xb8, 0x63, + 0xca, 0x33, 0x53, 0xf4, 0x1c, 0x33, 0x42, 0x61, 0x69, 0xd2, 0xc4, 0x14, 0x5d, 0xeb, 0x94, 0xd6, + 0xdc, 0xa9, 0x28, 0xa7, 0x61, 0x9a, 0xc2, 0x61, 0xbe, 0xea, 0xf5, 0x97, 0x6b, 0x50, 0x9a, 0xf9, + 0xa9, 0x99, 0x2d, 0xce, 0x7c, 0x5e, 0xed, 0x6d, 0x11, 0x48, 0x73, 0x91, 0x73, 0x9d, 0x87, 0xd6, + 0x88, 0xc7, 0x32, 0x72, 0xc9, 0x4f, 0x12, 0x7c, 0x24, 0x58, 0x1e, 0xee, 0x3f, 0x51, 0x73, 0xec, + 0x44, 0xcf, 0xdc, 0xd9, 0x20, 0x27, 0x3a, 0x49, 0x80, 0xd6, 0x37, 0x22, 0x96, 0x07, 0x3f, 0xd5, + 0x1d, 0xa8, 0x48, 0x9a, 0x1d, 0x8a, 0x26, 0x92, 0x55, 0x24, 0x48, 0xbf, 0xa7, 0x91, 0x0c, 0x90, + 0x56, 0x49, 0x8c, 0x22, 0x81, 0xfa, 0xfb, 0x00, 0x90, 0xe0, 0x52, 0x02, 0x43, 0x66, 0x41, 0x60, + 0x78, 0x21, 0x8f, 0xfc, 0x87, 0x50, 0x1b, 0x7b, 0xb3, 0x73, 0x3d, 0xc9, 0x91, 0x5d, 0x99, 0xa3, + 0x8a, 0x54, 0xc3, 0xe4, 0x5e, 0xcc, 0xb2, 0xa7, 0x35, 0xb7, 0xd2, 0xd3, 0xfa, 0x01, 0x14, 0xb9, + 0xe1, 0x3e, 0x10, 0xf7, 0xaa, 0x5e, 0x5a, 0xec, 0xe7, 0x3d, 0x11, 0x33, 0x1a, 0xd1, 0xb1, 0x36, + 0x6a, 0xe5, 0xe2, 0x49, 0x3b, 0xf9, 0x96, 0xd5, 0xcd, 0xe5, 0x9c, 0x11, 0x19, 0x7f, 0x47, 0xc9, + 0x90, 0x93, 0x92, 0x90, 0x10, 0x4e, 0x85, 0x35, 0x89, 0x84, 0x84, 0xa2, 0x2c, 0x24, 0x0c, 0xa7, + 0xdc, 0x86, 0x84, 0x42, 0xc2, 0xbb, 0x70, 0x49, 0x44, 0xa0, 0x63, 0x06, 0x1c, 0x4e, 0xa2, 0xe7, + 0x51, 0x4a, 0xe2, 0x36, 0xcd, 0x70, 0x4a, 0xd2, 0x37, 0x92, 0x7f, 0x09, 0x97, 0xc7, 0xc7, 0x86, + 0x7b, 0x64, 0xe9, 0xe1, 0xc8, 0xd1, 0xe9, 0x45, 0x65, 0x7d, 0x6a, 0xcc, 0x84, 0xd8, 0xf3, 0xe6, + 0x52, 0x63, 0x9b, 0x44, 0x3c, 0x1c, 0x39, 0x14, 0x46, 0x13, 0xfb, 0xe3, 0xb7, 0xc6, 0x8b, 0xf0, + 0x05, 0x6f, 0x14, 0x2c, 0x7a, 0xa3, 0x96, 0xa4, 0x99, 0xca, 0xb2, 0x34, 0x73, 0xfd, 0x5f, 0xe7, + 0xa0, 0x20, 0x62, 0xe8, 0xee, 0x42, 0xce, 0xf4, 0xbd, 0x59, 0x1c, 0xe9, 0xb6, 0x42, 0xba, 0xa0, + 0x9f, 0x5f, 0x41, 0x41, 0xe4, 0x1e, 0x14, 0x0c, 0xd3, 0xd4, 0x27, 0x27, 0x69, 0x8f, 0xd1, 0xc2, + 0x41, 0xbf, 0xb7, 0xa6, 0xe5, 0x0d, 0x3a, 0xf1, 0x3f, 0x81, 0x32, 0xd2, 0x73, 0x63, 0x58, 0x4a, + 0x5f, 0x5a, 0x3e, 0x92, 0xf7, 0xd6, 0xb4, 0x92, 0x11, 0x1d, 0xcf, 0x3f, 0x4e, 0xdb, 0xde, 0xf8, + 0x79, 0x79, 0x7d, 0x29, 0xeb, 0x45, 0x56, 0xb8, 0xdf, 0x00, 0x6e, 0x8c, 0x89, 0xb9, 0x4d, 0x5e, + 0x76, 0x4e, 0x2c, 0xf1, 0xa6, 0xbd, 0x35, 0x8d, 0xef, 0xb9, 0x88, 0x57, 0x7d, 0x14, 0xd9, 0xc5, + 0xe2, 0x1f, 0x4a, 0x58, 0x31, 0x32, 0xc8, 0x2b, 0x62, 0xe3, 0x18, 0x31, 0x0e, 0xcc, 0x66, 0x9a, + 0x51, 0xd8, 0x4e, 0x71, 0x29, 0x5b, 0xcc, 0x91, 0x28, 0x5b, 0xcc, 0x9e, 0x1e, 0x42, 0x85, 0x4c, + 0x54, 0x22, 0x5f, 0x69, 0x69, 0x68, 0x13, 0x86, 0x42, 0x86, 0xf7, 0x84, 0xbd, 0x34, 0xa3, 0x7e, + 0xfa, 0x96, 0x6c, 0xdb, 0xbc, 0xb1, 0x72, 0xa0, 0xb4, 0xd8, 0xcc, 0xc9, 0x3b, 0xab, 0xf1, 0x3c, + 0x6c, 0x07, 0xaa, 0x86, 0x74, 0xd2, 0x08, 0x43, 0xe7, 0x8d, 0x15, 0xf3, 0x14, 0xd3, 0x50, 0x19, + 0x52, 0x3a, 0x71, 0xc0, 0x5d, 0xd7, 0xe0, 0xea, 0xea, 0xa5, 0x2c, 0x47, 0x92, 0xe4, 0x78, 0x24, + 0x89, 0x9a, 0x7e, 0x68, 0x22, 0x7d, 0x39, 0x54, 0x8a, 0x2b, 0xf9, 0x1c, 0x75, 0x64, 0x79, 0xf3, + 0x56, 0xa0, 0x18, 0x3d, 0xcf, 0x4a, 0xe1, 0xa4, 0xcd, 0xfe, 0xc1, 0x57, 0x4a, 0x06, 0xc1, 0x9d, + 0xde, 0x60, 0xd8, 0xe8, 0x09, 0xf7, 0x6a, 0xa7, 0x27, 0xdc, 0xab, 0xea, 0x3f, 0xcf, 0x42, 0x39, + 0x36, 0x0f, 0xff, 0x70, 0xc5, 0x38, 0xd6, 0x38, 0xb3, 0xb2, 0xc6, 0xb9, 0x20, 0xa9, 0xc9, 0x0f, + 0x4e, 0x6c, 0xa6, 0xe5, 0xa1, 0x60, 0xf9, 0xf6, 0x59, 0xfe, 0x7b, 0xde, 0x3e, 0x93, 0xc3, 0x07, + 0x0b, 0xe9, 0xf0, 0xc1, 0x85, 0x27, 0x7a, 0x8b, 0x14, 0xa6, 0x22, 0x3f, 0xd1, 0x7b, 0x61, 0x7c, + 0x4a, 0xe9, 0xe2, 0xf8, 0x14, 0xfa, 0x8d, 0xa9, 0x27, 0xb6, 0x75, 0x2a, 0xa2, 0xe8, 0x44, 0x2a, + 0x7d, 0x7c, 0xc0, 0x73, 0x8e, 0x8f, 0xef, 0xc1, 0x8a, 0xd8, 0x7d, 0xb8, 0x3c, 0x39, 0x89, 0x9f, + 0x23, 0x4c, 0x14, 0xac, 0x2a, 0x75, 0x63, 0x25, 0x4e, 0xfd, 0xdb, 0x19, 0x80, 0xc4, 0x86, 0xfa, + 0x2b, 0x1b, 0x78, 0x24, 0x1d, 0x3a, 0xfb, 0x1d, 0x3a, 0xf4, 0x73, 0x1e, 0x65, 0x50, 0xbf, 0x81, + 0x72, 0x6c, 0x35, 0xff, 0xe1, 0x6b, 0xec, 0x85, 0xaa, 0xfc, 0xad, 0xc8, 0xd8, 0x15, 0x9b, 0x9d, + 0x7f, 0xd5, 0xb1, 0x48, 0x55, 0x9f, 0x7d, 0x4e, 0xf5, 0x67, 0xdc, 0xe2, 0x14, 0x57, 0xfe, 0x6b, + 0xde, 0x58, 0xf2, 0x9a, 0xcf, 0xa5, 0xd6, 0xbc, 0x3a, 0x17, 0x66, 0xb3, 0x5f, 0xbd, 0xea, 0x17, + 0xea, 0xf0, 0x5f, 0x66, 0x22, 0xdb, 0x4e, 0xfc, 0xc8, 0xe3, 0x85, 0x82, 0xd6, 0x6a, 0xf3, 0xd4, + 0x8b, 0x54, 0xf7, 0x9d, 0xda, 0x66, 0xee, 0xbb, 0xb4, 0xcd, 0x37, 0x21, 0xcf, 0x0f, 0x84, 0xfc, + 0x45, 0x9a, 0x26, 0xc7, 0x3f, 0xf7, 0x59, 0x74, 0x55, 0x15, 0x82, 0x25, 0xef, 0xef, 0xe5, 0xa8, + 0xdc, 0xe8, 0x49, 0x77, 0x4c, 0xa0, 0xb2, 0x5f, 0x4e, 0x94, 0xce, 0x17, 0x1f, 0x93, 0x5f, 0x9b, + 0xba, 0xf9, 0x0f, 0xd6, 0x61, 0x23, 0xe5, 0x30, 0xfb, 0x01, 0x8d, 0x59, 0xc9, 0xcd, 0xb3, 0xab, + 0xb9, 0xf9, 0x0f, 0x79, 0x6e, 0xe8, 0x7f, 0xc9, 0x09, 0x90, 0x8a, 0x31, 0x2b, 0xa5, 0x63, 0xcc, + 0x90, 0x9b, 0x56, 0x53, 0x52, 0xf9, 0x2a, 0xf9, 0x3d, 0xb3, 0x52, 0x7e, 0xbf, 0x19, 0xff, 0xa2, + 0x52, 0xa7, 0xc5, 0x15, 0xcb, 0x0d, 0x4d, 0x82, 0xb0, 0x4f, 0xe1, 0x1a, 0x97, 0x6a, 0xb8, 0x20, + 0xa7, 0x7b, 0x13, 0x3d, 0xfe, 0xbd, 0x25, 0x11, 0x37, 0x77, 0x95, 0x13, 0xf0, 0x37, 0xf3, 0x27, + 0x8d, 0x08, 0xab, 0x76, 0x60, 0x23, 0xe5, 0xbd, 0x94, 0x7e, 0xbb, 0x2d, 0x23, 0xff, 0x76, 0x1b, + 0xdb, 0x86, 0xfc, 0xe9, 0xb1, 0xe5, 0x5b, 0x2b, 0xde, 0xb8, 0xe3, 0x08, 0xf5, 0x47, 0x50, 0x95, + 0x23, 0x29, 0xd8, 0x3b, 0x90, 0xb7, 0x43, 0x6b, 0x1a, 0xe9, 0x56, 0x57, 0x97, 0x83, 0x2d, 0x48, + 0x91, 0xe6, 0x44, 0xea, 0x2f, 0x32, 0xa0, 0x2c, 0xe2, 0xa4, 0x1f, 0x98, 0xcb, 0x5c, 0xf0, 0x03, + 0x73, 0xeb, 0xa9, 0x46, 0xae, 0xfa, 0x8d, 0xb8, 0xf8, 0x9d, 0xad, 0xdc, 0x05, 0xef, 0x6c, 0xb1, + 0x37, 0xa0, 0xe4, 0x5b, 0xf4, 0xeb, 0x5d, 0xe6, 0x8a, 0x58, 0xe6, 0x18, 0xa7, 0xfe, 0x6e, 0x06, + 0x8a, 0x22, 0xec, 0x63, 0xa5, 0xb2, 0xfb, 0x16, 0x14, 0xf9, 0x2f, 0x79, 0x45, 0xca, 0xff, 0x52, + 0x1c, 0x64, 0x84, 0x67, 0x37, 0x79, 0x30, 0x4c, 0x5a, 0xf9, 0x3d, 0x70, 0x0c, 0x57, 0x23, 0xb8, + 0xf8, 0x29, 0x0c, 0x63, 0x2a, 0xae, 0xdf, 0xf1, 0xd7, 0x32, 0x80, 0x40, 0xfc, 0xa6, 0xdd, 0x8f, + 0xa1, 0x28, 0xc2, 0x4a, 0x56, 0x36, 0xe5, 0x79, 0xbf, 0x61, 0xb5, 0x0d, 0x90, 0xc4, 0x99, 0xac, + 0x2a, 0x41, 0xbd, 0x0b, 0xa5, 0x28, 0xb4, 0x04, 0xd7, 0x5f, 0x52, 0xb5, 0x88, 0x55, 0x97, 0x1b, + 0xe3, 0x88, 0x87, 0x60, 0xbb, 0xde, 0xf8, 0x84, 0xac, 0x6a, 0xef, 0x41, 0xc9, 0x89, 0x1e, 0xdd, + 0xcd, 0x5c, 0xfc, 0xe8, 0x6e, 0x4c, 0xc4, 0xee, 0x42, 0xcc, 0x8e, 0x9f, 0xa7, 0x2d, 0xab, 0x8d, + 0xe8, 0xc2, 0x07, 0xad, 0xb2, 0x07, 0xc2, 0x7a, 0xd4, 0xa5, 0x47, 0x68, 0x52, 0x06, 0x9b, 0x54, + 0x9b, 0x34, 0x89, 0x4c, 0xad, 0x41, 0x55, 0xf6, 0x87, 0xab, 0x0d, 0xd8, 0xda, 0xb7, 0x42, 0x03, + 0x79, 0xd6, 0x60, 0x6c, 0xb8, 0x48, 0xcf, 0xd7, 0x2f, 0x7e, 0xa4, 0xd7, 0xef, 0x22, 0x9d, 0xc6, + 0x89, 0xd4, 0x5f, 0xe4, 0x40, 0x59, 0xc4, 0x21, 0x33, 0x89, 0x6f, 0x5a, 0x66, 0xa2, 0x07, 0xc5, + 0x9d, 0xf8, 0xf7, 0x60, 0x68, 0x5d, 0xa4, 0x7e, 0xec, 0x84, 0x83, 0xa4, 0x80, 0xd5, 0xd4, 0xcb, + 0xdc, 0x25, 0x3b, 0xd8, 0xe3, 0x6f, 0x73, 0xbf, 0xc4, 0xdf, 0xc4, 0x70, 0xbc, 0x31, 0x2d, 0xeb, + 0x2a, 0x3d, 0x81, 0xd1, 0xf5, 0xc6, 0x98, 0x2b, 0x52, 0xb8, 0x79, 0x90, 0x56, 0x55, 0x2b, 0x09, + 0x2d, 0x9b, 0x9c, 0x06, 0x22, 0x8c, 0x35, 0x0c, 0xc4, 0xbd, 0xa1, 0x12, 0x07, 0x0c, 0x83, 0xe8, + 0xbd, 0xd2, 0xb1, 0xf8, 0x65, 0x8e, 0x2c, 0xbd, 0x57, 0xda, 0x74, 0xe9, 0x4a, 0x0d, 0xfd, 0x90, + 0xcc, 0x58, 0xfc, 0xd0, 0x8f, 0x78, 0x0d, 0x16, 0x51, 0xaf, 0xf1, 0xdf, 0x2e, 0xf1, 0xad, 0x20, + 0xe0, 0x2f, 0x30, 0xf1, 0xb7, 0x91, 0xaa, 0x11, 0x30, 0x7e, 0xea, 0x49, 0xfc, 0x72, 0x0c, 0x92, + 0x80, 0x78, 0xea, 0x89, 0xff, 0x6e, 0x0c, 0x12, 0x5c, 0x83, 0xd2, 0xb7, 0x9e, 0x6b, 0x91, 0xe2, + 0x5e, 0xa1, 0x56, 0x15, 0x31, 0xbd, 0x6f, 0xcc, 0xd4, 0x3f, 0xcf, 0xc0, 0xe5, 0xc5, 0x51, 0xa5, + 0x05, 0x53, 0x85, 0x52, 0xb3, 0xdf, 0xd5, 0x7b, 0x8d, 0xfd, 0xb6, 0xb2, 0xc6, 0x36, 0xa1, 0xd2, + 0xdf, 0xf9, 0xa2, 0xdd, 0x1c, 0x72, 0x40, 0x86, 0xee, 0x57, 0x0e, 0xf4, 0xbd, 0x4e, 0xab, 0xd5, + 0xee, 0x71, 0x2d, 0xa5, 0xbf, 0xf3, 0x85, 0xde, 0xed, 0x37, 0xf9, 0x0f, 0x4d, 0x44, 0xde, 0xf7, + 0x81, 0x92, 0x23, 0x8f, 0x37, 0xc5, 0x84, 0x62, 0x32, 0xcf, 0x43, 0x1e, 0x9f, 0x0e, 0xf4, 0x66, + 0x6f, 0xa8, 0x14, 0x30, 0xd5, 0x3b, 0xec, 0x76, 0x29, 0x45, 0xb1, 0x4d, 0xcd, 0xfe, 0xfe, 0x81, + 0xd6, 0x1e, 0x0c, 0xf4, 0x41, 0xe7, 0x67, 0x6d, 0xa5, 0x44, 0x35, 0x6b, 0x9d, 0x47, 0x9d, 0x1e, + 0x07, 0x94, 0x59, 0x11, 0xb2, 0xfb, 0x9d, 0x1e, 0xbf, 0x57, 0xba, 0xdf, 0xf8, 0x52, 0xa9, 0xe0, + 0xc7, 0xe0, 0x70, 0x5f, 0xa9, 0xde, 0x7d, 0x15, 0xaa, 0xf2, 0xaf, 0x35, 0x51, 0x94, 0xa3, 0xe7, + 0x5a, 0xfc, 0x0d, 0xd3, 0xee, 0xb7, 0x1f, 0x2a, 0x99, 0xbb, 0xbf, 0x25, 0xbd, 0x82, 0x4f, 0x34, + 0xc2, 0x19, 0x40, 0x97, 0xe8, 0xf8, 0x2d, 0x3d, 0x32, 0xfd, 0xd3, 0xa5, 0xbe, 0xbd, 0xc6, 0x60, + 0x8f, 0xbb, 0x09, 0x04, 0x86, 0x00, 0xd9, 0xe4, 0xed, 0x4b, 0xba, 0x24, 0x4b, 0x9f, 0xb1, 0xb3, + 0x3d, 0x4f, 0xf7, 0x17, 0x3b, 0x03, 0xec, 0x9c, 0x02, 0x55, 0xfc, 0x8a, 0x71, 0xc5, 0xbb, 0x2a, + 0x54, 0xa4, 0xe7, 0x8a, 0xa9, 0x0e, 0x23, 0x38, 0x16, 0xcf, 0x69, 0xa2, 0xba, 0xa9, 0x64, 0xee, + 0xbe, 0x81, 0x27, 0x86, 0xfc, 0x58, 0x30, 0x40, 0xa1, 0xe7, 0xf9, 0x53, 0xc3, 0x11, 0x74, 0xd6, + 0x3c, 0x40, 0xba, 0xf7, 0xe0, 0xca, 0xca, 0xa7, 0x8f, 0x29, 0x52, 0xd7, 0x9e, 0xce, 0x1c, 0x8b, + 0x07, 0x9b, 0xee, 0x9d, 0x8f, 0x7c, 0xdb, 0x54, 0x32, 0x77, 0x1f, 0x46, 0xf7, 0xec, 0xa2, 0xba, + 0xbb, 0xfd, 0x46, 0x8b, 0x4f, 0x6e, 0x7c, 0x89, 0x77, 0xb8, 0xc3, 0x9f, 0xca, 0xd4, 0xda, 0x83, + 0xc3, 0xee, 0x50, 0x5c, 0x18, 0xbe, 0xfb, 0x39, 0xd4, 0x2f, 0x8a, 0xba, 0xc4, 0x16, 0x35, 0xf7, + 0x1a, 0x14, 0xd9, 0x8a, 0x93, 0xd9, 0xd7, 0x79, 0x2a, 0xc3, 0x03, 0x83, 0xbb, 0x6d, 0x8a, 0xc8, + 0xb8, 0xfb, 0xf3, 0x8c, 0xc4, 0xc2, 0xa2, 0xc8, 0xb9, 0x18, 0x20, 0x66, 0x49, 0x06, 0x69, 0x96, + 0x61, 0x2a, 0x19, 0x76, 0x15, 0x58, 0x0a, 0xd4, 0xf5, 0xc6, 0x86, 0xa3, 0xac, 0x53, 0xec, 0x45, + 0x04, 0xa7, 0xf8, 0x66, 0x25, 0xcb, 0x5e, 0x81, 0x6b, 0x31, 0xac, 0xeb, 0x9d, 0x1e, 0xf8, 0x36, + 0xea, 0xda, 0xe7, 0x1c, 0x9d, 0xdb, 0xf9, 0xc9, 0x9f, 0xfd, 0xf2, 0x66, 0xe6, 0x5f, 0xfe, 0xf2, + 0x66, 0xe6, 0x3f, 0xfc, 0xf2, 0xe6, 0xda, 0x2f, 0xfe, 0xe3, 0xcd, 0xcc, 0xcf, 0xe4, 0xdf, 0x74, + 0x9e, 0x1a, 0xa1, 0x6f, 0x9f, 0xf1, 0x4d, 0x13, 0x25, 0x5c, 0xeb, 0xbd, 0xd9, 0xc9, 0xd1, 0x7b, + 0xb3, 0xd1, 0x7b, 0xc8, 0x99, 0x46, 0x05, 0xfa, 0xf5, 0xe6, 0x07, 0xff, 0x33, 0x00, 0x00, 0xff, + 0xff, 0xf5, 0x10, 0xd1, 0x3f, 0x1d, 0x7a, 0x00, 0x00, } func (m *Type) Marshal() (dAtA []byte, err error) { @@ -17063,10 +17154,43 @@ func (m *UpdateCtx) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } - if m.PartitionIdx != 0 { - i = encodeVarintPlan(dAtA, i, uint64(m.PartitionIdx)) + if len(m.DeleteCols) > 0 { + for iNdEx := len(m.DeleteCols) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.DeleteCols[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintPlan(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x42 + } + } + if len(m.InsertCols) > 0 { + for iNdEx := len(m.InsertCols) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.InsertCols[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintPlan(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x3a + } + } + if m.NewPartitionIdx != 0 { + i = encodeVarintPlan(dAtA, i, uint64(m.NewPartitionIdx)) i-- - dAtA[i] = 0x38 + dAtA[i] = 0x30 + } + if m.OldPartitionIdx != 0 { + i = encodeVarintPlan(dAtA, i, uint64(m.OldPartitionIdx)) + i-- + dAtA[i] = 0x28 } if len(m.PartitionTableNames) > 0 { for iNdEx := len(m.PartitionTableNames) - 1; iNdEx >= 0; iNdEx-- { @@ -17074,7 +17198,7 @@ func (m *UpdateCtx) MarshalToSizedBuffer(dAtA []byte) (int, error) { copy(dAtA[i:], m.PartitionTableNames[iNdEx]) i = encodeVarintPlan(dAtA, i, uint64(len(m.PartitionTableNames[iNdEx]))) i-- - dAtA[i] = 0x32 + dAtA[i] = 0x22 } } if len(m.PartitionTableIds) > 0 { @@ -17093,35 +17217,7 @@ func (m *UpdateCtx) MarshalToSizedBuffer(dAtA []byte) (int, error) { copy(dAtA[i:], dAtA63[:j62]) i = encodeVarintPlan(dAtA, i, uint64(j62)) i-- - dAtA[i] = 0x2a - } - if len(m.DeleteCols) > 0 { - for iNdEx := len(m.DeleteCols) - 1; iNdEx >= 0; iNdEx-- { - { - size, err := m.DeleteCols[iNdEx].MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintPlan(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x22 - } - } - if len(m.InsertCols) > 0 { - for iNdEx := len(m.InsertCols) - 1; iNdEx >= 0; iNdEx-- { - { - size, err := m.InsertCols[iNdEx].MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintPlan(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x1a - } + dAtA[i] = 0x1a } if m.TableDef != nil { { @@ -17675,6 +17771,38 @@ func (m *Node) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if len(m.DedupColTypes) > 0 { + for iNdEx := len(m.DedupColTypes) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.DedupColTypes[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintPlan(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x4 + i-- + dAtA[i] = 0xa2 + } + } + if len(m.DedupColName) > 0 { + i -= len(m.DedupColName) + copy(dAtA[i:], m.DedupColName) + i = encodeVarintPlan(dAtA, i, uint64(len(m.DedupColName))) + i-- + dAtA[i] = 0x4 + i-- + dAtA[i] = 0x9a + } + if m.OnDuplicateAction != 0 { + i = encodeVarintPlan(dAtA, i, uint64(m.OnDuplicateAction)) + i-- + dAtA[i] = 0x4 + i-- + dAtA[i] = 0x90 + } if m.PostDmlCtx != nil { { size, err := m.PostDmlCtx.MarshalToSizedBuffer(dAtA[:i]) @@ -18256,6 +18384,22 @@ func (m *Node) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0xb2 } + if len(m.InsertDeleteCols) > 0 { + for iNdEx := len(m.InsertDeleteCols) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.InsertDeleteCols[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintPlan(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xaa + } + } if m.RowsetData != nil { { size, err := m.RowsetData.MarshalToSizedBuffer(dAtA[:i]) @@ -18723,6 +18867,16 @@ func (m *LockTarget) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if m.FilterColRelPos != 0 { + i = encodeVarintPlan(dAtA, i, uint64(m.FilterColRelPos)) + i-- + dAtA[i] = 0x60 + } + if m.PrimaryColRelPos != 0 { + i = encodeVarintPlan(dAtA, i, uint64(m.PrimaryColRelPos)) + i-- + dAtA[i] = 0x58 + } if m.Mode != 0 { i = encodeVarintPlan(dAtA, i, uint64(m.Mode)) i-- @@ -25414,18 +25568,6 @@ func (m *UpdateCtx) ProtoSize() (n int) { l = m.TableDef.ProtoSize() n += 1 + l + sovPlan(uint64(l)) } - if len(m.InsertCols) > 0 { - for _, e := range m.InsertCols { - l = e.ProtoSize() - n += 1 + l + sovPlan(uint64(l)) - } - } - if len(m.DeleteCols) > 0 { - for _, e := range m.DeleteCols { - l = e.ProtoSize() - n += 1 + l + sovPlan(uint64(l)) - } - } if len(m.PartitionTableIds) > 0 { l = 0 for _, e := range m.PartitionTableIds { @@ -25439,8 +25581,23 @@ func (m *UpdateCtx) ProtoSize() (n int) { n += 1 + l + sovPlan(uint64(l)) } } - if m.PartitionIdx != 0 { - n += 1 + sovPlan(uint64(m.PartitionIdx)) + if m.OldPartitionIdx != 0 { + n += 1 + sovPlan(uint64(m.OldPartitionIdx)) + } + if m.NewPartitionIdx != 0 { + n += 1 + sovPlan(uint64(m.NewPartitionIdx)) + } + if len(m.InsertCols) > 0 { + for _, e := range m.InsertCols { + l = e.ProtoSize() + n += 1 + l + sovPlan(uint64(l)) + } + } + if len(m.DeleteCols) > 0 { + for _, e := range m.DeleteCols { + l = e.ProtoSize() + n += 1 + l + sovPlan(uint64(l)) + } } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) @@ -25766,6 +25923,12 @@ func (m *Node) ProtoSize() (n int) { l = m.RowsetData.ProtoSize() n += 2 + l + sovPlan(uint64(l)) } + if len(m.InsertDeleteCols) > 0 { + for _, e := range m.InsertDeleteCols { + l = e.ProtoSize() + n += 2 + l + sovPlan(uint64(l)) + } + } l = len(m.ExtraOptions) if l > 0 { n += 2 + l + sovPlan(uint64(l)) @@ -25952,6 +26115,19 @@ func (m *Node) ProtoSize() (n int) { l = m.PostDmlCtx.ProtoSize() n += 2 + l + sovPlan(uint64(l)) } + if m.OnDuplicateAction != 0 { + n += 2 + sovPlan(uint64(m.OnDuplicateAction)) + } + l = len(m.DedupColName) + if l > 0 { + n += 2 + l + sovPlan(uint64(l)) + } + if len(m.DedupColTypes) > 0 { + for _, e := range m.DedupColTypes { + l = e.ProtoSize() + n += 2 + l + sovPlan(uint64(l)) + } + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -26089,6 +26265,12 @@ func (m *LockTarget) ProtoSize() (n int) { if m.Mode != 0 { n += 1 + sovPlan(uint64(m.Mode)) } + if m.PrimaryColRelPos != 0 { + n += 1 + sovPlan(uint64(m.PrimaryColRelPos)) + } + if m.FilterColRelPos != 0 { + n += 1 + sovPlan(uint64(m.FilterColRelPos)) + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -38563,74 +38745,6 @@ func (m *UpdateCtx) Unmarshal(dAtA []byte) error { } iNdEx = postIndex case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field InsertCols", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPlan - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthPlan - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthPlan - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.InsertCols = append(m.InsertCols, &Expr{}) - if err := m.InsertCols[len(m.InsertCols)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 4: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field DeleteCols", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPlan - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthPlan - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthPlan - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.DeleteCols = append(m.DeleteCols, &Expr{}) - if err := m.DeleteCols[len(m.DeleteCols)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 5: if wireType == 0 { var v uint64 for shift := uint(0); ; shift += 7 { @@ -38706,7 +38820,7 @@ func (m *UpdateCtx) Unmarshal(dAtA []byte) error { } else { return fmt.Errorf("proto: wrong wireType = %d for field PartitionTableIds", wireType) } - case 6: + case 4: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field PartitionTableNames", wireType) } @@ -38738,11 +38852,11 @@ func (m *UpdateCtx) Unmarshal(dAtA []byte) error { } m.PartitionTableNames = append(m.PartitionTableNames, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex - case 7: + case 5: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field PartitionIdx", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field OldPartitionIdx", wireType) } - m.PartitionIdx = 0 + m.OldPartitionIdx = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowPlan @@ -38752,11 +38866,98 @@ func (m *UpdateCtx) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.PartitionIdx |= int32(b&0x7F) << shift + m.OldPartitionIdx |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field NewPartitionIdx", wireType) + } + m.NewPartitionIdx = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPlan + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.NewPartitionIdx |= int32(b&0x7F) << shift if b < 0x80 { break } } + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field InsertCols", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPlan + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthPlan + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthPlan + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.InsertCols = append(m.InsertCols, ColRef{}) + if err := m.InsertCols[len(m.InsertCols)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DeleteCols", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPlan + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthPlan + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthPlan + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.DeleteCols = append(m.DeleteCols, ColRef{}) + if err := m.DeleteCols[len(m.DeleteCols)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipPlan(dAtA[iNdEx:]) @@ -40958,6 +41159,40 @@ func (m *Node) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 21: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field InsertDeleteCols", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPlan + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthPlan + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthPlan + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.InsertDeleteCols = append(m.InsertDeleteCols, &Expr{}) + if err := m.InsertDeleteCols[len(m.InsertDeleteCols)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex case 22: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field ExtraOptions", wireType) @@ -42393,6 +42628,91 @@ func (m *Node) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 66: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field OnDuplicateAction", wireType) + } + m.OnDuplicateAction = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPlan + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.OnDuplicateAction |= Node_OnDuplicateAction(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 67: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DedupColName", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPlan + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthPlan + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthPlan + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.DedupColName = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 68: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DedupColTypes", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPlan + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthPlan + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthPlan + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.DedupColTypes = append(m.DedupColTypes, Type{}) + if err := m.DedupColTypes[len(m.DedupColTypes)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipPlan(dAtA[iNdEx:]) @@ -43350,6 +43670,44 @@ func (m *LockTarget) Unmarshal(dAtA []byte) error { break } } + case 11: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field PrimaryColRelPos", wireType) + } + m.PrimaryColRelPos = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPlan + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.PrimaryColRelPos |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 12: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field FilterColRelPos", wireType) + } + m.FilterColRelPos = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPlan + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.FilterColRelPos |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipPlan(dAtA[iNdEx:]) diff --git a/pkg/sql/colexec/dedupjoin/join.go b/pkg/sql/colexec/dedupjoin/join.go new file mode 100644 index 000000000000..d951aa22cd6b --- /dev/null +++ b/pkg/sql/colexec/dedupjoin/join.go @@ -0,0 +1,298 @@ +// Copyright 2021 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package dedupjoin + +import ( + "bytes" + "strings" + "time" + + "github.com/matrixorigin/matrixone/pkg/catalog" + "github.com/matrixorigin/matrixone/pkg/common/bitmap" + "github.com/matrixorigin/matrixone/pkg/common/hashmap" + "github.com/matrixorigin/matrixone/pkg/common/moerr" + "github.com/matrixorigin/matrixone/pkg/container/batch" + "github.com/matrixorigin/matrixone/pkg/container/types" + "github.com/matrixorigin/matrixone/pkg/container/vector" + "github.com/matrixorigin/matrixone/pkg/pb/plan" + "github.com/matrixorigin/matrixone/pkg/sql/colexec" + "github.com/matrixorigin/matrixone/pkg/vm" + "github.com/matrixorigin/matrixone/pkg/vm/message" + "github.com/matrixorigin/matrixone/pkg/vm/process" +) + +const opName = "dedup_join" + +func (dedupJoin *DedupJoin) String(buf *bytes.Buffer) { + buf.WriteString(opName) + buf.WriteString(": dedup join ") +} + +func (dedupJoin *DedupJoin) OpType() vm.OpType { + return vm.DedupJoin +} + +func (dedupJoin *DedupJoin) Prepare(proc *process.Process) (err error) { + if dedupJoin.OpAnalyzer == nil { + dedupJoin.OpAnalyzer = process.NewAnalyzer(dedupJoin.GetIdx(), dedupJoin.IsFirst, dedupJoin.IsLast, "dedup join") + } else { + dedupJoin.OpAnalyzer.Reset() + } + + if len(dedupJoin.ctr.vecs) == 0 { + dedupJoin.ctr.vecs = make([]*vector.Vector, len(dedupJoin.Conditions[0])) + dedupJoin.ctr.evecs = make([]evalVector, len(dedupJoin.Conditions[0])) + for i := range dedupJoin.ctr.evecs { + dedupJoin.ctr.evecs[i].executor, err = colexec.NewExpressionExecutor(proc, dedupJoin.Conditions[0][i]) + if err != nil { + return err + } + } + } + + return err +} + +func (dedupJoin *DedupJoin) Call(proc *process.Process) (vm.CallResult, error) { + if err, isCancel := vm.CancelCheck(proc); isCancel { + return vm.CancelResult, err + } + + analyzer := dedupJoin.OpAnalyzer + analyzer.Start() + defer analyzer.Stop() + + ctr := &dedupJoin.ctr + result := vm.NewCallResult() + var err error + for { + switch ctr.state { + case Build: + err = dedupJoin.build(analyzer, proc) + if err != nil { + return result, err + } + + if ctr.mp == nil && !dedupJoin.IsShuffle { + ctr.state = End + } else { + ctr.state = Probe + } + + case Probe: + result, err = vm.ChildrenCall(dedupJoin.GetChildren(0), proc, analyzer) + if err != nil { + return result, err + } + + bat := result.Batch + if bat == nil { + ctr.state = Finalize + dedupJoin.ctr.buf = nil + continue + } + if bat.IsEmpty() { + continue + } + + if ctr.batchRowCount == 0 { + continue + } + + if err := ctr.probe(bat, dedupJoin, proc, analyzer); err != nil { + return result, err + } + + case Finalize: + if dedupJoin.ctr.buf == nil { + dedupJoin.ctr.lastPos = 0 + err := ctr.finalize(dedupJoin, proc) + if err != nil { + return result, err + } + } + + if dedupJoin.ctr.lastPos >= len(dedupJoin.ctr.buf) { + ctr.state = End + continue + } + + result.Batch = dedupJoin.ctr.buf[dedupJoin.ctr.lastPos] + dedupJoin.ctr.lastPos++ + result.Status = vm.ExecHasMore + analyzer.Output(result.Batch) + return result, nil + + default: + result.Batch = nil + result.Status = vm.ExecStop + return result, nil + } + } +} + +func (dedupJoin *DedupJoin) build(analyzer process.Analyzer, proc *process.Process) (err error) { + ctr := &dedupJoin.ctr + start := time.Now() + defer analyzer.WaitStop(start) + ctr.mp, err = message.ReceiveJoinMap(dedupJoin.JoinMapTag, dedupJoin.IsShuffle, dedupJoin.ShuffleIdx, proc.GetMessageBoard(), proc.Ctx) + if err != nil { + return + } + if ctr.mp != nil { + ctr.maxAllocSize = max(ctr.maxAllocSize, ctr.mp.Size()) + } + ctr.batches = ctr.mp.GetBatches() + ctr.batchRowCount = ctr.mp.GetRowCount() + if ctr.batchRowCount > 0 { + ctr.matched = &bitmap.Bitmap{} + ctr.matched.InitWithSize(ctr.batchRowCount) + } + return +} + +func (ctr *container) finalize(ap *DedupJoin, proc *process.Process) error { + ctr.handledLast = true + + if ctr.matched == nil { + return nil + } + + if ap.NumCPU > 1 { + if !ap.IsMerger { + ap.Channel <- ctr.matched + return nil + } else { + for cnt := 1; cnt < int(ap.NumCPU); cnt++ { + v := colexec.ReceiveBitmapFromChannel(proc.Ctx, ap.Channel) + if v != nil { + ctr.matched.Or(v) + } else { + return nil + } + } + close(ap.Channel) + } + } + + if ctr.matched.Count() == 0 { + ap.ctr.buf = ctr.batches + ctr.batches = nil + return nil + } + + count := ctr.batchRowCount - int64(ctr.matched.Count()) + ctr.matched.Negate() + sels := make([]int32, 0, count) + itr := ctr.matched.Iterator() + for itr.HasNext() { + r := itr.Next() + sels = append(sels, int32(r)) + } + + n := (len(sels)-1)/colexec.DefaultBatchSize + 1 + ap.ctr.buf = make([]*batch.Batch, n) + for k := range ap.ctr.buf { + ap.ctr.buf[k] = batch.NewWithSize(len(ap.Result)) + for i, pos := range ap.Result { + ap.ctr.buf[k].Vecs[i] = vector.NewVec(ap.RightTypes[pos]) + } + var newsels []int32 + if (k+1)*colexec.DefaultBatchSize <= len(sels) { + newsels = sels[k*colexec.DefaultBatchSize : (k+1)*colexec.DefaultBatchSize] + } else { + newsels = sels[k*colexec.DefaultBatchSize:] + } + for _, sel := range newsels { + idx1, idx2 := sel/colexec.DefaultBatchSize, sel%colexec.DefaultBatchSize + for j, pos := range ap.Result { + if err := ap.ctr.buf[k].Vecs[j].UnionOne(ctr.batches[idx1].Vecs[pos], int64(idx2), proc.Mp()); err != nil { + return err + } + } + } + ap.ctr.buf[k].SetRowCount(len(newsels)) + } + return nil +} + +func (ctr *container) probe(bat *batch.Batch, ap *DedupJoin, proc *process.Process, analyzer process.Analyzer) error { + analyzer.Input(bat) + + if err := ctr.evalJoinCondition(bat, proc); err != nil { + return err + } + count := bat.RowCount() + itr := ctr.mp.NewIterator() + isPessimistic := proc.GetTxnOperator().Txn().IsPessimistic() + for i := 0; i < count; i += hashmap.UnitLimit { + n := count - i + if n > hashmap.UnitLimit { + n = hashmap.UnitLimit + } + vals, zvals := itr.Find(i, n, ctr.vecs) + for k := 0; k < n; k++ { + if zvals[k] == 0 || vals[k] == 0 { + continue + } + + switch ap.OnDuplicateAction { + case plan.Node_ERROR: + // do nothing for txn.mode = Optimistic + if isPessimistic { + var rowStr string + if len(ap.DedupColTypes) == 1 { + if ap.DedupColName == catalog.IndexTableIndexColName { + if ctr.vecs[0].GetType().Oid == types.T_varchar { + t, _, schema, err := types.DecodeTuple(ctr.vecs[0].GetBytesAt(i + k)) + if err == nil && len(schema) > 1 { + rowStr = t.ErrString(make([]int32, len(schema))) + } + } + } + + if len(rowStr) == 0 { + rowStr = ctr.vecs[0].RowToString(i + k) + } + } else { + rowItems, err := types.StringifyTuple(ctr.vecs[0].GetBytesAt(i+k), ap.DedupColTypes) + if err != nil { + return err + } + rowStr = "(" + strings.Join(rowItems, ",") + ")" + } + return moerr.NewDuplicateEntry(proc.Ctx, rowStr, ap.DedupColName) + } + case plan.Node_IGNORE: + ctr.matched.Add(vals[k] - 1) + case plan.Node_UPDATE: // TODO + } + } + } + + return nil +} + +func (ctr *container) evalJoinCondition(bat *batch.Batch, proc *process.Process) error { + for i := range ctr.evecs { + vec, err := ctr.evecs[i].executor.Eval(proc, []*batch.Batch{bat}, nil) + if err != nil { + return err + } + ctr.vecs[i] = vec + ctr.evecs[i].vec = vec + } + return nil +} diff --git a/pkg/sql/colexec/dedupjoin/join_test.go b/pkg/sql/colexec/dedupjoin/join_test.go new file mode 100644 index 000000000000..4c38d2e20ad1 --- /dev/null +++ b/pkg/sql/colexec/dedupjoin/join_test.go @@ -0,0 +1,291 @@ +// Copyright 2021 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package dedupjoin + +import ( + "bytes" + "context" + "testing" + + "github.com/matrixorigin/matrixone/pkg/common/mpool" + "github.com/matrixorigin/matrixone/pkg/container/batch" + "github.com/matrixorigin/matrixone/pkg/container/types" + "github.com/matrixorigin/matrixone/pkg/pb/plan" + "github.com/matrixorigin/matrixone/pkg/sql/colexec" + "github.com/matrixorigin/matrixone/pkg/sql/colexec/hashbuild" + "github.com/matrixorigin/matrixone/pkg/testutil" + "github.com/matrixorigin/matrixone/pkg/vm" + "github.com/matrixorigin/matrixone/pkg/vm/message" + "github.com/matrixorigin/matrixone/pkg/vm/process" + "github.com/stretchr/testify/require" +) + +const ( + Rows = 10 // default rows + BenchmarkRows = 100000 // default rows for benchmark +) + +// add unit tests for cases +type joinTestCase struct { + arg *DedupJoin + flgs []bool // flgs[i] == true: nullable + types []types.Type + proc *process.Process + cancel context.CancelFunc + barg *hashbuild.HashBuild +} + +var ( + tag int32 + tcs []joinTestCase +) + +func init() { + tcs = []joinTestCase{ + newTestCase([]bool{false}, []types.Type{types.T_int32.ToType()}, []int32{0}, + [][]*plan.Expr{ + { + newExpr(0, types.T_int32.ToType()), + }, + { + newExpr(0, types.T_int32.ToType()), + }, + }), + newTestCase([]bool{true}, []types.Type{types.T_int32.ToType()}, []int32{1}, + [][]*plan.Expr{ + { + newExpr(0, types.T_int32.ToType()), + }, + { + newExpr(0, types.T_int32.ToType()), + }, + }), + } +} + +func TestString(t *testing.T) { + buf := new(bytes.Buffer) + for _, tc := range tcs { + tc.arg.String(buf) + } +} + +func TestDedupJoin(t *testing.T) { + for _, tc := range tcs { + resetChildren(tc.arg) + resetHashBuildChildren(tc.barg) + err := tc.arg.Prepare(tc.proc) + require.NoError(t, err) + tc.barg.IsDedup = true + err = tc.barg.Prepare(tc.proc) + require.NoError(t, err) + + res, err := tc.barg.Call(tc.proc) + require.NoError(t, err) + require.Equal(t, res.Batch == nil, true) + res, err = tc.arg.Call(tc.proc) + require.NoError(t, err) + require.Equal(t, true, res.Batch == nil) + + tc.arg.Reset(tc.proc, false, nil) + tc.barg.Reset(tc.proc, false, nil) + + resetChildren(tc.arg) + resetHashBuildChildren(tc.barg) + tc.proc.GetMessageBoard().Reset() + err = tc.arg.Prepare(tc.proc) + require.NoError(t, err) + tc.barg.IsDedup = true + err = tc.barg.Prepare(tc.proc) + require.NoError(t, err) + + res, err = tc.barg.Call(tc.proc) + require.NoError(t, err) + require.Equal(t, res.Batch == nil, true) + res, err = tc.arg.Call(tc.proc) + require.NoError(t, err) + require.Equal(t, true, res.Batch == nil) + + tc.arg.Reset(tc.proc, false, nil) + tc.barg.Reset(tc.proc, false, nil) + + tc.arg.Free(tc.proc, false, nil) + tc.barg.Free(tc.proc, false, nil) + + resetChildren(tc.arg) + resetHashBuildChildren(tc.barg) + tc.proc.GetMessageBoard().Reset() + tc.arg.OnDuplicateAction = plan.Node_IGNORE + err = tc.arg.Prepare(tc.proc) + require.NoError(t, err) + tc.barg.IsDedup = true + tc.barg.OnDuplicateAction = plan.Node_IGNORE + err = tc.barg.Prepare(tc.proc) + require.NoError(t, err) + + res, err = tc.barg.Call(tc.proc) + require.NoError(t, err) + require.Equal(t, res.Batch == nil, true) + res, err = tc.arg.Call(tc.proc) + require.NoError(t, err) + require.Equal(t, true, res.Batch == nil) + + tc.arg.Reset(tc.proc, false, nil) + tc.barg.Reset(tc.proc, false, nil) + + tc.arg.Free(tc.proc, false, nil) + tc.barg.Free(tc.proc, false, nil) + + tc.proc.Free() + require.Equal(t, int64(0), tc.proc.Mp().CurrNB()) + } +} + +/* + func BenchmarkJoin(b *testing.B) { + for i := 0; i < b.N; i++ { + tcs = []joinTestCase{ + newTestCase([]bool{false}, []types.Type{types.T_int8.ToType()}, []int32{0}, + [][]*plan.Expr{ + { + newExpr(0, types.T_int8.ToType()), + }, + { + newExpr(0, types.T_int8.ToType()), + }, + }), + newTestCase([]bool{true}, []types.Type{types.T_int8.ToType()}, []int32{0}, + [][]*plan.Expr{ + { + newExpr(0, types.T_int8.ToType()), + }, + { + newExpr(0, types.T_int8.ToType()), + }, + }), + } + t := new(testing.T) + for _, tc := range tcs { + bats := hashBuild(t, tc) + err := tc.arg.Prepare(tc.proc) + require.NoError(t, err) + tc.proc.Reg.MergeReceivers[0].Ch <- testutil.NewRegMsg(newBatch(tc.types, tc.proc, Rows)) + tc.proc.Reg.MergeReceivers[0].Ch <- testutil.NewRegMsg(batch.EmptyBatch) + tc.proc.Reg.MergeReceivers[0].Ch <- testutil.NewRegMsg(newBatch(tc.types, tc.proc, Rows)) + tc.proc.Reg.MergeReceivers[0].Ch <- testutil.NewRegMsg(newBatch(tc.types, tc.proc, Rows)) + tc.proc.Reg.MergeReceivers[0].Ch <- testutil.NewRegMsg(newBatch(tc.types, tc.proc, Rows)) + tc.proc.Reg.MergeReceivers[0].Ch <- nil + tc.proc.Reg.MergeReceivers[1].Ch <- testutil.NewRegMsg(bats[0]) + tc.proc.Reg.MergeReceivers[1].Ch <- testutil.NewRegMsg(bats[1]) + for { + ok, err := tc.arg.Call(tc.proc) + if ok.Status == vm.ExecStop || err != nil { + break + } + } + } + } + } +*/ +func newExpr(pos int32, typ types.Type) *plan.Expr { + return &plan.Expr{ + Typ: plan.Type{ + Scale: typ.Scale, + Width: typ.Width, + Id: int32(typ.Oid), + }, + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + ColPos: pos, + }, + }, + } +} + +func newTestCase(flgs []bool, ts []types.Type, rp []int32, cs [][]*plan.Expr) joinTestCase { + proc := testutil.NewProcessWithMPool("", mpool.MustNewZero()) + proc.SetMessageBoard(message.NewMessageBoard()) + _, cancel := context.WithCancel(context.Background()) + //args := make([]*plan.Expr, 0, 2) + //args = append(args, &plan.Expr{ + // Typ: plan.Type{ + // Id: int32(ts[0].Oid), + // }, + // Expr: &plan.Expr_Col{ + // Col: &plan.ColRef{ + // RelPos: 0, + // ColPos: 0, + // }, + // }, + //}) + //args = append(args, &plan.Expr{ + // Typ: plan.Type{ + // Id: int32(ts[0].Oid), + // }, + // Expr: &plan.Expr_Col{ + // Col: &plan.ColRef{ + // RelPos: 1, + // ColPos: 0, + // }, + // }, + //}) + tag++ + return joinTestCase{ + types: ts, + flgs: flgs, + proc: proc, + cancel: cancel, + arg: &DedupJoin{ + RightTypes: ts, + Conditions: cs, + OperatorBase: vm.OperatorBase{ + OperatorInfo: vm.OperatorInfo{ + Idx: 0, + IsFirst: false, + IsLast: false, + }, + }, + JoinMapTag: tag, + }, + barg: &hashbuild.HashBuild{ + NeedHashMap: true, + Conditions: cs[1], + OperatorBase: vm.OperatorBase{ + OperatorInfo: vm.OperatorInfo{ + Idx: 0, + IsFirst: false, + IsLast: false, + }, + }, + NeedAllocateSels: true, + JoinMapTag: tag, + JoinMapRefCnt: 1, + }, + } +} + +func resetChildren(arg *DedupJoin) { + bat := colexec.MakeMockBatchs() + op := colexec.NewMockOperator().WithBatchs([]*batch.Batch{bat}) + arg.Children = nil + arg.AppendChild(op) +} + +func resetHashBuildChildren(arg *hashbuild.HashBuild) { + bat := colexec.MakeMockBatchs() + op := colexec.NewMockOperator().WithBatchs([]*batch.Batch{bat}) + arg.Children = nil + arg.AppendChild(op) +} diff --git a/pkg/sql/colexec/dedupjoin/types.go b/pkg/sql/colexec/dedupjoin/types.go new file mode 100644 index 000000000000..279521ccdb08 --- /dev/null +++ b/pkg/sql/colexec/dedupjoin/types.go @@ -0,0 +1,195 @@ +// Copyright 2021 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package dedupjoin + +import ( + "github.com/matrixorigin/matrixone/pkg/common/bitmap" + "github.com/matrixorigin/matrixone/pkg/common/reuse" + "github.com/matrixorigin/matrixone/pkg/container/batch" + "github.com/matrixorigin/matrixone/pkg/container/types" + "github.com/matrixorigin/matrixone/pkg/container/vector" + "github.com/matrixorigin/matrixone/pkg/pb/plan" + "github.com/matrixorigin/matrixone/pkg/sql/colexec" + "github.com/matrixorigin/matrixone/pkg/vm" + "github.com/matrixorigin/matrixone/pkg/vm/message" + "github.com/matrixorigin/matrixone/pkg/vm/process" +) + +var _ vm.Operator = new(DedupJoin) + +const ( + Build = iota + Probe + Finalize + End +) + +type evalVector struct { + executor colexec.ExpressionExecutor + vec *vector.Vector +} + +type container struct { + state int + lastPos int + + batches []*batch.Batch + batchRowCount int64 + + expr colexec.ExpressionExecutor + + evecs []evalVector + vecs []*vector.Vector + + mp *message.JoinMap + + matched *bitmap.Bitmap + handledLast bool + + maxAllocSize int64 + buf []*batch.Batch +} + +type DedupJoin struct { + ctr container + Result []int32 + RightTypes []types.Type + Conditions [][]*plan.Expr + + IsShuffle bool + ShuffleIdx int32 + RuntimeFilterSpecs []*plan.RuntimeFilterSpec + JoinMapTag int32 + + Channel chan *bitmap.Bitmap + NumCPU uint64 + IsMerger bool + + OnDuplicateAction plan.Node_OnDuplicateAction + DedupColName string + DedupColTypes []plan.Type + + vm.OperatorBase + colexec.Projection +} + +func (dedupJoin *DedupJoin) GetOperatorBase() *vm.OperatorBase { + return &dedupJoin.OperatorBase +} + +func init() { + reuse.CreatePool( + func() *DedupJoin { + return &DedupJoin{} + }, + func(a *DedupJoin) { + *a = DedupJoin{} + }, + reuse.DefaultOptions[DedupJoin](). + WithEnableChecker(), + ) +} + +func (dedupJoin DedupJoin) TypeName() string { + return opName +} + +func NewArgument() *DedupJoin { + return reuse.Alloc[DedupJoin](nil) +} + +func (dedupJoin *DedupJoin) Release() { + if dedupJoin != nil { + reuse.Free[DedupJoin](dedupJoin, nil) + } +} + +func (dedupJoin *DedupJoin) Reset(proc *process.Process, pipelineFailed bool, err error) { + ctr := &dedupJoin.ctr + if !ctr.handledLast && dedupJoin.NumCPU > 1 && !dedupJoin.IsMerger { + dedupJoin.Channel <- nil + } + if dedupJoin.OpAnalyzer != nil { + dedupJoin.OpAnalyzer.Alloc(ctr.maxAllocSize) + } + ctr.maxAllocSize = 0 + + ctr.cleanBuf(proc) + ctr.cleanHashMap() + ctr.resetExprExecutor() + ctr.resetEvalVectors() + ctr.handledLast = false + ctr.state = Build + ctr.lastPos = 0 +} + +func (dedupJoin *DedupJoin) Free(proc *process.Process, pipelineFailed bool, err error) { + ctr := &dedupJoin.ctr + ctr.cleanBuf(proc) + ctr.cleanEvalVectors() + ctr.cleanHashMap() + ctr.cleanExprExecutor() + +} + +func (ctr *container) resetExprExecutor() { + if ctr.expr != nil { + ctr.expr.ResetForNextQuery() + } +} + +func (ctr *container) cleanExprExecutor() { + if ctr.expr != nil { + ctr.expr.Free() + ctr.expr = nil + } +} + +func (ctr *container) cleanBuf(proc *process.Process) { + if ctr.matched != nil && ctr.matched.Count() == 0 { + // hash map will free these batches + ctr.buf = nil + return + } + for _, bat := range ctr.buf { + bat.Clean(proc.GetMPool()) + } + ctr.buf = nil +} + +func (ctr *container) cleanHashMap() { + if ctr.mp != nil { + ctr.mp.Free() + ctr.mp = nil + } +} + +func (ctr *container) cleanEvalVectors() { + for i := range ctr.evecs { + if ctr.evecs[i].executor != nil { + ctr.evecs[i].executor.Free() + } + ctr.evecs[i].vec = nil + } + ctr.evecs = nil +} + +func (ctr *container) resetEvalVectors() { + for i := range ctr.evecs { + if ctr.evecs[i].executor != nil { + ctr.evecs[i].executor.ResetForNextQuery() + } + } +} diff --git a/pkg/sql/colexec/deletion/types.go b/pkg/sql/colexec/deletion/types.go index 11589911f712..f665306ee8a0 100644 --- a/pkg/sql/colexec/deletion/types.go +++ b/pkg/sql/colexec/deletion/types.go @@ -209,7 +209,7 @@ func (deletion *Deletion) Free(proc *process.Process, pipelineFailed bool, err e ctr.source = nil } -func (deletion *Deletion) AffectedRows() uint64 { +func (deletion *Deletion) GetAffectedRows() uint64 { return deletion.ctr.affectedRows } diff --git a/pkg/sql/colexec/dispatch/dispatch.go b/pkg/sql/colexec/dispatch/dispatch.go index 485a71689d23..a832d6502cb2 100644 --- a/pkg/sql/colexec/dispatch/dispatch.go +++ b/pkg/sql/colexec/dispatch/dispatch.go @@ -18,13 +18,10 @@ import ( "bytes" "context" - "github.com/matrixorigin/matrixone/pkg/container/pSpool" - - "github.com/matrixorigin/matrixone/pkg/logutil" - "github.com/google/uuid" - "github.com/matrixorigin/matrixone/pkg/common/moerr" + "github.com/matrixorigin/matrixone/pkg/container/pSpool" + "github.com/matrixorigin/matrixone/pkg/logutil" "github.com/matrixorigin/matrixone/pkg/sql/colexec" "github.com/matrixorigin/matrixone/pkg/vm" "github.com/matrixorigin/matrixone/pkg/vm/process" diff --git a/pkg/sql/colexec/external/external.go b/pkg/sql/colexec/external/external.go index af9f660e46d3..098c4a112285 100644 --- a/pkg/sql/colexec/external/external.go +++ b/pkg/sql/colexec/external/external.go @@ -823,16 +823,6 @@ func makeType(typ *plan.Type, flag bool) types.Type { return types.New(types.T(typ.Id), typ.Width, typ.Scale) } -func initBatch(batchSize int, proc *process.Process, bat *batch.Batch) error { - if err := bat.PreExtend(proc.GetMPool(), batchSize); err != nil { - return err - } - for i := range bat.Vecs { - bat.Vecs[i].SetLength(batchSize) - } - return nil -} - func getRealAttrCnt(attrs []string, cols []*plan.ColDef) int { cnt := 0 for i := 0; i < len(attrs); i++ { @@ -1275,11 +1265,7 @@ func getOneRowDataNonRestrictive(bat *batch.Batch, line []csvparser.Field, rowId continue } vec := bat.Vecs[colIdx] - if param.Cols[colIdx].Hidden { - nulls.Add(vec.GetNulls(), uint64(rowIdx)) - continue - } - nulls.Add(vec.GetNulls(), uint64(rowIdx)) + vector.AppendBytes(vec, nil, true, mp) } return nil } @@ -1296,7 +1282,7 @@ func getColData(bat *batch.Batch, line []csvparser.Field, rowIdx int, param *Ext vec := bat.Vecs[colIdx] if param.Cols[colIdx].Hidden { - nulls.Add(vec.GetNulls(), uint64(rowIdx)) + vector.AppendBytes(vec, nil, true, mp) return nil } @@ -1312,12 +1298,12 @@ func getColData(bat *batch.Batch, line []csvparser.Field, rowIdx int, param *Ext isNullOrEmpty = isNullOrEmpty || len(field.Val) == 0 } if isNullOrEmpty { - nulls.Add(vec.GetNulls(), uint64(rowIdx)) + vector.AppendBytes(vec, nil, true, mp) return nil } if param.ParallelLoad { - err := vector.SetStringAt(vec, rowIdx, field.Val, mp) + err := vector.AppendBytes(vec, []byte(field.Val), false, mp) if err != nil { return err } @@ -1330,7 +1316,7 @@ func getColData(bat *batch.Batch, line []csvparser.Field, rowIdx int, param *Ext if err != nil { return moerr.NewInternalErrorf(param.Ctx, "the input value '%s' is not bool type for column %d", field.Val, colIdx) } - if err := vector.SetFixedAtNoTypeCheck(vec, rowIdx, b); err != nil { + if err = vector.AppendFixed(vec, b, false, mp); err != nil { return err } case types.T_bit: @@ -1346,13 +1332,14 @@ func getColData(bat *batch.Batch, line []csvparser.Field, rowIdx int, param *Ext if val > uint64(1<= size { + if curBatchSize >= param.maxBatchSize { break } } - for i := range bat.Vecs { - bat.Vecs[i].SetLength(cnt) - } n := bat.Vecs[0].Length() if unexpectEOF && n > 0 { n-- diff --git a/pkg/sql/colexec/hashbuild/build.go b/pkg/sql/colexec/hashbuild/build.go index 425c41bdfa12..dddbd8a628c4 100644 --- a/pkg/sql/colexec/hashbuild/build.go +++ b/pkg/sql/colexec/hashbuild/build.go @@ -43,6 +43,10 @@ func (hashBuild *HashBuild) Prepare(proc *process.Process) (err error) { } if hashBuild.NeedHashMap { + hashBuild.ctr.hashmapBuilder.IsDedup = hashBuild.IsDedup + hashBuild.ctr.hashmapBuilder.OnDuplicateAction = hashBuild.OnDuplicateAction + hashBuild.ctr.hashmapBuilder.DedupColName = hashBuild.DedupColName + hashBuild.ctr.hashmapBuilder.DedupColTypes = hashBuild.DedupColTypes return hashBuild.ctr.hashmapBuilder.Prepare(hashBuild.Conditions, proc) } return nil @@ -82,6 +86,7 @@ func (hashBuild *HashBuild) Call(proc *process.Process) (vm.CallResult, error) { if ctr.hashmapBuilder.InputBatchRowCount > 0 { jm = message.NewJoinMap(ctr.hashmapBuilder.MultiSels, ctr.hashmapBuilder.IntHashMap, ctr.hashmapBuilder.StrHashMap, ctr.hashmapBuilder.Batches.Buf, proc.Mp()) jm.SetPushedRuntimeFilterIn(ctr.runtimeFilterIn) + //jm.SetIgnoreRows(ctr.hashmapBuilder.IgnoreRows) if ap.NeedBatches { jm.SetRowCount(int64(ctr.hashmapBuilder.InputBatchRowCount)) } diff --git a/pkg/sql/colexec/hashbuild/types.go b/pkg/sql/colexec/hashbuild/types.go index 5009a2807659..552613d03120 100644 --- a/pkg/sql/colexec/hashbuild/types.go +++ b/pkg/sql/colexec/hashbuild/types.go @@ -16,9 +16,8 @@ package hashbuild import ( "github.com/matrixorigin/matrixone/pkg/common/reuse" - pbplan "github.com/matrixorigin/matrixone/pkg/pb/plan" + "github.com/matrixorigin/matrixone/pkg/pb/plan" "github.com/matrixorigin/matrixone/pkg/sql/colexec/hashmap_util" - "github.com/matrixorigin/matrixone/pkg/sql/plan" "github.com/matrixorigin/matrixone/pkg/vm" "github.com/matrixorigin/matrixone/pkg/vm/message" "github.com/matrixorigin/matrixone/pkg/vm/process" @@ -48,7 +47,13 @@ type HashBuild struct { Conditions []*plan.Expr JoinMapTag int32 JoinMapRefCnt int32 - RuntimeFilterSpec *pbplan.RuntimeFilterSpec + RuntimeFilterSpec *plan.RuntimeFilterSpec + + IsDedup bool + OnDuplicateAction plan.Node_OnDuplicateAction + DedupColName string + DedupColTypes []plan.Type + vm.OperatorBase } diff --git a/pkg/sql/colexec/hashmap_util/hashmap_util.go b/pkg/sql/colexec/hashmap_util/hashmap_util.go index 4d5928d3552b..bf94931abc8d 100644 --- a/pkg/sql/colexec/hashmap_util/hashmap_util.go +++ b/pkg/sql/colexec/hashmap_util/hashmap_util.go @@ -16,20 +16,19 @@ package hashmap_util import ( "runtime" + "strings" - "github.com/matrixorigin/matrixone/pkg/vm/message" - - "github.com/matrixorigin/matrixone/pkg/container/batch" - - "github.com/matrixorigin/matrixone/pkg/vm/process" - - "github.com/matrixorigin/matrixone/pkg/sql/colexec" - + "github.com/matrixorigin/matrixone/pkg/catalog" + "github.com/matrixorigin/matrixone/pkg/common/bitmap" "github.com/matrixorigin/matrixone/pkg/common/hashmap" + "github.com/matrixorigin/matrixone/pkg/common/moerr" + "github.com/matrixorigin/matrixone/pkg/container/batch" "github.com/matrixorigin/matrixone/pkg/container/types" "github.com/matrixorigin/matrixone/pkg/container/vector" - pbplan "github.com/matrixorigin/matrixone/pkg/pb/plan" - "github.com/matrixorigin/matrixone/pkg/sql/plan" + "github.com/matrixorigin/matrixone/pkg/pb/plan" + "github.com/matrixorigin/matrixone/pkg/sql/colexec" + "github.com/matrixorigin/matrixone/pkg/vm/message" + "github.com/matrixorigin/matrixone/pkg/vm/process" ) type HashmapBuilder struct { @@ -43,6 +42,13 @@ type HashmapBuilder struct { Batches colexec.Batches executor []colexec.ExpressionExecutor UniqueJoinKeys []*vector.Vector + + IsDedup bool + OnDuplicateAction plan.Node_OnDuplicateAction + DedupColName string + DedupColTypes []plan.Type + + IgnoreRows *bitmap.Bitmap } func (hb *HashmapBuilder) GetSize() int64 { @@ -72,7 +78,7 @@ func (hb *HashmapBuilder) Prepare(Conditions []*plan.Expr, proc *process.Process hb.keyWidth = 0 hb.InputBatchRowCount = 0 for i, expr := range Conditions { - if _, ok := Conditions[i].Expr.(*pbplan.Expr_Col); !ok { + if _, ok := Conditions[i].Expr.(*plan.Expr_Col); !ok { hb.needDupVec = true } typ := expr.Typ @@ -196,7 +202,7 @@ func (hb *HashmapBuilder) BuildHashmap(hashOnPK bool, needAllocateSels bool, nee itr = hb.StrHashMap.NewIterator() } - if hashOnPK { + if hashOnPK || hb.IsDedup { // if hash on primary key, prealloc hashmap size to the count of batch if hb.keyWidth <= 8 { err = hb.IntHashMap.PreAlloc(uint64(hb.InputBatchRowCount)) @@ -215,11 +221,17 @@ func (hb *HashmapBuilder) BuildHashmap(hashOnPK bool, needAllocateSels bool, nee } } + if hb.IsDedup && hb.InputBatchRowCount > 0 { + hb.IgnoreRows = &bitmap.Bitmap{} + hb.IgnoreRows.InitWithSize(int64(hb.InputBatchRowCount)) + } + var ( cardinality uint64 sels []int32 ) + vOld := uint64(0) for i := 0; i < hb.InputBatchRowCount; i += hashmap.UnitLimit { if i%(hashmap.UnitLimit*32) == 0 { runtime.Gosched() @@ -231,7 +243,7 @@ func (hb *HashmapBuilder) BuildHashmap(hashOnPK bool, needAllocateSels bool, nee // if not hash on primary key, estimate the hashmap size after 8192 rows //preAlloc to improve performance and reduce memory reAlloc - if !hashOnPK && hb.InputBatchRowCount > hashmap.HashMapSizeThreshHold && i == hashmap.HashMapSizeEstimate { + if !hashOnPK && !hb.IsDedup && hb.InputBatchRowCount > hashmap.HashMapSizeThreshHold && i == hashmap.HashMapSizeEstimate { if hb.keyWidth <= 8 { groupCount := hb.IntHashMap.GroupCount() rate := float64(groupCount) / float64(i) @@ -267,7 +279,40 @@ func (hb *HashmapBuilder) BuildHashmap(hashOnPK bool, needAllocateSels bool, nee } ai := int64(v) - 1 - if !hashOnPK && needAllocateSels { + if hb.IsDedup { + if v <= vOld { + switch hb.OnDuplicateAction { + case plan.Node_ERROR: + var rowStr string + if len(hb.DedupColTypes) == 1 { + if hb.DedupColName == catalog.IndexTableIndexColName { + if hb.vecs[vecIdx1][0].GetType().Oid == types.T_varchar { + t, _, schema, err := types.DecodeTuple(hb.vecs[vecIdx1][0].GetBytesAt(vecIdx2 + k)) + if err == nil && len(schema) > 1 { + rowStr = t.ErrString(make([]int32, len(schema))) + } + } + } + + if len(rowStr) == 0 { + rowStr = hb.vecs[vecIdx1][0].RowToString(vecIdx2 + k) + } + } else { + rowItems, err := types.StringifyTuple(hb.vecs[vecIdx1][0].GetBytesAt(vecIdx2+k), hb.DedupColTypes) + if err != nil { + return err + } + rowStr = "(" + strings.Join(rowItems, ",") + ")" + } + return moerr.NewDuplicateEntry(proc.Ctx, rowStr, hb.DedupColName) + case plan.Node_IGNORE: + hb.IgnoreRows.Add(uint64(i + k)) + case plan.Node_UPDATE: + } + } else { + vOld = v + } + } else if !hashOnPK && needAllocateSels { hb.MultiSels.InsertSel(int32(ai), int32(i+k)) } } @@ -280,7 +325,7 @@ func (hb *HashmapBuilder) BuildHashmap(hashOnPK bool, needAllocateSels bool, nee } } - if hashOnPK { + if hashOnPK || hb.IsDedup { for j, vec := range hb.vecs[vecIdx1] { err = hb.UniqueJoinKeys[j].UnionBatch(vec, int64(vecIdx2), n, nil, proc.Mp()) if err != nil { @@ -313,6 +358,13 @@ func (hb *HashmapBuilder) BuildHashmap(hashOnPK bool, needAllocateSels bool, nee } } + if hb.IsDedup { + err := hb.Batches.Shrink(hb.IgnoreRows, proc) + if err != nil { + return err + } + } + // if groupcount == inputrowcount, it means building hashmap on unique rows // we can free sels now if hb.keyWidth <= 8 { diff --git a/pkg/sql/colexec/insert/types.go b/pkg/sql/colexec/insert/types.go index 1afc50ab9cb3..34afff0ce2f5 100644 --- a/pkg/sql/colexec/insert/types.go +++ b/pkg/sql/colexec/insert/types.go @@ -137,14 +137,10 @@ func (insert *Insert) Free(proc *process.Process, pipelineFailed bool, err error } } -func (insert *Insert) AffectedRows() uint64 { +func (insert *Insert) GetAffectedRows() uint64 { return insert.ctr.affectedRows } -func (insert *Insert) GetAffectedRows() *uint64 { - return &insert.ctr.affectedRows -} - func (insert *Insert) initBufForS3() { attrs := []string{catalog.BlockMeta_TableIdx_Insert, catalog.BlockMeta_BlockInfo, catalog.ObjectMeta_ObjectStats} insert.ctr.buf = batch.NewWithSize(len(attrs)) diff --git a/pkg/sql/colexec/join/join.go b/pkg/sql/colexec/join/join.go index 8abac4b16217..702fffc10d61 100644 --- a/pkg/sql/colexec/join/join.go +++ b/pkg/sql/colexec/join/join.go @@ -118,16 +118,16 @@ func (innerJoin *InnerJoin) Call(proc *process.Process) (vm.CallResult, error) { continue } ctr.inbat = bat - ctr.lastrow = 0 + ctr.lastRow = 0 } - startrow := innerJoin.ctr.lastrow + startrow := innerJoin.ctr.lastRow if err := ctr.probe(innerJoin, proc, &probeResult); err != nil { return result, err } - if innerJoin.ctr.lastrow == 0 { + if innerJoin.ctr.lastRow == 0 { innerJoin.ctr.inbat = nil - } else if innerJoin.ctr.lastrow == startrow { + } else if innerJoin.ctr.lastRow == startrow { return result, moerr.NewInternalErrorNoCtx("inner join hanging") } @@ -201,11 +201,11 @@ func (ctr *container) probe(ap *InnerJoin, proc *process.Process, result *vm.Cal } itr := ctr.itr rowCount := 0 - for i := ap.ctr.lastrow; i < count; i += hashmap.UnitLimit { + for i := ap.ctr.lastRow; i < count; i += hashmap.UnitLimit { if rowCount >= colexec.DefaultBatchSize { ctr.rbat.AddRowCount(rowCount) result.Batch = ctr.rbat - ap.ctr.lastrow = i + ap.ctr.lastRow = i return nil } n := count - i @@ -273,7 +273,7 @@ func (ctr *container) probe(ap *InnerJoin, proc *process.Process, result *vm.Cal ctr.rbat.AddRowCount(rowCount) result.Batch = ctr.rbat - ap.ctr.lastrow = 0 + ap.ctr.lastRow = 0 return nil } diff --git a/pkg/sql/colexec/join/types.go b/pkg/sql/colexec/join/types.go index d5988d2dc848..d620cd3979f0 100644 --- a/pkg/sql/colexec/join/types.go +++ b/pkg/sql/colexec/join/types.go @@ -40,7 +40,7 @@ type container struct { itr hashmap.Iterator batchRowCount int64 - lastrow int + lastRow int inbat *batch.Batch rbat *batch.Batch @@ -126,7 +126,7 @@ func (innerJoin *InnerJoin) Reset(proc *process.Process, pipelineFailed bool, er ctr.resetExprExecutor() ctr.cleanHashMap() ctr.inbat = nil - ctr.lastrow = 0 + ctr.lastRow = 0 ctr.state = Build ctr.batchRowCount = 0 diff --git a/pkg/sql/colexec/join_util.go b/pkg/sql/colexec/join_util.go index d8d21ed637a1..f587e6c823c3 100644 --- a/pkg/sql/colexec/join_util.go +++ b/pkg/sql/colexec/join_util.go @@ -15,9 +15,11 @@ package colexec import ( + "github.com/matrixorigin/matrixone/pkg/common/bitmap" "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/common/mpool" "github.com/matrixorigin/matrixone/pkg/container/batch" + "github.com/matrixorigin/matrixone/pkg/container/vector" "github.com/matrixorigin/matrixone/pkg/vm/process" ) @@ -25,6 +27,14 @@ type Batches struct { Buf []*batch.Batch } +func (bs *Batches) RowCount() int { + var count int + for _, b := range bs.Buf { + count += b.RowCount() + } + return count +} + func (bs *Batches) Clean(mp *mpool.MPool) { for i := range bs.Buf { bs.Buf[i].Clean(mp) @@ -101,6 +111,50 @@ func (bs *Batches) CopyIntoBatches(src *batch.Batch, proc *process.Process) (err return nil } +func (bs *Batches) Shrink(ignoreRow *bitmap.Bitmap, proc *process.Process) error { + if ignoreRow.Count() == 0 { + return nil + } + + ignoreRow.Negate() + count := int64(ignoreRow.Count()) + sels := make([]int32, 0, count) + itr := ignoreRow.Iterator() + for itr.HasNext() { + r := itr.Next() + sels = append(sels, int32(r)) + } + + n := (len(sels)-1)/DefaultBatchSize + 1 + newBuf := make([]*batch.Batch, n) + for i := range newBuf { + newBuf[i] = batch.NewWithSize(len(bs.Buf[i].Vecs)) + for j, vec := range bs.Buf[0].Vecs { + newBuf[i].Vecs[j] = vector.NewVec(*vec.GetType()) + } + var newsels []int32 + if (i+1)*DefaultBatchSize <= len(sels) { + newsels = sels[i*DefaultBatchSize : (i+1)*DefaultBatchSize] + } else { + newsels = sels[i*DefaultBatchSize:] + } + for _, sel := range newsels { + idx1, idx2 := sel/DefaultBatchSize, sel%DefaultBatchSize + for j, vec := range bs.Buf[idx1].Vecs { + if err := newBuf[i].Vecs[j].UnionOne(vec, int64(idx2), proc.Mp()); err != nil { + return err + } + } + } + newBuf[i].SetRowCount(len(newsels)) + } + + bs.Clean(proc.Mp()) + bs.Buf = newBuf + + return nil +} + func appendToFixedSizeFromOffset(dst *batch.Batch, src *batch.Batch, offset int, proc *process.Process) (int, error) { var err error if dst == nil { diff --git a/pkg/sql/colexec/join_util_test.go b/pkg/sql/colexec/join_util_test.go index 9e2c28e245d4..032749b2104c 100644 --- a/pkg/sql/colexec/join_util_test.go +++ b/pkg/sql/colexec/join_util_test.go @@ -19,6 +19,7 @@ import ( "github.com/stretchr/testify/require" + "github.com/matrixorigin/matrixone/pkg/common/bitmap" "github.com/matrixorigin/matrixone/pkg/common/mpool" "github.com/matrixorigin/matrixone/pkg/container/types" "github.com/matrixorigin/matrixone/pkg/testutil" @@ -59,6 +60,14 @@ func TestBatches(t *testing.T) { require.Equal(t, 8192, batches.Buf[0].RowCount()) require.Equal(t, 8192, batches.Buf[1].RowCount()) require.Equal(t, 20, batches.Buf[2].RowCount()) + + rowCnt := batches.RowCount() + bm := &bitmap.Bitmap{} + bm.InitWithSize(int64(rowCnt)) + bm.AddRange(1000, 11000) + batches.Shrink(bm, proc) + require.Equal(t, rowCnt-10000, batches.RowCount()) + batches.Clean(proc.Mp()) require.Equal(t, int64(0), proc.Mp().CurrNB()) } diff --git a/pkg/sql/colexec/left/join.go b/pkg/sql/colexec/left/join.go index 5ab8e71746f8..aba2cef08c75 100644 --- a/pkg/sql/colexec/left/join.go +++ b/pkg/sql/colexec/left/join.go @@ -105,7 +105,7 @@ func (leftJoin *LeftJoin) Call(proc *process.Process) (vm.CallResult, error) { continue } ctr.inbat = bat - ctr.lastrow = 0 + ctr.lastRow = 0 } if ctr.rbat == nil { @@ -128,7 +128,7 @@ func (leftJoin *LeftJoin) Call(proc *process.Process) (vm.CallResult, error) { } } - startrow := leftJoin.ctr.lastrow + startRow := leftJoin.ctr.lastRow if ctr.mp == nil { err = ctr.emptyProbe(leftJoin, proc, &probeResult) } else { @@ -137,9 +137,9 @@ func (leftJoin *LeftJoin) Call(proc *process.Process) (vm.CallResult, error) { if err != nil { return result, err } - if leftJoin.ctr.lastrow == 0 { + if leftJoin.ctr.lastRow == 0 { leftJoin.ctr.inbat = nil - } else if leftJoin.ctr.lastrow == startrow { + } else if leftJoin.ctr.lastRow == startRow { return result, moerr.NewInternalErrorNoCtx("left join hanging") } @@ -187,7 +187,7 @@ func (ctr *container) emptyProbe(ap *LeftJoin, proc *process.Process, result *vm } ctr.rbat.AddRowCount(ap.ctr.inbat.RowCount()) result.Batch = ctr.rbat - ap.ctr.lastrow = 0 + ap.ctr.lastRow = 0 return nil } @@ -210,10 +210,10 @@ func (ctr *container) probe(ap *LeftJoin, proc *process.Process, result *vm.Call ctr.itr = ctr.mp.NewIterator() } itr := ctr.itr - for i := ap.ctr.lastrow; i < count; i += hashmap.UnitLimit { + for i := ap.ctr.lastRow; i < count; i += hashmap.UnitLimit { if ctr.rbat.RowCount() >= colexec.DefaultBatchSize { result.Batch = ctr.rbat - ap.ctr.lastrow = i + ap.ctr.lastRow = i return nil } n := count - i @@ -371,7 +371,7 @@ func (ctr *container) probe(ap *LeftJoin, proc *process.Process, result *vm.Call ctr.rbat.SetRowCount(ctr.rbat.RowCount() + rowCount) } result.Batch = ctr.rbat - ap.ctr.lastrow = 0 + ap.ctr.lastRow = 0 return nil } diff --git a/pkg/sql/colexec/left/types.go b/pkg/sql/colexec/left/types.go index 3a2e0b88d3a9..76e66f221f31 100644 --- a/pkg/sql/colexec/left/types.go +++ b/pkg/sql/colexec/left/types.go @@ -39,7 +39,7 @@ type container struct { state int itr hashmap.Iterator batchRowCount int64 - lastrow int + lastRow int inbat *batch.Batch rbat *batch.Batch @@ -114,7 +114,7 @@ func (leftJoin *LeftJoin) Reset(proc *process.Process, pipelineFailed bool, err ctr.resetExprExecutor() ctr.cleanHashMap() ctr.inbat = nil - ctr.lastrow = 0 + ctr.lastRow = 0 ctr.state = Build ctr.batchRowCount = 0 diff --git a/pkg/sql/colexec/mergeblock/types.go b/pkg/sql/colexec/mergeblock/types.go index 21ff5be4d712..9f32cd546a37 100644 --- a/pkg/sql/colexec/mergeblock/types.go +++ b/pkg/sql/colexec/mergeblock/types.go @@ -290,6 +290,6 @@ func (mergeBlock *MergeBlock) cleanMp(proc *process.Process) { mergeBlock.container.mp2 = nil } -func (mergeBlock *MergeBlock) AffectedRows() uint64 { +func (mergeBlock *MergeBlock) GetAffectedRows() uint64 { return mergeBlock.container.affectedRows } diff --git a/pkg/sql/colexec/mergedelete/mergedelete_test.go b/pkg/sql/colexec/mergedelete/mergedelete_test.go index 58700ef30684..aa39fd3b439b 100644 --- a/pkg/sql/colexec/mergedelete/mergedelete_test.go +++ b/pkg/sql/colexec/mergedelete/mergedelete_test.go @@ -17,9 +17,10 @@ package mergedelete import ( "bytes" "context" - "github.com/matrixorigin/matrixone/pkg/vm/process" "testing" + "github.com/matrixorigin/matrixone/pkg/vm/process" + "github.com/matrixorigin/matrixone/pkg/catalog" "github.com/matrixorigin/matrixone/pkg/container/batch" "github.com/matrixorigin/matrixone/pkg/container/types" @@ -176,20 +177,20 @@ func TestMergeDelete(t *testing.T) { resetChildren(&argument1, batch1) _, err = argument1.Call(proc) require.NoError(t, err) - require.Equal(t, uint64(15), argument1.AffectedRows()) + require.Equal(t, uint64(15), argument1.GetAffectedRows()) argument1.Reset(proc, false, err) resetChildren(&argument1, batch2) _, err = argument1.Call(proc) require.NoError(t, err) - require.Equal(t, uint64(60), argument1.AffectedRows()) + require.Equal(t, uint64(60), argument1.GetAffectedRows()) argument1.ctr.affectedRows = 0 argument1.Reset(proc, false, err) resetChildren(&argument1, nil) _, err = argument1.Call(proc) require.NoError(t, err) - require.Equal(t, uint64(0), argument1.AffectedRows()) + require.Equal(t, uint64(0), argument1.GetAffectedRows()) var partitionSources []engine.Relation partitionSources = append(partitionSources, &mockRelation{}) @@ -215,7 +216,7 @@ func TestMergeDelete(t *testing.T) { argument2.OpAnalyzer = process.NewAnalyzer(0, false, false, "mergedelete") _, err = argument2.Call(proc) require.NoError(t, err) - require.Equal(t, uint64(45), argument2.AffectedRows()) + require.Equal(t, uint64(45), argument2.GetAffectedRows()) // free resource argument1.Free(proc, false, nil) diff --git a/pkg/sql/colexec/mergedelete/types.go b/pkg/sql/colexec/mergedelete/types.go index c44298e2ccf0..a6a33a6aa210 100644 --- a/pkg/sql/colexec/mergedelete/types.go +++ b/pkg/sql/colexec/mergedelete/types.go @@ -108,6 +108,6 @@ func (mergeDelete *MergeDelete) Free(proc *process.Process, pipelineFailed bool, } } -func (mergeDelete *MergeDelete) AffectedRows() uint64 { +func (mergeDelete *MergeDelete) GetAffectedRows() uint64 { return mergeDelete.ctr.affectedRows } diff --git a/pkg/sql/colexec/mock_util.go b/pkg/sql/colexec/mock_util.go index 61a58e3d74b3..d2f791d4a44e 100644 --- a/pkg/sql/colexec/mock_util.go +++ b/pkg/sql/colexec/mock_util.go @@ -44,10 +44,8 @@ func NewMockOperator() *MockOperator { return &MockOperator{} } -func (op *MockOperator) ResetBatchs(proc *process.Process, batchs []*batch.Batch) *MockOperator { - op.Free(proc, false, nil) - op.WithBatchs(batchs) - return op +func (op *MockOperator) ResetBatchs() { + op.batchs = nil } func (op *MockOperator) WithBatchs(batchs []*batch.Batch) *MockOperator { @@ -68,6 +66,7 @@ func (op *MockOperator) Reset(proc *process.Process, pipelineFailed bool, err er op.batchs[i].CleanOnlyData() } } + op.current = 0 } func (op *MockOperator) Free(proc *process.Process, pipelineFailed bool, err error) { @@ -98,9 +97,6 @@ func (op *MockOperator) Call(proc *process.Process) (vm.CallResult, error) { result.Status = vm.ExecStop return result, nil } - if op.current > 0 { - op.batchs[op.current-1].CleanOnlyData() - } result.Batch = op.batchs[op.current] op.current = op.current + 1 return result, nil diff --git a/pkg/sql/colexec/multi_update/delete.go b/pkg/sql/colexec/multi_update/delete.go index f74023a02fca..c2c58a56969a 100644 --- a/pkg/sql/colexec/multi_update/delete.go +++ b/pkg/sql/colexec/multi_update/delete.go @@ -18,6 +18,7 @@ import ( "github.com/matrixorigin/matrixone/pkg/catalog" "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/container/batch" + "github.com/matrixorigin/matrixone/pkg/container/types" "github.com/matrixorigin/matrixone/pkg/container/vector" "github.com/matrixorigin/matrixone/pkg/vm/process" ) @@ -26,25 +27,43 @@ func (update *MultiUpdate) delete_table( proc *process.Process, updateCtx *MultiUpdateCtx, inputBatch *batch.Batch, - deleteBatch *batch.Batch) (err error) { - deleteBatch.CleanOnlyData() + idx int, +) (err error) { - rowIdIdx := updateCtx.deleteCols[0] + // init buf + ctr := &update.ctr + if ctr.deleteBuf[idx] == nil { + mainPkIdx := updateCtx.DeleteCols[1] + ctr.deleteBuf[idx] = newDeleteBatch(inputBatch, mainPkIdx) + } + deleteBatch := ctr.deleteBuf[idx] + rowIdIdx := updateCtx.DeleteCols[0] + rowIdVec := inputBatch.Vecs[rowIdIdx] + if rowIdVec.IsConstNull() { + return + } + rowCount := inputBatch.RowCount() + + if len(updateCtx.PartitionTableIDs) > 0 { + partTableNulls := inputBatch.Vecs[updateCtx.OldPartitionIdx].GetNulls() + partTableIDs := vector.MustFixedColWithTypeCheck[int32](inputBatch.Vecs[updateCtx.OldPartitionIdx]) + + for partIdx := range len(updateCtx.PartitionTableIDs) { + rowIdNulls := rowIdVec.GetNulls() + if rowIdNulls.Count() == rowCount { + continue + } - if len(updateCtx.partitionTableIDs) > 0 { - for partIdx := range len(updateCtx.partitionTableIDs) { deleteBatch.CleanOnlyData() expected := int32(partIdx) - partTableIDs := vector.MustFixedColWithTypeCheck[int32](inputBatch.Vecs[updateCtx.partitionIdx]) - rowIdNulls := inputBatch.Vecs[rowIdIdx].GetNulls() for i, partition := range partTableIDs { - if !inputBatch.Vecs[updateCtx.partitionIdx].GetNulls().Contains(uint64(i)) { + if !partTableNulls.Contains(uint64(i)) { if partition == -1 { return moerr.NewInvalidInput(proc.Ctx, "Table has no partition for value from column_list") } else if partition == expected { if !rowIdNulls.Contains(uint64(i)) { - for deleteIdx, inputIdx := range updateCtx.deleteCols { + for deleteIdx, inputIdx := range updateCtx.DeleteCols { err = deleteBatch.Vecs[deleteIdx].UnionOne(inputBatch.Vecs[inputIdx], int64(i), proc.Mp()) if err != nil { return err @@ -55,20 +74,30 @@ func (update *MultiUpdate) delete_table( } } - err = updateCtx.partitionSources[partIdx].Delete(proc.Ctx, deleteBatch, catalog.Row_ID) - if err != nil { - return err + rowCount := deleteBatch.Vecs[0].Length() + if rowCount > 0 { + deleteBatch.SetRowCount(rowCount) + tableType := update.ctr.updateCtxInfos[updateCtx.TableDef.Name].tableType + update.addDeleteAffectRows(tableType, uint64(rowCount)) + source := update.ctr.updateCtxInfos[updateCtx.TableDef.Name].Sources[partIdx] + err = source.Delete(proc.Ctx, deleteBatch, catalog.Row_ID) + if err != nil { + return err + } } } } else { deleteBatch.CleanOnlyData() + if rowIdVec.HasNull() { + // multi delete or delete unique table with null value + rowIdNulls := rowIdVec.GetNulls() + if rowIdNulls.Count() == rowCount { + return + } - if inputBatch.Vecs[rowIdIdx].HasNull() { - // multi delete - rowIdNulls := inputBatch.Vecs[rowIdIdx].GetNulls() - for i := 0; i < inputBatch.RowCount(); i++ { + for i := 0; i < rowCount; i++ { if !rowIdNulls.Contains(uint64(i)) { - for deleteIdx, inputIdx := range updateCtx.deleteCols { + for deleteIdx, inputIdx := range updateCtx.DeleteCols { err = deleteBatch.Vecs[deleteIdx].UnionOne(inputBatch.Vecs[inputIdx], int64(i), proc.Mp()) if err != nil { return err @@ -78,15 +107,29 @@ func (update *MultiUpdate) delete_table( } } else { - for deleteIdx, inputIdx := range updateCtx.deleteCols { + for deleteIdx, inputIdx := range updateCtx.DeleteCols { err = deleteBatch.Vecs[deleteIdx].UnionBatch(inputBatch.Vecs[inputIdx], 0, inputBatch.Vecs[inputIdx].Length(), nil, proc.GetMPool()) if err != nil { return err } } } - err = updateCtx.source.Delete(proc.Ctx, deleteBatch, catalog.Row_ID) + rowCount := deleteBatch.Vecs[0].Length() + if rowCount > 0 { + deleteBatch.SetRowCount(rowCount) + tableType := update.ctr.updateCtxInfos[updateCtx.TableDef.Name].tableType + update.addDeleteAffectRows(tableType, uint64(rowCount)) + source := update.ctr.updateCtxInfos[updateCtx.TableDef.Name].Sources[0] + err = source.Delete(proc.Ctx, deleteBatch, catalog.Row_ID) + } } return } + +func newDeleteBatch(inputBatch *batch.Batch, mainPkIdx int) *batch.Batch { + buf := batch.New([]string{catalog.Row_ID, "pk"}) + buf.SetVector(0, vector.NewVec(types.T_Rowid.ToType())) + buf.SetVector(1, vector.NewVec(*inputBatch.Vecs[mainPkIdx].GetType())) + return buf +} diff --git a/pkg/sql/colexec/multi_update/delete_test.go b/pkg/sql/colexec/multi_update/delete_test.go index b74f33147350..27257aa2f4a2 100644 --- a/pkg/sql/colexec/multi_update/delete_test.go +++ b/pkg/sql/colexec/multi_update/delete_test.go @@ -94,8 +94,8 @@ func buildDeleteTestCase(t *testing.T, hasUniqueKey bool, hasSecondaryKey bool, batchs, affectRows := prepareTestDeleteBatchs(proc.GetMPool(), 2, hasUniqueKey, hasSecondaryKey, isPartition) multiUpdateCtxs := prepareTestDeleteMultiUpdateCtx(hasUniqueKey, hasSecondaryKey, isPartition) - toWriteS3 := false - retCase := buildTestCase(multiUpdateCtxs, eng, batchs, affectRows, toWriteS3) + action := UpdateWriteTable + retCase := buildTestCase(multiUpdateCtxs, eng, batchs, affectRows, action) return proc, retCase } @@ -103,10 +103,10 @@ func buildDeleteS3TestCase(t *testing.T, hasUniqueKey bool, hasSecondaryKey bool _, ctrl, proc := prepareTestCtx(t, true) eng := prepareTestEng(ctrl) - batchs, affectRows := prepareTestDeleteBatchs(proc.GetMPool(), 200, hasUniqueKey, hasSecondaryKey, isPartition) + batchs, _ := prepareTestDeleteBatchs(proc.GetMPool(), 200, hasUniqueKey, hasSecondaryKey, isPartition) multiUpdateCtxs := prepareTestDeleteMultiUpdateCtx(hasUniqueKey, hasSecondaryKey, isPartition) - toWriteS3 := true - retCase := buildTestCase(multiUpdateCtxs, eng, batchs, affectRows, toWriteS3) + action := UpdateWriteS3 + retCase := buildTestCase(multiUpdateCtxs, eng, batchs, 0, action) return proc, retCase } @@ -114,6 +114,9 @@ func prepareTestDeleteBatchs(mp *mpool.MPool, size int, hasUniqueKey bool, hasSe var bats = make([]*batch.Batch, size) affectRows := 0 partitionCount := 3 + mainObjectID := types.NewObjectid() + uniqueObjectID := types.NewObjectid() + secondaryObjectID := types.NewObjectid() for i := 0; i < size; i++ { rowCount := colexec.DefaultBatchSize if i == size-1 { @@ -122,7 +125,7 @@ func prepareTestDeleteBatchs(mp *mpool.MPool, size int, hasUniqueKey bool, hasSe rows := makeTestPkArray(int64(affectRows), rowCount) columnA := testutil.MakeInt64Vector(rows, nil) - columnRowID := testutil.NewRowidVector(rowCount, types.T_Rowid.ToType(), mp, false, nil) + columnRowID := makeTestRowIDVector(mp, mainObjectID, uint16(i), rowCount) attrs := []string{"main_rowid", "a"} bat := &batch.Batch{ @@ -131,7 +134,7 @@ func prepareTestDeleteBatchs(mp *mpool.MPool, size int, hasUniqueKey bool, hasSe } if hasUniqueKey { - columnRowID := testutil.NewRowidVector(rowCount, types.T_Rowid.ToType(), mp, false, nil) + columnRowID := makeTestRowIDVector(mp, uniqueObjectID, uint16(i), rowCount) columnPk := testutil.NewStringVector(rowCount, types.T_varchar.ToType(), mp, false, nil) bat.Vecs = append(bat.Vecs, columnRowID) bat.Vecs = append(bat.Vecs, columnPk) @@ -139,7 +142,7 @@ func prepareTestDeleteBatchs(mp *mpool.MPool, size int, hasUniqueKey bool, hasSe } if hasSecondaryKey { - columnRowID := testutil.NewRowidVector(rowCount, types.T_Rowid.ToType(), mp, false, nil) + columnRowID := makeTestRowIDVector(mp, secondaryObjectID, uint16(i), rowCount) columnPk := testutil.NewStringVector(rowCount, types.T_varchar.ToType(), mp, false, nil) bat.Vecs = append(bat.Vecs, columnRowID) bat.Vecs = append(bat.Vecs, columnPk) @@ -166,10 +169,9 @@ func prepareTestDeleteMultiUpdateCtx(hasUniqueKey bool, hasSecondaryKey bool, is objRef, tableDef := getTestMainTable(isPartition) updateCtx := &MultiUpdateCtx{ - ref: objRef, - tableDef: tableDef, - tableType: updateMainTable, - deleteCols: []int{0, 1}, //row_id & pk + ObjRef: objRef, + TableDef: tableDef, + DeleteCols: []int{0, 1}, //row_id & pk } updateCtxs := []*MultiUpdateCtx{updateCtx} colCount := 2 @@ -190,10 +192,9 @@ func prepareTestDeleteMultiUpdateCtx(hasUniqueKey bool, hasSecondaryKey bool, is uniqueObjRef, uniqueTableDef := getTestUniqueIndexTable(uniqueTblName, isPartition) updateCtxs = append(updateCtxs, &MultiUpdateCtx{ - ref: uniqueObjRef, - tableDef: uniqueTableDef, - tableType: updateUniqueIndexTable, - deleteCols: []int{2, 3}, //row_id & pk + ObjRef: uniqueObjRef, + TableDef: uniqueTableDef, + DeleteCols: []int{2, 3}, //row_id & pk }) colCount += 2 } @@ -219,23 +220,20 @@ func prepareTestDeleteMultiUpdateCtx(hasUniqueKey bool, hasSecondaryKey bool, is } colCount += 2 updateCtxs = append(updateCtxs, &MultiUpdateCtx{ - ref: secondaryIdxObjRef, - tableDef: secondaryIdxTableDef, - tableType: updateSecondaryIndexTable, - deleteCols: secondaryPkPos, + ObjRef: secondaryIdxObjRef, + TableDef: secondaryIdxTableDef, + DeleteCols: secondaryPkPos, }) } if isPartition { - for i, updateCtx := range updateCtxs { - partTblIDs := make([]int32, len(tableDef.Partition.PartitionTableNames)) - for j := range tableDef.Partition.PartitionTableNames { - partTblIDs[j] = int32(i*1000 + j) - } - updateCtx.partitionIdx = colCount - updateCtx.partitionTableIDs = partTblIDs - updateCtx.partitionTableNames = tableDef.Partition.PartitionTableNames + partTblIDs := make([]uint64, len(tableDef.Partition.PartitionTableNames)) + for j := range tableDef.Partition.PartitionTableNames { + partTblIDs[j] = uint64(1000 + j) } + updateCtxs[0].OldPartitionIdx = colCount + updateCtxs[0].PartitionTableIDs = partTblIDs + updateCtxs[0].PartitionTableNames = tableDef.Partition.PartitionTableNames } return updateCtxs diff --git a/pkg/sql/colexec/multi_update/insert.go b/pkg/sql/colexec/multi_update/insert.go index 3b999c1fdd51..40c6128be806 100644 --- a/pkg/sql/colexec/multi_update/insert.go +++ b/pkg/sql/colexec/multi_update/insert.go @@ -25,6 +25,8 @@ import ( "github.com/matrixorigin/matrixone/pkg/vm/process" ) +//@todo add test case: only insert hidden table + func (update *MultiUpdate) insert_main_table( proc *process.Process, tableIndex int, @@ -34,22 +36,22 @@ func (update *MultiUpdate) insert_main_table( // init buffer if ctr.insertBuf[tableIndex] == nil { - bat := batch.NewWithSize(len(updateCtx.insertCols)) - attrs := make([]string, 0, len(updateCtx.tableDef.Cols)-1) - for _, col := range updateCtx.tableDef.Cols { + bat := batch.NewWithSize(len(updateCtx.InsertCols)) + attrs := make([]string, 0, len(updateCtx.TableDef.Cols)-1) + for _, col := range updateCtx.TableDef.Cols { if col.Name == catalog.Row_ID { continue } bat.Vecs[len(attrs)] = vector.NewVec(plan.MakeTypeByPlan2Type(col.Typ)) - attrs = append(attrs, col.Name) + attrs = append(attrs, col.GetOriginCaseName()) } bat.SetAttributes(attrs) ctr.insertBuf[tableIndex] = bat } // preinsert: check not null column - for insertIdx, inputIdx := range updateCtx.insertCols { - col := updateCtx.tableDef.Cols[insertIdx] + for insertIdx, inputIdx := range updateCtx.InsertCols { + col := updateCtx.TableDef.Cols[insertIdx] if col.Default != nil && !col.Default.NullAbility { if inputBatch.Vecs[inputIdx].HasNull() { return moerr.NewConstraintViolation(proc.Ctx, fmt.Sprintf("Column '%s' cannot be null", col.Name)) @@ -73,12 +75,12 @@ func (update *MultiUpdate) insert_uniuqe_index_table( if ctr.insertBuf[tableIndex] == nil { ctr.insertBuf[tableIndex] = batch.NewWithSize(2) ctr.insertBuf[tableIndex].Attrs = []string{catalog.IndexTableIndexColName, catalog.IndexTablePrimaryColName} - for insertIdx, inputIdx := range updateCtx.insertCols { + for insertIdx, inputIdx := range updateCtx.InsertCols { ctr.insertBuf[tableIndex].Vecs[insertIdx] = vector.NewVec(*inputBatch.Vecs[inputIdx].GetType()) } } - idxPkPos := updateCtx.insertCols[0] + idxPkPos := updateCtx.InsertCols[0] if inputBatch.Vecs[idxPkPos].HasNull() { err = update.check_null_and_insert_table(proc, updateCtx, inputBatch, ctr.insertBuf[tableIndex]) } else { @@ -96,14 +98,19 @@ func (update *MultiUpdate) insert_secondary_index_table( // init buf if ctr.insertBuf[tableIndex] == nil { - ctr.insertBuf[tableIndex] = batch.NewWithSize(2) - ctr.insertBuf[tableIndex].Attrs = []string{catalog.IndexTableIndexColName, catalog.IndexTablePrimaryColName} - for insertIdx, inputIdx := range updateCtx.insertCols { + attrs := make([]string, 0, len(update.MultiUpdateCtx[tableIndex].TableDef.Cols)) + for _, col := range update.MultiUpdateCtx[tableIndex].TableDef.Cols { + if col.Name != catalog.Row_ID { + attrs = append(attrs, col.Name) + } + } + ctr.insertBuf[tableIndex] = batch.New(attrs) + for insertIdx, inputIdx := range updateCtx.InsertCols { ctr.insertBuf[tableIndex].Vecs[insertIdx] = vector.NewVec(*inputBatch.Vecs[inputIdx].GetType()) } } - idxPkPos := updateCtx.insertCols[0] + idxPkPos := updateCtx.InsertCols[0] if inputBatch.Vecs[idxPkPos].HasNull() { err = update.check_null_and_insert_table(proc, updateCtx, inputBatch, ctr.insertBuf[tableIndex]) } else { @@ -117,19 +124,20 @@ func (update *MultiUpdate) insert_table( updateCtx *MultiUpdateCtx, inputBatch *batch.Batch, insertBatch *batch.Batch) (err error) { - if len(updateCtx.partitionTableIDs) > 0 { - for partIdx := range len(updateCtx.partitionTableIDs) { + if len(updateCtx.PartitionTableIDs) > 0 { + partTableNulls := inputBatch.Vecs[updateCtx.NewPartitionIdx].GetNulls() + partTableIDs := vector.MustFixedColWithTypeCheck[int32](inputBatch.Vecs[updateCtx.NewPartitionIdx]) + + for partIdx := range len(updateCtx.PartitionTableIDs) { insertBatch.CleanOnlyData() expected := int32(partIdx) - partTableIDs := vector.MustFixedColWithTypeCheck[int32](inputBatch.Vecs[updateCtx.partitionIdx]) - partTableNulls := inputBatch.Vecs[updateCtx.partitionIdx].GetNulls() for i, partition := range partTableIDs { if !partTableNulls.Contains(uint64(i)) { if partition == -1 { return moerr.NewInvalidInput(proc.Ctx, "Table has no partition for value from column_list") } else if partition == expected { - for insertIdx, inputIdx := range updateCtx.insertCols { + for insertIdx, inputIdx := range updateCtx.InsertCols { err = insertBatch.Vecs[insertIdx].UnionOne(inputBatch.Vecs[inputIdx], int64(i), proc.Mp()) if err != nil { return err @@ -139,20 +147,34 @@ func (update *MultiUpdate) insert_table( } } - err = updateCtx.partitionSources[partIdx].Write(proc.Ctx, insertBatch) - if err != nil { - return err + rowCount := insertBatch.Vecs[0].Length() + if rowCount > 0 { + insertBatch.SetRowCount(rowCount) + tableType := update.ctr.updateCtxInfos[updateCtx.TableDef.Name].tableType + update.addInsertAffectRows(tableType, uint64(rowCount)) + source := update.ctr.updateCtxInfos[updateCtx.TableDef.Name].Sources[partIdx] + err = source.Write(proc.Ctx, insertBatch) + if err != nil { + return err + } } } } else { insertBatch.CleanOnlyData() - for insertIdx, inputIdx := range updateCtx.insertCols { + for insertIdx, inputIdx := range updateCtx.InsertCols { err = insertBatch.Vecs[insertIdx].UnionBatch(inputBatch.Vecs[inputIdx], 0, inputBatch.Vecs[inputIdx].Length(), nil, proc.GetMPool()) if err != nil { return err } } - err = updateCtx.source.Write(proc.Ctx, insertBatch) + rowCount := insertBatch.Vecs[0].Length() + if rowCount > 0 { + insertBatch.SetRowCount(rowCount) + tableType := update.ctr.updateCtxInfos[updateCtx.TableDef.Name].tableType + update.addInsertAffectRows(tableType, uint64(rowCount)) + source := update.ctr.updateCtxInfos[updateCtx.TableDef.Name].Sources[0] + err = source.Write(proc.Ctx, insertBatch) + } } return } @@ -163,18 +185,19 @@ func (update *MultiUpdate) check_null_and_insert_table( inputBatch *batch.Batch, insertBatch *batch.Batch) (err error) { - idxPkPos := updateCtx.insertCols[0] - mainPkPos := updateCtx.insertCols[1] + idxPkPos := updateCtx.InsertCols[0] + mainPkPos := updateCtx.InsertCols[1] idxPkVec := inputBatch.Vecs[idxPkPos] mainPkVec := inputBatch.Vecs[mainPkPos] - idxPkNulls := inputBatch.Vecs[updateCtx.insertCols[0]].GetNulls() + idxPkNulls := inputBatch.Vecs[updateCtx.InsertCols[0]].GetNulls() + + if len(updateCtx.PartitionTableIDs) > 0 { + partTableIDs := vector.MustFixedColWithTypeCheck[int32](inputBatch.Vecs[updateCtx.NewPartitionIdx]) + partTableNulls := inputBatch.Vecs[updateCtx.NewPartitionIdx].GetNulls() - if len(updateCtx.partitionTableIDs) > 0 { - for partIdx := range len(updateCtx.partitionTableIDs) { + for partIdx := range len(updateCtx.PartitionTableIDs) { insertBatch.CleanOnlyData() expected := int32(partIdx) - partTableIDs := vector.MustFixedColWithTypeCheck[int32](inputBatch.Vecs[updateCtx.partitionIdx]) - partTableNulls := inputBatch.Vecs[updateCtx.partitionIdx].GetNulls() for i, partition := range partTableIDs { if !partTableNulls.Contains(uint64(i)) { @@ -196,9 +219,16 @@ func (update *MultiUpdate) check_null_and_insert_table( } } - err = updateCtx.partitionSources[partIdx].Write(proc.Ctx, insertBatch) - if err != nil { - return err + newRowCount := insertBatch.Vecs[0].Length() + if newRowCount > 0 { + insertBatch.SetRowCount(newRowCount) + tableType := update.ctr.updateCtxInfos[updateCtx.TableDef.Name].tableType + update.addInsertAffectRows(tableType, uint64(newRowCount)) + source := update.ctr.updateCtxInfos[updateCtx.TableDef.Name].Sources[partIdx] + err = source.Write(proc.Ctx, insertBatch) + if err != nil { + return err + } } } } else { @@ -218,7 +248,14 @@ func (update *MultiUpdate) check_null_and_insert_table( } } - err = updateCtx.source.Write(proc.Ctx, insertBatch) + newRowCount := insertBatch.Vecs[0].Length() + if newRowCount > 0 { + insertBatch.SetRowCount(newRowCount) + tableType := update.ctr.updateCtxInfos[updateCtx.TableDef.Name].tableType + update.addInsertAffectRows(tableType, uint64(newRowCount)) + source := update.ctr.updateCtxInfos[updateCtx.TableDef.Name].Sources[0] + err = source.Write(proc.Ctx, insertBatch) + } } return } diff --git a/pkg/sql/colexec/multi_update/insert_test.go b/pkg/sql/colexec/multi_update/insert_test.go index 4ff23b1e044b..93053fba4c22 100644 --- a/pkg/sql/colexec/multi_update/insert_test.go +++ b/pkg/sql/colexec/multi_update/insert_test.go @@ -115,6 +115,23 @@ func TestInsertS3PartitionTable(t *testing.T) { runTestCases(t, proc, []*testCase{case1}) } +func TestFlushS3Info(t *testing.T) { + hasUniqueKey := false + hasSecondaryKey := false + isPartition := false + + _, ctrl, proc := prepareTestCtx(t, true) + eng := prepareTestEng(ctrl) + + batchs, rowCount := buildFlushS3InfoBatch(proc.GetMPool(), hasUniqueKey, hasSecondaryKey, isPartition) + + multiUpdateCtxs := prepareTestInsertMultiUpdateCtx(hasUniqueKey, hasSecondaryKey, isPartition) + action := UpdateFlushS3Info + retCase := buildTestCase(multiUpdateCtxs, eng, batchs, rowCount, action) + + runTestCases(t, proc, []*testCase{retCase}) +} + // ----- util function ---- func buildInsertTestCase(t *testing.T, hasUniqueKey bool, hasSecondaryKey bool, isPartition bool) (*process.Process, *testCase) { _, ctrl, proc := prepareTestCtx(t, false) @@ -122,8 +139,8 @@ func buildInsertTestCase(t *testing.T, hasUniqueKey bool, hasSecondaryKey bool, batchs, affectRows := prepareTestInsertBatchs(proc.GetMPool(), 2, hasUniqueKey, hasSecondaryKey, isPartition) multiUpdateCtxs := prepareTestInsertMultiUpdateCtx(hasUniqueKey, hasSecondaryKey, isPartition) - toWriteS3 := false - retCase := buildTestCase(multiUpdateCtxs, eng, batchs, affectRows, toWriteS3) + action := UpdateWriteTable + retCase := buildTestCase(multiUpdateCtxs, eng, batchs, affectRows, action) return proc, retCase } @@ -131,10 +148,10 @@ func buildInsertS3TestCase(t *testing.T, hasUniqueKey bool, hasSecondaryKey bool _, ctrl, proc := prepareTestCtx(t, true) eng := prepareTestEng(ctrl) - batchs, affectRows := prepareTestInsertBatchs(proc.GetMPool(), 500, hasUniqueKey, hasSecondaryKey, isPartition) + batchs, _ := prepareTestInsertBatchs(proc.GetMPool(), 220, hasUniqueKey, hasSecondaryKey, isPartition) multiUpdateCtxs := prepareTestInsertMultiUpdateCtx(hasUniqueKey, hasSecondaryKey, isPartition) - toWriteS3 := true - retCase := buildTestCase(multiUpdateCtxs, eng, batchs, affectRows, toWriteS3) + action := UpdateWriteS3 + retCase := buildTestCase(multiUpdateCtxs, eng, batchs, 0, action) return proc, retCase } @@ -190,10 +207,9 @@ func prepareTestInsertMultiUpdateCtx(hasUniqueKey bool, hasSecondaryKey bool, is objRef, tableDef := getTestMainTable(isPartition) updateCtx := &MultiUpdateCtx{ - ref: objRef, - tableDef: tableDef, - tableType: updateMainTable, - insertCols: []int{0, 1, 2, 3}, + ObjRef: objRef, + TableDef: tableDef, + InsertCols: []int{0, 1, 2, 3}, } colCount := 4 updateCtxs := []*MultiUpdateCtx{updateCtx} @@ -214,10 +230,9 @@ func prepareTestInsertMultiUpdateCtx(hasUniqueKey bool, hasSecondaryKey bool, is uniqueObjRef, uniqueTableDef := getTestUniqueIndexTable(uniqueTblName, isPartition) updateCtxs = append(updateCtxs, &MultiUpdateCtx{ - ref: uniqueObjRef, - tableDef: uniqueTableDef, - tableType: updateUniqueIndexTable, - insertCols: []int{4, 0}, + ObjRef: uniqueObjRef, + TableDef: uniqueTableDef, + InsertCols: []int{4, 0}, }) colCount += 1 } @@ -241,25 +256,49 @@ func prepareTestInsertMultiUpdateCtx(hasUniqueKey bool, hasSecondaryKey bool, is secondaryPkPos += 1 } updateCtxs = append(updateCtxs, &MultiUpdateCtx{ - ref: secondaryIdxObjRef, - tableDef: secondaryIdxTableDef, - tableType: updateSecondaryIndexTable, - insertCols: []int{secondaryPkPos, 0}, + ObjRef: secondaryIdxObjRef, + TableDef: secondaryIdxTableDef, + InsertCols: []int{secondaryPkPos, 0}, }) colCount += 1 } if isPartition { - for i, updateCtx := range updateCtxs { - partTblIDs := make([]int32, len(tableDef.Partition.PartitionTableNames)) - for j := range tableDef.Partition.PartitionTableNames { - partTblIDs[j] = int32(i*1000 + j) - } - updateCtx.partitionIdx = colCount - updateCtx.partitionTableIDs = partTblIDs - updateCtx.partitionTableNames = tableDef.Partition.PartitionTableNames + partTblIDs := make([]uint64, len(tableDef.Partition.PartitionTableNames)) + for j := range tableDef.Partition.PartitionTableNames { + partTblIDs[j] = uint64(1000 + j) } + updateCtxs[0].NewPartitionIdx = colCount + updateCtxs[0].PartitionTableIDs = partTblIDs + updateCtxs[0].PartitionTableNames = tableDef.Partition.PartitionTableNames } return updateCtxs } + +func buildFlushS3InfoBatch(mp *mpool.MPool, hasUniqueKey bool, hasSecondaryKey bool, isPartition bool) ([]*batch.Batch, uint64) { + insertBats, _ := prepareTestInsertBatchs(mp, 5, hasUniqueKey, hasSecondaryKey, isPartition) + retBat := batch.NewWithSize(6) + action := uint8(actionInsert) + + retBat.Vecs[0] = testutil.NewUInt8Vector(5, types.T_uint8.ToType(), mp, false, []uint8{action, action, action, action, action}) + retBat.Vecs[1] = testutil.NewUInt16Vector(5, types.T_uint16.ToType(), mp, false, []uint16{0, 0, 0, 0, 0}) //idx + retBat.Vecs[2] = testutil.NewUInt16Vector(5, types.T_uint16.ToType(), mp, false, []uint16{0, 0, 0, 0, 0}) //partIdx + retBat.Vecs[3] = vector.NewVec(types.T_uint64.ToType()) //rowCount + retBat.Vecs[4] = testutil.NewStringVector(5, types.T_varchar.ToType(), mp, false, []string{"", "", "", "", ""}) //name + retBat.Vecs[5] = vector.NewVec(types.T_text.ToType()) //batch bytes + + totalRowCount := 0 + for _, bat := range insertBats { + totalRowCount += bat.RowCount() + _ = vector.AppendFixed(retBat.Vecs[3], bat.RowCount(), false, mp) + + val, _ := bat.MarshalBinary() + _ = vector.AppendBytes(retBat.Vecs[5], val, false, mp) + + bat.Clean(mp) + } + + retBat.SetRowCount(retBat.Vecs[0].Length()) + return []*batch.Batch{retBat}, uint64(totalRowCount) +} diff --git a/pkg/sql/colexec/multi_update/multi_update.go b/pkg/sql/colexec/multi_update/multi_update.go index 51a59c102ef1..0597acb7fe54 100644 --- a/pkg/sql/colexec/multi_update/multi_update.go +++ b/pkg/sql/colexec/multi_update/multi_update.go @@ -16,11 +16,12 @@ package multi_update import ( "bytes" + "fmt" + "strings" "time" "github.com/matrixorigin/matrixone/pkg/catalog" "github.com/matrixorigin/matrixone/pkg/container/batch" - "github.com/matrixorigin/matrixone/pkg/container/types" "github.com/matrixorigin/matrixone/pkg/container/vector" "github.com/matrixorigin/matrixone/pkg/sql/colexec" v2 "github.com/matrixorigin/matrixone/pkg/util/metric/v2" @@ -43,16 +44,53 @@ func (update *MultiUpdate) Prepare(proc *process.Process) error { update.OpAnalyzer.Reset() } + if update.ctr.updateCtxInfos == nil { + update.ctr.updateCtxInfos = make(map[string]*updateCtxInfo) + for _, updateCtx := range update.MultiUpdateCtx { + info := new(updateCtxInfo) + for _, col := range updateCtx.TableDef.Cols { + if col.Name != catalog.Row_ID { + info.insertAttrs = append(info.insertAttrs, col.Name) + } + } + + tableType := UpdateMainTable + if strings.HasPrefix(updateCtx.TableDef.Name, catalog.UniqueIndexTableNamePrefix) { + tableType = UpdateUniqueIndexTable + } else if strings.HasPrefix(updateCtx.TableDef.Name, catalog.SecondaryIndexTableNamePrefix) { + tableType = UpdateSecondaryIndexTable + } + info.tableType = tableType + update.ctr.updateCtxInfos[updateCtx.TableDef.Name] = info + } + } + + for _, updateCtx := range update.MultiUpdateCtx { + info := update.ctr.updateCtxInfos[updateCtx.TableDef.Name] + info.Sources = nil + if update.Action != UpdateWriteS3 { + rel, partitionRels, err := colexec.GetRelAndPartitionRelsByObjRef(proc.Ctx, proc, update.Engine, updateCtx.ObjRef, updateCtx.PartitionTableNames) + if err != nil { + return err + } + if len(updateCtx.PartitionTableNames) > 0 { + info.Sources = append(info.Sources, partitionRels...) + } else { + info.Sources = append(info.Sources, rel) + } + } + } + if len(update.ctr.insertBuf) == 0 { update.ctr.insertBuf = make([]*batch.Batch, len(update.MultiUpdateCtx)) } if len(update.ctr.deleteBuf) == 0 { update.ctr.deleteBuf = make([]*batch.Batch, len(update.MultiUpdateCtx)) } + update.ctr.affectedRows = 0 - eng := update.Engine - - if update.ToWriteS3 { + switch update.Action { + case UpdateWriteS3: if update.ctr.s3Writer == nil { writer, err := newS3Writer(update) if err != nil { @@ -61,28 +99,32 @@ func (update *MultiUpdate) Prepare(proc *process.Process) error { update.ctr.s3Writer = writer } - writer := update.ctr.s3Writer - for _, updateCtx := range writer.updateCtxs { - ref := updateCtx.ref - partitionNames := updateCtx.partitionTableNames - rel, partitionRels, err := colexec.GetRelAndPartitionRelsByObjRef(proc.Ctx, proc, eng, ref, partitionNames) - if err != nil { - return err - } - updateCtx.source = rel - updateCtx.partitionSources = partitionRels + case UpdateFlushS3Info: + //resort updateCtxs + writer, err := newS3Writer(update) + if err != nil { + return err } - } else { - for _, updateCtx := range update.MultiUpdateCtx { - ref := updateCtx.ref - partitionNames := updateCtx.partitionTableNames - rel, partitionRels, err := colexec.GetRelAndPartitionRelsByObjRef(proc.Ctx, proc, eng, ref, partitionNames) - if err != nil { - return err - } - updateCtx.source = rel - updateCtx.partitionSources = partitionRels + + update.MultiUpdateCtx = writer.updateCtxs + + err = writer.free(proc) + if err != nil { + return err } + writer.updateCtxs = nil + + case UpdateWriteTable: + //do nothing + } + + mainCtx := update.MultiUpdateCtx[0] + if len(mainCtx.DeleteCols) > 0 && len(mainCtx.InsertCols) > 0 { + update.ctr.action = actionUpdate + } else if len(mainCtx.InsertCols) > 0 { + update.ctr.action = actionInsert + } else { + update.ctr.action = actionDelete } return nil @@ -102,10 +144,17 @@ func (update *MultiUpdate) Call(proc *process.Process) (vm.CallResult, error) { analyzer.Stop() }() - if update.ToWriteS3 { + switch update.Action { + case UpdateWriteS3: return update.update_s3(proc, analyzer) + case UpdateWriteTable: + return update.update(proc, analyzer) + case UpdateFlushS3Info: + return update.updateFlushS3Info(proc, analyzer) + default: } - return update.update(proc, analyzer) + + panic(fmt.Sprintf("unexpected multi_update.UpdateAction: %#v", update.Action)) } func (update *MultiUpdate) update_s3(proc *process.Process, analyzer process.Analyzer) (vm.CallResult, error) { @@ -131,7 +180,6 @@ func (update *MultiUpdate) update_s3(proc *process.Process, analyzer process.Ana continue } - ctr.affectedRows += uint64(input.Batch.RowCount()) err = ctr.s3Writer.append(proc, input.Batch) if err != nil { return vm.CancelResult, err @@ -140,11 +188,17 @@ func (update *MultiUpdate) update_s3(proc *process.Process, analyzer process.Ana } if ctr.state == vm.Eval { - err := ctr.s3Writer.flushTailAndWriteToWorkspace(proc, update) + ctr.state = vm.End + err := ctr.s3Writer.flushTailAndWriteToOutput(proc) if err != nil { return vm.CancelResult, err } - return vm.NewCallResult(), nil + if ctr.s3Writer.outputBat.RowCount() == 0 { + return vm.CancelResult, err + } + result := vm.NewCallResult() + result.Batch = ctr.s3Writer.outputBat + return result, nil } return vm.CancelResult, nil @@ -165,39 +219,112 @@ func (update *MultiUpdate) update(proc *process.Process, analyzer process.Analyz return vm.CancelResult, err } - update.ctr.affectedRows += uint64(input.Batch.RowCount()) analyzer.Output(input.Batch) return input, nil } +func (update *MultiUpdate) updateFlushS3Info(proc *process.Process, analyzer process.Analyzer) (vm.CallResult, error) { + input, err := vm.ChildrenCall(update.GetChildren(0), proc, analyzer) + if err != nil { + return input, err + } + + if input.Batch == nil || input.Batch.IsEmpty() { + return input, nil + } + + actions := vector.MustFixedColNoTypeCheck[uint8](input.Batch.Vecs[0]) + updateCtxIdx := vector.MustFixedColNoTypeCheck[uint16](input.Batch.Vecs[1]) + partitionIdx := vector.MustFixedColNoTypeCheck[uint16](input.Batch.Vecs[2]) + rowCounts := vector.MustFixedColNoTypeCheck[uint64](input.Batch.Vecs[3]) + nameData, nameArea := vector.MustVarlenaRawData(input.Batch.Vecs[4]) + batData, batArea := vector.MustVarlenaRawData(input.Batch.Vecs[5]) + + ctx := proc.Ctx + batBufs := make(map[actionType]*batch.Batch) + defer func() { + for _, bat := range batBufs { + bat.Clean(proc.Mp()) + } + }() + + for i, action := range actions { + updateCtx := update.MultiUpdateCtx[updateCtxIdx[i]] + + switch actionType(action) { + case actionDelete: + if batBufs[actionDelete] == nil { + batBufs[actionDelete] = batch.NewOffHeapEmpty() + } else { + batBufs[actionDelete].CleanOnlyData() + } + if err := batBufs[actionDelete].UnmarshalBinary(batData[i].GetByteSlice(batArea)); err != nil { + return input, err + } + tableType := update.ctr.updateCtxInfos[updateCtx.TableDef.Name].tableType + update.addDeleteAffectRows(tableType, rowCounts[i]) + name := nameData[i].UnsafeGetString(nameArea) + source := update.ctr.updateCtxInfos[updateCtx.TableDef.Name].Sources[partitionIdx[i]] + err = source.Delete(ctx, batBufs[actionDelete], name) + + case actionInsert: + if batBufs[actionInsert] == nil { + batBufs[actionInsert] = batch.NewOffHeapEmpty() + } else { + batBufs[actionInsert].CleanOnlyData() + } + if err := batBufs[actionInsert].UnmarshalBinary(batData[i].GetByteSlice(batArea)); err != nil { + return input, err + } + + tableType := update.ctr.updateCtxInfos[updateCtx.TableDef.Name].tableType + update.addInsertAffectRows(tableType, rowCounts[i]) + source := update.ctr.updateCtxInfos[updateCtx.TableDef.Name].Sources[partitionIdx[i]] + err = source.Write(ctx, batBufs[actionInsert]) + + case actionUpdate: + if batBufs[actionUpdate] == nil { + batBufs[actionUpdate] = batch.NewOffHeapEmpty() + } else { + batBufs[actionUpdate].CleanOnlyData() + } + if err := batBufs[actionUpdate].UnmarshalBinary(batData[i].GetByteSlice(batArea)); err != nil { + return input, err + } + + err = update.updateOneBatch(proc, batBufs[actionUpdate]) + default: + panic("unexpected multi_update.actionType") + } + + if err != nil { + return vm.CancelResult, err + } + } + + return input, nil +} + func (update *MultiUpdate) updateOneBatch(proc *process.Process, bat *batch.Batch) (err error) { - ctr := &update.ctr for i, updateCtx := range update.MultiUpdateCtx { // delete rows - if len(updateCtx.deleteCols) > 0 { - // init buf - if ctr.deleteBuf[i] == nil { - mainPkIdx := updateCtx.deleteCols[1] - buf := batch.New([]string{catalog.Row_ID, "pk"}) - buf.SetVector(0, vector.NewVec(types.T_Rowid.ToType())) - buf.SetVector(1, vector.NewVec(*bat.Vecs[mainPkIdx].GetType())) - ctr.deleteBuf[i] = buf - } - err = update.delete_table(proc, updateCtx, bat, ctr.deleteBuf[i]) + if len(updateCtx.DeleteCols) > 0 { + err = update.delete_table(proc, updateCtx, bat, i) if err != nil { return } } // insert rows - if len(updateCtx.insertCols) > 0 { - switch updateCtx.tableType { - case updateMainTable: + if len(updateCtx.InsertCols) > 0 { + tableType := update.ctr.updateCtxInfos[updateCtx.TableDef.Name].tableType + switch tableType { + case UpdateMainTable: err = update.insert_main_table(proc, i, bat) - case updateUniqueIndexTable: + case UpdateUniqueIndexTable: err = update.insert_uniuqe_index_table(proc, i, bat) - case updateSecondaryIndexTable: + case UpdateSecondaryIndexTable: err = update.insert_secondary_index_table(proc, i, bat) } if err != nil { diff --git a/pkg/sql/colexec/multi_update/s3writer.go b/pkg/sql/colexec/multi_update/s3writer.go index 30f1428dc615..7e0a11061586 100644 --- a/pkg/sql/colexec/multi_update/s3writer.go +++ b/pkg/sql/colexec/multi_update/s3writer.go @@ -15,6 +15,10 @@ package multi_update import ( + "bytes" + "fmt" + "slices" + "github.com/matrixorigin/matrixone/pkg/catalog" "github.com/matrixorigin/matrixone/pkg/common/mpool" "github.com/matrixorigin/matrixone/pkg/container/batch" @@ -23,7 +27,9 @@ import ( "github.com/matrixorigin/matrixone/pkg/container/vector" "github.com/matrixorigin/matrixone/pkg/objectio" "github.com/matrixorigin/matrixone/pkg/sql/colexec" + "github.com/matrixorigin/matrixone/pkg/sql/colexec/deletion" "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/blockio" + "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/options" "github.com/matrixorigin/matrixone/pkg/vm/process" ) @@ -32,90 +38,135 @@ const ( DeleteWriteS3Threshold uint64 = 16 * mpool.MB TagS3SizeForMOLogger uint64 = 1 * mpool.MB + + RowIDIdx = 0 + PkIdx = 1 +) + +var ( + DeleteBatchAttrs = []string{catalog.Row_ID, "pk"} ) +type deleteBlockData struct { + bitmap *nulls.Nulls + typ int8 + bat *batch.Batch +} + +type deleteBlockInfo struct { + name string + bat *batch.Batch + rawRowCount uint64 +} + +func newDeleteBlockData(inputBatch *batch.Batch, pkIdx int) *deleteBlockData { + data := &deleteBlockData{ + bitmap: nulls.NewWithSize(int(options.DefaultBlockMaxRows)), + typ: deletion.RawRowIdBatch, + bat: newDeleteBatch(inputBatch, pkIdx), + } + return data +} + type s3Writer struct { cacheBatchs *batch.CompactBatchs + segmentMap map[string]int32 action actionType updateCtxs []*MultiUpdateCtx + updateCtxInfos map[string]*updateCtxInfo seqnums [][]uint16 sortIdxs []int pkIdxs []int schemaVersions []uint32 isClusterBys []bool - deleteBlockInfo [][]*batch.Batch - insertBlockInfo [][]*batch.Batch + deleteBlockMap [][]map[types.Blockid]*deleteBlockData + deleteBlockInfo [][]*deleteBlockInfo + insertBlockInfo [][]*batch.Batch + insertBlockRowCount [][]uint64 deleteBuf []*batch.Batch insertBuf []*batch.Batch + outputBat *batch.Batch + batchSize uint64 flushThreshold uint64 checkSizeCols []int + buf bytes.Buffer } func newS3Writer(update *MultiUpdate) (*s3Writer, error) { tableCount := len(update.MultiUpdateCtx) writer := &s3Writer{ cacheBatchs: batch.NewCompactBatchs(), + segmentMap: update.SegmentMap, + updateCtxInfos: update.ctr.updateCtxInfos, seqnums: make([][]uint16, 0, tableCount), sortIdxs: make([]int, 0, tableCount), pkIdxs: make([]int, 0, tableCount), schemaVersions: make([]uint32, 0, tableCount), isClusterBys: make([]bool, 0, tableCount), - deleteBuf: make([]*batch.Batch, tableCount), - insertBuf: make([]*batch.Batch, tableCount), - deleteBlockInfo: make([][]*batch.Batch, tableCount), - insertBlockInfo: make([][]*batch.Batch, tableCount), + deleteBuf: make([]*batch.Batch, tableCount), + insertBuf: make([]*batch.Batch, tableCount), + deleteBlockInfo: make([][]*deleteBlockInfo, tableCount), + insertBlockInfo: make([][]*batch.Batch, tableCount), + insertBlockRowCount: make([][]uint64, tableCount), + deleteBlockMap: make([][]map[types.Blockid]*deleteBlockData, tableCount), } var thisUpdateCtxs []*MultiUpdateCtx var mainUpdateCtx *MultiUpdateCtx for _, updateCtx := range update.MultiUpdateCtx { - if updateCtx.tableType == updateMainTable { + tableType := update.ctr.updateCtxInfos[updateCtx.TableDef.Name].tableType + if tableType == UpdateMainTable { mainUpdateCtx = updateCtx break } } for _, updateCtx := range update.MultiUpdateCtx { - if updateCtx.tableType != updateMainTable { + tableType := update.ctr.updateCtxInfos[updateCtx.TableDef.Name].tableType + if tableType != UpdateMainTable { thisUpdateCtxs = append(thisUpdateCtxs, updateCtx) - appendCfgToWriter(writer, updateCtx.tableDef) + appendCfgToWriter(writer, updateCtx.TableDef) } } // main table allways at the end for s3writer.updateCtxs // because main table will write to s3 at last - thisUpdateCtxs = append(thisUpdateCtxs, mainUpdateCtx) - appendCfgToWriter(writer, mainUpdateCtx.tableDef) + if mainUpdateCtx != nil { + // only insert into hidden table + thisUpdateCtxs = append(thisUpdateCtxs, mainUpdateCtx) + appendCfgToWriter(writer, mainUpdateCtx.TableDef) + } writer.updateCtxs = thisUpdateCtxs - if len(mainUpdateCtx.deleteCols) > 0 && len(mainUpdateCtx.insertCols) > 0 { + upCtx := thisUpdateCtxs[len(thisUpdateCtxs)-1] + if len(upCtx.DeleteCols) > 0 && len(upCtx.InsertCols) > 0 { //update writer.action = actionUpdate writer.flushThreshold = InsertWriteS3Threshold - writer.checkSizeCols = append(writer.checkSizeCols, mainUpdateCtx.insertCols...) - } else if len(mainUpdateCtx.insertCols) > 0 { + writer.checkSizeCols = append(writer.checkSizeCols, upCtx.InsertCols...) + } else if len(upCtx.InsertCols) > 0 { //insert writer.action = actionInsert writer.flushThreshold = InsertWriteS3Threshold - writer.checkSizeCols = append(writer.checkSizeCols, mainUpdateCtx.insertCols...) + writer.checkSizeCols = append(writer.checkSizeCols, upCtx.InsertCols...) } else { //delete writer.action = actionDelete writer.flushThreshold = DeleteWriteS3Threshold - writer.checkSizeCols = append(writer.checkSizeCols, mainUpdateCtx.deleteCols...) + writer.checkSizeCols = append(writer.checkSizeCols, upCtx.DeleteCols...) } return writer, nil } func (writer *s3Writer) append(proc *process.Process, inBatch *batch.Batch) (err error) { - err = writer.cacheBatchs.Push(proc.Mp(), inBatch) + err = writer.cacheBatchs.Extend(proc.Mp(), inBatch) if err != nil { return } @@ -129,25 +180,122 @@ func (writer *s3Writer) append(proc *process.Process, inBatch *batch.Batch) (err return } +func (writer *s3Writer) prepareDeleteBatchs(proc *process.Process, idx int, partIdx int, src []*batch.Batch) ([]*batch.Batch, error) { + defer func() { + for _, bat := range src { + bat.Clean(proc.GetMPool()) + } + }() + // split delete batchs by BlockID + if writer.deleteBlockMap[idx][partIdx] == nil { + writer.deleteBlockMap[idx][partIdx] = make(map[types.Blockid]*deleteBlockData) + } + blockMap := writer.deleteBlockMap[idx][partIdx] + + for _, bat := range src { + rowIDVec := bat.GetVector(RowIDIdx) + if rowIDVec.IsConstNull() { + continue + } + nulls := rowIDVec.GetNulls() + if nulls.Count() == bat.RowCount() { + continue + } + + rowIDs := vector.MustFixedColWithTypeCheck[types.Rowid](rowIDVec) + for i, rowID := range rowIDs { + if nulls.Contains(uint64(i)) { + continue + } + + blkid := rowID.CloneBlockID() + segid := rowID.CloneSegmentID() + // blkOffset := rowID.GetBlockOffset() + rowOffset := rowID.GetRowOffset() + + if blockMap[blkid] == nil { + blockMap[blkid] = newDeleteBlockData(bat, 1) + err := blockMap[blkid].bat.PreExtend(proc.GetMPool(), colexec.DefaultBatchSize) + if err != nil { + return nil, err + } + strSegid := string(segid[:]) + if writer.segmentMap[strSegid] == colexec.TxnWorkSpaceIdType { + blockMap[blkid].typ = deletion.RawBatchOffset + } else if writer.segmentMap[strSegid] == colexec.CnBlockIdType { + blockMap[blkid].typ = deletion.CNBlockOffset + } else { + blockMap[blkid].typ = deletion.RawRowIdBatch + } + } + + block := blockMap[blkid] + + bitmap := block.bitmap + if bitmap.Contains(uint64(rowOffset)) { + continue + } else { + bitmap.Add(uint64(rowOffset)) + } + + vector.AppendFixed(block.bat.GetVector(RowIDIdx), rowID, false, proc.GetMPool()) + block.bat.GetVector(PkIdx).UnionOne(bat.GetVector(PkIdx), int64(i), proc.GetMPool()) + block.bat.SetRowCount(block.bat.Vecs[0].Length()) + } + } + + //collect batchs that can be flush + blkids := make([]types.Blockid, 0, len(blockMap)) + for blkid, data := range blockMap { + //Don't flush rowids belong to uncommitted cn block and raw data batch in txn's workspace. + if data.typ != deletion.RawRowIdBatch { + continue + } + blkids = append(blkids, blkid) + } + slices.SortFunc(blkids, func(a, b types.Blockid) int { + return a.Compare(&b) + }) + deleteBats := batch.NewCompactBatchs() + for _, blkid := range blkids { + bat := blockMap[blkid].bat + delete(blockMap, blkid) + err := deleteBats.Push(proc.GetMPool(), bat) + if err != nil { + return nil, err + } + } + retBatchs := deleteBats.TakeBatchs() + return retBatchs, nil +} + func (writer *s3Writer) sortAndSync(proc *process.Process) (err error) { var bats []*batch.Batch onlyDelete := writer.action == actionDelete for i, updateCtx := range writer.updateCtxs { - parititionCount := len(updateCtx.partitionTableIDs) + parititionCount := len(updateCtx.PartitionTableIDs) + tableType := writer.updateCtxInfos[updateCtx.TableDef.Name].tableType // delete s3 - if len(updateCtx.deleteCols) > 0 { + if len(updateCtx.DeleteCols) > 0 { if parititionCount == 0 { // normal table - if onlyDelete && updateCtx.tableType == updateMainTable { - bats, err = fetchMainTableBatchs(proc, writer.cacheBatchs, -1, 0, updateCtx.deleteCols) + if onlyDelete && tableType == UpdateMainTable { + bats, err = fetchMainTableBatchs(proc, writer.cacheBatchs, -1, 0, updateCtx.DeleteCols, DeleteBatchAttrs) } else { - bats, err = cloneSomeVecFromCompactBatchs(proc, writer.cacheBatchs, -1, 0, updateCtx.deleteCols) + bats, err = cloneSomeVecFromCompactBatchs(proc, writer.cacheBatchs, -1, 0, updateCtx.DeleteCols, DeleteBatchAttrs, writer.sortIdxs[i]) } if err != nil { return } - err = writer.sortAndSyncOneTable(proc, i, 0, true, bats) + + var delBatchs []*batch.Batch + delBatchs, err = writer.prepareDeleteBatchs(proc, i, 0, bats) + if err != nil { + return + } + + err = writer.sortAndSyncOneTable(proc, i, 0, true, delBatchs) if err != nil { return } @@ -155,15 +303,21 @@ func (writer *s3Writer) sortAndSync(proc *process.Process) (err error) { // partition table lastIdx := parititionCount - 1 for getPartitionIdx := range parititionCount { - if onlyDelete && updateCtx.tableType == updateMainTable && getPartitionIdx == lastIdx { - bats, err = fetchMainTableBatchs(proc, writer.cacheBatchs, updateCtx.partitionIdx, getPartitionIdx, updateCtx.deleteCols) + if onlyDelete && tableType == UpdateMainTable && getPartitionIdx == lastIdx { + bats, err = fetchMainTableBatchs(proc, writer.cacheBatchs, updateCtx.OldPartitionIdx, getPartitionIdx, updateCtx.DeleteCols, DeleteBatchAttrs) } else { - bats, err = cloneSomeVecFromCompactBatchs(proc, writer.cacheBatchs, updateCtx.partitionIdx, getPartitionIdx, updateCtx.deleteCols) + bats, err = cloneSomeVecFromCompactBatchs(proc, writer.cacheBatchs, updateCtx.OldPartitionIdx, getPartitionIdx, updateCtx.DeleteCols, DeleteBatchAttrs, writer.sortIdxs[i]) } if err != nil { return } - err = writer.sortAndSyncOneTable(proc, i, int16(getPartitionIdx), true, bats) + + var delBatchs []*batch.Batch + delBatchs, err = writer.prepareDeleteBatchs(proc, i, getPartitionIdx, bats) + if err != nil { + return + } + err = writer.sortAndSyncOneTable(proc, i, int16(getPartitionIdx), true, delBatchs) if err != nil { return } @@ -172,13 +326,15 @@ func (writer *s3Writer) sortAndSync(proc *process.Process) (err error) { } // insert s3 - if len(updateCtx.insertCols) > 0 { + if len(updateCtx.InsertCols) > 0 { + insertAttrs := writer.updateCtxInfos[updateCtx.TableDef.Name].insertAttrs + tableType := writer.updateCtxInfos[updateCtx.TableDef.Name].tableType if parititionCount == 0 { // normal table - if updateCtx.tableType == updateMainTable { - bats, err = fetchMainTableBatchs(proc, writer.cacheBatchs, -1, 0, updateCtx.insertCols) + if tableType == UpdateMainTable { + bats, err = fetchMainTableBatchs(proc, writer.cacheBatchs, -1, 0, updateCtx.InsertCols, insertAttrs) } else { - bats, err = cloneSomeVecFromCompactBatchs(proc, writer.cacheBatchs, -1, 0, updateCtx.insertCols) + bats, err = cloneSomeVecFromCompactBatchs(proc, writer.cacheBatchs, -1, 0, updateCtx.InsertCols, insertAttrs, writer.sortIdxs[i]) } if err != nil { return @@ -191,10 +347,10 @@ func (writer *s3Writer) sortAndSync(proc *process.Process) (err error) { // partition table lastIdx := parititionCount - 1 for getPartitionIdx := range parititionCount { - if updateCtx.tableType == updateMainTable && getPartitionIdx == lastIdx { - bats, err = fetchMainTableBatchs(proc, writer.cacheBatchs, updateCtx.partitionIdx, getPartitionIdx, updateCtx.insertCols) + if tableType == UpdateMainTable && getPartitionIdx == lastIdx { + bats, err = fetchMainTableBatchs(proc, writer.cacheBatchs, updateCtx.NewPartitionIdx, getPartitionIdx, updateCtx.InsertCols, insertAttrs) } else { - bats, err = cloneSomeVecFromCompactBatchs(proc, writer.cacheBatchs, updateCtx.partitionIdx, getPartitionIdx, updateCtx.insertCols) + bats, err = cloneSomeVecFromCompactBatchs(proc, writer.cacheBatchs, updateCtx.NewPartitionIdx, getPartitionIdx, updateCtx.InsertCols, insertAttrs, writer.sortIdxs[i]) } if err != nil { return @@ -209,6 +365,9 @@ func (writer *s3Writer) sortAndSync(proc *process.Process) (err error) { } writer.batchSize = 0 + for _, bat := range writer.cacheBatchs.TakeBatchs() { + bat.Clean(proc.GetMPool()) + } return } @@ -222,7 +381,15 @@ func (writer *s3Writer) sortAndSyncOneTable( var blockInfos []objectio.BlockInfo var objStats objectio.ObjectStats + if len(bats) == 0 { + return nil + } + sortIndx := writer.sortIdxs[idx] + rowCount := 0 + for _, bat := range bats { + rowCount += bat.RowCount() + } if isDelete { sortIndx = 0 } @@ -244,7 +411,12 @@ func (writer *s3Writer) sortAndSyncOneTable( if err != nil { return } - return writer.fillBlockInfoBat(proc, idx, partitionIdx, blockInfos, objStats, isDelete) + + if isDelete { + return writer.fillDeleteBlockInfo(proc, idx, partitionIdx, objStats, rowCount) + } else { + return writer.fillInsertBlockInfo(proc, idx, partitionIdx, blockInfos, objStats, rowCount) + } } // need sort @@ -295,55 +467,71 @@ func (writer *s3Writer) sortAndSyncOneTable( if err != nil { return } - return writer.fillBlockInfoBat(proc, idx, partitionIdx, blockInfos, objStats, isDelete) + + if isDelete { + return writer.fillDeleteBlockInfo(proc, idx, partitionIdx, objStats, rowCount) + } else { + return writer.fillInsertBlockInfo(proc, idx, partitionIdx, blockInfos, objStats, rowCount) + } } -func (writer *s3Writer) fillBlockInfoBat( +func (writer *s3Writer) fillDeleteBlockInfo( proc *process.Process, idx int, partitionIdx int16, - blockInfos []objectio.BlockInfo, objStats objectio.ObjectStats, - isDelete bool) (err error) { + rowCount int) (err error) { // init buf - var targetBat *batch.Batch - if isDelete { - if writer.deleteBlockInfo[idx][partitionIdx] == nil { - attrs := []string{catalog.BlockMeta_TableIdx_Insert, catalog.BlockMeta_BlockInfo, catalog.ObjectMeta_ObjectStats} - blockInfoBat := batch.NewWithSize(len(attrs)) - blockInfoBat.Attrs = attrs - blockInfoBat.Vecs[0] = vector.NewVec(types.T_int16.ToType()) - blockInfoBat.Vecs[1] = vector.NewVec(types.T_text.ToType()) - blockInfoBat.Vecs[2] = vector.NewVec(types.T_binary.ToType()) - writer.deleteBlockInfo[idx][partitionIdx] = blockInfoBat - } - targetBat = writer.deleteBlockInfo[idx][partitionIdx] - } else { - if writer.insertBlockInfo[idx][partitionIdx] == nil { - attrs := []string{catalog.BlockMeta_TableIdx_Insert, catalog.BlockMeta_BlockInfo, catalog.ObjectMeta_ObjectStats} - blockInfoBat := batch.NewWithSize(len(attrs)) - blockInfoBat.Attrs = attrs - blockInfoBat.Vecs[0] = vector.NewVec(types.T_int16.ToType()) - blockInfoBat.Vecs[1] = vector.NewVec(types.T_text.ToType()) - blockInfoBat.Vecs[2] = vector.NewVec(types.T_binary.ToType()) - writer.insertBlockInfo[idx][partitionIdx] = blockInfoBat + if writer.deleteBlockInfo[idx][partitionIdx] == nil { + attrs := []string{catalog.ObjectMeta_ObjectStats} + blockInfoBat := batch.NewWithSize(len(attrs)) + blockInfoBat.Attrs = attrs + blockInfoBat.Vecs[0] = vector.NewVec(types.T_text.ToType()) + writer.deleteBlockInfo[idx][partitionIdx] = &deleteBlockInfo{ + name: "", + bat: blockInfoBat, } - targetBat = writer.insertBlockInfo[idx][partitionIdx] + } + + targetBloInfo := writer.deleteBlockInfo[idx][partitionIdx] + + objId := objStats.ObjectName().ObjectId()[:] + targetBloInfo.name = fmt.Sprintf("%s|%d", objId, deletion.FlushDeltaLoc) + targetBloInfo.rawRowCount += uint64(rowCount) + if err = vector.AppendBytes( + targetBloInfo.bat.GetVector(0), objStats.Marshal(), false, proc.GetMPool()); err != nil { + return } + targetBloInfo.bat.SetRowCount(targetBloInfo.bat.Vecs[0].Length()) + return +} - for _, blkInfo := range blockInfos { - if err = vector.AppendFixed( - targetBat.Vecs[0], - partitionIdx, - false, - proc.GetMPool()); err != nil { - return - } +func (writer *s3Writer) fillInsertBlockInfo( + proc *process.Process, + idx int, + partitionIdx int16, + blockInfos []objectio.BlockInfo, + objStats objectio.ObjectStats, + rowCount int) (err error) { + // init buf + if writer.insertBlockInfo[idx][partitionIdx] == nil { + attrs := []string{catalog.BlockMeta_BlockInfo, catalog.ObjectMeta_ObjectStats} + blockInfoBat := batch.NewWithSize(len(attrs)) + blockInfoBat.Attrs = attrs + blockInfoBat.Vecs[0] = vector.NewVec(types.T_text.ToType()) + blockInfoBat.Vecs[1] = vector.NewVec(types.T_binary.ToType()) + writer.insertBlockInfo[idx][partitionIdx] = blockInfoBat + writer.insertBlockRowCount[idx][partitionIdx] = 0 + } + + writer.insertBlockRowCount[idx][partitionIdx] += uint64(rowCount) + targetBat := writer.insertBlockInfo[idx][partitionIdx] + for _, blkInfo := range blockInfos { if err = vector.AppendBytes( - targetBat.Vecs[1], + targetBat.Vecs[0], objectio.EncodeBlockInfo(&blkInfo), false, proc.GetMPool()); err != nil { @@ -351,19 +539,17 @@ func (writer *s3Writer) fillBlockInfoBat( } } - if err = vector.AppendBytes(targetBat.Vecs[2], + if err = vector.AppendBytes(targetBat.Vecs[1], objStats.Marshal(), false, proc.GetMPool()); err != nil { return } - targetBat.SetRowCount(targetBat.Vecs[0].Length()) return } -func (writer *s3Writer) flushTailAndWriteToWorkspace(proc *process.Process, update *MultiUpdate) (err error) { - - if writer.batchSize > TagS3SizeForMOLogger { +func (writer *s3Writer) flushTailAndWriteToOutput(proc *process.Process) (err error) { + if (writer.action == actionDelete && writer.batchSize > 0) || writer.batchSize > TagS3SizeForMOLogger { //write tail batch to s3 err = writer.sortAndSync(proc) if err != nil { @@ -371,21 +557,48 @@ func (writer *s3Writer) flushTailAndWriteToWorkspace(proc *process.Process, upda } } - //write block info to workspace - for i, bats := range writer.deleteBlockInfo { - if len(bats) == 1 { + if writer.outputBat == nil { + writer.outputBat = makeS3OutputBatch() + } + mp := proc.GetMPool() + + //write delete block info to workspace + for i, partBlockInfos := range writer.deleteBlockInfo { + if len(partBlockInfos) == 1 { // normal table - if bats[0] != nil && bats[0].RowCount() > 0 { - err = writer.updateCtxs[i].source.Write(proc.Ctx, bats[0]) + if partBlockInfos[0] != nil && partBlockInfos[0].bat.RowCount() > 0 { + err = writer.addBatchToOutput(mp, actionDelete, i, 0, partBlockInfos[0].rawRowCount, partBlockInfos[0].name, partBlockInfos[0].bat) if err != nil { return } } } else { // partition table - for partIdx, bat := range bats { - if bat != nil && bat.RowCount() > 0 { - err = writer.updateCtxs[i].partitionSources[partIdx].Write(proc.Ctx, bat) + for partIdx, blockInfo := range partBlockInfos { + if blockInfo != nil && blockInfo.bat.RowCount() > 0 { + err = writer.addBatchToOutput(mp, actionDelete, i, partIdx, blockInfo.rawRowCount, blockInfo.name, blockInfo.bat) + if err != nil { + return + } + } + } + } + } + + //write delete batch (which not flush to s3) to workspace + for i, blocks := range writer.deleteBlockMap { + if len(blocks) == 1 { + // normal table + for blockID, blockData := range blocks[0] { + name := fmt.Sprintf("%s|%d", blockID, blockData.typ) + err = writer.addBatchToOutput(mp, actionDelete, i, 0, uint64(blockData.bat.RowCount()), name, blockData.bat) + } + } else { + // partition table + for partIdx, blockDatas := range blocks { + for blockID, blockData := range blockDatas { + name := fmt.Sprintf("%s|%d", blockID, blockData.typ) + err = writer.addBatchToOutput(mp, actionDelete, i, partIdx, uint64(blockData.bat.RowCount()), name, blockData.bat) if err != nil { return } @@ -393,11 +606,14 @@ func (writer *s3Writer) flushTailAndWriteToWorkspace(proc *process.Process, upda } } } + + //write insert block info to workspace for i, bats := range writer.insertBlockInfo { if len(bats) == 1 { // normal table if bats[0] != nil && bats[0].RowCount() > 0 { - err = writer.updateCtxs[i].source.Write(proc.Ctx, bats[0]) + resetMergeBlockForOldCN(proc, bats[0]) + err = writer.addBatchToOutput(mp, actionInsert, i, 0, writer.insertBlockRowCount[i][0], "", bats[0]) if err != nil { return } @@ -406,7 +622,8 @@ func (writer *s3Writer) flushTailAndWriteToWorkspace(proc *process.Process, upda // partition table for partIdx, bat := range bats { if bat != nil && bat.RowCount() > 0 { - err = writer.updateCtxs[i].partitionSources[partIdx].Write(proc.Ctx, bat) + resetMergeBlockForOldCN(proc, bat) + err = writer.addBatchToOutput(mp, actionInsert, i, partIdx, writer.insertBlockRowCount[i][partIdx], "", bat) if err != nil { return } @@ -417,11 +634,22 @@ func (writer *s3Writer) flushTailAndWriteToWorkspace(proc *process.Process, upda //write tail batch to workspace bats := writer.cacheBatchs.TakeBatchs() - for _, bat := range bats { - err = update.updateOneBatch(proc, bat) + defer func() { + for i := range bats { + if bats[i] != nil { + bats[i].Clean(proc.GetMPool()) + bats[i] = nil + } + } + }() + + for i, bat := range bats { + err = writer.addBatchToOutput(mp, actionUpdate, 0, 0, uint64(bat.RowCount()), "", bat) if err != nil { return } + bat.Clean(proc.Mp()) + bats[i] = nil } return nil @@ -439,10 +667,26 @@ func (writer *s3Writer) reset(proc *process.Process) (err error) { } } } - for _, bats := range writer.deleteBlockInfo { - for _, bat := range bats { - if bat != nil { - bat.CleanOnlyData() + for _, rowCounts := range writer.insertBlockRowCount { + for i := range rowCounts { + rowCounts[i] = 0 + } + } + for _, partBlockInfos := range writer.deleteBlockInfo { + for _, blockInfo := range partBlockInfos { + if blockInfo != nil { + blockInfo.bat.CleanOnlyData() + blockInfo.rawRowCount = 0 + } + } + } + for _, datas := range writer.deleteBlockMap { + for _, data := range datas { + for k, block := range data { + if block != nil && block.bat != nil { + block.bat.Clean(proc.Mp()) + } + delete(data, k) } } } @@ -456,7 +700,11 @@ func (writer *s3Writer) reset(proc *process.Process) (err error) { writer.deleteBuf[i].CleanOnlyData() } } + if writer.outputBat != nil { + writer.outputBat.CleanOnlyData() + } writer.batchSize = 0 + writer.buf.Reset() return } @@ -473,10 +721,11 @@ func (writer *s3Writer) free(proc *process.Process) (err error) { } } writer.insertBlockInfo = nil - for _, bats := range writer.deleteBlockInfo { - for _, bat := range bats { - if bat != nil { - bat.Clean(proc.Mp()) + writer.insertBlockRowCount = nil + for _, partBlockInfos := range writer.deleteBlockInfo { + for _, blockInfo := range partBlockInfos { + if blockInfo != nil { + blockInfo.bat.Clean(proc.Mp()) } } } @@ -496,5 +745,82 @@ func (writer *s3Writer) free(proc *process.Process) (err error) { } writer.deleteBuf = nil + for _, datas := range writer.deleteBlockMap { + for _, data := range datas { + for _, block := range data { + if block != nil && block.bat != nil { + block.bat.Clean(proc.Mp()) + } + } + } + } + writer.deleteBlockMap = nil + + if writer.outputBat != nil { + writer.outputBat.Clean(proc.Mp()) + writer.outputBat = nil + } + writer.buf.Reset() + return } + +func (writer *s3Writer) addBatchToOutput( + mp *mpool.MPool, + action actionType, + idx int, + partIdx int, + rowCount uint64, + name string, + bat *batch.Batch) (err error) { + output := writer.outputBat + + err = vector.AppendFixed(output.Vecs[0], uint8(action), false, mp) + if err != nil { + return + } + + err = vector.AppendFixed(output.Vecs[1], uint16(idx), false, mp) + if err != nil { + return + } + + err = vector.AppendFixed(output.Vecs[2], uint16(partIdx), false, mp) + if err != nil { + return + } + + err = vector.AppendFixed(output.Vecs[3], rowCount, false, mp) + if err != nil { + return + } + + err = vector.AppendBytes(output.Vecs[4], []byte(name), false, mp) + if err != nil { + return + } + + var val []byte + val, err = bat.MarshalBinaryWithBuffer(&writer.buf) + if err != nil { + return + } + + err = vector.AppendBytes(output.Vecs[5], val, false, mp) + if err != nil { + return + } + output.SetRowCount(output.Vecs[0].Length()) + return +} + +func makeS3OutputBatch() *batch.Batch { + bat := batch.NewWithSize(6) + bat.Vecs[0] = vector.NewVec(types.T_uint8.ToType()) // action type 0=actionInsert, 1=actionDelete + bat.Vecs[1] = vector.NewVec(types.T_uint16.ToType()) // index of update.UpdateCtxs + bat.Vecs[2] = vector.NewVec(types.T_uint16.ToType()) // index of partitions + bat.Vecs[3] = vector.NewVec(types.T_uint64.ToType()) // rowCount of s3 blocks + bat.Vecs[4] = vector.NewVec(types.T_varchar.ToType()) // name for delete. empty for insert + bat.Vecs[5] = vector.NewVec(types.T_text.ToType()) // originBatch.MarshalBinary() + return bat +} diff --git a/pkg/sql/colexec/multi_update/s3writer_util.go b/pkg/sql/colexec/multi_update/s3writer_util.go index 1929801e2676..076647c1c3e4 100644 --- a/pkg/sql/colexec/multi_update/s3writer_util.go +++ b/pkg/sql/colexec/multi_update/s3writer_util.go @@ -18,15 +18,19 @@ import ( "github.com/matrixorigin/matrixone/pkg/catalog" "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/container/batch" + "github.com/matrixorigin/matrixone/pkg/container/types" "github.com/matrixorigin/matrixone/pkg/container/vector" "github.com/matrixorigin/matrixone/pkg/defines" "github.com/matrixorigin/matrixone/pkg/fileservice" + "github.com/matrixorigin/matrixone/pkg/logutil" "github.com/matrixorigin/matrixone/pkg/objectio" "github.com/matrixorigin/matrixone/pkg/sql/colexec" "github.com/matrixorigin/matrixone/pkg/sql/plan" + "github.com/matrixorigin/matrixone/pkg/vm/engine/disttae" "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/blockio" "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/index" "github.com/matrixorigin/matrixone/pkg/vm/process" + "go.uber.org/zap" ) func generateBlockWriter(writer *s3Writer, @@ -40,8 +44,10 @@ func generateBlockWriter(writer *s3Writer, return nil, err } seqnums := writer.seqnums[idx] + sortIdx := writer.sortIdxs[idx] if isDelete { seqnums = nil + sortIdx = 0 } blockWriter, err := blockio.NewBlockWriterNew( s3, @@ -54,19 +60,17 @@ func generateBlockWriter(writer *s3Writer, return nil, err } - if writer.sortIdxs[idx] > -1 { - blockWriter.SetSortKey(uint16(writer.sortIdxs[idx])) + if sortIdx > -1 { + blockWriter.SetSortKey(uint16(sortIdx)) } if isDelete { - if writer.pkIdxs[idx] > -1 { - blockWriter.SetPrimaryKeyWithType( - uint16(writer.pkIdxs[idx]), - index.HBF, - index.ObjectPrefixFn, - index.BlockPrefixFn, - ) - } + blockWriter.SetPrimaryKeyWithType( + 0, + index.HBF, + index.ObjectPrefixFn, + index.BlockPrefixFn, + ) } else { if writer.pkIdxs[idx] > -1 { blockWriter.SetPrimaryKey(uint16(writer.pkIdxs[idx])) @@ -98,30 +102,37 @@ func appendCfgToWriter(writer *s3Writer, tableDef *plan.TableDef) { thisIdx := len(writer.sortIdxs) writer.seqnums = append(writer.seqnums, seqnums) writer.sortIdxs = append(writer.sortIdxs, sortIdx) - writer.pkIdxs = append(writer.sortIdxs, pkIdx) + writer.pkIdxs = append(writer.pkIdxs, pkIdx) writer.schemaVersions = append(writer.schemaVersions, tableDef.Version) writer.isClusterBys = append(writer.isClusterBys, tableDef.ClusterBy != nil) if tableDef.Partition == nil { - writer.deleteBlockInfo[thisIdx] = make([]*batch.Batch, 1) + writer.deleteBlockMap[thisIdx] = make([]map[types.Blockid]*deleteBlockData, 1) + writer.deleteBlockInfo[thisIdx] = make([]*deleteBlockInfo, 1) writer.insertBlockInfo[thisIdx] = make([]*batch.Batch, 1) + writer.insertBlockRowCount[thisIdx] = make([]uint64, 1) } else { partitionCount := len(tableDef.Partition.PartitionTableNames) - writer.deleteBlockInfo[thisIdx] = make([]*batch.Batch, partitionCount) + writer.deleteBlockMap[thisIdx] = make([]map[types.Blockid]*deleteBlockData, partitionCount) + writer.deleteBlockInfo[thisIdx] = make([]*deleteBlockInfo, partitionCount) writer.insertBlockInfo[thisIdx] = make([]*batch.Batch, partitionCount) + writer.insertBlockRowCount[thisIdx] = make([]uint64, partitionCount) } } -// cloneSomeVecFromCompactBatchs for hidden table. copy vec to new batch +// cloneSomeVecFromCompactBatchs copy vec to new batch func cloneSomeVecFromCompactBatchs( proc *process.Process, src *batch.CompactBatchs, partitionIdxInBatch int, getPartitionIdx int, - cols []int) ([]*batch.Batch, error) { + cols []int, + attrs []string, + sortIdx int) ([]*batch.Batch, error) { var err error var newBat *batch.Batch - bats := make([]*batch.Batch, src.Length()) + bats := make([]*batch.Batch, 0, src.Length()) + defer func() { if err != nil { for _, bat := range bats { @@ -139,17 +150,23 @@ func cloneSomeVecFromCompactBatchs( expect := int32(getPartitionIdx) for i := 0; i < src.Length(); i++ { newBat = batch.NewWithSize(len(cols)) + newBat.Attrs = attrs oldBat := src.Get(i) rid2pid := vector.MustFixedColWithTypeCheck[int32](oldBat.Vecs[partitionIdxInBatch]) nulls := oldBat.Vecs[partitionIdxInBatch].GetNulls() + sortNulls := oldBat.Vecs[cols[sortIdx]].GetNulls() for newColIdx, oldColIdx := range cols { typ := oldBat.Vecs[oldColIdx].GetType() newBat.Vecs[newColIdx] = vector.NewVec(*typ) } + err = newBat.PreExtend(proc.GetMPool(), colexec.DefaultBatchSize) + if err != nil { + return nil, err + } for rowIdx, partition := range rid2pid { - if !nulls.Contains(uint64(i)) { + if !nulls.Contains(uint64(rowIdx)) && !sortNulls.Contains(uint64(rowIdx)) { if partition == -1 { return nil, moerr.NewInvalidInput(proc.Ctx, "Table has no partition for value from column_list") } else if partition == expect { @@ -161,22 +178,52 @@ func cloneSomeVecFromCompactBatchs( } } } - newBat.SetRowCount(newBat.Vecs[0].Length()) - bats[i] = newBat + if newBat.Vecs[0].Length() > 0 { + newBat.SetRowCount(newBat.Vecs[0].Length()) + bats = append(bats, newBat) + } else { + newBat.Clean(proc.GetMPool()) + } newBat = nil } } else { for i := 0; i < src.Length(); i++ { newBat = batch.NewWithSize(len(cols)) + newBat.Attrs = attrs oldBat := src.Get(i) - for newColIdx, oldColIdx := range cols { - newBat.Vecs[newColIdx], err = oldBat.Vecs[oldColIdx].Dup(proc.GetMPool()) - if err != nil { - return nil, err + + sortVec := oldBat.Vecs[cols[sortIdx]] + if sortVec.HasNull() { + sortNulls := sortVec.GetNulls() + for newColIdx, oldColIdx := range cols { + typ := oldBat.Vecs[oldColIdx].GetType() + newBat.Vecs[newColIdx] = vector.NewVec(*typ) + } + + for j := 0; j < oldBat.RowCount(); j++ { + if !sortNulls.Contains(uint64(j)) { + for newColIdx, oldColIdx := range cols { + if err = newBat.Vecs[newColIdx].UnionOne(oldBat.Vecs[oldColIdx], int64(j), proc.GetMPool()); err != nil { + return nil, err + } + } + } + } + } else { + for newColIdx, oldColIdx := range cols { + newBat.Vecs[newColIdx], err = oldBat.Vecs[oldColIdx].Dup(proc.GetMPool()) + if err != nil { + return nil, err + } } } - newBat.SetRowCount(newBat.Vecs[0].Length()) - bats[i] = newBat + + if newBat.Vecs[0].Length() > 0 { + newBat.SetRowCount(newBat.Vecs[0].Length()) + bats = append(bats, newBat) + } else { + newBat.Clean(proc.GetMPool()) + } newBat = nil } } @@ -190,12 +237,14 @@ func fetchMainTableBatchs( src *batch.CompactBatchs, partitionIdxInBatch int, getPartitionIdx int, - cols []int) ([]*batch.Batch, error) { + cols []int, + attrs []string) ([]*batch.Batch, error) { var err error var newBat *batch.Batch mp := proc.GetMPool() retBats := make([]*batch.Batch, src.Length()) srcBats := src.TakeBatchs() + defer func() { for i, bat := range srcBats { for _, vec := range bat.Vecs { @@ -224,6 +273,7 @@ func fetchMainTableBatchs( expect := int32(getPartitionIdx) for i, oldBat := range srcBats { newBat = batch.NewWithSize(len(cols)) + newBat.Attrs = attrs rid2pid := vector.MustFixedColWithTypeCheck[int32](oldBat.Vecs[partitionIdxInBatch]) nulls := oldBat.Vecs[partitionIdxInBatch].GetNulls() @@ -231,6 +281,10 @@ func fetchMainTableBatchs( typ := oldBat.Vecs[oldColIdx].GetType() newBat.Vecs[newColIdx] = vector.NewVec(*typ) } + err = newBat.PreExtend(proc.GetMPool(), colexec.DefaultBatchSize) + if err != nil { + return nil, err + } for rowIdx, partition := range rid2pid { if !nulls.Contains(uint64(i)) { @@ -252,10 +306,23 @@ func fetchMainTableBatchs( } else { for i, oldBat := range srcBats { newBat = batch.NewWithSize(len(cols)) + newBat.Attrs = attrs for j, idx := range cols { oldVec := oldBat.Vecs[idx] srcBats[i].ReplaceVector(oldVec, nil, 0) - newBat.Vecs[j] = oldVec + + //expand constant vector + if oldVec.IsConst() { + newVec := vector.NewVec(*oldVec.GetType()) + err = vector.GetUnionAllFunction(*oldVec.GetType(), mp)(newVec, oldVec) + if err != nil { + return nil, err + } + oldVec.Free(mp) + newBat.Vecs[j] = newVec + } else { + newBat.Vecs[j] = oldVec + } } newBat.SetRowCount(newBat.Vecs[0].Length()) retBats[i] = newBat @@ -286,3 +353,32 @@ func syncThenGetBlockInfoAndStats(proc *process.Process, blockWriter *blockio.Bl return blkInfos, stats, err } + +func resetMergeBlockForOldCN(proc *process.Process, bat *batch.Batch) error { + if bat.Attrs[len(bat.Attrs)-1] != catalog.ObjectMeta_ObjectStats { + // bat comes from old CN, no object stats vec in it + bat.Attrs = append(bat.Attrs, catalog.ObjectMeta_ObjectStats) + bat.Vecs = append(bat.Vecs, vector.NewVec(types.T_binary.ToType())) + + blkVec := bat.Vecs[0] + destVec := bat.Vecs[1] + fs, err := fileservice.Get[fileservice.FileService](proc.Base.FileService, defines.SharedFileServiceName) + if err != nil { + logutil.Error("get fs failed when split object stats. ", zap.Error(err)) + return err + } + // var objDataMeta objectio.ObjectDataMeta + var objStats objectio.ObjectStats + for idx := 0; idx < bat.RowCount(); idx++ { + blkInfo := objectio.DecodeBlockInfo(blkVec.GetBytesAt(idx)) + objStats, _, err = disttae.ConstructObjStatsByLoadObjMeta(proc.Ctx, blkInfo.MetaLocation(), fs) + if err != nil { + return err + } + vector.AppendBytes(destVec, objStats.Marshal(), false, proc.GetMPool()) + } + + vector.AppendBytes(destVec, objStats.Marshal(), false, proc.GetMPool()) + } + return nil +} diff --git a/pkg/sql/colexec/multi_update/types.go b/pkg/sql/colexec/multi_update/types.go index f5232e5d0fa9..14d229da513f 100644 --- a/pkg/sql/colexec/multi_update/types.go +++ b/pkg/sql/colexec/multi_update/types.go @@ -27,12 +27,20 @@ var _ vm.Operator = new(MultiUpdate) const opName = "MultiUpdate" -type updateTableType int +type UpdateAction int const ( - updateMainTable updateTableType = iota - updateUniqueIndexTable - updateSecondaryIndexTable + UpdateWriteTable UpdateAction = iota + UpdateWriteS3 + UpdateFlushS3Info +) + +type UpdateTableType int + +const ( + UpdateMainTable UpdateTableType = iota + UpdateUniqueIndexTable + UpdateSecondaryIndexTable ) type actionType int @@ -60,39 +68,48 @@ type MultiUpdate struct { ctr container MultiUpdateCtx []*MultiUpdateCtx - ToWriteS3 bool + Action UpdateAction IsOnduplicateKeyUpdate bool Engine engine.Engine + SegmentMap map[string]int32 + vm.OperatorBase } +type updateCtxInfo struct { + Sources []engine.Relation + tableType UpdateTableType + insertAttrs []string +} + type container struct { state vm.CtrState affectedRows uint64 + action actionType - s3Writer *s3Writer + s3Writer *s3Writer + updateCtxInfos map[string]*updateCtxInfo insertBuf []*batch.Batch deleteBuf []*batch.Batch } type MultiUpdateCtx struct { - ref *plan.ObjectRef - tableDef *plan.TableDef - - tableType updateTableType + ObjRef *plan.ObjectRef + TableDef *plan.TableDef - insertCols []int - deleteCols []int + InsertCols []int + DeleteCols []int - partitionTableIDs []int32 // Align array index with the partition number - partitionTableNames []string // Align array index with the partition number - partitionIdx int // The array index position of the partition expression column + PartitionTableIDs []uint64 // Align array index with the partition number + PartitionTableNames []string // Align array index with the partition number + OldPartitionIdx int // The array index position of the partition expression column for delete + NewPartitionIdx int // The array index position of the partition expression column for insert - source engine.Relation - partitionSources []engine.Relation // Align array index with the partition number + // Source engine.Relation + // PartitionSources []engine.Relation // Align array index with the partition number } func (update MultiUpdate) TypeName() string { @@ -128,8 +145,10 @@ func (update *MultiUpdate) Reset(proc *process.Process, pipelineFailed bool, err if update.ctr.s3Writer != nil { update.ctr.s3Writer.reset(proc) } + for _, info := range update.ctr.updateCtxInfos { + info.Sources = nil + } update.ctr.state = vm.Build - update.ctr.affectedRows = 0 } func (update *MultiUpdate) Free(proc *process.Process, pipelineFailed bool, err error) { @@ -152,8 +171,36 @@ func (update *MultiUpdate) Free(proc *process.Process, pipelineFailed bool, err update.ctr.s3Writer.free(proc) update.ctr.s3Writer = nil } + + update.ctr.updateCtxInfos = nil } func (update *MultiUpdate) GetAffectedRows() uint64 { return update.ctr.affectedRows } + +func (update *MultiUpdate) SetAffectedRows(affectedRows uint64) { + update.ctr.affectedRows = affectedRows +} + +func (update *MultiUpdate) addInsertAffectRows(tableType UpdateTableType, rowCount uint64) { + if tableType != UpdateMainTable { + return + } + switch update.ctr.action { + case actionInsert: + update.ctr.affectedRows += rowCount + } +} + +func (update *MultiUpdate) addDeleteAffectRows(tableType UpdateTableType, rowCount uint64) { + if tableType != UpdateMainTable { + return + } + switch update.ctr.action { + case actionDelete: + update.ctr.affectedRows += rowCount + case actionUpdate: + update.ctr.affectedRows += rowCount + } +} diff --git a/pkg/sql/colexec/multi_update/update_test.go b/pkg/sql/colexec/multi_update/update_test.go index c0332b3816be..545ada827400 100644 --- a/pkg/sql/colexec/multi_update/update_test.go +++ b/pkg/sql/colexec/multi_update/update_test.go @@ -107,8 +107,8 @@ func buildUpdateTestCase(t *testing.T, hasUniqueKey bool, hasSecondaryKey bool, batchs, affectRows := prepareUpdateTestBatchs(proc.GetMPool(), 3, hasUniqueKey, hasSecondaryKey, isPartition) multiUpdateCtxs := prepareTestUpdateMultiUpdateCtx(hasUniqueKey, hasSecondaryKey, isPartition) - toWriteS3 := false - retCase := buildTestCase(multiUpdateCtxs, eng, batchs, affectRows, toWriteS3) + action := UpdateWriteTable + retCase := buildTestCase(multiUpdateCtxs, eng, batchs, affectRows, action) return proc, retCase } @@ -116,10 +116,10 @@ func buildUpdateS3TestCase(t *testing.T, hasUniqueKey bool, hasSecondaryKey bool _, ctrl, proc := prepareTestCtx(t, true) eng := prepareTestEng(ctrl) - batchs, affectRows := prepareUpdateTestBatchs(proc.GetMPool(), 500, hasUniqueKey, hasSecondaryKey, isPartition) + batchs, _ := prepareUpdateTestBatchs(proc.GetMPool(), 220, hasUniqueKey, hasSecondaryKey, isPartition) multiUpdateCtxs := prepareTestUpdateMultiUpdateCtx(hasUniqueKey, hasSecondaryKey, isPartition) - toWriteS3 := true - retCase := buildTestCase(multiUpdateCtxs, eng, batchs, affectRows, toWriteS3) + action := UpdateWriteS3 + retCase := buildTestCase(multiUpdateCtxs, eng, batchs, 0, action) return proc, retCase } @@ -127,6 +127,9 @@ func prepareUpdateTestBatchs(mp *mpool.MPool, size int, hasUniqueKey bool, hasSe var bats = make([]*batch.Batch, size) affectRows := 0 partitionCount := 3 + mainObjectID := types.NewObjectid() + uniqueObjectID := types.NewObjectid() + secondaryObjectID := types.NewObjectid() for i := 0; i < size; i++ { rowCount := colexec.DefaultBatchSize if i == size-1 { @@ -138,7 +141,7 @@ func prepareUpdateTestBatchs(mp *mpool.MPool, size int, hasUniqueKey bool, hasSe columnB := testutil.NewStringVector(rowCount, types.T_varchar.ToType(), mp, false, nil) columnC := testutil.NewInt32Vector(rowCount, types.T_int32.ToType(), mp, false, nil) columnD := testutil.NewInt32Vector(rowCount, types.T_int32.ToType(), mp, false, nil) - columnRowID := testutil.NewRowidVector(rowCount, types.T_Rowid.ToType(), mp, false, nil) + columnRowID := makeTestRowIDVector(mp, mainObjectID, uint16(i), rowCount) attrs := []string{"a", "b", "c", "d", catalog.Row_ID} bat := &batch.Batch{ @@ -147,7 +150,7 @@ func prepareUpdateTestBatchs(mp *mpool.MPool, size int, hasUniqueKey bool, hasSe } if hasUniqueKey { - columnRowID := testutil.NewRowidVector(rowCount, types.T_Rowid.ToType(), mp, false, nil) + columnRowID := makeTestRowIDVector(mp, uniqueObjectID, uint16(i), rowCount) columnOldPk := testutil.NewStringVector(rowCount, types.T_varchar.ToType(), mp, true, nil) columnNewPk := testutil.NewStringVector(rowCount, types.T_varchar.ToType(), mp, true, nil) bat.Vecs = append(bat.Vecs, columnRowID) @@ -157,7 +160,7 @@ func prepareUpdateTestBatchs(mp *mpool.MPool, size int, hasUniqueKey bool, hasSe } if hasSecondaryKey { - columnRowID := testutil.NewRowidVector(rowCount, types.T_Rowid.ToType(), mp, false, nil) + columnRowID := makeTestRowIDVector(mp, secondaryObjectID, uint16(i), rowCount) columnOldPk := testutil.NewStringVector(rowCount, types.T_varchar.ToType(), mp, true, nil) columnNewPk := testutil.NewStringVector(rowCount, types.T_varchar.ToType(), mp, true, nil) bat.Vecs = append(bat.Vecs, columnRowID) @@ -180,95 +183,6 @@ func prepareUpdateTestBatchs(mp *mpool.MPool, size int, hasUniqueKey bool, hasSe return bats, uint64(affectRows) } -// func prepareUpdateTestBatchs(hasUniqueKey bool, hasSecondaryKey bool, isPartition bool) ([]*batch.Batch, uint64) { -// // create table t1(a big int primary key, b varchar(10) not null, c int, d int); -// // update t1 set c = 10 -// affectRows := 0 -// segmentID := objectio.NewSegmentid() - -// //old data -// blkId1 := objectio.NewBlockid(segmentID, 0, 1) -// bat1ColumnA := []int64{1, 2, 3} -// bat1ColumnB := []string{"1a", "2b", "3c"} -// bat1ColumnC := []int32{10, 10, 10} // set c = 10 -// bat1ColumnD := []int32{21, 22, 23} -// bat1ColumnRowID := []types.Rowid{*objectio.NewRowid(blkId1, 0), *objectio.NewRowid(blkId1, 1), *objectio.NewRowid(blkId1, 2)} - -// blkId2 := objectio.NewBlockid(segmentID, 0, 2) -// bat2ColumnA := []int64{4, 5, 6} -// bat2ColumnB := []string{"4a", "5b", "6c"} -// bat2ColumnC := []int32{10, 10, 10} // set c = 10 -// bat2ColumnD := []int32{24, 25, 26} -// bat2ColumnRowID := []types.Rowid{*objectio.NewRowid(blkId2, 0), *objectio.NewRowid(blkId2, 1), *objectio.NewRowid(blkId2, 2)} -// attrs := []string{"a", "b", "c", "d", "new_c", catalog.Row_ID} - -// bat1 := &batch.Batch{ -// Vecs: []*vector.Vector{ -// testutil.MakeInt64Vector(bat1ColumnA, nil), -// testutil.MakeVarcharVector(bat1ColumnB, nil), -// testutil.MakeInt32Vector(bat1ColumnC, nil), -// testutil.MakeInt32Vector(bat1ColumnD, nil), -// testutil.MakeRowIdVector(bat1ColumnRowID, nil), -// }, -// Attrs: attrs, -// Cnt: 1, -// } -// bat1.SetRowCount(bat1.Vecs[0].Length()) -// affectRows += bat1.RowCount() - -// bat2 := &batch.Batch{ -// Vecs: []*vector.Vector{ -// testutil.MakeInt64Vector(bat2ColumnA, nil), -// testutil.MakeVarcharVector(bat2ColumnB, nil), -// testutil.MakeInt32Vector(bat2ColumnC, nil), -// testutil.MakeInt32Vector(bat2ColumnD, nil), -// testutil.MakeRowIdVector(bat2ColumnRowID, nil), -// }, -// Attrs: attrs, -// Cnt: 1, -// } -// bat2.SetRowCount(bat2.Vecs[0].Length()) -// affectRows += bat2.RowCount() - -// if hasUniqueKey { -// blkId3 := objectio.NewBlockid(segmentID, 0, 3) -// bat1.Vecs = append(bat1.Vecs, testutil.MakeRowIdVector([]types.Rowid{*objectio.NewRowid(blkId3, 0), *objectio.NewRowid(blkId3, 1), *objectio.NewRowid(blkId3, 2)}, nil)) -// bat1.Vecs = append(bat1.Vecs, testutil.MakeVarcharVector([]string{"bat1_uk_1", "bat1_uk_2", "bat1_uk_3"}, nil)) -// bat1.Vecs = append(bat1.Vecs, testutil.MakeVarcharVector([]string{"bat1_uk_11", "bat1_uk_22", "bat1_uk_33"}, nil)) -// bat1.Attrs = append(bat1.Attrs, "bat1_uk_rowid", "bat1_uk_old_pk", "bat1_uk_new_pk") - -// blkId4 := objectio.NewBlockid(segmentID, 0, 4) -// bat2.Vecs = append(bat2.Vecs, testutil.MakeRowIdVector([]types.Rowid{*objectio.NewRowid(blkId4, 0), *objectio.NewRowid(blkId4, 1), *objectio.NewRowid(blkId4, 2)}, nil)) -// bat2.Vecs = append(bat2.Vecs, testutil.MakeVarcharVector([]string{"bat2_uk_1", "bat2_uk_2", "bat2_uk_3"}, nil)) -// bat2.Vecs = append(bat2.Vecs, testutil.MakeVarcharVector([]string{"bat2_uk_11", "bat2_uk_22", "bat2_uk_33"}, nil)) -// bat2.Attrs = append(bat2.Attrs, "bat2_uk_rowid", "bat2_uk_old_pk", "bat2_uk_new_pk") -// } - -// if hasSecondaryKey { -// blkId5 := objectio.NewBlockid(segmentID, 0, 5) -// bat1.Vecs = append(bat1.Vecs, testutil.MakeRowIdVector([]types.Rowid{*objectio.NewRowid(blkId5, 0), *objectio.NewRowid(blkId5, 1), *objectio.NewRowid(blkId5, 2)}, nil)) -// bat1.Vecs = append(bat1.Vecs, testutil.MakeVarcharVector([]string{"bat1_sk_1", "bat1_sk_2", "bat1_sk_3"}, nil)) -// bat1.Vecs = append(bat1.Vecs, testutil.MakeVarcharVector([]string{"bat1_sk_11", "bat1_sk_22", "bat1_sk_33"}, nil)) -// bat1.Attrs = append(bat1.Attrs, "bat1_sk_pk", "bat1_sk_old_pk", "bat1_sk_new_pk") - -// blkId6 := objectio.NewBlockid(segmentID, 0, 6) -// bat2.Vecs = append(bat2.Vecs, testutil.MakeRowIdVector([]types.Rowid{*objectio.NewRowid(blkId6, 0), *objectio.NewRowid(blkId6, 1), *objectio.NewRowid(blkId6, 2)}, nil)) -// bat2.Vecs = append(bat2.Vecs, testutil.MakeVarcharVector([]string{"bat2_sk_1", "bat2_sk_2", "bat2_sk_3"}, nil)) -// bat2.Vecs = append(bat2.Vecs, testutil.MakeVarcharVector([]string{"bat2_sk_11", "bat2_sk_22", "bat2_sk_33"}, nil)) -// bat2.Attrs = append(bat2.Attrs, "bat2_sk_pk", "bat1_sk_old_pk", "bat1_sk_new_pk") -// } - -// if isPartition { -// bat1.Vecs = append(bat1.Vecs, testutil.MakeInt32Vector([]int32{0, 1, 2}, nil)) -// bat1.Attrs = append(bat1.Attrs, "bat1_part_idx") - -// bat2.Vecs = append(bat2.Vecs, testutil.MakeInt32Vector([]int32{0, 1, 2}, nil)) -// bat2.Attrs = append(bat2.Attrs, "bat2_part_idx") -// } - -// return []*batch.Batch{bat1, bat2}, uint64(affectRows) -// } - func prepareTestUpdateMultiUpdateCtx(hasUniqueKey bool, hasSecondaryKey bool, isPartition bool) []*MultiUpdateCtx { // create table t1(a big int primary key, b varchar(10) not null, c int, d int); @@ -277,11 +191,10 @@ func prepareTestUpdateMultiUpdateCtx(hasUniqueKey bool, hasSecondaryKey bool, is // if only update main table, attrs = ["a","b","new_c","d","row_id"] updateCtx := &MultiUpdateCtx{ - ref: objRef, - tableDef: tableDef, - tableType: updateMainTable, - insertCols: []int{0, 1, 2, 3}, //a, b, new_c, d - deleteCols: []int{4, 0}, //row_id, a + ObjRef: objRef, + TableDef: tableDef, + InsertCols: []int{0, 1, 2, 3}, //a, b, new_c, d + DeleteCols: []int{4, 0}, //row_id, a } updateCtxs := []*MultiUpdateCtx{updateCtx} colCount := 5 @@ -303,11 +216,10 @@ func prepareTestUpdateMultiUpdateCtx(hasUniqueKey bool, hasSecondaryKey bool, is // if update main table with uk, attrs = ["a","b","new_c","d","row_id","uk_del_rowid","uk_del_pk","uk_new_pk"] updateCtxs = append(updateCtxs, &MultiUpdateCtx{ - ref: uniqueObjRef, - tableDef: uniqueTableDef, - tableType: updateUniqueIndexTable, - insertCols: []int{7, 0}, //uk_pk & main_tbl_pk - deleteCols: []int{5, 6}, //del_row_id & del_pk + ObjRef: uniqueObjRef, + TableDef: uniqueTableDef, + InsertCols: []int{7, 0}, //uk_pk & main_tbl_pk + DeleteCols: []int{5, 6}, //del_row_id & del_pk }) colCount += 3 } @@ -336,24 +248,22 @@ func prepareTestUpdateMultiUpdateCtx(hasUniqueKey bool, hasSecondaryKey bool, is } colCount += 3 updateCtxs = append(updateCtxs, &MultiUpdateCtx{ - ref: secondaryIdxObjRef, - tableDef: secondaryIdxTableDef, - tableType: updateSecondaryIndexTable, - insertCols: insertCols, - deleteCols: deleteCols, + ObjRef: secondaryIdxObjRef, + TableDef: secondaryIdxTableDef, + InsertCols: insertCols, + DeleteCols: deleteCols, }) } if isPartition { - for i, updateCtx := range updateCtxs { - partTblIDs := make([]int32, len(tableDef.Partition.PartitionTableNames)) - for j := range tableDef.Partition.PartitionTableNames { - partTblIDs[j] = int32(i*1000 + j) - } - updateCtx.partitionIdx = colCount - updateCtx.partitionTableIDs = partTblIDs - updateCtx.partitionTableNames = tableDef.Partition.PartitionTableNames + partTblIDs := make([]uint64, len(tableDef.Partition.PartitionTableNames)) + for j := range tableDef.Partition.PartitionTableNames { + partTblIDs[j] = uint64(1000 + j) } + updateCtxs[0].OldPartitionIdx = colCount + updateCtxs[0].NewPartitionIdx = colCount + updateCtxs[0].PartitionTableIDs = partTblIDs + updateCtxs[0].PartitionTableNames = tableDef.Partition.PartitionTableNames } return updateCtxs diff --git a/pkg/sql/colexec/multi_update/util_for_test.go b/pkg/sql/colexec/multi_update/util_for_test.go index 603f126d19a2..61306a55ceb4 100644 --- a/pkg/sql/colexec/multi_update/util_for_test.go +++ b/pkg/sql/colexec/multi_update/util_for_test.go @@ -22,8 +22,10 @@ import ( "github.com/golang/mock/gomock" "github.com/matrixorigin/matrixone/pkg/catalog" + "github.com/matrixorigin/matrixone/pkg/common/mpool" "github.com/matrixorigin/matrixone/pkg/container/batch" "github.com/matrixorigin/matrixone/pkg/container/types" + "github.com/matrixorigin/matrixone/pkg/container/vector" "github.com/matrixorigin/matrixone/pkg/defines" "github.com/matrixorigin/matrixone/pkg/fileservice" mock_frontend "github.com/matrixorigin/matrixone/pkg/frontend/test" @@ -57,17 +59,17 @@ func runTestCases(t *testing.T, proc *process.Process, tcs []*testCase) { var err error var res vm.CallResult - dupBatchs := func(bats []*batch.Batch) []*batch.Batch { - ret := make([]*batch.Batch, len(bats)) - for i, bat := range bats { - ret[i], _ = bat.Dup(proc.GetMPool()) - } - return ret - } + // dupBatchs := func(bats []*batch.Batch) []*batch.Batch { + // ret := make([]*batch.Batch, len(bats)) + // for i, bat := range bats { + // ret[i], _ = bat.Dup(proc.GetMPool()) + // } + // return ret + // } // logutil.Info("begin to run multi_update test") for _, tc := range tcs { - child := colexec.NewMockOperator().WithBatchs(dupBatchs(tc.inputBatchs)) + child := colexec.NewMockOperator().WithBatchs(tc.inputBatchs) tc.op.AppendChild(child) err = tc.op.Prepare(proc) require.NoError(t, err) @@ -92,12 +94,13 @@ func runTestCases(t *testing.T, proc *process.Process, tcs []*testCase) { continue } require.NoError(t, err) - require.Equal(t, tc.op.GetAffectedRows(), tc.affectedRows) + require.Equal(t, tc.affectedRows, tc.op.GetAffectedRows()) + child.ResetBatchs() tc.op.Children[0].Reset(proc, false, nil) tc.op.Reset(proc, false, nil) - child.ResetBatchs(proc, tc.inputBatchs) + child.WithBatchs(tc.inputBatchs) err = tc.op.Prepare(proc) require.NoError(t, err) for { @@ -312,13 +315,13 @@ func buildTestCase( eng engine.Engine, inputBats []*batch.Batch, affectRows uint64, - toWriteS3 bool) *testCase { + action UpdateAction) *testCase { retCase := &testCase{ op: &MultiUpdate{ ctr: container{}, MultiUpdateCtx: multiUpdateCtxs, - ToWriteS3: toWriteS3, + Action: action, IsOnduplicateKeyUpdate: false, Engine: eng, OperatorBase: vm.OperatorBase{ @@ -348,7 +351,7 @@ func makeTestPkArray(from int64, rowCount int) []int64 { func makeTestPartitionArray(rowCount int, partitionCount int) []int32 { val := make([]int32, rowCount) for i := 0; i < rowCount; i++ { - val[i] = int32(i / partitionCount) + val[i] = int32(i % partitionCount) } return val } @@ -360,3 +363,16 @@ func makeTestVarcharArray(rowCount int) []string { } return val } + +func makeTestRowIDVector(m *mpool.MPool, objectID *types.Objectid, blockNum uint16, rowCount int) *vector.Vector { + blockID := types.NewBlockidWithObjectID(objectID, blockNum+1000) + vec := vector.NewVec(types.T_Rowid.ToType()) + for i := 0; i < rowCount; i++ { + rowID := types.NewRowid(blockID, uint32(i)+1) + if err := vector.AppendFixed(vec, *rowID, false, m); err != nil { + vec.Free(m) + return nil + } + } + return vec +} diff --git a/pkg/sql/colexec/preinsert/preinsert.go b/pkg/sql/colexec/preinsert/preinsert.go index f986ab606ec2..a4278b8f0224 100644 --- a/pkg/sql/colexec/preinsert/preinsert.go +++ b/pkg/sql/colexec/preinsert/preinsert.go @@ -18,8 +18,6 @@ import ( "bytes" "time" - "go.uber.org/zap" - "github.com/matrixorigin/matrixone/pkg/catalog" "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/container/batch" @@ -30,6 +28,7 @@ import ( "github.com/matrixorigin/matrixone/pkg/vm" "github.com/matrixorigin/matrixone/pkg/vm/engine/disttae" "github.com/matrixorigin/matrixone/pkg/vm/process" + "go.uber.org/zap" ) const opName = "preinsert" diff --git a/pkg/sql/colexec/preinsert/types.go b/pkg/sql/colexec/preinsert/types.go index ffe2a5f2bd1e..acd661ee5fc9 100644 --- a/pkg/sql/colexec/preinsert/types.go +++ b/pkg/sql/colexec/preinsert/types.go @@ -17,10 +17,9 @@ package preinsert import ( "github.com/matrixorigin/matrixone/pkg/common/reuse" "github.com/matrixorigin/matrixone/pkg/container/batch" + pb "github.com/matrixorigin/matrixone/pkg/pb/plan" "github.com/matrixorigin/matrixone/pkg/sql/colexec" "github.com/matrixorigin/matrixone/pkg/sql/plan" - - pb "github.com/matrixorigin/matrixone/pkg/pb/plan" "github.com/matrixorigin/matrixone/pkg/vm" "github.com/matrixorigin/matrixone/pkg/vm/process" ) diff --git a/pkg/sql/colexec/right/join.go b/pkg/sql/colexec/right/join.go index c3763d9c86ec..6ed62f7ee213 100644 --- a/pkg/sql/colexec/right/join.go +++ b/pkg/sql/colexec/right/join.go @@ -100,7 +100,7 @@ func (rightJoin *RightJoin) Call(proc *process.Process) (vm.CallResult, error) { bat := result.Batch if bat == nil { - ctr.state = SendLast + ctr.state = Finalize continue } if bat.IsEmpty() { @@ -109,32 +109,34 @@ func (rightJoin *RightJoin) Call(proc *process.Process) (vm.CallResult, error) { if ctr.mp == nil { continue } - rightJoin.ctr.buf = bat - rightJoin.ctr.lastpos = 0 + ctr.buf = bat + ctr.lastPos = 0 } - startrow := rightJoin.ctr.lastpos + startRow := ctr.lastPos if err := ctr.probe(rightJoin, proc, analyzer, &result); err != nil { return result, err } - if rightJoin.ctr.lastpos == 0 { - rightJoin.ctr.buf = nil - } else if rightJoin.ctr.lastpos == startrow { + if ctr.lastPos == 0 { + ctr.buf = nil + } else if ctr.lastPos == startRow { return result, moerr.NewInternalErrorNoCtx("right join hanging") } analyzer.Output(result.Batch) return result, nil - case SendLast: - setNil, err := ctr.sendLast(rightJoin, proc, analyzer, &result) + case Finalize: + err := ctr.finalize(rightJoin, proc, &result) if err != nil { return result, err } ctr.state = End - if setNil { + if result.Batch == nil { continue } + + result.Status = vm.ExecNext analyzer.Output(result.Batch) return result, nil @@ -166,24 +168,27 @@ func (rightJoin *RightJoin) build(analyzer process.Analyzer, proc *process.Proce return nil } -func (ctr *container) sendLast(ap *RightJoin, proc *process.Process, analyzer process.Analyzer, result *vm.CallResult) (bool, error) { +func (ctr *container) finalize(ap *RightJoin, proc *process.Process, result *vm.CallResult) error { ctr.handledLast = true if ctr.matched == nil { - return true, nil + result.Batch = nil + return nil } if ap.NumCPU > 1 { if !ap.IsMerger { ap.Channel <- ctr.matched - return true, nil + result.Batch = nil + return nil } else { for cnt := 1; cnt < int(ap.NumCPU); cnt++ { v := colexec.ReceiveBitmapFromChannel(proc.Ctx, ap.Channel) if v != nil { ctr.matched.Or(v) } else { - return true, nil + result.Batch = nil + return nil } } close(ap.Channel) @@ -201,19 +206,19 @@ func (ctr *container) sendLast(ap *RightJoin, proc *process.Process, analyzer pr ap.resetRBat() if err := ctr.rbat.PreExtend(proc.Mp(), len(sels)); err != nil { - return false, err + return err } for i, rp := range ap.Result { if rp.Rel == 0 { if err := vector.AppendMultiFixed(ctr.rbat.Vecs[i], 0, true, int(count), proc.Mp()); err != nil { - return false, err + return err } } else { for _, sel := range sels { idx1, idx2 := sel/colexec.DefaultBatchSize, sel%colexec.DefaultBatchSize if err := ctr.rbat.Vecs[i].UnionOne(ctr.batches[idx1].Vecs[rp.Pos], int64(idx2), proc.Mp()); err != nil { - return false, err + return err } } } @@ -221,7 +226,7 @@ func (ctr *container) sendLast(ap *RightJoin, proc *process.Process, analyzer pr } ctr.rbat.AddRowCount(len(sels)) result.Batch = ctr.rbat - return false, nil + return nil } func (ctr *container) probe(ap *RightJoin, proc *process.Process, analyzer process.Analyzer, result *vm.CallResult) error { @@ -243,11 +248,11 @@ func (ctr *container) probe(ap *RightJoin, proc *process.Process, analyzer proce itr := ctr.itr rowCountIncrese := 0 - for i := ap.ctr.lastpos; i < count; i += hashmap.UnitLimit { + for i := ap.ctr.lastPos; i < count; i += hashmap.UnitLimit { if rowCountIncrese >= colexec.DefaultBatchSize { ctr.rbat.AddRowCount(rowCountIncrese) result.Batch = ctr.rbat - ap.ctr.lastpos = i + ap.ctr.lastPos = i return nil } n := count - i @@ -374,7 +379,7 @@ func (ctr *container) probe(ap *RightJoin, proc *process.Process, analyzer proce ctr.rbat.AddRowCount(rowCountIncrese) //anal.Output(ctr.rbat, isLast) result.Batch = ctr.rbat - ap.ctr.lastpos = 0 + ap.ctr.lastPos = 0 return nil } diff --git a/pkg/sql/colexec/right/types.go b/pkg/sql/colexec/right/types.go index e21f99b594ec..d447c526b85a 100644 --- a/pkg/sql/colexec/right/types.go +++ b/pkg/sql/colexec/right/types.go @@ -33,7 +33,7 @@ var _ vm.Operator = new(RightJoin) const ( Build = iota Probe - SendLast + Finalize End ) @@ -45,7 +45,7 @@ type evalVector struct { type container struct { state int itr hashmap.Iterator - lastpos int + lastPos int batches []*batch.Batch batchRowCount int64 @@ -135,7 +135,7 @@ func (rightJoin *RightJoin) Reset(proc *process.Process, pipelineFailed bool, er ctr.matched = nil ctr.handledLast = false ctr.state = Build - ctr.lastpos = 0 + ctr.lastPos = 0 if rightJoin.OpAnalyzer != nil { rightJoin.OpAnalyzer.Alloc(ctr.maxAllocSize) diff --git a/pkg/sql/colexec/rightanti/join.go b/pkg/sql/colexec/rightanti/join.go index 15d999eee77e..83939fe05950 100644 --- a/pkg/sql/colexec/rightanti/join.go +++ b/pkg/sql/colexec/rightanti/join.go @@ -117,28 +117,25 @@ func (rightAnti *RightAnti) Call(proc *process.Process) (vm.CallResult, error) { continue case SendLast: - if rightAnti.ctr.buf == nil { - rightAnti.ctr.lastpos = 0 - setNil, err := ctr.sendLast(rightAnti, proc, analyzer) + if ctr.buf == nil { + ctr.lastPos = 0 + err := ctr.finalize(rightAnti, proc) if err != nil { return result, err } - if setNil { - ctr.state = End - } + } + + if ctr.lastPos >= len(ctr.buf) { + ctr.state = End continue - } else { - if rightAnti.ctr.lastpos >= len(rightAnti.ctr.buf) { - ctr.state = End - continue - } - result.Batch = rightAnti.ctr.buf[rightAnti.ctr.lastpos] - rightAnti.ctr.lastpos++ - result.Status = vm.ExecHasMore - analyzer.Output(result.Batch) - return result, nil } + result.Batch = ctr.buf[ctr.lastPos] + ctr.lastPos++ + result.Status = vm.ExecHasMore + analyzer.Output(result.Batch) + return result, nil + default: result.Batch = nil result.Status = vm.ExecStop @@ -167,24 +164,24 @@ func (rightAnti *RightAnti) build(analyzer process.Analyzer, proc *process.Proce return nil } -func (ctr *container) sendLast(ap *RightAnti, proc *process.Process, analyzer process.Analyzer) (bool, error) { +func (ctr *container) finalize(ap *RightAnti, proc *process.Process) error { ctr.handledLast = true if ctr.matched == nil { - return true, nil + return nil } if ap.NumCPU > 1 { if !ap.IsMerger { ap.Channel <- ctr.matched - return true, nil + return nil } else { for cnt := 1; cnt < int(ap.NumCPU); cnt++ { v := colexec.ReceiveBitmapFromChannel(proc.Ctx, ap.Channel) if v != nil { ctr.matched.Or(v) } else { - return true, nil + return nil } } close(ap.Channel) @@ -211,19 +208,19 @@ func (ctr *container) sendLast(ap *RightAnti, proc *process.Process, analyzer pr } } if err := ctr.rbat.PreExtend(proc.Mp(), len(sels)); err != nil { - return false, err + return err } for j, pos := range ap.Result { for _, sel := range sels { idx1, idx2 := sel/colexec.DefaultBatchSize, sel%colexec.DefaultBatchSize if err := ctr.rbat.Vecs[j].UnionOne(ctr.batches[idx1].Vecs[pos], int64(idx2), proc.Mp()); err != nil { - return false, err + return err } } } ctr.rbat.AddRowCount(len(sels)) ctr.buf = []*batch.Batch{ctr.rbat} - return false, nil + return nil } else { n := (len(sels)-1)/colexec.DefaultBatchSize + 1 ctr.buf = make([]*batch.Batch, n) @@ -233,7 +230,7 @@ func (ctr *container) sendLast(ap *RightAnti, proc *process.Process, analyzer pr ctr.buf[k].Vecs[i] = vector.NewOffHeapVecWithType(ap.RightTypes[pos]) } if err := ctr.buf[k].PreExtend(proc.Mp(), colexec.DefaultBatchSize); err != nil { - return false, err + return err } var newsels []int32 if (k+1)*colexec.DefaultBatchSize <= len(sels) { @@ -245,13 +242,13 @@ func (ctr *container) sendLast(ap *RightAnti, proc *process.Process, analyzer pr for _, sel := range newsels { idx1, idx2 := sel/colexec.DefaultBatchSize, sel%colexec.DefaultBatchSize if err := ctr.buf[k].Vecs[j].UnionOne(ctr.batches[idx1].Vecs[pos], int64(idx2), proc.Mp()); err != nil { - return false, err + return err } } } ctr.buf[k].SetRowCount(len(newsels)) } - return false, nil + return nil } } diff --git a/pkg/sql/colexec/rightanti/types.go b/pkg/sql/colexec/rightanti/types.go index 27fb6a03e590..c0af45f5e525 100644 --- a/pkg/sql/colexec/rightanti/types.go +++ b/pkg/sql/colexec/rightanti/types.go @@ -45,7 +45,7 @@ type evalVector struct { type container struct { state int itr hashmap.Iterator - lastpos int + lastPos int batches []*batch.Batch batchRowCount int64 @@ -143,7 +143,7 @@ func (rightAnti *RightAnti) Reset(proc *process.Process, pipelineFailed bool, er ctr.matched = nil ctr.handledLast = false ctr.state = Build - ctr.lastpos = 0 + ctr.lastPos = 0 } func (rightAnti *RightAnti) Free(proc *process.Process, pipelineFailed bool, err error) { diff --git a/pkg/sql/colexec/rightsemi/join.go b/pkg/sql/colexec/rightsemi/join.go index d0d7edc28c73..1f3ee9fe0a83 100644 --- a/pkg/sql/colexec/rightsemi/join.go +++ b/pkg/sql/colexec/rightsemi/join.go @@ -99,7 +99,7 @@ func (rightSemi *RightSemi) Call(proc *process.Process) (vm.CallResult, error) { bat := result.Batch if bat == nil { - ctr.state = SendLast + ctr.state = Finalize continue } if bat.IsEmpty() { @@ -115,29 +115,26 @@ func (rightSemi *RightSemi) Call(proc *process.Process) (vm.CallResult, error) { } continue - case SendLast: - if rightSemi.ctr.buf == nil { - rightSemi.ctr.lastpos = 0 - setNil, err := ctr.sendLast(rightSemi, proc, analyzer) + case Finalize: + if ctr.buf == nil { + ctr.lastPos = 0 + err := ctr.finalize(rightSemi, proc) if err != nil { return result, err } - if setNil { - ctr.state = End - } + } + + if ctr.lastPos >= len(ctr.buf) { + ctr.state = End continue - } else { - if rightSemi.ctr.lastpos >= len(rightSemi.ctr.buf) { - ctr.state = End - continue - } - result.Batch = rightSemi.ctr.buf[rightSemi.ctr.lastpos] - rightSemi.ctr.lastpos++ - result.Status = vm.ExecHasMore - analyzer.Output(result.Batch) - return result, nil } + result.Batch = ctr.buf[ctr.lastPos] + ctr.lastPos++ + result.Status = vm.ExecHasMore + analyzer.Output(result.Batch) + return result, nil + default: result.Batch = nil result.Status = vm.ExecStop @@ -166,24 +163,24 @@ func (rightSemi *RightSemi) build(analyzer process.Analyzer, proc *process.Proce return nil } -func (ctr *container) sendLast(ap *RightSemi, proc *process.Process, analyzer process.Analyzer) (bool, error) { +func (ctr *container) finalize(ap *RightSemi, proc *process.Process) error { ctr.handledLast = true if ctr.matched == nil { - return true, nil + return nil } if ap.NumCPU > 1 { if !ap.IsMerger { ap.Channel <- ctr.matched - return true, nil + return nil } else { for cnt := 1; cnt < int(ap.NumCPU); cnt++ { v := colexec.ReceiveBitmapFromChannel(proc.Ctx, ap.Channel) if v != nil { ctr.matched.Or(v) } else { - return true, nil + return nil } } close(ap.Channel) @@ -209,20 +206,20 @@ func (ctr *container) sendLast(ap *RightSemi, proc *process.Process, analyzer pr } } if err := ctr.rbat.PreExtend(proc.Mp(), len(sels)); err != nil { - return false, err + return err } for j, pos := range ap.Result { for _, sel := range sels { idx1, idx2 := sel/colexec.DefaultBatchSize, sel%colexec.DefaultBatchSize if err := ctr.rbat.Vecs[j].UnionOne(ctr.batches[idx1].Vecs[pos], int64(idx2), proc.Mp()); err != nil { - return false, err + return err } } } ctr.rbat.AddRowCount(len(sels)) ctr.buf = []*batch.Batch{ctr.rbat} - return false, nil + return nil } else { n := (len(sels)-1)/colexec.DefaultBatchSize + 1 ctr.buf = make([]*batch.Batch, n) @@ -232,7 +229,7 @@ func (ctr *container) sendLast(ap *RightSemi, proc *process.Process, analyzer pr ctr.buf[k].Vecs[i] = vector.NewOffHeapVecWithType(ap.RightTypes[pos]) } if err := ctr.buf[k].PreExtend(proc.Mp(), colexec.DefaultBatchSize); err != nil { - return false, err + return err } var newsels []int32 if (k+1)*colexec.DefaultBatchSize <= len(sels) { @@ -244,13 +241,13 @@ func (ctr *container) sendLast(ap *RightSemi, proc *process.Process, analyzer pr for _, sel := range newsels { idx1, idx2 := sel/colexec.DefaultBatchSize, sel%colexec.DefaultBatchSize if err := ap.ctr.buf[k].Vecs[i].UnionOne(ctr.batches[idx1].Vecs[pos], int64(idx2), proc.Mp()); err != nil { - return false, err + return err } } } ctr.buf[k].SetRowCount(len(newsels)) } - return false, nil + return nil } } diff --git a/pkg/sql/colexec/rightsemi/types.go b/pkg/sql/colexec/rightsemi/types.go index 14188bc77fca..8c03fe25c027 100644 --- a/pkg/sql/colexec/rightsemi/types.go +++ b/pkg/sql/colexec/rightsemi/types.go @@ -33,7 +33,7 @@ var _ vm.Operator = new(RightSemi) const ( Build = iota Probe - SendLast + Finalize End ) @@ -45,7 +45,7 @@ type evalVector struct { type container struct { state int itr hashmap.Iterator - lastpos int + lastPos int batches []*batch.Batch batchRowCount int64 @@ -144,7 +144,7 @@ func (rightSemi *RightSemi) Reset(proc *process.Process, pipelineFailed bool, er ctr.matched = nil ctr.handledLast = false ctr.state = Build - ctr.lastpos = 0 + ctr.lastPos = 0 } func (rightSemi *RightSemi) Free(proc *process.Process, pipelineFailed bool, err error) { diff --git a/pkg/sql/colexec/shuffle/shuffle.go b/pkg/sql/colexec/shuffle/shuffle.go index 79c270c66f7b..391f0df8ff05 100644 --- a/pkg/sql/colexec/shuffle/shuffle.go +++ b/pkg/sql/colexec/shuffle/shuffle.go @@ -16,8 +16,7 @@ package shuffle import ( "bytes" - - "github.com/matrixorigin/matrixone/pkg/vm/message" + "fmt" "github.com/matrixorigin/matrixone/pkg/container/batch" "github.com/matrixorigin/matrixone/pkg/container/types" @@ -25,6 +24,7 @@ import ( "github.com/matrixorigin/matrixone/pkg/pb/plan" plan2 "github.com/matrixorigin/matrixone/pkg/sql/plan" "github.com/matrixorigin/matrixone/pkg/vm" + "github.com/matrixorigin/matrixone/pkg/vm/message" "github.com/matrixorigin/matrixone/pkg/vm/process" ) @@ -321,6 +321,18 @@ func getShuffledSelsByHashWithoutNull(ap *Shuffle, bat *batch.Batch) [][]int32 { regIndex := plan2.SimpleInt64HashToRange(uint64(v), bucketNum) sels[regIndex] = append(sels[regIndex], int32(row)) } + case types.T_decimal64: + groupByCol := vector.MustFixedColNoTypeCheck[types.Decimal64](groupByVec) + for row, v := range groupByCol { + regIndex := plan2.SimpleInt64HashToRange(uint64(v), bucketNum) + sels[regIndex] = append(sels[regIndex], int32(row)) + } + case types.T_decimal128: + groupByCol := vector.MustFixedColNoTypeCheck[types.Decimal128](groupByVec) + for row, v := range groupByCol { + regIndex := plan2.SimpleInt64HashToRange(uint64(v.B0_63^v.B64_127), bucketNum) + sels[regIndex] = append(sels[regIndex], int32(row)) + } case types.T_char, types.T_varchar, types.T_text: groupByCol, area := vector.MustVarlenaRawData(groupByVec) for row := range groupByCol { @@ -328,7 +340,7 @@ func getShuffledSelsByHashWithoutNull(ap *Shuffle, bat *batch.Batch) [][]int32 { sels[regIndex] = append(sels[regIndex], int32(row)) } default: - panic("unsupported shuffle type, wrong plan!") //something got wrong here! + panic(fmt.Sprintf("unsupported shuffle type %v, wrong plan!", groupByVec.GetType())) //something got wrong here! } return sels } diff --git a/pkg/sql/colexec/shufflebuild/build.go b/pkg/sql/colexec/shufflebuild/build.go index 5a95f9ce491c..c7a6846a786d 100644 --- a/pkg/sql/colexec/shufflebuild/build.go +++ b/pkg/sql/colexec/shufflebuild/build.go @@ -17,9 +17,8 @@ package shufflebuild import ( "bytes" - "github.com/matrixorigin/matrixone/pkg/vm/message" - "github.com/matrixorigin/matrixone/pkg/vm" + "github.com/matrixorigin/matrixone/pkg/vm/message" "github.com/matrixorigin/matrixone/pkg/vm/process" ) @@ -44,6 +43,10 @@ func (shuffleBuild *ShuffleBuild) Prepare(proc *process.Process) (err error) { if shuffleBuild.RuntimeFilterSpec == nil { panic("there must be runtime filter in shuffle build!") } + shuffleBuild.ctr.hashmapBuilder.IsDedup = shuffleBuild.IsDedup + shuffleBuild.ctr.hashmapBuilder.OnDuplicateAction = shuffleBuild.OnDuplicateAction + shuffleBuild.ctr.hashmapBuilder.DedupColName = shuffleBuild.DedupColName + shuffleBuild.ctr.hashmapBuilder.DedupColTypes = shuffleBuild.DedupColTypes return shuffleBuild.ctr.hashmapBuilder.Prepare(shuffleBuild.Conditions, proc) } diff --git a/pkg/sql/colexec/shufflebuild/types.go b/pkg/sql/colexec/shufflebuild/types.go index 03787169e680..1611425814a1 100644 --- a/pkg/sql/colexec/shufflebuild/types.go +++ b/pkg/sql/colexec/shufflebuild/types.go @@ -16,9 +16,8 @@ package shufflebuild import ( "github.com/matrixorigin/matrixone/pkg/common/reuse" - pbplan "github.com/matrixorigin/matrixone/pkg/pb/plan" + "github.com/matrixorigin/matrixone/pkg/pb/plan" "github.com/matrixorigin/matrixone/pkg/sql/colexec/hashmap_util" - "github.com/matrixorigin/matrixone/pkg/sql/plan" "github.com/matrixorigin/matrixone/pkg/vm" "github.com/matrixorigin/matrixone/pkg/vm/message" "github.com/matrixorigin/matrixone/pkg/vm/process" @@ -44,9 +43,15 @@ type ShuffleBuild struct { NeedBatches bool NeedAllocateSels bool Conditions []*plan.Expr - RuntimeFilterSpec *pbplan.RuntimeFilterSpec + RuntimeFilterSpec *plan.RuntimeFilterSpec JoinMapTag int32 ShuffleIdx int32 + + IsDedup bool + OnDuplicateAction plan.Node_OnDuplicateAction + DedupColName string + DedupColTypes []plan.Type + vm.OperatorBase } diff --git a/pkg/sql/compile/compile.go b/pkg/sql/compile/compile.go index b96c60deb2b4..f3a3743343dd 100644 --- a/pkg/sql/compile/compile.go +++ b/pkg/sql/compile/compile.go @@ -26,9 +26,6 @@ import ( "strings" "time" - "github.com/panjf2000/ants/v2" - "go.uber.org/zap" - "github.com/matrixorigin/matrixone/pkg/catalog" "github.com/matrixorigin/matrixone/pkg/cnservice/cnclient" "github.com/matrixorigin/matrixone/pkg/common/moerr" @@ -64,6 +61,7 @@ import ( "github.com/matrixorigin/matrixone/pkg/sql/colexec/mergedelete" "github.com/matrixorigin/matrixone/pkg/sql/colexec/mergerecursive" "github.com/matrixorigin/matrixone/pkg/sql/colexec/minus" + "github.com/matrixorigin/matrixone/pkg/sql/colexec/multi_update" "github.com/matrixorigin/matrixone/pkg/sql/colexec/output" "github.com/matrixorigin/matrixone/pkg/sql/colexec/sample" "github.com/matrixorigin/matrixone/pkg/sql/parsers/tree" @@ -84,6 +82,8 @@ import ( "github.com/matrixorigin/matrixone/pkg/vm/engine/disttae" "github.com/matrixorigin/matrixone/pkg/vm/message" "github.com/matrixorigin/matrixone/pkg/vm/process" + "github.com/panjf2000/ants/v2" + "go.uber.org/zap" ) // Note: Now the cost going from stat is actually the number of rows, so we can only estimate a number for the size of each row. @@ -342,11 +342,19 @@ func (c *Compile) run(s *Scope) error { } mergeArg := s.RootOp.(*mergedelete.MergeDelete) if mergeArg.AddAffectedRows { - c.addAffectedRows(mergeArg.AffectedRows()) + c.addAffectedRows(mergeArg.GetAffectedRows()) } return nil case Remote: err := s.RemoteRun(c) + //@FIXME not a good choice + if _, ok := s.RootOp.(*multi_update.MultiUpdate); ok { + if len(s.PreScopes) > 0 { + for _, ps := range s.PreScopes[0].PreScopes { + c.addAffectedRows(ps.affectedRows()) + } + } + } c.addAffectedRows(s.affectedRows()) return err case CreateDatabase: @@ -864,16 +872,15 @@ func (c *Compile) compileQuery(qry *plan.Query) ([]*Scope, error) { }() for i := len(qry.Steps) - 1; i >= 0; i-- { var scopes []*Scope - var scope *Scope scopes, err = c.compilePlanScope(int32(i), qry.Steps[i], qry.Nodes) if err != nil { return nil, err } - scope, err = c.compileSteps(qry, scopes, qry.Steps[i]) + scopes, err = c.compileSteps(qry, scopes, qry.Steps[i]) if err != nil { return nil, err } - steps = append(steps, scope) + steps = append(steps, scopes...) } return steps, err @@ -906,21 +913,15 @@ func (c *Compile) compileSinkScan(qry *plan.Query, nodeId int32) error { return nil } -func (c *Compile) compileSteps(qry *plan.Query, ss []*Scope, step int32) (*Scope, error) { +func (c *Compile) compileSteps(qry *plan.Query, ss []*Scope, step int32) ([]*Scope, error) { if qry.Nodes[step].NodeType == plan.Node_SINK { - return ss[0], nil + return ss, nil } switch qry.StmtType { - case plan.Query_DELETE: - updateScopesLastFlag(ss) - return ss[0], nil - case plan.Query_INSERT: + case plan.Query_DELETE, plan.Query_INSERT, plan.Query_UPDATE: updateScopesLastFlag(ss) - return ss[0], nil - case plan.Query_UPDATE: - updateScopesLastFlag(ss) - return ss[0], nil + return ss, nil default: var rs *Scope if c.IsSingleScope(ss) { @@ -936,7 +937,7 @@ func (c *Compile) compileSteps(qry *plan.Query, ss []*Scope, step int32) (*Scope WithFunc(c.fill). WithBlock(c.needBlock), ) - return rs, nil + return []*Scope{rs}, nil } } @@ -1242,6 +1243,18 @@ func (c *Compile) compilePlanScope(step int32, curNodeIdx int32, ns []*plan.Node n.NotCacheable = true c.setAnalyzeCurrent(ss, int(curNodeIdx)) return c.compileInsert(ns, n, ss) + case plan.Node_MULTI_UPDATE: + for _, updateCtx := range n.UpdateCtxList { + c.appendMetaTables(updateCtx.ObjRef) + } + ss, err = c.compilePlanScope(step, n.Children[0], ns) + if err != nil { + return nil, err + } + + n.NotCacheable = true + c.setAnalyzeCurrent(ss, int(curNodeIdx)) + return c.compileMultiUpdate(ns, n, ss) case plan.Node_LOCK_OP: ss, err = c.compilePlanScope(step, n.Children[0], ns) if err != nil { @@ -2267,8 +2280,8 @@ func (c *Compile) compileJoin(node, left, right *plan.Node, probeScopes, buildSc return c.compileShuffleJoin(node, left, right, probeScopes, buildScopes) } - rs := c.compileProbeSideForBoradcastJoin(node, left, right, probeScopes) - return c.compileBuildSideForBoradcastJoin(node, rs, buildScopes) + rs := c.compileProbeSideForBroadcastJoin(node, left, right, probeScopes) + return c.compileBuildSideForBroadcastJoin(node, rs, buildScopes) } func (c *Compile) compileShuffleJoin(node, left, right *plan.Node, lefts, rights []*Scope) []*Scope { @@ -2346,6 +2359,14 @@ func (c *Compile) compileShuffleJoin(node, left, right *plan.Node, lefts, rights op.SetAnalyzeControl(c.anal.curNodeIdx, currentFirstFlag) shuffleJoins[i].setRootOperator(op) } + case plan.Node_DEDUP: + for i := range shuffleJoins { + op := constructDedupJoin(node, rightTyps, c.proc) + op.ShuffleIdx = int32(i) + op.SetAnalyzeControl(c.anal.curNodeIdx, currentFirstFlag) + shuffleJoins[i].setRootOperator(op) + } + default: panic(moerr.NewNYI(c.proc.Ctx, fmt.Sprintf("shuffle join do not support join type '%v'", node.JoinType))) } @@ -2378,7 +2399,7 @@ func (c *Compile) newProbeScopeListForBroadcastJoin(probeScopes []*Scope, forceO return probeScopes } -func (c *Compile) compileProbeSideForBoradcastJoin(node, left, right *plan.Node, probeScopes []*Scope) []*Scope { +func (c *Compile) compileProbeSideForBroadcastJoin(node, left, right *plan.Node, probeScopes []*Scope) []*Scope { var rs []*Scope isEq := plan2.IsEquiJoin2(node.OnList) @@ -2546,6 +2567,15 @@ func (c *Compile) compileProbeSideForBoradcastJoin(node, left, right *plan.Node, } c.anal.isFirst = false } + case plan.Node_DEDUP: + rs = c.newProbeScopeListForBroadcastJoin(probeScopes, true) + currentFirstFlag := c.anal.isFirst + for i := range rs { + op := constructDedupJoin(node, rightTyps, c.proc) + op.SetAnalyzeControl(c.anal.curNodeIdx, currentFirstFlag) + rs[i].setRootOperator(op) + } + c.anal.isFirst = false case plan.Node_MARK: rs = c.newProbeScopeListForBroadcastJoin(probeScopes, false) currentFirstFlag := c.anal.isFirst @@ -2569,7 +2599,7 @@ func (c *Compile) compileProbeSideForBoradcastJoin(node, left, right *plan.Node, return rs } -func (c *Compile) compileBuildSideForBoradcastJoin(node *plan.Node, rs, buildScopes []*Scope) []*Scope { +func (c *Compile) compileBuildSideForBroadcastJoin(node *plan.Node, rs, buildScopes []*Scope) []*Scope { if !c.IsSingleScope(buildScopes) { // first merge scopes of build side, will optimize this in the future buildScopes = c.mergeShuffleScopesIfNeeded(buildScopes, false) buildScopes = []*Scope{c.newMergeScope(buildScopes)} @@ -3242,6 +3272,64 @@ func (c *Compile) compileInsert(ns []*plan.Node, n *plan.Node, ss []*Scope) ([]* return ss, nil } +func (c *Compile) compileMultiUpdate(ns []*plan.Node, n *plan.Node, ss []*Scope) ([]*Scope, error) { + // Determine whether to Write S3 + toWriteS3 := n.Stats.GetCost()*float64(SingleLineSizeEstimate) > + float64(DistributedThreshold) || c.anal.qry.LoadWriteS3 + + currentFirstFlag := c.anal.isFirst + if toWriteS3 { + if len(ss) == 1 && ss[0].NodeInfo.Mcpu == 1 { + mcpu := c.getParallelSizeForExternalScan(n, ncpu) + if mcpu > 1 { + oldScope := ss[0] + + ss = make([]*Scope, mcpu) + for i := 0; i < mcpu; i++ { + ss[i] = c.newEmptyMergeScope() + mergeArg := merge.NewArgument() + mergeArg.SetAnalyzeControl(c.anal.curNodeIdx, currentFirstFlag) + ss[i].setRootOperator(mergeArg) + ss[i].Proc = c.proc.NewNoContextChildProc(1) + ss[i].NodeInfo = engine.Node{Addr: oldScope.NodeInfo.Addr, Mcpu: 1} + } + _, dispatchOp := constructDispatchLocalAndRemote(0, ss, oldScope) + dispatchOp.FuncId = dispatch.SendToAnyLocalFunc + dispatchOp.SetAnalyzeControl(c.anal.curNodeIdx, false) + oldScope.setRootOperator(dispatchOp) + + ss[0].PreScopes = append(ss[0].PreScopes, oldScope) + } + } + + for i := range ss { + multiUpdateArg := constructMultiUpdate(n, c.e) + multiUpdateArg.Action = multi_update.UpdateWriteS3 + multiUpdateArg.SetAnalyzeControl(c.anal.curNodeIdx, currentFirstFlag) + ss[i].setRootOperator(multiUpdateArg) + } + + rs := ss[0] + if len(ss) > 1 || ss[0].NodeInfo.Mcpu > 1 { + rs = c.newMergeScope(ss) + } + + multiUpdateArg := constructMultiUpdate(n, c.e) + multiUpdateArg.Action = multi_update.UpdateFlushS3Info + rs.setRootOperator(multiUpdateArg) + ss = []*Scope{rs} + } else { + for i := range ss { + multiUpdateArg := constructMultiUpdate(n, c.e) + multiUpdateArg.Action = multi_update.UpdateWriteTable + multiUpdateArg.SetAnalyzeControl(c.anal.curNodeIdx, currentFirstFlag) + ss[i].setRootOperator(multiUpdateArg) + } + } + c.anal.isFirst = false + return ss, nil +} + func (c *Compile) compilePreInsertUk(n *plan.Node, ss []*Scope) []*Scope { currentFirstFlag := c.anal.isFirst for i := range ss { @@ -3664,7 +3752,8 @@ func (c *Compile) mergeScopesByCN(ss []*Scope) []*Scope { } func (c *Compile) newShuffleJoinScopeList(probeScopes, buildScopes []*Scope, n *plan.Node) []*Scope { - if len(c.cnList) <= 1 { + cnlist := c.cnList + if len(cnlist) <= 1 { n.Stats.HashmapStats.ShuffleTypeForMultiCN = plan.ShuffleTypeForMultiCN_Simple } @@ -3673,10 +3762,19 @@ func (c *Compile) newShuffleJoinScopeList(probeScopes, buildScopes []*Scope, n * probeScopes = c.mergeShuffleScopesIfNeeded(probeScopes, true) } buildScopes = c.mergeShuffleScopesIfNeeded(buildScopes, true) + if n.JoinType == plan.Node_DEDUP && len(cnlist) > 1 { + //merge build side to avoid bugs + if !c.IsSingleScope(probeScopes) { + probeScopes = []*Scope{c.newMergeScope(probeScopes)} + } + if !c.IsSingleScope(buildScopes) { + buildScopes = []*Scope{c.newMergeScope(buildScopes)} + } + } - dop := plan2.GetShuffleDop(ncpu, len(c.cnList), n.Stats.HashmapStats.HashmapSize) + dop := plan2.GetShuffleDop(ncpu, len(cnlist), n.Stats.HashmapStats.HashmapSize) - bucketNum := len(c.cnList) * dop + bucketNum := len(cnlist) * dop shuffleJoins := make([]*Scope, 0, bucketNum) shuffleBuilds := make([]*Scope, 0, bucketNum) @@ -3684,7 +3782,7 @@ func (c *Compile) newShuffleJoinScopeList(probeScopes, buildScopes []*Scope, n * lenRight := len(buildScopes) if !reuse { - for _, cn := range c.cnList { + for _, cn := range cnlist { probes := make([]*Scope, dop) builds := make([]*Scope, dop) for i := range probes { @@ -3732,7 +3830,7 @@ func (c *Compile) newShuffleJoinScopeList(probeScopes, buildScopes []*Scope, n * shuffleProbeOp.SetAnalyzeControl(c.anal.curNodeIdx, currentFirstFlag) probeScopes[i].setRootOperator(shuffleProbeOp) - if len(c.cnList) > 1 && probeScopes[i].NodeInfo.Mcpu > 1 { // merge here to avoid bugs, delete this in the future + if len(cnlist) > 1 && probeScopes[i].NodeInfo.Mcpu > 1 { // merge here to avoid bugs, delete this in the future probeScopes[i] = c.newMergeScopeByCN([]*Scope{probeScopes[i]}, probeScopes[i].NodeInfo) } @@ -3757,7 +3855,7 @@ func (c *Compile) newShuffleJoinScopeList(probeScopes, buildScopes []*Scope, n * shuffleBuildOp.SetAnalyzeControl(c.anal.curNodeIdx, currentFirstFlag) buildScopes[i].setRootOperator(shuffleBuildOp) - if len(c.cnList) > 1 && buildScopes[i].NodeInfo.Mcpu > 1 { // merge here to avoid bugs, delete this in the future + if len(cnlist) > 1 && buildScopes[i].NodeInfo.Mcpu > 1 { // merge here to avoid bugs, delete this in the future buildScopes[i] = c.newMergeScopeByCN([]*Scope{buildScopes[i]}, buildScopes[i].NodeInfo) } @@ -4724,9 +4822,9 @@ func (s *Scope) affectedRows() uint64 { for op != nil { if arg, ok := op.(vm.ModificationArgument); ok { if marg, ok := arg.(*mergeblock.MergeBlock); ok { - return marg.AffectedRows() + return marg.GetAffectedRows() } - affectedRows += arg.AffectedRows() + affectedRows += arg.GetAffectedRows() } if op.GetOperatorBase().NumChildren() == 0 { op = nil diff --git a/pkg/sql/compile/debugTools.go b/pkg/sql/compile/debugTools.go index 12181614f4ea..6742c0ca98bf 100644 --- a/pkg/sql/compile/debugTools.go +++ b/pkg/sql/compile/debugTools.go @@ -85,7 +85,9 @@ var debugInstructionNames = map[vm.OpType]string{ vm.ValueScan: "valuescan", vm.TableFunction: "tablefunction", vm.OnDuplicateKey: "on duplicate key", + vm.DedupJoin: "dedup join", vm.Apply: "apply", + vm.MultiUpdate: "multi update", vm.PostDml: "postdml", } diff --git a/pkg/sql/compile/fuzzyCheck.go b/pkg/sql/compile/fuzzyCheck.go index 7bee36ee4a58..82741d48fbc5 100644 --- a/pkg/sql/compile/fuzzyCheck.go +++ b/pkg/sql/compile/fuzzyCheck.go @@ -23,10 +23,9 @@ import ( "strings" "time" - "github.com/matrixorigin/matrixone/pkg/common/reuse" - "github.com/matrixorigin/matrixone/pkg/catalog" "github.com/matrixorigin/matrixone/pkg/common/moerr" + "github.com/matrixorigin/matrixone/pkg/common/reuse" "github.com/matrixorigin/matrixone/pkg/container/batch" "github.com/matrixorigin/matrixone/pkg/container/nulls" "github.com/matrixorigin/matrixone/pkg/container/types" diff --git a/pkg/sql/compile/operator.go b/pkg/sql/compile/operator.go index 2d392fabcf92..4be61d224520 100644 --- a/pkg/sql/compile/operator.go +++ b/pkg/sql/compile/operator.go @@ -18,28 +18,9 @@ import ( "context" "fmt" - "github.com/matrixorigin/matrixone/pkg/common/bitmap" - - "github.com/matrixorigin/matrixone/pkg/vm/message" - - "github.com/matrixorigin/matrixone/pkg/sql/colexec/apply" - "github.com/matrixorigin/matrixone/pkg/sql/colexec/mergeblock" - "github.com/matrixorigin/matrixone/pkg/sql/colexec/postdml" - "github.com/matrixorigin/matrixone/pkg/sql/colexec/productl2" - "github.com/matrixorigin/matrixone/pkg/sql/colexec/table_scan" - "github.com/matrixorigin/matrixone/pkg/sql/colexec/unionall" - "github.com/matrixorigin/matrixone/pkg/sql/colexec/value_scan" - - "github.com/matrixorigin/matrixone/pkg/sql/colexec/shufflebuild" - - "github.com/matrixorigin/matrixone/pkg/sql/colexec/aggexec" - - "github.com/matrixorigin/matrixone/pkg/sql/colexec/indexbuild" - "github.com/matrixorigin/matrixone/pkg/sql/colexec/indexjoin" - "github.com/google/uuid" - "github.com/matrixorigin/matrixone/pkg/catalog" + "github.com/matrixorigin/matrixone/pkg/common/bitmap" "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/container/batch" "github.com/matrixorigin/matrixone/pkg/container/types" @@ -48,8 +29,11 @@ import ( "github.com/matrixorigin/matrixone/pkg/pb/plan" "github.com/matrixorigin/matrixone/pkg/pb/timestamp" "github.com/matrixorigin/matrixone/pkg/sql/colexec" + "github.com/matrixorigin/matrixone/pkg/sql/colexec/aggexec" "github.com/matrixorigin/matrixone/pkg/sql/colexec/anti" + "github.com/matrixorigin/matrixone/pkg/sql/colexec/apply" "github.com/matrixorigin/matrixone/pkg/sql/colexec/connector" + "github.com/matrixorigin/matrixone/pkg/sql/colexec/dedupjoin" "github.com/matrixorigin/matrixone/pkg/sql/colexec/deletion" "github.com/matrixorigin/matrixone/pkg/sql/colexec/dispatch" "github.com/matrixorigin/matrixone/pkg/sql/colexec/external" @@ -58,6 +42,8 @@ import ( "github.com/matrixorigin/matrixone/pkg/sql/colexec/fuzzyfilter" "github.com/matrixorigin/matrixone/pkg/sql/colexec/group" "github.com/matrixorigin/matrixone/pkg/sql/colexec/hashbuild" + "github.com/matrixorigin/matrixone/pkg/sql/colexec/indexbuild" + "github.com/matrixorigin/matrixone/pkg/sql/colexec/indexjoin" "github.com/matrixorigin/matrixone/pkg/sql/colexec/insert" "github.com/matrixorigin/matrixone/pkg/sql/colexec/intersect" "github.com/matrixorigin/matrixone/pkg/sql/colexec/intersectall" @@ -67,20 +53,24 @@ import ( "github.com/matrixorigin/matrixone/pkg/sql/colexec/lockop" "github.com/matrixorigin/matrixone/pkg/sql/colexec/loopjoin" "github.com/matrixorigin/matrixone/pkg/sql/colexec/merge" + "github.com/matrixorigin/matrixone/pkg/sql/colexec/mergeblock" "github.com/matrixorigin/matrixone/pkg/sql/colexec/mergecte" "github.com/matrixorigin/matrixone/pkg/sql/colexec/mergegroup" "github.com/matrixorigin/matrixone/pkg/sql/colexec/mergeorder" "github.com/matrixorigin/matrixone/pkg/sql/colexec/mergerecursive" "github.com/matrixorigin/matrixone/pkg/sql/colexec/mergetop" "github.com/matrixorigin/matrixone/pkg/sql/colexec/minus" + "github.com/matrixorigin/matrixone/pkg/sql/colexec/multi_update" "github.com/matrixorigin/matrixone/pkg/sql/colexec/offset" "github.com/matrixorigin/matrixone/pkg/sql/colexec/onduplicatekey" "github.com/matrixorigin/matrixone/pkg/sql/colexec/order" "github.com/matrixorigin/matrixone/pkg/sql/colexec/partition" + "github.com/matrixorigin/matrixone/pkg/sql/colexec/postdml" "github.com/matrixorigin/matrixone/pkg/sql/colexec/preinsert" "github.com/matrixorigin/matrixone/pkg/sql/colexec/preinsertsecondaryindex" "github.com/matrixorigin/matrixone/pkg/sql/colexec/preinsertunique" "github.com/matrixorigin/matrixone/pkg/sql/colexec/product" + "github.com/matrixorigin/matrixone/pkg/sql/colexec/productl2" "github.com/matrixorigin/matrixone/pkg/sql/colexec/projection" "github.com/matrixorigin/matrixone/pkg/sql/colexec/right" "github.com/matrixorigin/matrixone/pkg/sql/colexec/rightanti" @@ -88,17 +78,22 @@ import ( "github.com/matrixorigin/matrixone/pkg/sql/colexec/sample" "github.com/matrixorigin/matrixone/pkg/sql/colexec/semi" "github.com/matrixorigin/matrixone/pkg/sql/colexec/shuffle" + "github.com/matrixorigin/matrixone/pkg/sql/colexec/shufflebuild" "github.com/matrixorigin/matrixone/pkg/sql/colexec/single" "github.com/matrixorigin/matrixone/pkg/sql/colexec/source" "github.com/matrixorigin/matrixone/pkg/sql/colexec/table_function" + "github.com/matrixorigin/matrixone/pkg/sql/colexec/table_scan" "github.com/matrixorigin/matrixone/pkg/sql/colexec/timewin" "github.com/matrixorigin/matrixone/pkg/sql/colexec/top" + "github.com/matrixorigin/matrixone/pkg/sql/colexec/unionall" + "github.com/matrixorigin/matrixone/pkg/sql/colexec/value_scan" "github.com/matrixorigin/matrixone/pkg/sql/colexec/window" "github.com/matrixorigin/matrixone/pkg/sql/parsers/tree" plan2 "github.com/matrixorigin/matrixone/pkg/sql/plan" "github.com/matrixorigin/matrixone/pkg/sql/plan/function" "github.com/matrixorigin/matrixone/pkg/vm" "github.com/matrixorigin/matrixone/pkg/vm/engine" + "github.com/matrixorigin/matrixone/pkg/vm/message" "github.com/matrixorigin/matrixone/pkg/vm/process" ) @@ -545,6 +540,36 @@ func dupOperator(sourceOp vm.Operator, index int, maxParallel int) vm.Operator { op.TableFunction.SetInfo(&info) op.SetInfo(&info) return op + case vm.MultiUpdate: + t := sourceOp.(*multi_update.MultiUpdate) + op := multi_update.NewArgument() + op.MultiUpdateCtx = t.MultiUpdateCtx + op.Action = t.Action + op.IsOnduplicateKeyUpdate = t.IsOnduplicateKeyUpdate + op.Engine = t.Engine + op.SegmentMap = t.SegmentMap + op.SetInfo(&info) + return op + case vm.DedupJoin: + t := sourceOp.(*dedupjoin.DedupJoin) + op := dedupjoin.NewArgument() + if t.Channel == nil { + t.Channel = make(chan *bitmap.Bitmap, maxParallel) + } + op.Channel = t.Channel + op.NumCPU = uint64(maxParallel) + op.IsMerger = (index == 0) + op.Result = append(op.Result, t.Result...) + op.RightTypes = append(op.RightTypes, t.RightTypes...) + op.Conditions = append(op.Conditions, t.Conditions...) + op.IsShuffle = t.IsShuffle + op.ShuffleIdx = t.ShuffleIdx + op.RuntimeFilterSpecs = append(op.RuntimeFilterSpecs, t.RuntimeFilterSpecs...) + op.JoinMapTag = t.JoinMapTag + op.OnDuplicateAction = t.OnDuplicateAction + op.DedupColName = t.DedupColName + op.DedupColTypes = t.DedupColTypes + return op case vm.PostDml: t := sourceOp.(*postdml.PostDml) op := postdml.NewArgument() @@ -555,7 +580,7 @@ func dupOperator(sourceOp vm.Operator, index int, maxParallel int) vm.Operator { panic(fmt.Sprintf("unexpected instruction type '%d' to dup", sourceOp.OpType())) } -func constructRestrict(n *plan.Node, filterExpr *plan2.Expr) *filter.Filter { +func constructRestrict(n *plan.Node, filterExpr *plan.Expr) *filter.Filter { op := filter.NewArgument() op.E = filterExpr op.IsEnd = n.IsEnd @@ -743,6 +768,38 @@ func constructLockOp(n *plan.Node, eng engine.Engine) (*lockop.LockOp, error) { return arg, nil } +func constructMultiUpdate(n *plan.Node, eg engine.Engine) *multi_update.MultiUpdate { + arg := multi_update.NewArgument() + arg.Engine = eg + arg.SegmentMap = colexec.Get().GetCnSegmentMap() + + arg.MultiUpdateCtx = make([]*multi_update.MultiUpdateCtx, len(n.UpdateCtxList)) + for i, updateCtx := range n.UpdateCtxList { + insertCols := make([]int, len(updateCtx.InsertCols)) + for j, col := range updateCtx.InsertCols { + insertCols[j] = int(col.ColPos) + } + + deleteCols := make([]int, len(updateCtx.DeleteCols)) + for j, col := range updateCtx.DeleteCols { + deleteCols[j] = int(col.ColPos) + } + + arg.MultiUpdateCtx[i] = &multi_update.MultiUpdateCtx{ + ObjRef: updateCtx.ObjRef, + TableDef: updateCtx.TableDef, + InsertCols: insertCols, + DeleteCols: deleteCols, + PartitionTableIDs: updateCtx.PartitionTableIds, + PartitionTableNames: updateCtx.PartitionTableNames, + OldPartitionIdx: int(updateCtx.OldPartitionIdx), + NewPartitionIdx: int(updateCtx.NewPartitionIdx), + } + } + + return arg +} + func constructInsert(n *plan.Node, eg engine.Engine) *insert.Insert { oldCtx := n.InsertCtx var attrs []string @@ -1018,6 +1075,35 @@ func constructSingle(n *plan.Node, typs []types.Type, proc *process.Process) *si return arg } +func constructDedupJoin(n *plan.Node, right_typs []types.Type, proc *process.Process) *dedupjoin.DedupJoin { + result := make([]int32, len(n.ProjectList)) + for i, expr := range n.ProjectList { + _, result[i] = constructJoinResult(expr, proc) + } + cond, conds := extraJoinConditions(n.OnList) + if cond != nil { + panic("dedupjoin should not have non-equi join condition") + } + arg := dedupjoin.NewArgument() + arg.RightTypes = right_typs + arg.Result = result + arg.Conditions = constructJoinConditions(conds, proc) + arg.RuntimeFilterSpecs = n.RuntimeFilterBuildList + arg.OnDuplicateAction = n.OnDuplicateAction + arg.DedupColName = n.DedupColName + arg.DedupColTypes = n.DedupColTypes + arg.IsShuffle = n.Stats.HashmapStats != nil && n.Stats.HashmapStats.Shuffle + for i := range n.SendMsgList { + if n.SendMsgList[i].MsgType == int32(message.MsgJoinMap) { + arg.JoinMapTag = n.SendMsgList[i].MsgTag + } + } + if arg.JoinMapTag <= 0 { + panic("wrong joinmap tag!") + } + return arg +} + func constructProduct(n *plan.Node, typs []types.Type, proc *process.Process) *product.Product { result := make([]colexec.ResultPos, len(n.ProjectList)) for i, expr := range n.ProjectList { @@ -1687,6 +1773,21 @@ func constructHashBuild(op vm.Operator, proc *process.Process, mcpu int32) *hash ret.NeedBatches = true ret.NeedAllocateSels = true ret.JoinMapTag = arg.JoinMapTag + + case vm.DedupJoin: + arg := op.(*dedupjoin.DedupJoin) + ret.NeedHashMap = true + ret.Conditions = arg.Conditions[1] + ret.NeedBatches = true + ret.IsDedup = true + ret.OnDuplicateAction = arg.OnDuplicateAction + ret.DedupColName = arg.DedupColName + ret.DedupColTypes = arg.DedupColTypes + if len(arg.RuntimeFilterSpecs) > 0 { + ret.RuntimeFilterSpec = arg.RuntimeFilterSpecs[0] + } + ret.JoinMapTag = arg.JoinMapTag + default: ret.Release() panic(moerr.NewInternalErrorf(proc.Ctx, "unsupport join type '%v'", op.OpType())) @@ -1805,6 +1906,21 @@ func constructShuffleBuild(op vm.Operator, proc *process.Process) *shufflebuild. ret.JoinMapTag = arg.JoinMapTag ret.ShuffleIdx = arg.ShuffleIdx + case vm.DedupJoin: + arg := op.(*dedupjoin.DedupJoin) + ret.Conditions = arg.Conditions[1] + ret.NeedBatches = true + ret.NeedBatches = true + ret.IsDedup = true + ret.OnDuplicateAction = arg.OnDuplicateAction + ret.DedupColName = arg.DedupColName + ret.DedupColTypes = arg.DedupColTypes + if len(arg.RuntimeFilterSpecs) > 0 { + ret.RuntimeFilterSpec = plan2.DeepCopyRuntimeFilterSpec(arg.RuntimeFilterSpecs[0]) + } + ret.JoinMapTag = arg.JoinMapTag + ret.ShuffleIdx = arg.ShuffleIdx + default: ret.Release() panic(moerr.NewInternalErrorf(proc.Ctx, "unsupported type for shuffle join: '%v'", op.OpType())) diff --git a/pkg/sql/compile/remoterun.go b/pkg/sql/compile/remoterun.go index 59882a84d687..1bc370a8ac62 100644 --- a/pkg/sql/compile/remoterun.go +++ b/pkg/sql/compile/remoterun.go @@ -18,8 +18,7 @@ import ( "fmt" "unsafe" - "github.com/matrixorigin/matrixone/pkg/vm/engine/engine_util" - + "github.com/google/uuid" "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/common/mpool" "github.com/matrixorigin/matrixone/pkg/container/batch" @@ -31,6 +30,7 @@ import ( "github.com/matrixorigin/matrixone/pkg/sql/colexec/anti" "github.com/matrixorigin/matrixone/pkg/sql/colexec/apply" "github.com/matrixorigin/matrixone/pkg/sql/colexec/connector" + "github.com/matrixorigin/matrixone/pkg/sql/colexec/dedupjoin" "github.com/matrixorigin/matrixone/pkg/sql/colexec/deletion" "github.com/matrixorigin/matrixone/pkg/sql/colexec/dispatch" "github.com/matrixorigin/matrixone/pkg/sql/colexec/external" @@ -54,6 +54,7 @@ import ( "github.com/matrixorigin/matrixone/pkg/sql/colexec/mergerecursive" "github.com/matrixorigin/matrixone/pkg/sql/colexec/mergetop" "github.com/matrixorigin/matrixone/pkg/sql/colexec/minus" + "github.com/matrixorigin/matrixone/pkg/sql/colexec/multi_update" "github.com/matrixorigin/matrixone/pkg/sql/colexec/offset" "github.com/matrixorigin/matrixone/pkg/sql/colexec/onduplicatekey" "github.com/matrixorigin/matrixone/pkg/sql/colexec/order" @@ -82,9 +83,8 @@ import ( plan2 "github.com/matrixorigin/matrixone/pkg/sql/plan" "github.com/matrixorigin/matrixone/pkg/vm" "github.com/matrixorigin/matrixone/pkg/vm/engine" + "github.com/matrixorigin/matrixone/pkg/vm/engine/engine_util" "github.com/matrixorigin/matrixone/pkg/vm/process" - - "github.com/google/uuid" ) // encodeScope generate a pipeline.Pipeline from Scope, encode pipeline, and returns. @@ -415,7 +415,7 @@ func convertToPipelineInstruction(op vm.Operator, proc *process.Process, ctx *sc } case *deletion.Deletion: in.Delete = &pipeline.Deletion{ - AffectedRows: t.AffectedRows(), + AffectedRows: t.GetAffectedRows(), RemoteDelete: t.RemoteDelete, SegmentMap: t.SegmentMap, IBucket: t.IBucket, @@ -777,6 +777,10 @@ func convertToPipelineInstruction(op vm.Operator, proc *process.Process, ctx *sc JoinMapTag: t.JoinMapTag, JoinMapRefCnt: t.JoinMapRefCnt, RuntimeFilterSpec: t.RuntimeFilterSpec, + IsDedup: t.IsDedup, + OnDuplicateAction: t.OnDuplicateAction, + DedupColName: t.DedupColName, + DedupColTypes: t.DedupColTypes, } case *shufflebuild.ShuffleBuild: in.ShuffleBuild = &pipeline.Shufflebuild{ @@ -787,11 +791,27 @@ func convertToPipelineInstruction(op vm.Operator, proc *process.Process, ctx *sc RuntimeFilterSpec: t.RuntimeFilterSpec, JoinMapTag: t.JoinMapTag, ShuffleIdx: t.ShuffleIdx, + IsDedup: t.IsDedup, + OnDuplicateAction: t.OnDuplicateAction, + DedupColName: t.DedupColName, + DedupColTypes: t.DedupColTypes, } case *indexbuild.IndexBuild: in.IndexBuild = &pipeline.Indexbuild{ RuntimeFilterSpec: t.RuntimeFilterSpec, } + case *dedupjoin.DedupJoin: + in.DedupJoin = &pipeline.DedupJoin{ + LeftCond: t.Conditions[0], + RightCond: t.Conditions[1], + RuntimeFilterBuildList: t.RuntimeFilterSpecs, + IsShuffle: t.IsShuffle, + JoinMapTag: t.JoinMapTag, + ShuffleIdx: t.ShuffleIdx, + OnDuplicateAction: t.OnDuplicateAction, + DedupColName: t.DedupColName, + DedupColTypes: t.DedupColTypes, + } case *apply.Apply: relList, colList := getRelColList(t.Result) in.Apply = &pipeline.Apply{ @@ -808,6 +828,34 @@ func convertToPipelineInstruction(op vm.Operator, proc *process.Process, ctx *sc Params: t.TableFunction.Params, Name: t.TableFunction.FuncName, } + case *multi_update.MultiUpdate: + updateCtxList := make([]*plan.UpdateCtx, len(t.MultiUpdateCtx)) + for i, muCtx := range t.MultiUpdateCtx { + updateCtxList[i] = &plan.UpdateCtx{ + ObjRef: muCtx.ObjRef, + TableDef: muCtx.TableDef, + PartitionTableIds: muCtx.PartitionTableIDs, + PartitionTableNames: muCtx.PartitionTableNames, + OldPartitionIdx: int32(muCtx.OldPartitionIdx), + NewPartitionIdx: int32(muCtx.NewPartitionIdx), + } + + updateCtxList[i].InsertCols = make([]plan.ColRef, len(muCtx.InsertCols)) + for j, pos := range muCtx.InsertCols { + updateCtxList[i].InsertCols[j].ColPos = int32(pos) + } + + updateCtxList[i].DeleteCols = make([]plan.ColRef, len(muCtx.DeleteCols)) + for j, pos := range muCtx.DeleteCols { + updateCtxList[i].DeleteCols[j].ColPos = int32(pos) + } + } + in.MultiUpdate = &pipeline.MultiUpdate{ + AffectedRows: t.GetAffectedRows(), + Action: uint32(t.Action), + UpdateCtxList: updateCtxList, + SegmentMap: t.SegmentMap, + } case *postdml.PostDml: in.PostDml = &pipeline.PostDml{ AddAffectedRows: t.PostDmlCtx.AddAffectedRows, @@ -1255,6 +1303,10 @@ func convertToVmOperator(opr *pipeline.Instruction, ctx *scopeContext, eng engin arg.JoinMapTag = t.JoinMapTag arg.JoinMapRefCnt = t.JoinMapRefCnt arg.RuntimeFilterSpec = t.RuntimeFilterSpec + arg.IsDedup = t.IsDedup + arg.OnDuplicateAction = t.OnDuplicateAction + arg.DedupColName = t.DedupColName + arg.DedupColTypes = t.DedupColTypes op = arg case vm.ShuffleBuild: arg := shufflebuild.NewArgument() @@ -1266,11 +1318,27 @@ func convertToVmOperator(opr *pipeline.Instruction, ctx *scopeContext, eng engin arg.RuntimeFilterSpec = t.RuntimeFilterSpec arg.JoinMapTag = t.JoinMapTag arg.ShuffleIdx = t.ShuffleIdx + arg.IsDedup = t.IsDedup + arg.OnDuplicateAction = t.OnDuplicateAction + arg.DedupColName = t.DedupColName + arg.DedupColTypes = t.DedupColTypes op = arg case vm.IndexBuild: arg := indexbuild.NewArgument() arg.RuntimeFilterSpec = opr.GetIndexBuild().RuntimeFilterSpec op = arg + case vm.DedupJoin: + arg := dedupjoin.NewArgument() + t := opr.GetDedupJoin() + arg.Conditions = [][]*plan.Expr{t.LeftCond, t.RightCond} + arg.RuntimeFilterSpecs = t.RuntimeFilterBuildList + arg.IsShuffle = t.IsShuffle + arg.JoinMapTag = t.JoinMapTag + arg.ShuffleIdx = t.ShuffleIdx + arg.OnDuplicateAction = t.OnDuplicateAction + arg.DedupColName = t.DedupColName + arg.DedupColTypes = t.DedupColTypes + op = arg case vm.Apply: arg := apply.NewArgument() t := opr.GetApply() @@ -1285,6 +1353,38 @@ func convertToVmOperator(opr *pipeline.Instruction, ctx *scopeContext, eng engin arg.TableFunction.FuncName = opr.TableFunction.Name arg.TableFunction.Params = opr.TableFunction.Params op = arg + case vm.MultiUpdate: + arg := multi_update.NewArgument() + t := opr.GetMultiUpdate() + arg.SetAffectedRows(t.AffectedRows) + arg.Action = multi_update.UpdateAction(t.Action) + arg.SegmentMap = t.SegmentMap + + arg.MultiUpdateCtx = make([]*multi_update.MultiUpdateCtx, len(t.UpdateCtxList)) + for i, muCtx := range t.UpdateCtxList { + + arg.MultiUpdateCtx[i] = &multi_update.MultiUpdateCtx{ + ObjRef: muCtx.ObjRef, + TableDef: muCtx.TableDef, + PartitionTableIDs: muCtx.PartitionTableIds, + PartitionTableNames: muCtx.PartitionTableNames, + OldPartitionIdx: int(muCtx.OldPartitionIdx), + NewPartitionIdx: int(muCtx.NewPartitionIdx), + } + + arg.MultiUpdateCtx[i].InsertCols = make([]int, len(muCtx.InsertCols)) + for j, pos := range muCtx.InsertCols { + arg.MultiUpdateCtx[i].InsertCols[j] = int(pos.ColPos) + } + + arg.MultiUpdateCtx[i].DeleteCols = make([]int, len(muCtx.DeleteCols)) + for j, pos := range muCtx.DeleteCols { + arg.MultiUpdateCtx[i].DeleteCols[j] = int(pos.ColPos) + } + } + + op = arg + case vm.PostDml: t := opr.GetPostDml() arg := postdml.NewArgument() diff --git a/pkg/sql/compile/remoterunClient.go b/pkg/sql/compile/remoterunClient.go index bad0cfa361ee..00099f23b971 100644 --- a/pkg/sql/compile/remoterunClient.go +++ b/pkg/sql/compile/remoterunClient.go @@ -20,10 +20,6 @@ import ( "fmt" "time" - "github.com/matrixorigin/matrixone/pkg/vm/process" - - "go.uber.org/zap" - "github.com/matrixorigin/matrixone/pkg/cnservice/cnclient" "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/common/morpc" @@ -36,6 +32,8 @@ import ( "github.com/matrixorigin/matrixone/pkg/sql/models" v2 "github.com/matrixorigin/matrixone/pkg/util/metric/v2" "github.com/matrixorigin/matrixone/pkg/vm" + "github.com/matrixorigin/matrixone/pkg/vm/process" + "go.uber.org/zap" ) // MaxRpcTime is a default timeout time to rpc context if user never set this deadline. diff --git a/pkg/sql/compile/remoterunServer.go b/pkg/sql/compile/remoterunServer.go index 0ec9cd86e151..45dd4569240f 100644 --- a/pkg/sql/compile/remoterunServer.go +++ b/pkg/sql/compile/remoterunServer.go @@ -23,8 +23,6 @@ import ( "time" "github.com/google/uuid" - "go.uber.org/zap" - "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/common/morpc" "github.com/matrixorigin/matrixone/pkg/common/mpool" @@ -47,6 +45,7 @@ import ( v2 "github.com/matrixorigin/matrixone/pkg/util/metric/v2" "github.com/matrixorigin/matrixone/pkg/vm/engine" "github.com/matrixorigin/matrixone/pkg/vm/process" + "go.uber.org/zap" ) // CnServerMessageHandler receive and deal the message from cn-client. diff --git a/pkg/sql/parsers/dialect/mysql/keywords.go b/pkg/sql/parsers/dialect/mysql/keywords.go index 836384d7be97..e017d5e03d47 100644 --- a/pkg/sql/parsers/dialect/mysql/keywords.go +++ b/pkg/sql/parsers/dialect/mysql/keywords.go @@ -646,5 +646,6 @@ func init() { "cdc": CDC, "rollup": ROLLUP, "apply": APPLY, + "dedup": DEDUP, } } diff --git a/pkg/sql/parsers/dialect/mysql/mysql_sql.go b/pkg/sql/parsers/dialect/mysql/mysql_sql.go index 9480d45ae9c4..cdcb6375b0c9 100644 --- a/pkg/sql/parsers/dialect/mysql/mysql_sql.go +++ b/pkg/sql/parsers/dialect/mysql/mysql_sql.go @@ -95,560 +95,561 @@ const USE = 57418 const FORCE = 57419 const CROSS_L2 = 57420 const APPLY = 57421 -const LOWER_THAN_ON = 57422 -const ON = 57423 -const USING = 57424 -const SUBQUERY_AS_EXPR = 57425 -const LOWER_THAN_STRING = 57426 -const ID = 57427 -const AT_ID = 57428 -const AT_AT_ID = 57429 -const STRING = 57430 -const VALUE_ARG = 57431 -const LIST_ARG = 57432 -const COMMENT = 57433 -const COMMENT_KEYWORD = 57434 -const QUOTE_ID = 57435 -const STAGE = 57436 -const CREDENTIALS = 57437 -const STAGES = 57438 -const SNAPSHOTS = 57439 -const INTEGRAL = 57440 -const HEX = 57441 -const FLOAT = 57442 -const HEXNUM = 57443 -const BIT_LITERAL = 57444 -const NULL = 57445 -const TRUE = 57446 -const FALSE = 57447 -const LOWER_THAN_CHARSET = 57448 -const CHARSET = 57449 -const UNIQUE = 57450 -const KEY = 57451 -const OR = 57452 -const PIPE_CONCAT = 57453 -const XOR = 57454 -const AND = 57455 -const NOT = 57456 -const BETWEEN = 57457 -const CASE = 57458 -const WHEN = 57459 -const THEN = 57460 -const ELSE = 57461 -const END = 57462 -const ELSEIF = 57463 -const LOWER_THAN_EQ = 57464 -const LE = 57465 -const GE = 57466 -const NE = 57467 -const NULL_SAFE_EQUAL = 57468 -const IS = 57469 -const LIKE = 57470 -const REGEXP = 57471 -const IN = 57472 -const ASSIGNMENT = 57473 -const ILIKE = 57474 -const SHIFT_LEFT = 57475 -const SHIFT_RIGHT = 57476 -const DIV = 57477 -const MOD = 57478 -const UNARY = 57479 -const COLLATE = 57480 -const BINARY = 57481 -const UNDERSCORE_BINARY = 57482 -const INTERVAL = 57483 -const OUT = 57484 -const INOUT = 57485 -const BEGIN = 57486 -const START = 57487 -const TRANSACTION = 57488 -const COMMIT = 57489 -const ROLLBACK = 57490 -const WORK = 57491 -const CONSISTENT = 57492 -const SNAPSHOT = 57493 -const CHAIN = 57494 -const NO = 57495 -const RELEASE = 57496 -const PRIORITY = 57497 -const QUICK = 57498 -const BIT = 57499 -const TINYINT = 57500 -const SMALLINT = 57501 -const MEDIUMINT = 57502 -const INT = 57503 -const INTEGER = 57504 -const BIGINT = 57505 -const INTNUM = 57506 -const REAL = 57507 -const DOUBLE = 57508 -const FLOAT_TYPE = 57509 -const DECIMAL = 57510 -const NUMERIC = 57511 -const DECIMAL_VALUE = 57512 -const TIME = 57513 -const TIMESTAMP = 57514 -const DATETIME = 57515 -const YEAR = 57516 -const CHAR = 57517 -const VARCHAR = 57518 -const BOOL = 57519 -const CHARACTER = 57520 -const VARBINARY = 57521 -const NCHAR = 57522 -const TEXT = 57523 -const TINYTEXT = 57524 -const MEDIUMTEXT = 57525 -const LONGTEXT = 57526 -const DATALINK = 57527 -const BLOB = 57528 -const TINYBLOB = 57529 -const MEDIUMBLOB = 57530 -const LONGBLOB = 57531 -const JSON = 57532 -const ENUM = 57533 -const UUID = 57534 -const VECF32 = 57535 -const VECF64 = 57536 -const GEOMETRY = 57537 -const POINT = 57538 -const LINESTRING = 57539 -const POLYGON = 57540 -const GEOMETRYCOLLECTION = 57541 -const MULTIPOINT = 57542 -const MULTILINESTRING = 57543 -const MULTIPOLYGON = 57544 -const INT1 = 57545 -const INT2 = 57546 -const INT3 = 57547 -const INT4 = 57548 -const INT8 = 57549 -const S3OPTION = 57550 -const STAGEOPTION = 57551 -const SQL_SMALL_RESULT = 57552 -const SQL_BIG_RESULT = 57553 -const SQL_BUFFER_RESULT = 57554 -const LOW_PRIORITY = 57555 -const HIGH_PRIORITY = 57556 -const DELAYED = 57557 -const CREATE = 57558 -const ALTER = 57559 -const DROP = 57560 -const RENAME = 57561 -const ANALYZE = 57562 -const PHYPLAN = 57563 -const ADD = 57564 -const RETURNS = 57565 -const SCHEMA = 57566 -const TABLE = 57567 -const SEQUENCE = 57568 -const INDEX = 57569 -const VIEW = 57570 -const TO = 57571 -const IGNORE = 57572 -const IF = 57573 -const PRIMARY = 57574 -const COLUMN = 57575 -const CONSTRAINT = 57576 -const SPATIAL = 57577 -const FULLTEXT = 57578 -const FOREIGN = 57579 -const KEY_BLOCK_SIZE = 57580 -const SHOW = 57581 -const DESCRIBE = 57582 -const EXPLAIN = 57583 -const DATE = 57584 -const ESCAPE = 57585 -const REPAIR = 57586 -const OPTIMIZE = 57587 -const TRUNCATE = 57588 -const MAXVALUE = 57589 -const PARTITION = 57590 -const REORGANIZE = 57591 -const LESS = 57592 -const THAN = 57593 -const PROCEDURE = 57594 -const TRIGGER = 57595 -const STATUS = 57596 -const VARIABLES = 57597 -const ROLE = 57598 -const PROXY = 57599 -const AVG_ROW_LENGTH = 57600 -const STORAGE = 57601 -const DISK = 57602 -const MEMORY = 57603 -const CHECKSUM = 57604 -const COMPRESSION = 57605 -const DATA = 57606 -const DIRECTORY = 57607 -const DELAY_KEY_WRITE = 57608 -const ENCRYPTION = 57609 -const ENGINE = 57610 -const MAX_ROWS = 57611 -const MIN_ROWS = 57612 -const PACK_KEYS = 57613 -const ROW_FORMAT = 57614 -const STATS_AUTO_RECALC = 57615 -const STATS_PERSISTENT = 57616 -const STATS_SAMPLE_PAGES = 57617 -const DYNAMIC = 57618 -const COMPRESSED = 57619 -const REDUNDANT = 57620 -const COMPACT = 57621 -const FIXED = 57622 -const COLUMN_FORMAT = 57623 -const AUTO_RANDOM = 57624 -const ENGINE_ATTRIBUTE = 57625 -const SECONDARY_ENGINE_ATTRIBUTE = 57626 -const INSERT_METHOD = 57627 -const RESTRICT = 57628 -const CASCADE = 57629 -const ACTION = 57630 -const PARTIAL = 57631 -const SIMPLE = 57632 -const CHECK = 57633 -const ENFORCED = 57634 -const RANGE = 57635 -const LIST = 57636 -const ALGORITHM = 57637 -const LINEAR = 57638 -const PARTITIONS = 57639 -const SUBPARTITION = 57640 -const SUBPARTITIONS = 57641 -const CLUSTER = 57642 -const TYPE = 57643 -const ANY = 57644 -const SOME = 57645 -const EXTERNAL = 57646 -const LOCALFILE = 57647 -const URL = 57648 -const PREPARE = 57649 -const DEALLOCATE = 57650 -const RESET = 57651 -const EXTENSION = 57652 -const RETENTION = 57653 -const PERIOD = 57654 -const INCREMENT = 57655 -const CYCLE = 57656 -const MINVALUE = 57657 -const PUBLICATION = 57658 -const SUBSCRIPTIONS = 57659 -const PUBLICATIONS = 57660 -const PROPERTIES = 57661 -const PARSER = 57662 -const VISIBLE = 57663 -const INVISIBLE = 57664 -const BTREE = 57665 -const HASH = 57666 -const RTREE = 57667 -const BSI = 57668 -const IVFFLAT = 57669 -const MASTER = 57670 -const ZONEMAP = 57671 -const LEADING = 57672 -const BOTH = 57673 -const TRAILING = 57674 -const UNKNOWN = 57675 -const LISTS = 57676 -const OP_TYPE = 57677 -const REINDEX = 57678 -const EXPIRE = 57679 -const ACCOUNT = 57680 -const ACCOUNTS = 57681 -const UNLOCK = 57682 -const DAY = 57683 -const NEVER = 57684 -const PUMP = 57685 -const MYSQL_COMPATIBILITY_MODE = 57686 -const UNIQUE_CHECK_ON_AUTOINCR = 57687 -const MODIFY = 57688 -const CHANGE = 57689 -const SECOND = 57690 -const ASCII = 57691 -const COALESCE = 57692 -const COLLATION = 57693 -const HOUR = 57694 -const MICROSECOND = 57695 -const MINUTE = 57696 -const MONTH = 57697 -const QUARTER = 57698 -const REPEAT = 57699 -const REVERSE = 57700 -const ROW_COUNT = 57701 -const WEEK = 57702 -const REVOKE = 57703 -const FUNCTION = 57704 -const PRIVILEGES = 57705 -const TABLESPACE = 57706 -const EXECUTE = 57707 -const SUPER = 57708 -const GRANT = 57709 -const OPTION = 57710 -const REFERENCES = 57711 -const REPLICATION = 57712 -const SLAVE = 57713 -const CLIENT = 57714 -const USAGE = 57715 -const RELOAD = 57716 -const FILE = 57717 -const TEMPORARY = 57718 -const ROUTINE = 57719 -const EVENT = 57720 -const SHUTDOWN = 57721 -const NULLX = 57722 -const AUTO_INCREMENT = 57723 -const APPROXNUM = 57724 -const SIGNED = 57725 -const UNSIGNED = 57726 -const ZEROFILL = 57727 -const ENGINES = 57728 -const LOW_CARDINALITY = 57729 -const AUTOEXTEND_SIZE = 57730 -const ADMIN_NAME = 57731 -const RANDOM = 57732 -const SUSPEND = 57733 -const ATTRIBUTE = 57734 -const HISTORY = 57735 -const REUSE = 57736 -const CURRENT = 57737 -const OPTIONAL = 57738 -const FAILED_LOGIN_ATTEMPTS = 57739 -const PASSWORD_LOCK_TIME = 57740 -const UNBOUNDED = 57741 -const SECONDARY = 57742 -const RESTRICTED = 57743 -const USER = 57744 -const IDENTIFIED = 57745 -const CIPHER = 57746 -const ISSUER = 57747 -const X509 = 57748 -const SUBJECT = 57749 -const SAN = 57750 -const REQUIRE = 57751 -const SSL = 57752 -const NONE = 57753 -const PASSWORD = 57754 -const SHARED = 57755 -const EXCLUSIVE = 57756 -const MAX_QUERIES_PER_HOUR = 57757 -const MAX_UPDATES_PER_HOUR = 57758 -const MAX_CONNECTIONS_PER_HOUR = 57759 -const MAX_USER_CONNECTIONS = 57760 -const FORMAT = 57761 -const VERBOSE = 57762 -const CONNECTION = 57763 -const TRIGGERS = 57764 -const PROFILES = 57765 -const LOAD = 57766 -const INLINE = 57767 -const INFILE = 57768 -const TERMINATED = 57769 -const OPTIONALLY = 57770 -const ENCLOSED = 57771 -const ESCAPED = 57772 -const STARTING = 57773 -const LINES = 57774 -const ROWS = 57775 -const IMPORT = 57776 -const DISCARD = 57777 -const JSONTYPE = 57778 -const MODUMP = 57779 -const OVER = 57780 -const PRECEDING = 57781 -const FOLLOWING = 57782 -const GROUPS = 57783 -const DATABASES = 57784 -const TABLES = 57785 -const SEQUENCES = 57786 -const EXTENDED = 57787 -const FULL = 57788 -const PROCESSLIST = 57789 -const FIELDS = 57790 -const COLUMNS = 57791 -const OPEN = 57792 -const ERRORS = 57793 -const WARNINGS = 57794 -const INDEXES = 57795 -const SCHEMAS = 57796 -const NODE = 57797 -const LOCKS = 57798 -const ROLES = 57799 -const TABLE_NUMBER = 57800 -const COLUMN_NUMBER = 57801 -const TABLE_VALUES = 57802 -const TABLE_SIZE = 57803 -const NAMES = 57804 -const GLOBAL = 57805 -const PERSIST = 57806 -const SESSION = 57807 -const ISOLATION = 57808 -const LEVEL = 57809 -const READ = 57810 -const WRITE = 57811 -const ONLY = 57812 -const REPEATABLE = 57813 -const COMMITTED = 57814 -const UNCOMMITTED = 57815 -const SERIALIZABLE = 57816 -const LOCAL = 57817 -const EVENTS = 57818 -const PLUGINS = 57819 -const CURRENT_TIMESTAMP = 57820 -const DATABASE = 57821 -const CURRENT_TIME = 57822 -const LOCALTIME = 57823 -const LOCALTIMESTAMP = 57824 -const UTC_DATE = 57825 -const UTC_TIME = 57826 -const UTC_TIMESTAMP = 57827 -const REPLACE = 57828 -const CONVERT = 57829 -const SEPARATOR = 57830 -const TIMESTAMPDIFF = 57831 -const CURRENT_DATE = 57832 -const CURRENT_USER = 57833 -const CURRENT_ROLE = 57834 -const SECOND_MICROSECOND = 57835 -const MINUTE_MICROSECOND = 57836 -const MINUTE_SECOND = 57837 -const HOUR_MICROSECOND = 57838 -const HOUR_SECOND = 57839 -const HOUR_MINUTE = 57840 -const DAY_MICROSECOND = 57841 -const DAY_SECOND = 57842 -const DAY_MINUTE = 57843 -const DAY_HOUR = 57844 -const YEAR_MONTH = 57845 -const SQL_TSI_HOUR = 57846 -const SQL_TSI_DAY = 57847 -const SQL_TSI_WEEK = 57848 -const SQL_TSI_MONTH = 57849 -const SQL_TSI_QUARTER = 57850 -const SQL_TSI_YEAR = 57851 -const SQL_TSI_SECOND = 57852 -const SQL_TSI_MINUTE = 57853 -const RECURSIVE = 57854 -const CONFIG = 57855 -const DRAINER = 57856 -const SOURCE = 57857 -const STREAM = 57858 -const HEADERS = 57859 -const CONNECTOR = 57860 -const CONNECTORS = 57861 -const DAEMON = 57862 -const PAUSE = 57863 -const CANCEL = 57864 -const TASK = 57865 -const RESUME = 57866 -const MATCH = 57867 -const AGAINST = 57868 -const BOOLEAN = 57869 -const LANGUAGE = 57870 -const WITH = 57871 -const QUERY = 57872 -const EXPANSION = 57873 -const WITHOUT = 57874 -const VALIDATION = 57875 -const UPGRADE = 57876 -const RETRY = 57877 -const ADDDATE = 57878 -const BIT_AND = 57879 -const BIT_OR = 57880 -const BIT_XOR = 57881 -const CAST = 57882 -const COUNT = 57883 -const APPROX_COUNT = 57884 -const APPROX_COUNT_DISTINCT = 57885 -const SERIAL_EXTRACT = 57886 -const APPROX_PERCENTILE = 57887 -const CURDATE = 57888 -const CURTIME = 57889 -const DATE_ADD = 57890 -const DATE_SUB = 57891 -const EXTRACT = 57892 -const GROUP_CONCAT = 57893 -const MAX = 57894 -const MID = 57895 -const MIN = 57896 -const NOW = 57897 -const POSITION = 57898 -const SESSION_USER = 57899 -const STD = 57900 -const STDDEV = 57901 -const MEDIAN = 57902 -const CLUSTER_CENTERS = 57903 -const KMEANS = 57904 -const STDDEV_POP = 57905 -const STDDEV_SAMP = 57906 -const SUBDATE = 57907 -const SUBSTR = 57908 -const SUBSTRING = 57909 -const SUM = 57910 -const SYSDATE = 57911 -const SYSTEM_USER = 57912 -const TRANSLATE = 57913 -const TRIM = 57914 -const VARIANCE = 57915 -const VAR_POP = 57916 -const VAR_SAMP = 57917 -const AVG = 57918 -const RANK = 57919 -const ROW_NUMBER = 57920 -const DENSE_RANK = 57921 -const BIT_CAST = 57922 -const BITMAP_BIT_POSITION = 57923 -const BITMAP_BUCKET_NUMBER = 57924 -const BITMAP_COUNT = 57925 -const BITMAP_CONSTRUCT_AGG = 57926 -const BITMAP_OR_AGG = 57927 -const NEXTVAL = 57928 -const SETVAL = 57929 -const CURRVAL = 57930 -const LASTVAL = 57931 -const ARROW = 57932 -const ROW = 57933 -const OUTFILE = 57934 -const HEADER = 57935 -const MAX_FILE_SIZE = 57936 -const FORCE_QUOTE = 57937 -const PARALLEL = 57938 -const STRICT = 57939 -const UNUSED = 57940 -const BINDINGS = 57941 -const DO = 57942 -const DECLARE = 57943 -const LOOP = 57944 -const WHILE = 57945 -const LEAVE = 57946 -const ITERATE = 57947 -const UNTIL = 57948 -const CALL = 57949 -const PREV = 57950 -const SLIDING = 57951 -const FILL = 57952 -const SPBEGIN = 57953 -const BACKEND = 57954 -const SERVERS = 57955 -const HANDLER = 57956 -const PERCENT = 57957 -const SAMPLE = 57958 -const MO_TS = 57959 -const PITR = 57960 -const CDC = 57961 -const GROUPING = 57962 -const SETS = 57963 -const CUBE = 57964 -const ROLLUP = 57965 -const LOGSERVICE = 57966 -const REPLICAS = 57967 -const STORES = 57968 -const SETTINGS = 57969 -const KILL = 57970 -const BACKUP = 57971 -const FILESYSTEM = 57972 -const PARALLELISM = 57973 -const RESTORE = 57974 -const QUERY_RESULT = 57975 +const DEDUP = 57422 +const LOWER_THAN_ON = 57423 +const ON = 57424 +const USING = 57425 +const SUBQUERY_AS_EXPR = 57426 +const LOWER_THAN_STRING = 57427 +const ID = 57428 +const AT_ID = 57429 +const AT_AT_ID = 57430 +const STRING = 57431 +const VALUE_ARG = 57432 +const LIST_ARG = 57433 +const COMMENT = 57434 +const COMMENT_KEYWORD = 57435 +const QUOTE_ID = 57436 +const STAGE = 57437 +const CREDENTIALS = 57438 +const STAGES = 57439 +const SNAPSHOTS = 57440 +const INTEGRAL = 57441 +const HEX = 57442 +const FLOAT = 57443 +const HEXNUM = 57444 +const BIT_LITERAL = 57445 +const NULL = 57446 +const TRUE = 57447 +const FALSE = 57448 +const LOWER_THAN_CHARSET = 57449 +const CHARSET = 57450 +const UNIQUE = 57451 +const KEY = 57452 +const OR = 57453 +const PIPE_CONCAT = 57454 +const XOR = 57455 +const AND = 57456 +const NOT = 57457 +const BETWEEN = 57458 +const CASE = 57459 +const WHEN = 57460 +const THEN = 57461 +const ELSE = 57462 +const END = 57463 +const ELSEIF = 57464 +const LOWER_THAN_EQ = 57465 +const LE = 57466 +const GE = 57467 +const NE = 57468 +const NULL_SAFE_EQUAL = 57469 +const IS = 57470 +const LIKE = 57471 +const REGEXP = 57472 +const IN = 57473 +const ASSIGNMENT = 57474 +const ILIKE = 57475 +const SHIFT_LEFT = 57476 +const SHIFT_RIGHT = 57477 +const DIV = 57478 +const MOD = 57479 +const UNARY = 57480 +const COLLATE = 57481 +const BINARY = 57482 +const UNDERSCORE_BINARY = 57483 +const INTERVAL = 57484 +const OUT = 57485 +const INOUT = 57486 +const BEGIN = 57487 +const START = 57488 +const TRANSACTION = 57489 +const COMMIT = 57490 +const ROLLBACK = 57491 +const WORK = 57492 +const CONSISTENT = 57493 +const SNAPSHOT = 57494 +const CHAIN = 57495 +const NO = 57496 +const RELEASE = 57497 +const PRIORITY = 57498 +const QUICK = 57499 +const BIT = 57500 +const TINYINT = 57501 +const SMALLINT = 57502 +const MEDIUMINT = 57503 +const INT = 57504 +const INTEGER = 57505 +const BIGINT = 57506 +const INTNUM = 57507 +const REAL = 57508 +const DOUBLE = 57509 +const FLOAT_TYPE = 57510 +const DECIMAL = 57511 +const NUMERIC = 57512 +const DECIMAL_VALUE = 57513 +const TIME = 57514 +const TIMESTAMP = 57515 +const DATETIME = 57516 +const YEAR = 57517 +const CHAR = 57518 +const VARCHAR = 57519 +const BOOL = 57520 +const CHARACTER = 57521 +const VARBINARY = 57522 +const NCHAR = 57523 +const TEXT = 57524 +const TINYTEXT = 57525 +const MEDIUMTEXT = 57526 +const LONGTEXT = 57527 +const DATALINK = 57528 +const BLOB = 57529 +const TINYBLOB = 57530 +const MEDIUMBLOB = 57531 +const LONGBLOB = 57532 +const JSON = 57533 +const ENUM = 57534 +const UUID = 57535 +const VECF32 = 57536 +const VECF64 = 57537 +const GEOMETRY = 57538 +const POINT = 57539 +const LINESTRING = 57540 +const POLYGON = 57541 +const GEOMETRYCOLLECTION = 57542 +const MULTIPOINT = 57543 +const MULTILINESTRING = 57544 +const MULTIPOLYGON = 57545 +const INT1 = 57546 +const INT2 = 57547 +const INT3 = 57548 +const INT4 = 57549 +const INT8 = 57550 +const S3OPTION = 57551 +const STAGEOPTION = 57552 +const SQL_SMALL_RESULT = 57553 +const SQL_BIG_RESULT = 57554 +const SQL_BUFFER_RESULT = 57555 +const LOW_PRIORITY = 57556 +const HIGH_PRIORITY = 57557 +const DELAYED = 57558 +const CREATE = 57559 +const ALTER = 57560 +const DROP = 57561 +const RENAME = 57562 +const ANALYZE = 57563 +const PHYPLAN = 57564 +const ADD = 57565 +const RETURNS = 57566 +const SCHEMA = 57567 +const TABLE = 57568 +const SEQUENCE = 57569 +const INDEX = 57570 +const VIEW = 57571 +const TO = 57572 +const IGNORE = 57573 +const IF = 57574 +const PRIMARY = 57575 +const COLUMN = 57576 +const CONSTRAINT = 57577 +const SPATIAL = 57578 +const FULLTEXT = 57579 +const FOREIGN = 57580 +const KEY_BLOCK_SIZE = 57581 +const SHOW = 57582 +const DESCRIBE = 57583 +const EXPLAIN = 57584 +const DATE = 57585 +const ESCAPE = 57586 +const REPAIR = 57587 +const OPTIMIZE = 57588 +const TRUNCATE = 57589 +const MAXVALUE = 57590 +const PARTITION = 57591 +const REORGANIZE = 57592 +const LESS = 57593 +const THAN = 57594 +const PROCEDURE = 57595 +const TRIGGER = 57596 +const STATUS = 57597 +const VARIABLES = 57598 +const ROLE = 57599 +const PROXY = 57600 +const AVG_ROW_LENGTH = 57601 +const STORAGE = 57602 +const DISK = 57603 +const MEMORY = 57604 +const CHECKSUM = 57605 +const COMPRESSION = 57606 +const DATA = 57607 +const DIRECTORY = 57608 +const DELAY_KEY_WRITE = 57609 +const ENCRYPTION = 57610 +const ENGINE = 57611 +const MAX_ROWS = 57612 +const MIN_ROWS = 57613 +const PACK_KEYS = 57614 +const ROW_FORMAT = 57615 +const STATS_AUTO_RECALC = 57616 +const STATS_PERSISTENT = 57617 +const STATS_SAMPLE_PAGES = 57618 +const DYNAMIC = 57619 +const COMPRESSED = 57620 +const REDUNDANT = 57621 +const COMPACT = 57622 +const FIXED = 57623 +const COLUMN_FORMAT = 57624 +const AUTO_RANDOM = 57625 +const ENGINE_ATTRIBUTE = 57626 +const SECONDARY_ENGINE_ATTRIBUTE = 57627 +const INSERT_METHOD = 57628 +const RESTRICT = 57629 +const CASCADE = 57630 +const ACTION = 57631 +const PARTIAL = 57632 +const SIMPLE = 57633 +const CHECK = 57634 +const ENFORCED = 57635 +const RANGE = 57636 +const LIST = 57637 +const ALGORITHM = 57638 +const LINEAR = 57639 +const PARTITIONS = 57640 +const SUBPARTITION = 57641 +const SUBPARTITIONS = 57642 +const CLUSTER = 57643 +const TYPE = 57644 +const ANY = 57645 +const SOME = 57646 +const EXTERNAL = 57647 +const LOCALFILE = 57648 +const URL = 57649 +const PREPARE = 57650 +const DEALLOCATE = 57651 +const RESET = 57652 +const EXTENSION = 57653 +const RETENTION = 57654 +const PERIOD = 57655 +const INCREMENT = 57656 +const CYCLE = 57657 +const MINVALUE = 57658 +const PUBLICATION = 57659 +const SUBSCRIPTIONS = 57660 +const PUBLICATIONS = 57661 +const PROPERTIES = 57662 +const PARSER = 57663 +const VISIBLE = 57664 +const INVISIBLE = 57665 +const BTREE = 57666 +const HASH = 57667 +const RTREE = 57668 +const BSI = 57669 +const IVFFLAT = 57670 +const MASTER = 57671 +const ZONEMAP = 57672 +const LEADING = 57673 +const BOTH = 57674 +const TRAILING = 57675 +const UNKNOWN = 57676 +const LISTS = 57677 +const OP_TYPE = 57678 +const REINDEX = 57679 +const EXPIRE = 57680 +const ACCOUNT = 57681 +const ACCOUNTS = 57682 +const UNLOCK = 57683 +const DAY = 57684 +const NEVER = 57685 +const PUMP = 57686 +const MYSQL_COMPATIBILITY_MODE = 57687 +const UNIQUE_CHECK_ON_AUTOINCR = 57688 +const MODIFY = 57689 +const CHANGE = 57690 +const SECOND = 57691 +const ASCII = 57692 +const COALESCE = 57693 +const COLLATION = 57694 +const HOUR = 57695 +const MICROSECOND = 57696 +const MINUTE = 57697 +const MONTH = 57698 +const QUARTER = 57699 +const REPEAT = 57700 +const REVERSE = 57701 +const ROW_COUNT = 57702 +const WEEK = 57703 +const REVOKE = 57704 +const FUNCTION = 57705 +const PRIVILEGES = 57706 +const TABLESPACE = 57707 +const EXECUTE = 57708 +const SUPER = 57709 +const GRANT = 57710 +const OPTION = 57711 +const REFERENCES = 57712 +const REPLICATION = 57713 +const SLAVE = 57714 +const CLIENT = 57715 +const USAGE = 57716 +const RELOAD = 57717 +const FILE = 57718 +const TEMPORARY = 57719 +const ROUTINE = 57720 +const EVENT = 57721 +const SHUTDOWN = 57722 +const NULLX = 57723 +const AUTO_INCREMENT = 57724 +const APPROXNUM = 57725 +const SIGNED = 57726 +const UNSIGNED = 57727 +const ZEROFILL = 57728 +const ENGINES = 57729 +const LOW_CARDINALITY = 57730 +const AUTOEXTEND_SIZE = 57731 +const ADMIN_NAME = 57732 +const RANDOM = 57733 +const SUSPEND = 57734 +const ATTRIBUTE = 57735 +const HISTORY = 57736 +const REUSE = 57737 +const CURRENT = 57738 +const OPTIONAL = 57739 +const FAILED_LOGIN_ATTEMPTS = 57740 +const PASSWORD_LOCK_TIME = 57741 +const UNBOUNDED = 57742 +const SECONDARY = 57743 +const RESTRICTED = 57744 +const USER = 57745 +const IDENTIFIED = 57746 +const CIPHER = 57747 +const ISSUER = 57748 +const X509 = 57749 +const SUBJECT = 57750 +const SAN = 57751 +const REQUIRE = 57752 +const SSL = 57753 +const NONE = 57754 +const PASSWORD = 57755 +const SHARED = 57756 +const EXCLUSIVE = 57757 +const MAX_QUERIES_PER_HOUR = 57758 +const MAX_UPDATES_PER_HOUR = 57759 +const MAX_CONNECTIONS_PER_HOUR = 57760 +const MAX_USER_CONNECTIONS = 57761 +const FORMAT = 57762 +const VERBOSE = 57763 +const CONNECTION = 57764 +const TRIGGERS = 57765 +const PROFILES = 57766 +const LOAD = 57767 +const INLINE = 57768 +const INFILE = 57769 +const TERMINATED = 57770 +const OPTIONALLY = 57771 +const ENCLOSED = 57772 +const ESCAPED = 57773 +const STARTING = 57774 +const LINES = 57775 +const ROWS = 57776 +const IMPORT = 57777 +const DISCARD = 57778 +const JSONTYPE = 57779 +const MODUMP = 57780 +const OVER = 57781 +const PRECEDING = 57782 +const FOLLOWING = 57783 +const GROUPS = 57784 +const DATABASES = 57785 +const TABLES = 57786 +const SEQUENCES = 57787 +const EXTENDED = 57788 +const FULL = 57789 +const PROCESSLIST = 57790 +const FIELDS = 57791 +const COLUMNS = 57792 +const OPEN = 57793 +const ERRORS = 57794 +const WARNINGS = 57795 +const INDEXES = 57796 +const SCHEMAS = 57797 +const NODE = 57798 +const LOCKS = 57799 +const ROLES = 57800 +const TABLE_NUMBER = 57801 +const COLUMN_NUMBER = 57802 +const TABLE_VALUES = 57803 +const TABLE_SIZE = 57804 +const NAMES = 57805 +const GLOBAL = 57806 +const PERSIST = 57807 +const SESSION = 57808 +const ISOLATION = 57809 +const LEVEL = 57810 +const READ = 57811 +const WRITE = 57812 +const ONLY = 57813 +const REPEATABLE = 57814 +const COMMITTED = 57815 +const UNCOMMITTED = 57816 +const SERIALIZABLE = 57817 +const LOCAL = 57818 +const EVENTS = 57819 +const PLUGINS = 57820 +const CURRENT_TIMESTAMP = 57821 +const DATABASE = 57822 +const CURRENT_TIME = 57823 +const LOCALTIME = 57824 +const LOCALTIMESTAMP = 57825 +const UTC_DATE = 57826 +const UTC_TIME = 57827 +const UTC_TIMESTAMP = 57828 +const REPLACE = 57829 +const CONVERT = 57830 +const SEPARATOR = 57831 +const TIMESTAMPDIFF = 57832 +const CURRENT_DATE = 57833 +const CURRENT_USER = 57834 +const CURRENT_ROLE = 57835 +const SECOND_MICROSECOND = 57836 +const MINUTE_MICROSECOND = 57837 +const MINUTE_SECOND = 57838 +const HOUR_MICROSECOND = 57839 +const HOUR_SECOND = 57840 +const HOUR_MINUTE = 57841 +const DAY_MICROSECOND = 57842 +const DAY_SECOND = 57843 +const DAY_MINUTE = 57844 +const DAY_HOUR = 57845 +const YEAR_MONTH = 57846 +const SQL_TSI_HOUR = 57847 +const SQL_TSI_DAY = 57848 +const SQL_TSI_WEEK = 57849 +const SQL_TSI_MONTH = 57850 +const SQL_TSI_QUARTER = 57851 +const SQL_TSI_YEAR = 57852 +const SQL_TSI_SECOND = 57853 +const SQL_TSI_MINUTE = 57854 +const RECURSIVE = 57855 +const CONFIG = 57856 +const DRAINER = 57857 +const SOURCE = 57858 +const STREAM = 57859 +const HEADERS = 57860 +const CONNECTOR = 57861 +const CONNECTORS = 57862 +const DAEMON = 57863 +const PAUSE = 57864 +const CANCEL = 57865 +const TASK = 57866 +const RESUME = 57867 +const MATCH = 57868 +const AGAINST = 57869 +const BOOLEAN = 57870 +const LANGUAGE = 57871 +const WITH = 57872 +const QUERY = 57873 +const EXPANSION = 57874 +const WITHOUT = 57875 +const VALIDATION = 57876 +const UPGRADE = 57877 +const RETRY = 57878 +const ADDDATE = 57879 +const BIT_AND = 57880 +const BIT_OR = 57881 +const BIT_XOR = 57882 +const CAST = 57883 +const COUNT = 57884 +const APPROX_COUNT = 57885 +const APPROX_COUNT_DISTINCT = 57886 +const SERIAL_EXTRACT = 57887 +const APPROX_PERCENTILE = 57888 +const CURDATE = 57889 +const CURTIME = 57890 +const DATE_ADD = 57891 +const DATE_SUB = 57892 +const EXTRACT = 57893 +const GROUP_CONCAT = 57894 +const MAX = 57895 +const MID = 57896 +const MIN = 57897 +const NOW = 57898 +const POSITION = 57899 +const SESSION_USER = 57900 +const STD = 57901 +const STDDEV = 57902 +const MEDIAN = 57903 +const CLUSTER_CENTERS = 57904 +const KMEANS = 57905 +const STDDEV_POP = 57906 +const STDDEV_SAMP = 57907 +const SUBDATE = 57908 +const SUBSTR = 57909 +const SUBSTRING = 57910 +const SUM = 57911 +const SYSDATE = 57912 +const SYSTEM_USER = 57913 +const TRANSLATE = 57914 +const TRIM = 57915 +const VARIANCE = 57916 +const VAR_POP = 57917 +const VAR_SAMP = 57918 +const AVG = 57919 +const RANK = 57920 +const ROW_NUMBER = 57921 +const DENSE_RANK = 57922 +const BIT_CAST = 57923 +const BITMAP_BIT_POSITION = 57924 +const BITMAP_BUCKET_NUMBER = 57925 +const BITMAP_COUNT = 57926 +const BITMAP_CONSTRUCT_AGG = 57927 +const BITMAP_OR_AGG = 57928 +const NEXTVAL = 57929 +const SETVAL = 57930 +const CURRVAL = 57931 +const LASTVAL = 57932 +const ARROW = 57933 +const ROW = 57934 +const OUTFILE = 57935 +const HEADER = 57936 +const MAX_FILE_SIZE = 57937 +const FORCE_QUOTE = 57938 +const PARALLEL = 57939 +const STRICT = 57940 +const UNUSED = 57941 +const BINDINGS = 57942 +const DO = 57943 +const DECLARE = 57944 +const LOOP = 57945 +const WHILE = 57946 +const LEAVE = 57947 +const ITERATE = 57948 +const UNTIL = 57949 +const CALL = 57950 +const PREV = 57951 +const SLIDING = 57952 +const FILL = 57953 +const SPBEGIN = 57954 +const BACKEND = 57955 +const SERVERS = 57956 +const HANDLER = 57957 +const PERCENT = 57958 +const SAMPLE = 57959 +const MO_TS = 57960 +const PITR = 57961 +const CDC = 57962 +const GROUPING = 57963 +const SETS = 57964 +const CUBE = 57965 +const ROLLUP = 57966 +const LOGSERVICE = 57967 +const REPLICAS = 57968 +const STORES = 57969 +const SETTINGS = 57970 +const KILL = 57971 +const BACKUP = 57972 +const FILESYSTEM = 57973 +const PARALLELISM = 57974 +const RESTORE = 57975 +const QUERY_RESULT = 57976 var yyToknames = [...]string{ "$end", @@ -730,6 +731,7 @@ var yyToknames = [...]string{ "FORCE", "CROSS_L2", "APPLY", + "DEDUP", "LOWER_THAN_ON", "ON", "USING", @@ -1314,7 +1316,7 @@ const yyEofCode = 1 const yyErrCode = 2 const yyInitialStackSize = 16 -//line mysql_sql.y:12684 +//line mysql_sql.y:12701 //line yacctab:1 var yyExca = [...]int{ @@ -1326,1018 +1328,1164 @@ var yyExca = [...]int{ 22, 791, -2, 784, -1, 161, - 242, 1211, - 244, 1110, - -2, 1157, + 243, 1213, + 245, 1112, + -2, 1159, -1, 188, 43, 614, - 244, 614, - 271, 621, + 245, 614, 272, 621, - 470, 614, + 273, 621, + 471, 614, -2, 649, -1, 228, - 654, 1981, + 655, 1984, -2, 518, -1, 535, - 654, 2103, + 655, 2106, -2, 398, -1, 593, - 654, 2162, + 655, 2165, -2, 396, -1, 594, - 654, 2163, + 655, 2166, -2, 397, -1, 595, - 654, 2164, + 655, 2167, -2, 399, -1, 733, - 323, 176, - 442, 176, + 324, 176, 443, 176, - -2, 1883, + 444, 176, + -2, 1886, -1, 800, - 84, 1669, - -2, 2039, + 85, 1671, + -2, 2042, -1, 801, - 84, 1687, - -2, 2010, + 85, 1690, + -2, 2013, -1, 805, - 84, 1688, - -2, 2038, + 85, 1691, + -2, 2041, -1, 839, - 84, 1596, - -2, 2240, + 85, 1598, + -2, 2243, -1, 840, - 84, 1597, - -2, 2239, + 85, 1599, + -2, 2242, -1, 841, - 84, 1598, - -2, 2229, + 85, 1600, + -2, 2232, -1, 842, - 84, 2201, - -2, 2222, + 85, 2204, + -2, 2225, -1, 843, - 84, 2202, - -2, 2223, + 85, 2205, + -2, 2226, -1, 844, - 84, 2203, - -2, 2231, + 85, 2206, + -2, 2234, -1, 845, - 84, 2204, - -2, 2211, + 85, 2207, + -2, 2214, -1, 846, - 84, 2205, - -2, 2220, + 85, 2208, + -2, 2223, -1, 847, - 84, 2206, - -2, 2232, + 85, 2209, + -2, 2235, -1, 848, - 84, 2207, - -2, 2233, + 85, 2210, + -2, 2236, -1, 849, - 84, 2208, - -2, 2238, + 85, 2211, + -2, 2241, -1, 850, - 84, 2209, - -2, 2243, + 85, 2212, + -2, 2246, -1, 851, - 84, 2210, - -2, 2244, + 85, 2213, + -2, 2247, -1, 852, - 84, 1665, - -2, 2077, + 85, 1667, + -2, 2080, -1, 853, - 84, 1666, - -2, 1867, + 85, 1668, + -2, 1870, -1, 854, - 84, 1667, - -2, 2086, + 85, 1669, + -2, 2089, -1, 855, - 84, 1668, - -2, 1876, + 85, 1670, + -2, 1879, -1, 857, - 84, 1671, - -2, 1884, - -1, 858, - 84, 1672, - -2, 2110, - -1, 860, - 84, 1675, - -2, 1903, - -1, 862, - 84, 1677, - -2, 2122, + 85, 1673, + -2, 1887, + -1, 859, + 85, 1675, + -2, 2113, + -1, 861, + 85, 1678, + -2, 1906, -1, 863, - 84, 1678, - -2, 2121, + 85, 1680, + -2, 2125, -1, 864, - 84, 1679, - -2, 1948, + 85, 1681, + -2, 2124, -1, 865, - 84, 1680, - -2, 2034, - -1, 868, - 84, 1683, - -2, 2133, - -1, 870, - 84, 1685, + 85, 1682, + -2, 1951, + -1, 866, + 85, 1683, + -2, 2037, + -1, 869, + 85, 1686, -2, 2136, -1, 871, - 84, 1686, - -2, 2138, + 85, 1688, + -2, 2139, -1, 872, - 84, 1689, - -2, 2146, + 85, 1689, + -2, 2141, -1, 873, - 84, 1690, - -2, 2019, + 85, 1692, + -2, 2149, -1, 874, - 84, 1691, - -2, 2064, + 85, 1693, + -2, 2022, -1, 875, - 84, 1692, - -2, 2029, + 85, 1694, + -2, 2067, -1, 876, - 84, 1693, - -2, 2054, - -1, 887, - 84, 1574, - -2, 2234, + 85, 1695, + -2, 2032, + -1, 877, + 85, 1696, + -2, 2057, -1, 888, - 84, 1575, - -2, 2235, + 85, 1576, + -2, 2237, -1, 889, - 84, 1576, - -2, 2236, - -1, 989, - 465, 649, + 85, 1577, + -2, 2238, + -1, 890, + 85, 1578, + -2, 2239, + -1, 990, 466, 649, + 467, 649, -2, 615, - -1, 1040, - 126, 1867, - 137, 1867, - 157, 1867, - -2, 1841, - -1, 1157, + -1, 1041, + 127, 1870, + 138, 1870, + 158, 1870, + -2, 1844, + -1, 1158, 22, 818, -2, 767, - -1, 1267, + -1, 1268, 11, 791, 22, 791, - -2, 1451, - -1, 1351, + -2, 1453, + -1, 1352, 22, 818, -2, 767, - -1, 1705, - 84, 1740, - -2, 2036, -1, 1706, - 84, 1741, - -2, 2037, - -1, 1878, - 85, 982, - -2, 988, - -1, 2330, - 109, 1149, - 153, 1149, - 192, 1149, - 195, 1149, - 284, 1149, - -2, 1142, - -1, 2489, + 85, 1743, + -2, 2039, + -1, 1707, + 85, 1744, + -2, 2040, + -1, 1879, + 86, 984, + -2, 990, + -1, 2331, + 110, 1151, + 154, 1151, + 193, 1151, + 196, 1151, + 285, 1151, + -2, 1144, + -1, 2492, 11, 791, 22, 791, - -2, 923, - -1, 2523, - 85, 1827, - 158, 1827, - -2, 2021, - -1, 2524, - 85, 1827, - 158, 1827, - -2, 2020, - -1, 2525, - 85, 1803, - 158, 1803, - -2, 2007, + -2, 925, -1, 2526, - 85, 1804, - 158, 1804, - -2, 2012, + 86, 1830, + 159, 1830, + -2, 2024, -1, 2527, - 85, 1805, - 158, 1805, - -2, 1936, + 86, 1830, + 159, 1830, + -2, 2023, -1, 2528, - 85, 1806, - 158, 1806, - -2, 1930, + 86, 1806, + 159, 1806, + -2, 2010, -1, 2529, - 85, 1807, - 158, 1807, - -2, 1857, + 86, 1807, + 159, 1807, + -2, 2015, -1, 2530, - 85, 1808, - 158, 1808, - -2, 2009, + 86, 1808, + 159, 1808, + -2, 1939, -1, 2531, - 85, 1809, - 158, 1809, - -2, 1934, + 86, 1809, + 159, 1809, + -2, 1933, -1, 2532, - 85, 1810, - 158, 1810, - -2, 1929, + 86, 1810, + 159, 1810, + -2, 1860, -1, 2533, - 85, 1811, - 158, 1811, - -2, 1917, + 86, 1811, + 159, 1811, + -2, 2012, -1, 2534, - 85, 1827, - 158, 1827, - -2, 1918, + 86, 1812, + 159, 1812, + -2, 1937, -1, 2535, - 85, 1827, - 158, 1827, - -2, 1919, + 86, 1813, + 159, 1813, + -2, 1932, + -1, 2536, + 86, 1814, + 159, 1814, + -2, 1920, -1, 2537, - 85, 1816, - 158, 1816, - -2, 2054, + 86, 1830, + 159, 1830, + -2, 1921, -1, 2538, - 85, 1793, - 158, 1793, - -2, 2039, - -1, 2539, - 85, 1825, - 158, 1825, - -2, 2010, + 86, 1830, + 159, 1830, + -2, 1922, -1, 2540, - 85, 1825, - 158, 1825, - -2, 2038, + 86, 1819, + 159, 1819, + -2, 2057, -1, 2541, - 85, 1825, - 158, 1825, - -2, 1885, + 86, 1796, + 159, 1796, + -2, 2042, -1, 2542, - 85, 1823, - 158, 1823, - -2, 2029, + 86, 1828, + 159, 1828, + -2, 2013, -1, 2543, - 85, 1820, - 158, 1820, - -2, 1908, + 86, 1828, + 159, 1828, + -2, 2041, -1, 2544, - 84, 1774, - 85, 1774, - 158, 1774, - 400, 1774, - 401, 1774, - 402, 1774, - -2, 1856, + 86, 1828, + 159, 1828, + -2, 1888, -1, 2545, - 84, 1775, - 85, 1775, - 158, 1775, - 400, 1775, - 401, 1775, - 402, 1775, - -2, 1858, + 86, 1826, + 159, 1826, + -2, 2032, -1, 2546, - 84, 1776, - 85, 1776, - 158, 1776, - 400, 1776, - 401, 1776, - 402, 1776, - -2, 2082, + 86, 1823, + 159, 1823, + -2, 1911, -1, 2547, - 84, 1778, + 85, 1777, + 86, 1777, + 159, 1777, + 401, 1777, + 402, 1777, + 403, 1777, + -2, 1859, + -1, 2548, 85, 1778, - 158, 1778, - 400, 1778, + 86, 1778, + 159, 1778, 401, 1778, 402, 1778, - -2, 2011, - -1, 2548, - 84, 1780, - 85, 1780, - 158, 1780, - 400, 1780, - 401, 1780, - 402, 1780, - -2, 1991, + 403, 1778, + -2, 1861, -1, 2549, - 84, 1782, - 85, 1782, - 158, 1782, - 400, 1782, - 401, 1782, - 402, 1782, - -2, 1935, + 85, 1779, + 86, 1779, + 159, 1779, + 401, 1779, + 402, 1779, + 403, 1779, + -2, 2085, -1, 2550, - 84, 1784, - 85, 1784, - 158, 1784, - 400, 1784, - 401, 1784, - 402, 1784, - -2, 1913, + 85, 1781, + 86, 1781, + 159, 1781, + 401, 1781, + 402, 1781, + 403, 1781, + -2, 2014, -1, 2551, - 84, 1785, + 85, 1783, + 86, 1783, + 159, 1783, + 401, 1783, + 402, 1783, + 403, 1783, + -2, 1994, + -1, 2552, 85, 1785, - 158, 1785, - 400, 1785, + 86, 1785, + 159, 1785, 401, 1785, 402, 1785, - -2, 1914, - -1, 2552, - 84, 1787, + 403, 1785, + -2, 1938, + -1, 2553, 85, 1787, - 158, 1787, - 400, 1787, + 86, 1787, + 159, 1787, 401, 1787, 402, 1787, - -2, 1855, - -1, 2553, - 85, 1830, - 158, 1830, - 400, 1830, - 401, 1830, - 402, 1830, - -2, 1890, + 403, 1787, + -2, 1916, -1, 2554, - 85, 1830, - 158, 1830, - 400, 1830, - 401, 1830, - 402, 1830, - -2, 1904, + 85, 1788, + 86, 1788, + 159, 1788, + 401, 1788, + 402, 1788, + 403, 1788, + -2, 1917, -1, 2555, - 85, 1833, - 158, 1833, - 400, 1833, - 401, 1833, - 402, 1833, - -2, 1886, + 85, 1790, + 86, 1790, + 159, 1790, + 401, 1790, + 402, 1790, + 403, 1790, + -2, 1858, -1, 2556, - 85, 1833, - 158, 1833, - 400, 1833, + 86, 1833, + 159, 1833, 401, 1833, 402, 1833, - -2, 1951, + 403, 1833, + -2, 1893, -1, 2557, - 85, 1830, - 158, 1830, - 400, 1830, - 401, 1830, - 402, 1830, - -2, 1973, - -1, 2774, - 109, 1149, - 153, 1149, - 192, 1149, - 195, 1149, - 284, 1149, - -2, 1143, - -1, 2792, - 82, 711, - 158, 711, - -2, 1327, - -1, 3214, - 195, 1149, - 308, 1414, - -2, 1386, - -1, 3395, - 109, 1149, - 153, 1149, - 192, 1149, - 195, 1149, - -2, 1267, - -1, 3397, - 109, 1149, - 153, 1149, - 192, 1149, - 195, 1149, - -2, 1267, - -1, 3409, - 82, 711, - 158, 711, - -2, 1327, - -1, 3430, - 195, 1149, - 308, 1414, - -2, 1387, - -1, 3582, - 109, 1149, - 153, 1149, - 192, 1149, - 195, 1149, - -2, 1268, - -1, 3610, - 85, 1229, - 158, 1229, - -2, 1149, - -1, 3754, - 85, 1229, - 158, 1229, - -2, 1149, - -1, 3918, - 85, 1233, - 158, 1233, - -2, 1149, - -1, 3969, - 85, 1234, - 158, 1234, - -2, 1149, + 86, 1833, + 159, 1833, + 401, 1833, + 402, 1833, + 403, 1833, + -2, 1907, + -1, 2558, + 86, 1836, + 159, 1836, + 401, 1836, + 402, 1836, + 403, 1836, + -2, 1889, + -1, 2559, + 86, 1836, + 159, 1836, + 401, 1836, + 402, 1836, + 403, 1836, + -2, 1954, + -1, 2560, + 86, 1833, + 159, 1833, + 401, 1833, + 402, 1833, + 403, 1833, + -2, 1976, + -1, 2777, + 110, 1151, + 154, 1151, + 193, 1151, + 196, 1151, + 285, 1151, + -2, 1145, + -1, 2795, + 83, 711, + 159, 711, + -2, 1329, + -1, 3219, + 196, 1151, + 309, 1416, + -2, 1388, + -1, 3401, + 110, 1151, + 154, 1151, + 193, 1151, + 196, 1151, + -2, 1269, + -1, 3403, + 110, 1151, + 154, 1151, + 193, 1151, + 196, 1151, + -2, 1269, + -1, 3415, + 83, 711, + 159, 711, + -2, 1329, + -1, 3436, + 196, 1151, + 309, 1416, + -2, 1389, + -1, 3588, + 110, 1151, + 154, 1151, + 193, 1151, + 196, 1151, + -2, 1270, + -1, 3616, + 86, 1231, + 159, 1231, + -2, 1151, + -1, 3760, + 86, 1231, + 159, 1231, + -2, 1151, + -1, 3924, + 86, 1235, + 159, 1235, + -2, 1151, + -1, 3975, + 86, 1236, + 159, 1236, + -2, 1151, } const yyPrivate = 57344 -const yyLast = 52969 +const yyLast = 52977 var yyAct = [...]int{ - 767, 743, 4019, 769, 3991, 2822, 217, 1963, 1607, 4011, - 3922, 3928, 1685, 3415, 3514, 3929, 3233, 3820, 3921, 3754, - 752, 3200, 3843, 3801, 3878, 3732, 3307, 3702, 3638, 3444, - 745, 2816, 3792, 1303, 3308, 3753, 3821, 3570, 2734, 1745, - 3671, 797, 2819, 3518, 1039, 3723, 633, 1158, 1451, 3382, - 3377, 3802, 3804, 1911, 1457, 2795, 3209, 3509, 2377, 3171, - 651, 1519, 657, 657, 1747, 1732, 3431, 1681, 657, 675, - 684, 3579, 3552, 684, 1688, 1152, 3591, 3133, 3157, 3398, - 3305, 2934, 3584, 202, 37, 2935, 2058, 3160, 1750, 2061, - 3369, 2933, 2911, 741, 2074, 2845, 3229, 2023, 3218, 65, - 3211, 2930, 3400, 2097, 2483, 3349, 3001, 2130, 2172, 2647, - 3293, 2519, 2380, 3273, 1923, 2763, 3138, 2922, 2960, 1512, - 3140, 3134, 3136, 692, 2612, 696, 3135, 3180, 2341, 1148, - 2775, 2286, 2309, 3131, 137, 735, 3108, 3048, 2168, 2285, - 2591, 1592, 2155, 1596, 2139, 3217, 2974, 36, 740, 2138, - 2131, 1843, 2984, 681, 2103, 1597, 2573, 2054, 2167, 2484, - 2027, 2469, 1417, 2024, 961, 1600, 2521, 2752, 2747, 2847, - 2464, 2827, 1608, 2378, 1953, 2787, 1887, 2340, 1423, 213, - 8, 212, 7, 1679, 2330, 6, 1033, 2517, 1384, 1585, - 1746, 2169, 1559, 633, 1096, 744, 1497, 2321, 1491, 1440, - 650, 1628, 1922, 1460, 734, 2684, 1739, 1719, 1528, 753, - 1670, 1174, 2202, 2137, 23, 2324, 2134, 217, 2119, 217, - 1684, 1087, 1088, 2373, 2093, 1611, 689, 1883, 657, 1566, - 1678, 1032, 632, 998, 2491, 1886, 1452, 960, 2465, 1427, - 1436, 666, 27, 742, 1496, 1550, 1751, 111, 1493, 1461, - 891, 16, 203, 698, 24, 699, 937, 1558, 195, 984, - 1373, 17, 10, 943, 199, 1304, 2683, 683, 1349, 958, - 893, 695, 14, 894, 2410, 1235, 1236, 1237, 1234, 15, - 2176, 1235, 1236, 1237, 1234, 3811, 33, 1235, 1236, 1237, - 1234, 3720, 2720, 968, 653, 2720, 1048, 2720, 2493, 1084, - 1083, 3018, 1085, 3017, 3545, 951, 3412, 952, 736, 3187, - 2186, 680, 1153, 3385, 3300, 1154, 2635, 2579, 2576, 1856, - 676, 2577, 1573, 2574, 662, 1569, 1079, 1080, 201, 669, - 1066, 1045, 652, 2284, 1495, 913, 1368, 3779, 1342, 911, - 687, 678, 2290, 3118, 1047, 932, 1080, 1018, 679, 1620, - 1080, 1420, 1421, 1422, 2294, 677, 1857, 658, 1371, 946, - 3103, 942, 3101, 3098, 965, 966, 3100, 4003, 1474, 1850, - 1619, 1153, 1364, 1571, 3507, 1008, 2997, 1235, 1236, 1237, - 1234, 8, 2995, 7, 2712, 2710, 2108, 1078, 1235, 1236, - 1237, 1234, 3787, 3678, 3672, 3510, 3306, 2152, 3806, 2133, - 892, 1298, 1067, 736, 3077, 2125, 2418, 4025, 3800, 200, - 4000, 3739, 903, 2331, 3686, 200, 3524, 923, 1233, 3957, - 3553, 1379, 1197, 3399, 2332, 2621, 2173, 2714, 1606, 200, - 3557, 2781, 3798, 200, 912, 3707, 2629, 2665, 910, 200, - 200, 200, 200, 61, 191, 162, 3854, 3684, 1536, 200, - 61, 191, 162, 3075, 3904, 3740, 1378, 1615, 1010, 1626, - 1377, 1009, 1376, 136, 913, 911, 1043, 1049, 1409, 1044, - 1374, 694, 2928, 2325, 1392, 1061, 1056, 1051, 1055, 1059, - 2779, 3020, 1380, 2511, 1859, 3709, 196, 1612, 3009, 1623, - 948, 200, 941, 1013, 1011, 2184, 1012, 1232, 2512, 994, - 196, 945, 944, 1064, 196, 908, 2967, 1054, 969, 2071, - 1614, 196, 1625, 196, 1470, 2968, 2969, 1471, 926, 1651, - 196, 904, 933, 1671, 2037, 882, 1675, 881, 883, 884, - 2782, 885, 886, 2592, 2736, 971, 1498, 2498, 1500, 136, - 2497, 1007, 940, 2499, 3102, 3099, 2038, 2039, 1868, 1869, - 1674, 1458, 1459, 200, 61, 191, 162, 1225, 1062, 3932, - 3933, 950, 196, 1937, 2749, 1687, 939, 1065, 1230, 1448, - 938, 2273, 2737, 1042, 2750, 1041, 925, 200, 61, 191, - 162, 1456, 931, 3956, 1019, 1455, 1458, 1459, 3809, 1052, - 200, 61, 191, 162, 3809, 3892, 3808, 3891, 3807, 3890, - 993, 991, 3808, 3807, 929, 1473, 1015, 3204, 1391, 3995, - 3996, 3897, 3880, 1063, 3534, 3901, 3202, 1169, 1572, 1570, - 3790, 3309, 990, 2748, 196, 3880, 3793, 3794, 3795, 3796, - 3883, 3003, 3002, 3004, 964, 2715, 1676, 3309, 2616, 1177, - 1180, 1637, 949, 3675, 1163, 970, 1003, 2188, 196, 200, - 61, 191, 162, 1053, 1166, 657, 657, 3817, 2045, 1691, - 1673, 196, 161, 1660, 198, 2055, 657, 1162, 930, 999, - 1017, 3322, 2755, 1172, 3153, 3370, 1177, 1180, 712, 711, - 718, 708, 1666, 2180, 2923, 188, 684, 684, 2453, 657, - 715, 716, 3038, 717, 721, 2866, 3562, 702, 3906, 3907, - 2459, 3375, 2320, 3711, 3712, 1000, 1004, 726, 2116, 1579, - 1578, 3902, 3903, 2738, 949, 2739, 3931, 1228, 1229, 3151, - 196, 3456, 3899, 3036, 693, 987, 1227, 985, 989, 1007, - 1060, 1161, 2626, 986, 983, 982, 187, 988, 973, 974, - 972, 975, 976, 977, 978, 947, 1005, 1016, 1006, 2416, - 2713, 2185, 1275, 1484, 3533, 681, 681, 681, 1367, 1001, - 1002, 1393, 3535, 2180, 2069, 2070, 1057, 1200, 1472, 1058, - 1446, 1223, 1224, 1048, 3508, 3148, 3149, 1672, 1690, 1689, - 3810, 2996, 2456, 2457, 936, 2917, 3719, 3325, 1155, 3716, - 3042, 3150, 2719, 2049, 2455, 3559, 997, 3147, 1090, 1162, - 3353, 1154, 996, 2732, 1154, 906, 3665, 1154, 1045, 2291, - 2174, 2462, 2163, 2174, 2174, 1621, 3019, 992, 2514, 1192, - 1222, 1047, 3016, 1858, 3158, 649, 3525, 3471, 2207, 1307, - 3964, 3232, 3230, 3231, 2191, 2193, 2194, 3169, 1179, 1178, - 3181, 2733, 1080, 907, 3206, 1080, 1048, 2175, 730, 1080, - 1080, 732, 3468, 1080, 1080, 3744, 731, 3738, 3836, 3831, - 1697, 1700, 1701, 1308, 2788, 2913, 1154, 3736, 686, 1068, - 1050, 1698, 2187, 1171, 2448, 1179, 1178, 682, 685, 2926, - 2327, 1045, 2575, 3461, 924, 922, 1014, 3109, 3822, 1181, - 2383, 2821, 3666, 1574, 1047, 995, 3838, 3201, 703, 705, - 704, 967, 962, 3416, 3844, 3423, 3905, 963, 1370, 710, - 1372, 1435, 3145, 680, 680, 680, 3685, 3710, 3362, 1269, - 2305, 714, 676, 676, 676, 892, 1389, 651, 729, 3706, - 3360, 3122, 1157, 1156, 2451, 707, 1044, 3159, 1347, 62, - 3472, 1352, 163, 678, 678, 678, 1189, 2711, 163, 1150, - 679, 679, 679, 1185, 1186, 3558, 3816, 677, 677, 677, - 961, 2630, 163, 1860, 1276, 1191, 163, 1661, 197, 3629, - 1662, 3235, 163, 163, 163, 163, 1212, 1168, 1375, 1213, - 2514, 682, 163, 4031, 909, 2428, 3361, 2754, 1458, 1459, - 3154, 1271, 1272, 1273, 1274, 2427, 1447, 1458, 1459, 2924, - 1165, 1167, 1170, 3521, 2056, 682, 3694, 1215, 3695, 3039, - 2817, 2818, 1183, 2821, 657, 3159, 3745, 1486, 682, 1217, - 3713, 657, 1218, 3618, 163, 633, 633, 3898, 3737, 2382, - 2761, 2396, 1454, 1508, 2384, 633, 633, 2376, 2399, 1523, - 1523, 1507, 657, 62, 2758, 2759, 1190, 709, 713, 719, - 1220, 720, 722, 1433, 2046, 723, 724, 725, 3624, 2757, - 727, 728, 3697, 684, 1551, 651, 1205, 62, 3563, 1207, - 1562, 1562, 2449, 2450, 3207, 1319, 1320, 682, 1667, 2192, - 62, 217, 2867, 1432, 2868, 2869, 163, 1431, 2385, 1210, - 633, 1530, 4014, 3696, 3845, 2398, 3724, 1208, 1450, 1449, - 3210, 1525, 1008, 3758, 1149, 1699, 1521, 1521, 3920, 3097, - 163, 2419, 1386, 1387, 951, 3146, 952, 3401, 1396, 1397, - 1398, 1399, 1400, 163, 1402, 2979, 2980, 1390, 1266, 2376, - 1408, 1485, 1216, 2767, 2770, 2771, 2772, 2768, 2769, 62, - 2397, 3505, 1604, 3312, 1426, 1385, 3167, 1609, 694, 3230, - 3231, 1434, 2393, 1211, 1618, 3877, 1517, 1518, 1444, 1353, - 2962, 2964, 1351, 1494, 3234, 3226, 1463, 1464, 3113, 1466, - 1467, 1221, 1468, 2622, 1197, 2503, 2414, 1442, 1443, 1201, - 2177, 2044, 163, 1649, 2021, 1010, 2386, 1401, 1009, 2048, - 3264, 3041, 1395, 2725, 1407, 1406, 1219, 1523, 1405, 1523, - 1162, 1862, 1404, 1020, 1627, 1203, 2304, 1429, 688, 3363, - 3631, 1502, 1504, 955, 956, 957, 706, 1206, 1209, 2189, - 2190, 1515, 1516, 1419, 3227, 1437, 1441, 1441, 1441, 1416, - 1214, 4015, 2864, 1613, 3757, 2203, 3350, 1644, 1645, 1580, - 1624, 1475, 1476, 1202, 3050, 3049, 1462, 1414, 1048, 1465, - 1437, 1437, 2729, 681, 2298, 1048, 681, 681, 953, 918, - 1594, 1595, 1196, 1383, 1686, 1552, 1871, 1523, 3694, 1659, - 3695, 2300, 2299, 1381, 1382, 3620, 1575, 3168, 1872, 3619, - 1617, 2886, 2887, 1506, 1162, 1749, 3689, 1583, 3919, 1586, - 1587, 1008, 3543, 1599, 3625, 3626, 1603, 1780, 1781, 1798, - 1784, 1588, 1589, 1394, 3115, 1602, 2297, 662, 1799, 1531, - 917, 1537, 1870, 1543, 920, 919, 914, 2440, 3690, 2387, - 1204, 1806, 3803, 1808, 3697, 1809, 1810, 1811, 1563, 1648, - 1549, 2963, 915, 1564, 1733, 2413, 3592, 1683, 1647, 1008, - 4027, 4033, 1707, 1708, 1709, 1710, 1711, 1712, 1713, 1714, - 1715, 1716, 1717, 1718, 2895, 3696, 1771, 3887, 1730, 1731, - 1233, 3270, 2793, 2392, 4012, 4013, 1162, 2390, 1428, 1428, - 950, 2514, 1861, 3266, 1010, 3313, 1864, 1009, 1866, 1783, - 1197, 1635, 2594, 1664, 1638, 1873, 3366, 1875, 1876, 1702, - 4021, 4009, 1551, 3186, 2481, 1630, 1841, 1884, 1523, 1889, - 1890, 3324, 1892, 1486, 657, 3971, 2885, 1807, 2278, 657, - 2312, 680, 1523, 2182, 680, 680, 961, 1680, 2726, 1912, - 676, 2621, 1010, 676, 676, 1009, 1863, 1657, 3228, 3239, - 1852, 1523, 1658, 2313, 2314, 2241, 1654, 1486, 2240, 1656, - 1844, 678, 675, 1797, 678, 678, 1655, 1652, 679, 1677, - 1021, 679, 679, 1682, 3943, 677, 3940, 1653, 677, 677, - 3934, 2096, 1936, 4022, 3972, 3237, 1788, 1789, 1790, 2794, - 1721, 1943, 1943, 1159, 1486, 3107, 1486, 1486, 3972, 1804, - 657, 657, 1805, 2010, 1884, 2014, 2323, 3916, 1523, 2018, - 2019, 1159, 3871, 3870, 2034, 3105, 633, 3864, 1636, 1818, - 1819, 1639, 1640, 2794, 3839, 3827, 1728, 1729, 2482, 3270, - 633, 1233, 1523, 1235, 1236, 1237, 1234, 1893, 3777, 1840, - 1940, 2482, 2482, 1767, 3776, 3771, 1891, 3944, 3770, 3941, - 1764, 2982, 2741, 2217, 1766, 1763, 1765, 1769, 1770, 657, - 1884, 1523, 1768, 2079, 1668, 657, 657, 657, 692, 692, - 3769, 2716, 3768, 1847, 1194, 2089, 2090, 2091, 2092, 3073, - 3917, 2611, 2098, 2599, 1965, 1233, 1233, 2173, 1812, 217, - 2217, 1669, 217, 217, 2036, 217, 2012, 2182, 3828, 2072, - 3690, 1880, 1881, 1882, 3691, 2369, 2283, 2277, 1946, 1848, - 2276, 3778, 3748, 1895, 1896, 1897, 1898, 2345, 2217, 1842, - 3747, 2217, 2094, 2322, 2064, 2065, 2896, 2898, 2899, 2900, - 2897, 1920, 1921, 2248, 2164, 1798, 1798, 2141, 1888, 2067, - 1233, 1195, 2041, 2217, 2043, 2217, 1798, 1798, 1930, 1931, - 2020, 1195, 1904, 2157, 2050, 2062, 2063, 1879, 1348, 2081, - 2082, 2083, 3722, 3477, 1924, 1929, 1926, 1927, 1941, 1415, - 2057, 1917, 2107, 1909, 1736, 2110, 2111, 1934, 2113, 2032, - 1933, 1945, 1908, 1913, 1509, 2182, 1912, 2078, 1914, 1915, - 1523, 2171, 1919, 2182, 2383, 2386, 2151, 4040, 1947, 1948, - 4023, 1437, 2035, 1925, 1928, 3425, 1774, 1775, 1776, 1777, - 1778, 1779, 1772, 1773, 2143, 1441, 3412, 2986, 2796, 3391, - 1935, 2624, 1613, 1938, 1939, 2383, 2386, 1441, 2017, 1942, - 1944, 2011, 1197, 2623, 1048, 2217, 2514, 1048, 2615, 2165, - 2363, 656, 656, 2016, 2236, 1048, 681, 664, 2022, 3342, - 2221, 2147, 2162, 2101, 2356, 2087, 1479, 1480, 737, 1482, - 1483, 2051, 1487, 1488, 1489, 1632, 2040, 1284, 2042, 1045, - 3639, 3640, 3641, 3645, 3643, 3644, 3642, 3338, 3426, 2136, - 1045, 1680, 1047, 1182, 1071, 1076, 1077, 2076, 3247, 2655, - 2136, 2077, 3392, 1047, 2084, 2085, 1538, 1539, 1540, 1541, - 1542, 1146, 1544, 1545, 1546, 1547, 1548, 2957, 2102, 2702, - 1554, 1555, 1556, 1557, 1141, 2104, 3374, 1140, 1136, 1137, - 1138, 1139, 3343, 2660, 2690, 2659, 2658, 2656, 2387, 3655, - 2121, 3475, 2682, 2382, 2376, 2381, 1266, 2379, 2384, 2637, - 2200, 2201, 1048, 1235, 1236, 1237, 1234, 2066, 1250, 2371, - 3339, 2619, 2153, 2607, 3191, 2216, 1912, 2601, 3033, 2387, - 2150, 3248, 2148, 2142, 2382, 2376, 2381, 916, 2379, 2384, - 3832, 1424, 2159, 2596, 2161, 1425, 2588, 1045, 2288, 2289, - 2482, 2292, 2345, 2586, 2295, 1235, 1236, 1237, 1234, 2355, - 1047, 2584, 2385, 4034, 2657, 2582, 2344, 1233, 735, 2166, - 3999, 657, 657, 657, 680, 1233, 1511, 664, 2411, 2271, - 1438, 2219, 1233, 676, 3833, 2196, 657, 657, 657, 657, - 2179, 782, 138, 2385, 2345, 3812, 2597, 138, 3721, 2342, - 2602, 2215, 2279, 2195, 678, 896, 897, 898, 899, 2348, - 1486, 679, 896, 897, 898, 899, 2597, 3682, 677, 2589, - 3622, 2197, 3182, 1721, 1787, 1786, 2587, 3621, 1235, 1236, - 1237, 1234, 2204, 3752, 2583, 2209, 1486, 3607, 2583, 2345, - 2255, 2254, 2239, 1081, 1082, 2230, 2229, 1469, 1086, 1073, - 1074, 1075, 2228, 2405, 2218, 2181, 1641, 663, 2198, 2199, - 138, 2160, 1249, 1248, 1258, 1259, 1251, 1252, 1253, 1254, - 1255, 1256, 1257, 1250, 2213, 2278, 2249, 2250, 1510, 2252, - 3593, 2243, 1787, 1786, 1534, 921, 2259, 1249, 1248, 1258, - 1259, 1251, 1252, 1253, 1254, 1255, 1256, 1257, 1250, 1439, - 2412, 3183, 3404, 3402, 2360, 2661, 2662, 3566, 2362, 3384, - 2364, 657, 1943, 1233, 1233, 1233, 3271, 3262, 1233, 1233, - 2486, 2486, 2034, 2486, 3594, 1233, 3254, 2217, 2182, 1642, - 1513, 3249, 2272, 2274, 2275, 3162, 2920, 1824, 2919, 2765, - 2280, 1514, 2721, 633, 633, 3184, 3405, 3403, 2634, 1424, - 2365, 1162, 2600, 1425, 2505, 901, 2146, 1523, 657, 2145, - 2144, 1411, 901, 1253, 1254, 1255, 1256, 1257, 1250, 2375, - 1410, 657, 2306, 2374, 1164, 2574, 3298, 1162, 2558, 651, - 2644, 1307, 2568, 2105, 1740, 1562, 2210, 2034, 1046, 1727, - 2563, 2988, 2565, 138, 1740, 1817, 217, 1567, 1874, 2105, - 1237, 1234, 2357, 2368, 3889, 1724, 1726, 1723, 138, 1725, - 138, 1234, 2349, 3634, 3633, 1308, 3005, 2856, 2500, 2490, - 2501, 1048, 2854, 2352, 2509, 2488, 2833, 2492, 2358, 2831, - 2361, 2359, 3604, 3567, 3568, 3613, 2604, 4005, 4004, 2506, - 2507, 2522, 4030, 1802, 3947, 2494, 1235, 1236, 1237, 1234, - 2350, 2351, 3560, 2617, 1286, 3299, 1045, 2171, 1803, 3915, - 2353, 2354, 3914, 2764, 1523, 2516, 1523, 1285, 1523, 1047, - 3834, 2388, 2389, 1162, 2394, 1235, 1236, 1237, 1234, 2569, - 1441, 2636, 2735, 1567, 2562, 3773, 1249, 1248, 1258, 1259, - 1251, 1252, 1253, 1254, 1255, 1256, 1257, 1250, 3761, 1235, - 1236, 1237, 1234, 2704, 2458, 2705, 4029, 1523, 1162, 2463, - 3301, 3561, 2668, 1258, 1259, 1251, 1252, 1253, 1254, 1255, - 1256, 1257, 1250, 2495, 3372, 2907, 3751, 2675, 3741, 1502, - 1504, 3673, 1523, 2627, 3596, 2417, 3595, 2631, 2420, 2421, - 2422, 2423, 2424, 2425, 2426, 3417, 3406, 2429, 2430, 2431, - 2432, 2433, 2434, 2435, 2436, 2437, 2438, 2439, 2080, 2441, - 2442, 2443, 2444, 2445, 2513, 2446, 2510, 3371, 2663, 1235, - 1236, 1237, 1234, 3250, 1521, 3152, 2905, 2561, 3029, 2559, - 2578, 2723, 2724, 3373, 2906, 2727, 2903, 2679, 2680, 1235, - 1236, 1237, 1234, 2676, 1235, 1236, 1237, 1234, 2646, 1521, - 2613, 2614, 1162, 2570, 656, 1151, 1162, 3000, 2892, 1235, - 1236, 1237, 1234, 1523, 2652, 1160, 1486, 2633, 1568, 2999, - 2890, 2889, 2014, 2560, 2888, 2880, 2628, 2677, 2874, 2674, - 2792, 2873, 2567, 3925, 2666, 2904, 2798, 2232, 1188, 1235, - 1236, 1237, 1234, 2609, 2642, 2902, 2872, 2625, 2620, 2871, - 2618, 1785, 2717, 2590, 2808, 2502, 2282, 2124, 2708, 2123, - 1235, 1236, 1237, 1234, 1162, 2122, 2742, 2891, 2118, 2648, - 2522, 2648, 2830, 2117, 2073, 1235, 1236, 1237, 1234, 1162, - 1162, 1162, 1943, 1867, 1680, 1162, 1865, 2840, 2841, 2842, - 2843, 1162, 2850, 1633, 2851, 2852, 1366, 2853, 2664, 2855, - 2654, 2638, 2639, 2776, 730, 2231, 1144, 732, 2780, 3378, - 2850, 3539, 731, 2777, 1251, 1252, 1253, 1254, 1255, 1256, - 1257, 1250, 2486, 1048, 1235, 1236, 1237, 1234, 2825, 3383, - 2762, 2809, 1235, 1236, 1237, 1234, 2908, 2641, 1235, 1236, - 1237, 1234, 1965, 2825, 2836, 2837, 633, 3139, 4026, 2839, - 3714, 3715, 4024, 2014, 3515, 2846, 3997, 1162, 2034, 2034, - 2034, 2034, 2034, 2799, 1143, 3963, 3962, 1238, 3959, 2744, - 3895, 2746, 1162, 2034, 3527, 1268, 2486, 3894, 3703, 1505, - 2914, 3875, 2828, 3819, 1278, 2824, 2828, 3066, 3571, 2760, - 3797, 3788, 2965, 2743, 1523, 2789, 3765, 3760, 3759, 3718, - 2835, 1235, 1236, 1237, 1234, 657, 657, 2791, 2783, 3705, - 1287, 2685, 2686, 8, 3704, 7, 3674, 2691, 2800, 2797, - 3615, 2936, 3575, 3564, 2811, 3546, 3544, 2805, 2806, 3541, - 3538, 3537, 2810, 1888, 2813, 3513, 2936, 3511, 3485, 3482, - 3602, 2826, 3479, 2912, 2224, 2832, 3368, 3358, 3065, 3052, - 3351, 3335, 2838, 2953, 3333, 2807, 3328, 3260, 3259, 2829, - 3257, 217, 138, 138, 138, 1046, 217, 3256, 3251, 3245, - 3244, 2966, 3163, 3126, 3125, 1235, 1236, 1237, 1234, 2870, - 3526, 2882, 2801, 3121, 3119, 2751, 3117, 2804, 1798, 3114, - 1798, 3112, 2287, 3015, 1249, 1248, 1258, 1259, 1251, 1252, - 1253, 1254, 1255, 1256, 1257, 1250, 3028, 1235, 1236, 1237, - 1234, 2915, 1523, 3043, 3040, 3035, 2921, 4032, 3465, 2998, - 2972, 2937, 2938, 2939, 2940, 2941, 1235, 1236, 1237, 1234, - 2901, 2951, 2918, 2954, 2893, 2956, 2955, 3330, 1267, 1235, - 1236, 1237, 1234, 2214, 2989, 1235, 1236, 1237, 1234, 2993, - 2883, 2881, 2970, 2973, 1241, 1242, 1243, 1244, 1245, 1246, - 1247, 1239, 2877, 1481, 1235, 1236, 1237, 1234, 2876, 2875, - 1492, 3010, 1594, 1595, 2730, 1844, 2722, 2862, 2863, 2983, - 3014, 2718, 3021, 838, 837, 2823, 1048, 2610, 2301, 2296, - 2293, 1529, 2878, 2879, 2127, 2120, 3012, 1048, 3985, 1855, - 1599, 1854, 1634, 1603, 1587, 1315, 3022, 1311, 1310, 1147, - 2987, 905, 1602, 3851, 1588, 1589, 2990, 3116, 2916, 2991, - 1235, 1236, 1237, 1234, 3120, 3847, 3069, 3037, 3123, 3124, - 3699, 3068, 3011, 3698, 3687, 3006, 1162, 3008, 3032, 3013, - 3683, 3024, 3142, 3540, 3025, 200, 3023, 191, 162, 3522, - 3031, 2212, 3156, 1235, 1236, 1237, 1234, 657, 1235, 1236, - 1237, 1234, 3397, 3396, 3395, 3044, 3365, 3347, 3345, 3172, - 1162, 3344, 3341, 657, 3067, 1162, 1162, 3340, 3051, 3045, - 2701, 3334, 3064, 3332, 2034, 2342, 1354, 3190, 3314, 3060, - 3061, 3058, 3057, 3304, 3059, 3106, 3303, 3289, 1561, 1561, - 2825, 1235, 1236, 1237, 1234, 3288, 2405, 1235, 1236, 1237, - 1234, 3166, 3192, 3129, 3104, 3071, 196, 3984, 3216, 3062, - 3219, 3054, 3219, 3219, 3053, 3047, 2981, 1162, 1235, 1236, - 1237, 1234, 2740, 2585, 2825, 3055, 3056, 2581, 2700, 2825, - 2825, 2580, 3111, 2260, 2253, 3240, 3110, 2776, 2247, 2246, - 3236, 2245, 2244, 1523, 1523, 2699, 3175, 2242, 2238, 2237, - 3144, 3179, 3203, 3205, 3127, 1235, 1236, 1237, 1234, 1048, - 3194, 1048, 2235, 2226, 2223, 2222, 1048, 2126, 1483, 1838, - 1837, 3188, 1235, 1236, 1237, 1234, 1836, 3199, 1801, 1800, - 1791, 2825, 1535, 1533, 3946, 3165, 3869, 1305, 200, 3128, - 657, 3846, 1048, 3174, 1045, 2698, 3783, 3142, 3177, 3178, - 3185, 3780, 3767, 3189, 3241, 3242, 3215, 1047, 3762, 1486, - 1521, 1521, 2014, 2014, 3668, 3667, 3224, 2375, 3198, 3649, - 3632, 2374, 1235, 1236, 1237, 1234, 1532, 3628, 3078, 3079, - 663, 3606, 3590, 3495, 3080, 3081, 3082, 3083, 3225, 3084, - 3085, 3086, 3087, 3088, 3089, 3090, 3091, 3092, 3093, 3094, - 2697, 3238, 1692, 1693, 1694, 1695, 1696, 3220, 3221, 196, - 3493, 3463, 138, 1162, 3462, 3459, 3197, 2668, 3458, 3424, - 3421, 3419, 3214, 770, 780, 3386, 3302, 1235, 1236, 1237, - 1234, 3063, 1582, 771, 3246, 772, 776, 779, 775, 773, - 774, 1593, 1584, 1598, 1737, 1601, 1590, 1418, 1741, 1742, - 1743, 1744, 3222, 2909, 2834, 2785, 2784, 2778, 1782, 2745, - 2703, 2595, 3252, 3267, 3268, 2504, 1792, 3253, 3255, 3258, - 3193, 3261, 657, 3265, 2447, 3195, 3196, 2522, 3072, 2343, - 138, 3863, 2696, 3278, 2315, 2281, 1722, 138, 777, 196, - 2086, 1878, 1851, 1894, 1665, 1616, 1591, 1365, 1899, 1350, - 138, 1346, 1345, 138, 138, 3285, 3286, 3287, 3282, 1235, - 1236, 1237, 1234, 1344, 1343, 1342, 138, 1341, 1340, 1845, - 778, 1339, 1338, 1337, 3291, 2695, 1336, 1335, 3297, 1334, - 1333, 1332, 1249, 1248, 1258, 1259, 1251, 1252, 1253, 1254, - 1255, 1256, 1257, 1250, 1331, 2098, 3355, 3315, 1330, 3357, - 2640, 1329, 1235, 1236, 1237, 1234, 1328, 1327, 3316, 1326, - 1325, 3317, 1324, 1323, 1322, 2952, 2694, 1321, 1318, 1949, - 1950, 3321, 1317, 3336, 1249, 1248, 1258, 1259, 1251, 1252, - 1253, 1254, 1255, 1256, 1257, 1250, 3320, 1316, 1314, 657, - 2014, 3359, 3326, 1235, 1236, 1237, 1234, 1916, 1313, 1312, - 3390, 3977, 2693, 1309, 3269, 1248, 1258, 1259, 1251, 1252, - 1253, 1254, 1255, 1256, 1257, 1250, 2486, 2034, 3409, 1302, - 1301, 1299, 1932, 3281, 1298, 1297, 3348, 1296, 2075, 1235, - 1236, 1237, 1234, 1295, 2075, 2075, 2075, 1294, 1293, 1292, - 1291, 3427, 1290, 1289, 1162, 3364, 3352, 1288, 3354, 2648, - 2692, 1283, 3367, 3216, 1282, 1281, 1280, 1162, 1279, 1199, - 1048, 1145, 3274, 3275, 3861, 3859, 3857, 1048, 1162, 3460, - 3474, 2347, 2329, 1187, 1523, 3975, 3411, 1235, 1236, 1237, - 1234, 1845, 2689, 3379, 3280, 3930, 1845, 1845, 2688, 3277, - 2766, 3381, 2515, 2129, 1198, 657, 2944, 2014, 3497, 2948, - 2946, 1162, 3279, 3457, 2949, 2947, 3498, 2943, 3428, 1235, - 1236, 1237, 1234, 2945, 3408, 1235, 1236, 1237, 1234, 2942, - 3407, 3467, 2950, 3888, 2477, 2478, 3418, 3450, 3420, 121, - 217, 3414, 2846, 64, 3799, 3476, 2106, 63, 3611, 2109, - 2608, 1521, 2112, 1162, 3486, 2114, 2598, 3489, 2687, 1412, - 3161, 3464, 3027, 3499, 3469, 3496, 3466, 1906, 1907, 1901, - 1902, 1903, 3212, 3473, 3213, 2936, 3318, 3319, 2415, 3480, - 3470, 3478, 3483, 3292, 3481, 1235, 1236, 1237, 1234, 3484, - 2003, 2681, 1576, 2593, 3542, 2632, 3490, 1629, 3488, 3487, - 1610, 3491, 2302, 3549, 2088, 659, 1193, 1162, 3520, 660, - 2156, 2613, 2614, 661, 3137, 3130, 2812, 2936, 1235, 1236, - 1237, 1234, 2205, 3506, 2786, 2367, 2338, 1162, 1523, 1523, - 1910, 1877, 3988, 3172, 3516, 2033, 1787, 1786, 3517, 3547, - 3548, 1361, 1362, 3583, 3764, 3583, 1249, 1248, 1258, 1259, - 1251, 1252, 1253, 1254, 1255, 1256, 1257, 1250, 1359, 1360, - 1162, 3600, 1162, 3243, 3577, 3578, 3572, 2460, 3410, 1357, - 1358, 2825, 3603, 2454, 3605, 1355, 1356, 3413, 2015, 1523, - 3556, 3387, 3388, 3389, 3555, 3554, 1478, 3393, 3394, 3573, - 1477, 1226, 3576, 3284, 3574, 1521, 1733, 657, 3565, 1162, - 1162, 1453, 2975, 1162, 1162, 3411, 2303, 2158, 3501, 3588, - 138, 3587, 1430, 138, 138, 2206, 138, 1403, 3953, 2211, - 3580, 3951, 3651, 3609, 1686, 2143, 1686, 2220, 3908, 3457, - 3646, 3599, 3885, 3884, 1912, 1048, 3660, 3612, 3636, 3637, - 3882, 3823, 3647, 3648, 3616, 3784, 1733, 3663, 3669, 3670, - 3536, 3662, 3294, 3450, 3601, 3512, 1046, 2985, 3608, 138, - 3337, 1523, 2671, 3311, 2227, 3310, 3295, 1046, 3614, 1428, - 3551, 2400, 2234, 2370, 3656, 1631, 3979, 3978, 3657, 3356, - 2316, 2317, 2318, 138, 3700, 3030, 2728, 3658, 2331, 1235, - 1236, 1237, 1234, 3693, 2251, 2333, 2334, 2335, 2336, 2256, - 2257, 2258, 3652, 3978, 2261, 2262, 2263, 2264, 2265, 2266, - 2267, 2268, 2269, 2270, 3680, 3676, 2667, 2225, 1369, 1184, - 3979, 2643, 3681, 1159, 3688, 3692, 961, 2858, 1521, 3630, - 3727, 3733, 3290, 1735, 2859, 2860, 2861, 204, 3, 3528, - 1445, 3529, 72, 1235, 1236, 1237, 1234, 1162, 1235, 1236, - 1237, 1234, 896, 897, 898, 899, 3717, 1159, 3756, 3750, - 1235, 1236, 1237, 1234, 1267, 2, 4001, 4002, 1, 3728, - 2709, 3729, 3520, 1849, 3730, 1363, 2472, 2476, 2477, 2478, - 2473, 2480, 2474, 2479, 3746, 3742, 2475, 3597, 3598, 900, - 1162, 895, 1499, 2496, 2068, 1523, 1527, 1853, 902, 2958, - 2959, 3283, 2961, 2731, 2178, 2925, 3725, 2452, 2319, 3155, - 1413, 1686, 3763, 954, 1793, 1646, 1070, 1176, 1643, 1175, - 1492, 1048, 1173, 1738, 1813, 1814, 1815, 1816, 784, 3772, - 1820, 1821, 1822, 1823, 1825, 1826, 1827, 1828, 1829, 1830, - 1831, 1832, 1833, 1834, 1835, 2132, 2910, 2884, 3815, 3659, - 3805, 3987, 4018, 3945, 3990, 1663, 3774, 3785, 768, 3876, - 3789, 3949, 1521, 3791, 3679, 2183, 1231, 1529, 3007, 980, - 825, 795, 1300, 1622, 3824, 3076, 3074, 1072, 794, 3376, - 2075, 2756, 3664, 2978, 3735, 3813, 3653, 1069, 981, 2115, - 3654, 3786, 3677, 1577, 1581, 2366, 3743, 3842, 3818, 3610, - 3208, 2820, 3841, 1605, 3837, 3422, 3532, 3826, 1162, 3530, - 3531, 700, 2047, 631, 1030, 3650, 1523, 2128, 701, 3866, - 1845, 2346, 1845, 3900, 3873, 3766, 3856, 3858, 3860, 3862, - 3840, 3835, 934, 2328, 935, 927, 2774, 3849, 2773, 3874, - 1703, 1845, 1845, 1240, 1720, 3095, 3096, 3855, 1249, 1248, - 1258, 1259, 1251, 1252, 1253, 1254, 1255, 1256, 1257, 1250, - 1277, 739, 2208, 2753, 3881, 3879, 1523, 2466, 3445, 3733, - 2971, 71, 70, 1561, 69, 68, 225, 3865, 786, 3893, - 3781, 3782, 224, 1521, 3701, 3918, 3569, 3872, 3992, 765, - 764, 3926, 763, 762, 3911, 3909, 761, 760, 2471, 2470, - 3912, 3913, 2468, 2467, 2472, 2476, 2477, 2478, 2473, 2480, - 2474, 2479, 2029, 2028, 2475, 2095, 3170, 2849, 3942, 2844, - 1954, 1952, 1490, 2603, 2395, 2606, 3935, 3910, 3936, 2402, - 3937, 1951, 3938, 1521, 3939, 3927, 3327, 3952, 3523, 3954, - 3955, 3852, 3853, 3627, 3950, 3948, 2894, 3519, 1900, 1162, - 3805, 2391, 1971, 3958, 2865, 1968, 1967, 2857, 3623, 3617, - 1999, 3731, 3582, 3429, 3430, 3436, 3756, 2337, 1095, 1091, - 3775, 3967, 1771, 2489, 1093, 1094, 1092, 2653, 3969, 3970, - 3968, 3976, 3973, 3263, 3974, 3986, 2372, 3994, 3132, 2645, - 3993, 2311, 2651, 3980, 3981, 3982, 3983, 2310, 2308, 2307, - 1388, 3814, 3896, 2669, 2670, 4006, 3550, 1162, 3998, 2520, - 2518, 2672, 2673, 3965, 1142, 3276, 3272, 3841, 2140, 4007, - 2154, 4008, 3026, 4010, 2030, 2026, 2025, 2678, 2927, 4016, - 4020, 2461, 3708, 4017, 1905, 928, 2326, 41, 2033, 2790, - 118, 105, 179, 3825, 56, 178, 55, 138, 3829, 3830, - 116, 176, 54, 100, 4028, 99, 1692, 1845, 115, 174, - 53, 209, 208, 3994, 4036, 211, 3993, 4035, 210, 207, - 2571, 1686, 2572, 206, 1565, 4020, 4037, 205, 3886, 3850, - 3586, 4041, 3868, 890, 44, 43, 180, 42, 106, 57, - 40, 39, 38, 200, 61, 191, 162, 34, 13, 12, - 35, 22, 21, 1650, 20, 26, 32, 200, 61, 191, - 162, 192, 31, 131, 130, 30, 129, 128, 183, 127, - 126, 125, 193, 124, 123, 192, 29, 19, 48, 47, - 46, 9, 183, 119, 2802, 2803, 193, 114, 112, 1767, - 28, 136, 113, 110, 109, 108, 1764, 107, 103, 101, - 1766, 1763, 1765, 1769, 1770, 136, 122, 1261, 1768, 1265, - 83, 82, 81, 96, 196, 95, 94, 93, 92, 91, - 122, 89, 90, 979, 80, 1262, 1264, 1260, 196, 1263, - 1249, 1248, 1258, 1259, 1251, 1252, 1253, 1254, 1255, 1256, - 1257, 1250, 79, 78, 77, 76, 98, 104, 102, 87, - 97, 88, 86, 85, 84, 75, 74, 73, 160, 159, - 158, 157, 156, 154, 2976, 2977, 3960, 3961, 155, 153, - 152, 151, 150, 149, 148, 49, 50, 51, 52, 170, - 169, 171, 173, 175, 172, 177, 167, 165, 168, 166, - 164, 144, 145, 66, 146, 147, 11, 117, 18, 25, - 4, 0, 0, 0, 0, 144, 145, 0, 146, 147, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 3434, 0, 1845, 0, 138, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 138, 0, 0, 0, 0, - 0, 1752, 1753, 1754, 1755, 1756, 1757, 1758, 1759, 1760, - 1761, 1762, 1774, 1775, 1776, 1777, 1778, 1779, 1772, 1773, - 0, 3446, 0, 161, 189, 198, 190, 120, 0, 0, - 0, 0, 0, 0, 3437, 0, 0, 161, 189, 198, - 190, 120, 0, 0, 0, 3432, 188, 182, 181, 0, - 3454, 3455, 0, 67, 0, 0, 3433, 0, 0, 0, - 188, 182, 181, 2992, 0, 2994, 0, 67, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1845, 0, 0, 0, 0, 1845, - 0, 0, 0, 3438, 0, 0, 0, 0, 0, 0, - 2156, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 184, 185, 186, 0, 0, 2033, - 2033, 2033, 2033, 2033, 0, 0, 0, 0, 184, 185, - 186, 0, 0, 0, 2033, 3046, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 194, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 194, 0, 3070, 0, 0, 0, 0, 132, 0, - 0, 0, 187, 0, 133, 0, 3164, 0, 0, 0, - 0, 0, 132, 0, 0, 0, 187, 0, 133, 3453, - 0, 2381, 3176, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 3442, 0, 0, 0, - 0, 0, 138, 0, 0, 0, 0, 138, 0, 0, - 2000, 134, 0, 0, 0, 1961, 0, 0, 3439, 3443, - 3441, 3440, 0, 0, 60, 134, 0, 0, 138, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 60, 138, - 0, 0, 0, 0, 0, 2003, 1970, 0, 0, 0, - 0, 0, 0, 0, 0, 2004, 2005, 0, 3448, 3449, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 62, 0, 0, 0, 0, 0, 0, - 0, 1969, 0, 0, 0, 0, 0, 62, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1977, 2075, - 0, 0, 0, 3223, 0, 0, 3456, 0, 142, 197, - 0, 143, 0, 0, 0, 0, 163, 0, 3435, 0, - 0, 58, 142, 197, 3447, 143, 0, 0, 0, 0, - 163, 0, 0, 0, 0, 58, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1993, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 135, 45, 0, - 0, 0, 0, 0, 59, 0, 0, 0, 5, 0, - 0, 135, 45, 0, 0, 0, 0, 0, 59, 0, - 0, 1046, 0, 138, 0, 139, 140, 0, 138, 141, - 0, 0, 0, 0, 0, 2033, 0, 0, 0, 139, - 140, 3323, 0, 141, 0, 0, 2000, 0, 1960, 1962, - 1959, 1961, 0, 1956, 138, 0, 0, 0, 1981, 0, - 0, 0, 0, 0, 3452, 0, 0, 0, 0, 1987, - 0, 0, 0, 0, 0, 0, 0, 1972, 0, 1955, - 0, 2003, 1970, 0, 0, 0, 0, 0, 0, 1975, - 2009, 2004, 2005, 1976, 1978, 1980, 0, 1982, 1983, 1984, - 1988, 1989, 1990, 1992, 1995, 1996, 1997, 0, 0, 0, - 0, 0, 0, 0, 1985, 1994, 1986, 1969, 0, 0, - 0, 0, 0, 0, 0, 0, 1964, 0, 0, 0, - 0, 0, 0, 0, 1977, 0, 0, 0, 0, 0, - 3451, 0, 0, 0, 0, 3329, 0, 0, 0, 0, - 2001, 0, 3331, 1235, 1236, 1237, 1234, 0, 2075, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1957, 1958, 0, - 0, 0, 0, 0, 3346, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 1998, 0, 0, 0, 0, - 0, 0, 1993, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 1974, 0, 0, 0, 0, 0, 0, 1973, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 1771, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1991, 0, 0, 0, 0, 0, 0, - 0, 0, 1979, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 2075, 2007, 2006, 0, 0, 0, - 0, 0, 0, 0, 1960, 2815, 1959, 0, 0, 2814, - 0, 0, 0, 0, 1981, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 1987, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 1975, 2009, 0, 1966, 1976, - 1978, 1980, 0, 1982, 1983, 1984, 1988, 1989, 1990, 1992, - 1995, 1996, 1997, 0, 1845, 0, 0, 0, 0, 0, - 1985, 1994, 1986, 0, 0, 0, 0, 0, 0, 0, - 1845, 0, 1964, 3492, 0, 0, 3494, 1114, 0, 0, - 2002, 0, 0, 2008, 0, 0, 0, 0, 0, 0, - 0, 0, 138, 3500, 0, 0, 2001, 0, 0, 138, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1767, - 0, 0, 0, 1957, 1958, 0, 1764, 0, 0, 0, - 1766, 1763, 1765, 1769, 1770, 0, 0, 0, 1768, 0, - 0, 1998, 0, 0, 0, 0, 0, 0, 2033, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1974, 0, - 0, 0, 0, 0, 0, 1973, 0, 0, 0, 1114, - 0, 0, 0, 0, 0, 0, 3635, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1991, - 0, 0, 0, 0, 0, 0, 0, 0, 1979, 0, - 1099, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 2007, 2006, 0, 0, 0, 0, 0, 0, 0, - 1122, 1126, 1128, 1130, 1132, 1133, 1135, 0, 1140, 1136, - 1137, 1138, 1139, 0, 1117, 1118, 1119, 1120, 1097, 1098, - 1123, 0, 1100, 0, 1102, 1103, 1104, 1105, 1101, 1106, - 1107, 1108, 1109, 1110, 1113, 1115, 1111, 1112, 1121, 0, - 0, 138, 0, 0, 1966, 0, 1125, 1127, 1129, 1131, - 1134, 1752, 1753, 1754, 1755, 1756, 1757, 1758, 1759, 1760, - 1761, 1762, 1774, 1775, 1776, 1777, 1778, 1779, 1772, 1773, - 0, 0, 1099, 0, 0, 0, 1089, 0, 0, 0, - 0, 0, 0, 0, 0, 1116, 2002, 0, 0, 2008, - 1114, 0, 1122, 1126, 1128, 1130, 1132, 1133, 1135, 0, - 1140, 1136, 1137, 1138, 1139, 0, 1117, 1118, 1119, 1120, - 1097, 1098, 1123, 0, 1100, 0, 1102, 1103, 1104, 1105, - 1101, 1106, 1107, 1108, 1109, 1110, 1113, 1115, 1111, 1112, - 1121, 0, 0, 0, 0, 0, 0, 138, 1125, 1127, - 1129, 1131, 1134, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1116, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1099, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 2649, 2650, 0, 0, - 0, 0, 0, 1122, 1126, 1128, 1130, 1132, 1133, 1135, - 0, 1140, 1136, 1137, 1138, 1139, 0, 1117, 1118, 1119, - 1120, 1097, 1098, 1123, 0, 1100, 0, 1102, 1103, 1104, - 1105, 1101, 1106, 1107, 1108, 1109, 1110, 1113, 1115, 1111, - 1112, 1121, 0, 0, 0, 0, 1287, 0, 0, 1125, - 1127, 1129, 1131, 1134, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1116, 0, + 767, 743, 4025, 769, 3997, 2825, 217, 4017, 1608, 1964, + 3928, 1686, 3421, 3934, 3520, 3935, 3826, 3927, 3760, 3205, + 752, 3849, 3238, 3807, 3884, 3644, 3313, 3450, 3738, 1682, + 2819, 3708, 745, 3798, 1304, 3759, 3314, 3576, 1746, 3827, + 2737, 3677, 797, 2822, 1040, 3729, 633, 3524, 1520, 1159, + 3808, 3383, 3810, 1452, 3515, 1458, 3388, 1912, 3214, 2378, + 651, 1733, 657, 657, 3437, 2798, 3597, 3585, 657, 675, + 684, 3162, 1153, 684, 1689, 3558, 3176, 3138, 3311, 3404, + 741, 202, 2062, 3590, 2937, 2938, 2936, 3165, 37, 3375, + 1751, 2914, 2848, 2098, 2075, 3234, 3355, 2933, 3216, 3006, + 3406, 696, 3223, 2650, 2024, 3299, 2059, 2131, 2173, 2522, + 2965, 2486, 3278, 2381, 1924, 2766, 2925, 3143, 1513, 3145, + 3139, 3222, 3141, 692, 2615, 3140, 2310, 2778, 3185, 137, + 740, 2156, 1597, 2286, 2594, 735, 2411, 3113, 2287, 2524, + 2342, 2979, 2140, 2169, 2139, 1844, 2104, 2132, 3053, 2576, + 1149, 1609, 1586, 2989, 1598, 2055, 65, 681, 2470, 2487, + 1593, 2168, 3136, 1601, 962, 2028, 1034, 2750, 2850, 2465, + 2755, 36, 2830, 2379, 1954, 2790, 213, 8, 2331, 212, + 7, 6, 1888, 2520, 2203, 1747, 2341, 1560, 1680, 2170, + 1629, 1498, 1097, 633, 744, 1492, 1529, 650, 2322, 2374, + 1441, 1740, 734, 2687, 753, 1418, 742, 2325, 1671, 1720, + 2138, 27, 632, 1175, 2135, 1612, 689, 217, 2120, 217, + 2094, 1088, 1089, 1923, 1567, 2686, 1884, 1033, 657, 1679, + 1497, 2494, 1685, 999, 1887, 666, 961, 1437, 892, 1752, + 2466, 699, 16, 1494, 1551, 111, 1559, 698, 199, 24, + 683, 1461, 938, 14, 1748, 17, 1374, 959, 1453, 10, + 203, 195, 2025, 1305, 985, 23, 1786, 944, 695, 1462, + 1350, 1236, 1237, 1238, 1235, 894, 895, 3817, 653, 1385, + 680, 1236, 1237, 1238, 1235, 2177, 3726, 736, 2723, 952, + 2723, 953, 2723, 2496, 15, 1085, 3418, 3192, 1084, 3023, + 1086, 1236, 1237, 1238, 1235, 3022, 2187, 1154, 3551, 3391, + 3306, 676, 1155, 2638, 2579, 1067, 2582, 1857, 2580, 1574, + 2577, 1570, 678, 687, 1081, 1080, 1046, 662, 33, 201, + 933, 652, 2285, 1369, 669, 1421, 1422, 1423, 1496, 3785, + 1343, 658, 1081, 914, 947, 912, 943, 1081, 1428, 2291, + 3123, 1858, 2295, 1049, 1621, 1372, 3106, 3103, 3108, 3105, + 4009, 1475, 1019, 679, 1851, 1365, 1154, 1572, 1048, 1236, + 1237, 1238, 1235, 3513, 3002, 1620, 3000, 2109, 8, 2715, + 2713, 7, 736, 1079, 3793, 3684, 3678, 1068, 3516, 3312, + 1236, 1237, 1238, 1235, 2153, 1299, 3812, 677, 2134, 893, + 3082, 2126, 924, 2419, 4031, 3806, 904, 4006, 3692, 1234, + 1198, 3963, 2624, 2174, 3745, 3530, 3559, 200, 200, 200, + 61, 191, 162, 2717, 200, 3405, 200, 2333, 1607, 200, + 61, 191, 162, 3910, 200, 1616, 1627, 3804, 200, 61, + 191, 162, 913, 3563, 911, 2632, 3713, 3690, 2668, 1537, + 3860, 200, 61, 191, 162, 1375, 1380, 200, 3746, 1379, + 1062, 1057, 1052, 1056, 1060, 1613, 1624, 1378, 200, 192, + 1377, 1044, 1045, 914, 136, 949, 183, 942, 912, 1050, + 193, 1393, 3025, 3080, 1410, 694, 946, 945, 1065, 1615, + 1626, 196, 1055, 1638, 2185, 1860, 196, 1652, 196, 136, + 200, 196, 1170, 927, 2332, 3014, 196, 934, 1014, 1012, + 196, 1013, 2931, 2326, 122, 905, 136, 1381, 200, 61, + 191, 162, 2784, 196, 2514, 2515, 1233, 941, 883, 196, + 882, 884, 885, 2972, 886, 887, 909, 2739, 3107, 3104, + 196, 3715, 2501, 1063, 2595, 2500, 951, 2038, 2502, 2973, + 2974, 940, 1066, 2039, 2040, 939, 1499, 1672, 1501, 1471, + 1676, 926, 1472, 200, 61, 191, 162, 932, 1869, 1870, + 1008, 2072, 2782, 1457, 1053, 2740, 1449, 1456, 1459, 1460, + 1206, 3540, 1167, 1208, 1675, 200, 61, 191, 162, 930, + 196, 1459, 1460, 1226, 3907, 1938, 3938, 3939, 1064, 1020, + 144, 145, 3209, 146, 147, 1213, 1688, 1231, 1214, 2752, + 3815, 1209, 1573, 1571, 3207, 1392, 1043, 1042, 3814, 2753, + 3813, 1016, 2785, 3815, 3898, 2274, 3962, 950, 3814, 3897, + 3903, 2718, 3813, 3896, 3796, 196, 1216, 3007, 1054, 3886, + 161, 1661, 198, 4001, 4002, 3889, 3681, 3008, 3315, 3009, + 1474, 1178, 1181, 931, 1692, 657, 657, 196, 3799, 3800, + 3801, 3802, 2619, 188, 1164, 3315, 657, 1163, 2751, 3886, + 1677, 2189, 161, 189, 198, 190, 120, 3912, 3913, 1178, + 1181, 2460, 3823, 2050, 2056, 1018, 684, 684, 3328, 657, + 3908, 3909, 3376, 1202, 1674, 188, 182, 181, 3381, 2454, + 2181, 2046, 67, 1667, 2758, 2869, 3568, 2321, 3158, 2117, + 2926, 3717, 3718, 950, 3043, 1061, 2741, 2742, 1211, 1204, + 3462, 3539, 3905, 3700, 2629, 3701, 1580, 1579, 3041, 3541, + 948, 1207, 1210, 3156, 1228, 1218, 187, 2417, 1219, 730, + 3514, 3695, 732, 3001, 2181, 1201, 2716, 731, 1229, 1230, + 2186, 1058, 1276, 3937, 1059, 1368, 3152, 1203, 2920, 681, + 681, 681, 1017, 184, 185, 186, 1221, 1485, 1394, 937, + 2456, 1091, 3816, 1691, 1690, 3163, 2457, 2458, 1447, 3703, + 3477, 3725, 1212, 3331, 2517, 3047, 3722, 2722, 1173, 3153, + 3154, 1155, 1155, 3565, 2735, 1193, 194, 3671, 2175, 1163, + 2175, 2175, 1155, 1046, 3359, 3155, 2463, 1224, 1225, 1156, + 3702, 1673, 3024, 1473, 2164, 1223, 2292, 132, 1859, 649, + 1622, 187, 3021, 133, 3237, 3531, 2070, 2071, 3211, 3474, + 1049, 1308, 2736, 3970, 1205, 3235, 3236, 907, 1081, 3842, + 3174, 3186, 2208, 3837, 1081, 1048, 1081, 1081, 1217, 1081, + 1180, 1179, 2176, 1081, 1069, 1051, 3750, 2791, 682, 1215, + 3744, 1155, 686, 1169, 2192, 2194, 2195, 682, 2188, 925, + 923, 3742, 3150, 2916, 2578, 908, 1046, 2449, 1180, 1179, + 134, 685, 680, 680, 680, 3911, 2929, 1222, 3164, 1698, + 1701, 1702, 2328, 60, 3672, 1575, 3467, 3114, 1270, 3828, + 1699, 1015, 3844, 1049, 3422, 3850, 3206, 2820, 2821, 1371, + 2824, 1373, 1220, 676, 676, 676, 2824, 3691, 1048, 893, + 62, 1162, 3429, 1436, 678, 678, 678, 1390, 651, 62, + 3368, 3240, 1190, 1151, 1158, 693, 1186, 1187, 1157, 1045, + 1182, 1348, 62, 2714, 1353, 1662, 197, 682, 1663, 3712, + 3366, 163, 163, 163, 1272, 1273, 1274, 1275, 163, 1277, + 163, 962, 1192, 163, 1376, 679, 679, 679, 163, 3564, + 3127, 2633, 163, 2306, 3716, 1861, 3478, 142, 197, 2452, + 143, 3164, 3822, 1459, 1460, 163, 3635, 952, 1172, 953, + 58, 163, 682, 2429, 4037, 1459, 1460, 2428, 3367, 677, + 677, 677, 163, 3527, 1448, 1166, 1168, 1171, 2384, 62, + 4020, 1184, 2764, 2517, 682, 657, 910, 3751, 1487, 1509, + 2757, 737, 657, 2057, 3159, 2927, 633, 633, 3719, 2384, + 2387, 3044, 3743, 1508, 163, 3696, 633, 633, 1455, 3697, + 1524, 1524, 1191, 657, 2397, 1434, 3700, 3904, 3701, 1433, + 2377, 2400, 163, 1309, 62, 3851, 135, 45, 3212, 3624, + 2450, 2451, 1432, 59, 684, 1552, 651, 5, 3730, 1522, + 1522, 1563, 1563, 1320, 1321, 3151, 62, 2761, 2762, 3569, + 2049, 3215, 217, 3630, 139, 140, 1150, 3102, 141, 1526, + 3764, 633, 2760, 2870, 1531, 2871, 2872, 163, 2047, 3926, + 1668, 2420, 3703, 1451, 1450, 3407, 2377, 2394, 2399, 2193, + 3645, 3646, 3647, 3651, 3649, 3650, 3648, 1267, 3511, 163, + 2967, 2969, 3318, 1386, 3239, 1391, 2033, 3235, 3236, 694, + 3883, 1495, 1486, 3702, 1700, 2770, 2773, 2774, 2775, 2771, + 2772, 3231, 1198, 1605, 1354, 3118, 2625, 2383, 1610, 2506, + 4021, 2415, 2385, 2398, 2178, 1619, 1518, 1519, 2984, 2985, + 2045, 2022, 2387, 2388, 1402, 1352, 2728, 3269, 2383, 2377, + 2382, 3046, 2380, 2385, 3637, 1408, 1407, 1395, 1406, 1443, + 1444, 1009, 1863, 1405, 1650, 1021, 688, 3369, 656, 656, + 1009, 3232, 1503, 1505, 664, 2867, 2898, 1396, 1524, 954, + 1524, 1163, 1516, 1517, 1387, 1388, 2386, 1628, 2190, 2191, + 1397, 1398, 1399, 1400, 1401, 3356, 1403, 1415, 1417, 2732, + 1420, 3763, 1409, 2299, 2204, 3172, 1384, 2386, 1872, 1581, + 1197, 956, 957, 958, 2414, 1873, 1082, 1083, 1614, 1476, + 1477, 1087, 3055, 3054, 951, 1625, 1438, 1442, 1442, 1442, + 3549, 1584, 1463, 1587, 1588, 1466, 3120, 1576, 681, 2298, + 2305, 681, 681, 1553, 1011, 1589, 1590, 1010, 1524, 1871, + 1507, 1438, 1438, 1011, 1660, 915, 1010, 1645, 1646, 3925, + 1595, 1596, 1009, 4018, 4019, 1163, 1750, 2441, 2889, 2890, + 2301, 2300, 2968, 1600, 1532, 2388, 1604, 1734, 1781, 1782, + 1799, 1785, 1618, 916, 1603, 3598, 1049, 4033, 1538, 1800, + 1544, 662, 3626, 1049, 1550, 919, 3625, 1430, 1564, 2393, + 3631, 3632, 1807, 2391, 1809, 2313, 1810, 1811, 1812, 2484, + 1708, 1709, 1710, 1711, 1712, 1713, 1714, 1715, 1716, 1717, + 1718, 1719, 1565, 1684, 1382, 1383, 1731, 1732, 2314, 2315, + 4039, 4027, 4015, 3191, 664, 3319, 3173, 3893, 3696, 1429, + 1234, 2242, 3809, 2796, 2241, 1011, 918, 1163, 1010, 1649, + 921, 920, 1703, 1862, 1429, 1160, 1160, 1865, 1648, 1867, + 2183, 680, 1784, 1665, 680, 680, 1874, 3977, 1876, 1877, + 3275, 1022, 2729, 1552, 2324, 1808, 3233, 1658, 1885, 1524, + 1890, 1891, 1842, 1893, 1487, 657, 1669, 2517, 1864, 3271, + 657, 1198, 676, 1524, 2888, 676, 676, 962, 1631, 3949, + 1913, 3946, 2097, 678, 4028, 3978, 678, 678, 1655, 2797, + 1681, 1659, 1524, 1636, 1845, 1657, 1639, 1234, 1487, 1654, + 1683, 1656, 1798, 675, 1678, 1653, 1789, 1790, 1791, 2899, + 2901, 2902, 2903, 2900, 3940, 1687, 2597, 2485, 1195, 1805, + 3978, 3922, 1806, 1937, 679, 3372, 1722, 679, 679, 3330, + 1729, 1730, 1944, 1944, 2279, 1487, 2624, 1487, 1487, 1819, + 1820, 657, 657, 1196, 2011, 1885, 2015, 2357, 3877, 1524, + 2019, 2020, 3950, 3244, 3947, 2035, 3876, 633, 677, 1841, + 2485, 677, 677, 1892, 1637, 2797, 2217, 1640, 1641, 3242, + 3870, 633, 2323, 1524, 3845, 3275, 770, 780, 1894, 3112, + 1670, 3833, 1941, 1234, 1349, 3110, 771, 2218, 772, 776, + 779, 775, 773, 774, 3923, 1196, 1236, 1237, 1238, 1235, + 657, 1885, 1524, 1848, 2080, 2485, 657, 657, 657, 692, + 692, 1072, 1077, 1078, 2987, 2744, 2090, 2091, 2092, 2093, + 2719, 1234, 1813, 2099, 2095, 3783, 1966, 2614, 2602, 1234, + 217, 2174, 2013, 217, 217, 3782, 217, 1236, 1237, 1238, + 1235, 777, 2216, 2218, 4046, 2073, 2037, 2183, 1843, 1947, + 1814, 1815, 1816, 1817, 3834, 3777, 1821, 1822, 1823, 1824, + 1826, 1827, 1828, 1829, 1830, 1831, 1832, 1833, 1834, 1835, + 1836, 2051, 1849, 778, 2370, 2284, 1799, 1799, 2142, 2278, + 2277, 1853, 2042, 2356, 2044, 2065, 2066, 1799, 1799, 2249, + 3776, 1889, 1915, 1916, 2158, 2063, 2064, 1880, 3784, 2165, + 2082, 2083, 2084, 2068, 2079, 1905, 2021, 1930, 2346, 1772, + 1416, 2108, 3775, 1910, 2111, 2112, 2058, 2114, 1909, 1935, + 1737, 3774, 1198, 2036, 1918, 1510, 3754, 1913, 2218, 4029, + 1920, 1524, 2172, 1881, 1882, 1883, 3418, 3753, 1926, 3728, + 2991, 3483, 2152, 2799, 3431, 1896, 1897, 1898, 1899, 3397, + 2627, 3348, 2144, 1925, 2626, 1927, 1928, 3344, 2618, 2364, + 1948, 1949, 1438, 2218, 2237, 3252, 2962, 1614, 1424, 1934, + 2222, 2012, 1943, 1945, 2163, 2705, 1442, 1236, 1237, 1238, + 1235, 2018, 2693, 2017, 2166, 2218, 2023, 2102, 1442, 2088, + 1633, 681, 2148, 2041, 2218, 2043, 1285, 1183, 2052, 2183, + 1236, 1237, 1238, 1235, 2685, 1046, 1147, 1074, 1075, 1076, + 2183, 2640, 2218, 1946, 2517, 1142, 1046, 3432, 2137, 2622, + 1239, 3661, 3398, 3078, 3349, 2085, 2086, 2077, 1269, 2137, + 3345, 2078, 1049, 1914, 1681, 1049, 2610, 1279, 3253, 2485, + 3380, 656, 1152, 1049, 3481, 2103, 2214, 1048, 2346, 1267, + 2604, 2105, 1161, 2599, 1929, 1234, 2591, 2589, 1048, 897, + 898, 899, 900, 1288, 2587, 2067, 1251, 3196, 2201, 2202, + 1936, 2122, 3187, 1939, 1940, 1189, 1768, 1234, 2585, 2384, + 2387, 3038, 2154, 1765, 1234, 917, 4040, 1767, 1764, 1766, + 1770, 1771, 2346, 2345, 2143, 1769, 1470, 1913, 2151, 2280, + 2256, 2149, 897, 898, 899, 900, 2255, 1788, 1787, 2600, + 2240, 2231, 2230, 1046, 680, 2229, 4005, 2162, 2219, 2289, + 2290, 2412, 2293, 2605, 3838, 2296, 2600, 2182, 3818, 2592, + 2590, 1642, 3599, 1788, 1787, 2197, 1514, 2586, 2167, 735, + 1049, 2220, 657, 657, 657, 676, 1425, 1515, 1439, 3727, + 1426, 2586, 3188, 2180, 2272, 1048, 678, 657, 657, 657, + 657, 3410, 3408, 1512, 2160, 3688, 2346, 3628, 3839, 2196, + 2343, 3627, 2279, 1234, 3613, 2205, 3600, 3572, 3304, 1234, + 2349, 1487, 1535, 1234, 1234, 1234, 3390, 2577, 1234, 1722, + 3276, 2218, 2199, 2200, 2198, 3267, 3189, 679, 3259, 1425, + 2183, 2210, 2647, 1426, 1643, 3411, 3409, 1487, 1480, 1481, + 902, 1483, 1484, 2388, 1488, 1489, 1490, 3254, 2383, 2377, + 2382, 1825, 2380, 2385, 2406, 3167, 2250, 2251, 2571, 2253, + 2923, 677, 2922, 922, 2372, 2768, 2260, 2161, 2724, 1775, + 1776, 1777, 1778, 1779, 1780, 1773, 1774, 1818, 1539, 1540, + 1541, 1542, 1543, 902, 1545, 1546, 1547, 1548, 1549, 2637, + 2603, 2508, 1555, 1556, 1557, 1558, 1511, 2147, 1440, 2146, + 2145, 2413, 2361, 1412, 1411, 1165, 2363, 2386, 2365, 1728, + 2106, 2993, 657, 1944, 1254, 1255, 1256, 1257, 1258, 1251, + 1741, 2489, 2489, 2035, 2489, 1725, 1727, 1724, 1875, 1726, + 2281, 2273, 2275, 2276, 1259, 1260, 1252, 1253, 1254, 1255, + 1256, 1257, 1258, 1251, 633, 633, 1236, 1237, 1238, 1235, + 3895, 2366, 1163, 1741, 1235, 2211, 3640, 3307, 1524, 657, + 1252, 1253, 1254, 1255, 1256, 1257, 1258, 1251, 1238, 1235, + 2376, 2307, 657, 2375, 1236, 1237, 1238, 1235, 1163, 2561, + 651, 3619, 1562, 1562, 1308, 2581, 1563, 2512, 2035, 3639, + 1568, 2566, 2106, 2568, 3010, 2859, 2857, 217, 2418, 2836, + 2834, 2421, 2422, 2423, 2424, 2425, 2426, 2427, 3573, 3574, + 2430, 2431, 2432, 2433, 2434, 2435, 2436, 2437, 2438, 2439, + 2440, 3566, 2442, 2443, 2444, 2445, 2446, 2493, 2447, 1287, + 4036, 1482, 2362, 2491, 2350, 2495, 4011, 2607, 1493, 2503, + 4010, 2504, 1286, 3378, 2910, 2497, 3057, 2369, 2389, 2390, + 3953, 2395, 1046, 2707, 2620, 2708, 1803, 2767, 2172, 1530, + 2509, 2510, 3921, 3920, 3840, 1524, 2519, 1524, 2738, 1524, + 3779, 1804, 2353, 3767, 1163, 2908, 2906, 2359, 2572, 1049, + 2360, 3567, 2639, 1242, 1243, 1244, 1245, 1246, 1247, 1248, + 1240, 3757, 2565, 2895, 1048, 4035, 2630, 1236, 1237, 1238, + 1235, 1442, 3384, 3379, 2909, 1568, 2459, 2464, 1524, 1163, + 1503, 1505, 4032, 2671, 1236, 1237, 1238, 1235, 3747, 2498, + 2616, 2617, 2677, 3305, 1236, 1237, 1238, 1235, 2678, 1236, + 1237, 1238, 1235, 1524, 3679, 2907, 2905, 1522, 2649, 3602, + 3601, 3423, 1236, 1237, 1238, 1235, 1693, 1694, 1695, 1696, + 1697, 2573, 3412, 2894, 2516, 2513, 2666, 3377, 3255, 3157, + 2351, 2352, 1522, 1236, 1237, 1238, 1235, 3071, 3034, 3005, + 2354, 2355, 1569, 2564, 3004, 2893, 2892, 2562, 2891, 2883, + 2233, 2679, 2726, 2727, 3389, 2877, 2730, 2876, 1738, 2875, + 2682, 2683, 1742, 1743, 1744, 1745, 1236, 1237, 1238, 1235, + 2874, 2720, 1783, 1163, 2593, 2505, 1309, 1163, 2283, 2655, + 1793, 3931, 2636, 2125, 1524, 2680, 2124, 1487, 1236, 1237, + 1238, 1235, 2123, 2015, 2119, 2118, 3144, 2645, 2074, 3070, + 2631, 2795, 2525, 2612, 1868, 2669, 1866, 2801, 1236, 1237, + 1238, 1235, 3545, 2651, 1145, 2651, 1634, 2623, 1367, 2232, + 2621, 4030, 2711, 2628, 1506, 2811, 1236, 1237, 1238, 1235, + 3720, 3721, 3521, 1846, 4003, 1163, 3969, 3968, 3965, 1236, + 1237, 1238, 1235, 2833, 2641, 2642, 1236, 1237, 1238, 1235, + 1163, 1163, 1163, 1944, 3901, 3900, 1163, 3533, 2843, 2844, + 2845, 2846, 1163, 2853, 2667, 2854, 2855, 1681, 2856, 2657, + 2858, 2644, 3709, 1144, 730, 2779, 3881, 732, 2225, 2780, + 3825, 2853, 731, 2783, 1236, 1237, 1238, 1235, 3577, 3803, + 782, 138, 3532, 2489, 3794, 3771, 138, 2826, 2634, 2812, + 3766, 2765, 3765, 2563, 3724, 3711, 3710, 2911, 3680, 3621, + 2802, 1917, 2570, 3581, 1966, 3570, 3552, 633, 2754, 1236, + 1237, 1238, 1235, 3550, 2015, 3547, 3544, 3543, 1163, 2035, + 2035, 2035, 2035, 2035, 2035, 3519, 1933, 3517, 3491, 2792, + 2081, 2688, 2689, 3488, 3485, 1163, 2035, 2694, 2915, 2489, + 3374, 1049, 3990, 2917, 2831, 2827, 663, 3364, 2831, 138, + 2746, 1236, 1237, 1238, 1235, 2970, 2763, 1524, 2814, 3357, + 2838, 3341, 3339, 2786, 1236, 1237, 1238, 1235, 657, 657, + 3334, 8, 3265, 2747, 7, 2749, 2800, 2794, 3264, 3262, + 3261, 1895, 2215, 3256, 3250, 1846, 1900, 2213, 3249, 3168, + 1846, 1846, 3131, 2813, 3130, 2816, 3126, 3124, 3471, 3122, + 2865, 2866, 2829, 3336, 3119, 3117, 1889, 2835, 3074, 2288, + 3048, 3045, 2841, 3952, 3003, 2881, 2882, 2745, 2977, 2958, + 2904, 2525, 2896, 2886, 217, 1236, 1237, 1238, 1235, 217, + 1236, 1237, 1238, 1235, 2884, 1236, 1237, 1238, 1235, 2873, + 2107, 2919, 2885, 2110, 2880, 2879, 2113, 2878, 2733, 2115, + 2725, 1799, 2721, 1799, 2971, 2613, 3020, 1950, 1951, 3073, + 1236, 1237, 1238, 1235, 2810, 1236, 1237, 1238, 1235, 3033, + 2302, 2918, 2297, 2921, 2294, 1524, 2128, 1047, 3040, 2828, + 2924, 2121, 138, 838, 837, 4038, 1236, 1237, 1238, 1235, + 2955, 1856, 1855, 1635, 2828, 2839, 2840, 138, 2961, 138, + 2842, 2959, 1316, 1312, 2157, 2994, 2849, 1311, 1148, 906, + 2998, 2832, 3991, 3857, 2978, 2988, 2076, 3853, 2975, 3705, + 3704, 3693, 2076, 2076, 2076, 1427, 3689, 3546, 3528, 3403, + 1588, 1845, 1435, 3402, 3401, 3371, 3019, 3353, 3015, 1445, + 1589, 1590, 2960, 3351, 3350, 3347, 3346, 1464, 1465, 3026, + 1467, 1468, 2804, 1469, 1595, 1596, 3340, 2807, 3338, 3017, + 3320, 3310, 1600, 3309, 3295, 1604, 3294, 3197, 3134, 3027, + 3109, 2995, 2939, 1603, 2992, 2996, 3076, 3037, 3067, 3059, + 3121, 2940, 2941, 2942, 2943, 2944, 2945, 3125, 3058, 2939, + 3042, 3128, 3129, 3016, 3013, 3011, 3030, 3018, 3029, 1163, + 3072, 3052, 2986, 2743, 3028, 3147, 1049, 2588, 200, 2207, + 191, 162, 2584, 2212, 2583, 3161, 2261, 1049, 3036, 2254, + 657, 2221, 3049, 2248, 3050, 2247, 2704, 1236, 1237, 1238, + 1235, 2246, 3177, 1163, 2245, 2243, 657, 2239, 1163, 1163, + 2238, 2236, 2227, 2224, 3062, 3069, 3064, 2035, 2343, 2703, + 3195, 2223, 3056, 1236, 1237, 1238, 1235, 2127, 2228, 1839, + 3111, 3060, 3061, 3065, 3066, 1838, 2235, 1837, 1802, 2406, + 1801, 1792, 200, 3063, 3171, 1536, 1236, 1237, 1238, 1235, + 196, 3221, 1534, 3224, 3875, 3224, 3224, 1306, 2252, 3852, + 1163, 3789, 3786, 2257, 2258, 2259, 3773, 3768, 2262, 2263, + 2264, 2265, 2266, 2267, 2268, 2269, 2270, 2271, 3245, 2702, + 3115, 2779, 3116, 2701, 3674, 3673, 1524, 1524, 3655, 3241, + 2700, 3132, 3149, 3180, 3638, 3634, 3208, 3210, 3184, 2699, + 3199, 3612, 3596, 3869, 3501, 3133, 1236, 1237, 1238, 1235, + 1236, 1237, 1238, 1235, 196, 1522, 1522, 1236, 1237, 1238, + 1235, 3193, 3499, 3469, 3204, 3170, 1236, 1237, 1238, 1235, + 3083, 3084, 1046, 657, 3246, 3247, 3085, 3086, 3087, 3088, + 3147, 3089, 3090, 3091, 3092, 3093, 3094, 3095, 3096, 3097, + 3098, 3099, 1487, 3220, 2698, 2015, 2015, 3229, 3194, 1049, + 2376, 1049, 3203, 2375, 3190, 3179, 1049, 3983, 2697, 3468, + 3182, 3183, 3465, 2658, 1048, 3464, 3225, 3226, 3430, 3427, + 3230, 1236, 1237, 1238, 1235, 3981, 2696, 3425, 3219, 3936, + 2695, 3392, 1049, 3068, 3243, 1236, 1237, 1238, 1235, 1583, + 1594, 1141, 1137, 1138, 1139, 1140, 1585, 2663, 1163, 2662, + 2661, 2659, 2671, 1236, 1237, 1238, 1235, 1236, 1237, 1238, + 1235, 3308, 1599, 2828, 1602, 1591, 1419, 3251, 2317, 2318, + 2319, 1250, 1249, 1259, 1260, 1252, 1253, 1254, 1255, 1256, + 1257, 1258, 1251, 2334, 2335, 2336, 2337, 2912, 2837, 2788, + 2244, 2787, 3272, 3273, 2781, 2748, 3257, 2828, 2706, 3258, + 3260, 3263, 2828, 2828, 3266, 2598, 3270, 657, 3166, 2692, + 2507, 2448, 2344, 2316, 2282, 3283, 1723, 3284, 2660, 3286, + 2691, 196, 2087, 1879, 1846, 1852, 1846, 1666, 1617, 1592, + 1366, 1351, 1347, 1346, 1345, 3288, 1236, 1237, 1238, 1235, + 1344, 3291, 3292, 3293, 1343, 1846, 1846, 1236, 1237, 1238, + 1235, 1342, 1341, 1340, 2828, 1339, 1338, 3297, 3303, 1337, + 1484, 138, 138, 138, 1047, 1249, 1259, 1260, 1252, 1253, + 1254, 1255, 1256, 1257, 1258, 1251, 3227, 1562, 1336, 1335, + 2099, 3361, 3321, 1334, 3363, 3285, 2690, 1333, 1332, 3323, + 1331, 1330, 1329, 3322, 1328, 1327, 1326, 3326, 1325, 1324, + 1323, 1322, 1921, 1922, 2684, 1319, 3327, 1318, 1493, 1317, + 3342, 1315, 3198, 1236, 1237, 1238, 1235, 3200, 3201, 1931, + 1932, 3332, 1314, 1313, 657, 2015, 1310, 2606, 3365, 2609, + 1303, 1236, 1237, 1238, 1235, 3396, 1302, 1268, 1300, 1942, + 1299, 1298, 1297, 1296, 1295, 1294, 1293, 1292, 1291, 1290, + 1289, 2489, 2035, 3415, 1284, 1530, 1283, 2651, 1282, 2664, + 2665, 1281, 1280, 3354, 1200, 1146, 3867, 3282, 2076, 3865, + 2674, 3863, 3358, 3360, 3279, 3280, 3433, 3466, 2348, 1163, + 2957, 2330, 1188, 2670, 3370, 2769, 2518, 2130, 3221, 1199, + 2948, 3373, 1163, 2648, 2956, 2949, 2654, 1236, 1237, 1238, + 1235, 2947, 2525, 1163, 2946, 3480, 121, 2672, 2673, 1524, + 1236, 1237, 1238, 1235, 2646, 2675, 2676, 64, 3385, 2952, + 2950, 3417, 3387, 3617, 2953, 2951, 2954, 63, 2479, 2480, + 657, 2681, 2015, 3894, 3805, 2611, 1163, 3503, 1522, 3463, + 2601, 1236, 1237, 1238, 1235, 3504, 3274, 1413, 3424, 3217, + 3426, 3218, 1049, 1907, 1908, 3032, 3414, 3482, 2416, 1049, + 1693, 1846, 3413, 3456, 3420, 3476, 217, 3287, 3393, 3394, + 3395, 1736, 659, 3298, 3399, 3400, 1902, 1903, 1904, 1163, + 3492, 3324, 3325, 660, 3495, 2004, 1355, 1577, 3475, 3472, + 3470, 3505, 2596, 661, 3502, 3479, 2616, 2617, 1236, 1237, + 1238, 1235, 2861, 2635, 1630, 3484, 3486, 1611, 3487, 2862, + 2863, 2864, 2303, 2089, 3490, 1194, 3142, 3135, 3494, 3497, + 3548, 3496, 2815, 2789, 3493, 2368, 2339, 3489, 1911, 3555, + 1878, 1788, 1787, 1163, 3526, 1362, 1363, 3994, 2805, 2806, + 1360, 1361, 1358, 1359, 1356, 1357, 3770, 3512, 3248, 2461, + 2455, 2016, 1479, 1163, 1524, 1524, 1478, 1227, 3290, 3177, + 2980, 2304, 3523, 3522, 2159, 1431, 1404, 3553, 3554, 1454, + 3959, 3589, 3957, 3589, 3914, 3891, 3890, 3888, 3829, 3790, + 3669, 3668, 3607, 1522, 1734, 3518, 1163, 3606, 1163, 3343, + 3317, 3578, 3583, 3584, 3316, 3301, 2401, 3609, 2371, 3611, + 1632, 3300, 3579, 2990, 1429, 1524, 3985, 3984, 3984, 3562, + 3561, 3560, 3362, 3507, 3035, 2731, 3580, 3571, 2332, 2226, + 1370, 1185, 3985, 657, 3636, 1163, 1163, 2793, 3296, 1163, + 1163, 3594, 1160, 3434, 1734, 3582, 1533, 3593, 204, 3, + 663, 3417, 1446, 72, 2, 4007, 3473, 3605, 2144, 4008, + 3586, 3615, 1, 3652, 3657, 3542, 3463, 2849, 2712, 1850, + 1913, 3618, 3666, 3642, 3643, 1364, 3622, 3653, 3654, 3557, + 901, 3614, 138, 896, 3675, 3676, 1500, 2499, 2069, 1846, + 3456, 3620, 1528, 897, 898, 899, 900, 1524, 1160, 1854, + 2939, 2474, 2478, 2479, 2480, 2475, 2483, 2476, 2481, 903, + 2963, 2477, 3416, 2482, 2964, 3289, 2966, 3662, 2734, 3663, + 3706, 3419, 2179, 2928, 2453, 3658, 1522, 3664, 1049, 3699, + 2320, 3160, 1414, 955, 3534, 1794, 3535, 1647, 2467, 1071, + 1177, 1644, 1176, 2939, 1174, 3687, 1739, 784, 3682, 2133, + 138, 3686, 2913, 2887, 3665, 3993, 4024, 138, 3951, 3694, + 3996, 3698, 962, 1664, 768, 3882, 3795, 3739, 3733, 2997, + 138, 2999, 3955, 138, 138, 2474, 2478, 2479, 2480, 2475, + 2483, 2476, 2481, 1163, 3659, 2477, 138, 2482, 3660, 3797, + 1846, 3685, 2184, 3723, 3756, 1846, 3762, 2828, 1232, 3012, + 981, 825, 795, 1301, 1623, 3734, 2157, 3081, 3526, 3079, + 3736, 3735, 1073, 2358, 2981, 2982, 794, 3382, 2759, 3670, + 2983, 3752, 3741, 3748, 1070, 982, 1163, 2116, 3792, 3683, + 1578, 1524, 1582, 2367, 3749, 3848, 3616, 3213, 2823, 1606, + 3843, 3051, 3428, 3538, 3536, 3537, 3731, 700, 2048, 3769, + 1687, 631, 1687, 1031, 3656, 2129, 701, 2347, 3906, 3772, + 1522, 935, 2329, 936, 3778, 928, 2777, 2776, 1704, 3075, + 1241, 1721, 3100, 1262, 3101, 1266, 1278, 739, 2209, 3780, + 2756, 3451, 2976, 71, 3821, 70, 3811, 69, 68, 225, + 786, 1263, 1265, 1261, 3791, 1264, 1250, 1249, 1259, 1260, + 1252, 1253, 1254, 1255, 1256, 1257, 1258, 1251, 224, 3707, + 3575, 3878, 3830, 3998, 1049, 765, 764, 763, 762, 761, + 760, 2472, 2473, 3819, 2471, 2469, 2468, 2030, 2029, 3758, + 2096, 3175, 3603, 3604, 3824, 2852, 2847, 1955, 1953, 1491, + 3847, 2396, 2403, 1952, 1163, 3933, 3832, 3333, 3529, 3858, + 3859, 3633, 1524, 2897, 3525, 3872, 1901, 2392, 1972, 2868, + 3879, 3862, 3864, 3866, 3868, 1969, 3846, 1968, 3781, 2860, + 3629, 3623, 3841, 2000, 3855, 3880, 3737, 3588, 3435, 3436, + 3442, 1522, 3861, 1250, 1249, 1259, 1260, 1252, 1253, 1254, + 1255, 1256, 1257, 1258, 1251, 3787, 3788, 2338, 1096, 1092, + 3871, 3887, 1524, 3885, 1094, 3739, 1095, 1093, 2656, 3268, + 2373, 3137, 2312, 2311, 2001, 2309, 2308, 1389, 3820, 3899, + 3902, 3924, 3556, 2523, 2521, 1143, 3281, 3932, 3277, 3228, + 3915, 1522, 2141, 3917, 2155, 3031, 2031, 1687, 3918, 3919, + 2027, 3831, 2026, 2930, 2462, 3714, 3835, 3836, 1906, 2004, + 3916, 929, 2327, 41, 3948, 118, 105, 179, 56, 3941, + 178, 3942, 55, 3943, 969, 3944, 3169, 116, 3945, 176, + 54, 100, 99, 3958, 115, 3960, 3961, 3856, 174, 53, + 3956, 3954, 3181, 209, 208, 1163, 3811, 211, 210, 3964, + 207, 2574, 2575, 3761, 206, 2034, 1566, 205, 3892, 3592, + 3874, 891, 44, 1978, 3762, 3973, 43, 180, 42, 200, + 61, 191, 162, 3975, 3976, 3974, 106, 57, 40, 3982, + 3980, 3992, 39, 4000, 3979, 38, 3999, 192, 3986, 3987, + 3988, 3989, 34, 13, 183, 12, 966, 967, 193, 35, + 22, 4012, 21, 1163, 4004, 1651, 20, 1009, 26, 32, + 31, 131, 130, 30, 4013, 3847, 4014, 136, 129, 4016, + 128, 127, 126, 4022, 125, 124, 4026, 123, 29, 4023, + 138, 1994, 122, 138, 138, 19, 138, 48, 47, 46, + 9, 196, 119, 114, 112, 28, 113, 110, 109, 108, + 4034, 107, 103, 101, 83, 82, 81, 96, 95, 4000, + 4042, 94, 3999, 4041, 93, 92, 91, 89, 90, 2076, + 980, 4026, 4043, 80, 3966, 3967, 1047, 4047, 79, 138, + 78, 77, 76, 98, 104, 102, 87, 1047, 97, 2803, + 1011, 88, 86, 1010, 85, 84, 75, 74, 2808, 2809, + 73, 160, 159, 138, 158, 157, 156, 154, 155, 153, + 152, 151, 150, 1982, 149, 148, 49, 50, 144, 145, + 51, 146, 147, 3335, 1988, 52, 170, 169, 171, 173, + 3337, 995, 175, 172, 177, 167, 165, 168, 166, 164, + 970, 66, 11, 117, 1976, 2010, 18, 25, 1977, 1979, + 1981, 4, 1983, 1984, 1985, 1989, 1990, 1991, 1993, 1996, + 1997, 1998, 3352, 0, 0, 0, 0, 972, 0, 1986, + 1995, 1987, 0, 0, 0, 0, 0, 3440, 0, 0, + 0, 0, 0, 0, 1268, 0, 0, 0, 0, 0, + 161, 189, 198, 190, 120, 0, 0, 0, 0, 3971, + 0, 0, 0, 3329, 0, 2002, 0, 0, 0, 0, + 0, 0, 0, 188, 182, 181, 0, 0, 3452, 0, + 67, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 3443, 994, 992, 0, 0, 0, 0, 0, 0, + 0, 0, 3438, 0, 0, 0, 0, 3460, 3461, 0, + 1999, 0, 0, 3439, 991, 0, 0, 1687, 0, 0, + 0, 0, 0, 0, 0, 0, 965, 1975, 0, 0, + 0, 0, 0, 0, 1974, 0, 0, 971, 1004, 0, + 0, 184, 185, 186, 0, 0, 0, 0, 0, 0, + 3444, 0, 0, 0, 0, 0, 0, 0, 1992, 0, + 0, 1000, 0, 0, 0, 0, 0, 1980, 0, 0, + 0, 0, 1846, 0, 194, 0, 0, 0, 0, 0, + 2076, 0, 0, 0, 0, 0, 0, 0, 1846, 0, + 0, 3498, 0, 0, 3500, 132, 0, 1001, 1005, 187, + 0, 133, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 3506, 0, 0, 0, 0, 988, 0, 986, + 990, 1008, 0, 0, 0, 987, 984, 983, 0, 989, + 974, 975, 973, 976, 977, 978, 979, 0, 1006, 0, + 1007, 0, 0, 0, 0, 0, 3459, 0, 2382, 0, + 0, 1002, 1003, 0, 2001, 0, 0, 0, 134, 1962, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 60, 0, 3448, 0, 3595, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 2076, 0, 998, 2004, + 1971, 0, 0, 0, 997, 3445, 3449, 3447, 3446, 2005, + 2006, 0, 0, 0, 0, 0, 0, 0, 0, 993, + 0, 0, 0, 0, 0, 3610, 0, 0, 0, 0, + 62, 0, 0, 0, 0, 1970, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 3454, 3455, 0, 0, 0, + 0, 0, 0, 1978, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 2492, 0, 142, 197, 0, 143, 0, + 0, 0, 0, 163, 0, 0, 0, 0, 58, 1250, + 1249, 1259, 1260, 1252, 1253, 1254, 1255, 1256, 1257, 1258, + 1251, 0, 0, 3462, 0, 0, 0, 996, 0, 0, + 0, 0, 0, 968, 963, 3441, 0, 0, 0, 964, + 0, 3453, 0, 0, 0, 0, 0, 0, 0, 3202, + 0, 1994, 0, 3608, 0, 0, 0, 0, 2034, 0, + 0, 0, 0, 0, 0, 0, 0, 138, 0, 0, + 712, 711, 718, 708, 135, 45, 0, 0, 0, 0, + 0, 59, 715, 716, 0, 717, 721, 0, 0, 702, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 726, + 0, 0, 139, 140, 0, 0, 141, 1250, 1249, 1259, + 1260, 1252, 1253, 1254, 1255, 1256, 1257, 1258, 1251, 3641, + 0, 0, 0, 1961, 1963, 1960, 0, 0, 1957, 0, + 0, 0, 0, 1982, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1988, 730, 0, 0, 732, 0, + 0, 0, 1973, 731, 1956, 3077, 0, 0, 0, 0, + 0, 0, 0, 0, 1976, 2010, 0, 0, 1977, 1979, + 1981, 3458, 1983, 1984, 1985, 1989, 1990, 1991, 1993, 1996, + 1997, 1998, 0, 0, 0, 0, 0, 2001, 0, 1986, + 1995, 1987, 1962, 0, 0, 0, 0, 0, 0, 0, + 0, 1965, 0, 0, 0, 0, 0, 1115, 0, 1250, + 1249, 1259, 1260, 1252, 1253, 1254, 1255, 1256, 1257, 1258, + 1251, 0, 2004, 1971, 0, 2002, 0, 0, 0, 0, + 0, 0, 2005, 2006, 1250, 1249, 1259, 1260, 1252, 1253, + 1254, 1255, 1256, 1257, 1258, 1251, 0, 3457, 0, 0, + 0, 0, 1958, 1959, 0, 1288, 0, 0, 1970, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1999, 0, 0, 0, 0, 0, 1978, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1975, 0, 0, + 0, 703, 705, 704, 1974, 138, 0, 0, 0, 0, + 0, 0, 710, 0, 0, 138, 0, 0, 0, 0, + 0, 0, 0, 0, 714, 0, 0, 0, 1992, 0, + 3854, 729, 0, 0, 0, 0, 0, 1980, 707, 0, + 0, 1100, 697, 0, 0, 0, 0, 0, 0, 0, + 2008, 2007, 0, 0, 1994, 0, 0, 0, 0, 0, + 0, 1123, 1127, 1129, 1131, 1133, 1134, 1136, 0, 1141, + 1137, 1138, 1139, 1140, 0, 1118, 1119, 1120, 1121, 1098, + 1099, 1124, 0, 1101, 0, 1103, 1104, 1105, 1106, 1102, + 1107, 1108, 1109, 1110, 1111, 1114, 1116, 1112, 1113, 1122, + 0, 0, 0, 1967, 0, 0, 0, 1126, 1128, 1130, + 1132, 1135, 0, 0, 3929, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1961, 2818, 1960, 0, + 0, 2817, 0, 0, 0, 0, 1982, 0, 0, 2034, + 2034, 2034, 2034, 2034, 2034, 2003, 1117, 1988, 2009, 0, + 709, 713, 719, 0, 720, 722, 2034, 0, 723, 724, + 725, 0, 0, 727, 728, 0, 0, 1976, 2010, 0, + 0, 1977, 1979, 1981, 0, 1983, 1984, 1985, 1989, 1990, + 1991, 1993, 1996, 1997, 1998, 0, 0, 0, 3929, 0, + 0, 0, 1986, 1995, 1987, 0, 712, 711, 718, 708, + 0, 1115, 0, 0, 1965, 0, 0, 0, 715, 716, + 0, 717, 721, 0, 0, 702, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 726, 0, 0, 2002, 0, + 0, 0, 0, 2643, 0, 0, 0, 0, 0, 0, + 3929, 0, 0, 0, 138, 0, 0, 0, 0, 138, + 0, 0, 0, 0, 0, 1958, 1959, 1250, 1249, 1259, + 1260, 1252, 1253, 1254, 1255, 1256, 1257, 1258, 1251, 0, + 138, 730, 0, 1999, 732, 0, 0, 0, 0, 731, + 0, 138, 0, 0, 0, 0, 0, 2652, 2653, 0, + 1975, 1772, 0, 0, 0, 0, 0, 1974, 712, 711, + 718, 708, 4045, 0, 0, 0, 0, 0, 0, 0, + 715, 716, 0, 717, 721, 0, 0, 702, 0, 706, + 0, 1992, 0, 0, 0, 1100, 0, 726, 0, 1090, + 1980, 0, 0, 0, 0, 1115, 0, 0, 0, 0, + 0, 0, 0, 2008, 2007, 1123, 1127, 1129, 1131, 1133, + 1134, 1136, 0, 1141, 1137, 1138, 1139, 1140, 0, 1118, + 1119, 1120, 1121, 1098, 1099, 1124, 0, 1101, 0, 1103, + 1104, 1105, 1106, 1102, 1107, 1108, 1109, 1110, 1111, 1114, + 1116, 1112, 1113, 1122, 1236, 1237, 1238, 1235, 0, 0, + 0, 1126, 1128, 1130, 1132, 1135, 1967, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 703, 705, 704, + 0, 0, 0, 0, 0, 0, 0, 0, 710, 0, + 1117, 1125, 0, 0, 0, 0, 0, 0, 2003, 0, + 714, 2009, 0, 0, 0, 0, 0, 729, 0, 0, + 0, 0, 0, 1047, 707, 138, 0, 0, 1768, 1100, + 138, 0, 0, 1772, 0, 1765, 0, 2034, 0, 1767, + 1764, 1766, 1770, 1771, 0, 0, 0, 1769, 0, 1123, + 1127, 1129, 1131, 1133, 1134, 1136, 138, 1141, 1137, 1138, + 1139, 1140, 0, 1118, 1119, 1120, 1121, 1098, 1099, 1124, + 0, 1101, 0, 1103, 1104, 1105, 1106, 1102, 1107, 1108, + 1109, 1110, 1111, 1114, 1116, 1112, 1113, 1122, 0, 703, + 705, 704, 0, 0, 0, 1126, 1128, 1130, 1132, 1135, + 710, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 714, 0, 0, 0, 0, 0, 0, 729, + 0, 0, 0, 0, 0, 0, 707, 2206, 0, 0, + 0, 0, 0, 0, 1117, 0, 709, 713, 719, 0, + 720, 722, 0, 0, 723, 724, 725, 0, 0, 727, + 728, 1250, 1249, 1259, 1260, 1252, 1253, 1254, 1255, 1256, + 1257, 1258, 1251, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1753, 1754, 1755, 1756, 1757, 1758, 1759, 1760, 1761, 1762, + 1763, 1775, 1776, 1777, 1778, 1779, 1780, 1773, 1774, 0, + 1768, 0, 0, 0, 0, 0, 0, 1765, 0, 0, + 0, 1767, 1764, 1766, 1770, 1771, 0, 0, 0, 1769, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 709, 713, + 719, 0, 720, 722, 0, 0, 723, 724, 725, 0, + 0, 727, 728, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1125, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 706, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 1753, 1754, 1755, 1756, 1757, 1758, 1759, 1760, + 1761, 1762, 1763, 1775, 1776, 1777, 1778, 1779, 1780, 1773, + 1774, 0, 0, 0, 0, 0, 138, 0, 0, 0, 0, 0, 0, 138, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 3848, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 706, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1125, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 802, 0, 0, 0, 0, 0, 0, - 1124, 0, 391, 0, 519, 552, 541, 625, 507, 0, - 0, 0, 0, 0, 0, 754, 0, 0, 0, 330, - 0, 0, 360, 556, 538, 548, 539, 524, 525, 526, - 533, 340, 527, 528, 529, 499, 530, 500, 531, 532, - 793, 555, 506, 423, 375, 573, 572, 0, 0, 861, - 869, 0, 0, 0, 0, 3923, 0, 0, 0, 0, + 0, 0, 2034, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 802, + 0, 0, 0, 0, 0, 0, 0, 0, 391, 0, + 519, 552, 541, 625, 507, 0, 0, 0, 0, 0, + 0, 754, 0, 0, 0, 330, 138, 0, 360, 556, + 538, 548, 539, 524, 525, 526, 533, 340, 527, 528, + 529, 499, 530, 500, 531, 532, 793, 555, 506, 423, + 375, 573, 572, 0, 0, 862, 870, 0, 0, 0, + 0, 0, 0, 0, 0, 858, 0, 0, 0, 0, + 746, 0, 0, 783, 838, 837, 770, 780, 0, 0, + 303, 223, 501, 621, 503, 502, 771, 0, 772, 776, + 779, 775, 773, 774, 0, 853, 0, 0, 0, 0, + 0, 0, 738, 750, 0, 755, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 138, 0, 0, 0, 0, 0, 0, 747, + 748, 0, 0, 0, 0, 803, 0, 749, 0, 0, + 798, 777, 781, 0, 0, 0, 0, 293, 429, 447, + 304, 418, 460, 309, 426, 299, 390, 415, 0, 0, + 295, 445, 425, 372, 350, 351, 294, 0, 409, 328, + 342, 325, 388, 778, 801, 805, 324, 876, 799, 455, + 297, 0, 454, 387, 441, 446, 373, 367, 0, 296, + 443, 371, 366, 354, 332, 877, 355, 356, 346, 399, + 364, 400, 347, 377, 376, 378, 0, 0, 0, 0, + 0, 483, 484, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 614, 796, 0, 618, + 0, 457, 0, 0, 860, 0, 0, 0, 428, 0, + 0, 357, 0, 0, 0, 800, 0, 412, 393, 873, + 0, 0, 410, 362, 442, 401, 448, 430, 456, 406, + 402, 288, 431, 327, 374, 300, 302, 322, 329, 331, + 333, 334, 383, 384, 396, 417, 433, 434, 435, 326, + 310, 411, 311, 344, 312, 289, 318, 316, 319, 419, + 320, 291, 397, 439, 0, 339, 407, 370, 292, 369, + 398, 438, 437, 301, 464, 470, 471, 560, 138, 476, + 645, 646, 647, 485, 0, 403, 490, 491, 492, 494, + 495, 496, 497, 561, 578, 545, 515, 478, 569, 512, + 516, 517, 581, 1796, 1795, 1797, 469, 358, 359, 0, + 337, 285, 286, 640, 857, 389, 583, 616, 617, 508, + 0, 872, 852, 854, 855, 859, 863, 864, 865, 866, + 867, 869, 871, 875, 639, 0, 562, 577, 643, 576, + 636, 395, 0, 416, 574, 521, 0, 566, 540, 0, + 567, 536, 571, 0, 510, 0, 424, 450, 462, 479, + 482, 511, 596, 597, 598, 290, 481, 600, 601, 602, + 603, 604, 605, 606, 599, 874, 543, 520, 546, 461, + 523, 522, 0, 0, 557, 804, 558, 559, 379, 380, + 381, 382, 861, 584, 308, 480, 405, 0, 544, 0, + 0, 0, 0, 0, 0, 0, 0, 549, 550, 547, + 648, 0, 607, 608, 0, 0, 474, 475, 336, 343, + 493, 345, 307, 394, 338, 459, 352, 0, 486, 551, + 487, 610, 613, 611, 612, 386, 348, 349, 420, 353, + 363, 408, 458, 392, 413, 305, 449, 422, 368, 537, + 564, 883, 856, 882, 884, 885, 881, 886, 887, 868, + 759, 0, 811, 879, 878, 880, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 592, 591, 590, + 589, 588, 587, 586, 585, 0, 0, 534, 436, 317, + 279, 313, 314, 321, 637, 634, 440, 638, 766, 287, + 514, 361, 0, 404, 335, 579, 580, 0, 0, 845, + 818, 819, 820, 756, 821, 815, 816, 757, 817, 846, + 809, 842, 843, 785, 812, 822, 841, 823, 844, 847, + 848, 888, 889, 829, 813, 251, 890, 826, 849, 840, + 839, 824, 810, 850, 851, 792, 787, 827, 828, 814, + 833, 834, 835, 758, 806, 807, 808, 830, 831, 788, + 789, 790, 791, 0, 0, 0, 465, 466, 467, 489, + 0, 451, 513, 635, 0, 0, 0, 0, 0, 0, + 0, 563, 575, 609, 0, 619, 620, 622, 624, 836, + 626, 427, 0, 832, 629, 630, 627, 365, 414, 432, + 421, 802, 641, 504, 505, 642, 615, 0, 751, 0, + 391, 0, 519, 552, 541, 625, 507, 0, 0, 0, + 0, 0, 0, 754, 0, 0, 0, 330, 1847, 0, + 360, 556, 538, 548, 539, 524, 525, 526, 533, 340, + 527, 528, 529, 499, 530, 500, 531, 532, 793, 555, + 506, 423, 375, 573, 572, 0, 0, 862, 870, 0, + 0, 0, 0, 0, 0, 0, 0, 858, 0, 2060, + 0, 0, 746, 0, 0, 783, 838, 837, 770, 780, + 0, 0, 303, 223, 501, 621, 503, 502, 771, 0, + 772, 776, 779, 775, 773, 774, 0, 853, 0, 0, + 0, 0, 0, 0, 738, 750, 0, 755, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 747, 748, 0, 0, 0, 0, 803, 0, 749, + 0, 0, 2061, 777, 781, 0, 0, 0, 0, 293, + 429, 447, 304, 418, 460, 309, 426, 299, 390, 415, + 0, 0, 295, 445, 425, 372, 350, 351, 294, 0, + 409, 328, 342, 325, 388, 778, 801, 805, 324, 876, + 799, 455, 297, 0, 454, 387, 441, 446, 373, 367, + 0, 296, 443, 371, 366, 354, 332, 877, 355, 356, + 346, 399, 364, 400, 347, 377, 376, 378, 0, 0, + 0, 0, 0, 483, 484, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 614, 796, + 0, 618, 0, 457, 0, 0, 860, 0, 0, 0, + 428, 0, 0, 357, 0, 0, 0, 800, 0, 412, + 393, 873, 0, 0, 410, 362, 442, 401, 448, 430, + 456, 406, 402, 288, 431, 327, 374, 300, 302, 322, + 329, 331, 333, 334, 383, 384, 396, 417, 433, 434, + 435, 326, 310, 411, 311, 344, 312, 289, 318, 316, + 319, 419, 320, 291, 397, 439, 0, 339, 407, 370, + 292, 369, 398, 438, 437, 301, 464, 470, 471, 560, + 0, 476, 645, 646, 647, 485, 0, 403, 490, 491, + 492, 494, 495, 496, 497, 561, 578, 545, 515, 478, + 569, 512, 516, 517, 581, 0, 0, 0, 469, 358, + 359, 0, 337, 285, 286, 640, 857, 389, 583, 616, + 617, 508, 0, 872, 852, 854, 855, 859, 863, 864, + 865, 866, 867, 869, 871, 875, 639, 0, 562, 577, + 643, 576, 636, 395, 0, 416, 574, 521, 0, 566, + 540, 0, 567, 536, 571, 0, 510, 0, 424, 450, + 462, 479, 482, 511, 596, 597, 598, 290, 481, 600, + 601, 602, 603, 604, 605, 606, 599, 874, 543, 520, + 546, 461, 523, 522, 0, 0, 557, 804, 558, 559, + 379, 380, 381, 382, 861, 584, 308, 480, 405, 0, + 544, 0, 0, 0, 0, 0, 0, 0, 0, 549, + 550, 547, 648, 0, 607, 608, 0, 0, 474, 475, + 336, 343, 493, 345, 307, 394, 338, 459, 352, 0, + 486, 551, 487, 610, 613, 611, 612, 386, 348, 349, + 420, 353, 363, 408, 458, 392, 413, 305, 449, 422, + 368, 537, 564, 883, 856, 882, 884, 885, 881, 886, + 887, 868, 759, 0, 811, 879, 878, 880, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 592, + 591, 590, 589, 588, 587, 586, 585, 0, 0, 534, + 436, 317, 279, 313, 314, 321, 637, 634, 440, 638, + 766, 287, 514, 361, 0, 404, 335, 579, 580, 0, + 0, 845, 818, 819, 820, 756, 821, 815, 816, 757, + 817, 846, 809, 842, 843, 785, 812, 822, 841, 823, + 844, 847, 848, 888, 889, 829, 813, 251, 890, 826, + 849, 840, 839, 824, 810, 850, 851, 792, 787, 827, + 828, 814, 833, 834, 835, 758, 806, 807, 808, 830, + 831, 788, 789, 790, 791, 0, 0, 0, 465, 466, + 467, 489, 0, 451, 513, 635, 0, 0, 0, 0, + 0, 0, 0, 563, 575, 609, 0, 619, 620, 622, + 624, 836, 626, 427, 0, 832, 629, 630, 627, 365, + 414, 432, 421, 0, 641, 504, 505, 642, 615, 0, + 751, 200, 802, 0, 0, 0, 0, 0, 0, 0, + 0, 391, 0, 519, 552, 541, 625, 507, 0, 0, + 0, 0, 0, 0, 754, 0, 0, 0, 330, 0, + 0, 360, 556, 538, 548, 539, 524, 525, 526, 533, + 340, 527, 528, 529, 499, 530, 500, 531, 532, 1271, + 555, 506, 423, 375, 573, 572, 0, 0, 862, 870, + 0, 0, 0, 0, 0, 0, 0, 0, 858, 0, 0, 0, 0, 746, 0, 0, 783, 838, 837, 770, 780, 0, 0, 303, 223, 501, 621, 503, 502, 771, 0, 772, 776, 779, 775, 773, 774, 0, 853, 0, - 0, 0, 1124, 0, 0, 738, 750, 0, 755, 0, + 0, 0, 0, 0, 0, 738, 750, 0, 755, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 747, 748, 0, 0, 0, 0, 803, 3923, + 0, 0, 747, 748, 0, 0, 0, 0, 803, 0, 749, 0, 0, 798, 777, 781, 0, 0, 0, 0, 293, 429, 447, 304, 418, 460, 309, 426, 299, 390, 415, 0, 0, 295, 445, 425, 372, 350, 351, 294, 0, 409, 328, 342, 325, 388, 778, 801, 805, 324, - 875, 799, 455, 297, 0, 454, 387, 441, 446, 373, - 367, 3923, 296, 443, 371, 366, 354, 332, 876, 355, + 876, 799, 455, 297, 0, 454, 387, 441, 446, 373, + 367, 0, 296, 443, 371, 366, 354, 332, 877, 355, 356, 346, 399, 364, 400, 347, 377, 376, 378, 0, 0, 0, 0, 0, 483, 484, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 614, - 796, 0, 618, 0, 457, 0, 0, 859, 0, 0, - 0, 428, 0, 1124, 357, 0, 0, 0, 800, 0, - 412, 393, 872, 4039, 0, 410, 362, 442, 401, 448, + 796, 0, 618, 0, 457, 0, 0, 860, 0, 0, + 0, 428, 0, 0, 357, 0, 0, 0, 800, 0, + 412, 393, 873, 0, 0, 410, 362, 442, 401, 448, 430, 456, 406, 402, 288, 431, 327, 374, 300, 302, 322, 329, 331, 333, 334, 383, 384, 396, 417, 433, 434, 435, 326, 310, 411, 311, 344, 312, 289, 318, @@ -2345,30 +2493,30 @@ var yyAct = [...]int{ 370, 292, 369, 398, 438, 437, 301, 464, 470, 471, 560, 0, 476, 645, 646, 647, 485, 0, 403, 490, 491, 492, 494, 495, 496, 497, 561, 578, 545, 515, - 478, 569, 512, 516, 517, 581, 1795, 1794, 1796, 469, + 478, 569, 512, 516, 517, 581, 0, 0, 0, 469, 358, 359, 0, 337, 285, 286, 640, 857, 389, 583, - 616, 617, 508, 0, 871, 852, 854, 855, 858, 862, - 863, 864, 865, 866, 868, 870, 874, 639, 0, 562, + 616, 617, 508, 0, 872, 852, 854, 855, 859, 863, + 864, 865, 866, 867, 869, 871, 875, 639, 0, 562, 577, 643, 576, 636, 395, 0, 416, 574, 521, 0, 566, 540, 0, 567, 536, 571, 0, 510, 0, 424, 450, 462, 479, 482, 511, 596, 597, 598, 290, 481, - 600, 601, 602, 603, 604, 605, 606, 599, 873, 543, + 600, 601, 602, 603, 604, 605, 606, 599, 874, 543, 520, 546, 461, 523, 522, 0, 0, 557, 804, 558, - 559, 379, 380, 381, 382, 860, 584, 308, 480, 405, + 559, 379, 380, 381, 382, 861, 584, 308, 480, 405, 0, 544, 0, 0, 0, 0, 0, 0, 0, 0, 549, 550, 547, 648, 0, 607, 608, 0, 0, 474, 475, 336, 343, 493, 345, 307, 394, 338, 459, 352, 0, 486, 551, 487, 610, 613, 611, 612, 386, 348, 349, 420, 353, 363, 408, 458, 392, 413, 305, 449, - 422, 368, 537, 564, 882, 856, 881, 883, 884, 880, - 885, 886, 867, 759, 0, 811, 878, 877, 879, 0, + 422, 368, 537, 564, 883, 856, 882, 884, 885, 881, + 886, 887, 868, 759, 0, 811, 879, 878, 880, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 592, 591, 590, 589, 588, 587, 586, 585, 0, 0, 534, 436, 317, 279, 313, 314, 321, 637, 634, 440, - 638, 766, 287, 514, 361, 0, 404, 335, 579, 580, + 638, 766, 287, 514, 361, 163, 404, 335, 579, 580, 0, 0, 845, 818, 819, 820, 756, 821, 815, 816, 757, 817, 846, 809, 842, 843, 785, 812, 822, 841, - 823, 844, 847, 848, 887, 888, 829, 813, 251, 889, + 823, 844, 847, 848, 888, 889, 829, 813, 251, 890, 826, 849, 840, 839, 824, 810, 850, 851, 792, 787, 827, 828, 814, 833, 834, 835, 758, 806, 807, 808, 830, 831, 788, 789, 790, 791, 0, 0, 0, 465, @@ -2378,138 +2526,11 @@ var yyAct = [...]int{ 365, 414, 432, 421, 802, 641, 504, 505, 642, 615, 0, 751, 0, 391, 0, 519, 552, 541, 625, 507, 0, 0, 0, 0, 0, 0, 754, 0, 0, 0, - 330, 1846, 0, 360, 556, 538, 548, 539, 524, 525, + 330, 4044, 0, 360, 556, 538, 548, 539, 524, 525, 526, 533, 340, 527, 528, 529, 499, 530, 500, 531, 532, 793, 555, 506, 423, 375, 573, 572, 0, 0, - 861, 869, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 2059, 0, 0, 746, 0, 0, 783, 838, 837, - 770, 780, 0, 0, 303, 223, 501, 621, 503, 502, - 771, 0, 772, 776, 779, 775, 773, 774, 0, 853, - 0, 0, 0, 0, 0, 0, 738, 750, 0, 755, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 747, 748, 0, 0, 0, 0, 803, - 0, 749, 0, 0, 2060, 777, 781, 0, 0, 0, - 0, 293, 429, 447, 304, 418, 460, 309, 426, 299, - 390, 415, 0, 0, 295, 445, 425, 372, 350, 351, - 294, 0, 409, 328, 342, 325, 388, 778, 801, 805, - 324, 875, 799, 455, 297, 0, 454, 387, 441, 446, - 373, 367, 0, 296, 443, 371, 366, 354, 332, 876, - 355, 356, 346, 399, 364, 400, 347, 377, 376, 378, - 0, 0, 0, 0, 0, 483, 484, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 614, 796, 0, 618, 0, 457, 0, 0, 859, 0, - 0, 0, 428, 0, 0, 357, 0, 0, 0, 800, - 0, 412, 393, 872, 0, 0, 410, 362, 442, 401, - 448, 430, 456, 406, 402, 288, 431, 327, 374, 300, - 302, 322, 329, 331, 333, 334, 383, 384, 396, 417, - 433, 434, 435, 326, 310, 411, 311, 344, 312, 289, - 318, 316, 319, 419, 320, 291, 397, 439, 0, 339, - 407, 370, 292, 369, 398, 438, 437, 301, 464, 470, - 471, 560, 0, 476, 645, 646, 647, 485, 0, 403, - 490, 491, 492, 494, 495, 496, 497, 561, 578, 545, - 515, 478, 569, 512, 516, 517, 581, 0, 0, 0, - 469, 358, 359, 0, 337, 285, 286, 640, 857, 389, - 583, 616, 617, 508, 0, 871, 852, 854, 855, 858, - 862, 863, 864, 865, 866, 868, 870, 874, 639, 0, - 562, 577, 643, 576, 636, 395, 0, 416, 574, 521, - 0, 566, 540, 0, 567, 536, 571, 0, 510, 0, - 424, 450, 462, 479, 482, 511, 596, 597, 598, 290, - 481, 600, 601, 602, 603, 604, 605, 606, 599, 873, - 543, 520, 546, 461, 523, 522, 0, 0, 557, 804, - 558, 559, 379, 380, 381, 382, 860, 584, 308, 480, - 405, 0, 544, 0, 0, 0, 0, 0, 0, 0, - 0, 549, 550, 547, 648, 0, 607, 608, 0, 0, - 474, 475, 336, 343, 493, 345, 307, 394, 338, 459, - 352, 0, 486, 551, 487, 610, 613, 611, 612, 386, - 348, 349, 420, 353, 363, 408, 458, 392, 413, 305, - 449, 422, 368, 537, 564, 882, 856, 881, 883, 884, - 880, 885, 886, 867, 759, 0, 811, 878, 877, 879, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 592, 591, 590, 589, 588, 587, 586, 585, 0, - 0, 534, 436, 317, 279, 313, 314, 321, 637, 634, - 440, 638, 766, 287, 514, 361, 0, 404, 335, 579, - 580, 0, 0, 845, 818, 819, 820, 756, 821, 815, - 816, 757, 817, 846, 809, 842, 843, 785, 812, 822, - 841, 823, 844, 847, 848, 887, 888, 829, 813, 251, - 889, 826, 849, 840, 839, 824, 810, 850, 851, 792, - 787, 827, 828, 814, 833, 834, 835, 758, 806, 807, - 808, 830, 831, 788, 789, 790, 791, 0, 0, 0, - 465, 466, 467, 489, 0, 451, 513, 635, 0, 0, - 0, 0, 0, 0, 0, 563, 575, 609, 0, 619, - 620, 622, 624, 836, 626, 427, 0, 832, 629, 630, - 627, 365, 414, 432, 421, 0, 641, 504, 505, 642, - 615, 0, 751, 200, 802, 0, 0, 0, 0, 0, - 0, 0, 0, 391, 0, 519, 552, 541, 625, 507, - 0, 0, 0, 0, 0, 0, 754, 0, 0, 0, - 330, 0, 0, 360, 556, 538, 548, 539, 524, 525, - 526, 533, 340, 527, 528, 529, 499, 530, 500, 531, - 532, 1270, 555, 506, 423, 375, 573, 572, 0, 0, - 861, 869, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 746, 0, 0, 783, 838, 837, - 770, 780, 0, 0, 303, 223, 501, 621, 503, 502, - 771, 0, 772, 776, 779, 775, 773, 774, 0, 853, - 0, 0, 0, 0, 0, 0, 738, 750, 0, 755, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 747, 748, 0, 0, 0, 0, 803, - 0, 749, 0, 0, 798, 777, 781, 0, 0, 0, - 0, 293, 429, 447, 304, 418, 460, 309, 426, 299, - 390, 415, 0, 0, 295, 445, 425, 372, 350, 351, - 294, 0, 409, 328, 342, 325, 388, 778, 801, 805, - 324, 875, 799, 455, 297, 0, 454, 387, 441, 446, - 373, 367, 0, 296, 443, 371, 366, 354, 332, 876, - 355, 356, 346, 399, 364, 400, 347, 377, 376, 378, - 0, 0, 0, 0, 0, 483, 484, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 614, 796, 0, 618, 0, 457, 0, 0, 859, 0, - 0, 0, 428, 0, 0, 357, 0, 0, 0, 800, - 0, 412, 393, 872, 0, 0, 410, 362, 442, 401, - 448, 430, 456, 406, 402, 288, 431, 327, 374, 300, - 302, 322, 329, 331, 333, 334, 383, 384, 396, 417, - 433, 434, 435, 326, 310, 411, 311, 344, 312, 289, - 318, 316, 319, 419, 320, 291, 397, 439, 0, 339, - 407, 370, 292, 369, 398, 438, 437, 301, 464, 470, - 471, 560, 0, 476, 645, 646, 647, 485, 0, 403, - 490, 491, 492, 494, 495, 496, 497, 561, 578, 545, - 515, 478, 569, 512, 516, 517, 581, 0, 0, 0, - 469, 358, 359, 0, 337, 285, 286, 640, 857, 389, - 583, 616, 617, 508, 0, 871, 852, 854, 855, 858, - 862, 863, 864, 865, 866, 868, 870, 874, 639, 0, - 562, 577, 643, 576, 636, 395, 0, 416, 574, 521, - 0, 566, 540, 0, 567, 536, 571, 0, 510, 0, - 424, 450, 462, 479, 482, 511, 596, 597, 598, 290, - 481, 600, 601, 602, 603, 604, 605, 606, 599, 873, - 543, 520, 546, 461, 523, 522, 0, 0, 557, 804, - 558, 559, 379, 380, 381, 382, 860, 584, 308, 480, - 405, 0, 544, 0, 0, 0, 0, 0, 0, 0, - 0, 549, 550, 547, 648, 0, 607, 608, 0, 0, - 474, 475, 336, 343, 493, 345, 307, 394, 338, 459, - 352, 0, 486, 551, 487, 610, 613, 611, 612, 386, - 348, 349, 420, 353, 363, 408, 458, 392, 413, 305, - 449, 422, 368, 537, 564, 882, 856, 881, 883, 884, - 880, 885, 886, 867, 759, 0, 811, 878, 877, 879, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 592, 591, 590, 589, 588, 587, 586, 585, 0, - 0, 534, 436, 317, 279, 313, 314, 321, 637, 634, - 440, 638, 766, 287, 514, 361, 163, 404, 335, 579, - 580, 0, 0, 845, 818, 819, 820, 756, 821, 815, - 816, 757, 817, 846, 809, 842, 843, 785, 812, 822, - 841, 823, 844, 847, 848, 887, 888, 829, 813, 251, - 889, 826, 849, 840, 839, 824, 810, 850, 851, 792, - 787, 827, 828, 814, 833, 834, 835, 758, 806, 807, - 808, 830, 831, 788, 789, 790, 791, 0, 0, 0, - 465, 466, 467, 489, 0, 451, 513, 635, 0, 0, - 0, 0, 0, 0, 0, 563, 575, 609, 0, 619, - 620, 622, 624, 836, 626, 427, 0, 832, 629, 630, - 627, 365, 414, 432, 421, 802, 641, 504, 505, 642, - 615, 0, 751, 0, 391, 0, 519, 552, 541, 625, - 507, 0, 0, 0, 0, 0, 0, 754, 0, 0, - 0, 330, 4038, 0, 360, 556, 538, 548, 539, 524, - 525, 526, 533, 340, 527, 528, 529, 499, 530, 500, - 531, 532, 793, 555, 506, 423, 375, 573, 572, 0, - 0, 861, 869, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 746, 0, 0, 783, 838, + 862, 870, 0, 0, 0, 0, 0, 0, 0, 0, + 858, 0, 0, 0, 0, 746, 0, 0, 783, 838, 837, 770, 780, 0, 0, 303, 223, 501, 621, 503, 502, 771, 0, 772, 776, 779, 775, 773, 774, 0, 853, 0, 0, 0, 0, 0, 0, 738, 750, 0, @@ -2520,14 +2541,14 @@ var yyAct = [...]int{ 0, 0, 293, 429, 447, 304, 418, 460, 309, 426, 299, 390, 415, 0, 0, 295, 445, 425, 372, 350, 351, 294, 0, 409, 328, 342, 325, 388, 778, 801, - 805, 324, 875, 799, 455, 297, 0, 454, 387, 441, + 805, 324, 876, 799, 455, 297, 0, 454, 387, 441, 446, 373, 367, 0, 296, 443, 371, 366, 354, 332, - 876, 355, 356, 346, 399, 364, 400, 347, 377, 376, + 877, 355, 356, 346, 399, 364, 400, 347, 377, 376, 378, 0, 0, 0, 0, 0, 483, 484, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 614, 796, 0, 618, 0, 457, 0, 0, 859, + 0, 614, 796, 0, 618, 0, 457, 0, 0, 860, 0, 0, 0, 428, 0, 0, 357, 0, 0, 0, - 800, 0, 412, 393, 872, 0, 0, 410, 362, 442, + 800, 0, 412, 393, 873, 0, 0, 410, 362, 442, 401, 448, 430, 456, 406, 402, 288, 431, 327, 374, 300, 302, 322, 329, 331, 333, 334, 383, 384, 396, 417, 433, 434, 435, 326, 310, 411, 311, 344, 312, @@ -2537,29 +2558,29 @@ var yyAct = [...]int{ 403, 490, 491, 492, 494, 495, 496, 497, 561, 578, 545, 515, 478, 569, 512, 516, 517, 581, 0, 0, 0, 469, 358, 359, 0, 337, 285, 286, 640, 857, - 389, 583, 616, 617, 508, 0, 871, 852, 854, 855, - 858, 862, 863, 864, 865, 866, 868, 870, 874, 639, + 389, 583, 616, 617, 508, 0, 872, 852, 854, 855, + 859, 863, 864, 865, 866, 867, 869, 871, 875, 639, 0, 562, 577, 643, 576, 636, 395, 0, 416, 574, 521, 0, 566, 540, 0, 567, 536, 571, 0, 510, 0, 424, 450, 462, 479, 482, 511, 596, 597, 598, 290, 481, 600, 601, 602, 603, 604, 605, 606, 599, - 873, 543, 520, 546, 461, 523, 522, 0, 0, 557, - 804, 558, 559, 379, 380, 381, 382, 860, 584, 308, + 874, 543, 520, 546, 461, 523, 522, 0, 0, 557, + 804, 558, 559, 379, 380, 381, 382, 861, 584, 308, 480, 405, 0, 544, 0, 0, 0, 0, 0, 0, 0, 0, 549, 550, 547, 648, 0, 607, 608, 0, 0, 474, 475, 336, 343, 493, 345, 307, 394, 338, 459, 352, 0, 486, 551, 487, 610, 613, 611, 612, 386, 348, 349, 420, 353, 363, 408, 458, 392, 413, - 305, 449, 422, 368, 537, 564, 882, 856, 881, 883, - 884, 880, 885, 886, 867, 759, 0, 811, 878, 877, - 879, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 305, 449, 422, 368, 537, 564, 883, 856, 882, 884, + 885, 881, 886, 887, 868, 759, 0, 811, 879, 878, + 880, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 592, 591, 590, 589, 588, 587, 586, 585, 0, 0, 534, 436, 317, 279, 313, 314, 321, 637, 634, 440, 638, 766, 287, 514, 361, 0, 404, 335, 579, 580, 0, 0, 845, 818, 819, 820, 756, 821, 815, 816, 757, 817, 846, 809, 842, 843, 785, 812, - 822, 841, 823, 844, 847, 848, 887, 888, 829, 813, - 251, 889, 826, 849, 840, 839, 824, 810, 850, 851, + 822, 841, 823, 844, 847, 848, 888, 889, 829, 813, + 251, 890, 826, 849, 840, 839, 824, 810, 850, 851, 792, 787, 827, 828, 814, 833, 834, 835, 758, 806, 807, 808, 830, 831, 788, 789, 790, 791, 0, 0, 0, 465, 466, 467, 489, 0, 451, 513, 635, 0, @@ -2571,71 +2592,8 @@ var yyAct = [...]int{ 0, 0, 330, 0, 0, 360, 556, 538, 548, 539, 524, 525, 526, 533, 340, 527, 528, 529, 499, 530, 500, 531, 532, 793, 555, 506, 423, 375, 573, 572, - 0, 0, 861, 869, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 746, 0, 0, 783, - 838, 837, 770, 780, 0, 0, 303, 223, 501, 621, - 503, 502, 771, 0, 772, 776, 779, 775, 773, 774, - 0, 853, 0, 0, 0, 0, 0, 0, 738, 750, - 0, 755, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 747, 748, 0, 0, 0, - 0, 803, 0, 749, 0, 0, 798, 777, 781, 0, - 0, 0, 0, 293, 429, 447, 304, 418, 460, 309, - 426, 299, 390, 415, 0, 0, 295, 445, 425, 372, - 350, 351, 294, 0, 409, 328, 342, 325, 388, 778, - 801, 805, 324, 875, 799, 455, 297, 0, 454, 387, - 441, 446, 373, 367, 0, 296, 443, 371, 366, 354, - 332, 876, 355, 356, 346, 399, 364, 400, 347, 377, - 376, 378, 0, 0, 0, 0, 0, 483, 484, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 614, 796, 0, 618, 0, 457, 0, 0, - 859, 0, 0, 0, 428, 0, 0, 357, 0, 0, - 0, 800, 0, 412, 393, 872, 3924, 0, 410, 362, - 442, 401, 448, 430, 456, 406, 402, 288, 431, 327, - 374, 300, 302, 322, 329, 331, 333, 334, 383, 384, - 396, 417, 433, 434, 435, 326, 310, 411, 311, 344, - 312, 289, 318, 316, 319, 419, 320, 291, 397, 439, - 0, 339, 407, 370, 292, 369, 398, 438, 437, 301, - 464, 470, 471, 560, 0, 476, 645, 646, 647, 485, - 0, 403, 490, 491, 492, 494, 495, 496, 497, 561, - 578, 545, 515, 478, 569, 512, 516, 517, 581, 0, - 0, 0, 469, 358, 359, 0, 337, 285, 286, 640, - 857, 389, 583, 616, 617, 508, 0, 871, 852, 854, - 855, 858, 862, 863, 864, 865, 866, 868, 870, 874, - 639, 0, 562, 577, 643, 576, 636, 395, 0, 416, - 574, 521, 0, 566, 540, 0, 567, 536, 571, 0, - 510, 0, 424, 450, 462, 479, 482, 511, 596, 597, - 598, 290, 481, 600, 601, 602, 603, 604, 605, 606, - 599, 873, 543, 520, 546, 461, 523, 522, 0, 0, - 557, 804, 558, 559, 379, 380, 381, 382, 860, 584, - 308, 480, 405, 0, 544, 0, 0, 0, 0, 0, - 0, 0, 0, 549, 550, 547, 648, 0, 607, 608, - 0, 0, 474, 475, 336, 343, 493, 345, 307, 394, - 338, 459, 352, 0, 486, 551, 487, 610, 613, 611, - 612, 386, 348, 349, 420, 353, 363, 408, 458, 392, - 413, 305, 449, 422, 368, 537, 564, 882, 856, 881, - 883, 884, 880, 885, 886, 867, 759, 0, 811, 878, - 877, 879, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 592, 591, 590, 589, 588, 587, 586, - 585, 0, 0, 534, 436, 317, 279, 313, 314, 321, - 637, 634, 440, 638, 766, 287, 514, 361, 0, 404, - 335, 579, 580, 0, 0, 845, 818, 819, 820, 756, - 821, 815, 816, 757, 817, 846, 809, 842, 843, 785, - 812, 822, 841, 823, 844, 847, 848, 887, 888, 829, - 813, 251, 889, 826, 849, 840, 839, 824, 810, 850, - 851, 792, 787, 827, 828, 814, 833, 834, 835, 758, - 806, 807, 808, 830, 831, 788, 789, 790, 791, 0, - 0, 0, 465, 466, 467, 489, 0, 451, 513, 635, - 0, 0, 0, 0, 0, 0, 0, 563, 575, 609, - 0, 619, 620, 622, 624, 836, 626, 427, 0, 832, - 629, 630, 627, 365, 414, 432, 421, 802, 641, 504, - 505, 642, 615, 0, 751, 0, 391, 0, 519, 552, - 541, 625, 507, 0, 0, 0, 0, 0, 0, 754, - 0, 0, 0, 330, 1846, 0, 360, 556, 538, 548, - 539, 524, 525, 526, 533, 340, 527, 528, 529, 499, - 530, 500, 531, 532, 793, 555, 506, 423, 375, 573, - 572, 0, 0, 861, 869, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 746, 0, 0, + 0, 0, 862, 870, 0, 0, 0, 0, 0, 0, + 0, 0, 858, 0, 0, 0, 0, 746, 0, 0, 783, 838, 837, 770, 780, 0, 0, 303, 223, 501, 621, 503, 502, 771, 0, 772, 776, 779, 775, 773, 774, 0, 853, 0, 0, 0, 0, 0, 0, 738, @@ -2646,14 +2604,14 @@ var yyAct = [...]int{ 0, 0, 0, 0, 293, 429, 447, 304, 418, 460, 309, 426, 299, 390, 415, 0, 0, 295, 445, 425, 372, 350, 351, 294, 0, 409, 328, 342, 325, 388, - 778, 801, 805, 324, 875, 799, 455, 297, 0, 454, + 778, 801, 805, 324, 876, 799, 455, 297, 0, 454, 387, 441, 446, 373, 367, 0, 296, 443, 371, 366, - 354, 332, 876, 355, 356, 346, 399, 364, 400, 347, + 354, 332, 877, 355, 356, 346, 399, 364, 400, 347, 377, 376, 378, 0, 0, 0, 0, 0, 483, 484, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 614, 796, 0, 618, 0, 457, 0, - 0, 859, 0, 0, 0, 428, 0, 0, 357, 0, - 0, 0, 800, 0, 412, 393, 872, 0, 0, 410, + 0, 860, 0, 0, 0, 428, 0, 0, 357, 0, + 0, 0, 800, 0, 412, 393, 873, 3930, 0, 410, 362, 442, 401, 448, 430, 456, 406, 402, 288, 431, 327, 374, 300, 302, 322, 329, 331, 333, 334, 383, 384, 396, 417, 433, 434, 435, 326, 310, 411, 311, @@ -2663,29 +2621,29 @@ var yyAct = [...]int{ 485, 0, 403, 490, 491, 492, 494, 495, 496, 497, 561, 578, 545, 515, 478, 569, 512, 516, 517, 581, 0, 0, 0, 469, 358, 359, 0, 337, 285, 286, - 640, 857, 389, 583, 616, 617, 508, 0, 871, 852, - 854, 855, 858, 862, 863, 864, 865, 866, 868, 870, - 874, 639, 0, 562, 577, 643, 576, 636, 395, 0, + 640, 857, 389, 583, 616, 617, 508, 0, 872, 852, + 854, 855, 859, 863, 864, 865, 866, 867, 869, 871, + 875, 639, 0, 562, 577, 643, 576, 636, 395, 0, 416, 574, 521, 0, 566, 540, 0, 567, 536, 571, 0, 510, 0, 424, 450, 462, 479, 482, 511, 596, 597, 598, 290, 481, 600, 601, 602, 603, 604, 605, - 606, 599, 873, 543, 520, 546, 461, 523, 522, 0, - 0, 557, 804, 558, 559, 379, 380, 381, 382, 860, + 606, 599, 874, 543, 520, 546, 461, 523, 522, 0, + 0, 557, 804, 558, 559, 379, 380, 381, 382, 861, 584, 308, 480, 405, 0, 544, 0, 0, 0, 0, 0, 0, 0, 0, 549, 550, 547, 648, 0, 607, 608, 0, 0, 474, 475, 336, 343, 493, 345, 307, 394, 338, 459, 352, 0, 486, 551, 487, 610, 613, 611, 612, 386, 348, 349, 420, 353, 363, 408, 458, - 392, 413, 305, 449, 422, 368, 537, 564, 882, 856, - 881, 883, 884, 880, 885, 886, 867, 759, 0, 811, - 878, 877, 879, 0, 0, 0, 0, 0, 0, 0, + 392, 413, 305, 449, 422, 368, 537, 564, 883, 856, + 882, 884, 885, 881, 886, 887, 868, 759, 0, 811, + 879, 878, 880, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 592, 591, 590, 589, 588, 587, 586, 585, 0, 0, 534, 436, 317, 279, 313, 314, 321, 637, 634, 440, 638, 766, 287, 514, 361, 0, 404, 335, 579, 580, 0, 0, 845, 818, 819, 820, 756, 821, 815, 816, 757, 817, 846, 809, 842, 843, - 785, 812, 822, 841, 823, 844, 847, 848, 887, 888, - 829, 813, 251, 889, 826, 849, 840, 839, 824, 810, + 785, 812, 822, 841, 823, 844, 847, 848, 888, 889, + 829, 813, 251, 890, 826, 849, 840, 839, 824, 810, 850, 851, 792, 787, 827, 828, 814, 833, 834, 835, 758, 806, 807, 808, 830, 831, 788, 789, 790, 791, 0, 0, 0, 465, 466, 467, 489, 0, 451, 513, @@ -2694,329 +2652,13 @@ var yyAct = [...]int{ 832, 629, 630, 627, 365, 414, 432, 421, 802, 641, 504, 505, 642, 615, 0, 751, 0, 391, 0, 519, 552, 541, 625, 507, 0, 0, 0, 0, 0, 0, - 754, 0, 0, 0, 330, 0, 0, 360, 556, 538, + 754, 0, 0, 0, 330, 1847, 0, 360, 556, 538, 548, 539, 524, 525, 526, 533, 340, 527, 528, 529, 499, 530, 500, 531, 532, 793, 555, 506, 423, 375, - 573, 572, 0, 0, 861, 869, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 746, 0, - 0, 783, 838, 837, 770, 780, 0, 0, 303, 223, - 501, 621, 503, 502, 771, 0, 772, 776, 779, 775, - 773, 774, 0, 853, 0, 0, 0, 0, 0, 0, - 738, 750, 0, 755, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 747, 748, 1560, - 0, 0, 0, 803, 0, 749, 0, 0, 798, 777, - 781, 0, 0, 0, 0, 293, 429, 447, 304, 418, - 460, 309, 426, 299, 390, 415, 0, 0, 295, 445, - 425, 372, 350, 351, 294, 0, 409, 328, 342, 325, - 388, 778, 801, 805, 324, 875, 799, 455, 297, 0, - 454, 387, 441, 446, 373, 367, 0, 296, 443, 371, - 366, 354, 332, 876, 355, 356, 346, 399, 364, 400, - 347, 377, 376, 378, 0, 0, 0, 0, 0, 483, - 484, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 614, 796, 0, 618, 0, 457, - 0, 0, 859, 0, 0, 0, 428, 0, 0, 357, - 0, 0, 0, 800, 0, 412, 393, 872, 0, 0, - 410, 362, 442, 401, 448, 430, 456, 406, 402, 288, - 431, 327, 374, 300, 302, 322, 329, 331, 333, 334, - 383, 384, 396, 417, 433, 434, 435, 326, 310, 411, - 311, 344, 312, 289, 318, 316, 319, 419, 320, 291, - 397, 439, 0, 339, 407, 370, 292, 369, 398, 438, - 437, 301, 464, 470, 471, 560, 0, 476, 645, 646, - 647, 485, 0, 403, 490, 491, 492, 494, 495, 496, - 497, 561, 578, 545, 515, 478, 569, 512, 516, 517, - 581, 0, 0, 0, 469, 358, 359, 0, 337, 285, - 286, 640, 857, 389, 583, 616, 617, 508, 0, 871, - 852, 854, 855, 858, 862, 863, 864, 865, 866, 868, - 870, 874, 639, 0, 562, 577, 643, 576, 636, 395, - 0, 416, 574, 521, 0, 566, 540, 0, 567, 536, - 571, 0, 510, 0, 424, 450, 462, 479, 482, 511, - 596, 597, 598, 290, 481, 600, 601, 602, 603, 604, - 605, 606, 599, 873, 543, 520, 546, 461, 523, 522, - 0, 0, 557, 804, 558, 559, 379, 380, 381, 382, - 860, 584, 308, 480, 405, 0, 544, 0, 0, 0, - 0, 0, 0, 0, 0, 549, 550, 547, 648, 0, - 607, 608, 0, 0, 474, 475, 336, 343, 493, 345, - 307, 394, 338, 459, 352, 0, 486, 551, 487, 610, - 613, 611, 612, 386, 348, 349, 420, 353, 363, 408, - 458, 392, 413, 305, 449, 422, 368, 537, 564, 882, - 856, 881, 883, 884, 880, 885, 886, 867, 759, 0, - 811, 878, 877, 879, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 592, 591, 590, 589, 588, - 587, 586, 585, 0, 0, 534, 436, 317, 279, 313, - 314, 321, 637, 634, 440, 638, 766, 287, 514, 361, - 0, 404, 335, 579, 580, 0, 0, 845, 818, 819, - 820, 756, 821, 815, 816, 757, 817, 846, 809, 842, - 843, 785, 812, 822, 841, 823, 844, 847, 848, 887, - 888, 829, 813, 251, 889, 826, 849, 840, 839, 824, - 810, 850, 851, 792, 787, 827, 828, 814, 833, 834, - 835, 758, 806, 807, 808, 830, 831, 788, 789, 790, - 791, 0, 0, 0, 465, 466, 467, 489, 0, 451, - 513, 635, 0, 0, 0, 0, 0, 0, 0, 563, - 575, 609, 0, 619, 620, 622, 624, 836, 626, 427, - 0, 832, 629, 630, 627, 365, 414, 432, 421, 0, - 641, 504, 505, 642, 615, 802, 751, 0, 2233, 0, - 0, 0, 0, 0, 391, 0, 519, 552, 541, 625, - 507, 0, 0, 0, 0, 0, 0, 754, 0, 0, - 0, 330, 0, 0, 360, 556, 538, 548, 539, 524, - 525, 526, 533, 340, 527, 528, 529, 499, 530, 500, - 531, 532, 793, 555, 506, 423, 375, 573, 572, 0, - 0, 861, 869, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 746, 0, 0, 783, 838, - 837, 770, 780, 0, 0, 303, 223, 501, 621, 503, - 502, 771, 0, 772, 776, 779, 775, 773, 774, 0, - 853, 0, 0, 0, 0, 0, 0, 738, 750, 0, - 755, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 747, 748, 0, 0, 0, 0, - 803, 0, 749, 0, 0, 798, 777, 781, 0, 0, - 0, 0, 293, 429, 447, 304, 418, 460, 309, 426, - 299, 390, 415, 0, 0, 295, 445, 425, 372, 350, - 351, 294, 0, 409, 328, 342, 325, 388, 778, 801, - 805, 324, 875, 799, 455, 297, 0, 454, 387, 441, - 446, 373, 367, 0, 296, 443, 371, 366, 354, 332, - 876, 355, 356, 346, 399, 364, 400, 347, 377, 376, - 378, 0, 0, 0, 0, 0, 483, 484, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 614, 796, 0, 618, 0, 457, 0, 0, 859, - 0, 0, 0, 428, 0, 0, 357, 0, 0, 0, - 800, 0, 412, 393, 872, 0, 0, 410, 362, 442, - 401, 448, 430, 456, 406, 402, 288, 431, 327, 374, - 300, 302, 322, 329, 331, 333, 334, 383, 384, 396, - 417, 433, 434, 435, 326, 310, 411, 311, 344, 312, - 289, 318, 316, 319, 419, 320, 291, 397, 439, 0, - 339, 407, 370, 292, 369, 398, 438, 437, 301, 464, - 470, 471, 560, 0, 476, 645, 646, 647, 485, 0, - 403, 490, 491, 492, 494, 495, 496, 497, 561, 578, - 545, 515, 478, 569, 512, 516, 517, 581, 0, 0, - 0, 469, 358, 359, 0, 337, 285, 286, 640, 857, - 389, 583, 616, 617, 508, 0, 871, 852, 854, 855, - 858, 862, 863, 864, 865, 866, 868, 870, 874, 639, - 0, 562, 577, 643, 576, 636, 395, 0, 416, 574, - 521, 0, 566, 540, 0, 567, 536, 571, 0, 510, - 0, 424, 450, 462, 479, 482, 511, 596, 597, 598, - 290, 481, 600, 601, 602, 603, 604, 605, 606, 599, - 873, 543, 520, 546, 461, 523, 522, 0, 0, 557, - 804, 558, 559, 379, 380, 381, 382, 860, 584, 308, - 480, 405, 0, 544, 0, 0, 0, 0, 0, 0, - 0, 0, 549, 550, 547, 648, 0, 607, 608, 0, - 0, 474, 475, 336, 343, 493, 345, 307, 394, 338, - 459, 352, 0, 486, 551, 487, 610, 613, 611, 612, - 386, 348, 349, 420, 353, 363, 408, 458, 392, 413, - 305, 449, 422, 368, 537, 564, 882, 856, 881, 883, - 884, 880, 885, 886, 867, 759, 0, 811, 878, 877, - 879, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 592, 591, 590, 589, 588, 587, 586, 585, - 0, 0, 534, 436, 317, 279, 313, 314, 321, 637, - 634, 440, 638, 766, 287, 514, 361, 0, 404, 335, - 579, 580, 0, 0, 845, 818, 819, 820, 756, 821, - 815, 816, 757, 817, 846, 809, 842, 843, 785, 812, - 822, 841, 823, 844, 847, 848, 887, 888, 829, 813, - 251, 889, 826, 849, 840, 839, 824, 810, 850, 851, - 792, 787, 827, 828, 814, 833, 834, 835, 758, 806, - 807, 808, 830, 831, 788, 789, 790, 791, 0, 0, - 0, 465, 466, 467, 489, 0, 451, 513, 635, 0, - 0, 0, 0, 0, 0, 0, 563, 575, 609, 0, - 619, 620, 622, 624, 836, 626, 427, 0, 832, 629, - 630, 627, 365, 414, 432, 421, 802, 641, 504, 505, - 642, 615, 0, 751, 0, 391, 0, 519, 552, 541, - 625, 507, 0, 0, 0, 0, 0, 0, 754, 0, - 0, 0, 330, 0, 0, 360, 556, 538, 548, 539, - 524, 525, 526, 533, 340, 527, 528, 529, 499, 530, - 500, 531, 532, 793, 555, 506, 423, 375, 573, 572, - 0, 0, 861, 869, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 746, 0, 0, 783, - 838, 837, 770, 780, 0, 0, 303, 223, 501, 621, - 503, 502, 771, 0, 772, 776, 779, 775, 773, 774, - 0, 853, 0, 0, 0, 0, 0, 0, 738, 750, - 0, 755, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 747, 748, 1839, 0, 0, - 0, 803, 0, 749, 0, 0, 798, 777, 781, 0, - 0, 0, 0, 293, 429, 447, 304, 418, 460, 309, - 426, 299, 390, 415, 0, 0, 295, 445, 425, 372, - 350, 351, 294, 0, 409, 328, 342, 325, 388, 778, - 801, 805, 324, 875, 799, 455, 297, 0, 454, 387, - 441, 446, 373, 367, 0, 296, 443, 371, 366, 354, - 332, 876, 355, 356, 346, 399, 364, 400, 347, 377, - 376, 378, 0, 0, 0, 0, 0, 483, 484, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 614, 796, 0, 618, 0, 457, 0, 0, - 859, 0, 0, 0, 428, 0, 0, 357, 0, 0, - 0, 800, 0, 412, 393, 872, 0, 0, 410, 362, - 442, 401, 448, 430, 456, 406, 402, 288, 431, 327, - 374, 300, 302, 322, 329, 331, 333, 334, 383, 384, - 396, 417, 433, 434, 435, 326, 310, 411, 311, 344, - 312, 289, 318, 316, 319, 419, 320, 291, 397, 439, - 0, 339, 407, 370, 292, 369, 398, 438, 437, 301, - 464, 470, 471, 560, 0, 476, 645, 646, 647, 485, - 0, 403, 490, 491, 492, 494, 495, 496, 497, 561, - 578, 545, 515, 478, 569, 512, 516, 517, 581, 0, - 0, 0, 469, 358, 359, 0, 337, 285, 286, 640, - 857, 389, 583, 616, 617, 508, 0, 871, 852, 854, - 855, 858, 862, 863, 864, 865, 866, 868, 870, 874, - 639, 0, 562, 577, 643, 576, 636, 395, 0, 416, - 574, 521, 0, 566, 540, 0, 567, 536, 571, 0, - 510, 0, 424, 450, 462, 479, 482, 511, 596, 597, - 598, 290, 481, 600, 601, 602, 603, 604, 605, 606, - 599, 873, 543, 520, 546, 461, 523, 522, 0, 0, - 557, 804, 558, 559, 379, 380, 381, 382, 860, 584, - 308, 480, 405, 0, 544, 0, 0, 0, 0, 0, - 0, 0, 0, 549, 550, 547, 648, 0, 607, 608, - 0, 0, 474, 475, 336, 343, 493, 345, 307, 394, - 338, 459, 352, 0, 486, 551, 487, 610, 613, 611, - 612, 386, 348, 349, 420, 353, 363, 408, 458, 392, - 413, 305, 449, 422, 368, 537, 564, 882, 856, 881, - 883, 884, 880, 885, 886, 867, 759, 0, 811, 878, - 877, 879, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 592, 591, 590, 589, 588, 587, 586, - 585, 0, 0, 534, 436, 317, 279, 313, 314, 321, - 637, 634, 440, 638, 766, 287, 514, 361, 0, 404, - 335, 579, 580, 0, 0, 845, 818, 819, 820, 756, - 821, 815, 816, 757, 817, 846, 809, 842, 843, 785, - 812, 822, 841, 823, 844, 847, 848, 887, 888, 829, - 813, 251, 889, 826, 849, 840, 839, 824, 810, 850, - 851, 792, 787, 827, 828, 814, 833, 834, 835, 758, - 806, 807, 808, 830, 831, 788, 789, 790, 791, 0, - 0, 0, 465, 466, 467, 489, 0, 451, 513, 635, - 0, 0, 0, 0, 0, 0, 0, 563, 575, 609, - 0, 619, 620, 622, 624, 836, 626, 427, 0, 832, - 629, 630, 627, 365, 414, 432, 421, 802, 641, 504, - 505, 642, 615, 0, 751, 0, 391, 0, 519, 552, - 541, 625, 507, 0, 0, 0, 0, 0, 0, 754, - 0, 0, 0, 330, 0, 0, 360, 556, 538, 548, - 539, 524, 525, 526, 533, 340, 527, 528, 529, 499, - 530, 500, 531, 532, 793, 555, 506, 423, 375, 573, - 572, 0, 0, 861, 869, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 746, 0, 0, - 783, 838, 837, 770, 780, 0, 0, 303, 223, 501, - 621, 503, 502, 771, 0, 772, 776, 779, 775, 773, - 774, 0, 853, 0, 0, 0, 0, 0, 0, 738, - 750, 0, 755, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 747, 748, 0, 0, - 0, 0, 803, 0, 749, 0, 0, 798, 777, 781, - 0, 0, 0, 0, 293, 429, 447, 304, 418, 460, - 309, 426, 299, 390, 415, 0, 0, 295, 445, 425, - 372, 350, 351, 294, 0, 409, 328, 342, 325, 388, - 778, 801, 805, 324, 875, 799, 455, 297, 0, 454, - 387, 441, 446, 373, 367, 0, 296, 443, 371, 366, - 354, 332, 876, 355, 356, 346, 399, 364, 400, 347, - 377, 376, 378, 0, 0, 0, 0, 0, 483, 484, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 614, 796, 0, 618, 0, 457, 0, - 0, 859, 0, 0, 0, 428, 0, 0, 357, 0, - 0, 0, 800, 0, 412, 393, 872, 0, 0, 410, - 362, 442, 401, 448, 430, 456, 406, 402, 288, 431, - 327, 374, 300, 302, 322, 329, 331, 333, 334, 383, - 384, 396, 417, 433, 434, 435, 326, 310, 411, 311, - 344, 312, 289, 318, 316, 319, 419, 320, 291, 397, - 439, 0, 339, 407, 370, 292, 369, 398, 438, 437, - 301, 464, 470, 471, 560, 0, 476, 645, 646, 647, - 485, 0, 403, 490, 491, 492, 494, 495, 496, 497, - 561, 578, 545, 515, 478, 569, 512, 516, 517, 581, - 0, 0, 0, 469, 358, 359, 0, 337, 285, 286, - 640, 857, 389, 583, 616, 617, 508, 0, 871, 852, - 854, 855, 858, 862, 863, 864, 865, 866, 868, 870, - 874, 639, 0, 562, 577, 643, 576, 636, 395, 0, - 416, 574, 521, 0, 566, 540, 0, 567, 536, 571, - 0, 510, 0, 424, 450, 462, 479, 482, 511, 596, - 597, 598, 290, 481, 600, 601, 602, 603, 604, 605, - 606, 599, 873, 543, 520, 546, 461, 523, 522, 0, - 0, 557, 804, 558, 559, 379, 380, 381, 382, 860, - 584, 308, 480, 405, 0, 544, 0, 0, 0, 0, - 0, 0, 0, 0, 549, 550, 547, 648, 0, 607, - 608, 0, 0, 474, 475, 336, 343, 493, 345, 307, - 394, 338, 459, 352, 0, 486, 551, 487, 610, 613, - 611, 612, 386, 348, 349, 420, 353, 363, 408, 458, - 392, 413, 305, 449, 422, 368, 537, 564, 882, 856, - 881, 883, 884, 880, 885, 886, 867, 759, 0, 811, - 878, 877, 879, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 592, 591, 590, 589, 588, 587, - 586, 585, 0, 0, 534, 436, 317, 279, 313, 314, - 321, 637, 634, 440, 638, 766, 287, 514, 361, 0, - 404, 335, 579, 580, 0, 0, 845, 818, 819, 820, - 756, 821, 815, 816, 757, 817, 846, 809, 842, 843, - 785, 812, 822, 841, 823, 844, 847, 848, 887, 888, - 829, 813, 251, 889, 826, 849, 840, 839, 824, 810, - 850, 851, 792, 787, 827, 828, 814, 833, 834, 835, - 758, 806, 807, 808, 830, 831, 788, 789, 790, 791, - 0, 0, 0, 465, 466, 467, 489, 0, 451, 513, - 635, 0, 0, 0, 0, 0, 0, 0, 563, 575, - 609, 0, 619, 620, 622, 624, 836, 626, 427, 0, - 832, 629, 630, 627, 365, 414, 432, 421, 802, 641, - 504, 505, 642, 615, 0, 751, 0, 391, 0, 519, - 552, 541, 625, 507, 0, 0, 0, 0, 0, 0, - 754, 0, 0, 0, 330, 0, 0, 360, 556, 538, - 548, 539, 524, 525, 526, 533, 340, 527, 528, 529, - 499, 530, 500, 531, 532, 793, 555, 506, 423, 375, - 573, 572, 0, 0, 861, 869, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 746, 0, - 0, 783, 838, 837, 770, 780, 0, 0, 303, 223, - 501, 621, 503, 502, 771, 0, 772, 776, 779, 775, - 773, 774, 0, 853, 0, 0, 0, 0, 0, 0, - 738, 750, 0, 755, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 747, 748, 0, - 0, 0, 0, 803, 0, 749, 0, 0, 798, 777, - 781, 0, 0, 0, 0, 293, 429, 447, 304, 418, - 460, 309, 426, 299, 390, 415, 0, 0, 295, 445, - 425, 372, 350, 351, 294, 0, 409, 328, 342, 325, - 388, 778, 801, 805, 324, 875, 799, 455, 297, 0, - 454, 387, 441, 446, 373, 367, 0, 296, 443, 371, - 366, 354, 332, 876, 355, 356, 346, 399, 364, 400, - 347, 377, 376, 378, 0, 0, 0, 0, 0, 483, - 484, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 614, 796, 0, 618, 0, 457, - 0, 0, 859, 0, 0, 0, 428, 0, 0, 357, - 0, 0, 0, 800, 0, 412, 393, 872, 0, 0, - 410, 362, 442, 401, 448, 430, 456, 406, 402, 288, - 431, 327, 374, 300, 302, 322, 329, 331, 333, 334, - 383, 384, 396, 417, 433, 434, 435, 326, 310, 411, - 311, 344, 312, 289, 318, 316, 319, 419, 320, 291, - 397, 439, 0, 339, 407, 370, 292, 369, 398, 438, - 437, 301, 464, 470, 471, 560, 0, 476, 645, 646, - 647, 485, 0, 403, 490, 491, 492, 494, 495, 496, - 497, 561, 578, 545, 515, 478, 569, 512, 516, 517, - 581, 0, 0, 0, 469, 358, 359, 0, 337, 285, - 286, 640, 857, 389, 583, 616, 617, 508, 0, 871, - 852, 854, 855, 858, 862, 863, 864, 865, 866, 868, - 870, 874, 639, 0, 562, 577, 643, 576, 636, 395, - 0, 416, 574, 521, 0, 566, 540, 0, 567, 536, - 571, 0, 510, 0, 424, 450, 462, 479, 482, 511, - 596, 597, 598, 290, 481, 600, 601, 602, 603, 604, - 605, 606, 599, 873, 543, 520, 546, 461, 523, 522, - 0, 0, 557, 804, 558, 559, 379, 380, 381, 382, - 860, 584, 308, 480, 405, 0, 544, 0, 0, 0, - 0, 0, 0, 0, 0, 549, 550, 547, 648, 0, - 607, 608, 0, 0, 474, 475, 336, 343, 493, 345, - 307, 394, 338, 459, 352, 0, 486, 551, 487, 610, - 613, 611, 612, 386, 348, 349, 420, 353, 363, 408, - 458, 392, 413, 305, 449, 422, 368, 537, 564, 882, - 856, 881, 883, 884, 880, 885, 886, 867, 759, 0, - 811, 878, 877, 879, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 592, 591, 590, 589, 588, - 587, 586, 585, 0, 0, 534, 436, 317, 279, 313, - 314, 321, 637, 634, 440, 638, 766, 287, 514, 361, - 0, 404, 335, 579, 580, 0, 0, 845, 818, 819, - 820, 756, 821, 815, 816, 757, 817, 846, 809, 842, - 843, 785, 812, 822, 841, 823, 844, 847, 848, 887, - 888, 829, 813, 251, 889, 826, 849, 840, 839, 824, - 810, 850, 851, 792, 787, 827, 828, 814, 833, 834, - 835, 758, 806, 807, 808, 830, 831, 788, 789, 790, - 791, 0, 0, 0, 465, 466, 467, 489, 0, 451, - 513, 635, 0, 0, 0, 0, 0, 0, 0, 563, - 575, 609, 0, 619, 620, 622, 624, 836, 626, 427, - 0, 3502, 629, 3503, 3504, 365, 414, 432, 421, 802, - 641, 504, 505, 642, 615, 0, 751, 0, 391, 0, - 519, 552, 541, 625, 507, 0, 0, 0, 0, 0, - 0, 754, 0, 0, 0, 330, 0, 0, 360, 556, - 538, 548, 539, 524, 525, 526, 533, 340, 527, 528, - 529, 499, 530, 500, 531, 532, 793, 555, 506, 423, - 375, 573, 572, 0, 0, 861, 869, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 746, + 573, 572, 0, 0, 862, 870, 0, 0, 0, 0, + 0, 0, 0, 0, 858, 0, 0, 0, 0, 746, 0, 0, 783, 838, 837, 770, 780, 0, 0, 303, - 223, 501, 621, 503, 502, 2706, 0, 2707, 776, 779, + 223, 501, 621, 503, 502, 771, 0, 772, 776, 779, 775, 773, 774, 0, 853, 0, 0, 0, 0, 0, 0, 738, 750, 0, 755, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -3025,14 +2667,14 @@ var yyAct = [...]int{ 777, 781, 0, 0, 0, 0, 293, 429, 447, 304, 418, 460, 309, 426, 299, 390, 415, 0, 0, 295, 445, 425, 372, 350, 351, 294, 0, 409, 328, 342, - 325, 388, 778, 801, 805, 324, 875, 799, 455, 297, + 325, 388, 778, 801, 805, 324, 876, 799, 455, 297, 0, 454, 387, 441, 446, 373, 367, 0, 296, 443, - 371, 366, 354, 332, 876, 355, 356, 346, 399, 364, + 371, 366, 354, 332, 877, 355, 356, 346, 399, 364, 400, 347, 377, 376, 378, 0, 0, 0, 0, 0, 483, 484, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 614, 796, 0, 618, 0, - 457, 0, 0, 859, 0, 0, 0, 428, 0, 0, - 357, 0, 0, 0, 800, 0, 412, 393, 872, 0, + 457, 0, 0, 860, 0, 0, 0, 428, 0, 0, + 357, 0, 0, 0, 800, 0, 412, 393, 873, 0, 0, 410, 362, 442, 401, 448, 430, 456, 406, 402, 288, 431, 327, 374, 300, 302, 322, 329, 331, 333, 334, 383, 384, 396, 417, 433, 434, 435, 326, 310, @@ -3043,28 +2685,28 @@ var yyAct = [...]int{ 496, 497, 561, 578, 545, 515, 478, 569, 512, 516, 517, 581, 0, 0, 0, 469, 358, 359, 0, 337, 285, 286, 640, 857, 389, 583, 616, 617, 508, 0, - 871, 852, 854, 855, 858, 862, 863, 864, 865, 866, - 868, 870, 874, 639, 0, 562, 577, 643, 576, 636, + 872, 852, 854, 855, 859, 863, 864, 865, 866, 867, + 869, 871, 875, 639, 0, 562, 577, 643, 576, 636, 395, 0, 416, 574, 521, 0, 566, 540, 0, 567, 536, 571, 0, 510, 0, 424, 450, 462, 479, 482, 511, 596, 597, 598, 290, 481, 600, 601, 602, 603, - 604, 605, 606, 599, 873, 543, 520, 546, 461, 523, + 604, 605, 606, 599, 874, 543, 520, 546, 461, 523, 522, 0, 0, 557, 804, 558, 559, 379, 380, 381, - 382, 860, 584, 308, 480, 405, 0, 544, 0, 0, + 382, 861, 584, 308, 480, 405, 0, 544, 0, 0, 0, 0, 0, 0, 0, 0, 549, 550, 547, 648, 0, 607, 608, 0, 0, 474, 475, 336, 343, 493, 345, 307, 394, 338, 459, 352, 0, 486, 551, 487, 610, 613, 611, 612, 386, 348, 349, 420, 353, 363, 408, 458, 392, 413, 305, 449, 422, 368, 537, 564, - 882, 856, 881, 883, 884, 880, 885, 886, 867, 759, - 0, 811, 878, 877, 879, 0, 0, 0, 0, 0, + 883, 856, 882, 884, 885, 881, 886, 887, 868, 759, + 0, 811, 879, 878, 880, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 592, 591, 590, 589, 588, 587, 586, 585, 0, 0, 534, 436, 317, 279, 313, 314, 321, 637, 634, 440, 638, 766, 287, 514, 361, 0, 404, 335, 579, 580, 0, 0, 845, 818, 819, 820, 756, 821, 815, 816, 757, 817, 846, 809, 842, 843, 785, 812, 822, 841, 823, 844, 847, 848, - 887, 888, 829, 813, 251, 889, 826, 849, 840, 839, + 888, 889, 829, 813, 251, 890, 826, 849, 840, 839, 824, 810, 850, 851, 792, 787, 827, 828, 814, 833, 834, 835, 758, 806, 807, 808, 830, 831, 788, 789, 790, 791, 0, 0, 0, 465, 466, 467, 489, 0, @@ -3072,94 +2714,31 @@ var yyAct = [...]int{ 563, 575, 609, 0, 619, 620, 622, 624, 836, 626, 427, 0, 832, 629, 630, 627, 365, 414, 432, 421, 802, 641, 504, 505, 642, 615, 0, 751, 0, 391, - 0, 519, 552, 541, 625, 507, 0, 0, 1704, 0, + 0, 519, 552, 541, 625, 507, 0, 0, 0, 0, 0, 0, 754, 0, 0, 0, 330, 0, 0, 360, 556, 538, 548, 539, 524, 525, 526, 533, 340, 527, 528, 529, 499, 530, 500, 531, 532, 793, 555, 506, - 423, 375, 573, 572, 0, 0, 861, 869, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 746, 0, 0, 783, 838, 837, 770, 780, 0, 0, - 303, 223, 501, 621, 503, 502, 771, 0, 772, 776, - 779, 775, 773, 774, 0, 853, 0, 0, 0, 0, - 0, 0, 0, 750, 0, 755, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 747, - 748, 0, 0, 0, 0, 803, 0, 749, 0, 0, - 798, 777, 781, 0, 0, 0, 0, 293, 429, 447, - 304, 418, 460, 309, 426, 299, 390, 415, 0, 0, - 295, 445, 425, 372, 350, 351, 294, 0, 409, 328, - 342, 325, 388, 778, 801, 805, 324, 875, 799, 455, - 297, 0, 454, 387, 441, 446, 373, 367, 0, 296, - 443, 371, 366, 354, 332, 876, 355, 356, 346, 399, - 364, 400, 347, 377, 376, 378, 0, 0, 0, 0, - 0, 483, 484, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 614, 796, 0, 618, - 0, 457, 0, 0, 859, 0, 0, 0, 428, 0, - 0, 357, 0, 0, 0, 800, 0, 412, 393, 872, - 0, 0, 410, 362, 442, 401, 448, 430, 456, 406, - 402, 288, 431, 327, 374, 300, 302, 322, 329, 331, - 333, 334, 383, 384, 396, 417, 433, 434, 435, 326, - 310, 411, 311, 344, 312, 289, 318, 316, 319, 419, - 320, 291, 397, 439, 0, 339, 407, 370, 292, 369, - 398, 438, 437, 301, 464, 1705, 1706, 560, 0, 476, - 645, 646, 647, 485, 0, 403, 490, 491, 492, 494, - 495, 496, 497, 561, 578, 545, 515, 478, 569, 512, - 516, 517, 581, 0, 0, 0, 469, 358, 359, 0, - 337, 285, 286, 640, 857, 389, 583, 616, 617, 508, - 0, 871, 852, 854, 855, 858, 862, 863, 864, 865, - 866, 868, 870, 874, 639, 0, 562, 577, 643, 576, - 636, 395, 0, 416, 574, 521, 0, 566, 540, 0, - 567, 536, 571, 0, 510, 0, 424, 450, 462, 479, - 482, 511, 596, 597, 598, 290, 481, 600, 601, 602, - 603, 604, 605, 606, 599, 873, 543, 520, 546, 461, - 523, 522, 0, 0, 557, 804, 558, 559, 379, 380, - 381, 382, 860, 584, 308, 480, 405, 0, 544, 0, - 0, 0, 0, 0, 0, 0, 0, 549, 550, 547, - 648, 0, 607, 608, 0, 0, 474, 475, 336, 343, - 493, 345, 307, 394, 338, 459, 352, 0, 486, 551, - 487, 610, 613, 611, 612, 386, 348, 349, 420, 353, - 363, 408, 458, 392, 413, 305, 449, 422, 368, 537, - 564, 882, 856, 881, 883, 884, 880, 885, 886, 867, - 759, 0, 811, 878, 877, 879, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 592, 591, 590, - 589, 588, 587, 586, 585, 0, 0, 534, 436, 317, - 279, 313, 314, 321, 637, 634, 440, 638, 766, 287, - 514, 361, 0, 404, 335, 579, 580, 0, 0, 845, - 818, 819, 820, 756, 821, 815, 816, 757, 817, 846, - 809, 842, 843, 785, 812, 822, 841, 823, 844, 847, - 848, 887, 888, 829, 813, 251, 889, 826, 849, 840, - 839, 824, 810, 850, 851, 792, 787, 827, 828, 814, - 833, 834, 835, 758, 806, 807, 808, 830, 831, 788, - 789, 790, 791, 0, 0, 0, 465, 466, 467, 489, - 0, 451, 513, 635, 0, 0, 0, 0, 0, 0, - 0, 563, 575, 609, 0, 619, 620, 622, 624, 836, - 626, 427, 0, 832, 629, 630, 627, 365, 414, 432, - 421, 802, 641, 504, 505, 642, 615, 0, 751, 0, - 391, 0, 519, 552, 541, 625, 507, 0, 0, 0, - 0, 0, 0, 754, 0, 0, 0, 330, 0, 0, - 360, 556, 538, 548, 539, 524, 525, 526, 533, 340, - 527, 528, 529, 499, 530, 500, 531, 532, 793, 555, - 506, 423, 375, 573, 572, 0, 0, 861, 869, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 423, 375, 573, 572, 0, 0, 862, 870, 0, 0, + 0, 0, 0, 0, 0, 0, 858, 0, 0, 0, 0, 746, 0, 0, 783, 838, 837, 770, 780, 0, 0, 303, 223, 501, 621, 503, 502, 771, 0, 772, 776, 779, 775, 773, 774, 0, 853, 0, 0, 0, - 0, 0, 0, 0, 750, 0, 755, 0, 0, 0, + 0, 0, 0, 738, 750, 0, 755, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 747, 748, 0, 0, 0, 0, 803, 0, 749, 0, + 747, 748, 1561, 0, 0, 0, 803, 0, 749, 0, 0, 798, 777, 781, 0, 0, 0, 0, 293, 429, 447, 304, 418, 460, 309, 426, 299, 390, 415, 0, 0, 295, 445, 425, 372, 350, 351, 294, 0, 409, - 328, 342, 325, 388, 778, 801, 805, 324, 875, 799, + 328, 342, 325, 388, 778, 801, 805, 324, 876, 799, 455, 297, 0, 454, 387, 441, 446, 373, 367, 0, - 296, 443, 371, 366, 354, 332, 876, 355, 356, 346, + 296, 443, 371, 366, 354, 332, 877, 355, 356, 346, 399, 364, 400, 347, 377, 376, 378, 0, 0, 0, 0, 0, 483, 484, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 614, 796, 0, - 618, 0, 457, 0, 0, 859, 0, 0, 0, 428, + 618, 0, 457, 0, 0, 860, 0, 0, 0, 428, 0, 0, 357, 0, 0, 0, 800, 0, 412, 393, - 872, 0, 0, 410, 362, 442, 401, 448, 430, 456, + 873, 0, 0, 410, 362, 442, 401, 448, 430, 456, 406, 402, 288, 431, 327, 374, 300, 302, 322, 329, 331, 333, 334, 383, 384, 396, 417, 433, 434, 435, 326, 310, 411, 311, 344, 312, 289, 318, 316, 319, @@ -3169,1996 +2748,552 @@ var yyAct = [...]int{ 494, 495, 496, 497, 561, 578, 545, 515, 478, 569, 512, 516, 517, 581, 0, 0, 0, 469, 358, 359, 0, 337, 285, 286, 640, 857, 389, 583, 616, 617, - 508, 0, 871, 852, 854, 855, 858, 862, 863, 864, - 865, 866, 868, 870, 874, 639, 0, 562, 577, 643, + 508, 0, 872, 852, 854, 855, 859, 863, 864, 865, + 866, 867, 869, 871, 875, 639, 0, 562, 577, 643, 576, 636, 395, 0, 416, 574, 521, 0, 566, 540, 0, 567, 536, 571, 0, 510, 0, 424, 450, 462, 479, 482, 511, 596, 597, 598, 290, 481, 600, 601, - 602, 603, 604, 605, 606, 599, 873, 543, 520, 546, + 602, 603, 604, 605, 606, 599, 874, 543, 520, 546, 461, 523, 522, 0, 0, 557, 804, 558, 559, 379, - 380, 381, 382, 860, 584, 308, 480, 405, 0, 544, + 380, 381, 382, 861, 584, 308, 480, 405, 0, 544, 0, 0, 0, 0, 0, 0, 0, 0, 549, 550, 547, 648, 0, 607, 608, 0, 0, 474, 475, 336, 343, 493, 345, 307, 394, 338, 459, 352, 0, 486, 551, 487, 610, 613, 611, 612, 386, 348, 349, 420, 353, 363, 408, 458, 392, 413, 305, 449, 422, 368, - 537, 564, 882, 856, 881, 883, 884, 880, 885, 886, - 867, 759, 0, 811, 878, 877, 879, 0, 0, 0, + 537, 564, 883, 856, 882, 884, 885, 881, 886, 887, + 868, 759, 0, 811, 879, 878, 880, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 592, 591, 590, 589, 588, 587, 586, 585, 0, 0, 534, 436, 317, 279, 313, 314, 321, 637, 634, 440, 638, 766, 287, 514, 361, 0, 404, 335, 579, 580, 0, 0, 845, 818, 819, 820, 756, 821, 815, 816, 757, 817, 846, 809, 842, 843, 785, 812, 822, 841, 823, 844, - 847, 848, 887, 888, 829, 813, 251, 889, 826, 849, + 847, 848, 888, 889, 829, 813, 251, 890, 826, 849, 840, 839, 824, 810, 850, 851, 792, 787, 827, 828, 814, 833, 834, 835, 758, 806, 807, 808, 830, 831, 788, 789, 790, 791, 0, 0, 0, 465, 466, 467, 489, 0, 451, 513, 635, 0, 0, 0, 0, 0, 0, 0, 563, 575, 609, 0, 619, 620, 622, 624, 836, 626, 427, 0, 832, 629, 630, 627, 365, 414, - 432, 421, 802, 641, 504, 505, 642, 615, 0, 751, - 0, 391, 0, 519, 552, 541, 625, 507, 0, 0, - 0, 0, 0, 0, 754, 0, 0, 0, 330, 0, - 0, 360, 556, 538, 548, 539, 524, 525, 526, 533, - 340, 527, 528, 529, 499, 530, 500, 531, 532, 793, - 555, 506, 423, 375, 573, 572, 0, 0, 861, 869, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 783, 838, 837, 770, 780, - 0, 0, 303, 223, 501, 621, 503, 502, 771, 0, - 772, 776, 779, 775, 773, 774, 0, 853, 0, 0, - 0, 0, 0, 0, 738, 750, 0, 755, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 747, 748, 0, 0, 0, 0, 803, 0, 749, - 0, 0, 798, 777, 781, 0, 0, 0, 0, 293, - 429, 447, 304, 418, 460, 309, 426, 299, 390, 415, - 0, 0, 295, 445, 425, 372, 350, 351, 294, 0, - 409, 328, 342, 325, 388, 778, 801, 805, 324, 875, - 799, 455, 297, 0, 454, 387, 441, 446, 373, 367, - 0, 296, 443, 371, 366, 354, 332, 876, 355, 356, - 346, 399, 364, 400, 347, 377, 376, 378, 0, 0, - 0, 0, 0, 483, 484, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 614, 796, - 0, 618, 0, 457, 0, 0, 859, 0, 0, 0, - 428, 0, 0, 357, 0, 0, 0, 800, 0, 412, - 393, 872, 0, 0, 410, 362, 442, 401, 448, 430, - 456, 406, 402, 288, 431, 327, 374, 300, 302, 322, - 329, 331, 333, 334, 383, 384, 396, 417, 433, 434, - 435, 326, 310, 411, 311, 344, 312, 289, 318, 316, - 319, 419, 320, 291, 397, 439, 0, 339, 407, 370, - 292, 369, 398, 438, 437, 301, 464, 470, 471, 560, - 0, 476, 645, 646, 647, 485, 0, 403, 490, 491, - 492, 494, 495, 496, 497, 561, 578, 545, 515, 478, - 569, 512, 516, 517, 581, 0, 0, 0, 469, 358, - 359, 0, 337, 285, 286, 640, 857, 389, 583, 616, - 617, 508, 0, 871, 852, 854, 855, 858, 862, 863, - 864, 865, 866, 868, 870, 874, 639, 0, 562, 577, - 643, 576, 636, 395, 0, 416, 574, 521, 0, 566, - 540, 0, 567, 536, 571, 0, 510, 0, 424, 450, - 462, 479, 482, 511, 596, 597, 598, 290, 481, 600, - 601, 602, 603, 604, 605, 606, 599, 873, 543, 520, - 546, 461, 523, 522, 0, 0, 557, 804, 558, 559, - 379, 380, 381, 382, 860, 584, 308, 480, 405, 0, - 544, 0, 0, 0, 0, 0, 0, 0, 0, 549, - 550, 547, 648, 0, 607, 608, 0, 0, 474, 475, - 336, 343, 493, 345, 307, 394, 338, 459, 352, 0, - 486, 551, 487, 610, 613, 611, 612, 386, 348, 349, - 420, 353, 363, 408, 458, 392, 413, 305, 449, 422, - 368, 537, 564, 882, 856, 881, 883, 884, 880, 885, - 886, 867, 759, 0, 811, 878, 877, 879, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 592, - 591, 590, 589, 588, 587, 586, 585, 0, 0, 534, - 436, 317, 279, 313, 314, 321, 637, 634, 440, 638, - 766, 287, 514, 361, 0, 404, 335, 579, 580, 0, - 0, 845, 818, 819, 820, 756, 821, 815, 816, 757, - 817, 846, 809, 842, 843, 785, 812, 822, 841, 823, - 844, 847, 848, 887, 888, 829, 813, 251, 889, 826, - 849, 840, 839, 824, 810, 850, 851, 792, 787, 827, - 828, 814, 833, 834, 835, 758, 806, 807, 808, 830, - 831, 788, 789, 790, 791, 0, 0, 0, 465, 466, - 467, 489, 0, 451, 513, 635, 0, 0, 0, 0, - 0, 0, 0, 563, 575, 609, 0, 619, 620, 622, - 624, 836, 626, 427, 0, 832, 629, 630, 627, 365, - 414, 432, 421, 0, 641, 504, 505, 642, 615, 0, - 751, 200, 61, 191, 162, 0, 0, 0, 0, 0, - 0, 391, 0, 519, 552, 541, 625, 507, 0, 192, - 0, 0, 0, 0, 0, 0, 183, 0, 330, 0, - 193, 360, 556, 538, 548, 539, 524, 525, 526, 533, - 340, 527, 528, 529, 499, 530, 500, 531, 532, 136, - 555, 506, 423, 375, 573, 572, 0, 0, 0, 0, - 0, 0, 0, 0, 122, 0, 0, 0, 0, 0, - 0, 0, 196, 0, 0, 222, 0, 0, 0, 0, - 0, 0, 303, 223, 501, 621, 503, 502, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 306, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 214, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 293, - 429, 447, 304, 418, 460, 309, 426, 299, 390, 415, - 0, 0, 295, 445, 425, 372, 350, 351, 294, 0, - 409, 328, 342, 325, 388, 0, 444, 472, 324, 463, - 0, 455, 297, 0, 454, 387, 441, 446, 373, 367, - 0, 296, 443, 371, 366, 354, 332, 488, 355, 356, - 346, 399, 364, 400, 347, 377, 376, 378, 0, 0, - 0, 0, 0, 483, 484, 0, 0, 0, 0, 0, - 0, 161, 189, 198, 190, 120, 0, 0, 614, 0, - 0, 618, 0, 457, 0, 0, 215, 0, 0, 0, - 428, 0, 0, 357, 188, 182, 181, 473, 0, 412, - 393, 227, 0, 0, 410, 362, 442, 401, 448, 430, - 456, 406, 402, 288, 431, 327, 374, 300, 302, 322, - 329, 331, 333, 334, 383, 384, 396, 417, 433, 434, - 435, 326, 310, 411, 311, 344, 312, 289, 318, 316, - 319, 419, 320, 291, 397, 439, 0, 339, 407, 370, - 292, 369, 398, 438, 437, 301, 464, 470, 471, 560, - 0, 476, 593, 594, 595, 485, 0, 403, 490, 491, - 492, 494, 495, 496, 497, 561, 578, 545, 515, 478, - 569, 512, 516, 517, 581, 0, 0, 0, 469, 358, - 359, 0, 337, 285, 286, 452, 323, 389, 583, 616, - 617, 508, 0, 570, 509, 518, 315, 542, 554, 553, - 385, 468, 218, 565, 568, 498, 228, 0, 562, 577, - 535, 576, 229, 395, 0, 416, 574, 521, 0, 566, - 540, 0, 567, 536, 571, 0, 510, 0, 424, 450, - 462, 479, 482, 511, 596, 597, 598, 290, 481, 600, - 601, 602, 603, 604, 605, 606, 599, 453, 543, 520, - 546, 461, 523, 522, 0, 0, 557, 477, 558, 559, - 379, 380, 381, 382, 341, 584, 308, 480, 405, 134, - 544, 0, 0, 0, 0, 0, 0, 0, 0, 549, - 550, 547, 226, 0, 607, 608, 0, 0, 474, 475, - 336, 343, 493, 345, 307, 394, 338, 459, 352, 0, - 486, 551, 487, 610, 613, 611, 612, 386, 348, 349, - 420, 353, 363, 408, 458, 392, 413, 305, 449, 422, - 368, 537, 564, 0, 0, 0, 0, 0, 0, 0, - 0, 62, 0, 0, 274, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 592, - 591, 590, 589, 588, 587, 586, 585, 0, 0, 534, - 436, 317, 279, 313, 314, 321, 233, 298, 440, 234, - 0, 287, 514, 361, 163, 404, 335, 579, 580, 58, - 0, 235, 236, 237, 238, 239, 240, 241, 242, 280, - 243, 244, 245, 246, 247, 248, 249, 252, 253, 254, - 255, 256, 257, 258, 259, 582, 250, 251, 260, 261, - 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, - 272, 273, 0, 0, 0, 281, 282, 283, 284, 0, - 0, 275, 276, 277, 278, 0, 0, 0, 465, 466, - 467, 489, 0, 451, 513, 230, 45, 216, 219, 221, - 220, 0, 59, 563, 575, 609, 5, 619, 620, 622, - 624, 623, 626, 427, 0, 628, 629, 630, 627, 365, - 414, 432, 421, 139, 231, 504, 505, 232, 615, 200, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 391, - 0, 519, 552, 541, 625, 507, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 330, 0, 0, 360, - 556, 538, 548, 539, 524, 525, 526, 533, 340, 527, - 528, 529, 499, 530, 500, 531, 532, 136, 555, 506, - 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 196, 0, 0, 222, 0, 0, 0, 0, 0, 0, - 303, 223, 501, 621, 503, 502, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 306, 2383, 2386, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 293, 429, 447, - 304, 418, 460, 309, 426, 299, 390, 415, 0, 0, - 295, 445, 425, 372, 350, 351, 294, 0, 409, 328, - 342, 325, 388, 0, 444, 472, 324, 463, 0, 455, - 297, 0, 454, 387, 441, 446, 373, 367, 0, 296, - 443, 371, 366, 354, 332, 488, 355, 356, 346, 399, - 364, 400, 347, 377, 376, 378, 0, 0, 0, 0, - 0, 483, 484, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 614, 0, 0, 618, - 2387, 457, 0, 0, 0, 2382, 0, 2381, 428, 2379, - 2384, 357, 0, 0, 0, 473, 0, 412, 393, 644, - 0, 0, 410, 362, 442, 401, 448, 430, 456, 406, - 402, 288, 431, 327, 374, 300, 302, 322, 329, 331, - 333, 334, 383, 384, 396, 417, 433, 434, 435, 326, - 310, 411, 311, 344, 312, 289, 318, 316, 319, 419, - 320, 291, 397, 439, 2385, 339, 407, 370, 292, 369, - 398, 438, 437, 301, 464, 470, 471, 560, 0, 476, - 645, 646, 647, 485, 0, 403, 490, 491, 492, 494, - 495, 496, 497, 561, 578, 545, 515, 478, 569, 512, - 516, 517, 581, 0, 0, 0, 469, 358, 359, 0, - 337, 285, 286, 640, 323, 389, 583, 616, 617, 508, - 0, 570, 509, 518, 315, 542, 554, 553, 385, 468, - 0, 565, 568, 498, 639, 0, 562, 577, 643, 576, - 636, 395, 0, 416, 574, 521, 0, 566, 540, 0, - 567, 536, 571, 0, 510, 0, 424, 450, 462, 479, - 482, 511, 596, 597, 598, 290, 481, 600, 601, 602, - 603, 604, 605, 606, 599, 453, 543, 520, 546, 461, - 523, 522, 0, 0, 557, 477, 558, 559, 379, 380, - 381, 382, 341, 584, 308, 480, 405, 0, 544, 0, - 0, 0, 0, 0, 0, 0, 0, 549, 550, 547, - 648, 0, 607, 608, 0, 0, 474, 475, 336, 343, - 493, 345, 307, 394, 338, 459, 352, 0, 486, 551, - 487, 610, 613, 611, 612, 386, 348, 349, 420, 353, - 363, 408, 458, 392, 413, 305, 449, 422, 368, 537, - 564, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 592, 591, 590, - 589, 588, 587, 586, 585, 0, 0, 534, 436, 317, - 279, 313, 314, 321, 637, 634, 440, 638, 0, 287, - 514, 361, 163, 404, 335, 579, 580, 0, 0, 235, - 236, 237, 238, 239, 240, 241, 242, 280, 243, 244, - 245, 246, 247, 248, 249, 252, 253, 254, 255, 256, - 257, 258, 259, 582, 250, 251, 260, 261, 262, 263, - 264, 265, 266, 267, 268, 269, 270, 271, 272, 273, - 0, 0, 0, 281, 282, 283, 284, 0, 0, 275, - 276, 277, 278, 0, 0, 0, 465, 466, 467, 489, - 0, 451, 513, 635, 0, 0, 0, 0, 0, 0, - 0, 563, 575, 609, 0, 619, 620, 622, 624, 623, - 626, 427, 0, 628, 629, 630, 627, 365, 414, 432, - 421, 0, 641, 504, 505, 642, 615, 391, 0, 519, - 552, 541, 625, 507, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 330, 0, 0, 360, 556, 538, - 548, 539, 524, 525, 526, 533, 340, 527, 528, 529, - 499, 530, 500, 531, 532, 0, 555, 506, 423, 375, - 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1306, 0, - 0, 222, 0, 0, 770, 780, 0, 0, 303, 223, - 501, 621, 503, 502, 771, 0, 772, 776, 779, 775, - 773, 774, 0, 306, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 777, - 0, 0, 0, 0, 0, 293, 429, 447, 304, 418, - 460, 309, 426, 299, 390, 415, 0, 0, 295, 445, - 425, 372, 350, 351, 294, 0, 409, 328, 342, 325, - 388, 778, 444, 472, 324, 463, 0, 455, 297, 0, - 454, 387, 441, 446, 373, 367, 0, 296, 443, 371, - 366, 354, 332, 488, 355, 356, 346, 399, 364, 400, - 347, 377, 376, 378, 0, 0, 0, 0, 0, 483, - 484, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 614, 0, 0, 618, 0, 457, - 0, 0, 0, 0, 0, 0, 428, 0, 0, 357, - 0, 0, 0, 473, 0, 412, 393, 644, 0, 0, - 410, 362, 442, 401, 448, 430, 456, 406, 402, 288, - 431, 327, 374, 300, 302, 322, 329, 331, 333, 334, - 383, 384, 396, 417, 433, 434, 435, 326, 310, 411, - 311, 344, 312, 289, 318, 316, 319, 419, 320, 291, - 397, 439, 0, 339, 407, 370, 292, 369, 398, 438, - 437, 301, 464, 470, 471, 560, 0, 476, 645, 646, - 647, 485, 0, 403, 490, 491, 492, 494, 495, 496, - 497, 561, 578, 545, 515, 478, 569, 512, 516, 517, - 581, 0, 0, 0, 469, 358, 359, 0, 337, 285, - 286, 640, 323, 389, 583, 616, 617, 508, 0, 570, - 509, 518, 315, 542, 554, 553, 385, 468, 0, 565, - 568, 498, 639, 0, 562, 577, 643, 576, 636, 395, - 0, 416, 574, 521, 0, 566, 540, 0, 567, 536, - 571, 0, 510, 0, 424, 450, 462, 479, 482, 511, - 596, 597, 598, 290, 481, 600, 601, 602, 603, 604, - 605, 606, 599, 453, 543, 520, 546, 461, 523, 522, - 0, 0, 557, 477, 558, 559, 379, 380, 381, 382, - 341, 584, 308, 480, 405, 0, 544, 0, 0, 0, - 0, 0, 0, 0, 0, 549, 550, 547, 648, 0, - 607, 608, 0, 0, 474, 475, 336, 343, 493, 345, - 307, 394, 338, 459, 352, 0, 486, 551, 487, 610, - 613, 611, 612, 386, 348, 349, 420, 353, 363, 408, - 458, 392, 413, 305, 449, 422, 368, 537, 564, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 592, 591, 590, 589, 588, - 587, 586, 585, 0, 0, 534, 436, 317, 279, 313, - 314, 321, 637, 634, 440, 638, 0, 287, 514, 361, - 0, 404, 335, 579, 580, 0, 0, 235, 236, 237, - 238, 239, 240, 241, 242, 280, 243, 244, 245, 246, - 247, 248, 249, 252, 253, 254, 255, 256, 257, 258, - 259, 582, 250, 251, 260, 261, 262, 263, 264, 265, - 266, 267, 268, 269, 270, 271, 272, 273, 0, 0, - 0, 281, 282, 283, 284, 0, 0, 275, 276, 277, - 278, 0, 0, 0, 465, 466, 467, 489, 0, 451, - 513, 635, 0, 0, 0, 0, 0, 0, 0, 563, - 575, 609, 0, 619, 620, 622, 624, 623, 626, 427, - 0, 628, 629, 630, 627, 365, 414, 432, 421, 0, - 641, 504, 505, 642, 615, 200, 61, 191, 162, 0, - 0, 0, 0, 0, 0, 391, 667, 519, 552, 541, - 625, 507, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 330, 0, 0, 360, 556, 538, 548, 539, - 524, 525, 526, 533, 340, 527, 528, 529, 499, 530, - 500, 531, 532, 0, 555, 506, 423, 375, 573, 572, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 674, - 0, 0, 0, 0, 0, 0, 673, 0, 0, 222, - 0, 0, 0, 0, 0, 0, 303, 223, 501, 621, - 503, 502, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 306, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 293, 429, 447, 304, 418, 460, 309, - 426, 299, 390, 415, 0, 0, 295, 445, 425, 372, - 350, 351, 294, 0, 409, 328, 342, 325, 388, 0, - 444, 472, 324, 463, 0, 455, 297, 0, 454, 387, - 441, 446, 373, 367, 0, 296, 443, 371, 366, 354, - 332, 488, 355, 356, 346, 399, 364, 400, 347, 377, - 376, 378, 0, 0, 0, 0, 0, 483, 484, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 671, - 672, 0, 614, 0, 0, 618, 0, 457, 0, 0, - 0, 0, 0, 0, 428, 0, 0, 357, 0, 0, - 0, 473, 0, 412, 393, 644, 0, 0, 410, 362, - 442, 401, 448, 430, 456, 406, 402, 288, 431, 327, - 374, 300, 302, 322, 329, 331, 333, 334, 383, 384, - 396, 417, 433, 434, 435, 326, 310, 411, 311, 344, - 312, 289, 318, 316, 319, 419, 320, 291, 397, 439, - 0, 339, 407, 370, 292, 369, 398, 438, 437, 301, - 464, 470, 471, 560, 0, 476, 645, 646, 647, 485, - 0, 403, 490, 491, 492, 494, 495, 496, 497, 561, - 578, 545, 515, 478, 569, 512, 516, 517, 581, 0, - 0, 0, 469, 358, 359, 0, 337, 285, 286, 640, - 323, 389, 583, 616, 617, 508, 0, 570, 509, 518, - 315, 542, 554, 553, 385, 468, 0, 565, 568, 498, - 639, 0, 562, 577, 643, 576, 636, 395, 0, 416, - 574, 521, 0, 566, 540, 0, 567, 536, 571, 0, - 510, 0, 424, 450, 462, 479, 482, 511, 596, 597, - 598, 290, 481, 600, 601, 602, 603, 604, 605, 606, - 599, 453, 543, 520, 546, 461, 523, 522, 0, 0, - 557, 477, 558, 559, 379, 380, 381, 382, 668, 670, - 308, 480, 405, 682, 544, 0, 0, 0, 0, 0, - 0, 0, 0, 549, 550, 547, 648, 0, 607, 608, - 0, 0, 474, 475, 336, 343, 493, 345, 307, 394, - 338, 459, 352, 0, 486, 551, 487, 610, 613, 611, - 612, 386, 348, 349, 420, 353, 363, 408, 458, 392, - 413, 305, 449, 422, 368, 537, 564, 0, 0, 0, - 0, 0, 0, 0, 0, 62, 0, 0, 274, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 592, 591, 590, 589, 588, 587, 586, - 585, 0, 0, 534, 436, 317, 279, 313, 314, 321, - 637, 634, 440, 638, 0, 287, 514, 361, 163, 404, - 335, 579, 580, 0, 0, 235, 236, 237, 238, 239, - 240, 241, 242, 280, 243, 244, 245, 246, 247, 248, - 249, 252, 253, 254, 255, 256, 257, 258, 259, 582, - 250, 251, 260, 261, 262, 263, 264, 265, 266, 267, - 268, 269, 270, 271, 272, 273, 0, 0, 0, 281, - 282, 283, 284, 0, 0, 275, 276, 277, 278, 0, - 0, 0, 465, 466, 467, 489, 0, 451, 513, 635, - 0, 0, 0, 0, 0, 0, 0, 563, 575, 609, - 0, 619, 620, 622, 624, 623, 626, 427, 0, 628, - 629, 630, 627, 365, 414, 432, 421, 0, 641, 504, - 505, 642, 615, 391, 0, 519, 552, 541, 625, 507, - 0, 1114, 0, 0, 0, 0, 0, 0, 0, 0, - 330, 0, 0, 360, 556, 538, 548, 539, 524, 525, - 526, 533, 340, 527, 528, 529, 499, 530, 500, 531, - 532, 0, 555, 506, 423, 375, 573, 572, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 222, 0, 0, - 0, 0, 0, 0, 303, 223, 501, 621, 503, 502, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 306, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1099, 0, 0, 0, 0, 0, - 0, 293, 429, 447, 304, 418, 460, 309, 426, 299, - 390, 415, 0, 0, 2544, 2547, 2548, 2549, 2550, 2551, - 2552, 0, 2557, 2553, 2554, 2555, 2556, 0, 2539, 2540, - 2541, 2542, 1097, 2523, 2545, 0, 2524, 387, 2525, 2526, - 2527, 2528, 1101, 2529, 2530, 2531, 2532, 2533, 2536, 2537, - 2534, 2535, 2543, 399, 364, 400, 347, 377, 376, 378, - 1125, 1127, 1129, 1131, 1134, 483, 484, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 614, 0, 0, 618, 0, 457, 0, 0, 0, 0, - 0, 0, 428, 0, 0, 357, 0, 0, 0, 2538, - 0, 412, 393, 644, 0, 0, 410, 362, 442, 401, - 448, 430, 456, 406, 402, 288, 431, 327, 374, 300, - 302, 322, 329, 331, 333, 334, 383, 384, 396, 417, - 433, 434, 435, 326, 310, 411, 311, 344, 312, 289, - 318, 316, 319, 419, 320, 291, 397, 439, 0, 339, - 407, 370, 292, 369, 398, 438, 437, 301, 464, 470, - 471, 560, 0, 476, 645, 646, 647, 485, 0, 403, - 490, 491, 492, 494, 495, 496, 497, 561, 578, 545, - 515, 478, 569, 512, 516, 517, 581, 0, 0, 0, - 469, 358, 359, 0, 337, 285, 286, 640, 323, 389, - 583, 616, 617, 508, 0, 570, 509, 518, 315, 542, - 554, 553, 385, 468, 0, 565, 568, 498, 639, 0, - 562, 577, 643, 576, 636, 395, 0, 416, 574, 521, - 0, 566, 540, 0, 567, 536, 571, 0, 510, 0, - 424, 450, 462, 479, 482, 511, 596, 597, 598, 290, - 481, 600, 601, 602, 603, 604, 605, 606, 599, 453, - 543, 520, 546, 461, 523, 522, 0, 0, 557, 477, - 558, 559, 379, 380, 381, 382, 341, 584, 308, 480, - 405, 0, 544, 0, 0, 0, 0, 0, 0, 0, - 0, 549, 550, 547, 648, 0, 607, 608, 0, 0, - 474, 475, 336, 343, 493, 345, 307, 394, 338, 459, - 352, 0, 486, 551, 487, 610, 613, 611, 612, 386, - 348, 349, 420, 353, 363, 408, 458, 392, 413, 305, - 449, 422, 368, 537, 564, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 592, 591, 590, 589, 588, 587, 586, 585, 0, - 0, 534, 436, 317, 279, 313, 314, 321, 637, 634, - 440, 638, 0, 287, 2546, 361, 0, 404, 335, 579, - 580, 0, 0, 235, 236, 237, 238, 239, 240, 241, - 242, 280, 243, 244, 245, 246, 247, 248, 249, 252, - 253, 254, 255, 256, 257, 258, 259, 582, 250, 251, - 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, - 270, 271, 272, 273, 0, 0, 0, 281, 282, 283, - 284, 0, 0, 275, 276, 277, 278, 0, 0, 0, - 465, 466, 467, 489, 0, 451, 513, 635, 0, 0, - 0, 0, 0, 0, 0, 563, 575, 609, 0, 619, - 620, 622, 624, 623, 626, 427, 0, 628, 629, 630, - 627, 365, 414, 432, 421, 0, 641, 504, 505, 642, - 615, 391, 0, 519, 552, 541, 625, 507, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 330, 0, - 0, 360, 556, 538, 548, 539, 524, 525, 526, 533, - 340, 527, 528, 529, 499, 530, 500, 531, 532, 0, - 555, 506, 423, 375, 573, 572, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 222, 0, 0, 0, 0, - 0, 0, 303, 223, 501, 621, 503, 502, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 306, 2383, 2386, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 293, - 429, 447, 304, 418, 460, 309, 426, 299, 390, 415, - 0, 0, 295, 445, 425, 372, 350, 351, 294, 0, - 409, 328, 342, 325, 388, 0, 444, 472, 324, 463, - 0, 455, 297, 0, 454, 387, 441, 446, 373, 367, - 0, 296, 443, 371, 366, 354, 332, 488, 355, 356, - 346, 399, 364, 400, 347, 377, 376, 378, 0, 0, - 0, 0, 0, 483, 484, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 614, 0, - 0, 618, 2387, 457, 0, 0, 0, 2382, 0, 2381, - 428, 2379, 2384, 357, 0, 0, 0, 473, 0, 412, - 393, 644, 0, 0, 410, 362, 442, 401, 448, 430, - 456, 406, 402, 288, 431, 327, 374, 300, 302, 322, - 329, 331, 333, 334, 383, 384, 396, 417, 433, 434, - 435, 326, 310, 411, 311, 344, 312, 289, 318, 316, - 319, 419, 320, 291, 397, 439, 2385, 339, 407, 370, - 292, 369, 398, 438, 437, 301, 464, 470, 471, 560, - 0, 476, 645, 646, 647, 485, 0, 403, 490, 491, - 492, 494, 495, 496, 497, 561, 578, 545, 515, 478, - 569, 512, 516, 517, 581, 0, 0, 0, 469, 358, - 359, 0, 337, 285, 286, 640, 323, 389, 583, 616, - 617, 508, 0, 570, 509, 518, 315, 542, 554, 553, - 385, 468, 0, 565, 568, 498, 639, 0, 562, 577, - 643, 576, 636, 395, 0, 416, 574, 521, 0, 566, - 540, 0, 567, 536, 571, 0, 510, 0, 424, 450, - 462, 479, 482, 511, 596, 597, 598, 290, 481, 600, - 601, 602, 603, 604, 605, 606, 599, 453, 543, 520, - 546, 461, 523, 522, 0, 0, 557, 477, 558, 559, - 379, 380, 381, 382, 341, 584, 308, 480, 405, 0, - 544, 0, 0, 0, 0, 0, 0, 0, 0, 549, - 550, 547, 648, 0, 607, 608, 0, 0, 474, 475, - 336, 343, 493, 345, 307, 394, 338, 459, 352, 0, - 486, 551, 487, 610, 613, 611, 612, 386, 348, 349, - 420, 353, 363, 408, 458, 392, 413, 305, 449, 422, - 368, 537, 564, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 592, - 591, 590, 589, 588, 587, 586, 585, 0, 0, 534, - 436, 317, 279, 313, 314, 321, 637, 634, 440, 638, - 0, 287, 514, 361, 0, 404, 335, 579, 580, 0, - 0, 235, 236, 237, 238, 239, 240, 241, 242, 280, - 243, 244, 245, 246, 247, 248, 249, 252, 253, 254, - 255, 256, 257, 258, 259, 582, 250, 251, 260, 261, - 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, - 272, 273, 0, 0, 0, 281, 282, 283, 284, 0, - 0, 275, 276, 277, 278, 0, 0, 0, 465, 466, - 467, 489, 0, 451, 513, 635, 0, 0, 0, 0, - 0, 0, 0, 563, 575, 609, 0, 619, 620, 622, - 624, 623, 626, 427, 0, 628, 629, 630, 627, 365, - 414, 432, 421, 0, 641, 504, 505, 642, 615, 391, - 0, 519, 552, 541, 625, 507, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 330, 0, 0, 360, - 556, 538, 548, 539, 524, 525, 526, 533, 340, 527, - 528, 529, 499, 530, 500, 531, 532, 0, 555, 506, - 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 222, 0, 0, 0, 0, 0, 0, - 303, 223, 501, 621, 503, 502, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 306, 0, 2404, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 293, 429, 447, - 304, 418, 460, 309, 426, 299, 390, 415, 0, 0, - 295, 445, 425, 372, 350, 351, 294, 0, 409, 328, - 342, 325, 388, 0, 444, 472, 324, 463, 0, 455, - 297, 0, 454, 387, 441, 446, 373, 367, 0, 296, - 443, 371, 366, 354, 332, 488, 355, 356, 346, 399, - 364, 400, 347, 377, 376, 378, 0, 0, 0, 0, - 0, 483, 484, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 614, 0, 0, 618, - 2403, 457, 0, 0, 0, 2409, 2406, 2408, 428, 0, - 2407, 357, 0, 0, 0, 473, 0, 412, 393, 644, - 0, 2401, 410, 362, 442, 401, 448, 430, 456, 406, - 402, 288, 431, 327, 374, 300, 302, 322, 329, 331, - 333, 334, 383, 384, 396, 417, 433, 434, 435, 326, - 310, 411, 311, 344, 312, 289, 318, 316, 319, 419, - 320, 291, 397, 439, 0, 339, 407, 370, 292, 369, - 398, 438, 437, 301, 464, 470, 471, 560, 0, 476, - 645, 646, 647, 485, 0, 403, 490, 491, 492, 494, - 495, 496, 497, 561, 578, 545, 515, 478, 569, 512, - 516, 517, 581, 0, 0, 0, 469, 358, 359, 0, - 337, 285, 286, 640, 323, 389, 583, 616, 617, 508, - 0, 570, 509, 518, 315, 542, 554, 553, 385, 468, - 0, 565, 568, 498, 639, 0, 562, 577, 643, 576, - 636, 395, 0, 416, 574, 521, 0, 566, 540, 0, - 567, 536, 571, 0, 510, 0, 424, 450, 462, 479, - 482, 511, 596, 597, 598, 290, 481, 600, 601, 602, - 603, 604, 605, 606, 599, 453, 543, 520, 546, 461, - 523, 522, 0, 0, 557, 477, 558, 559, 379, 380, - 381, 382, 341, 584, 308, 480, 405, 0, 544, 0, - 0, 0, 0, 0, 0, 0, 0, 549, 550, 547, - 648, 0, 607, 608, 0, 0, 474, 475, 336, 343, - 493, 345, 307, 394, 338, 459, 352, 0, 486, 551, - 487, 610, 613, 611, 612, 386, 348, 349, 420, 353, - 363, 408, 458, 392, 413, 305, 449, 422, 368, 537, - 564, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 592, 591, 590, - 589, 588, 587, 586, 585, 0, 0, 534, 436, 317, - 279, 313, 314, 321, 637, 634, 440, 638, 0, 287, - 514, 361, 0, 404, 335, 579, 580, 0, 0, 235, - 236, 237, 238, 239, 240, 241, 242, 280, 243, 244, - 245, 246, 247, 248, 249, 252, 253, 254, 255, 256, - 257, 258, 259, 582, 250, 251, 260, 261, 262, 263, - 264, 265, 266, 267, 268, 269, 270, 271, 272, 273, - 0, 0, 0, 281, 282, 283, 284, 0, 0, 275, - 276, 277, 278, 0, 0, 0, 465, 466, 467, 489, - 0, 451, 513, 635, 0, 0, 0, 0, 0, 0, - 0, 563, 575, 609, 0, 619, 620, 622, 624, 623, - 626, 427, 0, 628, 629, 630, 627, 365, 414, 432, - 421, 0, 641, 504, 505, 642, 615, 391, 0, 519, - 552, 541, 625, 507, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 330, 0, 0, 360, 556, 538, - 548, 539, 524, 525, 526, 533, 340, 527, 528, 529, - 499, 530, 500, 531, 532, 0, 555, 506, 423, 375, - 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 222, 0, 0, 0, 0, 0, 0, 303, 223, - 501, 621, 503, 502, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 306, 0, 2404, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 293, 429, 447, 304, 418, - 460, 309, 426, 299, 390, 415, 0, 0, 295, 445, - 425, 372, 350, 351, 294, 0, 409, 328, 342, 325, - 388, 0, 444, 472, 324, 463, 0, 455, 297, 0, - 454, 387, 441, 446, 373, 367, 0, 296, 443, 371, - 366, 354, 332, 488, 355, 356, 346, 399, 364, 400, - 347, 377, 376, 378, 0, 0, 0, 0, 0, 483, - 484, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 614, 0, 0, 618, 2403, 457, - 0, 0, 0, 2409, 2406, 2408, 428, 0, 2407, 357, - 0, 0, 0, 473, 0, 412, 393, 644, 0, 0, - 410, 362, 442, 401, 448, 430, 456, 406, 402, 288, - 431, 327, 374, 300, 302, 322, 329, 331, 333, 334, - 383, 384, 396, 417, 433, 434, 435, 326, 310, 411, - 311, 344, 312, 289, 318, 316, 319, 419, 320, 291, - 397, 439, 0, 339, 407, 370, 292, 369, 398, 438, - 437, 301, 464, 470, 471, 560, 0, 476, 645, 646, - 647, 485, 0, 403, 490, 491, 492, 494, 495, 496, - 497, 561, 578, 545, 515, 478, 569, 512, 516, 517, - 581, 0, 0, 0, 469, 358, 359, 0, 337, 285, - 286, 640, 323, 389, 583, 616, 617, 508, 0, 570, - 509, 518, 315, 542, 554, 553, 385, 468, 0, 565, - 568, 498, 639, 0, 562, 577, 643, 576, 636, 395, - 0, 416, 574, 521, 0, 566, 540, 0, 567, 536, - 571, 0, 510, 0, 424, 450, 462, 479, 482, 511, - 596, 597, 598, 290, 481, 600, 601, 602, 603, 604, - 605, 606, 599, 453, 543, 520, 546, 461, 523, 522, - 0, 0, 557, 477, 558, 559, 379, 380, 381, 382, - 341, 584, 308, 480, 405, 0, 544, 0, 0, 0, - 0, 0, 0, 0, 0, 549, 550, 547, 648, 0, - 607, 608, 0, 0, 474, 475, 336, 343, 493, 345, - 307, 394, 338, 459, 352, 0, 486, 551, 487, 610, - 613, 611, 612, 386, 348, 349, 420, 353, 363, 408, - 458, 392, 413, 305, 449, 422, 368, 537, 564, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 592, 591, 590, 589, 588, - 587, 586, 585, 0, 0, 534, 436, 317, 279, 313, - 314, 321, 637, 634, 440, 638, 0, 287, 514, 361, - 0, 404, 335, 579, 580, 0, 0, 235, 236, 237, - 238, 239, 240, 241, 242, 280, 243, 244, 245, 246, - 247, 248, 249, 252, 253, 254, 255, 256, 257, 258, - 259, 582, 250, 251, 260, 261, 262, 263, 264, 265, - 266, 267, 268, 269, 270, 271, 272, 273, 0, 0, - 0, 281, 282, 283, 284, 0, 0, 275, 276, 277, - 278, 0, 0, 0, 465, 466, 467, 489, 0, 451, - 513, 635, 0, 0, 0, 0, 0, 0, 0, 563, - 575, 609, 0, 619, 620, 622, 624, 623, 626, 427, - 0, 628, 629, 630, 627, 365, 414, 432, 421, 0, - 641, 504, 505, 642, 615, 391, 0, 519, 552, 541, - 625, 507, 0, 0, 0, 0, 0, 2099, 0, 0, - 0, 0, 330, 0, 0, 360, 556, 538, 548, 539, - 524, 525, 526, 533, 340, 527, 528, 529, 499, 530, - 500, 531, 532, 0, 555, 506, 423, 375, 573, 572, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 222, - 0, 0, 2100, 0, 0, 0, 303, 223, 501, 621, - 503, 502, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 306, 0, 0, 1235, 1236, 1237, 1234, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 293, 429, 447, 304, 418, 460, 309, - 426, 299, 390, 415, 0, 0, 295, 445, 425, 372, - 350, 351, 294, 0, 409, 328, 342, 325, 388, 0, - 444, 472, 324, 463, 0, 455, 297, 0, 454, 387, - 441, 446, 373, 367, 0, 296, 443, 371, 366, 354, - 332, 488, 355, 356, 346, 399, 364, 400, 347, 377, - 376, 378, 0, 0, 0, 0, 0, 483, 484, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 614, 0, 0, 618, 0, 457, 0, 0, - 0, 0, 0, 0, 428, 0, 0, 357, 0, 0, - 0, 473, 0, 412, 393, 644, 0, 0, 410, 362, - 442, 401, 448, 430, 456, 406, 402, 288, 431, 327, - 374, 300, 302, 322, 329, 331, 333, 334, 383, 384, - 396, 417, 433, 434, 435, 326, 310, 411, 311, 344, - 312, 289, 318, 316, 319, 419, 320, 291, 397, 439, - 0, 339, 407, 370, 292, 369, 398, 438, 437, 301, - 464, 470, 471, 560, 0, 476, 645, 646, 647, 485, - 0, 403, 490, 491, 492, 494, 495, 496, 497, 561, - 578, 545, 515, 478, 569, 512, 516, 517, 581, 0, - 0, 0, 469, 358, 359, 0, 337, 285, 286, 640, - 323, 389, 583, 616, 617, 508, 0, 570, 509, 518, - 315, 542, 554, 553, 385, 468, 0, 565, 568, 498, - 639, 0, 562, 577, 643, 576, 636, 395, 0, 416, - 574, 521, 0, 566, 540, 0, 567, 536, 571, 0, - 510, 0, 424, 450, 462, 479, 482, 511, 596, 597, - 598, 290, 481, 600, 601, 602, 603, 604, 605, 606, - 599, 453, 543, 520, 546, 461, 523, 522, 0, 0, - 557, 477, 558, 559, 379, 380, 381, 382, 341, 584, - 308, 480, 405, 0, 544, 0, 0, 0, 0, 0, - 0, 0, 0, 549, 550, 547, 648, 0, 607, 608, - 0, 0, 474, 475, 336, 343, 493, 345, 307, 394, - 338, 459, 352, 0, 486, 551, 487, 610, 613, 611, - 612, 386, 348, 349, 420, 353, 363, 408, 458, 392, - 413, 305, 449, 422, 368, 537, 564, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 274, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 592, 591, 590, 589, 588, 587, 586, - 585, 0, 0, 534, 436, 317, 279, 313, 314, 321, - 637, 634, 440, 638, 0, 287, 514, 361, 0, 404, - 335, 579, 580, 0, 0, 235, 236, 237, 238, 239, - 240, 241, 242, 280, 243, 244, 245, 246, 247, 248, - 249, 252, 253, 254, 255, 256, 257, 258, 259, 582, - 250, 251, 260, 261, 262, 263, 264, 265, 266, 267, - 268, 269, 270, 271, 272, 273, 0, 0, 0, 281, - 282, 283, 284, 0, 0, 275, 276, 277, 278, 0, - 0, 0, 465, 466, 467, 489, 0, 451, 513, 635, - 0, 0, 0, 0, 0, 0, 0, 563, 575, 609, - 0, 619, 620, 622, 624, 623, 626, 427, 0, 628, - 629, 630, 627, 365, 414, 432, 421, 200, 641, 504, - 505, 642, 615, 0, 0, 0, 0, 391, 0, 519, - 552, 541, 625, 507, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 330, 0, 0, 360, 556, 538, - 548, 539, 524, 525, 526, 533, 340, 527, 528, 529, - 499, 530, 500, 531, 532, 136, 555, 506, 423, 375, - 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 196, 2149, - 0, 222, 0, 0, 0, 0, 0, 0, 303, 223, - 501, 621, 503, 502, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 306, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 293, 429, 447, 304, 418, - 460, 309, 426, 299, 390, 415, 0, 0, 295, 445, - 425, 372, 350, 351, 294, 0, 409, 328, 342, 325, - 388, 0, 444, 472, 324, 463, 0, 455, 297, 0, - 454, 387, 441, 446, 373, 367, 0, 296, 443, 371, - 366, 354, 332, 488, 355, 356, 346, 399, 364, 400, - 347, 377, 376, 378, 0, 0, 0, 0, 0, 483, - 484, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 614, 0, 0, 618, 0, 457, - 0, 0, 0, 0, 0, 0, 428, 0, 0, 357, - 0, 0, 0, 473, 0, 412, 393, 644, 0, 0, - 410, 362, 442, 401, 448, 430, 456, 406, 402, 288, - 431, 327, 374, 300, 302, 322, 329, 331, 333, 334, - 383, 384, 396, 417, 433, 434, 435, 326, 310, 411, - 311, 344, 312, 289, 318, 316, 319, 419, 320, 291, - 397, 439, 0, 339, 407, 370, 292, 369, 398, 438, - 437, 301, 464, 470, 471, 560, 0, 476, 645, 646, - 647, 485, 0, 403, 490, 491, 492, 494, 495, 496, - 497, 561, 578, 545, 515, 478, 569, 512, 516, 517, - 581, 0, 0, 0, 469, 358, 359, 0, 337, 285, - 286, 640, 323, 389, 583, 616, 617, 508, 0, 570, - 509, 518, 315, 542, 554, 553, 385, 468, 0, 565, - 568, 498, 639, 0, 562, 577, 643, 576, 636, 395, - 0, 416, 574, 521, 0, 566, 540, 0, 567, 536, - 571, 0, 510, 0, 424, 450, 462, 479, 482, 511, - 596, 597, 598, 290, 481, 600, 601, 602, 603, 604, - 605, 606, 599, 453, 543, 520, 546, 461, 523, 522, - 0, 0, 557, 477, 558, 559, 379, 380, 381, 382, - 341, 584, 308, 480, 405, 0, 544, 0, 0, 0, - 0, 0, 0, 0, 0, 549, 550, 547, 648, 0, - 607, 608, 0, 0, 474, 475, 336, 343, 493, 345, - 307, 394, 338, 459, 352, 0, 486, 551, 487, 610, - 613, 611, 612, 386, 348, 349, 420, 353, 363, 408, - 458, 392, 413, 305, 449, 422, 368, 537, 564, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 592, 591, 590, 589, 588, - 587, 586, 585, 0, 0, 534, 436, 317, 279, 313, - 314, 321, 637, 634, 440, 638, 0, 287, 514, 361, - 163, 404, 335, 579, 580, 0, 0, 235, 236, 237, - 238, 239, 240, 241, 242, 280, 243, 244, 245, 246, - 247, 248, 249, 252, 253, 254, 255, 256, 257, 258, - 259, 582, 250, 251, 260, 261, 262, 263, 264, 265, - 266, 267, 268, 269, 270, 271, 272, 273, 0, 0, - 0, 281, 282, 283, 284, 0, 0, 275, 276, 277, - 278, 0, 0, 0, 465, 466, 467, 489, 0, 451, - 513, 635, 0, 0, 0, 0, 0, 0, 0, 563, - 575, 609, 0, 619, 620, 622, 624, 623, 626, 427, - 0, 628, 629, 630, 627, 365, 414, 432, 421, 200, - 641, 504, 505, 642, 615, 0, 0, 0, 0, 391, - 0, 519, 552, 541, 625, 507, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 330, 0, 0, 360, - 556, 538, 548, 539, 524, 525, 526, 533, 340, 527, - 528, 529, 499, 530, 500, 531, 532, 136, 555, 506, - 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 196, 2135, 0, 222, 0, 0, 0, 0, 0, 0, - 303, 223, 501, 621, 503, 502, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 306, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 293, 429, 447, - 304, 418, 460, 309, 426, 299, 390, 415, 0, 0, - 295, 445, 425, 372, 350, 351, 294, 0, 409, 328, - 342, 325, 388, 0, 444, 472, 324, 463, 0, 455, - 297, 0, 454, 387, 441, 446, 373, 367, 0, 296, - 443, 371, 366, 354, 332, 488, 355, 356, 346, 399, - 364, 400, 347, 377, 376, 378, 0, 0, 0, 0, - 0, 483, 484, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 614, 0, 0, 618, - 0, 457, 0, 0, 0, 0, 0, 0, 428, 0, - 0, 357, 0, 0, 0, 473, 0, 412, 393, 644, - 0, 0, 410, 362, 442, 401, 448, 430, 456, 406, - 402, 288, 431, 327, 374, 300, 302, 322, 329, 331, - 333, 334, 383, 384, 396, 417, 433, 434, 435, 326, - 310, 411, 311, 344, 312, 289, 318, 316, 319, 419, - 320, 291, 397, 439, 0, 339, 407, 370, 292, 369, - 398, 438, 437, 301, 464, 470, 471, 560, 0, 476, - 645, 646, 647, 485, 0, 403, 490, 491, 492, 494, - 495, 496, 497, 561, 578, 545, 515, 478, 569, 512, - 516, 517, 581, 0, 0, 0, 469, 358, 359, 0, - 337, 285, 286, 640, 323, 389, 583, 616, 617, 508, - 0, 570, 509, 518, 315, 542, 554, 553, 385, 468, - 0, 565, 568, 498, 639, 0, 562, 577, 643, 576, - 636, 395, 0, 416, 574, 521, 0, 566, 540, 0, - 567, 536, 571, 0, 510, 0, 424, 450, 462, 479, - 482, 511, 596, 597, 598, 290, 481, 600, 601, 602, - 603, 604, 605, 606, 599, 453, 543, 520, 546, 461, - 523, 522, 0, 0, 557, 477, 558, 559, 379, 380, - 381, 382, 341, 584, 308, 480, 405, 0, 544, 0, - 0, 0, 0, 0, 0, 0, 0, 549, 550, 547, - 648, 0, 607, 608, 0, 0, 474, 475, 336, 343, - 493, 345, 307, 394, 338, 459, 352, 0, 486, 551, - 487, 610, 613, 611, 612, 386, 348, 349, 420, 353, - 363, 408, 458, 392, 413, 305, 449, 422, 368, 537, - 564, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 592, 591, 590, - 589, 588, 587, 586, 585, 0, 0, 534, 436, 317, - 279, 313, 314, 321, 637, 634, 440, 638, 0, 287, - 514, 361, 163, 404, 335, 579, 580, 0, 0, 235, - 236, 237, 238, 239, 240, 241, 242, 280, 243, 244, - 245, 246, 247, 248, 249, 252, 253, 254, 255, 256, - 257, 258, 259, 582, 250, 251, 260, 261, 262, 263, - 264, 265, 266, 267, 268, 269, 270, 271, 272, 273, - 0, 0, 0, 281, 282, 283, 284, 0, 0, 275, - 276, 277, 278, 0, 0, 0, 465, 466, 467, 489, - 0, 451, 513, 635, 0, 0, 0, 0, 0, 0, - 0, 563, 575, 609, 0, 619, 620, 622, 624, 623, - 626, 427, 0, 628, 629, 630, 627, 365, 414, 432, - 421, 0, 641, 504, 505, 642, 615, 391, 0, 519, - 552, 541, 625, 507, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 330, 1029, 0, 360, 556, 538, - 548, 539, 524, 525, 526, 533, 340, 527, 528, 529, - 499, 530, 500, 531, 532, 0, 555, 506, 423, 375, - 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 222, 1036, 1037, 0, 0, 0, 0, 303, 223, - 501, 621, 503, 502, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1040, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 293, 429, 1023, 304, 418, - 460, 309, 426, 299, 390, 415, 0, 0, 295, 445, - 425, 372, 350, 351, 294, 0, 409, 328, 342, 325, - 388, 0, 444, 472, 324, 463, 1010, 455, 297, 1009, - 454, 387, 441, 446, 373, 367, 0, 296, 443, 371, - 366, 354, 332, 488, 355, 356, 346, 399, 364, 400, - 347, 377, 376, 378, 0, 0, 0, 0, 0, 483, - 484, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 614, 0, 0, 618, 0, 457, - 0, 0, 0, 0, 0, 0, 428, 0, 0, 357, - 0, 0, 0, 473, 0, 412, 393, 644, 0, 0, - 410, 362, 442, 401, 448, 430, 456, 1027, 402, 288, - 431, 327, 374, 300, 302, 322, 329, 331, 333, 334, - 383, 384, 396, 417, 433, 434, 435, 326, 310, 411, - 311, 344, 312, 289, 318, 316, 319, 419, 320, 291, - 397, 439, 0, 339, 407, 370, 292, 369, 398, 438, - 437, 301, 464, 470, 471, 560, 0, 476, 645, 646, - 647, 485, 0, 403, 490, 491, 492, 494, 495, 496, - 497, 561, 578, 545, 515, 478, 569, 512, 516, 517, - 581, 0, 0, 0, 469, 358, 359, 0, 337, 285, - 286, 640, 323, 389, 583, 616, 617, 508, 0, 570, - 509, 518, 315, 542, 554, 553, 385, 468, 0, 565, - 568, 498, 639, 0, 562, 577, 643, 576, 636, 395, - 0, 416, 574, 521, 0, 566, 540, 0, 567, 536, - 571, 0, 510, 0, 424, 450, 462, 479, 482, 511, - 596, 597, 598, 290, 481, 600, 601, 602, 603, 604, - 605, 1028, 599, 453, 543, 520, 546, 461, 523, 522, - 0, 0, 557, 1031, 558, 559, 379, 380, 381, 382, - 341, 584, 1026, 480, 405, 0, 544, 0, 0, 0, - 0, 0, 0, 0, 0, 549, 550, 547, 648, 0, - 607, 608, 0, 0, 474, 475, 336, 343, 493, 345, - 307, 394, 338, 459, 352, 0, 486, 551, 487, 610, - 613, 611, 612, 1038, 1024, 1034, 1025, 353, 363, 408, - 458, 392, 413, 305, 449, 422, 1035, 537, 564, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 592, 591, 590, 589, 588, - 587, 586, 585, 0, 0, 534, 436, 317, 279, 313, - 314, 321, 637, 634, 440, 638, 0, 287, 514, 361, - 0, 404, 335, 579, 580, 0, 0, 235, 236, 237, - 238, 239, 240, 241, 242, 280, 243, 244, 245, 246, - 247, 248, 249, 252, 253, 254, 255, 256, 257, 258, - 259, 582, 250, 251, 260, 261, 262, 263, 264, 265, - 266, 267, 268, 269, 270, 271, 272, 273, 0, 0, - 0, 281, 282, 283, 284, 0, 0, 275, 276, 277, - 278, 0, 0, 0, 465, 466, 467, 489, 0, 451, - 513, 635, 0, 0, 0, 0, 0, 0, 0, 563, - 575, 609, 0, 619, 620, 622, 624, 623, 626, 427, - 0, 628, 629, 630, 627, 1022, 414, 432, 421, 200, - 641, 504, 505, 642, 615, 0, 0, 0, 0, 391, - 0, 519, 552, 541, 625, 507, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 330, 0, 0, 360, - 556, 538, 548, 539, 524, 525, 526, 533, 340, 527, - 528, 529, 499, 530, 500, 531, 532, 136, 555, 506, - 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 2031, 0, 0, 222, 0, 0, 0, 0, 0, 0, - 303, 223, 501, 621, 503, 502, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 306, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 293, 429, 447, - 304, 418, 460, 309, 426, 299, 390, 415, 0, 0, - 295, 445, 425, 372, 350, 351, 294, 0, 409, 328, - 342, 325, 388, 0, 444, 472, 324, 463, 0, 455, - 297, 0, 454, 387, 441, 446, 373, 367, 0, 296, - 443, 371, 366, 354, 332, 488, 355, 356, 346, 399, - 364, 400, 347, 377, 376, 378, 0, 0, 0, 0, - 0, 483, 484, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 614, 0, 0, 618, - 0, 457, 0, 0, 0, 0, 0, 0, 428, 0, - 0, 357, 0, 0, 0, 473, 0, 412, 393, 644, - 0, 0, 410, 362, 442, 401, 448, 430, 456, 406, - 402, 288, 431, 327, 374, 300, 302, 322, 329, 331, - 333, 334, 383, 384, 396, 417, 433, 434, 435, 326, - 310, 411, 311, 344, 312, 289, 318, 316, 319, 419, - 320, 291, 397, 439, 0, 339, 407, 370, 292, 369, - 398, 438, 437, 301, 464, 470, 471, 560, 0, 476, - 645, 646, 647, 485, 0, 403, 490, 491, 492, 494, - 495, 496, 497, 561, 578, 545, 515, 478, 569, 512, - 516, 517, 581, 0, 0, 0, 469, 358, 359, 0, - 337, 285, 286, 640, 323, 389, 583, 616, 617, 508, - 0, 570, 509, 518, 315, 542, 554, 553, 385, 468, - 0, 565, 568, 498, 639, 0, 562, 577, 643, 576, - 636, 395, 0, 416, 574, 521, 0, 566, 540, 0, - 567, 536, 571, 0, 510, 0, 424, 450, 462, 479, - 482, 511, 596, 597, 598, 290, 481, 600, 601, 602, - 603, 604, 605, 606, 599, 453, 543, 520, 546, 461, - 523, 522, 0, 0, 557, 477, 558, 559, 379, 380, - 381, 382, 341, 584, 308, 480, 405, 0, 544, 0, - 0, 0, 0, 0, 0, 0, 0, 549, 550, 547, - 648, 0, 607, 608, 0, 0, 474, 475, 336, 343, - 493, 345, 307, 394, 338, 459, 352, 0, 486, 551, - 487, 610, 613, 611, 612, 386, 348, 349, 420, 353, - 363, 408, 458, 392, 413, 305, 449, 422, 368, 537, - 564, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 592, 591, 590, - 589, 588, 587, 586, 585, 0, 0, 534, 436, 317, - 279, 313, 314, 321, 637, 634, 440, 638, 0, 287, - 514, 361, 163, 404, 335, 579, 580, 0, 0, 235, - 236, 237, 238, 239, 240, 241, 242, 280, 243, 244, - 245, 246, 247, 248, 249, 252, 253, 254, 255, 256, - 257, 258, 259, 582, 250, 251, 260, 261, 262, 263, - 264, 265, 266, 267, 268, 269, 270, 271, 272, 273, - 0, 0, 0, 281, 282, 283, 284, 0, 0, 275, - 276, 277, 278, 0, 0, 0, 465, 466, 467, 489, - 0, 451, 513, 635, 0, 0, 0, 0, 0, 0, - 0, 563, 575, 609, 0, 619, 620, 622, 624, 623, - 626, 427, 0, 628, 629, 630, 627, 365, 414, 432, - 421, 0, 641, 504, 505, 642, 615, 391, 0, 519, - 552, 541, 625, 507, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 330, 0, 0, 360, 556, 538, - 548, 539, 524, 525, 526, 533, 340, 527, 528, 529, - 499, 530, 500, 531, 532, 0, 555, 506, 423, 375, - 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 222, 1036, 1037, 0, 0, 0, 0, 303, 223, - 501, 621, 503, 502, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1040, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 293, 429, 447, 304, 418, - 460, 309, 426, 299, 390, 415, 0, 0, 295, 445, - 425, 372, 350, 351, 294, 0, 409, 328, 342, 325, - 388, 0, 444, 472, 324, 463, 1010, 455, 297, 1009, - 454, 387, 441, 446, 373, 367, 0, 296, 443, 371, - 366, 354, 332, 488, 355, 356, 346, 399, 364, 400, - 347, 377, 376, 378, 0, 0, 0, 0, 0, 483, - 484, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 614, 0, 0, 618, 0, 457, - 0, 0, 0, 0, 0, 0, 428, 0, 0, 357, - 0, 0, 0, 473, 0, 412, 393, 644, 0, 0, - 410, 362, 442, 401, 448, 430, 456, 406, 402, 288, - 431, 327, 374, 300, 302, 322, 329, 331, 333, 334, - 383, 384, 396, 417, 433, 434, 435, 326, 310, 411, - 311, 344, 312, 289, 318, 316, 319, 419, 320, 291, - 397, 439, 0, 339, 407, 370, 292, 369, 398, 438, - 437, 301, 464, 470, 471, 560, 0, 476, 645, 646, - 647, 485, 0, 403, 490, 491, 492, 494, 495, 496, - 497, 561, 578, 545, 515, 478, 569, 512, 516, 517, - 581, 0, 0, 0, 469, 358, 359, 0, 337, 285, - 286, 640, 323, 389, 583, 616, 617, 508, 0, 570, - 509, 518, 315, 542, 554, 553, 385, 468, 0, 565, - 568, 498, 639, 0, 562, 577, 643, 576, 636, 395, - 0, 416, 574, 521, 0, 566, 540, 0, 567, 536, - 571, 0, 510, 0, 424, 450, 462, 479, 482, 511, - 596, 597, 598, 290, 481, 600, 601, 602, 603, 604, - 605, 606, 599, 453, 543, 520, 546, 461, 523, 522, - 0, 0, 557, 477, 558, 559, 379, 380, 381, 382, - 341, 584, 308, 480, 405, 0, 544, 0, 0, 0, - 0, 0, 0, 0, 0, 549, 550, 547, 648, 0, - 607, 608, 0, 0, 474, 475, 336, 343, 493, 345, - 307, 394, 338, 459, 352, 0, 486, 551, 487, 610, - 613, 611, 612, 1038, 2052, 1034, 2053, 353, 363, 408, - 458, 392, 413, 305, 449, 422, 1035, 537, 564, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 592, 591, 590, 589, 588, - 587, 586, 585, 0, 0, 534, 436, 317, 279, 313, - 314, 321, 637, 634, 440, 638, 0, 287, 514, 361, - 0, 404, 335, 579, 580, 0, 0, 235, 236, 237, - 238, 239, 240, 241, 242, 280, 243, 244, 245, 246, - 247, 248, 249, 252, 253, 254, 255, 256, 257, 258, - 259, 582, 250, 251, 260, 261, 262, 263, 264, 265, - 266, 267, 268, 269, 270, 271, 272, 273, 0, 0, - 0, 281, 282, 283, 284, 0, 0, 275, 276, 277, - 278, 0, 0, 0, 465, 466, 467, 489, 0, 451, - 513, 635, 0, 0, 0, 0, 0, 0, 0, 563, - 575, 609, 0, 619, 620, 622, 624, 623, 626, 427, - 0, 628, 629, 630, 627, 365, 414, 432, 421, 0, - 641, 504, 505, 642, 615, 391, 0, 519, 552, 541, - 625, 507, 0, 0, 2929, 0, 0, 0, 0, 0, - 0, 0, 330, 0, 0, 360, 556, 538, 548, 539, - 524, 525, 526, 533, 340, 527, 528, 529, 499, 530, - 500, 531, 532, 0, 555, 506, 423, 375, 573, 572, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 222, - 0, 0, 0, 0, 0, 0, 303, 223, 501, 621, - 503, 502, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 306, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 293, 429, 447, 304, 418, 460, 309, - 426, 299, 390, 415, 0, 0, 295, 445, 425, 372, - 350, 351, 294, 0, 409, 328, 342, 325, 388, 0, - 444, 472, 324, 463, 0, 455, 297, 0, 454, 387, - 441, 446, 373, 367, 0, 296, 443, 371, 366, 354, - 332, 488, 355, 356, 346, 399, 364, 400, 347, 377, - 376, 378, 0, 0, 0, 0, 0, 483, 484, 0, - 0, 0, 0, 0, 0, 0, 0, 2932, 0, 0, - 0, 2931, 614, 0, 0, 618, 0, 457, 0, 0, - 0, 0, 0, 0, 428, 0, 0, 357, 0, 0, - 0, 473, 0, 412, 393, 644, 0, 0, 410, 362, - 442, 401, 448, 430, 456, 406, 402, 288, 431, 327, - 374, 300, 302, 322, 329, 331, 333, 334, 383, 384, - 396, 417, 433, 434, 435, 326, 310, 411, 311, 344, - 312, 289, 318, 316, 319, 419, 320, 291, 397, 439, - 0, 339, 407, 370, 292, 369, 398, 438, 437, 301, - 464, 470, 471, 560, 0, 476, 645, 646, 647, 485, - 0, 403, 490, 491, 492, 494, 495, 496, 497, 561, - 578, 545, 515, 478, 569, 512, 516, 517, 581, 0, - 0, 0, 469, 358, 359, 0, 337, 285, 286, 640, - 323, 389, 583, 616, 617, 508, 0, 570, 509, 518, - 315, 542, 554, 553, 385, 468, 0, 565, 568, 498, - 639, 0, 562, 577, 643, 576, 636, 395, 0, 416, - 574, 521, 0, 566, 540, 0, 567, 536, 571, 0, - 510, 0, 424, 450, 462, 479, 482, 511, 596, 597, - 598, 290, 481, 600, 601, 602, 603, 604, 605, 606, - 599, 453, 543, 520, 546, 461, 523, 522, 0, 0, - 557, 477, 558, 559, 379, 380, 381, 382, 341, 584, - 308, 480, 405, 0, 544, 0, 0, 0, 0, 0, - 0, 0, 0, 549, 550, 547, 648, 0, 607, 608, - 0, 0, 474, 475, 336, 343, 493, 345, 307, 394, - 338, 459, 352, 0, 486, 551, 487, 610, 613, 611, - 612, 386, 348, 349, 420, 353, 363, 408, 458, 392, - 413, 305, 449, 422, 368, 537, 564, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 274, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 592, 591, 590, 589, 588, 587, 586, - 585, 0, 0, 534, 436, 317, 279, 313, 314, 321, - 637, 634, 440, 638, 0, 287, 514, 361, 0, 404, - 335, 579, 580, 0, 0, 235, 236, 237, 238, 239, - 240, 241, 242, 280, 243, 244, 245, 246, 247, 248, - 249, 252, 253, 254, 255, 256, 257, 258, 259, 582, - 250, 251, 260, 261, 262, 263, 264, 265, 266, 267, - 268, 269, 270, 271, 272, 273, 0, 0, 0, 281, - 282, 283, 284, 0, 0, 275, 276, 277, 278, 0, - 0, 0, 465, 466, 467, 489, 0, 451, 513, 635, - 0, 0, 0, 0, 0, 0, 0, 563, 575, 609, - 0, 619, 620, 622, 624, 623, 626, 427, 0, 628, - 629, 630, 627, 365, 414, 432, 421, 0, 641, 504, - 505, 642, 615, 391, 0, 519, 552, 541, 625, 507, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 330, 1526, 0, 360, 556, 538, 548, 539, 524, 525, - 526, 533, 340, 527, 528, 529, 499, 530, 500, 531, - 532, 0, 555, 506, 423, 375, 573, 572, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 222, 0, 0, - 1524, 0, 0, 0, 303, 223, 501, 621, 503, 502, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 306, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1522, 0, 0, 0, 0, 0, - 0, 293, 429, 447, 304, 418, 460, 309, 426, 299, - 390, 415, 0, 0, 295, 445, 425, 372, 350, 351, - 294, 0, 409, 328, 342, 325, 388, 0, 444, 472, - 324, 463, 0, 455, 297, 0, 454, 387, 441, 446, - 373, 367, 0, 296, 443, 371, 366, 354, 332, 488, - 355, 356, 346, 399, 364, 400, 347, 377, 376, 378, - 0, 0, 0, 0, 0, 483, 484, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 614, 0, 0, 618, 0, 457, 0, 0, 0, 0, - 0, 0, 428, 0, 0, 357, 0, 0, 0, 473, - 0, 412, 393, 644, 0, 0, 410, 362, 442, 401, - 448, 430, 456, 406, 402, 288, 431, 327, 374, 300, - 302, 322, 329, 331, 333, 334, 383, 384, 396, 417, - 433, 434, 435, 326, 310, 411, 311, 344, 312, 289, - 318, 316, 319, 419, 320, 291, 397, 439, 0, 339, - 407, 370, 292, 369, 398, 438, 437, 301, 464, 470, - 471, 560, 0, 476, 645, 646, 647, 485, 0, 403, - 490, 491, 492, 494, 495, 496, 497, 561, 578, 545, - 515, 478, 569, 512, 516, 517, 581, 0, 0, 0, - 469, 358, 359, 0, 337, 285, 286, 640, 323, 389, - 583, 616, 617, 508, 0, 570, 509, 518, 315, 542, - 554, 553, 385, 468, 0, 565, 568, 498, 639, 0, - 562, 577, 643, 576, 636, 395, 0, 416, 574, 521, - 0, 566, 540, 0, 567, 536, 571, 0, 510, 0, - 424, 450, 462, 479, 482, 511, 596, 597, 598, 290, - 481, 600, 601, 602, 603, 604, 605, 606, 599, 453, - 543, 520, 546, 461, 523, 522, 0, 0, 557, 477, - 558, 559, 379, 380, 381, 382, 341, 584, 308, 480, - 405, 0, 544, 0, 0, 0, 0, 0, 0, 0, - 0, 549, 550, 547, 648, 0, 607, 608, 0, 0, - 474, 475, 336, 343, 493, 345, 307, 394, 338, 459, - 352, 0, 486, 551, 487, 610, 613, 611, 612, 386, - 348, 349, 420, 353, 363, 408, 458, 392, 413, 305, - 449, 422, 368, 537, 564, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 592, 591, 590, 589, 588, 587, 586, 585, 0, - 0, 534, 436, 317, 279, 313, 314, 321, 637, 634, - 440, 638, 0, 287, 514, 361, 0, 404, 335, 579, - 580, 0, 0, 235, 236, 237, 238, 239, 240, 241, - 242, 280, 243, 244, 245, 246, 247, 248, 249, 252, - 253, 254, 255, 256, 257, 258, 259, 582, 250, 251, - 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, - 270, 271, 272, 273, 0, 0, 0, 281, 282, 283, - 284, 0, 0, 275, 276, 277, 278, 0, 0, 0, - 465, 466, 467, 489, 0, 451, 513, 635, 0, 0, - 0, 0, 0, 0, 0, 563, 575, 609, 0, 619, - 620, 622, 624, 623, 626, 427, 0, 628, 629, 630, - 627, 365, 414, 432, 421, 0, 641, 504, 505, 642, - 615, 391, 0, 519, 552, 541, 625, 507, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 330, 1520, - 0, 360, 556, 538, 548, 539, 524, 525, 526, 533, - 340, 527, 528, 529, 499, 530, 500, 531, 532, 0, - 555, 506, 423, 375, 573, 572, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 222, 0, 0, 1524, 0, - 0, 0, 303, 223, 501, 621, 503, 502, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 306, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 1522, 0, 0, 0, 0, 0, 0, 293, - 429, 447, 304, 418, 460, 309, 426, 299, 390, 415, - 0, 0, 295, 445, 425, 372, 350, 351, 294, 0, - 409, 328, 342, 325, 388, 0, 444, 472, 324, 463, - 0, 455, 297, 0, 454, 387, 441, 446, 373, 367, - 0, 296, 443, 371, 366, 354, 332, 488, 355, 356, - 346, 399, 364, 400, 347, 377, 376, 378, 0, 0, - 0, 0, 0, 483, 484, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 614, 0, - 0, 618, 0, 457, 0, 0, 0, 0, 0, 0, - 428, 0, 0, 357, 0, 0, 0, 473, 0, 412, - 393, 644, 0, 0, 410, 362, 442, 401, 448, 430, - 456, 406, 402, 288, 431, 327, 374, 300, 302, 322, - 329, 331, 333, 334, 383, 384, 396, 417, 433, 434, - 435, 326, 310, 411, 311, 344, 312, 289, 318, 316, - 319, 419, 320, 291, 397, 439, 0, 339, 407, 370, - 292, 369, 398, 438, 437, 301, 464, 470, 471, 560, - 0, 476, 645, 646, 647, 485, 0, 403, 490, 491, - 492, 494, 495, 496, 497, 561, 578, 545, 515, 478, - 569, 512, 516, 517, 581, 0, 0, 0, 469, 358, - 359, 0, 337, 285, 286, 640, 323, 389, 583, 616, - 617, 508, 0, 570, 509, 518, 315, 542, 554, 553, - 385, 468, 0, 565, 568, 498, 639, 0, 562, 577, - 643, 576, 636, 395, 0, 416, 574, 521, 0, 566, - 540, 0, 567, 536, 571, 0, 510, 0, 424, 450, - 462, 479, 482, 511, 596, 597, 598, 290, 481, 600, - 601, 602, 603, 604, 605, 606, 599, 453, 543, 520, - 546, 461, 523, 522, 0, 0, 557, 477, 558, 559, - 379, 380, 381, 382, 341, 584, 308, 480, 405, 0, - 544, 0, 0, 0, 0, 0, 0, 0, 0, 549, - 550, 547, 648, 0, 607, 608, 0, 0, 474, 475, - 336, 343, 493, 345, 307, 394, 338, 459, 352, 0, - 486, 551, 487, 610, 613, 611, 612, 386, 348, 349, - 420, 353, 363, 408, 458, 392, 413, 305, 449, 422, - 368, 537, 564, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 592, - 591, 590, 589, 588, 587, 586, 585, 0, 0, 534, - 436, 317, 279, 313, 314, 321, 637, 634, 440, 638, - 0, 287, 514, 361, 0, 404, 335, 579, 580, 0, - 0, 235, 236, 237, 238, 239, 240, 241, 242, 280, - 243, 244, 245, 246, 247, 248, 249, 252, 253, 254, - 255, 256, 257, 258, 259, 582, 250, 251, 260, 261, - 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, - 272, 273, 0, 0, 0, 281, 282, 283, 284, 0, - 0, 275, 276, 277, 278, 0, 0, 0, 465, 466, - 467, 489, 0, 451, 513, 635, 0, 0, 0, 0, - 0, 0, 0, 563, 575, 609, 0, 619, 620, 622, - 624, 623, 626, 427, 0, 628, 629, 630, 627, 365, - 414, 432, 421, 0, 641, 504, 505, 642, 615, 391, - 0, 519, 552, 541, 625, 507, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 330, 0, 0, 360, - 556, 538, 548, 539, 524, 525, 526, 533, 340, 527, - 528, 529, 499, 530, 500, 531, 532, 0, 555, 506, - 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 3989, 0, 222, 838, 0, 0, 0, 0, 0, - 303, 223, 501, 621, 503, 502, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 306, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 293, 429, 447, - 304, 418, 460, 309, 426, 299, 390, 415, 0, 0, - 295, 445, 425, 372, 350, 351, 294, 0, 409, 328, - 342, 325, 388, 0, 444, 472, 324, 463, 0, 455, - 297, 0, 454, 387, 441, 446, 373, 367, 0, 296, - 443, 371, 366, 354, 332, 488, 355, 356, 346, 399, - 364, 400, 347, 377, 376, 378, 0, 0, 0, 0, - 0, 483, 484, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 614, 0, 0, 618, - 0, 457, 0, 0, 0, 0, 0, 0, 428, 0, - 0, 357, 0, 0, 0, 473, 0, 412, 393, 644, - 0, 0, 410, 362, 442, 401, 448, 430, 456, 406, - 402, 288, 431, 327, 374, 300, 302, 322, 329, 331, - 333, 334, 383, 384, 396, 417, 433, 434, 435, 326, - 310, 411, 311, 344, 312, 289, 318, 316, 319, 419, - 320, 291, 397, 439, 0, 339, 407, 370, 292, 369, - 398, 438, 437, 301, 464, 470, 471, 560, 0, 476, - 645, 646, 647, 485, 0, 403, 490, 491, 492, 494, - 495, 496, 497, 561, 578, 545, 515, 478, 569, 512, - 516, 517, 581, 0, 0, 0, 469, 358, 359, 0, - 337, 285, 286, 640, 323, 389, 583, 616, 617, 508, - 0, 570, 509, 518, 315, 542, 554, 553, 385, 468, - 0, 565, 568, 498, 639, 0, 562, 577, 643, 576, - 636, 395, 0, 416, 574, 521, 0, 566, 540, 0, - 567, 536, 571, 0, 510, 0, 424, 450, 462, 479, - 482, 511, 596, 597, 598, 290, 481, 600, 601, 602, - 603, 604, 605, 606, 599, 453, 543, 520, 546, 461, - 523, 522, 0, 0, 557, 477, 558, 559, 379, 380, - 381, 382, 341, 584, 308, 480, 405, 0, 544, 0, - 0, 0, 0, 0, 0, 0, 0, 549, 550, 547, - 648, 0, 607, 608, 0, 0, 474, 475, 336, 343, - 493, 345, 307, 394, 338, 459, 352, 0, 486, 551, - 487, 610, 613, 611, 612, 386, 348, 349, 420, 353, - 363, 408, 458, 392, 413, 305, 449, 422, 368, 537, - 564, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 592, 591, 590, - 589, 588, 587, 586, 585, 0, 0, 534, 436, 317, - 279, 313, 314, 321, 637, 634, 440, 638, 0, 287, - 514, 361, 0, 404, 335, 579, 580, 0, 0, 235, - 236, 237, 238, 239, 240, 241, 242, 280, 243, 244, - 245, 246, 247, 248, 249, 252, 253, 254, 255, 256, - 257, 258, 259, 582, 250, 251, 260, 261, 262, 263, - 264, 265, 266, 267, 268, 269, 270, 271, 272, 273, - 0, 0, 0, 281, 282, 283, 284, 0, 0, 275, - 276, 277, 278, 0, 0, 0, 465, 466, 467, 489, - 0, 451, 513, 635, 0, 0, 0, 0, 0, 0, - 0, 563, 575, 609, 0, 619, 620, 622, 624, 623, - 626, 427, 0, 628, 629, 630, 627, 365, 414, 432, - 421, 0, 641, 504, 505, 642, 615, 391, 0, 519, - 552, 541, 625, 507, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 330, 0, 0, 360, 556, 538, - 548, 539, 524, 525, 526, 533, 340, 527, 528, 529, - 499, 530, 500, 531, 532, 0, 555, 506, 423, 375, - 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 222, 0, 0, 1524, 0, 0, 0, 303, 223, - 501, 621, 503, 502, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 306, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1522, 0, - 0, 0, 0, 0, 0, 293, 429, 447, 304, 418, - 460, 309, 426, 299, 390, 415, 0, 0, 295, 445, - 425, 372, 350, 351, 294, 0, 409, 328, 342, 325, - 388, 0, 444, 472, 324, 463, 0, 455, 297, 0, - 454, 387, 441, 446, 373, 367, 0, 296, 443, 371, - 366, 354, 332, 488, 355, 356, 346, 399, 364, 400, - 347, 377, 376, 378, 0, 0, 0, 0, 0, 483, - 484, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 614, 0, 0, 618, 0, 457, - 0, 0, 0, 0, 0, 0, 428, 0, 0, 357, - 0, 0, 0, 473, 0, 412, 393, 644, 0, 0, - 410, 362, 442, 401, 448, 430, 456, 406, 402, 288, - 431, 327, 374, 300, 302, 322, 329, 331, 333, 334, - 383, 384, 396, 417, 433, 434, 435, 326, 310, 411, - 311, 344, 312, 289, 318, 316, 319, 419, 320, 291, - 397, 439, 0, 339, 407, 370, 292, 369, 398, 438, - 437, 301, 464, 470, 471, 560, 0, 476, 645, 646, - 647, 485, 0, 403, 490, 491, 492, 494, 495, 496, - 497, 561, 578, 545, 515, 478, 569, 512, 516, 517, - 581, 0, 0, 0, 469, 358, 359, 0, 337, 285, - 286, 640, 323, 389, 583, 616, 617, 508, 0, 570, - 509, 518, 315, 542, 554, 553, 385, 468, 0, 565, - 568, 498, 639, 0, 562, 577, 643, 576, 636, 395, - 0, 416, 574, 521, 0, 566, 540, 0, 567, 536, - 571, 0, 510, 0, 424, 450, 462, 479, 482, 511, - 596, 597, 598, 290, 481, 600, 601, 602, 603, 604, - 605, 606, 599, 453, 543, 520, 546, 461, 523, 522, - 0, 0, 557, 477, 558, 559, 379, 380, 381, 382, - 341, 584, 308, 480, 405, 0, 544, 0, 0, 0, - 0, 0, 0, 0, 0, 549, 550, 547, 648, 0, - 607, 608, 0, 0, 474, 475, 336, 343, 493, 345, - 307, 394, 338, 459, 352, 0, 486, 551, 487, 610, - 613, 611, 612, 386, 348, 349, 420, 353, 363, 408, - 458, 392, 413, 305, 449, 422, 368, 537, 564, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 592, 591, 590, 589, 588, - 587, 586, 585, 0, 0, 534, 436, 317, 279, 313, - 314, 321, 637, 634, 440, 638, 0, 287, 514, 361, - 0, 404, 335, 579, 580, 0, 0, 235, 236, 237, - 238, 239, 240, 241, 242, 280, 243, 244, 245, 246, - 247, 248, 249, 252, 253, 254, 255, 256, 257, 258, - 259, 582, 250, 251, 260, 261, 262, 263, 264, 265, - 266, 267, 268, 269, 270, 271, 272, 273, 0, 0, - 0, 281, 282, 283, 284, 0, 0, 275, 276, 277, - 278, 0, 0, 0, 465, 466, 467, 489, 0, 451, - 513, 635, 0, 0, 0, 0, 0, 0, 0, 563, - 575, 609, 0, 619, 620, 622, 624, 623, 626, 427, - 0, 628, 629, 630, 627, 365, 414, 432, 421, 0, - 641, 504, 505, 642, 615, 391, 0, 519, 552, 541, - 625, 507, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 330, 0, 0, 360, 556, 538, 548, 539, - 524, 525, 526, 533, 340, 527, 528, 529, 499, 530, - 500, 531, 532, 0, 555, 506, 423, 375, 573, 572, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 222, - 0, 0, 0, 0, 0, 0, 303, 223, 501, 621, - 503, 502, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 306, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 293, 429, 447, 304, 418, 460, 309, - 426, 299, 390, 415, 0, 0, 295, 445, 425, 372, - 350, 351, 294, 0, 409, 328, 342, 325, 388, 0, - 444, 472, 324, 463, 0, 455, 297, 0, 454, 387, - 441, 446, 373, 367, 0, 296, 443, 371, 366, 354, - 332, 488, 355, 356, 346, 399, 364, 400, 347, 377, - 376, 378, 0, 0, 0, 0, 0, 483, 484, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 614, 0, 0, 618, 0, 457, 0, 0, - 0, 0, 0, 0, 428, 0, 0, 357, 0, 0, - 0, 473, 0, 412, 393, 644, 0, 0, 410, 362, - 442, 401, 448, 430, 456, 406, 402, 288, 431, 327, - 374, 300, 302, 322, 329, 331, 333, 334, 383, 384, - 396, 417, 433, 434, 435, 326, 310, 411, 311, 344, - 312, 289, 318, 316, 319, 419, 320, 291, 397, 439, - 0, 339, 407, 370, 292, 369, 398, 438, 437, 301, - 464, 470, 471, 560, 0, 476, 645, 646, 647, 485, - 2448, 403, 490, 491, 492, 494, 495, 496, 497, 561, - 578, 545, 515, 478, 569, 512, 516, 517, 581, 0, - 0, 0, 469, 358, 359, 0, 337, 285, 286, 640, - 323, 389, 583, 616, 617, 508, 0, 570, 509, 518, - 315, 542, 554, 553, 385, 468, 0, 565, 568, 498, - 639, 0, 562, 577, 643, 576, 636, 395, 0, 416, - 574, 521, 0, 566, 540, 0, 567, 536, 571, 0, - 510, 0, 424, 450, 462, 479, 482, 511, 596, 597, - 598, 290, 481, 600, 601, 602, 603, 604, 605, 606, - 599, 453, 543, 520, 546, 461, 523, 522, 0, 0, - 557, 477, 558, 559, 379, 380, 381, 382, 341, 584, - 308, 480, 405, 0, 544, 0, 0, 0, 0, 0, - 0, 0, 0, 549, 550, 547, 648, 0, 607, 608, - 0, 0, 474, 475, 336, 343, 493, 345, 307, 394, - 338, 459, 352, 0, 486, 551, 487, 610, 613, 611, - 612, 386, 348, 349, 420, 353, 363, 408, 458, 392, - 413, 305, 449, 422, 368, 537, 564, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 274, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 592, 591, 590, 589, 588, 587, 586, - 585, 959, 0, 534, 436, 317, 279, 313, 314, 321, - 637, 634, 440, 638, 0, 287, 514, 361, 0, 404, - 335, 579, 580, 0, 0, 235, 236, 237, 238, 239, - 240, 241, 242, 280, 243, 244, 245, 246, 247, 248, - 249, 252, 253, 254, 255, 256, 257, 258, 259, 582, - 250, 251, 260, 261, 262, 263, 264, 265, 266, 267, - 268, 269, 270, 271, 272, 273, 0, 0, 0, 281, - 282, 283, 284, 0, 0, 275, 276, 277, 278, 0, - 0, 0, 465, 466, 467, 489, 0, 451, 513, 635, - 0, 0, 0, 0, 0, 0, 0, 563, 575, 609, - 0, 619, 620, 622, 624, 623, 626, 427, 0, 628, - 629, 630, 627, 365, 414, 432, 421, 0, 641, 504, - 505, 642, 615, 391, 0, 519, 552, 541, 625, 507, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 330, 0, 0, 360, 556, 538, 548, 539, 524, 525, - 526, 533, 340, 527, 528, 529, 499, 530, 500, 531, - 532, 0, 555, 506, 423, 375, 573, 572, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 222, 0, 0, - 1524, 0, 0, 0, 303, 223, 501, 621, 503, 502, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 306, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1734, 0, 0, 0, 0, 0, - 0, 293, 429, 447, 304, 418, 460, 309, 426, 299, - 390, 415, 0, 0, 295, 445, 425, 372, 350, 351, - 294, 0, 409, 328, 342, 325, 388, 0, 444, 472, - 324, 463, 0, 455, 297, 0, 454, 387, 441, 446, - 373, 367, 0, 296, 443, 371, 366, 354, 332, 488, - 355, 356, 346, 399, 364, 400, 347, 377, 376, 378, - 0, 0, 0, 0, 0, 483, 484, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 614, 0, 0, 618, 0, 457, 0, 0, 0, 0, - 0, 0, 428, 0, 0, 357, 0, 0, 0, 473, - 0, 412, 393, 644, 0, 0, 410, 362, 442, 401, - 448, 430, 456, 406, 402, 288, 431, 327, 374, 300, - 302, 322, 329, 331, 333, 334, 383, 384, 396, 417, - 433, 434, 435, 326, 310, 411, 311, 344, 312, 289, - 318, 316, 319, 419, 320, 291, 397, 439, 0, 339, - 407, 370, 292, 369, 398, 438, 437, 301, 464, 470, - 471, 560, 0, 476, 645, 646, 647, 485, 0, 403, - 490, 491, 492, 494, 495, 496, 497, 561, 578, 545, - 515, 478, 569, 512, 516, 517, 581, 0, 0, 0, - 469, 358, 359, 0, 337, 285, 286, 640, 323, 389, - 583, 616, 617, 508, 0, 570, 509, 518, 315, 542, - 554, 553, 385, 468, 0, 565, 568, 498, 639, 0, - 562, 577, 643, 576, 636, 395, 0, 416, 574, 521, - 0, 566, 540, 0, 567, 536, 571, 0, 510, 0, - 424, 450, 462, 479, 482, 511, 596, 597, 598, 290, - 481, 600, 601, 602, 603, 604, 605, 606, 599, 453, - 543, 520, 546, 461, 523, 522, 0, 0, 557, 477, - 558, 559, 379, 380, 381, 382, 341, 584, 308, 480, - 405, 0, 544, 0, 0, 0, 0, 0, 0, 0, - 0, 549, 550, 547, 648, 0, 607, 608, 0, 0, - 474, 475, 336, 343, 493, 345, 307, 394, 338, 459, - 352, 0, 486, 551, 487, 610, 613, 611, 612, 386, - 348, 349, 420, 353, 363, 408, 458, 392, 413, 305, - 449, 422, 368, 537, 564, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 592, 591, 590, 589, 588, 587, 586, 585, 0, - 0, 534, 436, 317, 279, 313, 314, 321, 637, 634, - 440, 638, 0, 287, 514, 361, 0, 404, 335, 579, - 580, 0, 0, 235, 236, 237, 238, 239, 240, 241, - 242, 280, 243, 244, 245, 246, 247, 248, 249, 252, - 253, 254, 255, 256, 257, 258, 259, 582, 250, 251, - 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, - 270, 271, 272, 273, 0, 0, 0, 281, 282, 283, - 284, 0, 0, 275, 276, 277, 278, 0, 0, 0, - 465, 466, 467, 489, 0, 451, 513, 635, 0, 0, - 0, 0, 0, 0, 0, 563, 575, 609, 0, 619, - 620, 622, 624, 623, 626, 427, 0, 628, 629, 630, - 627, 365, 414, 432, 421, 0, 641, 504, 505, 642, - 615, 391, 0, 519, 552, 541, 625, 507, 0, 0, - 0, 0, 0, 2485, 0, 0, 0, 0, 330, 0, - 0, 360, 556, 538, 548, 539, 524, 525, 526, 533, - 340, 527, 528, 529, 499, 530, 500, 531, 532, 0, - 555, 506, 423, 375, 573, 572, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 222, 0, 0, 2487, 0, - 0, 0, 303, 223, 501, 621, 503, 502, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 306, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 293, - 429, 447, 304, 418, 460, 309, 426, 299, 390, 415, - 0, 0, 295, 445, 425, 372, 350, 351, 294, 0, - 409, 328, 342, 325, 388, 0, 444, 472, 324, 463, - 0, 455, 297, 0, 454, 387, 441, 446, 373, 367, - 0, 296, 443, 371, 366, 354, 332, 488, 355, 356, - 346, 399, 364, 400, 347, 377, 376, 378, 0, 0, - 0, 0, 0, 483, 484, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 614, 0, - 0, 618, 0, 457, 0, 0, 0, 0, 0, 0, - 428, 0, 0, 357, 0, 0, 0, 473, 0, 412, - 393, 644, 0, 0, 410, 362, 442, 401, 448, 430, - 456, 406, 402, 288, 431, 327, 374, 300, 302, 322, - 329, 331, 333, 334, 383, 384, 396, 417, 433, 434, - 435, 326, 310, 411, 311, 344, 312, 289, 318, 316, - 319, 419, 320, 291, 397, 439, 0, 339, 407, 370, - 292, 369, 398, 438, 437, 301, 464, 470, 471, 560, - 0, 476, 645, 646, 647, 485, 0, 403, 490, 491, - 492, 494, 495, 496, 497, 561, 578, 545, 515, 478, - 569, 512, 516, 517, 581, 0, 0, 0, 469, 358, - 359, 0, 337, 285, 286, 640, 323, 389, 583, 616, - 617, 508, 0, 570, 509, 518, 315, 542, 554, 553, - 385, 468, 0, 565, 568, 498, 639, 0, 562, 577, - 643, 576, 636, 395, 0, 416, 574, 521, 0, 566, - 540, 0, 567, 536, 571, 0, 510, 0, 424, 450, - 462, 479, 482, 511, 596, 597, 598, 290, 481, 600, - 601, 602, 603, 604, 605, 606, 599, 453, 543, 520, - 546, 461, 523, 522, 0, 0, 557, 477, 558, 559, - 379, 380, 381, 382, 341, 584, 308, 480, 405, 0, - 544, 0, 0, 0, 0, 0, 0, 0, 0, 549, - 550, 547, 648, 0, 607, 608, 0, 0, 474, 475, - 336, 343, 493, 345, 307, 394, 338, 459, 352, 0, - 486, 551, 487, 610, 613, 611, 612, 386, 348, 349, - 420, 353, 363, 408, 458, 392, 413, 305, 449, 422, - 368, 537, 564, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 592, - 591, 590, 589, 588, 587, 586, 585, 0, 0, 534, - 436, 317, 279, 313, 314, 321, 637, 634, 440, 638, - 0, 287, 514, 361, 0, 404, 335, 579, 580, 0, - 0, 235, 236, 237, 238, 239, 240, 241, 242, 280, - 243, 244, 245, 246, 247, 248, 249, 252, 253, 254, - 255, 256, 257, 258, 259, 582, 250, 251, 260, 261, - 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, - 272, 273, 0, 0, 0, 281, 282, 283, 284, 0, - 0, 275, 276, 277, 278, 0, 0, 0, 465, 466, - 467, 489, 0, 451, 513, 635, 0, 0, 0, 0, - 0, 0, 0, 563, 575, 609, 0, 619, 620, 622, - 624, 623, 626, 427, 0, 628, 629, 630, 627, 365, - 414, 432, 421, 0, 641, 504, 505, 642, 615, 391, - 0, 519, 552, 541, 625, 507, 0, 0, 0, 0, - 0, 2099, 0, 0, 0, 0, 330, 0, 0, 360, - 556, 538, 548, 539, 524, 525, 526, 533, 340, 527, - 528, 529, 499, 530, 500, 531, 532, 0, 555, 506, - 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 222, 0, 0, 2100, 0, 0, 0, - 303, 223, 501, 621, 503, 502, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 306, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 293, 429, 447, - 304, 418, 460, 309, 426, 299, 390, 415, 0, 0, - 295, 445, 425, 372, 350, 351, 294, 0, 409, 328, - 342, 325, 388, 0, 444, 472, 324, 463, 0, 455, - 297, 0, 454, 387, 441, 446, 373, 367, 0, 296, - 443, 371, 366, 354, 332, 488, 355, 356, 346, 399, - 364, 400, 347, 377, 376, 378, 0, 0, 0, 0, - 0, 483, 484, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 614, 0, 0, 618, - 0, 457, 0, 0, 0, 0, 0, 0, 428, 0, - 0, 357, 0, 0, 0, 473, 0, 412, 393, 644, - 0, 0, 410, 362, 442, 401, 448, 430, 456, 406, - 402, 288, 431, 327, 374, 300, 302, 322, 329, 331, - 333, 334, 383, 384, 396, 417, 433, 434, 435, 326, - 310, 411, 311, 344, 312, 289, 318, 316, 319, 419, - 320, 291, 397, 439, 0, 339, 407, 370, 292, 369, - 398, 438, 437, 301, 464, 470, 471, 560, 0, 476, - 645, 646, 647, 485, 0, 403, 490, 491, 492, 494, - 495, 496, 497, 561, 578, 545, 515, 478, 569, 512, - 516, 517, 581, 0, 0, 0, 469, 358, 359, 0, - 337, 285, 286, 640, 323, 389, 583, 616, 617, 508, - 0, 570, 509, 518, 315, 542, 554, 553, 385, 468, - 0, 565, 568, 498, 639, 0, 562, 577, 643, 576, - 636, 395, 0, 416, 574, 521, 0, 566, 540, 0, - 567, 536, 571, 0, 510, 0, 424, 450, 462, 479, - 482, 511, 596, 597, 598, 290, 481, 600, 601, 602, - 603, 604, 605, 606, 599, 453, 543, 520, 546, 461, - 523, 522, 0, 0, 557, 477, 558, 559, 379, 380, - 381, 382, 341, 584, 308, 480, 405, 0, 544, 0, - 0, 0, 0, 0, 0, 0, 0, 549, 550, 547, - 648, 0, 607, 608, 0, 0, 474, 475, 336, 343, - 493, 345, 307, 394, 338, 459, 352, 0, 486, 551, - 487, 610, 613, 611, 612, 386, 348, 349, 420, 353, - 363, 408, 458, 392, 413, 305, 449, 422, 368, 537, - 564, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 592, 591, 590, - 589, 588, 587, 586, 585, 0, 0, 534, 436, 317, - 279, 313, 314, 321, 637, 634, 440, 638, 0, 287, - 514, 361, 0, 404, 335, 579, 580, 0, 0, 235, - 236, 237, 238, 239, 240, 241, 242, 280, 243, 244, - 245, 246, 247, 248, 249, 252, 253, 254, 255, 256, - 257, 258, 259, 582, 250, 251, 260, 261, 262, 263, - 264, 265, 266, 267, 268, 269, 270, 271, 272, 273, - 0, 0, 0, 281, 282, 283, 284, 0, 0, 275, - 276, 277, 278, 0, 0, 0, 465, 466, 467, 489, - 0, 451, 513, 635, 0, 0, 0, 0, 0, 0, - 0, 563, 575, 609, 0, 619, 620, 622, 624, 623, - 626, 427, 0, 628, 629, 630, 627, 365, 414, 432, - 421, 0, 641, 504, 505, 642, 615, 391, 0, 519, - 552, 541, 625, 507, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 330, 0, 0, 360, 556, 538, - 548, 539, 524, 525, 526, 533, 340, 527, 528, 529, - 499, 530, 500, 531, 532, 0, 555, 506, 423, 375, - 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 222, 0, 0, 3141, 3143, 0, 0, 303, 223, - 501, 621, 503, 502, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 306, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 293, 429, 447, 304, 418, - 460, 309, 426, 299, 390, 415, 0, 0, 295, 445, - 425, 372, 350, 351, 294, 0, 409, 328, 342, 325, - 388, 0, 444, 472, 324, 463, 0, 455, 297, 0, - 454, 387, 441, 446, 373, 367, 0, 296, 443, 371, - 366, 354, 332, 488, 355, 356, 346, 399, 364, 400, - 347, 377, 376, 378, 0, 0, 0, 0, 0, 483, - 484, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 614, 0, 0, 618, 0, 457, - 0, 0, 0, 0, 0, 0, 428, 0, 0, 357, - 0, 0, 0, 473, 0, 412, 393, 644, 0, 0, - 410, 362, 442, 401, 448, 430, 456, 406, 402, 288, - 431, 327, 374, 300, 302, 322, 329, 331, 333, 334, - 383, 384, 396, 417, 433, 434, 435, 326, 310, 411, - 311, 344, 312, 289, 318, 316, 319, 419, 320, 291, - 397, 439, 0, 339, 407, 370, 292, 369, 398, 438, - 437, 301, 464, 470, 471, 560, 0, 476, 645, 646, - 647, 485, 0, 403, 490, 491, 492, 494, 495, 496, - 497, 561, 578, 545, 515, 478, 569, 512, 516, 517, - 581, 0, 0, 0, 469, 358, 359, 0, 337, 285, - 286, 640, 323, 389, 583, 616, 617, 508, 0, 570, - 509, 518, 315, 542, 554, 553, 385, 468, 0, 565, - 568, 498, 639, 0, 562, 577, 643, 576, 636, 395, - 0, 416, 574, 521, 0, 566, 540, 0, 567, 536, - 571, 0, 510, 0, 424, 450, 462, 479, 482, 511, - 596, 597, 598, 290, 481, 600, 601, 602, 603, 604, - 605, 606, 599, 453, 543, 520, 546, 461, 523, 522, - 0, 0, 557, 477, 558, 559, 379, 380, 381, 382, - 341, 584, 308, 480, 405, 0, 544, 0, 0, 0, - 0, 0, 0, 0, 0, 549, 550, 547, 648, 0, - 607, 608, 0, 0, 474, 475, 336, 343, 493, 345, - 307, 394, 338, 459, 352, 0, 486, 551, 487, 610, - 613, 611, 612, 386, 348, 349, 420, 353, 363, 408, - 458, 392, 413, 305, 449, 422, 368, 537, 564, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 592, 591, 590, 589, 588, - 587, 586, 585, 0, 0, 534, 436, 317, 279, 313, - 314, 321, 637, 634, 440, 638, 0, 287, 514, 361, - 0, 404, 335, 579, 580, 0, 0, 235, 236, 237, - 238, 239, 240, 241, 242, 280, 243, 244, 245, 246, - 247, 248, 249, 252, 253, 254, 255, 256, 257, 258, - 259, 582, 250, 251, 260, 261, 262, 263, 264, 265, - 266, 267, 268, 269, 270, 271, 272, 273, 0, 0, - 0, 281, 282, 283, 284, 0, 0, 275, 276, 277, - 278, 0, 0, 0, 465, 466, 467, 489, 0, 451, - 513, 635, 0, 0, 0, 0, 0, 0, 0, 563, - 575, 609, 0, 619, 620, 622, 624, 623, 626, 427, - 0, 628, 629, 630, 627, 365, 414, 432, 421, 0, - 641, 504, 505, 642, 615, 391, 0, 519, 552, 541, - 625, 507, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 330, 2508, 0, 360, 556, 538, 548, 539, - 524, 525, 526, 533, 340, 527, 528, 529, 499, 530, - 500, 531, 532, 0, 555, 506, 423, 375, 573, 572, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 222, - 0, 0, 1524, 0, 0, 0, 303, 223, 501, 621, - 503, 502, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 306, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 293, 429, 447, 304, 418, 460, 309, - 426, 299, 390, 415, 0, 0, 295, 445, 425, 372, - 350, 351, 294, 0, 409, 328, 342, 325, 388, 0, - 444, 472, 324, 463, 0, 455, 297, 0, 454, 387, - 441, 446, 373, 367, 0, 296, 443, 371, 366, 354, - 332, 488, 355, 356, 346, 399, 364, 400, 347, 377, - 376, 378, 0, 0, 0, 0, 0, 483, 484, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 614, 0, 0, 618, 0, 457, 0, 0, - 0, 0, 0, 0, 428, 0, 0, 357, 0, 0, - 0, 473, 0, 412, 393, 644, 0, 0, 410, 362, - 442, 401, 448, 430, 456, 406, 402, 288, 431, 327, - 374, 300, 302, 322, 329, 331, 333, 334, 383, 384, - 396, 417, 433, 434, 435, 326, 310, 411, 311, 344, - 312, 289, 318, 316, 319, 419, 320, 291, 397, 439, - 0, 339, 407, 370, 292, 369, 398, 438, 437, 301, - 464, 470, 471, 560, 0, 476, 645, 646, 647, 485, - 0, 403, 490, 491, 492, 494, 495, 496, 497, 561, - 578, 545, 515, 478, 569, 512, 516, 517, 581, 0, - 0, 0, 469, 358, 359, 0, 337, 285, 286, 640, - 323, 389, 583, 616, 617, 508, 0, 570, 509, 518, - 315, 542, 554, 553, 385, 468, 0, 565, 568, 498, - 639, 0, 562, 577, 643, 576, 636, 395, 0, 416, - 574, 521, 0, 566, 540, 0, 567, 536, 571, 0, - 510, 0, 424, 450, 462, 479, 482, 511, 596, 597, - 598, 290, 481, 600, 601, 602, 603, 604, 605, 606, - 599, 453, 543, 520, 546, 461, 523, 522, 0, 0, - 557, 477, 558, 559, 379, 380, 381, 382, 341, 584, - 308, 480, 405, 0, 544, 0, 0, 0, 0, 0, - 0, 0, 0, 549, 550, 547, 648, 0, 607, 608, - 0, 0, 474, 475, 336, 343, 493, 345, 307, 394, - 338, 459, 352, 0, 486, 551, 487, 610, 613, 611, - 612, 386, 348, 349, 420, 353, 363, 408, 458, 392, - 413, 305, 449, 422, 368, 537, 564, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 274, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 592, 591, 590, 589, 588, 587, 586, - 585, 0, 0, 534, 436, 317, 279, 313, 314, 321, - 637, 634, 440, 638, 0, 287, 514, 361, 0, 404, - 335, 579, 580, 0, 0, 235, 236, 237, 238, 239, - 240, 241, 242, 280, 243, 244, 245, 246, 247, 248, - 249, 252, 253, 254, 255, 256, 257, 258, 259, 582, - 250, 251, 260, 261, 262, 263, 264, 265, 266, 267, - 268, 269, 270, 271, 272, 273, 0, 0, 0, 281, - 282, 283, 284, 0, 0, 275, 276, 277, 278, 0, - 0, 0, 465, 466, 467, 489, 0, 451, 513, 635, - 0, 0, 0, 0, 0, 0, 0, 563, 575, 609, - 0, 619, 620, 622, 624, 623, 626, 427, 0, 628, - 629, 630, 627, 365, 414, 432, 421, 0, 641, 504, - 505, 642, 615, 391, 0, 519, 552, 541, 625, 507, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 655, - 330, 0, 0, 360, 556, 538, 548, 539, 524, 525, - 526, 533, 340, 527, 528, 529, 499, 530, 500, 531, - 532, 0, 555, 506, 423, 375, 573, 572, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 222, 0, 0, - 0, 0, 0, 0, 303, 223, 501, 621, 503, 502, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 306, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 293, 429, 447, 304, 418, 460, 309, 426, 299, - 390, 415, 0, 0, 295, 445, 425, 372, 350, 351, - 294, 0, 409, 328, 342, 325, 388, 0, 444, 472, - 324, 463, 0, 455, 297, 0, 454, 387, 441, 446, - 373, 367, 0, 296, 443, 371, 366, 354, 332, 488, - 355, 356, 346, 399, 364, 400, 347, 377, 376, 378, - 0, 0, 0, 0, 0, 483, 484, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 614, 0, 0, 618, 0, 457, 0, 654, 0, 0, - 0, 0, 428, 0, 0, 357, 0, 0, 0, 473, - 0, 412, 393, 644, 0, 0, 410, 362, 442, 401, - 448, 430, 456, 406, 402, 288, 431, 327, 374, 300, - 302, 322, 329, 331, 333, 334, 383, 384, 396, 417, - 433, 434, 435, 326, 310, 411, 311, 344, 312, 289, - 318, 316, 319, 419, 320, 291, 397, 439, 0, 339, - 407, 370, 292, 369, 398, 438, 437, 301, 464, 470, - 471, 560, 0, 476, 645, 646, 647, 485, 0, 403, - 490, 491, 492, 494, 495, 496, 497, 561, 578, 545, - 515, 478, 569, 512, 516, 517, 581, 0, 0, 0, - 469, 358, 359, 0, 337, 285, 286, 640, 323, 389, - 583, 616, 617, 508, 0, 570, 509, 518, 315, 542, - 554, 553, 385, 468, 0, 565, 568, 498, 639, 0, - 562, 577, 643, 576, 636, 395, 0, 416, 574, 521, - 0, 566, 540, 0, 567, 536, 571, 0, 510, 0, - 424, 450, 462, 479, 482, 511, 596, 597, 598, 290, - 481, 600, 601, 602, 603, 604, 605, 606, 599, 453, - 543, 520, 546, 461, 523, 522, 0, 0, 557, 477, - 558, 559, 379, 380, 381, 382, 341, 584, 308, 480, - 405, 0, 544, 0, 0, 0, 0, 0, 0, 0, - 0, 549, 550, 547, 648, 0, 607, 608, 0, 0, - 474, 475, 336, 343, 493, 345, 307, 394, 338, 459, - 352, 0, 486, 551, 487, 610, 613, 611, 612, 386, - 348, 349, 420, 353, 363, 408, 458, 392, 413, 305, - 449, 422, 368, 537, 564, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 592, 591, 590, 589, 588, 587, 586, 585, 0, - 0, 534, 436, 317, 279, 313, 314, 321, 637, 634, - 440, 638, 0, 287, 514, 361, 0, 404, 335, 579, - 580, 0, 0, 235, 236, 237, 238, 239, 240, 241, - 242, 280, 243, 244, 245, 246, 247, 248, 249, 252, - 253, 254, 255, 256, 257, 258, 259, 582, 250, 251, - 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, - 270, 271, 272, 273, 0, 0, 0, 281, 282, 283, - 284, 0, 0, 275, 276, 277, 278, 0, 0, 0, - 465, 466, 467, 489, 0, 451, 513, 635, 0, 0, - 0, 0, 0, 0, 0, 563, 575, 609, 0, 619, - 620, 622, 624, 623, 626, 427, 0, 628, 629, 630, - 627, 365, 414, 432, 421, 0, 641, 504, 505, 642, - 615, 391, 0, 519, 552, 541, 625, 507, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 330, 0, - 0, 360, 556, 538, 548, 539, 524, 525, 526, 533, - 340, 527, 528, 529, 499, 530, 500, 531, 532, 0, - 555, 506, 423, 375, 573, 572, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 222, 838, 0, 0, 0, - 0, 0, 303, 223, 501, 621, 503, 502, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 306, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 293, - 429, 447, 304, 418, 460, 309, 426, 299, 390, 415, - 0, 0, 295, 445, 425, 372, 350, 351, 294, 0, - 409, 328, 342, 325, 388, 0, 444, 472, 324, 463, - 0, 455, 297, 0, 454, 387, 441, 446, 373, 367, - 0, 296, 443, 371, 366, 354, 332, 488, 355, 356, - 346, 399, 364, 400, 347, 377, 376, 378, 0, 0, - 0, 0, 0, 483, 484, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 614, 0, - 0, 618, 0, 457, 0, 0, 0, 0, 0, 0, - 428, 0, 0, 357, 0, 0, 0, 473, 0, 412, - 393, 644, 0, 0, 410, 362, 442, 401, 448, 430, - 456, 406, 402, 288, 431, 327, 374, 300, 302, 322, - 329, 331, 333, 334, 383, 384, 396, 417, 433, 434, - 435, 326, 310, 411, 311, 344, 312, 289, 318, 316, - 319, 419, 320, 291, 397, 439, 0, 339, 407, 370, - 292, 369, 398, 438, 437, 301, 464, 470, 471, 560, - 0, 476, 645, 646, 647, 485, 0, 403, 490, 491, - 492, 494, 495, 496, 497, 561, 578, 545, 515, 478, - 569, 512, 516, 517, 581, 0, 0, 0, 469, 358, - 359, 0, 337, 285, 286, 640, 323, 389, 583, 616, - 617, 508, 0, 570, 509, 518, 315, 542, 554, 553, - 385, 468, 0, 565, 568, 498, 639, 0, 562, 577, - 643, 576, 636, 395, 0, 416, 574, 521, 0, 566, - 540, 0, 567, 536, 571, 0, 510, 0, 424, 450, - 462, 479, 482, 511, 596, 597, 598, 290, 481, 600, - 601, 602, 603, 604, 605, 606, 599, 453, 543, 520, - 546, 461, 523, 522, 0, 0, 557, 477, 558, 559, - 379, 380, 381, 382, 341, 584, 308, 480, 405, 0, - 544, 0, 0, 0, 0, 0, 0, 0, 0, 549, - 550, 547, 648, 0, 607, 608, 0, 0, 474, 475, - 336, 343, 493, 345, 307, 394, 338, 459, 352, 0, - 486, 551, 487, 610, 613, 611, 612, 386, 348, 349, - 420, 353, 363, 408, 458, 392, 413, 305, 449, 422, - 368, 537, 564, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 592, - 591, 590, 589, 588, 587, 586, 585, 0, 0, 534, - 436, 317, 279, 313, 314, 321, 637, 634, 440, 638, - 0, 287, 514, 361, 0, 404, 335, 579, 580, 0, - 0, 235, 236, 237, 238, 239, 240, 241, 242, 280, - 243, 244, 245, 246, 247, 248, 249, 252, 253, 254, - 255, 256, 257, 258, 259, 582, 250, 251, 260, 261, - 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, - 272, 273, 0, 0, 0, 281, 282, 283, 284, 0, - 0, 275, 276, 277, 278, 0, 0, 0, 465, 466, - 467, 489, 0, 451, 513, 635, 0, 0, 0, 0, - 0, 0, 0, 563, 575, 609, 0, 619, 620, 622, - 624, 623, 626, 427, 0, 628, 629, 630, 627, 365, - 414, 432, 421, 0, 641, 504, 505, 642, 615, 391, - 0, 519, 552, 541, 625, 507, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 330, 0, 0, 360, - 556, 538, 548, 539, 524, 525, 526, 533, 340, 527, - 528, 529, 499, 530, 500, 531, 532, 0, 555, 506, - 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 3966, 0, 0, 222, 0, 0, 0, 0, 0, 0, - 303, 223, 501, 621, 503, 502, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 306, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 432, 421, 0, 641, 504, 505, 642, 615, 802, 751, + 0, 2234, 0, 0, 0, 0, 0, 391, 0, 519, + 552, 541, 625, 507, 0, 0, 0, 0, 0, 0, + 754, 0, 0, 0, 330, 0, 0, 360, 556, 538, + 548, 539, 524, 525, 526, 533, 340, 527, 528, 529, + 499, 530, 500, 531, 532, 793, 555, 506, 423, 375, + 573, 572, 0, 0, 862, 870, 0, 0, 0, 0, + 0, 0, 0, 0, 858, 0, 0, 0, 0, 746, + 0, 0, 783, 838, 837, 770, 780, 0, 0, 303, + 223, 501, 621, 503, 502, 771, 0, 772, 776, 779, + 775, 773, 774, 0, 853, 0, 0, 0, 0, 0, + 0, 738, 750, 0, 755, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 747, 748, + 0, 0, 0, 0, 803, 0, 749, 0, 0, 798, + 777, 781, 0, 0, 0, 0, 293, 429, 447, 304, + 418, 460, 309, 426, 299, 390, 415, 0, 0, 295, + 445, 425, 372, 350, 351, 294, 0, 409, 328, 342, + 325, 388, 778, 801, 805, 324, 876, 799, 455, 297, + 0, 454, 387, 441, 446, 373, 367, 0, 296, 443, + 371, 366, 354, 332, 877, 355, 356, 346, 399, 364, + 400, 347, 377, 376, 378, 0, 0, 0, 0, 0, + 483, 484, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 614, 796, 0, 618, 0, + 457, 0, 0, 860, 0, 0, 0, 428, 0, 0, + 357, 0, 0, 0, 800, 0, 412, 393, 873, 0, + 0, 410, 362, 442, 401, 448, 430, 456, 406, 402, + 288, 431, 327, 374, 300, 302, 322, 329, 331, 333, + 334, 383, 384, 396, 417, 433, 434, 435, 326, 310, + 411, 311, 344, 312, 289, 318, 316, 319, 419, 320, + 291, 397, 439, 0, 339, 407, 370, 292, 369, 398, + 438, 437, 301, 464, 470, 471, 560, 0, 476, 645, + 646, 647, 485, 0, 403, 490, 491, 492, 494, 495, + 496, 497, 561, 578, 545, 515, 478, 569, 512, 516, + 517, 581, 0, 0, 0, 469, 358, 359, 0, 337, + 285, 286, 640, 857, 389, 583, 616, 617, 508, 0, + 872, 852, 854, 855, 859, 863, 864, 865, 866, 867, + 869, 871, 875, 639, 0, 562, 577, 643, 576, 636, + 395, 0, 416, 574, 521, 0, 566, 540, 0, 567, + 536, 571, 0, 510, 0, 424, 450, 462, 479, 482, + 511, 596, 597, 598, 290, 481, 600, 601, 602, 603, + 604, 605, 606, 599, 874, 543, 520, 546, 461, 523, + 522, 0, 0, 557, 804, 558, 559, 379, 380, 381, + 382, 861, 584, 308, 480, 405, 0, 544, 0, 0, + 0, 0, 0, 0, 0, 0, 549, 550, 547, 648, + 0, 607, 608, 0, 0, 474, 475, 336, 343, 493, + 345, 307, 394, 338, 459, 352, 0, 486, 551, 487, + 610, 613, 611, 612, 386, 348, 349, 420, 353, 363, + 408, 458, 392, 413, 305, 449, 422, 368, 537, 564, + 883, 856, 882, 884, 885, 881, 886, 887, 868, 759, + 0, 811, 879, 878, 880, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 592, 591, 590, 589, + 588, 587, 586, 585, 0, 0, 534, 436, 317, 279, + 313, 314, 321, 637, 634, 440, 638, 766, 287, 514, + 361, 0, 404, 335, 579, 580, 0, 0, 845, 818, + 819, 820, 756, 821, 815, 816, 757, 817, 846, 809, + 842, 843, 785, 812, 822, 841, 823, 844, 847, 848, + 888, 889, 829, 813, 251, 890, 826, 849, 840, 839, + 824, 810, 850, 851, 792, 787, 827, 828, 814, 833, + 834, 835, 758, 806, 807, 808, 830, 831, 788, 789, + 790, 791, 0, 0, 0, 465, 466, 467, 489, 0, + 451, 513, 635, 0, 0, 0, 0, 0, 0, 0, + 563, 575, 609, 0, 619, 620, 622, 624, 836, 626, + 427, 0, 832, 629, 630, 627, 365, 414, 432, 421, + 802, 641, 504, 505, 642, 615, 0, 751, 0, 391, + 0, 519, 552, 541, 625, 507, 0, 0, 0, 0, + 0, 0, 754, 0, 0, 0, 330, 0, 0, 360, + 556, 538, 548, 539, 524, 525, 526, 533, 340, 527, + 528, 529, 499, 530, 500, 531, 532, 793, 555, 506, + 423, 375, 573, 572, 0, 0, 862, 870, 0, 0, + 0, 0, 0, 0, 0, 0, 858, 0, 0, 0, + 0, 746, 0, 0, 783, 838, 837, 770, 780, 0, + 0, 303, 223, 501, 621, 503, 502, 771, 0, 772, + 776, 779, 775, 773, 774, 0, 853, 0, 0, 0, + 0, 0, 0, 738, 750, 0, 755, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 293, 429, 447, - 304, 418, 460, 309, 426, 299, 390, 415, 0, 0, - 295, 445, 425, 372, 350, 351, 294, 0, 409, 328, - 342, 325, 388, 0, 444, 472, 324, 463, 0, 455, - 297, 0, 454, 387, 441, 446, 373, 367, 0, 296, - 443, 371, 366, 354, 332, 488, 355, 356, 346, 399, - 364, 400, 347, 377, 376, 378, 0, 0, 0, 0, - 0, 483, 484, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 614, 0, 0, 618, - 0, 457, 0, 0, 0, 0, 0, 0, 428, 0, - 0, 357, 0, 0, 0, 473, 0, 412, 393, 644, - 0, 0, 410, 362, 442, 401, 448, 430, 456, 406, - 402, 288, 431, 327, 374, 300, 302, 322, 329, 331, - 333, 334, 383, 384, 396, 417, 433, 434, 435, 326, - 310, 411, 311, 344, 312, 289, 318, 316, 319, 419, - 320, 291, 397, 439, 0, 339, 407, 370, 292, 369, - 398, 438, 437, 301, 464, 470, 471, 560, 0, 476, - 645, 646, 647, 485, 0, 403, 490, 491, 492, 494, - 495, 496, 497, 561, 578, 545, 515, 478, 569, 512, - 516, 517, 581, 0, 0, 0, 469, 358, 359, 0, - 337, 285, 286, 640, 323, 389, 583, 616, 617, 508, - 0, 570, 509, 518, 315, 542, 554, 553, 385, 468, - 0, 565, 568, 498, 639, 0, 562, 577, 643, 576, - 636, 395, 0, 416, 574, 521, 0, 566, 540, 0, - 567, 536, 571, 0, 510, 0, 424, 450, 462, 479, - 482, 511, 596, 597, 598, 290, 481, 600, 601, 602, - 603, 604, 605, 606, 599, 453, 543, 520, 546, 461, - 523, 522, 0, 0, 557, 477, 558, 559, 379, 380, - 381, 382, 341, 584, 308, 480, 405, 0, 544, 0, - 0, 0, 0, 0, 0, 0, 0, 549, 550, 547, - 648, 0, 607, 608, 0, 0, 474, 475, 336, 343, - 493, 345, 307, 394, 338, 459, 352, 0, 486, 551, - 487, 610, 613, 611, 612, 386, 348, 349, 420, 353, - 363, 408, 458, 392, 413, 305, 449, 422, 368, 537, - 564, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 592, 591, 590, - 589, 588, 587, 586, 585, 0, 0, 534, 436, 317, - 279, 313, 314, 321, 637, 634, 440, 638, 0, 287, - 514, 361, 0, 404, 335, 579, 580, 0, 0, 235, - 236, 237, 238, 239, 240, 241, 242, 280, 243, 244, - 245, 246, 247, 248, 249, 252, 253, 254, 255, 256, - 257, 258, 259, 582, 250, 251, 260, 261, 262, 263, - 264, 265, 266, 267, 268, 269, 270, 271, 272, 273, - 0, 0, 0, 281, 282, 283, 284, 0, 0, 275, - 276, 277, 278, 0, 0, 0, 465, 466, 467, 489, - 0, 451, 513, 635, 0, 0, 0, 0, 0, 0, - 0, 563, 575, 609, 0, 619, 620, 622, 624, 623, - 626, 427, 0, 628, 629, 630, 627, 365, 414, 432, - 421, 0, 641, 504, 505, 642, 615, 391, 0, 519, - 552, 541, 625, 507, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 330, 0, 0, 360, 556, 538, - 548, 539, 524, 525, 526, 533, 340, 527, 528, 529, - 499, 530, 500, 531, 532, 0, 555, 506, 423, 375, - 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 222, 0, 0, 3734, 0, 0, 0, 303, 223, - 501, 621, 503, 502, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 306, 0, 0, 0, 0, 0, 0, + 747, 748, 1840, 0, 0, 0, 803, 0, 749, 0, + 0, 798, 777, 781, 0, 0, 0, 0, 293, 429, + 447, 304, 418, 460, 309, 426, 299, 390, 415, 0, + 0, 295, 445, 425, 372, 350, 351, 294, 0, 409, + 328, 342, 325, 388, 778, 801, 805, 324, 876, 799, + 455, 297, 0, 454, 387, 441, 446, 373, 367, 0, + 296, 443, 371, 366, 354, 332, 877, 355, 356, 346, + 399, 364, 400, 347, 377, 376, 378, 0, 0, 0, + 0, 0, 483, 484, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 614, 796, 0, + 618, 0, 457, 0, 0, 860, 0, 0, 0, 428, + 0, 0, 357, 0, 0, 0, 800, 0, 412, 393, + 873, 0, 0, 410, 362, 442, 401, 448, 430, 456, + 406, 402, 288, 431, 327, 374, 300, 302, 322, 329, + 331, 333, 334, 383, 384, 396, 417, 433, 434, 435, + 326, 310, 411, 311, 344, 312, 289, 318, 316, 319, + 419, 320, 291, 397, 439, 0, 339, 407, 370, 292, + 369, 398, 438, 437, 301, 464, 470, 471, 560, 0, + 476, 645, 646, 647, 485, 0, 403, 490, 491, 492, + 494, 495, 496, 497, 561, 578, 545, 515, 478, 569, + 512, 516, 517, 581, 0, 0, 0, 469, 358, 359, + 0, 337, 285, 286, 640, 857, 389, 583, 616, 617, + 508, 0, 872, 852, 854, 855, 859, 863, 864, 865, + 866, 867, 869, 871, 875, 639, 0, 562, 577, 643, + 576, 636, 395, 0, 416, 574, 521, 0, 566, 540, + 0, 567, 536, 571, 0, 510, 0, 424, 450, 462, + 479, 482, 511, 596, 597, 598, 290, 481, 600, 601, + 602, 603, 604, 605, 606, 599, 874, 543, 520, 546, + 461, 523, 522, 0, 0, 557, 804, 558, 559, 379, + 380, 381, 382, 861, 584, 308, 480, 405, 0, 544, + 0, 0, 0, 0, 0, 0, 0, 0, 549, 550, + 547, 648, 0, 607, 608, 0, 0, 474, 475, 336, + 343, 493, 345, 307, 394, 338, 459, 352, 0, 486, + 551, 487, 610, 613, 611, 612, 386, 348, 349, 420, + 353, 363, 408, 458, 392, 413, 305, 449, 422, 368, + 537, 564, 883, 856, 882, 884, 885, 881, 886, 887, + 868, 759, 0, 811, 879, 878, 880, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 592, 591, + 590, 589, 588, 587, 586, 585, 0, 0, 534, 436, + 317, 279, 313, 314, 321, 637, 634, 440, 638, 766, + 287, 514, 361, 0, 404, 335, 579, 580, 0, 0, + 845, 818, 819, 820, 756, 821, 815, 816, 757, 817, + 846, 809, 842, 843, 785, 812, 822, 841, 823, 844, + 847, 848, 888, 889, 829, 813, 251, 890, 826, 849, + 840, 839, 824, 810, 850, 851, 792, 787, 827, 828, + 814, 833, 834, 835, 758, 806, 807, 808, 830, 831, + 788, 789, 790, 791, 0, 0, 0, 465, 466, 467, + 489, 0, 451, 513, 635, 0, 0, 0, 0, 0, + 0, 0, 563, 575, 609, 0, 619, 620, 622, 624, + 836, 626, 427, 0, 832, 629, 630, 627, 365, 414, + 432, 421, 802, 641, 504, 505, 642, 615, 0, 751, + 0, 391, 0, 519, 552, 541, 625, 507, 0, 0, + 0, 0, 0, 0, 754, 0, 0, 0, 330, 0, + 0, 360, 556, 538, 548, 539, 524, 525, 526, 533, + 340, 527, 528, 529, 499, 530, 500, 531, 532, 793, + 555, 506, 423, 375, 573, 572, 0, 0, 862, 870, + 0, 0, 0, 0, 0, 0, 0, 0, 858, 0, + 0, 0, 0, 746, 0, 0, 783, 838, 837, 770, + 780, 0, 0, 303, 223, 501, 621, 503, 502, 771, + 0, 772, 776, 779, 775, 773, 774, 0, 853, 0, + 0, 0, 0, 0, 0, 738, 750, 0, 755, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 747, 748, 0, 0, 0, 0, 803, 0, + 749, 0, 0, 798, 777, 781, 0, 0, 0, 0, + 293, 429, 447, 304, 418, 460, 309, 426, 299, 390, + 415, 0, 0, 295, 445, 425, 372, 350, 351, 294, + 0, 409, 328, 342, 325, 388, 778, 801, 805, 324, + 876, 799, 455, 297, 0, 454, 387, 441, 446, 373, + 367, 0, 296, 443, 371, 366, 354, 332, 877, 355, + 356, 346, 399, 364, 400, 347, 377, 376, 378, 0, + 0, 0, 0, 0, 483, 484, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 614, + 796, 0, 618, 0, 457, 0, 0, 860, 0, 0, + 0, 428, 0, 0, 357, 0, 0, 0, 800, 0, + 412, 393, 873, 0, 0, 410, 362, 442, 401, 448, + 430, 456, 406, 402, 288, 431, 327, 374, 300, 302, + 322, 329, 331, 333, 334, 383, 384, 396, 417, 433, + 434, 435, 326, 310, 411, 311, 344, 312, 289, 318, + 316, 319, 419, 320, 291, 397, 439, 0, 339, 407, + 370, 292, 369, 398, 438, 437, 301, 464, 470, 471, + 560, 0, 476, 645, 646, 647, 485, 0, 403, 490, + 491, 492, 494, 495, 496, 497, 561, 578, 545, 515, + 478, 569, 512, 516, 517, 581, 0, 0, 0, 469, + 358, 359, 0, 337, 285, 286, 640, 857, 389, 583, + 616, 617, 508, 0, 872, 852, 854, 855, 859, 863, + 864, 865, 866, 867, 869, 871, 875, 639, 0, 562, + 577, 643, 576, 636, 395, 0, 416, 574, 521, 0, + 566, 540, 0, 567, 536, 571, 0, 510, 0, 424, + 450, 462, 479, 482, 511, 596, 597, 598, 290, 481, + 600, 601, 602, 603, 604, 605, 606, 599, 874, 543, + 520, 546, 461, 523, 522, 0, 0, 557, 804, 558, + 559, 379, 380, 381, 382, 861, 584, 308, 480, 405, + 0, 544, 0, 0, 0, 0, 0, 0, 0, 0, + 549, 550, 547, 648, 0, 607, 608, 0, 0, 474, + 475, 336, 343, 493, 345, 307, 394, 338, 459, 352, + 0, 486, 551, 487, 610, 613, 611, 612, 386, 348, + 349, 420, 353, 363, 408, 458, 392, 413, 305, 449, + 422, 368, 537, 564, 883, 856, 882, 884, 885, 881, + 886, 887, 868, 759, 0, 811, 879, 878, 880, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 592, 591, 590, 589, 588, 587, 586, 585, 0, 0, + 534, 436, 317, 279, 313, 314, 321, 637, 634, 440, + 638, 766, 287, 514, 361, 0, 404, 335, 579, 580, + 0, 0, 845, 818, 819, 820, 756, 821, 815, 816, + 757, 817, 846, 809, 842, 843, 785, 812, 822, 841, + 823, 844, 847, 848, 888, 889, 829, 813, 251, 890, + 826, 849, 840, 839, 824, 810, 850, 851, 792, 787, + 827, 828, 814, 833, 834, 835, 758, 806, 807, 808, + 830, 831, 788, 789, 790, 791, 0, 0, 0, 465, + 466, 467, 489, 0, 451, 513, 635, 0, 0, 0, + 0, 0, 0, 0, 563, 575, 609, 0, 619, 620, + 622, 624, 836, 626, 427, 0, 832, 629, 630, 627, + 365, 414, 432, 421, 802, 641, 504, 505, 642, 615, + 0, 751, 0, 391, 0, 519, 552, 541, 625, 507, + 0, 0, 0, 0, 0, 0, 754, 0, 0, 0, + 330, 0, 0, 360, 556, 538, 548, 539, 524, 525, + 526, 533, 340, 527, 528, 529, 499, 530, 500, 531, + 532, 793, 555, 506, 423, 375, 573, 572, 0, 0, + 862, 870, 0, 0, 0, 0, 0, 0, 0, 0, + 858, 0, 0, 0, 0, 746, 0, 0, 783, 838, + 837, 770, 780, 0, 0, 303, 223, 501, 621, 503, + 502, 771, 0, 772, 776, 779, 775, 773, 774, 0, + 853, 0, 0, 0, 0, 0, 0, 738, 750, 0, + 755, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 293, 429, 447, 304, 418, - 460, 309, 426, 299, 390, 415, 0, 0, 295, 445, - 425, 372, 350, 351, 294, 0, 409, 328, 342, 325, - 388, 0, 444, 472, 324, 463, 0, 455, 297, 0, - 454, 387, 441, 446, 373, 367, 0, 296, 443, 371, - 366, 354, 332, 488, 355, 356, 346, 399, 364, 400, - 347, 377, 376, 378, 0, 0, 0, 0, 0, 483, - 484, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 614, 0, 0, 618, 0, 457, - 0, 0, 0, 0, 0, 0, 428, 0, 0, 357, - 0, 0, 0, 473, 0, 412, 393, 644, 0, 0, - 410, 362, 442, 401, 448, 430, 456, 406, 402, 288, - 431, 327, 374, 300, 302, 322, 329, 331, 333, 334, - 383, 384, 396, 417, 433, 434, 435, 326, 310, 411, - 311, 344, 312, 289, 318, 316, 319, 419, 320, 291, - 397, 439, 0, 339, 407, 370, 292, 369, 398, 438, - 437, 301, 464, 470, 471, 560, 0, 476, 645, 646, - 647, 485, 0, 403, 490, 491, 492, 494, 495, 496, - 497, 561, 578, 545, 515, 478, 569, 512, 516, 517, - 581, 0, 0, 0, 469, 358, 359, 0, 337, 285, - 286, 640, 323, 389, 583, 616, 617, 508, 0, 570, - 509, 518, 315, 542, 554, 553, 385, 468, 0, 565, - 568, 498, 639, 0, 562, 577, 643, 576, 636, 395, - 0, 416, 574, 521, 0, 566, 540, 0, 567, 536, - 571, 0, 510, 0, 424, 450, 462, 479, 482, 511, - 596, 597, 598, 290, 481, 600, 601, 602, 603, 604, - 605, 606, 599, 453, 543, 520, 546, 461, 523, 522, - 0, 0, 557, 477, 558, 559, 379, 380, 381, 382, - 341, 584, 308, 480, 405, 0, 544, 0, 0, 0, - 0, 0, 0, 0, 0, 549, 550, 547, 648, 0, - 607, 608, 0, 0, 474, 475, 336, 343, 493, 345, - 307, 394, 338, 459, 352, 0, 486, 551, 487, 610, - 613, 611, 612, 386, 348, 349, 420, 353, 363, 408, - 458, 392, 413, 305, 449, 422, 368, 537, 564, 0, + 0, 0, 0, 0, 747, 748, 0, 0, 0, 0, + 803, 0, 749, 0, 0, 798, 777, 781, 0, 0, + 0, 0, 293, 429, 447, 304, 418, 460, 309, 426, + 299, 390, 415, 0, 0, 295, 445, 425, 372, 350, + 351, 294, 0, 409, 328, 342, 325, 388, 778, 801, + 805, 324, 876, 799, 455, 297, 0, 454, 387, 441, + 446, 373, 367, 0, 296, 443, 371, 366, 354, 332, + 877, 355, 356, 346, 399, 364, 400, 347, 377, 376, + 378, 0, 0, 0, 0, 0, 483, 484, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 592, 591, 590, 589, 588, - 587, 586, 585, 0, 0, 534, 436, 317, 279, 313, - 314, 321, 637, 634, 440, 638, 0, 287, 514, 361, - 0, 404, 335, 579, 580, 0, 0, 235, 236, 237, - 238, 239, 240, 241, 242, 280, 243, 244, 245, 246, - 247, 248, 249, 252, 253, 254, 255, 256, 257, 258, - 259, 582, 250, 251, 260, 261, 262, 263, 264, 265, - 266, 267, 268, 269, 270, 271, 272, 273, 0, 0, - 0, 281, 282, 283, 284, 0, 0, 275, 276, 277, - 278, 0, 0, 0, 465, 466, 467, 489, 0, 451, - 513, 635, 0, 0, 0, 0, 0, 0, 0, 563, - 575, 609, 0, 619, 620, 622, 624, 623, 626, 427, - 0, 628, 629, 630, 627, 365, 414, 432, 421, 0, - 641, 504, 505, 642, 615, 391, 0, 519, 552, 541, - 625, 507, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 614, 796, 0, 618, 0, 457, 0, 0, 860, + 0, 0, 0, 428, 0, 0, 357, 0, 0, 0, + 800, 0, 412, 393, 873, 0, 0, 410, 362, 442, + 401, 448, 430, 456, 406, 402, 288, 431, 327, 374, + 300, 302, 322, 329, 331, 333, 334, 383, 384, 396, + 417, 433, 434, 435, 326, 310, 411, 311, 344, 312, + 289, 318, 316, 319, 419, 320, 291, 397, 439, 0, + 339, 407, 370, 292, 369, 398, 438, 437, 301, 464, + 470, 471, 560, 0, 476, 645, 646, 647, 485, 0, + 403, 490, 491, 492, 494, 495, 496, 497, 561, 578, + 545, 515, 478, 569, 512, 516, 517, 581, 0, 0, + 0, 469, 358, 359, 0, 337, 285, 286, 640, 857, + 389, 583, 616, 617, 508, 0, 872, 852, 854, 855, + 859, 863, 864, 865, 866, 867, 869, 871, 875, 639, + 0, 562, 577, 643, 576, 636, 395, 0, 416, 574, + 521, 0, 566, 540, 0, 567, 536, 571, 0, 510, + 0, 424, 450, 462, 479, 482, 511, 596, 597, 598, + 290, 481, 600, 601, 602, 603, 604, 605, 606, 599, + 874, 543, 520, 546, 461, 523, 522, 0, 0, 557, + 804, 558, 559, 379, 380, 381, 382, 861, 584, 308, + 480, 405, 0, 544, 0, 0, 0, 0, 0, 0, + 0, 0, 549, 550, 547, 648, 0, 607, 608, 0, + 0, 474, 475, 336, 343, 493, 345, 307, 394, 338, + 459, 352, 0, 486, 551, 487, 610, 613, 611, 612, + 386, 348, 349, 420, 353, 363, 408, 458, 392, 413, + 305, 449, 422, 368, 537, 564, 883, 856, 882, 884, + 885, 881, 886, 887, 868, 759, 0, 811, 879, 878, + 880, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 592, 591, 590, 589, 588, 587, 586, 585, + 0, 0, 534, 436, 317, 279, 313, 314, 321, 637, + 634, 440, 638, 766, 287, 514, 361, 0, 404, 335, + 579, 580, 0, 0, 845, 818, 819, 820, 756, 821, + 815, 816, 757, 817, 846, 809, 842, 843, 785, 812, + 822, 841, 823, 844, 847, 848, 888, 889, 829, 813, + 251, 890, 826, 849, 840, 839, 824, 810, 850, 851, + 792, 787, 827, 828, 814, 833, 834, 835, 758, 806, + 807, 808, 830, 831, 788, 789, 790, 791, 0, 0, + 0, 465, 466, 467, 489, 0, 451, 513, 635, 0, + 0, 0, 0, 0, 0, 0, 563, 575, 609, 0, + 619, 620, 622, 624, 836, 626, 427, 0, 3508, 629, + 3509, 3510, 365, 414, 432, 421, 802, 641, 504, 505, + 642, 615, 0, 751, 0, 391, 0, 519, 552, 541, + 625, 507, 0, 0, 0, 0, 0, 0, 754, 0, 0, 0, 330, 0, 0, 360, 556, 538, 548, 539, 524, 525, 526, 533, 340, 527, 528, 529, 499, 530, - 500, 531, 532, 0, 555, 506, 423, 375, 573, 572, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 222, - 0, 0, 0, 0, 0, 0, 303, 223, 501, 621, - 503, 502, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 306, 0, 0, 0, 0, 0, 0, 0, 0, + 500, 531, 532, 793, 555, 506, 423, 375, 573, 572, + 0, 0, 862, 870, 0, 0, 0, 0, 0, 0, + 0, 0, 858, 0, 0, 0, 0, 746, 0, 0, + 783, 838, 837, 770, 780, 0, 0, 303, 223, 501, + 621, 503, 502, 2709, 0, 2710, 776, 779, 775, 773, + 774, 0, 853, 0, 0, 0, 0, 0, 0, 738, + 750, 0, 755, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 747, 748, 0, 0, + 0, 0, 803, 0, 749, 0, 0, 798, 777, 781, + 0, 0, 0, 0, 293, 429, 447, 304, 418, 460, + 309, 426, 299, 390, 415, 0, 0, 295, 445, 425, + 372, 350, 351, 294, 0, 409, 328, 342, 325, 388, + 778, 801, 805, 324, 876, 799, 455, 297, 0, 454, + 387, 441, 446, 373, 367, 0, 296, 443, 371, 366, + 354, 332, 877, 355, 356, 346, 399, 364, 400, 347, + 377, 376, 378, 0, 0, 0, 0, 0, 483, 484, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 614, 796, 0, 618, 0, 457, 0, + 0, 860, 0, 0, 0, 428, 0, 0, 357, 0, + 0, 0, 800, 0, 412, 393, 873, 0, 0, 410, + 362, 442, 401, 448, 430, 456, 406, 402, 288, 431, + 327, 374, 300, 302, 322, 329, 331, 333, 334, 383, + 384, 396, 417, 433, 434, 435, 326, 310, 411, 311, + 344, 312, 289, 318, 316, 319, 419, 320, 291, 397, + 439, 0, 339, 407, 370, 292, 369, 398, 438, 437, + 301, 464, 470, 471, 560, 0, 476, 645, 646, 647, + 485, 0, 403, 490, 491, 492, 494, 495, 496, 497, + 561, 578, 545, 515, 478, 569, 512, 516, 517, 581, + 0, 0, 0, 469, 358, 359, 0, 337, 285, 286, + 640, 857, 389, 583, 616, 617, 508, 0, 872, 852, + 854, 855, 859, 863, 864, 865, 866, 867, 869, 871, + 875, 639, 0, 562, 577, 643, 576, 636, 395, 0, + 416, 574, 521, 0, 566, 540, 0, 567, 536, 571, + 0, 510, 0, 424, 450, 462, 479, 482, 511, 596, + 597, 598, 290, 481, 600, 601, 602, 603, 604, 605, + 606, 599, 874, 543, 520, 546, 461, 523, 522, 0, + 0, 557, 804, 558, 559, 379, 380, 381, 382, 861, + 584, 308, 480, 405, 0, 544, 0, 0, 0, 0, + 0, 0, 0, 0, 549, 550, 547, 648, 0, 607, + 608, 0, 0, 474, 475, 336, 343, 493, 345, 307, + 394, 338, 459, 352, 0, 486, 551, 487, 610, 613, + 611, 612, 386, 348, 349, 420, 353, 363, 408, 458, + 392, 413, 305, 449, 422, 368, 537, 564, 883, 856, + 882, 884, 885, 881, 886, 887, 868, 759, 0, 811, + 879, 878, 880, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 592, 591, 590, 589, 588, 587, + 586, 585, 0, 0, 534, 436, 317, 279, 313, 314, + 321, 637, 634, 440, 638, 766, 287, 514, 361, 0, + 404, 335, 579, 580, 0, 0, 845, 818, 819, 820, + 756, 821, 815, 816, 757, 817, 846, 809, 842, 843, + 785, 812, 822, 841, 823, 844, 847, 848, 888, 889, + 829, 813, 251, 890, 826, 849, 840, 839, 824, 810, + 850, 851, 792, 787, 827, 828, 814, 833, 834, 835, + 758, 806, 807, 808, 830, 831, 788, 789, 790, 791, + 0, 0, 0, 465, 466, 467, 489, 0, 451, 513, + 635, 0, 0, 0, 0, 0, 0, 0, 563, 575, + 609, 0, 619, 620, 622, 624, 836, 626, 427, 0, + 832, 629, 630, 627, 365, 414, 432, 421, 802, 641, + 504, 505, 642, 615, 0, 751, 0, 391, 0, 519, + 552, 541, 625, 507, 0, 0, 1705, 0, 0, 0, + 754, 0, 0, 0, 330, 0, 0, 360, 556, 538, + 548, 539, 524, 525, 526, 533, 340, 527, 528, 529, + 499, 530, 500, 531, 532, 793, 555, 506, 423, 375, + 573, 572, 0, 0, 862, 870, 0, 0, 0, 0, + 0, 0, 0, 0, 858, 0, 0, 0, 0, 746, + 0, 0, 783, 838, 837, 770, 780, 0, 0, 303, + 223, 501, 621, 503, 502, 771, 0, 772, 776, 779, + 775, 773, 774, 0, 853, 0, 0, 0, 0, 0, + 0, 0, 750, 0, 755, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 747, 748, + 0, 0, 0, 0, 803, 0, 749, 0, 0, 798, + 777, 781, 0, 0, 0, 0, 293, 429, 447, 304, + 418, 460, 309, 426, 299, 390, 415, 0, 0, 295, + 445, 425, 372, 350, 351, 294, 0, 409, 328, 342, + 325, 388, 778, 801, 805, 324, 876, 799, 455, 297, + 0, 454, 387, 441, 446, 373, 367, 0, 296, 443, + 371, 366, 354, 332, 877, 355, 356, 346, 399, 364, + 400, 347, 377, 376, 378, 0, 0, 0, 0, 0, + 483, 484, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 614, 796, 0, 618, 0, + 457, 0, 0, 860, 0, 0, 0, 428, 0, 0, + 357, 0, 0, 0, 800, 0, 412, 393, 873, 0, + 0, 410, 362, 442, 401, 448, 430, 456, 406, 402, + 288, 431, 327, 374, 300, 302, 322, 329, 331, 333, + 334, 383, 384, 396, 417, 433, 434, 435, 326, 310, + 411, 311, 344, 312, 289, 318, 316, 319, 419, 320, + 291, 397, 439, 0, 339, 407, 370, 292, 369, 398, + 438, 437, 301, 464, 1706, 1707, 560, 0, 476, 645, + 646, 647, 485, 0, 403, 490, 491, 492, 494, 495, + 496, 497, 561, 578, 545, 515, 478, 569, 512, 516, + 517, 581, 0, 0, 0, 469, 358, 359, 0, 337, + 285, 286, 640, 857, 389, 583, 616, 617, 508, 0, + 872, 852, 854, 855, 859, 863, 864, 865, 866, 867, + 869, 871, 875, 639, 0, 562, 577, 643, 576, 636, + 395, 0, 416, 574, 521, 0, 566, 540, 0, 567, + 536, 571, 0, 510, 0, 424, 450, 462, 479, 482, + 511, 596, 597, 598, 290, 481, 600, 601, 602, 603, + 604, 605, 606, 599, 874, 543, 520, 546, 461, 523, + 522, 0, 0, 557, 804, 558, 559, 379, 380, 381, + 382, 861, 584, 308, 480, 405, 0, 544, 0, 0, + 0, 0, 0, 0, 0, 0, 549, 550, 547, 648, + 0, 607, 608, 0, 0, 474, 475, 336, 343, 493, + 345, 307, 394, 338, 459, 352, 0, 486, 551, 487, + 610, 613, 611, 612, 386, 348, 349, 420, 353, 363, + 408, 458, 392, 413, 305, 449, 422, 368, 537, 564, + 883, 856, 882, 884, 885, 881, 886, 887, 868, 759, + 0, 811, 879, 878, 880, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 592, 591, 590, 589, + 588, 587, 586, 585, 0, 0, 534, 436, 317, 279, + 313, 314, 321, 637, 634, 440, 638, 766, 287, 514, + 361, 0, 404, 335, 579, 580, 0, 0, 845, 818, + 819, 820, 756, 821, 815, 816, 757, 817, 846, 809, + 842, 843, 785, 812, 822, 841, 823, 844, 847, 848, + 888, 889, 829, 813, 251, 890, 826, 849, 840, 839, + 824, 810, 850, 851, 792, 787, 827, 828, 814, 833, + 834, 835, 758, 806, 807, 808, 830, 831, 788, 789, + 790, 791, 0, 0, 0, 465, 466, 467, 489, 0, + 451, 513, 635, 0, 0, 0, 0, 0, 0, 0, + 563, 575, 609, 0, 619, 620, 622, 624, 836, 626, + 427, 0, 832, 629, 630, 627, 365, 414, 432, 421, + 802, 641, 504, 505, 642, 615, 0, 751, 0, 391, + 0, 519, 552, 541, 625, 507, 0, 0, 0, 0, + 0, 0, 754, 0, 0, 0, 330, 0, 0, 360, + 556, 538, 548, 539, 524, 525, 526, 533, 340, 527, + 528, 529, 499, 530, 500, 531, 532, 793, 555, 506, + 423, 375, 573, 572, 0, 0, 862, 870, 0, 0, + 0, 0, 0, 0, 0, 0, 858, 0, 0, 0, + 0, 746, 0, 0, 783, 838, 837, 770, 780, 0, + 0, 303, 223, 501, 621, 503, 502, 771, 0, 772, + 776, 779, 775, 773, 774, 0, 853, 0, 0, 0, + 0, 0, 0, 0, 750, 0, 755, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 293, 429, 447, 304, 418, 460, 309, - 426, 299, 390, 415, 0, 0, 295, 445, 425, 372, - 350, 351, 294, 0, 409, 328, 342, 325, 388, 0, - 444, 472, 324, 463, 0, 455, 297, 0, 454, 387, - 441, 446, 373, 367, 0, 296, 443, 371, 366, 354, - 332, 488, 355, 356, 346, 399, 364, 400, 347, 377, - 376, 378, 0, 0, 0, 0, 0, 483, 484, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 614, 0, 0, 618, 0, 457, 0, 0, - 0, 3867, 0, 0, 428, 0, 0, 357, 0, 0, - 0, 473, 0, 412, 393, 644, 0, 0, 410, 362, - 442, 401, 448, 430, 456, 406, 402, 288, 431, 327, - 374, 300, 302, 322, 329, 331, 333, 334, 383, 384, - 396, 417, 433, 434, 435, 326, 310, 411, 311, 344, - 312, 289, 318, 316, 319, 419, 320, 291, 397, 439, - 0, 339, 407, 370, 292, 369, 398, 438, 437, 301, - 464, 470, 471, 560, 0, 476, 645, 646, 647, 485, - 0, 403, 490, 491, 492, 494, 495, 496, 497, 561, - 578, 545, 515, 478, 569, 512, 516, 517, 581, 0, - 0, 0, 469, 358, 359, 0, 337, 285, 286, 640, - 323, 389, 583, 616, 617, 508, 0, 570, 509, 518, - 315, 542, 554, 553, 385, 468, 0, 565, 568, 498, - 639, 0, 562, 577, 643, 576, 636, 395, 0, 416, - 574, 521, 0, 566, 540, 0, 567, 536, 571, 0, - 510, 0, 424, 450, 462, 479, 482, 511, 596, 597, - 598, 290, 481, 600, 601, 602, 603, 604, 605, 606, - 599, 453, 543, 520, 546, 461, 523, 522, 0, 0, - 557, 477, 558, 559, 379, 380, 381, 382, 341, 584, - 308, 480, 405, 0, 544, 0, 0, 0, 0, 0, - 0, 0, 0, 549, 550, 547, 648, 0, 607, 608, - 0, 0, 474, 475, 336, 343, 493, 345, 307, 394, - 338, 459, 352, 0, 486, 551, 487, 610, 613, 611, - 612, 386, 348, 349, 420, 353, 363, 408, 458, 392, - 413, 305, 449, 422, 368, 537, 564, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 274, 0, + 747, 748, 0, 0, 0, 0, 803, 0, 749, 0, + 0, 798, 777, 781, 0, 0, 0, 0, 293, 429, + 447, 304, 418, 460, 309, 426, 299, 390, 415, 0, + 0, 295, 445, 425, 372, 350, 351, 294, 0, 409, + 328, 342, 325, 388, 778, 801, 805, 324, 876, 799, + 455, 297, 0, 454, 387, 441, 446, 373, 367, 0, + 296, 443, 371, 366, 354, 332, 877, 355, 356, 346, + 399, 364, 400, 347, 377, 376, 378, 0, 0, 0, + 0, 0, 483, 484, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 614, 796, 0, + 618, 0, 457, 0, 0, 860, 0, 0, 0, 428, + 0, 0, 357, 0, 0, 0, 800, 0, 412, 393, + 873, 0, 0, 410, 362, 442, 401, 448, 430, 456, + 406, 402, 288, 431, 327, 374, 300, 302, 322, 329, + 331, 333, 334, 383, 384, 396, 417, 433, 434, 435, + 326, 310, 411, 311, 344, 312, 289, 318, 316, 319, + 419, 320, 291, 397, 439, 0, 339, 407, 370, 292, + 369, 398, 438, 437, 301, 464, 470, 471, 560, 0, + 476, 645, 646, 647, 485, 0, 403, 490, 491, 492, + 494, 495, 496, 497, 561, 578, 545, 515, 478, 569, + 512, 516, 517, 581, 0, 0, 0, 469, 358, 359, + 0, 337, 285, 286, 640, 857, 389, 583, 616, 617, + 508, 0, 872, 852, 854, 855, 859, 863, 864, 865, + 866, 867, 869, 871, 875, 639, 0, 562, 577, 643, + 576, 636, 395, 0, 416, 574, 521, 0, 566, 540, + 0, 567, 536, 571, 0, 510, 0, 424, 450, 462, + 479, 482, 511, 596, 597, 598, 290, 481, 600, 601, + 602, 603, 604, 605, 606, 599, 874, 543, 520, 546, + 461, 523, 522, 0, 0, 557, 804, 558, 559, 379, + 380, 381, 382, 861, 584, 308, 480, 405, 0, 544, + 0, 0, 0, 0, 0, 0, 0, 0, 549, 550, + 547, 648, 0, 607, 608, 0, 0, 474, 475, 336, + 343, 493, 345, 307, 394, 338, 459, 352, 0, 486, + 551, 487, 610, 613, 611, 612, 386, 348, 349, 420, + 353, 363, 408, 458, 392, 413, 305, 449, 422, 368, + 537, 564, 883, 856, 882, 884, 885, 881, 886, 887, + 868, 759, 0, 811, 879, 878, 880, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 592, 591, + 590, 589, 588, 587, 586, 585, 0, 0, 534, 436, + 317, 279, 313, 314, 321, 637, 634, 440, 638, 766, + 287, 514, 361, 0, 404, 335, 579, 580, 0, 0, + 845, 818, 819, 820, 756, 821, 815, 816, 757, 817, + 846, 809, 842, 843, 785, 812, 822, 841, 823, 844, + 847, 848, 888, 889, 829, 813, 251, 890, 826, 849, + 840, 839, 824, 810, 850, 851, 792, 787, 827, 828, + 814, 833, 834, 835, 758, 806, 807, 808, 830, 831, + 788, 789, 790, 791, 0, 0, 0, 465, 466, 467, + 489, 0, 451, 513, 635, 0, 0, 0, 0, 0, + 0, 0, 563, 575, 609, 0, 619, 620, 622, 624, + 836, 626, 427, 0, 832, 629, 630, 627, 365, 414, + 432, 421, 802, 641, 504, 505, 642, 615, 0, 751, + 0, 391, 0, 519, 552, 541, 625, 507, 0, 0, + 0, 0, 0, 0, 754, 0, 0, 0, 330, 0, + 0, 360, 556, 538, 548, 539, 524, 525, 526, 533, + 340, 527, 528, 529, 499, 530, 500, 531, 532, 793, + 555, 506, 423, 375, 573, 572, 0, 0, 862, 870, + 0, 0, 0, 0, 0, 0, 0, 0, 858, 0, + 0, 0, 0, 0, 0, 0, 783, 838, 837, 770, + 780, 0, 0, 303, 223, 501, 621, 503, 502, 771, + 0, 772, 776, 779, 775, 773, 774, 0, 853, 0, + 0, 0, 0, 0, 0, 738, 750, 0, 755, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 592, 591, 590, 589, 588, 587, 586, - 585, 0, 0, 534, 436, 317, 279, 313, 314, 321, - 637, 634, 440, 638, 0, 287, 514, 361, 0, 404, - 335, 579, 580, 0, 0, 235, 236, 237, 238, 239, - 240, 241, 242, 280, 243, 244, 245, 246, 247, 248, - 249, 252, 253, 254, 255, 256, 257, 258, 259, 582, - 250, 251, 260, 261, 262, 263, 264, 265, 266, 267, - 268, 269, 270, 271, 272, 273, 0, 0, 0, 281, - 282, 283, 284, 0, 0, 275, 276, 277, 278, 0, - 0, 0, 465, 466, 467, 489, 0, 451, 513, 635, - 0, 0, 0, 0, 0, 0, 0, 563, 575, 609, - 0, 619, 620, 622, 624, 623, 626, 427, 0, 628, - 629, 630, 627, 365, 414, 432, 421, 0, 641, 504, - 505, 642, 615, 391, 0, 519, 552, 541, 625, 507, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 330, 0, 0, 360, 556, 538, 548, 539, 524, 525, - 526, 533, 340, 527, 528, 529, 499, 530, 500, 531, - 532, 0, 555, 506, 423, 375, 573, 572, 0, 0, + 0, 0, 747, 748, 0, 0, 0, 0, 803, 0, + 749, 0, 0, 798, 777, 781, 0, 0, 0, 0, + 293, 429, 447, 304, 418, 460, 309, 426, 299, 390, + 415, 0, 0, 295, 445, 425, 372, 350, 351, 294, + 0, 409, 328, 342, 325, 388, 778, 801, 805, 324, + 876, 799, 455, 297, 0, 454, 387, 441, 446, 373, + 367, 0, 296, 443, 371, 366, 354, 332, 877, 355, + 356, 346, 399, 364, 400, 347, 377, 376, 378, 0, + 0, 0, 0, 0, 483, 484, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 614, + 796, 0, 618, 0, 457, 0, 0, 860, 0, 0, + 0, 428, 0, 0, 357, 0, 0, 0, 800, 0, + 412, 393, 873, 0, 0, 410, 362, 442, 401, 448, + 430, 456, 406, 402, 288, 431, 327, 374, 300, 302, + 322, 329, 331, 333, 334, 383, 384, 396, 417, 433, + 434, 435, 326, 310, 411, 311, 344, 312, 289, 318, + 316, 319, 419, 320, 291, 397, 439, 0, 339, 407, + 370, 292, 369, 398, 438, 437, 301, 464, 470, 471, + 560, 0, 476, 645, 646, 647, 485, 0, 403, 490, + 491, 492, 494, 495, 496, 497, 561, 578, 545, 515, + 478, 569, 512, 516, 517, 581, 0, 0, 0, 469, + 358, 359, 0, 337, 285, 286, 640, 857, 389, 583, + 616, 617, 508, 0, 872, 852, 854, 855, 859, 863, + 864, 865, 866, 867, 869, 871, 875, 639, 0, 562, + 577, 643, 576, 636, 395, 0, 416, 574, 521, 0, + 566, 540, 0, 567, 536, 571, 0, 510, 0, 424, + 450, 462, 479, 482, 511, 596, 597, 598, 290, 481, + 600, 601, 602, 603, 604, 605, 606, 599, 874, 543, + 520, 546, 461, 523, 522, 0, 0, 557, 804, 558, + 559, 379, 380, 381, 382, 861, 584, 308, 480, 405, + 0, 544, 0, 0, 0, 0, 0, 0, 0, 0, + 549, 550, 547, 648, 0, 607, 608, 0, 0, 474, + 475, 336, 343, 493, 345, 307, 394, 338, 459, 352, + 0, 486, 551, 487, 610, 613, 611, 612, 386, 348, + 349, 420, 353, 363, 408, 458, 392, 413, 305, 449, + 422, 368, 537, 564, 883, 856, 882, 884, 885, 881, + 886, 887, 868, 759, 0, 811, 879, 878, 880, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1748, 0, 0, 222, 0, 0, + 592, 591, 590, 589, 588, 587, 586, 585, 0, 0, + 534, 436, 317, 279, 313, 314, 321, 637, 634, 440, + 638, 766, 287, 514, 361, 0, 404, 335, 579, 580, + 0, 0, 845, 818, 819, 820, 756, 821, 815, 816, + 757, 817, 846, 809, 842, 843, 785, 812, 822, 841, + 823, 844, 847, 848, 888, 889, 829, 813, 251, 890, + 826, 849, 840, 839, 824, 810, 850, 851, 792, 787, + 827, 828, 814, 833, 834, 835, 758, 806, 807, 808, + 830, 831, 788, 789, 790, 791, 0, 0, 0, 465, + 466, 467, 489, 0, 451, 513, 635, 0, 0, 0, + 0, 0, 0, 0, 563, 575, 609, 0, 619, 620, + 622, 624, 836, 626, 427, 0, 832, 629, 630, 627, + 365, 414, 432, 421, 0, 641, 504, 505, 642, 615, + 0, 751, 200, 61, 191, 162, 0, 0, 0, 0, + 0, 0, 391, 0, 519, 552, 541, 625, 507, 0, + 192, 0, 0, 0, 0, 0, 0, 183, 0, 330, + 0, 193, 360, 556, 538, 548, 539, 524, 525, 526, + 533, 340, 527, 528, 529, 499, 530, 500, 531, 532, + 136, 555, 506, 423, 375, 573, 572, 0, 0, 0, + 0, 0, 0, 0, 0, 122, 0, 0, 0, 0, + 0, 0, 0, 0, 196, 0, 0, 222, 0, 0, 0, 0, 0, 0, 303, 223, 501, 621, 503, 502, 0, 0, 0, 0, 0, 0, 0, 0, 0, 306, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 214, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -5170,65 +3305,129 @@ var yyAct = [...]int{ 373, 367, 0, 296, 443, 371, 366, 354, 332, 488, 355, 356, 346, 399, 364, 400, 347, 377, 376, 378, 0, 0, 0, 0, 0, 483, 484, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 614, 0, 0, 618, 0, 457, 0, 0, 0, 0, - 0, 0, 428, 0, 0, 357, 0, 0, 0, 473, - 0, 412, 393, 644, 0, 0, 410, 362, 442, 401, + 0, 0, 0, 161, 189, 198, 190, 120, 0, 0, + 614, 0, 0, 618, 0, 457, 0, 0, 215, 0, + 0, 0, 428, 0, 0, 357, 188, 182, 181, 473, + 0, 412, 393, 227, 0, 0, 410, 362, 442, 401, 448, 430, 456, 406, 402, 288, 431, 327, 374, 300, 302, 322, 329, 331, 333, 334, 383, 384, 396, 417, 433, 434, 435, 326, 310, 411, 311, 344, 312, 289, 318, 316, 319, 419, 320, 291, 397, 439, 0, 339, 407, 370, 292, 369, 398, 438, 437, 301, 464, 470, - 471, 560, 0, 476, 645, 646, 647, 485, 0, 403, + 471, 560, 0, 476, 593, 594, 595, 485, 0, 403, 490, 491, 492, 494, 495, 496, 497, 561, 578, 545, 515, 478, 569, 512, 516, 517, 581, 0, 0, 0, - 469, 358, 359, 0, 337, 285, 286, 640, 323, 389, + 469, 358, 359, 0, 337, 285, 286, 452, 323, 389, 583, 616, 617, 508, 0, 570, 509, 518, 315, 542, - 554, 553, 385, 468, 0, 565, 568, 498, 639, 0, - 562, 577, 643, 576, 636, 395, 0, 416, 574, 521, + 554, 553, 385, 468, 218, 565, 568, 498, 228, 0, + 562, 577, 535, 576, 229, 395, 0, 416, 574, 521, 0, 566, 540, 0, 567, 536, 571, 0, 510, 0, 424, 450, 462, 479, 482, 511, 596, 597, 598, 290, 481, 600, 601, 602, 603, 604, 605, 606, 599, 453, 543, 520, 546, 461, 523, 522, 0, 0, 557, 477, 558, 559, 379, 380, 381, 382, 341, 584, 308, 480, - 405, 0, 544, 0, 0, 0, 0, 0, 0, 0, - 0, 549, 550, 547, 648, 0, 607, 608, 0, 0, + 405, 134, 544, 0, 0, 0, 0, 0, 0, 0, + 0, 549, 550, 547, 226, 0, 607, 608, 0, 0, 474, 475, 336, 343, 493, 345, 307, 394, 338, 459, 352, 0, 486, 551, 487, 610, 613, 611, 612, 386, 348, 349, 420, 353, 363, 408, 458, 392, 413, 305, 449, 422, 368, 537, 564, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, + 0, 0, 0, 62, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 592, 591, 590, 589, 588, 587, 586, 585, 0, - 0, 534, 436, 317, 279, 313, 314, 321, 637, 634, - 440, 638, 0, 287, 514, 361, 0, 404, 335, 579, - 580, 0, 0, 235, 236, 237, 238, 239, 240, 241, + 0, 534, 436, 317, 279, 313, 314, 321, 233, 298, + 440, 234, 0, 287, 514, 361, 163, 404, 335, 579, + 580, 58, 0, 235, 236, 237, 238, 239, 240, 241, 242, 280, 243, 244, 245, 246, 247, 248, 249, 252, 253, 254, 255, 256, 257, 258, 259, 582, 250, 251, 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, 273, 0, 0, 0, 281, 282, 283, 284, 0, 0, 275, 276, 277, 278, 0, 0, 0, - 465, 466, 467, 489, 0, 451, 513, 635, 0, 0, - 0, 0, 0, 0, 0, 563, 575, 609, 0, 619, + 465, 466, 467, 489, 0, 451, 513, 230, 45, 216, + 219, 221, 220, 0, 59, 563, 575, 609, 5, 619, 620, 622, 624, 623, 626, 427, 0, 628, 629, 630, - 627, 365, 414, 432, 421, 0, 641, 504, 505, 642, - 615, 391, 0, 519, 552, 541, 625, 507, 0, 0, + 627, 365, 414, 432, 421, 139, 231, 504, 505, 232, + 615, 200, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 391, 0, 519, 552, 541, 625, 507, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 330, 0, 0, 360, 556, 538, 548, 539, 524, 525, 526, 533, - 340, 527, 528, 529, 499, 530, 500, 531, 532, 0, + 340, 527, 528, 529, 499, 530, 500, 531, 532, 136, 555, 506, 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 3749, 0, 222, 0, 0, 0, 0, - 0, 0, 303, 223, 501, 621, 503, 502, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 306, 0, 0, + 0, 0, 0, 196, 0, 0, 222, 0, 0, 0, + 0, 0, 0, 303, 223, 501, 621, 503, 502, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 306, 2384, + 2387, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 293, 429, 447, 304, 418, 460, 309, 426, 299, 390, + 415, 0, 0, 295, 445, 425, 372, 350, 351, 294, + 0, 409, 328, 342, 325, 388, 0, 444, 472, 324, + 463, 0, 455, 297, 0, 454, 387, 441, 446, 373, + 367, 0, 296, 443, 371, 366, 354, 332, 488, 355, + 356, 346, 399, 364, 400, 347, 377, 376, 378, 0, + 0, 0, 0, 0, 483, 484, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 614, + 0, 0, 618, 2388, 457, 0, 0, 0, 2383, 0, + 2382, 428, 2380, 2385, 357, 0, 0, 0, 473, 0, + 412, 393, 644, 0, 0, 410, 362, 442, 401, 448, + 430, 456, 406, 402, 288, 431, 327, 374, 300, 302, + 322, 329, 331, 333, 334, 383, 384, 396, 417, 433, + 434, 435, 326, 310, 411, 311, 344, 312, 289, 318, + 316, 319, 419, 320, 291, 397, 439, 2386, 339, 407, + 370, 292, 369, 398, 438, 437, 301, 464, 470, 471, + 560, 0, 476, 645, 646, 647, 485, 0, 403, 490, + 491, 492, 494, 495, 496, 497, 561, 578, 545, 515, + 478, 569, 512, 516, 517, 581, 0, 0, 0, 469, + 358, 359, 0, 337, 285, 286, 640, 323, 389, 583, + 616, 617, 508, 0, 570, 509, 518, 315, 542, 554, + 553, 385, 468, 0, 565, 568, 498, 639, 0, 562, + 577, 643, 576, 636, 395, 0, 416, 574, 521, 0, + 566, 540, 0, 567, 536, 571, 0, 510, 0, 424, + 450, 462, 479, 482, 511, 596, 597, 598, 290, 481, + 600, 601, 602, 603, 604, 605, 606, 599, 453, 543, + 520, 546, 461, 523, 522, 0, 0, 557, 477, 558, + 559, 379, 380, 381, 382, 341, 584, 308, 480, 405, + 0, 544, 0, 0, 0, 0, 0, 0, 0, 0, + 549, 550, 547, 648, 0, 607, 608, 0, 0, 474, + 475, 336, 343, 493, 345, 307, 394, 338, 459, 352, + 0, 486, 551, 487, 610, 613, 611, 612, 386, 348, + 349, 420, 353, 363, 408, 458, 392, 413, 305, 449, + 422, 368, 537, 564, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 592, 591, 590, 589, 588, 587, 586, 585, 0, 0, + 534, 436, 317, 279, 313, 314, 321, 637, 634, 440, + 638, 0, 287, 514, 361, 163, 404, 335, 579, 580, + 0, 0, 235, 236, 237, 238, 239, 240, 241, 242, + 280, 243, 244, 245, 246, 247, 248, 249, 252, 253, + 254, 255, 256, 257, 258, 259, 582, 250, 251, 260, + 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, + 271, 272, 273, 0, 0, 0, 281, 282, 283, 284, + 0, 0, 275, 276, 277, 278, 0, 0, 0, 465, + 466, 467, 489, 0, 451, 513, 635, 0, 0, 0, + 0, 0, 0, 0, 563, 575, 609, 0, 619, 620, + 622, 624, 623, 626, 427, 0, 628, 629, 630, 627, + 365, 414, 432, 421, 0, 641, 504, 505, 642, 615, + 391, 0, 519, 552, 541, 625, 507, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 330, 0, 0, + 360, 556, 538, 548, 539, 524, 525, 526, 533, 340, + 527, 528, 529, 499, 530, 500, 531, 532, 0, 555, + 506, 423, 375, 573, 572, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 1307, 0, 0, 222, 0, 0, 770, 780, + 0, 0, 303, 223, 501, 621, 503, 502, 771, 0, + 772, 776, 779, 775, 773, 774, 0, 306, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 293, + 0, 0, 0, 777, 0, 0, 0, 0, 0, 293, 429, 447, 304, 418, 460, 309, 426, 299, 390, 415, 0, 0, 295, 445, 425, 372, 350, 351, 294, 0, - 409, 328, 342, 325, 388, 0, 444, 472, 324, 463, + 409, 328, 342, 325, 388, 778, 444, 472, 324, 463, 0, 455, 297, 0, 454, 387, 441, 446, 373, 367, 0, 296, 443, 371, 366, 354, 332, 488, 355, 356, 346, 399, 364, 400, 347, 377, 376, 378, 0, 0, @@ -5274,12 +3473,76 @@ var yyAct = [...]int{ 467, 489, 0, 451, 513, 635, 0, 0, 0, 0, 0, 0, 0, 563, 575, 609, 0, 619, 620, 622, 624, 623, 626, 427, 0, 628, 629, 630, 627, 365, - 414, 432, 421, 0, 641, 504, 505, 642, 615, 391, - 0, 519, 552, 541, 625, 507, 0, 0, 0, 0, + 414, 432, 421, 0, 641, 504, 505, 642, 615, 200, + 61, 191, 162, 0, 0, 0, 0, 0, 0, 391, + 667, 519, 552, 541, 625, 507, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 330, 0, 0, 360, 556, 538, 548, 539, 524, 525, 526, 533, 340, 527, 528, 529, 499, 530, 500, 531, 532, 0, 555, 506, 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 674, 0, 0, 0, 0, 0, 0, + 0, 673, 0, 0, 222, 0, 0, 0, 0, 0, + 0, 303, 223, 501, 621, 503, 502, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 306, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 293, 429, + 447, 304, 418, 460, 309, 426, 299, 390, 415, 0, + 0, 295, 445, 425, 372, 350, 351, 294, 0, 409, + 328, 342, 325, 388, 0, 444, 472, 324, 463, 0, + 455, 297, 0, 454, 387, 441, 446, 373, 367, 0, + 296, 443, 371, 366, 354, 332, 488, 355, 356, 346, + 399, 364, 400, 347, 377, 376, 378, 0, 0, 0, + 0, 0, 483, 484, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 671, 672, 0, 614, 0, 0, + 618, 0, 457, 0, 0, 0, 0, 0, 0, 428, + 0, 0, 357, 0, 0, 0, 473, 0, 412, 393, + 644, 0, 0, 410, 362, 442, 401, 448, 430, 456, + 406, 402, 288, 431, 327, 374, 300, 302, 322, 329, + 331, 333, 334, 383, 384, 396, 417, 433, 434, 435, + 326, 310, 411, 311, 344, 312, 289, 318, 316, 319, + 419, 320, 291, 397, 439, 0, 339, 407, 370, 292, + 369, 398, 438, 437, 301, 464, 470, 471, 560, 0, + 476, 645, 646, 647, 485, 0, 403, 490, 491, 492, + 494, 495, 496, 497, 561, 578, 545, 515, 478, 569, + 512, 516, 517, 581, 0, 0, 0, 469, 358, 359, + 0, 337, 285, 286, 640, 323, 389, 583, 616, 617, + 508, 0, 570, 509, 518, 315, 542, 554, 553, 385, + 468, 0, 565, 568, 498, 639, 0, 562, 577, 643, + 576, 636, 395, 0, 416, 574, 521, 0, 566, 540, + 0, 567, 536, 571, 0, 510, 0, 424, 450, 462, + 479, 482, 511, 596, 597, 598, 290, 481, 600, 601, + 602, 603, 604, 605, 606, 599, 453, 543, 520, 546, + 461, 523, 522, 0, 0, 557, 477, 558, 559, 379, + 380, 381, 382, 668, 670, 308, 480, 405, 682, 544, + 0, 0, 0, 0, 0, 0, 0, 0, 549, 550, + 547, 648, 0, 607, 608, 0, 0, 474, 475, 336, + 343, 493, 345, 307, 394, 338, 459, 352, 0, 486, + 551, 487, 610, 613, 611, 612, 386, 348, 349, 420, + 353, 363, 408, 458, 392, 413, 305, 449, 422, 368, + 537, 564, 0, 0, 0, 0, 0, 0, 0, 0, + 62, 0, 0, 274, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 592, 591, + 590, 589, 588, 587, 586, 585, 0, 0, 534, 436, + 317, 279, 313, 314, 321, 637, 634, 440, 638, 0, + 287, 514, 361, 163, 404, 335, 579, 580, 0, 0, + 235, 236, 237, 238, 239, 240, 241, 242, 280, 243, + 244, 245, 246, 247, 248, 249, 252, 253, 254, 255, + 256, 257, 258, 259, 582, 250, 251, 260, 261, 262, + 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, + 273, 0, 0, 0, 281, 282, 283, 284, 0, 0, + 275, 276, 277, 278, 0, 0, 0, 465, 466, 467, + 489, 0, 451, 513, 635, 0, 0, 0, 0, 0, + 0, 0, 563, 575, 609, 0, 619, 620, 622, 624, + 623, 626, 427, 0, 628, 629, 630, 627, 365, 414, + 432, 421, 0, 641, 504, 505, 642, 615, 391, 0, + 519, 552, 541, 625, 507, 0, 1115, 0, 0, 0, + 0, 0, 0, 0, 0, 330, 0, 0, 360, 556, + 538, 548, 539, 524, 525, 526, 533, 340, 527, 528, + 529, 499, 530, 500, 531, 532, 0, 555, 506, 423, + 375, 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 222, 0, 0, 0, 0, 0, 0, 303, 223, 501, 621, 503, 502, 0, 0, 0, 0, @@ -5288,17 +3551,17 @@ var yyAct = [...]int{ 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 293, 429, 447, + 1100, 0, 0, 0, 0, 0, 0, 293, 429, 447, 304, 418, 460, 309, 426, 299, 390, 415, 0, 0, - 295, 445, 425, 372, 350, 351, 294, 0, 409, 328, - 342, 325, 388, 0, 444, 472, 324, 463, 0, 455, - 297, 0, 454, 387, 441, 446, 373, 367, 0, 296, - 443, 371, 366, 354, 332, 488, 355, 356, 346, 399, - 364, 400, 347, 377, 376, 378, 0, 0, 0, 0, - 0, 483, 484, 0, 0, 0, 0, 0, 0, 0, + 2547, 2550, 2551, 2552, 2553, 2554, 2555, 0, 2560, 2556, + 2557, 2558, 2559, 0, 2542, 2543, 2544, 2545, 1098, 2526, + 2548, 0, 2527, 387, 2528, 2529, 2530, 2531, 1102, 2532, + 2533, 2534, 2535, 2536, 2539, 2540, 2537, 2538, 2546, 399, + 364, 400, 347, 377, 376, 378, 1126, 1128, 1130, 1132, + 1135, 483, 484, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 614, 0, 0, 618, - 0, 457, 0, 0, 0, 3661, 0, 0, 428, 0, - 0, 357, 0, 0, 0, 473, 0, 412, 393, 644, + 0, 457, 0, 0, 0, 0, 0, 0, 428, 0, + 0, 357, 0, 0, 0, 2541, 0, 412, 393, 644, 0, 0, 410, 362, 442, 401, 448, 430, 456, 406, 402, 288, 431, 327, 374, 300, 302, 322, 329, 331, 333, 334, 383, 384, 396, 417, 433, 434, 435, 326, @@ -5327,7 +3590,7 @@ var yyAct = [...]int{ 0, 0, 0, 0, 0, 0, 0, 592, 591, 590, 589, 588, 587, 586, 585, 0, 0, 534, 436, 317, 279, 313, 314, 321, 637, 634, 440, 638, 0, 287, - 514, 361, 0, 404, 335, 579, 580, 0, 0, 235, + 2549, 361, 0, 404, 335, 579, 580, 0, 0, 235, 236, 237, 238, 239, 240, 241, 242, 280, 243, 244, 245, 246, 247, 248, 249, 252, 253, 254, 255, 256, 257, 258, 259, 582, 250, 251, 260, 261, 262, 263, @@ -5344,9 +3607,72 @@ var yyAct = [...]int{ 499, 530, 500, 531, 532, 0, 555, 506, 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 222, 0, 0, 3173, 0, 0, 0, 303, 223, + 0, 0, 222, 0, 0, 0, 0, 0, 0, 303, + 223, 501, 621, 503, 502, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 306, 2384, 2387, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 293, 429, 447, 304, + 418, 460, 309, 426, 299, 390, 415, 0, 0, 295, + 445, 425, 372, 350, 351, 294, 0, 409, 328, 342, + 325, 388, 0, 444, 472, 324, 463, 0, 455, 297, + 0, 454, 387, 441, 446, 373, 367, 0, 296, 443, + 371, 366, 354, 332, 488, 355, 356, 346, 399, 364, + 400, 347, 377, 376, 378, 0, 0, 0, 0, 0, + 483, 484, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 614, 0, 0, 618, 2388, + 457, 0, 0, 0, 2383, 0, 2382, 428, 2380, 2385, + 357, 0, 0, 0, 473, 0, 412, 393, 644, 0, + 0, 410, 362, 442, 401, 448, 430, 456, 406, 402, + 288, 431, 327, 374, 300, 302, 322, 329, 331, 333, + 334, 383, 384, 396, 417, 433, 434, 435, 326, 310, + 411, 311, 344, 312, 289, 318, 316, 319, 419, 320, + 291, 397, 439, 2386, 339, 407, 370, 292, 369, 398, + 438, 437, 301, 464, 470, 471, 560, 0, 476, 645, + 646, 647, 485, 0, 403, 490, 491, 492, 494, 495, + 496, 497, 561, 578, 545, 515, 478, 569, 512, 516, + 517, 581, 0, 0, 0, 469, 358, 359, 0, 337, + 285, 286, 640, 323, 389, 583, 616, 617, 508, 0, + 570, 509, 518, 315, 542, 554, 553, 385, 468, 0, + 565, 568, 498, 639, 0, 562, 577, 643, 576, 636, + 395, 0, 416, 574, 521, 0, 566, 540, 0, 567, + 536, 571, 0, 510, 0, 424, 450, 462, 479, 482, + 511, 596, 597, 598, 290, 481, 600, 601, 602, 603, + 604, 605, 606, 599, 453, 543, 520, 546, 461, 523, + 522, 0, 0, 557, 477, 558, 559, 379, 380, 381, + 382, 341, 584, 308, 480, 405, 0, 544, 0, 0, + 0, 0, 0, 0, 0, 0, 549, 550, 547, 648, + 0, 607, 608, 0, 0, 474, 475, 336, 343, 493, + 345, 307, 394, 338, 459, 352, 0, 486, 551, 487, + 610, 613, 611, 612, 386, 348, 349, 420, 353, 363, + 408, 458, 392, 413, 305, 449, 422, 368, 537, 564, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 592, 591, 590, 589, + 588, 587, 586, 585, 0, 0, 534, 436, 317, 279, + 313, 314, 321, 637, 634, 440, 638, 0, 287, 514, + 361, 0, 404, 335, 579, 580, 0, 0, 235, 236, + 237, 238, 239, 240, 241, 242, 280, 243, 244, 245, + 246, 247, 248, 249, 252, 253, 254, 255, 256, 257, + 258, 259, 582, 250, 251, 260, 261, 262, 263, 264, + 265, 266, 267, 268, 269, 270, 271, 272, 273, 0, + 0, 0, 281, 282, 283, 284, 0, 0, 275, 276, + 277, 278, 0, 0, 0, 465, 466, 467, 489, 0, + 451, 513, 635, 0, 0, 0, 0, 0, 0, 0, + 563, 575, 609, 0, 619, 620, 622, 624, 623, 626, + 427, 0, 628, 629, 630, 627, 365, 414, 432, 421, + 0, 641, 504, 505, 642, 615, 391, 0, 519, 552, + 541, 625, 507, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 330, 0, 0, 360, 556, 538, 548, + 539, 524, 525, 526, 533, 340, 527, 528, 529, 499, + 530, 500, 531, 532, 0, 555, 506, 423, 375, 573, + 572, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 222, 0, 0, 0, 0, 0, 0, 303, 223, 501, 621, 503, 502, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 306, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 306, 0, 2405, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -5359,9 +3685,9 @@ var yyAct = [...]int{ 366, 354, 332, 488, 355, 356, 346, 399, 364, 400, 347, 377, 376, 378, 0, 0, 0, 0, 0, 483, 484, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 614, 0, 0, 618, 0, 457, - 0, 0, 0, 0, 0, 0, 428, 0, 0, 357, - 0, 0, 0, 473, 0, 412, 393, 644, 0, 0, + 0, 0, 0, 0, 614, 0, 0, 618, 2404, 457, + 0, 0, 0, 2410, 2407, 2409, 428, 0, 2408, 357, + 0, 0, 0, 473, 0, 412, 393, 644, 0, 2402, 410, 362, 442, 401, 448, 430, 456, 406, 402, 288, 431, 327, 374, 300, 302, 322, 329, 331, 333, 334, 383, 384, 396, 417, 433, 434, 435, 326, 310, 411, @@ -5406,13 +3732,76 @@ var yyAct = [...]int{ 524, 525, 526, 533, 340, 527, 528, 529, 499, 530, 500, 531, 532, 0, 555, 506, 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 222, 0, 0, 0, 0, 0, 0, 303, 223, 501, + 621, 503, 502, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 306, 0, 2405, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 293, 429, 447, 304, 418, 460, + 309, 426, 299, 390, 415, 0, 0, 295, 445, 425, + 372, 350, 351, 294, 0, 409, 328, 342, 325, 388, + 0, 444, 472, 324, 463, 0, 455, 297, 0, 454, + 387, 441, 446, 373, 367, 0, 296, 443, 371, 366, + 354, 332, 488, 355, 356, 346, 399, 364, 400, 347, + 377, 376, 378, 0, 0, 0, 0, 0, 483, 484, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 614, 0, 0, 618, 2404, 457, 0, + 0, 0, 2410, 2407, 2409, 428, 0, 2408, 357, 0, + 0, 0, 473, 0, 412, 393, 644, 0, 0, 410, + 362, 442, 401, 448, 430, 456, 406, 402, 288, 431, + 327, 374, 300, 302, 322, 329, 331, 333, 334, 383, + 384, 396, 417, 433, 434, 435, 326, 310, 411, 311, + 344, 312, 289, 318, 316, 319, 419, 320, 291, 397, + 439, 0, 339, 407, 370, 292, 369, 398, 438, 437, + 301, 464, 470, 471, 560, 0, 476, 645, 646, 647, + 485, 0, 403, 490, 491, 492, 494, 495, 496, 497, + 561, 578, 545, 515, 478, 569, 512, 516, 517, 581, + 0, 0, 0, 469, 358, 359, 0, 337, 285, 286, + 640, 323, 389, 583, 616, 617, 508, 0, 570, 509, + 518, 315, 542, 554, 553, 385, 468, 0, 565, 568, + 498, 639, 0, 562, 577, 643, 576, 636, 395, 0, + 416, 574, 521, 0, 566, 540, 0, 567, 536, 571, + 0, 510, 0, 424, 450, 462, 479, 482, 511, 596, + 597, 598, 290, 481, 600, 601, 602, 603, 604, 605, + 606, 599, 453, 543, 520, 546, 461, 523, 522, 0, + 0, 557, 477, 558, 559, 379, 380, 381, 382, 341, + 584, 308, 480, 405, 0, 544, 0, 0, 0, 0, + 0, 0, 0, 0, 549, 550, 547, 648, 0, 607, + 608, 0, 0, 474, 475, 336, 343, 493, 345, 307, + 394, 338, 459, 352, 0, 486, 551, 487, 610, 613, + 611, 612, 386, 348, 349, 420, 353, 363, 408, 458, + 392, 413, 305, 449, 422, 368, 537, 564, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 274, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 592, 591, 590, 589, 588, 587, + 586, 585, 0, 0, 534, 436, 317, 279, 313, 314, + 321, 637, 634, 440, 638, 0, 287, 514, 361, 0, + 404, 335, 579, 580, 0, 0, 235, 236, 237, 238, + 239, 240, 241, 242, 280, 243, 244, 245, 246, 247, + 248, 249, 252, 253, 254, 255, 256, 257, 258, 259, + 582, 250, 251, 260, 261, 262, 263, 264, 265, 266, + 267, 268, 269, 270, 271, 272, 273, 0, 0, 0, + 281, 282, 283, 284, 0, 0, 275, 276, 277, 278, + 0, 0, 0, 465, 466, 467, 489, 0, 451, 513, + 635, 0, 0, 0, 0, 0, 0, 0, 563, 575, + 609, 0, 619, 620, 622, 624, 623, 626, 427, 0, + 628, 629, 630, 627, 365, 414, 432, 421, 0, 641, + 504, 505, 642, 615, 391, 0, 519, 552, 541, 625, + 507, 0, 0, 0, 0, 0, 2100, 0, 0, 0, + 0, 330, 0, 0, 360, 556, 538, 548, 539, 524, + 525, 526, 533, 340, 527, 528, 529, 499, 530, 500, + 531, 532, 0, 555, 506, 423, 375, 573, 572, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 222, - 0, 0, 0, 0, 0, 0, 303, 223, 501, 621, + 0, 0, 2101, 0, 0, 0, 303, 223, 501, 621, 503, 502, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 306, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 306, 0, 0, 1236, 1237, 1238, 1235, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 3191, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 293, 429, 447, 304, 418, 460, 309, 426, 299, 390, 415, 0, 0, 295, 445, 425, 372, @@ -5462,14 +3851,204 @@ var yyAct = [...]int{ 0, 0, 465, 466, 467, 489, 0, 451, 513, 635, 0, 0, 0, 0, 0, 0, 0, 563, 575, 609, 0, 619, 620, 622, 624, 623, 626, 427, 0, 628, - 629, 630, 627, 365, 414, 432, 421, 0, 641, 504, - 505, 642, 615, 391, 0, 519, 552, 541, 625, 507, + 629, 630, 627, 365, 414, 432, 421, 200, 641, 504, + 505, 642, 615, 0, 0, 0, 0, 391, 0, 519, + 552, 541, 625, 507, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 330, 0, 0, 360, 556, 538, + 548, 539, 524, 525, 526, 533, 340, 527, 528, 529, + 499, 530, 500, 531, 532, 136, 555, 506, 423, 375, + 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 196, + 2150, 0, 222, 0, 0, 0, 0, 0, 0, 303, + 223, 501, 621, 503, 502, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 306, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 330, 0, 0, 360, 556, 538, 548, 539, 524, 525, - 526, 533, 340, 527, 528, 529, 499, 530, 500, 531, - 532, 0, 555, 506, 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 2031, 0, 0, 222, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 293, 429, 447, 304, + 418, 460, 309, 426, 299, 390, 415, 0, 0, 295, + 445, 425, 372, 350, 351, 294, 0, 409, 328, 342, + 325, 388, 0, 444, 472, 324, 463, 0, 455, 297, + 0, 454, 387, 441, 446, 373, 367, 0, 296, 443, + 371, 366, 354, 332, 488, 355, 356, 346, 399, 364, + 400, 347, 377, 376, 378, 0, 0, 0, 0, 0, + 483, 484, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 614, 0, 0, 618, 0, + 457, 0, 0, 0, 0, 0, 0, 428, 0, 0, + 357, 0, 0, 0, 473, 0, 412, 393, 644, 0, + 0, 410, 362, 442, 401, 448, 430, 456, 406, 402, + 288, 431, 327, 374, 300, 302, 322, 329, 331, 333, + 334, 383, 384, 396, 417, 433, 434, 435, 326, 310, + 411, 311, 344, 312, 289, 318, 316, 319, 419, 320, + 291, 397, 439, 0, 339, 407, 370, 292, 369, 398, + 438, 437, 301, 464, 470, 471, 560, 0, 476, 645, + 646, 647, 485, 0, 403, 490, 491, 492, 494, 495, + 496, 497, 561, 578, 545, 515, 478, 569, 512, 516, + 517, 581, 0, 0, 0, 469, 358, 359, 0, 337, + 285, 286, 640, 323, 389, 583, 616, 617, 508, 0, + 570, 509, 518, 315, 542, 554, 553, 385, 468, 0, + 565, 568, 498, 639, 0, 562, 577, 643, 576, 636, + 395, 0, 416, 574, 521, 0, 566, 540, 0, 567, + 536, 571, 0, 510, 0, 424, 450, 462, 479, 482, + 511, 596, 597, 598, 290, 481, 600, 601, 602, 603, + 604, 605, 606, 599, 453, 543, 520, 546, 461, 523, + 522, 0, 0, 557, 477, 558, 559, 379, 380, 381, + 382, 341, 584, 308, 480, 405, 0, 544, 0, 0, + 0, 0, 0, 0, 0, 0, 549, 550, 547, 648, + 0, 607, 608, 0, 0, 474, 475, 336, 343, 493, + 345, 307, 394, 338, 459, 352, 0, 486, 551, 487, + 610, 613, 611, 612, 386, 348, 349, 420, 353, 363, + 408, 458, 392, 413, 305, 449, 422, 368, 537, 564, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 592, 591, 590, 589, + 588, 587, 586, 585, 0, 0, 534, 436, 317, 279, + 313, 314, 321, 637, 634, 440, 638, 0, 287, 514, + 361, 163, 404, 335, 579, 580, 0, 0, 235, 236, + 237, 238, 239, 240, 241, 242, 280, 243, 244, 245, + 246, 247, 248, 249, 252, 253, 254, 255, 256, 257, + 258, 259, 582, 250, 251, 260, 261, 262, 263, 264, + 265, 266, 267, 268, 269, 270, 271, 272, 273, 0, + 0, 0, 281, 282, 283, 284, 0, 0, 275, 276, + 277, 278, 0, 0, 0, 465, 466, 467, 489, 0, + 451, 513, 635, 0, 0, 0, 0, 0, 0, 0, + 563, 575, 609, 0, 619, 620, 622, 624, 623, 626, + 427, 0, 628, 629, 630, 627, 365, 414, 432, 421, + 200, 641, 504, 505, 642, 615, 0, 0, 0, 0, + 391, 0, 519, 552, 541, 625, 507, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 330, 0, 0, + 360, 556, 538, 548, 539, 524, 525, 526, 533, 340, + 527, 528, 529, 499, 530, 500, 531, 532, 136, 555, + 506, 423, 375, 573, 572, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 196, 2136, 0, 222, 0, 0, 0, 0, + 0, 0, 303, 223, 501, 621, 503, 502, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 306, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 293, + 429, 447, 304, 418, 460, 309, 426, 299, 390, 415, + 0, 0, 295, 445, 425, 372, 350, 351, 294, 0, + 409, 328, 342, 325, 388, 0, 444, 472, 324, 463, + 0, 455, 297, 0, 454, 387, 441, 446, 373, 367, + 0, 296, 443, 371, 366, 354, 332, 488, 355, 356, + 346, 399, 364, 400, 347, 377, 376, 378, 0, 0, + 0, 0, 0, 483, 484, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 614, 0, + 0, 618, 0, 457, 0, 0, 0, 0, 0, 0, + 428, 0, 0, 357, 0, 0, 0, 473, 0, 412, + 393, 644, 0, 0, 410, 362, 442, 401, 448, 430, + 456, 406, 402, 288, 431, 327, 374, 300, 302, 322, + 329, 331, 333, 334, 383, 384, 396, 417, 433, 434, + 435, 326, 310, 411, 311, 344, 312, 289, 318, 316, + 319, 419, 320, 291, 397, 439, 0, 339, 407, 370, + 292, 369, 398, 438, 437, 301, 464, 470, 471, 560, + 0, 476, 645, 646, 647, 485, 0, 403, 490, 491, + 492, 494, 495, 496, 497, 561, 578, 545, 515, 478, + 569, 512, 516, 517, 581, 0, 0, 0, 469, 358, + 359, 0, 337, 285, 286, 640, 323, 389, 583, 616, + 617, 508, 0, 570, 509, 518, 315, 542, 554, 553, + 385, 468, 0, 565, 568, 498, 639, 0, 562, 577, + 643, 576, 636, 395, 0, 416, 574, 521, 0, 566, + 540, 0, 567, 536, 571, 0, 510, 0, 424, 450, + 462, 479, 482, 511, 596, 597, 598, 290, 481, 600, + 601, 602, 603, 604, 605, 606, 599, 453, 543, 520, + 546, 461, 523, 522, 0, 0, 557, 477, 558, 559, + 379, 380, 381, 382, 341, 584, 308, 480, 405, 0, + 544, 0, 0, 0, 0, 0, 0, 0, 0, 549, + 550, 547, 648, 0, 607, 608, 0, 0, 474, 475, + 336, 343, 493, 345, 307, 394, 338, 459, 352, 0, + 486, 551, 487, 610, 613, 611, 612, 386, 348, 349, + 420, 353, 363, 408, 458, 392, 413, 305, 449, 422, + 368, 537, 564, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 592, + 591, 590, 589, 588, 587, 586, 585, 0, 0, 534, + 436, 317, 279, 313, 314, 321, 637, 634, 440, 638, + 0, 287, 514, 361, 163, 404, 335, 579, 580, 0, + 0, 235, 236, 237, 238, 239, 240, 241, 242, 280, + 243, 244, 245, 246, 247, 248, 249, 252, 253, 254, + 255, 256, 257, 258, 259, 582, 250, 251, 260, 261, + 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, + 272, 273, 0, 0, 0, 281, 282, 283, 284, 0, + 0, 275, 276, 277, 278, 0, 0, 0, 465, 466, + 467, 489, 0, 451, 513, 635, 0, 0, 0, 0, + 0, 0, 0, 563, 575, 609, 0, 619, 620, 622, + 624, 623, 626, 427, 0, 628, 629, 630, 627, 365, + 414, 432, 421, 0, 641, 504, 505, 642, 615, 391, + 0, 519, 552, 541, 625, 507, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 330, 1030, 0, 360, + 556, 538, 548, 539, 524, 525, 526, 533, 340, 527, + 528, 529, 499, 530, 500, 531, 532, 0, 555, 506, + 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 222, 1037, 1038, 0, 0, 0, + 0, 303, 223, 501, 621, 503, 502, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1041, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 293, 429, + 1024, 304, 418, 460, 309, 426, 299, 390, 415, 0, + 0, 295, 445, 425, 372, 350, 351, 294, 0, 409, + 328, 342, 325, 388, 0, 444, 472, 324, 463, 1011, + 455, 297, 1010, 454, 387, 441, 446, 373, 367, 0, + 296, 443, 371, 366, 354, 332, 488, 355, 356, 346, + 399, 364, 400, 347, 377, 376, 378, 0, 0, 0, + 0, 0, 483, 484, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 614, 0, 0, + 618, 0, 457, 0, 0, 0, 0, 0, 0, 428, + 0, 0, 357, 0, 0, 0, 473, 0, 412, 393, + 644, 0, 0, 410, 362, 442, 401, 448, 430, 456, + 1028, 402, 288, 431, 327, 374, 300, 302, 322, 329, + 331, 333, 334, 383, 384, 396, 417, 433, 434, 435, + 326, 310, 411, 311, 344, 312, 289, 318, 316, 319, + 419, 320, 291, 397, 439, 0, 339, 407, 370, 292, + 369, 398, 438, 437, 301, 464, 470, 471, 560, 0, + 476, 645, 646, 647, 485, 0, 403, 490, 491, 492, + 494, 495, 496, 497, 561, 578, 545, 515, 478, 569, + 512, 516, 517, 581, 0, 0, 0, 469, 358, 359, + 0, 337, 285, 286, 640, 323, 389, 583, 616, 617, + 508, 0, 570, 509, 518, 315, 542, 554, 553, 385, + 468, 0, 565, 568, 498, 639, 0, 562, 577, 643, + 576, 636, 395, 0, 416, 574, 521, 0, 566, 540, + 0, 567, 536, 571, 0, 510, 0, 424, 450, 462, + 479, 482, 511, 596, 597, 598, 290, 481, 600, 601, + 602, 603, 604, 605, 1029, 599, 453, 543, 520, 546, + 461, 523, 522, 0, 0, 557, 1032, 558, 559, 379, + 380, 381, 382, 341, 584, 1027, 480, 405, 0, 544, + 0, 0, 0, 0, 0, 0, 0, 0, 549, 550, + 547, 648, 0, 607, 608, 0, 0, 474, 475, 336, + 343, 493, 345, 307, 394, 338, 459, 352, 0, 486, + 551, 487, 610, 613, 611, 612, 1039, 1025, 1035, 1026, + 353, 363, 408, 458, 392, 413, 305, 449, 422, 1036, + 537, 564, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 592, 591, + 590, 589, 588, 587, 586, 585, 0, 0, 534, 436, + 317, 279, 313, 314, 321, 637, 634, 440, 638, 0, + 287, 514, 361, 0, 404, 335, 579, 580, 0, 0, + 235, 236, 237, 238, 239, 240, 241, 242, 280, 243, + 244, 245, 246, 247, 248, 249, 252, 253, 254, 255, + 256, 257, 258, 259, 582, 250, 251, 260, 261, 262, + 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, + 273, 0, 0, 0, 281, 282, 283, 284, 0, 0, + 275, 276, 277, 278, 0, 0, 0, 465, 466, 467, + 489, 0, 451, 513, 635, 0, 0, 0, 0, 0, + 0, 0, 563, 575, 609, 0, 619, 620, 622, 624, + 623, 626, 427, 0, 628, 629, 630, 627, 1023, 414, + 432, 421, 200, 641, 504, 505, 642, 615, 0, 0, + 0, 0, 391, 0, 519, 552, 541, 625, 507, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 330, + 0, 0, 360, 556, 538, 548, 539, 524, 525, 526, + 533, 340, 527, 528, 529, 499, 530, 500, 531, 532, + 136, 555, 506, 423, 375, 573, 572, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 2032, 0, 0, 222, 0, 0, 0, 0, 0, 0, 303, 223, 501, 621, 503, 502, 0, 0, 0, 0, 0, 0, 0, 0, 0, 306, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -5515,7 +4094,7 @@ var yyAct = [...]int{ 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 592, 591, 590, 589, 588, 587, 586, 585, 0, 0, 534, 436, 317, 279, 313, 314, 321, 637, 634, - 440, 638, 0, 287, 514, 361, 0, 404, 335, 579, + 440, 638, 0, 287, 514, 361, 163, 404, 335, 579, 580, 0, 0, 235, 236, 237, 238, 239, 240, 241, 242, 280, 243, 244, 245, 246, 247, 248, 249, 252, 253, 254, 255, 256, 257, 258, 259, 582, 250, 251, @@ -5527,11 +4106,74 @@ var yyAct = [...]int{ 620, 622, 624, 623, 626, 427, 0, 628, 629, 630, 627, 365, 414, 432, 421, 0, 641, 504, 505, 642, 615, 391, 0, 519, 552, 541, 625, 507, 0, 0, - 3380, 0, 0, 0, 0, 0, 0, 0, 330, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 330, 0, 0, 360, 556, 538, 548, 539, 524, 525, 526, 533, 340, 527, 528, 529, 499, 530, 500, 531, 532, 0, 555, 506, 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 222, 1037, 1038, 0, + 0, 0, 0, 303, 223, 501, 621, 503, 502, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1041, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 293, 429, 447, 304, 418, 460, 309, 426, 299, 390, + 415, 0, 0, 295, 445, 425, 372, 350, 351, 294, + 0, 409, 328, 342, 325, 388, 0, 444, 472, 324, + 463, 1011, 455, 297, 1010, 454, 387, 441, 446, 373, + 367, 0, 296, 443, 371, 366, 354, 332, 488, 355, + 356, 346, 399, 364, 400, 347, 377, 376, 378, 0, + 0, 0, 0, 0, 483, 484, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 614, + 0, 0, 618, 0, 457, 0, 0, 0, 0, 0, + 0, 428, 0, 0, 357, 0, 0, 0, 473, 0, + 412, 393, 644, 0, 0, 410, 362, 442, 401, 448, + 430, 456, 406, 402, 288, 431, 327, 374, 300, 302, + 322, 329, 331, 333, 334, 383, 384, 396, 417, 433, + 434, 435, 326, 310, 411, 311, 344, 312, 289, 318, + 316, 319, 419, 320, 291, 397, 439, 0, 339, 407, + 370, 292, 369, 398, 438, 437, 301, 464, 470, 471, + 560, 0, 476, 645, 646, 647, 485, 0, 403, 490, + 491, 492, 494, 495, 496, 497, 561, 578, 545, 515, + 478, 569, 512, 516, 517, 581, 0, 0, 0, 469, + 358, 359, 0, 337, 285, 286, 640, 323, 389, 583, + 616, 617, 508, 0, 570, 509, 518, 315, 542, 554, + 553, 385, 468, 0, 565, 568, 498, 639, 0, 562, + 577, 643, 576, 636, 395, 0, 416, 574, 521, 0, + 566, 540, 0, 567, 536, 571, 0, 510, 0, 424, + 450, 462, 479, 482, 511, 596, 597, 598, 290, 481, + 600, 601, 602, 603, 604, 605, 606, 599, 453, 543, + 520, 546, 461, 523, 522, 0, 0, 557, 477, 558, + 559, 379, 380, 381, 382, 341, 584, 308, 480, 405, + 0, 544, 0, 0, 0, 0, 0, 0, 0, 0, + 549, 550, 547, 648, 0, 607, 608, 0, 0, 474, + 475, 336, 343, 493, 345, 307, 394, 338, 459, 352, + 0, 486, 551, 487, 610, 613, 611, 612, 1039, 2053, + 1035, 2054, 353, 363, 408, 458, 392, 413, 305, 449, + 422, 1036, 537, 564, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 592, 591, 590, 589, 588, 587, 586, 585, 0, 0, + 534, 436, 317, 279, 313, 314, 321, 637, 634, 440, + 638, 0, 287, 514, 361, 0, 404, 335, 579, 580, + 0, 0, 235, 236, 237, 238, 239, 240, 241, 242, + 280, 243, 244, 245, 246, 247, 248, 249, 252, 253, + 254, 255, 256, 257, 258, 259, 582, 250, 251, 260, + 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, + 271, 272, 273, 0, 0, 0, 281, 282, 283, 284, + 0, 0, 275, 276, 277, 278, 0, 0, 0, 465, + 466, 467, 489, 0, 451, 513, 635, 0, 0, 0, + 0, 0, 0, 0, 563, 575, 609, 0, 619, 620, + 622, 624, 623, 626, 427, 0, 628, 629, 630, 627, + 365, 414, 432, 421, 0, 641, 504, 505, 642, 615, + 391, 0, 519, 552, 541, 625, 507, 0, 0, 2932, + 0, 0, 0, 0, 0, 0, 0, 330, 0, 0, + 360, 556, 538, 548, 539, 524, 525, 526, 533, 340, + 527, 528, 529, 499, 530, 500, 531, 532, 0, 555, + 506, 423, 375, 573, 572, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 222, 0, 0, 0, 0, 0, 0, 303, 223, 501, 621, 503, 502, 0, 0, 0, 0, 0, 0, 0, 0, 0, 306, 0, 0, @@ -5547,7 +4189,7 @@ var yyAct = [...]int{ 0, 296, 443, 371, 366, 354, 332, 488, 355, 356, 346, 399, 364, 400, 347, 377, 376, 378, 0, 0, 0, 0, 0, 483, 484, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 614, 0, + 0, 0, 0, 2935, 0, 0, 0, 2934, 614, 0, 0, 618, 0, 457, 0, 0, 0, 0, 0, 0, 428, 0, 0, 357, 0, 0, 0, 473, 0, 412, 393, 644, 0, 0, 410, 362, 442, 401, 448, 430, @@ -5590,19 +4232,82 @@ var yyAct = [...]int{ 624, 623, 626, 427, 0, 628, 629, 630, 627, 365, 414, 432, 421, 0, 641, 504, 505, 642, 615, 391, 0, 519, 552, 541, 625, 507, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 330, 0, 0, 360, + 0, 0, 0, 0, 0, 0, 330, 1527, 0, 360, 556, 538, 548, 539, 524, 525, 526, 533, 340, 527, 528, 529, 499, 530, 500, 531, 532, 0, 555, 506, 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 222, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 222, 0, 0, 1525, 0, 0, + 0, 303, 223, 501, 621, 503, 502, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 306, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 1523, 0, 0, 0, 0, 0, 0, 293, 429, + 447, 304, 418, 460, 309, 426, 299, 390, 415, 0, + 0, 295, 445, 425, 372, 350, 351, 294, 0, 409, + 328, 342, 325, 388, 0, 444, 472, 324, 463, 0, + 455, 297, 0, 454, 387, 441, 446, 373, 367, 0, + 296, 443, 371, 366, 354, 332, 488, 355, 356, 346, + 399, 364, 400, 347, 377, 376, 378, 0, 0, 0, + 0, 0, 483, 484, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 614, 0, 0, + 618, 0, 457, 0, 0, 0, 0, 0, 0, 428, + 0, 0, 357, 0, 0, 0, 473, 0, 412, 393, + 644, 0, 0, 410, 362, 442, 401, 448, 430, 456, + 406, 402, 288, 431, 327, 374, 300, 302, 322, 329, + 331, 333, 334, 383, 384, 396, 417, 433, 434, 435, + 326, 310, 411, 311, 344, 312, 289, 318, 316, 319, + 419, 320, 291, 397, 439, 0, 339, 407, 370, 292, + 369, 398, 438, 437, 301, 464, 470, 471, 560, 0, + 476, 645, 646, 647, 485, 0, 403, 490, 491, 492, + 494, 495, 496, 497, 561, 578, 545, 515, 478, 569, + 512, 516, 517, 581, 0, 0, 0, 469, 358, 359, + 0, 337, 285, 286, 640, 323, 389, 583, 616, 617, + 508, 0, 570, 509, 518, 315, 542, 554, 553, 385, + 468, 0, 565, 568, 498, 639, 0, 562, 577, 643, + 576, 636, 395, 0, 416, 574, 521, 0, 566, 540, + 0, 567, 536, 571, 0, 510, 0, 424, 450, 462, + 479, 482, 511, 596, 597, 598, 290, 481, 600, 601, + 602, 603, 604, 605, 606, 599, 453, 543, 520, 546, + 461, 523, 522, 0, 0, 557, 477, 558, 559, 379, + 380, 381, 382, 341, 584, 308, 480, 405, 0, 544, + 0, 0, 0, 0, 0, 0, 0, 0, 549, 550, + 547, 648, 0, 607, 608, 0, 0, 474, 475, 336, + 343, 493, 345, 307, 394, 338, 459, 352, 0, 486, + 551, 487, 610, 613, 611, 612, 386, 348, 349, 420, + 353, 363, 408, 458, 392, 413, 305, 449, 422, 368, + 537, 564, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 592, 591, + 590, 589, 588, 587, 586, 585, 0, 0, 534, 436, + 317, 279, 313, 314, 321, 637, 634, 440, 638, 0, + 287, 514, 361, 0, 404, 335, 579, 580, 0, 0, + 235, 236, 237, 238, 239, 240, 241, 242, 280, 243, + 244, 245, 246, 247, 248, 249, 252, 253, 254, 255, + 256, 257, 258, 259, 582, 250, 251, 260, 261, 262, + 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, + 273, 0, 0, 0, 281, 282, 283, 284, 0, 0, + 275, 276, 277, 278, 0, 0, 0, 465, 466, 467, + 489, 0, 451, 513, 635, 0, 0, 0, 0, 0, + 0, 0, 563, 575, 609, 0, 619, 620, 622, 624, + 623, 626, 427, 0, 628, 629, 630, 627, 365, 414, + 432, 421, 0, 641, 504, 505, 642, 615, 391, 0, + 519, 552, 541, 625, 507, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 330, 1521, 0, 360, 556, + 538, 548, 539, 524, 525, 526, 533, 340, 527, 528, + 529, 499, 530, 500, 531, 532, 0, 555, 506, 423, + 375, 573, 572, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 222, 0, 0, 1525, 0, 0, 0, 303, 223, 501, 621, 503, 502, 0, 0, 0, 0, 0, 0, 0, 0, 0, 306, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 3296, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 293, 429, 447, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1523, 0, 0, 0, 0, 0, 0, 293, 429, 447, 304, 418, 460, 309, 426, 299, 390, 415, 0, 0, 295, 445, 425, 372, 350, 351, 294, 0, 409, 328, 342, 325, 388, 0, 444, 472, 324, 463, 0, 455, @@ -5658,13 +4363,76 @@ var yyAct = [...]int{ 499, 530, 500, 531, 532, 0, 555, 506, 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 222, 0, 0, 0, 0, 0, 0, 303, 223, + 3995, 0, 222, 838, 0, 0, 0, 0, 0, 303, + 223, 501, 621, 503, 502, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 306, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 293, 429, 447, 304, + 418, 460, 309, 426, 299, 390, 415, 0, 0, 295, + 445, 425, 372, 350, 351, 294, 0, 409, 328, 342, + 325, 388, 0, 444, 472, 324, 463, 0, 455, 297, + 0, 454, 387, 441, 446, 373, 367, 0, 296, 443, + 371, 366, 354, 332, 488, 355, 356, 346, 399, 364, + 400, 347, 377, 376, 378, 0, 0, 0, 0, 0, + 483, 484, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 614, 0, 0, 618, 0, + 457, 0, 0, 0, 0, 0, 0, 428, 0, 0, + 357, 0, 0, 0, 473, 0, 412, 393, 644, 0, + 0, 410, 362, 442, 401, 448, 430, 456, 406, 402, + 288, 431, 327, 374, 300, 302, 322, 329, 331, 333, + 334, 383, 384, 396, 417, 433, 434, 435, 326, 310, + 411, 311, 344, 312, 289, 318, 316, 319, 419, 320, + 291, 397, 439, 0, 339, 407, 370, 292, 369, 398, + 438, 437, 301, 464, 470, 471, 560, 0, 476, 645, + 646, 647, 485, 0, 403, 490, 491, 492, 494, 495, + 496, 497, 561, 578, 545, 515, 478, 569, 512, 516, + 517, 581, 0, 0, 0, 469, 358, 359, 0, 337, + 285, 286, 640, 323, 389, 583, 616, 617, 508, 0, + 570, 509, 518, 315, 542, 554, 553, 385, 468, 0, + 565, 568, 498, 639, 0, 562, 577, 643, 576, 636, + 395, 0, 416, 574, 521, 0, 566, 540, 0, 567, + 536, 571, 0, 510, 0, 424, 450, 462, 479, 482, + 511, 596, 597, 598, 290, 481, 600, 601, 602, 603, + 604, 605, 606, 599, 453, 543, 520, 546, 461, 523, + 522, 0, 0, 557, 477, 558, 559, 379, 380, 381, + 382, 341, 584, 308, 480, 405, 0, 544, 0, 0, + 0, 0, 0, 0, 0, 0, 549, 550, 547, 648, + 0, 607, 608, 0, 0, 474, 475, 336, 343, 493, + 345, 307, 394, 338, 459, 352, 0, 486, 551, 487, + 610, 613, 611, 612, 386, 348, 349, 420, 353, 363, + 408, 458, 392, 413, 305, 449, 422, 368, 537, 564, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 592, 591, 590, 589, + 588, 587, 586, 585, 0, 0, 534, 436, 317, 279, + 313, 314, 321, 637, 634, 440, 638, 0, 287, 514, + 361, 0, 404, 335, 579, 580, 0, 0, 235, 236, + 237, 238, 239, 240, 241, 242, 280, 243, 244, 245, + 246, 247, 248, 249, 252, 253, 254, 255, 256, 257, + 258, 259, 582, 250, 251, 260, 261, 262, 263, 264, + 265, 266, 267, 268, 269, 270, 271, 272, 273, 0, + 0, 0, 281, 282, 283, 284, 0, 0, 275, 276, + 277, 278, 0, 0, 0, 465, 466, 467, 489, 0, + 451, 513, 635, 0, 0, 0, 0, 0, 0, 0, + 563, 575, 609, 0, 619, 620, 622, 624, 623, 626, + 427, 0, 628, 629, 630, 627, 365, 414, 432, 421, + 0, 641, 504, 505, 642, 615, 391, 0, 519, 552, + 541, 625, 507, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 330, 0, 0, 360, 556, 538, 548, + 539, 524, 525, 526, 533, 340, 527, 528, 529, 499, + 530, 500, 531, 532, 0, 555, 506, 423, 375, 573, + 572, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 222, 0, 0, 1525, 0, 0, 0, 303, 223, 501, 621, 503, 502, 0, 0, 0, 0, 0, 0, 0, 0, 0, 306, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 3034, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1523, 0, 0, 0, 0, 0, 0, 293, 429, 447, 304, 418, 460, 309, 426, 299, 390, 415, 0, 0, 295, 445, 425, 372, 350, 351, 294, 0, 409, 328, 342, 325, @@ -5720,14 +4488,77 @@ var yyAct = [...]int{ 524, 525, 526, 533, 340, 527, 528, 529, 499, 530, 500, 531, 532, 0, 555, 506, 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 222, 0, 0, 0, 0, 0, 0, 303, 223, 501, + 621, 503, 502, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 306, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 293, 429, 447, 304, 418, 460, + 309, 426, 299, 390, 415, 0, 0, 295, 445, 425, + 372, 350, 351, 294, 0, 409, 328, 342, 325, 388, + 0, 444, 472, 324, 463, 0, 455, 297, 0, 454, + 387, 441, 446, 373, 367, 0, 296, 443, 371, 366, + 354, 332, 488, 355, 356, 346, 399, 364, 400, 347, + 377, 376, 378, 0, 0, 0, 0, 0, 483, 484, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 614, 0, 0, 618, 0, 457, 0, + 0, 0, 0, 0, 0, 428, 0, 0, 357, 0, + 0, 0, 473, 0, 412, 393, 644, 0, 0, 410, + 362, 442, 401, 448, 430, 456, 406, 402, 288, 431, + 327, 374, 300, 302, 322, 329, 331, 333, 334, 383, + 384, 396, 417, 433, 434, 435, 326, 310, 411, 311, + 344, 312, 289, 318, 316, 319, 419, 320, 291, 397, + 439, 0, 339, 407, 370, 292, 369, 398, 438, 437, + 301, 464, 470, 471, 560, 0, 476, 645, 646, 647, + 485, 2449, 403, 490, 491, 492, 494, 495, 496, 497, + 561, 578, 545, 515, 478, 569, 512, 516, 517, 581, + 0, 0, 0, 469, 358, 359, 0, 337, 285, 286, + 640, 323, 389, 583, 616, 617, 508, 0, 570, 509, + 518, 315, 542, 554, 553, 385, 468, 0, 565, 568, + 498, 639, 0, 562, 577, 643, 576, 636, 395, 0, + 416, 574, 521, 0, 566, 540, 0, 567, 536, 571, + 0, 510, 0, 424, 450, 462, 479, 482, 511, 596, + 597, 598, 290, 481, 600, 601, 602, 603, 604, 605, + 606, 599, 453, 543, 520, 546, 461, 523, 522, 0, + 0, 557, 477, 558, 559, 379, 380, 381, 382, 341, + 584, 308, 480, 405, 0, 544, 0, 0, 0, 0, + 0, 0, 0, 0, 549, 550, 547, 648, 0, 607, + 608, 0, 0, 474, 475, 336, 343, 493, 345, 307, + 394, 338, 459, 352, 0, 486, 551, 487, 610, 613, + 611, 612, 386, 348, 349, 420, 353, 363, 408, 458, + 392, 413, 305, 449, 422, 368, 537, 564, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 274, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 592, 591, 590, 589, 588, 587, + 586, 585, 960, 0, 534, 436, 317, 279, 313, 314, + 321, 637, 634, 440, 638, 0, 287, 514, 361, 0, + 404, 335, 579, 580, 0, 0, 235, 236, 237, 238, + 239, 240, 241, 242, 280, 243, 244, 245, 246, 247, + 248, 249, 252, 253, 254, 255, 256, 257, 258, 259, + 582, 250, 251, 260, 261, 262, 263, 264, 265, 266, + 267, 268, 269, 270, 271, 272, 273, 0, 0, 0, + 281, 282, 283, 284, 0, 0, 275, 276, 277, 278, + 0, 0, 0, 465, 466, 467, 489, 0, 451, 513, + 635, 0, 0, 0, 0, 0, 0, 0, 563, 575, + 609, 0, 619, 620, 622, 624, 623, 626, 427, 0, + 628, 629, 630, 627, 365, 414, 432, 421, 0, 641, + 504, 505, 642, 615, 391, 0, 519, 552, 541, 625, + 507, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 330, 0, 0, 360, 556, 538, 548, 539, 524, + 525, 526, 533, 340, 527, 528, 529, 499, 530, 500, + 531, 532, 0, 555, 506, 423, 375, 573, 572, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 222, - 0, 0, 1524, 0, 0, 0, 303, 223, 501, 621, + 0, 0, 1525, 0, 0, 0, 303, 223, 501, 621, 503, 502, 0, 0, 0, 0, 0, 0, 0, 0, 0, 306, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1735, 0, 0, 0, 0, 0, 0, 293, 429, 447, 304, 418, 460, 309, 426, 299, 390, 415, 0, 0, 295, 445, 425, 372, 350, 351, 294, 0, 409, 328, 342, 325, 388, 0, @@ -5778,13 +4609,76 @@ var yyAct = [...]int{ 0, 619, 620, 622, 624, 623, 626, 427, 0, 628, 629, 630, 627, 365, 414, 432, 421, 0, 641, 504, 505, 642, 615, 391, 0, 519, 552, 541, 625, 507, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 2488, 0, 0, 0, 0, 330, 0, 0, 360, 556, 538, 548, 539, 524, 525, 526, 533, 340, 527, 528, 529, 499, 530, 500, 531, 532, 0, 555, 506, 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 222, 0, + 0, 2490, 0, 0, 0, 303, 223, 501, 621, 503, + 502, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 306, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 293, 429, 447, 304, 418, 460, 309, 426, + 299, 390, 415, 0, 0, 295, 445, 425, 372, 350, + 351, 294, 0, 409, 328, 342, 325, 388, 0, 444, + 472, 324, 463, 0, 455, 297, 0, 454, 387, 441, + 446, 373, 367, 0, 296, 443, 371, 366, 354, 332, + 488, 355, 356, 346, 399, 364, 400, 347, 377, 376, + 378, 0, 0, 0, 0, 0, 483, 484, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 614, 0, 0, 618, 0, 457, 0, 0, 0, + 0, 0, 0, 428, 0, 0, 357, 0, 0, 0, + 473, 0, 412, 393, 644, 0, 0, 410, 362, 442, + 401, 448, 430, 456, 406, 402, 288, 431, 327, 374, + 300, 302, 322, 329, 331, 333, 334, 383, 384, 396, + 417, 433, 434, 435, 326, 310, 411, 311, 344, 312, + 289, 318, 316, 319, 419, 320, 291, 397, 439, 0, + 339, 407, 370, 292, 369, 398, 438, 437, 301, 464, + 470, 471, 560, 0, 476, 645, 646, 647, 485, 0, + 403, 490, 491, 492, 494, 495, 496, 497, 561, 578, + 545, 515, 478, 569, 512, 516, 517, 581, 0, 0, + 0, 469, 358, 359, 0, 337, 285, 286, 640, 323, + 389, 583, 616, 617, 508, 0, 570, 509, 518, 315, + 542, 554, 553, 385, 468, 0, 565, 568, 498, 639, + 0, 562, 577, 643, 576, 636, 395, 0, 416, 574, + 521, 0, 566, 540, 0, 567, 536, 571, 0, 510, + 0, 424, 450, 462, 479, 482, 511, 596, 597, 598, + 290, 481, 600, 601, 602, 603, 604, 605, 606, 599, + 453, 543, 520, 546, 461, 523, 522, 0, 0, 557, + 477, 558, 559, 379, 380, 381, 382, 341, 584, 308, + 480, 405, 0, 544, 0, 0, 0, 0, 0, 0, + 0, 0, 549, 550, 547, 648, 0, 607, 608, 0, + 0, 474, 475, 336, 343, 493, 345, 307, 394, 338, + 459, 352, 0, 486, 551, 487, 610, 613, 611, 612, + 386, 348, 349, 420, 353, 363, 408, 458, 392, 413, + 305, 449, 422, 368, 537, 564, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 592, 591, 590, 589, 588, 587, 586, 585, + 0, 0, 534, 436, 317, 279, 313, 314, 321, 637, + 634, 440, 638, 0, 287, 514, 361, 0, 404, 335, + 579, 580, 0, 0, 235, 236, 237, 238, 239, 240, + 241, 242, 280, 243, 244, 245, 246, 247, 248, 249, + 252, 253, 254, 255, 256, 257, 258, 259, 582, 250, + 251, 260, 261, 262, 263, 264, 265, 266, 267, 268, + 269, 270, 271, 272, 273, 0, 0, 0, 281, 282, + 283, 284, 0, 0, 275, 276, 277, 278, 0, 0, + 0, 465, 466, 467, 489, 0, 451, 513, 635, 0, + 0, 0, 0, 0, 0, 0, 563, 575, 609, 0, + 619, 620, 622, 624, 623, 626, 427, 0, 628, 629, + 630, 627, 365, 414, 432, 421, 0, 641, 504, 505, + 642, 615, 391, 0, 519, 552, 541, 625, 507, 0, + 0, 0, 0, 0, 2100, 0, 0, 0, 0, 330, + 0, 0, 360, 556, 538, 548, 539, 524, 525, 526, + 533, 340, 527, 528, 529, 499, 530, 500, 531, 532, + 0, 555, 506, 423, 375, 573, 572, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 222, 0, 0, - 2487, 0, 0, 0, 303, 223, 501, 621, 503, 502, + 2101, 0, 0, 0, 303, 223, 501, 621, 503, 502, 0, 0, 0, 0, 0, 0, 0, 0, 0, 306, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -5841,12 +4735,75 @@ var yyAct = [...]int{ 620, 622, 624, 623, 626, 427, 0, 628, 629, 630, 627, 365, 414, 432, 421, 0, 641, 504, 505, 642, 615, 391, 0, 519, 552, 541, 625, 507, 0, 0, - 2848, 0, 0, 0, 0, 0, 0, 0, 330, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 330, 0, 0, 360, 556, 538, 548, 539, 524, 525, 526, 533, 340, 527, 528, 529, 499, 530, 500, 531, 532, 0, 555, 506, 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 222, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 222, 0, 0, 3146, + 3148, 0, 0, 303, 223, 501, 621, 503, 502, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 306, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 293, 429, 447, 304, 418, 460, 309, 426, 299, 390, + 415, 0, 0, 295, 445, 425, 372, 350, 351, 294, + 0, 409, 328, 342, 325, 388, 0, 444, 472, 324, + 463, 0, 455, 297, 0, 454, 387, 441, 446, 373, + 367, 0, 296, 443, 371, 366, 354, 332, 488, 355, + 356, 346, 399, 364, 400, 347, 377, 376, 378, 0, + 0, 0, 0, 0, 483, 484, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 614, + 0, 0, 618, 0, 457, 0, 0, 0, 0, 0, + 0, 428, 0, 0, 357, 0, 0, 0, 473, 0, + 412, 393, 644, 0, 0, 410, 362, 442, 401, 448, + 430, 456, 406, 402, 288, 431, 327, 374, 300, 302, + 322, 329, 331, 333, 334, 383, 384, 396, 417, 433, + 434, 435, 326, 310, 411, 311, 344, 312, 289, 318, + 316, 319, 419, 320, 291, 397, 439, 0, 339, 407, + 370, 292, 369, 398, 438, 437, 301, 464, 470, 471, + 560, 0, 476, 645, 646, 647, 485, 0, 403, 490, + 491, 492, 494, 495, 496, 497, 561, 578, 545, 515, + 478, 569, 512, 516, 517, 581, 0, 0, 0, 469, + 358, 359, 0, 337, 285, 286, 640, 323, 389, 583, + 616, 617, 508, 0, 570, 509, 518, 315, 542, 554, + 553, 385, 468, 0, 565, 568, 498, 639, 0, 562, + 577, 643, 576, 636, 395, 0, 416, 574, 521, 0, + 566, 540, 0, 567, 536, 571, 0, 510, 0, 424, + 450, 462, 479, 482, 511, 596, 597, 598, 290, 481, + 600, 601, 602, 603, 604, 605, 606, 599, 453, 543, + 520, 546, 461, 523, 522, 0, 0, 557, 477, 558, + 559, 379, 380, 381, 382, 341, 584, 308, 480, 405, + 0, 544, 0, 0, 0, 0, 0, 0, 0, 0, + 549, 550, 547, 648, 0, 607, 608, 0, 0, 474, + 475, 336, 343, 493, 345, 307, 394, 338, 459, 352, + 0, 486, 551, 487, 610, 613, 611, 612, 386, 348, + 349, 420, 353, 363, 408, 458, 392, 413, 305, 449, + 422, 368, 537, 564, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 592, 591, 590, 589, 588, 587, 586, 585, 0, 0, + 534, 436, 317, 279, 313, 314, 321, 637, 634, 440, + 638, 0, 287, 514, 361, 0, 404, 335, 579, 580, + 0, 0, 235, 236, 237, 238, 239, 240, 241, 242, + 280, 243, 244, 245, 246, 247, 248, 249, 252, 253, + 254, 255, 256, 257, 258, 259, 582, 250, 251, 260, + 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, + 271, 272, 273, 0, 0, 0, 281, 282, 283, 284, + 0, 0, 275, 276, 277, 278, 0, 0, 0, 465, + 466, 467, 489, 0, 451, 513, 635, 0, 0, 0, + 0, 0, 0, 0, 563, 575, 609, 0, 619, 620, + 622, 624, 623, 626, 427, 0, 628, 629, 630, 627, + 365, 414, 432, 421, 0, 641, 504, 505, 642, 615, + 391, 0, 519, 552, 541, 625, 507, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 330, 2511, 0, + 360, 556, 538, 548, 539, 524, 525, 526, 533, 340, + 527, 528, 529, 499, 530, 500, 531, 532, 0, 555, + 506, 423, 375, 573, 572, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 222, 0, 0, 1525, 0, 0, 0, 303, 223, 501, 621, 503, 502, 0, 0, 0, 0, 0, 0, 0, 0, 0, 306, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -5904,18 +4861,81 @@ var yyAct = [...]int{ 624, 623, 626, 427, 0, 628, 629, 630, 627, 365, 414, 432, 421, 0, 641, 504, 505, 642, 615, 391, 0, 519, 552, 541, 625, 507, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 330, 0, 0, 360, + 0, 0, 0, 0, 0, 655, 330, 0, 0, 360, 556, 538, 548, 539, 524, 525, 526, 533, 340, 527, 528, 529, 499, 530, 500, 531, 532, 0, 555, 506, 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 222, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 222, 0, 0, 0, 0, 0, + 0, 303, 223, 501, 621, 503, 502, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 306, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 293, 429, + 447, 304, 418, 460, 309, 426, 299, 390, 415, 0, + 0, 295, 445, 425, 372, 350, 351, 294, 0, 409, + 328, 342, 325, 388, 0, 444, 472, 324, 463, 0, + 455, 297, 0, 454, 387, 441, 446, 373, 367, 0, + 296, 443, 371, 366, 354, 332, 488, 355, 356, 346, + 399, 364, 400, 347, 377, 376, 378, 0, 0, 0, + 0, 0, 483, 484, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 614, 0, 0, + 618, 0, 457, 0, 654, 0, 0, 0, 0, 428, + 0, 0, 357, 0, 0, 0, 473, 0, 412, 393, + 644, 0, 0, 410, 362, 442, 401, 448, 430, 456, + 406, 402, 288, 431, 327, 374, 300, 302, 322, 329, + 331, 333, 334, 383, 384, 396, 417, 433, 434, 435, + 326, 310, 411, 311, 344, 312, 289, 318, 316, 319, + 419, 320, 291, 397, 439, 0, 339, 407, 370, 292, + 369, 398, 438, 437, 301, 464, 470, 471, 560, 0, + 476, 645, 646, 647, 485, 0, 403, 490, 491, 492, + 494, 495, 496, 497, 561, 578, 545, 515, 478, 569, + 512, 516, 517, 581, 0, 0, 0, 469, 358, 359, + 0, 337, 285, 286, 640, 323, 389, 583, 616, 617, + 508, 0, 570, 509, 518, 315, 542, 554, 553, 385, + 468, 0, 565, 568, 498, 639, 0, 562, 577, 643, + 576, 636, 395, 0, 416, 574, 521, 0, 566, 540, + 0, 567, 536, 571, 0, 510, 0, 424, 450, 462, + 479, 482, 511, 596, 597, 598, 290, 481, 600, 601, + 602, 603, 604, 605, 606, 599, 453, 543, 520, 546, + 461, 523, 522, 0, 0, 557, 477, 558, 559, 379, + 380, 381, 382, 341, 584, 308, 480, 405, 0, 544, + 0, 0, 0, 0, 0, 0, 0, 0, 549, 550, + 547, 648, 0, 607, 608, 0, 0, 474, 475, 336, + 343, 493, 345, 307, 394, 338, 459, 352, 0, 486, + 551, 487, 610, 613, 611, 612, 386, 348, 349, 420, + 353, 363, 408, 458, 392, 413, 305, 449, 422, 368, + 537, 564, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 592, 591, + 590, 589, 588, 587, 586, 585, 0, 0, 534, 436, + 317, 279, 313, 314, 321, 637, 634, 440, 638, 0, + 287, 514, 361, 0, 404, 335, 579, 580, 0, 0, + 235, 236, 237, 238, 239, 240, 241, 242, 280, 243, + 244, 245, 246, 247, 248, 249, 252, 253, 254, 255, + 256, 257, 258, 259, 582, 250, 251, 260, 261, 262, + 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, + 273, 0, 0, 0, 281, 282, 283, 284, 0, 0, + 275, 276, 277, 278, 0, 0, 0, 465, 466, 467, + 489, 0, 451, 513, 635, 0, 0, 0, 0, 0, + 0, 0, 563, 575, 609, 0, 619, 620, 622, 624, + 623, 626, 427, 0, 628, 629, 630, 627, 365, 414, + 432, 421, 0, 641, 504, 505, 642, 615, 391, 0, + 519, 552, 541, 625, 507, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 330, 0, 0, 360, 556, + 538, 548, 539, 524, 525, 526, 533, 340, 527, 528, + 529, 499, 530, 500, 531, 532, 0, 555, 506, 423, + 375, 573, 572, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 222, 838, 0, 0, 0, 0, 0, 303, 223, 501, 621, 503, 502, 0, 0, 0, 0, 0, 0, 0, 0, 0, 306, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 2170, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 293, 429, 447, 304, 418, 460, 309, 426, 299, 390, 415, 0, 0, 295, 445, 425, 372, 350, 351, 294, 0, 409, 328, @@ -5971,8 +4991,71 @@ var yyAct = [...]int{ 548, 539, 524, 525, 526, 533, 340, 527, 528, 529, 499, 530, 500, 531, 532, 0, 555, 506, 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 3972, + 0, 0, 222, 0, 0, 0, 0, 0, 0, 303, + 223, 501, 621, 503, 502, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 306, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 293, 429, 447, 304, + 418, 460, 309, 426, 299, 390, 415, 0, 0, 295, + 445, 425, 372, 350, 351, 294, 0, 409, 328, 342, + 325, 388, 0, 444, 472, 324, 463, 0, 455, 297, + 0, 454, 387, 441, 446, 373, 367, 0, 296, 443, + 371, 366, 354, 332, 488, 355, 356, 346, 399, 364, + 400, 347, 377, 376, 378, 0, 0, 0, 0, 0, + 483, 484, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 614, 0, 0, 618, 0, + 457, 0, 0, 0, 0, 0, 0, 428, 0, 0, + 357, 0, 0, 0, 473, 0, 412, 393, 644, 0, + 0, 410, 362, 442, 401, 448, 430, 456, 406, 402, + 288, 431, 327, 374, 300, 302, 322, 329, 331, 333, + 334, 383, 384, 396, 417, 433, 434, 435, 326, 310, + 411, 311, 344, 312, 289, 318, 316, 319, 419, 320, + 291, 397, 439, 0, 339, 407, 370, 292, 369, 398, + 438, 437, 301, 464, 470, 471, 560, 0, 476, 645, + 646, 647, 485, 0, 403, 490, 491, 492, 494, 495, + 496, 497, 561, 578, 545, 515, 478, 569, 512, 516, + 517, 581, 0, 0, 0, 469, 358, 359, 0, 337, + 285, 286, 640, 323, 389, 583, 616, 617, 508, 0, + 570, 509, 518, 315, 542, 554, 553, 385, 468, 0, + 565, 568, 498, 639, 0, 562, 577, 643, 576, 636, + 395, 0, 416, 574, 521, 0, 566, 540, 0, 567, + 536, 571, 0, 510, 0, 424, 450, 462, 479, 482, + 511, 596, 597, 598, 290, 481, 600, 601, 602, 603, + 604, 605, 606, 599, 453, 543, 520, 546, 461, 523, + 522, 0, 0, 557, 477, 558, 559, 379, 380, 381, + 382, 341, 584, 308, 480, 405, 0, 544, 0, 0, + 0, 0, 0, 0, 0, 0, 549, 550, 547, 648, + 0, 607, 608, 0, 0, 474, 475, 336, 343, 493, + 345, 307, 394, 338, 459, 352, 0, 486, 551, 487, + 610, 613, 611, 612, 386, 348, 349, 420, 353, 363, + 408, 458, 392, 413, 305, 449, 422, 368, 537, 564, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 592, 591, 590, 589, + 588, 587, 586, 585, 0, 0, 534, 436, 317, 279, + 313, 314, 321, 637, 634, 440, 638, 0, 287, 514, + 361, 0, 404, 335, 579, 580, 0, 0, 235, 236, + 237, 238, 239, 240, 241, 242, 280, 243, 244, 245, + 246, 247, 248, 249, 252, 253, 254, 255, 256, 257, + 258, 259, 582, 250, 251, 260, 261, 262, 263, 264, + 265, 266, 267, 268, 269, 270, 271, 272, 273, 0, + 0, 0, 281, 282, 283, 284, 0, 0, 275, 276, + 277, 278, 0, 0, 0, 465, 466, 467, 489, 0, + 451, 513, 635, 0, 0, 0, 0, 0, 0, 0, + 563, 575, 609, 0, 619, 620, 622, 624, 623, 626, + 427, 0, 628, 629, 630, 627, 365, 414, 432, 421, + 0, 641, 504, 505, 642, 615, 391, 0, 519, 552, + 541, 625, 507, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 330, 0, 0, 360, 556, 538, 548, + 539, 524, 525, 526, 533, 340, 527, 528, 529, 499, + 530, 500, 531, 532, 0, 555, 506, 423, 375, 573, + 572, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 222, 0, 0, 2605, 0, 0, 0, 303, 223, + 0, 222, 0, 0, 3740, 0, 0, 0, 303, 223, 501, 621, 503, 502, 0, 0, 0, 0, 0, 0, 0, 0, 0, 306, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -6034,13 +5117,76 @@ var yyAct = [...]int{ 524, 525, 526, 533, 340, 527, 528, 529, 499, 530, 500, 531, 532, 0, 555, 506, 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 222, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 222, 0, 0, 0, 0, 0, 0, 303, 223, 501, + 621, 503, 502, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 306, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 293, 429, 447, 304, 418, 460, + 309, 426, 299, 390, 415, 0, 0, 295, 445, 425, + 372, 350, 351, 294, 0, 409, 328, 342, 325, 388, + 0, 444, 472, 324, 463, 0, 455, 297, 0, 454, + 387, 441, 446, 373, 367, 0, 296, 443, 371, 366, + 354, 332, 488, 355, 356, 346, 399, 364, 400, 347, + 377, 376, 378, 0, 0, 0, 0, 0, 483, 484, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 614, 0, 0, 618, 0, 457, 0, + 0, 0, 3873, 0, 0, 428, 0, 0, 357, 0, + 0, 0, 473, 0, 412, 393, 644, 0, 0, 410, + 362, 442, 401, 448, 430, 456, 406, 402, 288, 431, + 327, 374, 300, 302, 322, 329, 331, 333, 334, 383, + 384, 396, 417, 433, 434, 435, 326, 310, 411, 311, + 344, 312, 289, 318, 316, 319, 419, 320, 291, 397, + 439, 0, 339, 407, 370, 292, 369, 398, 438, 437, + 301, 464, 470, 471, 560, 0, 476, 645, 646, 647, + 485, 0, 403, 490, 491, 492, 494, 495, 496, 497, + 561, 578, 545, 515, 478, 569, 512, 516, 517, 581, + 0, 0, 0, 469, 358, 359, 0, 337, 285, 286, + 640, 323, 389, 583, 616, 617, 508, 0, 570, 509, + 518, 315, 542, 554, 553, 385, 468, 0, 565, 568, + 498, 639, 0, 562, 577, 643, 576, 636, 395, 0, + 416, 574, 521, 0, 566, 540, 0, 567, 536, 571, + 0, 510, 0, 424, 450, 462, 479, 482, 511, 596, + 597, 598, 290, 481, 600, 601, 602, 603, 604, 605, + 606, 599, 453, 543, 520, 546, 461, 523, 522, 0, + 0, 557, 477, 558, 559, 379, 380, 381, 382, 341, + 584, 308, 480, 405, 0, 544, 0, 0, 0, 0, + 0, 0, 0, 0, 549, 550, 547, 648, 0, 607, + 608, 0, 0, 474, 475, 336, 343, 493, 345, 307, + 394, 338, 459, 352, 0, 486, 551, 487, 610, 613, + 611, 612, 386, 348, 349, 420, 353, 363, 408, 458, + 392, 413, 305, 449, 422, 368, 537, 564, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 274, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 592, 591, 590, 589, 588, 587, + 586, 585, 0, 0, 534, 436, 317, 279, 313, 314, + 321, 637, 634, 440, 638, 0, 287, 514, 361, 0, + 404, 335, 579, 580, 0, 0, 235, 236, 237, 238, + 239, 240, 241, 242, 280, 243, 244, 245, 246, 247, + 248, 249, 252, 253, 254, 255, 256, 257, 258, 259, + 582, 250, 251, 260, 261, 262, 263, 264, 265, 266, + 267, 268, 269, 270, 271, 272, 273, 0, 0, 0, + 281, 282, 283, 284, 0, 0, 275, 276, 277, 278, + 0, 0, 0, 465, 466, 467, 489, 0, 451, 513, + 635, 0, 0, 0, 0, 0, 0, 0, 563, 575, + 609, 0, 619, 620, 622, 624, 623, 626, 427, 0, + 628, 629, 630, 627, 365, 414, 432, 421, 0, 641, + 504, 505, 642, 615, 391, 0, 519, 552, 541, 625, + 507, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 330, 0, 0, 360, 556, 538, 548, 539, 524, + 525, 526, 533, 340, 527, 528, 529, 499, 530, 500, + 531, 532, 0, 555, 506, 423, 375, 573, 572, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1749, 0, 0, 222, 0, 0, 0, 0, 0, 0, 303, 223, 501, 621, 503, 502, 0, 0, 0, 0, 0, 0, 0, 0, 0, 306, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 2566, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 293, 429, 447, 304, 418, 460, 309, 426, 299, 390, 415, 0, 0, 295, 445, 425, 372, @@ -6097,8 +5243,71 @@ var yyAct = [...]int{ 526, 533, 340, 527, 528, 529, 499, 530, 500, 531, 532, 0, 555, 506, 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 3755, 0, 222, 0, + 0, 0, 0, 0, 0, 303, 223, 501, 621, 503, + 502, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 306, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 293, 429, 447, 304, 418, 460, 309, 426, + 299, 390, 415, 0, 0, 295, 445, 425, 372, 350, + 351, 294, 0, 409, 328, 342, 325, 388, 0, 444, + 472, 324, 463, 0, 455, 297, 0, 454, 387, 441, + 446, 373, 367, 0, 296, 443, 371, 366, 354, 332, + 488, 355, 356, 346, 399, 364, 400, 347, 377, 376, + 378, 0, 0, 0, 0, 0, 483, 484, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 614, 0, 0, 618, 0, 457, 0, 0, 0, + 0, 0, 0, 428, 0, 0, 357, 0, 0, 0, + 473, 0, 412, 393, 644, 0, 0, 410, 362, 442, + 401, 448, 430, 456, 406, 402, 288, 431, 327, 374, + 300, 302, 322, 329, 331, 333, 334, 383, 384, 396, + 417, 433, 434, 435, 326, 310, 411, 311, 344, 312, + 289, 318, 316, 319, 419, 320, 291, 397, 439, 0, + 339, 407, 370, 292, 369, 398, 438, 437, 301, 464, + 470, 471, 560, 0, 476, 645, 646, 647, 485, 0, + 403, 490, 491, 492, 494, 495, 496, 497, 561, 578, + 545, 515, 478, 569, 512, 516, 517, 581, 0, 0, + 0, 469, 358, 359, 0, 337, 285, 286, 640, 323, + 389, 583, 616, 617, 508, 0, 570, 509, 518, 315, + 542, 554, 553, 385, 468, 0, 565, 568, 498, 639, + 0, 562, 577, 643, 576, 636, 395, 0, 416, 574, + 521, 0, 566, 540, 0, 567, 536, 571, 0, 510, + 0, 424, 450, 462, 479, 482, 511, 596, 597, 598, + 290, 481, 600, 601, 602, 603, 604, 605, 606, 599, + 453, 543, 520, 546, 461, 523, 522, 0, 0, 557, + 477, 558, 559, 379, 380, 381, 382, 341, 584, 308, + 480, 405, 0, 544, 0, 0, 0, 0, 0, 0, + 0, 0, 549, 550, 547, 648, 0, 607, 608, 0, + 0, 474, 475, 336, 343, 493, 345, 307, 394, 338, + 459, 352, 0, 486, 551, 487, 610, 613, 611, 612, + 386, 348, 349, 420, 353, 363, 408, 458, 392, 413, + 305, 449, 422, 368, 537, 564, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 592, 591, 590, 589, 588, 587, 586, 585, + 0, 0, 534, 436, 317, 279, 313, 314, 321, 637, + 634, 440, 638, 0, 287, 514, 361, 0, 404, 335, + 579, 580, 0, 0, 235, 236, 237, 238, 239, 240, + 241, 242, 280, 243, 244, 245, 246, 247, 248, 249, + 252, 253, 254, 255, 256, 257, 258, 259, 582, 250, + 251, 260, 261, 262, 263, 264, 265, 266, 267, 268, + 269, 270, 271, 272, 273, 0, 0, 0, 281, 282, + 283, 284, 0, 0, 275, 276, 277, 278, 0, 0, + 0, 465, 466, 467, 489, 0, 451, 513, 635, 0, + 0, 0, 0, 0, 0, 0, 563, 575, 609, 0, + 619, 620, 622, 624, 623, 626, 427, 0, 628, 629, + 630, 627, 365, 414, 432, 421, 0, 641, 504, 505, + 642, 615, 391, 0, 519, 552, 541, 625, 507, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 330, + 0, 0, 360, 556, 538, 548, 539, 524, 525, 526, + 533, 340, 527, 528, 529, 499, 530, 500, 531, 532, + 0, 555, 506, 423, 375, 573, 572, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 222, 0, 0, - 2564, 0, 0, 0, 303, 223, 501, 621, 503, 502, + 0, 0, 0, 0, 303, 223, 501, 621, 503, 502, 0, 0, 0, 0, 0, 0, 0, 0, 0, 306, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -6113,7 +5322,7 @@ var yyAct = [...]int{ 355, 356, 346, 399, 364, 400, 347, 377, 376, 378, 0, 0, 0, 0, 0, 483, 484, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 614, 0, 0, 618, 0, 457, 0, 0, 0, 0, + 614, 0, 0, 618, 0, 457, 0, 0, 0, 3667, 0, 0, 428, 0, 0, 357, 0, 0, 0, 473, 0, 412, 393, 644, 0, 0, 410, 362, 442, 401, 448, 430, 456, 406, 402, 288, 431, 327, 374, 300, @@ -6153,20 +5362,83 @@ var yyAct = [...]int{ 465, 466, 467, 489, 0, 451, 513, 635, 0, 0, 0, 0, 0, 0, 0, 563, 575, 609, 0, 619, 620, 622, 624, 623, 626, 427, 0, 628, 629, 630, - 627, 365, 414, 432, 421, 2339, 641, 504, 505, 642, + 627, 365, 414, 432, 421, 0, 641, 504, 505, 642, 615, 391, 0, 519, 552, 541, 625, 507, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 330, 0, 0, 360, 556, 538, 548, 539, 524, 525, 526, 533, 340, 527, 528, 529, 499, 530, 500, 531, 532, 0, 555, 506, 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 222, 0, 0, 3178, + 0, 0, 0, 303, 223, 501, 621, 503, 502, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 306, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 293, 429, 447, 304, 418, 460, 309, 426, 299, 390, + 415, 0, 0, 295, 445, 425, 372, 350, 351, 294, + 0, 409, 328, 342, 325, 388, 0, 444, 472, 324, + 463, 0, 455, 297, 0, 454, 387, 441, 446, 373, + 367, 0, 296, 443, 371, 366, 354, 332, 488, 355, + 356, 346, 399, 364, 400, 347, 377, 376, 378, 0, + 0, 0, 0, 0, 483, 484, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 614, + 0, 0, 618, 0, 457, 0, 0, 0, 0, 0, + 0, 428, 0, 0, 357, 0, 0, 0, 473, 0, + 412, 393, 644, 0, 0, 410, 362, 442, 401, 448, + 430, 456, 406, 402, 288, 431, 327, 374, 300, 302, + 322, 329, 331, 333, 334, 383, 384, 396, 417, 433, + 434, 435, 326, 310, 411, 311, 344, 312, 289, 318, + 316, 319, 419, 320, 291, 397, 439, 0, 339, 407, + 370, 292, 369, 398, 438, 437, 301, 464, 470, 471, + 560, 0, 476, 645, 646, 647, 485, 0, 403, 490, + 491, 492, 494, 495, 496, 497, 561, 578, 545, 515, + 478, 569, 512, 516, 517, 581, 0, 0, 0, 469, + 358, 359, 0, 337, 285, 286, 640, 323, 389, 583, + 616, 617, 508, 0, 570, 509, 518, 315, 542, 554, + 553, 385, 468, 0, 565, 568, 498, 639, 0, 562, + 577, 643, 576, 636, 395, 0, 416, 574, 521, 0, + 566, 540, 0, 567, 536, 571, 0, 510, 0, 424, + 450, 462, 479, 482, 511, 596, 597, 598, 290, 481, + 600, 601, 602, 603, 604, 605, 606, 599, 453, 543, + 520, 546, 461, 523, 522, 0, 0, 557, 477, 558, + 559, 379, 380, 381, 382, 341, 584, 308, 480, 405, + 0, 544, 0, 0, 0, 0, 0, 0, 0, 0, + 549, 550, 547, 648, 0, 607, 608, 0, 0, 474, + 475, 336, 343, 493, 345, 307, 394, 338, 459, 352, + 0, 486, 551, 487, 610, 613, 611, 612, 386, 348, + 349, 420, 353, 363, 408, 458, 392, 413, 305, 449, + 422, 368, 537, 564, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 592, 591, 590, 589, 588, 587, 586, 585, 0, 0, + 534, 436, 317, 279, 313, 314, 321, 637, 634, 440, + 638, 0, 287, 514, 361, 0, 404, 335, 579, 580, + 0, 0, 235, 236, 237, 238, 239, 240, 241, 242, + 280, 243, 244, 245, 246, 247, 248, 249, 252, 253, + 254, 255, 256, 257, 258, 259, 582, 250, 251, 260, + 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, + 271, 272, 273, 0, 0, 0, 281, 282, 283, 284, + 0, 0, 275, 276, 277, 278, 0, 0, 0, 465, + 466, 467, 489, 0, 451, 513, 635, 0, 0, 0, + 0, 0, 0, 0, 563, 575, 609, 0, 619, 620, + 622, 624, 623, 626, 427, 0, 628, 629, 630, 627, + 365, 414, 432, 421, 0, 641, 504, 505, 642, 615, + 391, 0, 519, 552, 541, 625, 507, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 330, 0, 0, + 360, 556, 538, 548, 539, 524, 525, 526, 533, 340, + 527, 528, 529, 499, 530, 500, 531, 532, 0, 555, + 506, 423, 375, 573, 572, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 222, 0, 0, 0, 0, 0, 0, 303, 223, 501, 621, 503, 502, 0, 0, 0, 0, 0, 0, 0, 0, 0, 306, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 3196, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 293, 429, 447, 304, 418, 460, 309, 426, 299, 390, 415, 0, 0, 295, 445, 425, 372, 350, 351, 294, 0, @@ -6223,7 +5495,70 @@ var yyAct = [...]int{ 528, 529, 499, 530, 500, 531, 532, 0, 555, 506, 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 222, 0, 0, 0, 1885, 0, 0, + 0, 2032, 0, 0, 222, 0, 0, 0, 0, 0, + 0, 303, 223, 501, 621, 503, 502, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 306, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 293, 429, + 447, 304, 418, 460, 309, 426, 299, 390, 415, 0, + 0, 295, 445, 425, 372, 350, 351, 294, 0, 409, + 328, 342, 325, 388, 0, 444, 472, 324, 463, 0, + 455, 297, 0, 454, 387, 441, 446, 373, 367, 0, + 296, 443, 371, 366, 354, 332, 488, 355, 356, 346, + 399, 364, 400, 347, 377, 376, 378, 0, 0, 0, + 0, 0, 483, 484, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 614, 0, 0, + 618, 0, 457, 0, 0, 0, 0, 0, 0, 428, + 0, 0, 357, 0, 0, 0, 473, 0, 412, 393, + 644, 0, 0, 410, 362, 442, 401, 448, 430, 456, + 406, 402, 288, 431, 327, 374, 300, 302, 322, 329, + 331, 333, 334, 383, 384, 396, 417, 433, 434, 435, + 326, 310, 411, 311, 344, 312, 289, 318, 316, 319, + 419, 320, 291, 397, 439, 0, 339, 407, 370, 292, + 369, 398, 438, 437, 301, 464, 470, 471, 560, 0, + 476, 645, 646, 647, 485, 0, 403, 490, 491, 492, + 494, 495, 496, 497, 561, 578, 545, 515, 478, 569, + 512, 516, 517, 581, 0, 0, 0, 469, 358, 359, + 0, 337, 285, 286, 640, 323, 389, 583, 616, 617, + 508, 0, 570, 509, 518, 315, 542, 554, 553, 385, + 468, 0, 565, 568, 498, 639, 0, 562, 577, 643, + 576, 636, 395, 0, 416, 574, 521, 0, 566, 540, + 0, 567, 536, 571, 0, 510, 0, 424, 450, 462, + 479, 482, 511, 596, 597, 598, 290, 481, 600, 601, + 602, 603, 604, 605, 606, 599, 453, 543, 520, 546, + 461, 523, 522, 0, 0, 557, 477, 558, 559, 379, + 380, 381, 382, 341, 584, 308, 480, 405, 0, 544, + 0, 0, 0, 0, 0, 0, 0, 0, 549, 550, + 547, 648, 0, 607, 608, 0, 0, 474, 475, 336, + 343, 493, 345, 307, 394, 338, 459, 352, 0, 486, + 551, 487, 610, 613, 611, 612, 386, 348, 349, 420, + 353, 363, 408, 458, 392, 413, 305, 449, 422, 368, + 537, 564, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 592, 591, + 590, 589, 588, 587, 586, 585, 0, 0, 534, 436, + 317, 279, 313, 314, 321, 637, 634, 440, 638, 0, + 287, 514, 361, 0, 404, 335, 579, 580, 0, 0, + 235, 236, 237, 238, 239, 240, 241, 242, 280, 243, + 244, 245, 246, 247, 248, 249, 252, 253, 254, 255, + 256, 257, 258, 259, 582, 250, 251, 260, 261, 262, + 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, + 273, 0, 0, 0, 281, 282, 283, 284, 0, 0, + 275, 276, 277, 278, 0, 0, 0, 465, 466, 467, + 489, 0, 451, 513, 635, 0, 0, 0, 0, 0, + 0, 0, 563, 575, 609, 0, 619, 620, 622, 624, + 623, 626, 427, 0, 628, 629, 630, 627, 365, 414, + 432, 421, 0, 641, 504, 505, 642, 615, 391, 0, + 519, 552, 541, 625, 507, 0, 0, 3386, 0, 0, + 0, 0, 0, 0, 0, 330, 0, 0, 360, 556, + 538, 548, 539, 524, 525, 526, 533, 340, 527, 528, + 529, 499, 530, 500, 531, 532, 0, 555, 506, 423, + 375, 573, 572, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 222, 0, 0, 0, 0, 0, 0, 303, 223, 501, 621, 503, 502, 0, 0, 0, 0, 0, 0, 0, 0, 0, 306, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -6280,18 +5615,81 @@ var yyAct = [...]int{ 0, 563, 575, 609, 0, 619, 620, 622, 624, 623, 626, 427, 0, 628, 629, 630, 627, 365, 414, 432, 421, 0, 641, 504, 505, 642, 615, 391, 0, 519, - 552, 541, 625, 507, 0, 2013, 0, 0, 0, 0, + 552, 541, 625, 507, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 330, 0, 0, 360, 556, 538, 548, 539, 524, 525, 526, 533, 340, 527, 528, 529, 499, 530, 500, 531, 532, 0, 555, 506, 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 222, 0, 0, 0, 0, 0, 0, 303, + 223, 501, 621, 503, 502, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 306, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 3302, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 293, 429, 447, 304, + 418, 460, 309, 426, 299, 390, 415, 0, 0, 295, + 445, 425, 372, 350, 351, 294, 0, 409, 328, 342, + 325, 388, 0, 444, 472, 324, 463, 0, 455, 297, + 0, 454, 387, 441, 446, 373, 367, 0, 296, 443, + 371, 366, 354, 332, 488, 355, 356, 346, 399, 364, + 400, 347, 377, 376, 378, 0, 0, 0, 0, 0, + 483, 484, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 614, 0, 0, 618, 0, + 457, 0, 0, 0, 0, 0, 0, 428, 0, 0, + 357, 0, 0, 0, 473, 0, 412, 393, 644, 0, + 0, 410, 362, 442, 401, 448, 430, 456, 406, 402, + 288, 431, 327, 374, 300, 302, 322, 329, 331, 333, + 334, 383, 384, 396, 417, 433, 434, 435, 326, 310, + 411, 311, 344, 312, 289, 318, 316, 319, 419, 320, + 291, 397, 439, 0, 339, 407, 370, 292, 369, 398, + 438, 437, 301, 464, 470, 471, 560, 0, 476, 645, + 646, 647, 485, 0, 403, 490, 491, 492, 494, 495, + 496, 497, 561, 578, 545, 515, 478, 569, 512, 516, + 517, 581, 0, 0, 0, 469, 358, 359, 0, 337, + 285, 286, 640, 323, 389, 583, 616, 617, 508, 0, + 570, 509, 518, 315, 542, 554, 553, 385, 468, 0, + 565, 568, 498, 639, 0, 562, 577, 643, 576, 636, + 395, 0, 416, 574, 521, 0, 566, 540, 0, 567, + 536, 571, 0, 510, 0, 424, 450, 462, 479, 482, + 511, 596, 597, 598, 290, 481, 600, 601, 602, 603, + 604, 605, 606, 599, 453, 543, 520, 546, 461, 523, + 522, 0, 0, 557, 477, 558, 559, 379, 380, 381, + 382, 341, 584, 308, 480, 405, 0, 544, 0, 0, + 0, 0, 0, 0, 0, 0, 549, 550, 547, 648, + 0, 607, 608, 0, 0, 474, 475, 336, 343, 493, + 345, 307, 394, 338, 459, 352, 0, 486, 551, 487, + 610, 613, 611, 612, 386, 348, 349, 420, 353, 363, + 408, 458, 392, 413, 305, 449, 422, 368, 537, 564, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 592, 591, 590, 589, + 588, 587, 586, 585, 0, 0, 534, 436, 317, 279, + 313, 314, 321, 637, 634, 440, 638, 0, 287, 514, + 361, 0, 404, 335, 579, 580, 0, 0, 235, 236, + 237, 238, 239, 240, 241, 242, 280, 243, 244, 245, + 246, 247, 248, 249, 252, 253, 254, 255, 256, 257, + 258, 259, 582, 250, 251, 260, 261, 262, 263, 264, + 265, 266, 267, 268, 269, 270, 271, 272, 273, 0, + 0, 0, 281, 282, 283, 284, 0, 0, 275, 276, + 277, 278, 0, 0, 0, 465, 466, 467, 489, 0, + 451, 513, 635, 0, 0, 0, 0, 0, 0, 0, + 563, 575, 609, 0, 619, 620, 622, 624, 623, 626, + 427, 0, 628, 629, 630, 627, 365, 414, 432, 421, + 0, 641, 504, 505, 642, 615, 391, 0, 519, 552, + 541, 625, 507, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 330, 0, 0, 360, 556, 538, 548, + 539, 524, 525, 526, 533, 340, 527, 528, 529, 499, + 530, 500, 531, 532, 0, 555, 506, 423, 375, 573, + 572, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 222, 0, 0, 0, 0, 0, 0, 303, 223, 501, 621, 503, 502, 0, 0, 0, 0, 0, 0, 0, 0, 0, 306, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 3039, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 293, 429, 447, 304, 418, 460, 309, 426, 299, 390, 415, 0, 0, 295, 445, @@ -6348,8 +5746,71 @@ var yyAct = [...]int{ 524, 525, 526, 533, 340, 527, 528, 529, 499, 530, 500, 531, 532, 0, 555, 506, 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 222, 0, 0, 1525, 0, 0, 0, 303, 223, 501, + 621, 503, 502, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 306, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 293, 429, 447, 304, 418, 460, + 309, 426, 299, 390, 415, 0, 0, 295, 445, 425, + 372, 350, 351, 294, 0, 409, 328, 342, 325, 388, + 0, 444, 472, 324, 463, 0, 455, 297, 0, 454, + 387, 441, 446, 373, 367, 0, 296, 443, 371, 366, + 354, 332, 488, 355, 356, 346, 399, 364, 400, 347, + 377, 376, 378, 0, 0, 0, 0, 0, 483, 484, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 614, 0, 0, 618, 0, 457, 0, + 0, 0, 0, 0, 0, 428, 0, 0, 357, 0, + 0, 0, 473, 0, 412, 393, 644, 0, 0, 410, + 362, 442, 401, 448, 430, 456, 406, 402, 288, 431, + 327, 374, 300, 302, 322, 329, 331, 333, 334, 383, + 384, 396, 417, 433, 434, 435, 326, 310, 411, 311, + 344, 312, 289, 318, 316, 319, 419, 320, 291, 397, + 439, 0, 339, 407, 370, 292, 369, 398, 438, 437, + 301, 464, 470, 471, 560, 0, 476, 645, 646, 647, + 485, 0, 403, 490, 491, 492, 494, 495, 496, 497, + 561, 578, 545, 515, 478, 569, 512, 516, 517, 581, + 0, 0, 0, 469, 358, 359, 0, 337, 285, 286, + 640, 323, 389, 583, 616, 617, 508, 0, 570, 509, + 518, 315, 542, 554, 553, 385, 468, 0, 565, 568, + 498, 639, 0, 562, 577, 643, 576, 636, 395, 0, + 416, 574, 521, 0, 566, 540, 0, 567, 536, 571, + 0, 510, 0, 424, 450, 462, 479, 482, 511, 596, + 597, 598, 290, 481, 600, 601, 602, 603, 604, 605, + 606, 599, 453, 543, 520, 546, 461, 523, 522, 0, + 0, 557, 477, 558, 559, 379, 380, 381, 382, 341, + 584, 308, 480, 405, 0, 544, 0, 0, 0, 0, + 0, 0, 0, 0, 549, 550, 547, 648, 0, 607, + 608, 0, 0, 474, 475, 336, 343, 493, 345, 307, + 394, 338, 459, 352, 0, 486, 551, 487, 610, 613, + 611, 612, 386, 348, 349, 420, 353, 363, 408, 458, + 392, 413, 305, 449, 422, 368, 537, 564, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 274, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 592, 591, 590, 589, 588, 587, + 586, 585, 0, 0, 534, 436, 317, 279, 313, 314, + 321, 637, 634, 440, 638, 0, 287, 514, 361, 0, + 404, 335, 579, 580, 0, 0, 235, 236, 237, 238, + 239, 240, 241, 242, 280, 243, 244, 245, 246, 247, + 248, 249, 252, 253, 254, 255, 256, 257, 258, 259, + 582, 250, 251, 260, 261, 262, 263, 264, 265, 266, + 267, 268, 269, 270, 271, 272, 273, 0, 0, 0, + 281, 282, 283, 284, 0, 0, 275, 276, 277, 278, + 0, 0, 0, 465, 466, 467, 489, 0, 451, 513, + 635, 0, 0, 0, 0, 0, 0, 0, 563, 575, + 609, 0, 619, 620, 622, 624, 623, 626, 427, 0, + 628, 629, 630, 627, 365, 414, 432, 421, 0, 641, + 504, 505, 642, 615, 391, 0, 519, 552, 541, 625, + 507, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 330, 0, 0, 360, 556, 538, 548, 539, 524, + 525, 526, 533, 340, 527, 528, 529, 499, 530, 500, + 531, 532, 0, 555, 506, 423, 375, 573, 572, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 222, - 0, 0, 1524, 0, 0, 0, 303, 223, 501, 621, + 0, 0, 2490, 0, 0, 0, 303, 223, 501, 621, 503, 502, 0, 0, 0, 0, 0, 0, 0, 0, 0, 306, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -6367,7 +5828,7 @@ var yyAct = [...]int{ 0, 0, 614, 0, 0, 618, 0, 457, 0, 0, 0, 0, 0, 0, 428, 0, 0, 357, 0, 0, 0, 473, 0, 412, 393, 644, 0, 0, 410, 362, - 442, 401, 448, 430, 456, 1918, 402, 288, 431, 327, + 442, 401, 448, 430, 456, 406, 402, 288, 431, 327, 374, 300, 302, 322, 329, 331, 333, 334, 383, 384, 396, 417, 433, 434, 435, 326, 310, 411, 311, 344, 312, 289, 318, 316, 319, 419, 320, 291, 397, 439, @@ -6406,10 +5867,73 @@ var yyAct = [...]int{ 0, 619, 620, 622, 624, 623, 626, 427, 0, 628, 629, 630, 627, 365, 414, 432, 421, 0, 641, 504, 505, 642, 615, 391, 0, 519, 552, 541, 625, 507, + 0, 0, 2851, 0, 0, 0, 0, 0, 0, 0, + 330, 0, 0, 360, 556, 538, 548, 539, 524, 525, + 526, 533, 340, 527, 528, 529, 499, 530, 500, 531, + 532, 0, 555, 506, 423, 375, 573, 572, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 222, 0, + 0, 0, 0, 0, 0, 303, 223, 501, 621, 503, + 502, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 306, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 330, 0, 0, 360, 556, 538, 548, 539, 524, 525, - 526, 533, 340, 527, 528, 529, 499, 530, 500, 531, - 532, 0, 555, 506, 423, 375, 573, 572, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 293, 429, 447, 304, 418, 460, 309, 426, + 299, 390, 415, 0, 0, 295, 445, 425, 372, 350, + 351, 294, 0, 409, 328, 342, 325, 388, 0, 444, + 472, 324, 463, 0, 455, 297, 0, 454, 387, 441, + 446, 373, 367, 0, 296, 443, 371, 366, 354, 332, + 488, 355, 356, 346, 399, 364, 400, 347, 377, 376, + 378, 0, 0, 0, 0, 0, 483, 484, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 614, 0, 0, 618, 0, 457, 0, 0, 0, + 0, 0, 0, 428, 0, 0, 357, 0, 0, 0, + 473, 0, 412, 393, 644, 0, 0, 410, 362, 442, + 401, 448, 430, 456, 406, 402, 288, 431, 327, 374, + 300, 302, 322, 329, 331, 333, 334, 383, 384, 396, + 417, 433, 434, 435, 326, 310, 411, 311, 344, 312, + 289, 318, 316, 319, 419, 320, 291, 397, 439, 0, + 339, 407, 370, 292, 369, 398, 438, 437, 301, 464, + 470, 471, 560, 0, 476, 645, 646, 647, 485, 0, + 403, 490, 491, 492, 494, 495, 496, 497, 561, 578, + 545, 515, 478, 569, 512, 516, 517, 581, 0, 0, + 0, 469, 358, 359, 0, 337, 285, 286, 640, 323, + 389, 583, 616, 617, 508, 0, 570, 509, 518, 315, + 542, 554, 553, 385, 468, 0, 565, 568, 498, 639, + 0, 562, 577, 643, 576, 636, 395, 0, 416, 574, + 521, 0, 566, 540, 0, 567, 536, 571, 0, 510, + 0, 424, 450, 462, 479, 482, 511, 596, 597, 598, + 290, 481, 600, 601, 602, 603, 604, 605, 606, 599, + 453, 543, 520, 546, 461, 523, 522, 0, 0, 557, + 477, 558, 559, 379, 380, 381, 382, 341, 584, 308, + 480, 405, 0, 544, 0, 0, 0, 0, 0, 0, + 0, 0, 549, 550, 547, 648, 0, 607, 608, 0, + 0, 474, 475, 336, 343, 493, 345, 307, 394, 338, + 459, 352, 0, 486, 551, 487, 610, 613, 611, 612, + 386, 348, 349, 420, 353, 363, 408, 458, 392, 413, + 305, 449, 422, 368, 537, 564, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 592, 591, 590, 589, 588, 587, 586, 585, + 0, 0, 534, 436, 317, 279, 313, 314, 321, 637, + 634, 440, 638, 0, 287, 514, 361, 0, 404, 335, + 579, 580, 0, 0, 235, 236, 237, 238, 239, 240, + 241, 242, 280, 243, 244, 245, 246, 247, 248, 249, + 252, 253, 254, 255, 256, 257, 258, 259, 582, 250, + 251, 260, 261, 262, 263, 264, 265, 266, 267, 268, + 269, 270, 271, 272, 273, 0, 0, 0, 281, 282, + 283, 284, 0, 0, 275, 276, 277, 278, 0, 0, + 0, 465, 466, 467, 489, 0, 451, 513, 635, 0, + 0, 0, 0, 0, 0, 0, 563, 575, 609, 0, + 619, 620, 622, 624, 623, 626, 427, 0, 628, 629, + 630, 627, 365, 414, 432, 421, 0, 641, 504, 505, + 642, 615, 391, 0, 519, 552, 541, 625, 507, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 330, + 0, 0, 360, 556, 538, 548, 539, 524, 525, 526, + 533, 340, 527, 528, 529, 499, 530, 500, 531, 532, + 0, 555, 506, 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 222, 0, 0, 0, 0, 0, 0, 303, 223, 501, 621, 503, 502, @@ -6417,7 +5941,7 @@ var yyAct = [...]int{ 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 2171, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 293, 429, 447, 304, 418, 460, 309, 426, 299, 390, 415, 0, 0, 295, 445, 425, 372, 350, 351, @@ -6427,7 +5951,7 @@ var yyAct = [...]int{ 355, 356, 346, 399, 364, 400, 347, 377, 376, 378, 0, 0, 0, 0, 0, 483, 484, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 614, 0, 0, 618, 0, 457, 0, 0, 1553, 0, + 614, 0, 0, 618, 0, 457, 0, 0, 0, 0, 0, 0, 428, 0, 0, 357, 0, 0, 0, 473, 0, 412, 393, 644, 0, 0, 410, 362, 442, 401, 448, 430, 456, 406, 402, 288, 431, 327, 374, 300, @@ -6469,18 +5993,81 @@ var yyAct = [...]int{ 620, 622, 624, 623, 626, 427, 0, 628, 629, 630, 627, 365, 414, 432, 421, 0, 641, 504, 505, 642, 615, 391, 0, 519, 552, 541, 625, 507, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 655, 330, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 330, 0, 0, 360, 556, 538, 548, 539, 524, 525, 526, 533, 340, 527, 528, 529, 499, 530, 500, 531, 532, 0, 555, 506, 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 222, 0, 0, 2608, + 0, 0, 0, 303, 223, 501, 621, 503, 502, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 306, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 293, 429, 447, 304, 418, 460, 309, 426, 299, 390, + 415, 0, 0, 295, 445, 425, 372, 350, 351, 294, + 0, 409, 328, 342, 325, 388, 0, 444, 472, 324, + 463, 0, 455, 297, 0, 454, 387, 441, 446, 373, + 367, 0, 296, 443, 371, 366, 354, 332, 488, 355, + 356, 346, 399, 364, 400, 347, 377, 376, 378, 0, + 0, 0, 0, 0, 483, 484, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 614, + 0, 0, 618, 0, 457, 0, 0, 0, 0, 0, + 0, 428, 0, 0, 357, 0, 0, 0, 473, 0, + 412, 393, 644, 0, 0, 410, 362, 442, 401, 448, + 430, 456, 406, 402, 288, 431, 327, 374, 300, 302, + 322, 329, 331, 333, 334, 383, 384, 396, 417, 433, + 434, 435, 326, 310, 411, 311, 344, 312, 289, 318, + 316, 319, 419, 320, 291, 397, 439, 0, 339, 407, + 370, 292, 369, 398, 438, 437, 301, 464, 470, 471, + 560, 0, 476, 645, 646, 647, 485, 0, 403, 490, + 491, 492, 494, 495, 496, 497, 561, 578, 545, 515, + 478, 569, 512, 516, 517, 581, 0, 0, 0, 469, + 358, 359, 0, 337, 285, 286, 640, 323, 389, 583, + 616, 617, 508, 0, 570, 509, 518, 315, 542, 554, + 553, 385, 468, 0, 565, 568, 498, 639, 0, 562, + 577, 643, 576, 636, 395, 0, 416, 574, 521, 0, + 566, 540, 0, 567, 536, 571, 0, 510, 0, 424, + 450, 462, 479, 482, 511, 596, 597, 598, 290, 481, + 600, 601, 602, 603, 604, 605, 606, 599, 453, 543, + 520, 546, 461, 523, 522, 0, 0, 557, 477, 558, + 559, 379, 380, 381, 382, 341, 584, 308, 480, 405, + 0, 544, 0, 0, 0, 0, 0, 0, 0, 0, + 549, 550, 547, 648, 0, 607, 608, 0, 0, 474, + 475, 336, 343, 493, 345, 307, 394, 338, 459, 352, + 0, 486, 551, 487, 610, 613, 611, 612, 386, 348, + 349, 420, 353, 363, 408, 458, 392, 413, 305, 449, + 422, 368, 537, 564, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 592, 591, 590, 589, 588, 587, 586, 585, 0, 0, + 534, 436, 317, 279, 313, 314, 321, 637, 634, 440, + 638, 0, 287, 514, 361, 0, 404, 335, 579, 580, + 0, 0, 235, 236, 237, 238, 239, 240, 241, 242, + 280, 243, 244, 245, 246, 247, 248, 249, 252, 253, + 254, 255, 256, 257, 258, 259, 582, 250, 251, 260, + 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, + 271, 272, 273, 0, 0, 0, 281, 282, 283, 284, + 0, 0, 275, 276, 277, 278, 0, 0, 0, 465, + 466, 467, 489, 0, 451, 513, 635, 0, 0, 0, + 0, 0, 0, 0, 563, 575, 609, 0, 619, 620, + 622, 624, 623, 626, 427, 0, 628, 629, 630, 627, + 365, 414, 432, 421, 0, 641, 504, 505, 642, 615, + 391, 0, 519, 552, 541, 625, 507, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 330, 0, 0, + 360, 556, 538, 548, 539, 524, 525, 526, 533, 340, + 527, 528, 529, 499, 530, 500, 531, 532, 0, 555, + 506, 423, 375, 573, 572, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 222, 0, 0, 0, 0, 0, 0, 303, 223, 501, 621, 503, 502, 0, 0, 0, 0, 0, 0, 0, 0, 0, 306, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 2569, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 293, 429, 447, 304, 418, 460, 309, 426, 299, 390, 415, 0, 0, 295, 445, 425, 372, 350, 351, 294, 0, @@ -6537,6 +6124,69 @@ var yyAct = [...]int{ 528, 529, 499, 530, 500, 531, 532, 0, 555, 506, 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 222, 0, 0, 2567, 0, 0, + 0, 303, 223, 501, 621, 503, 502, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 306, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 293, 429, + 447, 304, 418, 460, 309, 426, 299, 390, 415, 0, + 0, 295, 445, 425, 372, 350, 351, 294, 0, 409, + 328, 342, 325, 388, 0, 444, 472, 324, 463, 0, + 455, 297, 0, 454, 387, 441, 446, 373, 367, 0, + 296, 443, 371, 366, 354, 332, 488, 355, 356, 346, + 399, 364, 400, 347, 377, 376, 378, 0, 0, 0, + 0, 0, 483, 484, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 614, 0, 0, + 618, 0, 457, 0, 0, 0, 0, 0, 0, 428, + 0, 0, 357, 0, 0, 0, 473, 0, 412, 393, + 644, 0, 0, 410, 362, 442, 401, 448, 430, 456, + 406, 402, 288, 431, 327, 374, 300, 302, 322, 329, + 331, 333, 334, 383, 384, 396, 417, 433, 434, 435, + 326, 310, 411, 311, 344, 312, 289, 318, 316, 319, + 419, 320, 291, 397, 439, 0, 339, 407, 370, 292, + 369, 398, 438, 437, 301, 464, 470, 471, 560, 0, + 476, 645, 646, 647, 485, 0, 403, 490, 491, 492, + 494, 495, 496, 497, 561, 578, 545, 515, 478, 569, + 512, 516, 517, 581, 0, 0, 0, 469, 358, 359, + 0, 337, 285, 286, 640, 323, 389, 583, 616, 617, + 508, 0, 570, 509, 518, 315, 542, 554, 553, 385, + 468, 0, 565, 568, 498, 639, 0, 562, 577, 643, + 576, 636, 395, 0, 416, 574, 521, 0, 566, 540, + 0, 567, 536, 571, 0, 510, 0, 424, 450, 462, + 479, 482, 511, 596, 597, 598, 290, 481, 600, 601, + 602, 603, 604, 605, 606, 599, 453, 543, 520, 546, + 461, 523, 522, 0, 0, 557, 477, 558, 559, 379, + 380, 381, 382, 341, 584, 308, 480, 405, 0, 544, + 0, 0, 0, 0, 0, 0, 0, 0, 549, 550, + 547, 648, 0, 607, 608, 0, 0, 474, 475, 336, + 343, 493, 345, 307, 394, 338, 459, 352, 0, 486, + 551, 487, 610, 613, 611, 612, 386, 348, 349, 420, + 353, 363, 408, 458, 392, 413, 305, 449, 422, 368, + 537, 564, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 592, 591, + 590, 589, 588, 587, 586, 585, 0, 0, 534, 436, + 317, 279, 313, 314, 321, 637, 634, 440, 638, 0, + 287, 514, 361, 0, 404, 335, 579, 580, 0, 0, + 235, 236, 237, 238, 239, 240, 241, 242, 280, 243, + 244, 245, 246, 247, 248, 249, 252, 253, 254, 255, + 256, 257, 258, 259, 582, 250, 251, 260, 261, 262, + 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, + 273, 0, 0, 0, 281, 282, 283, 284, 0, 0, + 275, 276, 277, 278, 0, 0, 0, 465, 466, 467, + 489, 0, 451, 513, 635, 0, 0, 0, 0, 0, + 0, 0, 563, 575, 609, 0, 619, 620, 622, 624, + 623, 626, 427, 0, 628, 629, 630, 627, 365, 414, + 432, 421, 2340, 641, 504, 505, 642, 615, 391, 0, + 519, 552, 541, 625, 507, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 330, 0, 0, 360, 556, + 538, 548, 539, 524, 525, 526, 533, 340, 527, 528, + 529, 499, 530, 500, 531, 532, 0, 555, 506, 423, + 375, 573, 572, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 222, 0, 0, 0, 0, 0, 0, 303, 223, 501, 621, 503, 502, 0, 0, 0, 0, 0, 0, 0, 0, 0, 306, 0, 0, 0, 0, @@ -6552,7 +6202,7 @@ var yyAct = [...]int{ 443, 371, 366, 354, 332, 488, 355, 356, 346, 399, 364, 400, 347, 377, 376, 378, 0, 0, 0, 0, 0, 483, 484, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 614, 0, 665, 618, + 0, 0, 0, 0, 0, 0, 614, 0, 0, 618, 0, 457, 0, 0, 0, 0, 0, 0, 428, 0, 0, 357, 0, 0, 0, 473, 0, 412, 393, 644, 0, 0, 410, 362, 442, 401, 448, 430, 456, 406, @@ -6600,6 +6250,69 @@ var yyAct = [...]int{ 499, 530, 500, 531, 532, 0, 555, 506, 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 222, 0, 0, 0, 1886, 0, 0, 303, + 223, 501, 621, 503, 502, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 306, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 293, 429, 447, 304, + 418, 460, 309, 426, 299, 390, 415, 0, 0, 295, + 445, 425, 372, 350, 351, 294, 0, 409, 328, 342, + 325, 388, 0, 444, 472, 324, 463, 0, 455, 297, + 0, 454, 387, 441, 446, 373, 367, 0, 296, 443, + 371, 366, 354, 332, 488, 355, 356, 346, 399, 364, + 400, 347, 377, 376, 378, 0, 0, 0, 0, 0, + 483, 484, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 614, 0, 0, 618, 0, + 457, 0, 0, 0, 0, 0, 0, 428, 0, 0, + 357, 0, 0, 0, 473, 0, 412, 393, 644, 0, + 0, 410, 362, 442, 401, 448, 430, 456, 406, 402, + 288, 431, 327, 374, 300, 302, 322, 329, 331, 333, + 334, 383, 384, 396, 417, 433, 434, 435, 326, 310, + 411, 311, 344, 312, 289, 318, 316, 319, 419, 320, + 291, 397, 439, 0, 339, 407, 370, 292, 369, 398, + 438, 437, 301, 464, 470, 471, 560, 0, 476, 645, + 646, 647, 485, 0, 403, 490, 491, 492, 494, 495, + 496, 497, 561, 578, 545, 515, 478, 569, 512, 516, + 517, 581, 0, 0, 0, 469, 358, 359, 0, 337, + 285, 286, 640, 323, 389, 583, 616, 617, 508, 0, + 570, 509, 518, 315, 542, 554, 553, 385, 468, 0, + 565, 568, 498, 639, 0, 562, 577, 643, 576, 636, + 395, 0, 416, 574, 521, 0, 566, 540, 0, 567, + 536, 571, 0, 510, 0, 424, 450, 462, 479, 482, + 511, 596, 597, 598, 290, 481, 600, 601, 602, 603, + 604, 605, 606, 599, 453, 543, 520, 546, 461, 523, + 522, 0, 0, 557, 477, 558, 559, 379, 380, 381, + 382, 341, 584, 308, 480, 405, 0, 544, 0, 0, + 0, 0, 0, 0, 0, 0, 549, 550, 547, 648, + 0, 607, 608, 0, 0, 474, 475, 336, 343, 493, + 345, 307, 394, 338, 459, 352, 0, 486, 551, 487, + 610, 613, 611, 612, 386, 348, 349, 420, 353, 363, + 408, 458, 392, 413, 305, 449, 422, 368, 537, 564, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 592, 591, 590, 589, + 588, 587, 586, 585, 0, 0, 534, 436, 317, 279, + 313, 314, 321, 637, 634, 440, 638, 0, 287, 514, + 361, 0, 404, 335, 579, 580, 0, 0, 235, 236, + 237, 238, 239, 240, 241, 242, 280, 243, 244, 245, + 246, 247, 248, 249, 252, 253, 254, 255, 256, 257, + 258, 259, 582, 250, 251, 260, 261, 262, 263, 264, + 265, 266, 267, 268, 269, 270, 271, 272, 273, 0, + 0, 0, 281, 282, 283, 284, 0, 0, 275, 276, + 277, 278, 0, 0, 0, 465, 466, 467, 489, 0, + 451, 513, 635, 0, 0, 0, 0, 0, 0, 0, + 563, 575, 609, 0, 619, 620, 622, 624, 623, 626, + 427, 0, 628, 629, 630, 627, 365, 414, 432, 421, + 0, 641, 504, 505, 642, 615, 391, 0, 519, 552, + 541, 625, 507, 0, 2014, 0, 0, 0, 0, 0, + 0, 0, 0, 330, 0, 0, 360, 556, 538, 548, + 539, 524, 525, 526, 533, 340, 527, 528, 529, 499, + 530, 500, 531, 532, 0, 555, 506, 423, 375, 573, + 572, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 222, 0, 0, 0, 0, 0, 0, 303, 223, 501, 621, 503, 502, 0, 0, 0, 0, 0, 0, 0, 0, 0, 306, 0, 0, 0, 0, 0, 0, @@ -6644,7 +6357,7 @@ var yyAct = [...]int{ 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 592, 591, 590, 589, 588, - 587, 586, 585, 959, 0, 534, 436, 317, 279, 313, + 587, 586, 585, 0, 0, 534, 436, 317, 279, 313, 314, 321, 637, 634, 440, 638, 0, 287, 514, 361, 0, 404, 335, 579, 580, 0, 0, 235, 236, 237, 238, 239, 240, 241, 242, 280, 243, 244, 245, 246, @@ -6662,6 +6375,69 @@ var yyAct = [...]int{ 524, 525, 526, 533, 340, 527, 528, 529, 499, 530, 500, 531, 532, 0, 555, 506, 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 222, 0, 0, 1525, 0, 0, 0, 303, 223, 501, + 621, 503, 502, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 306, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 293, 429, 447, 304, 418, 460, + 309, 426, 299, 390, 415, 0, 0, 295, 445, 425, + 372, 350, 351, 294, 0, 409, 328, 342, 325, 388, + 0, 444, 472, 324, 463, 0, 455, 297, 0, 454, + 387, 441, 446, 373, 367, 0, 296, 443, 371, 366, + 354, 332, 488, 355, 356, 346, 399, 364, 400, 347, + 377, 376, 378, 0, 0, 0, 0, 0, 483, 484, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 614, 0, 0, 618, 0, 457, 0, + 0, 0, 0, 0, 0, 428, 0, 0, 357, 0, + 0, 0, 473, 0, 412, 393, 644, 0, 0, 410, + 362, 442, 401, 448, 430, 456, 1919, 402, 288, 431, + 327, 374, 300, 302, 322, 329, 331, 333, 334, 383, + 384, 396, 417, 433, 434, 435, 326, 310, 411, 311, + 344, 312, 289, 318, 316, 319, 419, 320, 291, 397, + 439, 0, 339, 407, 370, 292, 369, 398, 438, 437, + 301, 464, 470, 471, 560, 0, 476, 645, 646, 647, + 485, 0, 403, 490, 491, 492, 494, 495, 496, 497, + 561, 578, 545, 515, 478, 569, 512, 516, 517, 581, + 0, 0, 0, 469, 358, 359, 0, 337, 285, 286, + 640, 323, 389, 583, 616, 617, 508, 0, 570, 509, + 518, 315, 542, 554, 553, 385, 468, 0, 565, 568, + 498, 639, 0, 562, 577, 643, 576, 636, 395, 0, + 416, 574, 521, 0, 566, 540, 0, 567, 536, 571, + 0, 510, 0, 424, 450, 462, 479, 482, 511, 596, + 597, 598, 290, 481, 600, 601, 602, 603, 604, 605, + 606, 599, 453, 543, 520, 546, 461, 523, 522, 0, + 0, 557, 477, 558, 559, 379, 380, 381, 382, 341, + 584, 308, 480, 405, 0, 544, 0, 0, 0, 0, + 0, 0, 0, 0, 549, 550, 547, 648, 0, 607, + 608, 0, 0, 474, 475, 336, 343, 493, 345, 307, + 394, 338, 459, 352, 0, 486, 551, 487, 610, 613, + 611, 612, 386, 348, 349, 420, 353, 363, 408, 458, + 392, 413, 305, 449, 422, 368, 537, 564, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 274, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 592, 591, 590, 589, 588, 587, + 586, 585, 0, 0, 534, 436, 317, 279, 313, 314, + 321, 637, 634, 440, 638, 0, 287, 514, 361, 0, + 404, 335, 579, 580, 0, 0, 235, 236, 237, 238, + 239, 240, 241, 242, 280, 243, 244, 245, 246, 247, + 248, 249, 252, 253, 254, 255, 256, 257, 258, 259, + 582, 250, 251, 260, 261, 262, 263, 264, 265, 266, + 267, 268, 269, 270, 271, 272, 273, 0, 0, 0, + 281, 282, 283, 284, 0, 0, 275, 276, 277, 278, + 0, 0, 0, 465, 466, 467, 489, 0, 451, 513, + 635, 0, 0, 0, 0, 0, 0, 0, 563, 575, + 609, 0, 619, 620, 622, 624, 623, 626, 427, 0, + 628, 629, 630, 627, 365, 414, 432, 421, 0, 641, + 504, 505, 642, 615, 391, 0, 519, 552, 541, 625, + 507, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 330, 0, 0, 360, 556, 538, 548, 539, 524, + 525, 526, 533, 340, 527, 528, 529, 499, 530, 500, + 531, 532, 0, 555, 506, 423, 375, 573, 572, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 222, 0, 0, 0, 0, 0, 0, 303, 223, 501, 621, 503, 502, 0, 0, 0, 0, 0, 0, 0, 0, @@ -6679,7 +6455,7 @@ var yyAct = [...]int{ 376, 378, 0, 0, 0, 0, 0, 483, 484, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 614, 0, 0, 618, 0, 457, 0, 0, - 0, 0, 0, 0, 428, 0, 0, 357, 0, 0, + 1554, 0, 0, 0, 428, 0, 0, 357, 0, 0, 0, 473, 0, 412, 393, 644, 0, 0, 410, 362, 442, 401, 448, 430, 456, 406, 402, 288, 431, 327, 374, 300, 302, 322, 329, 331, 333, 334, 383, 384, @@ -6720,11 +6496,74 @@ var yyAct = [...]int{ 0, 619, 620, 622, 624, 623, 626, 427, 0, 628, 629, 630, 627, 365, 414, 432, 421, 0, 641, 504, 505, 642, 615, 391, 0, 519, 552, 541, 625, 507, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 655, 330, 0, 0, 360, 556, 538, 548, 539, 524, 525, 526, 533, 340, 527, 528, 529, 499, 530, 500, 531, 532, 0, 555, 506, 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 222, 0, + 0, 0, 0, 0, 0, 303, 223, 501, 621, 503, + 502, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 306, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 293, 429, 447, 304, 418, 460, 309, 426, + 299, 390, 415, 0, 0, 295, 445, 425, 372, 350, + 351, 294, 0, 409, 328, 342, 325, 388, 0, 444, + 472, 324, 463, 0, 455, 297, 0, 454, 387, 441, + 446, 373, 367, 0, 296, 443, 371, 366, 354, 332, + 488, 355, 356, 346, 399, 364, 400, 347, 377, 376, + 378, 0, 0, 0, 0, 0, 483, 484, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 614, 0, 0, 618, 0, 457, 0, 0, 0, + 0, 0, 0, 428, 0, 0, 357, 0, 0, 0, + 473, 0, 412, 393, 644, 0, 0, 410, 362, 442, + 401, 448, 430, 456, 406, 402, 288, 431, 327, 374, + 300, 302, 322, 329, 331, 333, 334, 383, 384, 396, + 417, 433, 434, 435, 326, 310, 411, 311, 344, 312, + 289, 318, 316, 319, 419, 320, 291, 397, 439, 0, + 339, 407, 370, 292, 369, 398, 438, 437, 301, 464, + 470, 471, 560, 0, 476, 645, 646, 647, 485, 0, + 403, 490, 491, 492, 494, 495, 496, 497, 561, 578, + 545, 515, 478, 569, 512, 516, 517, 581, 0, 0, + 0, 469, 358, 359, 0, 337, 285, 286, 640, 323, + 389, 583, 616, 617, 508, 0, 570, 509, 518, 315, + 542, 554, 553, 385, 468, 0, 565, 568, 498, 639, + 0, 562, 577, 643, 576, 636, 395, 0, 416, 574, + 521, 0, 566, 540, 0, 567, 536, 571, 0, 510, + 0, 424, 450, 462, 479, 482, 511, 596, 597, 598, + 290, 481, 600, 601, 602, 603, 604, 605, 606, 599, + 453, 543, 520, 546, 461, 523, 522, 0, 0, 557, + 477, 558, 559, 379, 380, 381, 382, 341, 584, 308, + 480, 405, 0, 544, 0, 0, 0, 0, 0, 0, + 0, 0, 549, 550, 547, 648, 0, 607, 608, 0, + 0, 474, 475, 336, 343, 493, 345, 307, 394, 338, + 459, 352, 0, 486, 551, 487, 610, 613, 611, 612, + 386, 348, 349, 420, 353, 363, 408, 458, 392, 413, + 305, 449, 422, 368, 537, 564, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 592, 591, 590, 589, 588, 587, 586, 585, + 0, 0, 534, 436, 317, 279, 313, 314, 321, 637, + 634, 440, 638, 0, 287, 514, 361, 0, 404, 335, + 579, 580, 0, 0, 235, 236, 237, 238, 239, 240, + 241, 242, 280, 243, 244, 245, 246, 247, 248, 249, + 252, 253, 254, 255, 256, 257, 258, 259, 582, 250, + 251, 260, 261, 262, 263, 264, 265, 266, 267, 268, + 269, 270, 271, 272, 273, 0, 0, 0, 281, 282, + 283, 284, 0, 0, 275, 276, 277, 278, 0, 0, + 0, 465, 466, 467, 489, 0, 451, 513, 635, 0, + 0, 0, 0, 0, 0, 0, 563, 575, 609, 0, + 619, 620, 622, 624, 623, 626, 427, 0, 628, 629, + 630, 627, 365, 414, 432, 421, 0, 641, 504, 505, + 642, 615, 391, 0, 519, 552, 541, 625, 507, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 330, + 0, 0, 360, 556, 538, 548, 539, 524, 525, 526, + 533, 340, 527, 528, 529, 499, 530, 500, 531, 532, + 0, 555, 506, 423, 375, 573, 572, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 222, 0, 0, 0, 0, 0, 0, 303, 223, 501, 621, 503, 502, 0, 0, 0, 0, 0, 0, 0, 0, 0, 306, @@ -6733,7 +6572,7 @@ var yyAct = [...]int{ 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 293, 429, 1503, 304, 418, 460, 309, 426, 299, + 0, 293, 429, 447, 304, 418, 460, 309, 426, 299, 390, 415, 0, 0, 295, 445, 425, 372, 350, 351, 294, 0, 409, 328, 342, 325, 388, 0, 444, 472, 324, 463, 0, 455, 297, 0, 454, 387, 441, 446, @@ -6741,7 +6580,7 @@ var yyAct = [...]int{ 355, 356, 346, 399, 364, 400, 347, 377, 376, 378, 0, 0, 0, 0, 0, 483, 484, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 614, 0, 0, 618, 0, 457, 0, 0, 0, 0, + 614, 0, 665, 618, 0, 457, 0, 0, 0, 0, 0, 0, 428, 0, 0, 357, 0, 0, 0, 473, 0, 412, 393, 644, 0, 0, 410, 362, 442, 401, 448, 430, 456, 406, 402, 288, 431, 327, 374, 300, @@ -6769,24 +6608,87 @@ var yyAct = [...]int{ 449, 422, 368, 537, 564, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 592, 591, 590, 589, 588, 587, 586, 585, 0, - 0, 534, 436, 317, 279, 313, 314, 321, 637, 634, - 440, 638, 0, 287, 514, 361, 0, 404, 335, 579, - 580, 0, 0, 235, 236, 237, 238, 239, 240, 241, - 242, 280, 243, 244, 245, 246, 247, 248, 249, 252, - 253, 254, 255, 256, 257, 258, 259, 582, 250, 251, - 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, - 270, 271, 272, 273, 0, 0, 0, 281, 282, 283, - 284, 0, 0, 275, 276, 277, 278, 0, 0, 0, - 465, 466, 467, 489, 0, 451, 513, 635, 0, 0, - 0, 0, 0, 0, 0, 563, 575, 609, 0, 619, - 620, 622, 624, 623, 626, 427, 0, 628, 629, 630, - 627, 365, 414, 432, 421, 0, 641, 504, 505, 642, - 615, 391, 0, 519, 552, 541, 625, 507, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 330, 0, - 0, 360, 556, 538, 548, 539, 524, 525, 526, 533, - 340, 527, 528, 529, 499, 530, 500, 531, 532, 0, - 555, 506, 423, 375, 573, 572, 0, 0, 0, 0, + 0, 592, 591, 590, 589, 588, 587, 586, 585, 0, + 0, 534, 436, 317, 279, 313, 314, 321, 637, 634, + 440, 638, 0, 287, 514, 361, 0, 404, 335, 579, + 580, 0, 0, 235, 236, 237, 238, 239, 240, 241, + 242, 280, 243, 244, 245, 246, 247, 248, 249, 252, + 253, 254, 255, 256, 257, 258, 259, 582, 250, 251, + 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, + 270, 271, 272, 273, 0, 0, 0, 281, 282, 283, + 284, 0, 0, 275, 276, 277, 278, 0, 0, 0, + 465, 466, 467, 489, 0, 451, 513, 635, 0, 0, + 0, 0, 0, 0, 0, 563, 575, 609, 0, 619, + 620, 622, 624, 623, 626, 427, 0, 628, 629, 630, + 627, 365, 414, 432, 421, 0, 641, 504, 505, 642, + 615, 391, 0, 519, 552, 541, 625, 507, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 330, 0, + 0, 360, 556, 538, 548, 539, 524, 525, 526, 533, + 340, 527, 528, 529, 499, 530, 500, 531, 532, 0, + 555, 506, 423, 375, 573, 572, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 222, 0, 0, 0, + 0, 0, 0, 303, 223, 501, 621, 503, 502, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 306, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 293, 429, 447, 304, 418, 460, 309, 426, 299, 390, + 415, 0, 0, 295, 445, 425, 372, 350, 351, 294, + 0, 409, 328, 342, 325, 388, 0, 444, 472, 324, + 463, 0, 455, 297, 0, 454, 387, 441, 446, 373, + 367, 0, 296, 443, 371, 366, 354, 332, 488, 355, + 356, 346, 399, 364, 400, 347, 377, 376, 378, 0, + 0, 0, 0, 0, 483, 484, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 614, + 0, 0, 618, 0, 457, 0, 0, 0, 0, 0, + 0, 428, 0, 0, 357, 0, 0, 0, 473, 0, + 412, 393, 644, 0, 0, 410, 362, 442, 401, 448, + 430, 456, 406, 402, 288, 431, 327, 374, 300, 302, + 322, 329, 331, 333, 334, 383, 384, 396, 417, 433, + 434, 435, 326, 310, 411, 311, 344, 312, 289, 318, + 316, 319, 419, 320, 291, 397, 439, 0, 339, 407, + 370, 292, 369, 398, 438, 437, 301, 464, 470, 471, + 560, 0, 476, 645, 646, 647, 485, 0, 403, 490, + 491, 492, 494, 495, 496, 497, 561, 578, 545, 515, + 478, 569, 512, 516, 517, 581, 0, 0, 0, 469, + 358, 359, 0, 337, 285, 286, 640, 323, 389, 583, + 616, 617, 508, 0, 570, 509, 518, 315, 542, 554, + 553, 385, 468, 0, 565, 568, 498, 639, 0, 562, + 577, 643, 576, 636, 395, 0, 416, 574, 521, 0, + 566, 540, 0, 567, 536, 571, 0, 510, 0, 424, + 450, 462, 479, 482, 511, 596, 597, 598, 290, 481, + 600, 601, 602, 603, 604, 605, 606, 599, 453, 543, + 520, 546, 461, 523, 522, 0, 0, 557, 477, 558, + 559, 379, 380, 381, 382, 341, 584, 308, 480, 405, + 0, 544, 0, 0, 0, 0, 0, 0, 0, 0, + 549, 550, 547, 648, 0, 607, 608, 0, 0, 474, + 475, 336, 343, 493, 345, 307, 394, 338, 459, 352, + 0, 486, 551, 487, 610, 613, 611, 612, 386, 348, + 349, 420, 353, 363, 408, 458, 392, 413, 305, 449, + 422, 368, 537, 564, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 592, 591, 590, 589, 588, 587, 586, 585, 960, 0, + 534, 436, 317, 279, 313, 314, 321, 637, 634, 440, + 638, 0, 287, 514, 361, 0, 404, 335, 579, 580, + 0, 0, 235, 236, 237, 238, 239, 240, 241, 242, + 280, 243, 244, 245, 246, 247, 248, 249, 252, 253, + 254, 255, 256, 257, 258, 259, 582, 250, 251, 260, + 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, + 271, 272, 273, 0, 0, 0, 281, 282, 283, 284, + 0, 0, 275, 276, 277, 278, 0, 0, 0, 465, + 466, 467, 489, 0, 451, 513, 635, 0, 0, 0, + 0, 0, 0, 0, 563, 575, 609, 0, 619, 620, + 622, 624, 623, 626, 427, 0, 628, 629, 630, 627, + 365, 414, 432, 421, 0, 641, 504, 505, 642, 615, + 391, 0, 519, 552, 541, 625, 507, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 330, 0, 0, + 360, 556, 538, 548, 539, 524, 525, 526, 533, 340, + 527, 528, 529, 499, 530, 500, 531, 532, 0, 555, + 506, 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 222, 0, 0, 0, 0, 0, 0, 303, 223, 501, 621, 503, 502, 0, 0, @@ -6796,7 +6698,7 @@ var yyAct = [...]int{ 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 293, - 429, 1501, 304, 418, 460, 309, 426, 299, 390, 415, + 429, 447, 304, 418, 460, 309, 426, 299, 390, 415, 0, 0, 295, 445, 425, 372, 350, 351, 294, 0, 409, 328, 342, 325, 388, 0, 444, 472, 324, 463, 0, 455, 297, 0, 454, 387, 441, 446, 373, 367, @@ -6851,6 +6753,69 @@ var yyAct = [...]int{ 528, 529, 499, 530, 500, 531, 532, 0, 555, 506, 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 222, 0, 0, 0, 0, 0, + 0, 303, 223, 501, 621, 503, 502, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 306, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 293, 429, + 1504, 304, 418, 460, 309, 426, 299, 390, 415, 0, + 0, 295, 445, 425, 372, 350, 351, 294, 0, 409, + 328, 342, 325, 388, 0, 444, 472, 324, 463, 0, + 455, 297, 0, 454, 387, 441, 446, 373, 367, 0, + 296, 443, 371, 366, 354, 332, 488, 355, 356, 346, + 399, 364, 400, 347, 377, 376, 378, 0, 0, 0, + 0, 0, 483, 484, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 614, 0, 0, + 618, 0, 457, 0, 0, 0, 0, 0, 0, 428, + 0, 0, 357, 0, 0, 0, 473, 0, 412, 393, + 644, 0, 0, 410, 362, 442, 401, 448, 430, 456, + 406, 402, 288, 431, 327, 374, 300, 302, 322, 329, + 331, 333, 334, 383, 384, 396, 417, 433, 434, 435, + 326, 310, 411, 311, 344, 312, 289, 318, 316, 319, + 419, 320, 291, 397, 439, 0, 339, 407, 370, 292, + 369, 398, 438, 437, 301, 464, 470, 471, 560, 0, + 476, 645, 646, 647, 485, 0, 403, 490, 491, 492, + 494, 495, 496, 497, 561, 578, 545, 515, 478, 569, + 512, 516, 517, 581, 0, 0, 0, 469, 358, 359, + 0, 337, 285, 286, 640, 323, 389, 583, 616, 617, + 508, 0, 570, 509, 518, 315, 542, 554, 553, 385, + 468, 0, 565, 568, 498, 639, 0, 562, 577, 643, + 576, 636, 395, 0, 416, 574, 521, 0, 566, 540, + 0, 567, 536, 571, 0, 510, 0, 424, 450, 462, + 479, 482, 511, 596, 597, 598, 290, 481, 600, 601, + 602, 603, 604, 605, 606, 599, 453, 543, 520, 546, + 461, 523, 522, 0, 0, 557, 477, 558, 559, 379, + 380, 381, 382, 341, 584, 308, 480, 405, 0, 544, + 0, 0, 0, 0, 0, 0, 0, 0, 549, 550, + 547, 648, 0, 607, 608, 0, 0, 474, 475, 336, + 343, 493, 345, 307, 394, 338, 459, 352, 0, 486, + 551, 487, 610, 613, 611, 612, 386, 348, 349, 420, + 353, 363, 408, 458, 392, 413, 305, 449, 422, 368, + 537, 564, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 592, 591, + 590, 589, 588, 587, 586, 585, 0, 0, 534, 436, + 317, 279, 313, 314, 321, 637, 634, 440, 638, 0, + 287, 514, 361, 0, 404, 335, 579, 580, 0, 0, + 235, 236, 237, 238, 239, 240, 241, 242, 280, 243, + 244, 245, 246, 247, 248, 249, 252, 253, 254, 255, + 256, 257, 258, 259, 582, 250, 251, 260, 261, 262, + 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, + 273, 0, 0, 0, 281, 282, 283, 284, 0, 0, + 275, 276, 277, 278, 0, 0, 0, 465, 466, 467, + 489, 0, 451, 513, 635, 0, 0, 0, 0, 0, + 0, 0, 563, 575, 609, 0, 619, 620, 622, 624, + 623, 626, 427, 0, 628, 629, 630, 627, 365, 414, + 432, 421, 0, 641, 504, 505, 642, 615, 391, 0, + 519, 552, 541, 625, 507, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 330, 0, 0, 360, 556, + 538, 548, 539, 524, 525, 526, 533, 340, 527, 528, + 529, 499, 530, 500, 531, 532, 0, 555, 506, 423, + 375, 573, 572, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 222, 0, 0, 0, 0, 0, 0, 303, 223, 501, 621, 503, 502, 0, 0, 0, 0, 0, 0, 0, 0, 0, 306, 0, 0, 0, 0, @@ -6858,7 +6823,7 @@ var yyAct = [...]int{ 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 293, 429, 447, + 0, 0, 0, 0, 0, 0, 0, 293, 429, 1502, 304, 418, 460, 309, 426, 299, 390, 415, 0, 0, 295, 445, 425, 372, 350, 351, 294, 0, 409, 328, 342, 325, 388, 0, 444, 472, 324, 463, 0, 455, @@ -6870,7 +6835,7 @@ var yyAct = [...]int{ 0, 457, 0, 0, 0, 0, 0, 0, 428, 0, 0, 357, 0, 0, 0, 473, 0, 412, 393, 644, 0, 0, 410, 362, 442, 401, 448, 430, 456, 406, - 402, 288, 431, 327, 374, 300, 302, 733, 329, 331, + 402, 288, 431, 327, 374, 300, 302, 322, 329, 331, 333, 334, 383, 384, 396, 417, 433, 434, 435, 326, 310, 411, 311, 344, 312, 289, 318, 316, 319, 419, 320, 291, 397, 439, 0, 339, 407, 370, 292, 369, @@ -6914,6 +6879,69 @@ var yyAct = [...]int{ 499, 530, 500, 531, 532, 0, 555, 506, 423, 375, 573, 572, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 222, 0, 0, 0, 0, 0, 0, 303, + 223, 501, 621, 503, 502, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 306, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 293, 429, 447, 304, + 418, 460, 309, 426, 299, 390, 415, 0, 0, 295, + 445, 425, 372, 350, 351, 294, 0, 409, 328, 342, + 325, 388, 0, 444, 472, 324, 463, 0, 455, 297, + 0, 454, 387, 441, 446, 373, 367, 0, 296, 443, + 371, 366, 354, 332, 488, 355, 356, 346, 399, 364, + 400, 347, 377, 376, 378, 0, 0, 0, 0, 0, + 483, 484, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 614, 0, 0, 618, 0, + 457, 0, 0, 0, 0, 0, 0, 428, 0, 0, + 357, 0, 0, 0, 473, 0, 412, 393, 644, 0, + 0, 410, 362, 442, 401, 448, 430, 456, 406, 402, + 288, 431, 327, 374, 300, 302, 733, 329, 331, 333, + 334, 383, 384, 396, 417, 433, 434, 435, 326, 310, + 411, 311, 344, 312, 289, 318, 316, 319, 419, 320, + 291, 397, 439, 0, 339, 407, 370, 292, 369, 398, + 438, 437, 301, 464, 470, 471, 560, 0, 476, 645, + 646, 647, 485, 0, 403, 490, 491, 492, 494, 495, + 496, 497, 561, 578, 545, 515, 478, 569, 512, 516, + 517, 581, 0, 0, 0, 469, 358, 359, 0, 337, + 285, 286, 640, 323, 389, 583, 616, 617, 508, 0, + 570, 509, 518, 315, 542, 554, 553, 385, 468, 0, + 565, 568, 498, 639, 0, 562, 577, 643, 576, 636, + 395, 0, 416, 574, 521, 0, 566, 540, 0, 567, + 536, 571, 0, 510, 0, 424, 450, 462, 479, 482, + 511, 596, 597, 598, 290, 481, 600, 601, 602, 603, + 604, 605, 606, 599, 453, 543, 520, 546, 461, 523, + 522, 0, 0, 557, 477, 558, 559, 379, 380, 381, + 382, 341, 584, 308, 480, 405, 0, 544, 0, 0, + 0, 0, 0, 0, 0, 0, 549, 550, 547, 648, + 0, 607, 608, 0, 0, 474, 475, 336, 343, 493, + 345, 307, 394, 338, 459, 352, 0, 486, 551, 487, + 610, 613, 611, 612, 386, 348, 349, 420, 353, 363, + 408, 458, 392, 413, 305, 449, 422, 368, 537, 564, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 592, 591, 590, 589, + 588, 587, 586, 585, 0, 0, 534, 436, 317, 279, + 313, 314, 321, 637, 634, 440, 638, 0, 287, 514, + 361, 0, 404, 335, 579, 580, 0, 0, 235, 236, + 237, 238, 239, 240, 241, 242, 280, 243, 244, 245, + 246, 247, 248, 249, 252, 253, 254, 255, 256, 257, + 258, 259, 582, 250, 251, 260, 261, 262, 263, 264, + 265, 266, 267, 268, 269, 270, 271, 272, 273, 0, + 0, 0, 281, 282, 283, 284, 0, 0, 275, 276, + 277, 278, 0, 0, 0, 465, 466, 467, 489, 0, + 451, 513, 635, 0, 0, 0, 0, 0, 0, 0, + 563, 575, 609, 0, 619, 620, 622, 624, 623, 626, + 427, 0, 628, 629, 630, 627, 365, 414, 432, 421, + 0, 641, 504, 505, 642, 615, 391, 0, 519, 552, + 541, 625, 507, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 330, 0, 0, 360, 556, 538, 548, + 539, 524, 525, 526, 533, 340, 527, 528, 529, 499, + 530, 500, 531, 532, 0, 555, 506, 423, 375, 573, + 572, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 222, 0, 0, 0, 0, 0, 0, 303, 223, 501, 621, 503, 502, 0, 0, 0, 0, 0, 0, 0, 0, 0, 306, 0, 0, 0, 0, 0, 0, @@ -6969,128 +6997,102 @@ var yyAct = [...]int{ 278, 0, 0, 0, 465, 466, 467, 489, 0, 451, 513, 635, 0, 0, 0, 0, 0, 0, 0, 563, 575, 609, 0, 619, 620, 622, 624, 623, 626, 427, - 0, 628, 629, 630, 627, 365, 414, 432, 421, 0, - 641, 504, 505, 642, 615, 712, 711, 718, 708, 0, - 0, 0, 0, 0, 0, 0, 0, 715, 716, 0, - 717, 721, 2000, 0, 702, 712, 711, 718, 708, 200, - 0, 0, 0, 0, 726, 0, 0, 715, 716, 0, - 717, 721, 0, 0, 702, 0, 0, 0, 0, 0, - 0, 3581, 0, 0, 726, 0, 0, 2003, 0, 0, - 2000, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 730, - 0, 0, 732, 0, 0, 0, 0, 731, 0, 0, - 0, 0, 0, 0, 0, 2003, 0, 0, 0, 730, - 196, 0, 732, 0, 0, 0, 0, 731, 0, 0, - 1977, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 628, 629, 630, 627, 365, 414, 432, 421, 2001, + 641, 504, 505, 642, 615, 0, 200, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 3755, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1977, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 3587, 0, + 0, 0, 0, 0, 2004, 0, 0, 2001, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1993, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 2000, 0, 0, 0, 0, + 0, 0, 2004, 0, 0, 2001, 0, 0, 196, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1978, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1993, 0, 0, 0, - 0, 0, 0, 0, 0, 703, 705, 704, 0, 0, - 2003, 0, 0, 0, 0, 0, 710, 0, 0, 0, - 0, 0, 0, 0, 0, 703, 705, 704, 714, 0, - 0, 0, 0, 0, 0, 729, 710, 0, 0, 0, - 1981, 0, 707, 0, 0, 0, 697, 0, 714, 0, - 0, 1987, 0, 0, 0, 729, 0, 0, 0, 0, - 0, 0, 707, 1977, 0, 0, 0, 0, 0, 0, - 0, 1975, 2009, 0, 0, 1976, 1978, 1980, 1981, 1982, - 1983, 1984, 1988, 1989, 1990, 1992, 1995, 1996, 1997, 1987, - 0, 0, 0, 0, 0, 0, 1985, 1994, 1986, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 2000, 1975, - 2009, 0, 0, 1976, 1978, 1980, 0, 1982, 1983, 1984, - 1988, 1989, 1990, 1992, 1995, 1996, 1997, 3726, 0, 0, - 0, 1993, 2001, 0, 1985, 1994, 1986, 0, 0, 0, - 0, 0, 0, 2003, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 709, 713, 719, 0, 720, 722, - 0, 0, 723, 724, 725, 0, 0, 727, 728, 0, - 2001, 0, 0, 0, 709, 713, 719, 1998, 720, 722, - 0, 0, 723, 724, 725, 0, 0, 727, 728, 0, - 0, 0, 0, 0, 1974, 0, 1977, 0, 0, 0, - 0, 1973, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1981, 0, 1998, 0, 0, 0, 0, - 0, 0, 0, 0, 1987, 1991, 0, 0, 0, 0, - 0, 0, 1974, 0, 1979, 0, 0, 0, 0, 1973, - 0, 0, 0, 0, 1975, 2009, 0, 0, 1976, 1978, - 1980, 0, 1982, 1983, 1984, 1988, 1989, 1990, 1992, 1995, - 1996, 1997, 0, 1991, 1993, 0, 0, 0, 0, 1985, - 1994, 1986, 1979, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 2004, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1978, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 2001, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 706, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1994, 0, 0, 0, + 0, 0, 0, 0, 1978, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 3585, 706, 0, 0, 1981, 0, 0, 0, - 1998, 0, 0, 0, 0, 0, 0, 1987, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1974, 0, 0, - 0, 0, 0, 0, 1973, 0, 0, 1975, 2009, 0, - 3589, 1976, 1978, 1980, 0, 1982, 1983, 1984, 1988, 1989, - 1990, 1992, 1995, 1996, 1997, 0, 0, 0, 1991, 0, - 0, 0, 1985, 1994, 1986, 0, 0, 1979, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 3732, 0, 0, 0, 1994, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 2001, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 1994, 0, 0, 0, 0, 0, 1982, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1988, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1976, + 2010, 0, 0, 1977, 1979, 1981, 1982, 1983, 1984, 1985, + 1989, 1990, 1991, 1993, 1996, 1997, 1998, 1988, 0, 0, + 0, 0, 0, 0, 1986, 1995, 1987, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1976, 2010, 0, + 0, 1977, 1979, 1981, 1982, 1983, 1984, 1985, 1989, 1990, + 1991, 1993, 1996, 1997, 1998, 1988, 0, 0, 0, 0, + 2002, 0, 1986, 1995, 1987, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1976, 2010, 0, 0, 1977, + 1979, 1981, 0, 1983, 1984, 1985, 1989, 1990, 1991, 1993, + 1996, 1997, 1998, 0, 0, 0, 0, 0, 2002, 0, + 1986, 1995, 1987, 0, 0, 1999, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1998, 0, 0, 0, 0, 0, 0, + 0, 0, 1975, 0, 0, 0, 0, 0, 0, 1974, + 0, 0, 0, 0, 0, 0, 2002, 0, 0, 0, + 0, 0, 0, 1999, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1992, 0, 0, 0, 0, 0, 0, + 1975, 0, 1980, 0, 0, 0, 0, 1974, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1974, 0, 0, 0, 0, 0, 0, 1973, 0, 0, - 0, 0, 0, 0, 0, 3589, 0, 0, 0, 0, + 0, 1999, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 1992, 0, 0, 0, 0, 0, 0, 1975, 0, + 1980, 0, 0, 0, 0, 1974, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1991, 0, 0, 0, 0, 0, 0, 0, 0, - 1979, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1992, + 0, 0, 0, 0, 0, 0, 0, 0, 1980, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 3591, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 3595, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 3589, + 0, 0, 0, 0, 0, 0, 3595, } var yyPact = [...]int{ - 4040, -1000, -1000, -1000, -323, 14998, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + 438, -1000, -1000, -1000, -323, 15199, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, 48942, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 470, 48942, - -318, 30730, 47058, -1000, -1000, 2742, -1000, 47686, 16902, 48942, - 554, 544, 48942, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, 49197, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 463, 49197, + -320, 30956, 47310, -1000, -1000, 2745, -1000, 47939, 17106, 49197, + 556, 537, 49197, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - 966, -1000, 51454, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, 900, 52092, 50826, 11203, -208, -1000, 1916, -26, - 2631, 488, -198, -202, 1150, 1169, 1144, 1144, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, 249, 983, 48314, -1000, -1000, -1000, -1000, -1000, -1000, + 943, -1000, 51713, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, 880, 4527, 51084, 11398, -210, -1000, 1793, -33, + 2568, 519, -193, -195, 1108, 1139, 1199, 1199, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 266, 251, - 961, 983, 21934, 115, 113, 1916, 478, -70, 234, -1000, - 1732, 4054, 203, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, 11203, 11203, 14998, -355, 14998, 11203, - 48942, 48942, -1000, -1000, -1000, -1000, -318, 47686, 900, 52092, - 11203, 2631, 488, -198, -202, -1000, -1000, -1000, -1000, -1000, + -1000, 233, 1000, 48568, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 3867, 265, + 942, 1000, 22146, 156, 155, 1793, 455, -59, 218, -1000, + 1519, 3926, 201, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, 11398, 11398, 15199, -360, 15199, 11398, + 49197, 49197, -1000, -1000, -1000, -1000, -320, 47939, 880, 4527, + 11398, 2568, 519, -193, -195, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -70, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -59, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, @@ -7105,8 +7107,8 @@ var yyPact = [...]int{ -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, 113, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, 155, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, @@ -7123,416 +7125,417 @@ var yyPact = [...]int{ -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, 5078, -1000, 1637, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 2384, - 3147, 1624, 2629, 839, 47058, 48942, -1000, 155, 839, -1000, - -1000, -1000, 1916, 3606, -1000, 48942, 48942, 206, 1958, -1000, - 577, 540, 436, 402, 354, 1606, -1000, -1000, -1000, -1000, - -1000, -1000, 731, 3562, -1000, 48942, 48942, 3161, 48942, -1000, - 2327, 773, -1000, 52112, 3330, 1473, 1016, 3183, -1000, -1000, - 3145, -1000, 387, 824, 734, 777, 465, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, 312, -1000, 3424, -1000, -1000, 341, - -1000, -1000, 329, -1000, -1000, -1000, 108, -1000, -1000, -1000, - -1000, -1000, -1000, 5, -1000, -1000, 1202, 2237, 11203, 2538, - -1000, 4001, 1663, -1000, -1000, -1000, 6780, 13727, 13727, 13727, - 13727, 48942, -1000, -1000, 2975, 11203, 3144, 3142, 3141, 3140, - -1000, -1000, -1000, -1000, -1000, -1000, 3137, 1590, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, 2074, -1000, -1000, - -1000, 14358, -1000, 3133, 3129, 3128, 3126, 3125, 3124, 3123, - 3119, 3113, 3111, 3110, 3107, 3106, 3105, 2823, 16264, 3089, - 2628, 2627, 3085, 3084, 3074, 2625, 3073, 3058, 3054, 2823, - 2823, 3053, 3050, 3049, 3048, 3046, 3045, 3043, 3042, 3037, - 3034, 3030, 3017, 3016, 3015, 3013, 3012, 3009, 3008, 3007, - 3004, 3003, 3001, 3000, 2999, 2988, 2987, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, 4930, -1000, 1597, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 2322, + 3120, 1588, 2567, 820, 47310, 49197, -1000, 149, 820, -1000, + -1000, -1000, 1793, 3517, -1000, 49197, 49197, 225, 1888, -1000, + 505, 425, 550, 413, 353, 1579, -1000, -1000, -1000, -1000, + -1000, -1000, 729, 3444, -1000, 49197, 49197, 3139, 49197, -1000, + 2326, 768, -1000, 4943, 3319, 1376, 983, 3147, -1000, -1000, + 3119, -1000, 364, 337, 362, 492, 459, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, 347, -1000, 3370, -1000, -1000, 348, + -1000, -1000, 359, -1000, -1000, -1000, 146, -1000, -1000, -1000, + -1000, -1000, -1000, 33, -1000, -1000, 1201, 2388, 11398, 2066, + -1000, 3586, 1635, -1000, -1000, -1000, 6968, 13926, 13926, 13926, + 13926, 49197, -1000, -1000, 2976, 11398, 3117, 3116, 3113, 3111, + -1000, -1000, -1000, -1000, -1000, -1000, 3109, 1578, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, 2048, -1000, -1000, + -1000, 14558, -1000, 3105, 3104, 3103, 3102, 3101, 3100, 3099, + 3098, 3097, 3096, 3095, 3093, 3091, 3085, 2752, 16467, 3081, + 2566, 2562, 3078, 3077, 3066, 2561, 3064, 3062, 3060, 2752, + 2752, 3056, 3055, 3054, 3053, 3051, 3050, 3049, 3047, 3046, + 3045, 3043, 3042, 3038, 3034, 3033, 3014, 3011, 3010, 3008, + 3007, 3006, 2999, 2995, 2989, 2988, 2987, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - 1480, -1000, 2985, 3572, 2895, -1000, 3403, 3397, 3386, 3369, - -254, 2983, 2306, -1000, -1000, 111, 3561, 48942, -277, 48942, - 438, -78, -80, -84, -1000, -64, -1000, -1000, 1104, -1000, - 1092, -1000, 897, 897, 897, 48942, 48942, 229, 1058, 897, - 897, 897, 897, 897, 943, 897, 3453, 960, 956, 953, - 952, 897, -35, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - 1954, 1945, 3260, 1074, -1000, -1000, -1000, -1000, 1491, 48942, - -1000, 2933, 438, -291, 1717, 1717, 3511, 3511, 3448, 816, - 812, 782, 1717, 600, -1000, 1873, 1873, 1873, 1873, 1717, - 526, 827, 3439, 3439, 121, 1873, 78, 1717, 1717, 78, - 1717, 1717, -1000, 1925, 272, -262, -1000, -1000, -1000, -1000, - 1873, 1873, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 3419, - 3415, 900, 900, 48942, 900, 223, 48942, 900, 900, 900, - 48942, 916, -310, 53, 50198, 49570, 2402, 2327, 768, 760, - 1506, 1862, -1000, 1914, 48942, 48942, 1914, 1914, 25078, 24450, - -1000, 48942, -1000, 3572, 2895, 2818, 1909, 2817, 2895, -92, - 438, 900, 900, 900, 900, 900, 321, 900, 900, 900, - 900, 900, 48942, 48942, 46430, 900, 900, 900, 900, 9304, - 9304, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - 14998, 2073, 2207, 202, -1, -301, 276, -1000, -1000, 48942, - 3311, 309, -1000, -1000, -1000, 2918, -1000, 2928, 2928, 2928, - 2928, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, 2928, 2928, 2932, 2982, -1000, -1000, 2927, 2927, 2927, - 2918, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, 2929, 2929, 2931, 2931, - 2929, 48942, -118, -1000, -1000, 11203, 48942, 3321, 426, 2981, - 839, -1000, -1000, 48942, 181, 428, 3572, 3318, 3439, 3514, - -1000, -1000, 1588, 2303, 2622, -1000, 354, -1000, 564, 354, - -1000, 636, 636, 1881, -1000, 1131, -1000, -1000, -1000, -1000, - -1000, -1000, 48942, 5, 429, -1000, -1000, 2605, 2980, -1000, - 650, 1463, 1554, -1000, 281, 665, 39522, 2327, 39522, 48942, - -1000, -1000, -1000, -1000, -1000, -1000, 105, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, 1365, -1000, 2986, 3461, 2809, -1000, 3352, 3350, 3348, + 3343, -262, 2985, 2257, -1000, -1000, 107, 3443, 49197, -281, + 49197, 423, -71, -74, -82, -1000, -30, -1000, -1000, 1174, + -1000, 1054, -1000, 874, 874, 874, 49197, 49197, 235, 931, + 874, 874, 874, 874, 874, 919, 874, 3382, 940, 935, + 933, 932, 874, -20, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, 1887, 1886, 3228, 1043, -1000, -1000, -1000, -1000, 1491, + 49197, -1000, 2921, 423, -308, 1751, 1751, 3426, 3426, 3381, + 790, 777, 773, 1751, 611, -1000, 1871, 1871, 1871, 1871, + 1751, 533, 831, 3387, 3387, 112, 1871, 106, 1751, 1751, + 106, 1751, 1751, -1000, 1804, 316, -270, -1000, -1000, -1000, + -1000, 1871, 1871, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + 3365, 3361, 880, 880, 49197, 880, 236, 49197, 880, 880, + 880, 49197, 883, -307, 72, 50455, 49826, 2276, 2326, 759, + 745, 1506, 1879, -1000, 1749, 49197, 49197, 1749, 1749, 25295, + 24666, -1000, 49197, -1000, 3461, 2809, 2746, 1836, 2739, 2809, + -92, 423, 880, 880, 880, 880, 880, 319, 880, 880, + 880, 880, 880, 49197, 49197, 46681, 880, 880, 880, 880, + 9496, 9496, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, 15199, 2094, 2160, 197, -8, -305, 277, -1000, -1000, + 49197, 3286, 325, -1000, -1000, -1000, 2894, -1000, 2901, 2901, + 2901, 2901, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, 2901, 2901, 2920, 2984, -1000, -1000, 2895, 2895, + 2895, 2894, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, 2917, 2917, 2919, + 2919, 2917, 49197, -119, -1000, -1000, 11398, 49197, 3308, 404, + 2983, 820, -1000, -1000, 49197, 185, 405, 3461, 3305, 3387, + 3419, -1000, -1000, 1572, 2255, 2552, -1000, 353, -1000, 416, + 353, -1000, 572, 572, 1785, -1000, 1170, -1000, -1000, -1000, + -1000, -1000, -1000, 49197, 33, 406, -1000, -1000, 2544, 2982, + -1000, 671, 1324, 1503, -1000, 314, 5045, 39762, 2326, 39762, + 49197, -1000, -1000, -1000, -1000, -1000, -1000, 145, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, 336, -1000, 11203, 11203, 11203, 11203, 11203, -1000, 755, - 13096, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 13727, 13727, - 13727, 13727, 13727, 13727, 13727, 13727, 13727, 13727, 13727, 13727, - 2972, 1981, 13727, 13727, 13727, 13727, 27590, 1909, 3508, 1496, - 317, 1663, 1663, 1663, 1663, 11203, -1000, 1984, 2237, 11203, - 11203, 11203, 11203, 33870, 48942, -1000, -1000, 4701, 11203, 11203, - 3741, 11203, 3364, 11203, 11203, 11203, 2815, 5509, 48942, 11203, - -1000, 2814, 2813, -1000, -1000, 2068, 11203, -1000, -1000, 11203, - -1000, -1000, 11203, 13727, 11203, -1000, 11203, 11203, 11203, -1000, - -1000, 1165, 3364, 3364, 3364, 1960, 11203, 11203, 3364, 3364, - 3364, 1902, 3364, 3364, 3364, 3364, 3364, 3364, 3364, 3364, - 3364, 3364, 3364, 2811, 2805, 2804, 10572, 3439, -208, -1000, - 8673, 3318, 3439, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -258, 2978, 48942, 2621, 2619, -333, 188, - 467, 48942, 1184, -1000, -1000, 48942, 2296, 48942, 2293, 62, - 1145, 1096, 1109, -1000, 48942, 1992, 48942, 48942, 3356, -1000, - 2977, 48942, 897, 897, 897, -1000, 44546, 39522, 48942, 48942, - 2327, 48942, 48942, 48942, 897, 897, 897, 897, 48942, -1000, - 3276, 39522, 3271, 916, -1000, 48942, 1491, 3355, 48942, -1000, - -1000, -1000, -1000, 3511, 13727, 13727, -1000, -1000, 11203, -1000, - 45802, 1873, 1717, 1717, -1000, -1000, 48942, -1000, -1000, -1000, - 1873, 48942, 1873, 1873, 3511, 1873, -1000, -1000, -1000, 1717, - 1717, -1000, -1000, 11203, -1000, -1000, 1873, 1873, -1000, -1000, - 3511, 48942, 103, 3511, 3511, 86, -1000, -1000, -1000, 1717, - 48942, 48942, 897, 48942, -1000, 48942, 48942, -1000, -1000, 48942, - 48942, 4474, 48942, 44546, 45174, 3407, -1000, 39522, 48942, 48942, - 1472, -1000, 938, 37010, -1000, 48942, 1416, -1000, 40, -1000, - 60, 53, 1914, 53, 1914, 935, -1000, 626, 761, 23194, - 575, 39522, 6140, -1000, -1000, 1914, 1914, 6140, 6140, 1674, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, 1461, -1000, 279, - 3439, -1000, -1000, -1000, -1000, -1000, 2284, -1000, 48942, 44546, - 39522, 2327, 48942, 900, 48942, 48942, 48942, 48942, 48942, -1000, - 2976, 1578, -1000, 3328, 48942, 48942, 48942, 48942, 1444, -1000, - -1000, 20042, 1576, 1444, -1000, 1987, -1000, 11203, 14998, -233, - 11203, 14998, 14998, 11203, 14998, -1000, 11203, 306, -1000, -1000, - -1000, -1000, 2283, -1000, 2278, -1000, -1000, -1000, -1000, -1000, - 2615, 2615, -1000, 2275, -1000, -1000, -1000, -1000, 2269, -1000, - -1000, 2267, -1000, -1000, -1000, -1000, -147, 2802, 1202, -1000, - 2614, 3182, -209, -1000, 21306, 48942, 48942, 426, -337, 1944, - 1943, 1940, -1000, -209, -1000, 20674, 48942, 3439, -1000, -212, - 3318, 11203, 48942, -1000, 3443, -1000, -1000, 354, -1000, -1000, - -1000, 636, 439, -1000, -1000, -1000, -1000, -1000, -1000, 1575, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - 457, -72, -73, 1456, -1000, 48942, -1000, -1000, 281, 39522, - 41406, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 268, -1000, - -1000, 192, -1000, 934, 263, 1880, -1000, -1000, 248, 220, - 211, 994, 2237, -1000, 1996, 1996, 2006, -1000, 729, -1000, - -1000, -1000, -1000, 2975, -1000, -1000, -1000, 2082, 3035, -1000, - 1928, 1928, 1678, 1678, 1678, 1678, 1678, 2281, 2281, -1000, - -1000, -1000, 6780, 2972, 13727, 13727, 13727, 13727, 975, 975, - 3669, 3267, -1000, -1000, -1000, -1000, 11203, 175, 1974, -1000, - 11203, 2726, 1826, 2618, 1753, 1879, -1000, 2918, 11203, 1573, + -1000, -1000, 330, -1000, 11398, 11398, 11398, 11398, 11398, -1000, + 783, 13294, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 13926, + 13926, 13926, 13926, 13926, 13926, 13926, 13926, 13926, 13926, 13926, + 13926, 2971, 1900, 13926, 13926, 13926, 13926, 27811, 1836, 3225, + 1501, 310, 1635, 1635, 1635, 1635, 11398, -1000, 1909, 2388, + 11398, 11398, 11398, 11398, 34101, 49197, -1000, -1000, 5031, 11398, + 11398, 4859, 11398, 3339, 11398, 11398, 11398, 2735, 5695, 49197, + 11398, -1000, 2734, 2732, -1000, -1000, 2080, 11398, -1000, -1000, + 11398, -1000, -1000, 11398, 13926, 11398, -1000, 11398, 11398, 11398, + -1000, -1000, 1457, 3339, 3339, 3339, 1841, 11398, 11398, 3339, + 3339, 3339, 1815, 3339, 3339, 3339, 3339, 3339, 3339, 3339, + 3339, 3339, 3339, 3339, 2731, 2729, 2723, 10766, 3387, -210, + -1000, 8864, 3305, 3387, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -264, 2980, 49197, 2551, 2550, -336, + 182, 478, 49197, 1165, -1000, -1000, 49197, 2245, 49197, 2243, + 81, 1101, 1057, 1065, -1000, 49197, 1921, 49197, 49197, 3335, + -1000, 2978, 49197, 874, 874, 874, -1000, 44794, 39762, 49197, + 49197, 2326, 49197, 49197, 49197, 874, 874, 874, 874, 49197, + -1000, 3263, 39762, 3237, 883, -1000, 49197, 1491, 3333, 49197, + -1000, -1000, -1000, -1000, 3426, 13926, 13926, -1000, -1000, 11398, + -1000, 46052, 1871, 1751, 1751, -1000, -1000, 49197, -1000, -1000, + -1000, 1871, 49197, 1871, 1871, 3426, 1871, -1000, -1000, -1000, + 1751, 1751, -1000, -1000, 11398, -1000, -1000, 1871, 1871, -1000, + -1000, 3426, 49197, 134, 3426, 3426, 125, -1000, -1000, -1000, + 1751, 49197, 49197, 874, 49197, -1000, 49197, 49197, -1000, -1000, + 49197, 49197, 4358, 49197, 44794, 45423, 3360, -1000, 39762, 49197, + 49197, 1487, -1000, 914, 37246, -1000, 49197, 1427, -1000, 62, + -1000, 66, 72, 1749, 72, 1749, 913, -1000, 669, 651, + 23408, 593, 39762, 6327, -1000, -1000, 1749, 1749, 6327, 6327, + 1651, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 1484, -1000, + 340, 3387, -1000, -1000, -1000, -1000, -1000, 2237, -1000, 49197, + 44794, 39762, 2326, 49197, 880, 49197, 49197, 49197, 49197, 49197, + -1000, 2977, 1571, -1000, 3317, 49197, 49197, 49197, 49197, 1405, + -1000, -1000, 20251, 1569, 1405, -1000, 1979, -1000, 11398, 15199, + -243, 11398, 15199, 15199, 11398, 15199, -1000, 11398, 306, -1000, + -1000, -1000, -1000, 2234, -1000, 2233, -1000, -1000, -1000, -1000, + -1000, 2540, 2540, -1000, 2231, -1000, -1000, -1000, -1000, 2225, + -1000, -1000, 2222, -1000, -1000, -1000, -1000, -152, 2721, 1201, + -1000, 2535, 3145, -211, -1000, 21517, 49197, 49197, 404, -341, + 1883, 1882, 1880, -1000, -211, -1000, 20884, 49197, 3387, -1000, + -216, 3305, 11398, 49197, -1000, 3380, -1000, -1000, 353, -1000, + -1000, -1000, 572, 441, -1000, -1000, -1000, -1000, -1000, -1000, + 1556, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, 458, -60, -65, 1480, -1000, 49197, -1000, -1000, 314, + 39762, 41649, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 254, + -1000, -1000, 196, -1000, 907, 279, 1781, -1000, -1000, 246, + 215, 234, 982, 2388, -1000, 1963, 1963, 1948, -1000, 758, + -1000, -1000, -1000, -1000, 2976, -1000, -1000, -1000, 1902, 2964, + -1000, 1878, 1878, 1655, 1655, 1655, 1655, 1655, 1926, 1926, + -1000, -1000, -1000, 6968, 2971, 13926, 13926, 13926, 13926, 963, + 963, 4564, 5201, -1000, -1000, -1000, -1000, 11398, 188, 1942, + -1000, 11398, 2502, 1627, 2497, 1423, 1772, -1000, 2894, 11398, + 1552, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, 2715, 2707, 2401, 3442, 2706, 11398, -1000, -1000, 1769, + 1766, 1765, -1000, 2273, 10134, -1000, -1000, -1000, 2705, 1546, + 2704, -1000, -1000, -1000, 2701, 1764, 1278, 2699, 2871, 2698, + 2695, 2689, 2687, 1470, 11398, 11398, 11398, 11398, 2683, 1760, + 1754, 11398, 11398, 11398, 11398, 2680, 11398, 11398, 11398, 11398, + 11398, 11398, 11398, 11398, 11398, 11398, 49197, 169, 169, 169, + 1461, 1460, -1000, -1000, 1753, -1000, 2388, -1000, -1000, 3305, + -1000, 2969, 2217, 1456, -1000, -1000, -317, 2468, 49197, 49197, + 180, 49197, 2533, -284, 49197, -1000, -1000, 2531, -1000, -1000, + -1000, 1090, 1051, 1119, 2529, 3316, 3377, 949, 49197, 1188, + 2968, 49197, 49197, 49197, 300, -1000, -1000, 1353, -1000, 279, + 16, 568, 1252, 3138, 3441, -120, 49197, 49197, 49197, 49197, + 3331, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 44165, + -1000, 2967, 1747, -1000, -1000, 1635, 1635, 2388, 3135, 49197, + 49197, 3426, 3426, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + 1871, 3426, 3426, 1464, 1751, 1871, -1000, -1000, 1871, -341, + -1000, 1871, -1000, -341, 1541, -341, 49197, -1000, -1000, -1000, + 3330, 2921, 1455, -1000, -1000, -1000, 3417, 1708, 855, 855, + 1072, 799, 3415, 18993, -1000, 1734, 1080, 904, 3255, 355, + -1000, 1734, -148, 835, 1734, 1734, 1734, 1734, 1734, 1734, + 1734, 714, 710, 1734, 1734, 1734, 1734, 1734, 1734, 1734, + 1734, 1734, 1734, 1734, 1123, 1734, 1734, 1734, 1734, 1734, + -1000, 1734, 2966, 548, -1000, -1000, -1000, -1000, -1000, -1000, + 782, 668, 292, 3359, 408, -1000, 414, 1353, 323, 3358, + 450, 49197, 49197, 3537, 1298, -1000, -1000, -1000, -1000, -1000, + 28440, 28440, 22779, 28440, -1000, 208, 1749, 72, 56, -1000, + -1000, 1427, 6327, 1427, 6327, 2214, -1000, -1000, 902, -1000, + -1000, 1252, -1000, 49197, 49197, -1000, -1000, 2965, 1874, -1000, + -1000, 16467, -1000, 6327, 6327, -1000, -1000, 30327, 49197, -1000, + 31, -1000, 38, 3305, -1000, 1248, -1000, -1000, 1412, 1252, + 3144, 49197, 1248, 1248, 1248, -1000, -1000, 17735, 49197, 49197, + -1000, -1000, -1000, -1000, 3426, 9496, -1000, 37246, -1000, -1000, + 43536, -1000, 42907, 3426, 1844, -1000, 15199, 2139, 195, -1000, + 256, -310, 194, 1971, 192, 2388, -1000, -1000, 2678, 2676, + 1732, -1000, 1718, 2671, 1711, 1710, 2213, -1000, 78, -1000, + 3292, 1297, -1000, 2960, -1000, 1707, 3219, -1000, 1409, -1000, + 1873, 1704, -1000, -1000, -1000, 11398, 42278, 11398, 1297, 1690, + 3214, 1409, 3305, 2514, -1000, 1408, -1000, 2193, 1540, 223, + -1000, -1000, -1000, 49197, 880, 2544, 1673, 41649, 1317, -1000, + 899, 1536, 1532, -1000, 39762, 339, 39762, -1000, 39762, -1000, + -1000, 424, -1000, 49197, 3304, -1000, -1000, -1000, 2468, 1872, + -340, 49197, -1000, -1000, -1000, -1000, -1000, 1665, -1000, 963, + 963, 4564, 4877, -1000, 13926, -1000, 13926, 3168, -1000, 1818, + -1000, 11398, 2126, 4646, 11398, 4646, 2798, 26553, 34101, -96, + 3299, 3137, 49197, -1000, -1000, 11398, 11398, -1000, 3124, -1000, + -1000, -1000, -1000, 11398, 11398, 2215, -1000, 49197, -1000, -1000, + -1000, -1000, 26553, -1000, 13926, -1000, -1000, -1000, -1000, 11398, + 1364, 1364, 3058, 1658, 169, 169, 169, 3040, 2974, 2963, + 1636, 169, 2884, 2880, 2862, 2848, 2793, 2784, 2777, 2773, + 2713, 2690, 1629, -1000, 2953, -1000, -1000, 2062, 12662, 8864, + -1000, -1000, 317, 1401, 2210, 2511, 133, -1000, 1851, -1000, + 2509, 49197, 49197, 1149, -1000, 49197, 3438, -1000, -1000, -1000, + -1000, 1047, 2507, -1000, 476, 2077, 219, 324, 2667, 1396, + -1000, -1000, 49197, -1000, -1000, -1000, 17735, 2921, 2950, 2921, + 200, 1734, 662, 39762, 738, -1000, 49197, 2069, 1848, 3143, + 794, 3284, 49197, 2949, 487, 2946, 2944, 3328, 526, 5064, + 49197, 1346, -1000, 1525, 3926, -1000, 49197, -1000, 2326, -1000, + 1751, -1000, -1000, 3426, -1000, -1000, 11398, 11398, 3426, 1751, + 1751, -1000, 1871, -1000, 49197, -1000, -341, 526, 5064, 3327, + 4651, 596, 2352, -1000, 49197, -1000, -1000, -1000, 897, -1000, + 1050, 874, 49197, 1998, 1050, 1997, 2943, -1000, -1000, 49197, + 49197, 49197, 49197, -1000, -1000, 49197, -1000, 49197, 49197, 49197, + 49197, 49197, 41020, -1000, 49197, 49197, -1000, 49197, 1994, 49197, + 1993, 3301, -1000, 1734, 1734, 999, -1000, -1000, 664, -1000, + 41020, 2209, 2198, 2196, 2194, 2506, 2504, 2503, 1734, 1734, + 2188, 2493, 40391, 2482, 1243, 2187, 2185, 2184, 2162, 2481, + 1155, -1000, 2479, 2145, 2144, 2113, 49197, 2942, 2397, 543, + -1000, -1000, 2077, 200, 1734, 396, 49197, 1845, 1843, 662, + 615, 562, 15, 24037, -1000, -1000, -1000, 49197, 37246, 37246, + 37246, 37246, 37246, 37246, -1000, 3176, 3162, 3167, -1000, 3192, + 3191, 3198, 3166, 3141, 49197, 37246, 2921, -1000, 40391, -1000, + -1000, -1000, 1836, 1620, 3463, 1044, 11398, 6327, -1000, -1000, + 47, 59, -1000, -1000, -1000, -1000, 39762, 2477, 593, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, 3376, 49197, 49197, 854, + 2666, 1395, -1000, -1000, -1000, 5064, 2901, 2901, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, 2901, 2901, 2920, + -1000, -1000, 2895, 2895, 2895, 2894, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, 2917, 2917, 2919, 2919, + 2917, -1000, -1000, 3424, -1000, 1386, -1000, -1000, 1522, -1000, + 3424, 1901, -328, 15199, 1897, 1802, -1000, 11398, 15199, 11398, + -244, 368, -247, -1000, -1000, -1000, 2473, -1000, -1000, -1000, + 2183, -1000, 2178, -1000, 187, 202, 1992, -211, 8864, 444, + 49197, -211, 49197, 8864, -1000, 49197, 168, -349, -355, 158, + 421, -211, 3376, 78, 11398, 3251, -1000, -1000, 49197, 2177, + -1000, -1000, -1000, 3437, 39762, 2326, 1675, 39133, -1000, 342, + -1000, 253, 623, 2470, -1000, 928, 131, 2469, 2468, -1000, + -1000, -1000, -1000, 13926, 1635, -1000, -1000, -1000, 2388, 11398, + 2665, -1000, 1063, 1063, 2121, 2652, 2643, -1000, 2901, 2901, + -1000, 2894, 2895, 2894, 1063, 1063, 2642, -1000, 2888, -1000, + 3299, -1000, 2253, 2664, -1000, 2523, 2472, 11398, -1000, 2640, + 4539, 1604, -23, -180, 169, 169, -1000, -1000, -1000, -1000, + 169, 169, 169, 169, -1000, 169, 169, 169, 169, 169, + 169, 169, 169, 169, 169, 169, 821, -94, -274, -95, + -275, -1000, 2634, 1366, -1000, -1000, -1000, -1000, -1000, 4859, + 1360, 578, 578, 2468, 2464, -1000, 898, 2463, 1087, 49197, + 2458, -286, -1000, 2456, -1000, -1000, 49197, 2455, -1000, 659, + 49197, 49197, 2453, 2451, 1188, 5064, 2632, 3322, 18364, 3321, + 2241, -1000, -1000, -1000, 29698, 615, -1000, -1000, -1000, 715, + 378, 2168, 607, -1000, 49197, 532, 2996, 1838, 2448, 49197, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, 3284, -1000, 1071, + 506, 35988, 15838, -1000, 411, 49197, -1000, 18364, 18364, 411, + 508, 1791, -1000, 820, 1270, 139, 37246, 49197, -1000, 36617, + 2631, -1000, 1252, 3426, -1000, 2388, 2388, -341, 3426, 3426, + 1751, -1000, -1000, 508, -1000, 411, -1000, 918, 19622, 591, + 529, 517, -1000, 716, -1000, -1000, 815, 3244, 5064, -1000, + 49197, -1000, 49197, -1000, 49197, 49197, 874, 11398, 3244, 49197, + 894, -1000, 1160, 496, 479, 814, 814, 1350, -1000, 3299, + -1000, -1000, 1334, -1000, -1000, -1000, -1000, 49197, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, 26553, 26553, 3357, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - 2800, 2799, 2537, 3560, 2798, 11203, -1000, -1000, 1877, 1871, - 1870, -1000, 2330, 9941, -1000, -1000, -1000, 2797, 1567, 2784, - -1000, -1000, -1000, 2783, 1867, 1353, 2782, 1833, 2777, 2776, - 2774, 2773, 1455, 11203, 11203, 11203, 11203, 2769, 1866, 1865, - 11203, 11203, 11203, 11203, 2768, 11203, 11203, 11203, 11203, 11203, - 11203, 11203, 11203, 11203, 11203, 48942, 116, 116, 116, 1432, - 1429, -1000, -1000, 1827, -1000, 2237, -1000, -1000, 3318, -1000, - 2971, 2266, 1428, -1000, -1000, -315, 2512, 48942, 48942, 174, - 48942, 2610, -281, 48942, -1000, -1000, 2609, -1000, -1000, -1000, - 1138, 1083, 1101, 2608, 3326, 3442, 896, 48942, 1274, 2970, - 48942, 48942, 48942, 296, -1000, -1000, 1445, -1000, 263, -23, - 557, 1222, 3160, 3531, -122, 48942, 48942, 48942, 48942, 3351, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 43918, -1000, - 2965, 1791, -1000, -1000, 1663, 1663, 2237, 3159, 48942, 48942, - 3511, 3511, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 1873, - 3511, 3511, 1711, 1717, 1873, -1000, -1000, 1873, -337, -1000, - 1873, -1000, -337, 1563, -337, 48942, -1000, -1000, -1000, 3350, - 2933, 1427, -1000, -1000, -1000, 3512, 1564, 879, 879, 1117, - 787, 3510, 18786, -1000, 1762, 1182, 930, 3295, 368, -1000, - 1762, -144, 846, 1762, 1762, 1762, 1762, 1762, 1762, 1762, - 713, 703, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, - 1762, 1762, 1762, 1154, 1762, 1762, 1762, 1762, 1762, -1000, - 1762, 2960, 546, -1000, -1000, -1000, -1000, -1000, -1000, 795, - 624, 282, 3402, 433, -1000, 421, 1445, 343, 3396, 456, - 48942, 48942, 3796, 1363, -1000, -1000, -1000, -1000, -1000, 28218, - 28218, 22566, 28218, -1000, 214, 1914, 53, 52, -1000, -1000, - 1416, 6140, 1416, 6140, 2265, -1000, -1000, 929, -1000, -1000, - 1222, -1000, 48942, 48942, -1000, -1000, 2951, 1938, -1000, -1000, - 16264, -1000, 6140, 6140, -1000, -1000, 30102, 48942, -1000, -9, - -1000, 12, 3318, -1000, 1213, -1000, -1000, 1409, 1222, 3181, - 48942, 1213, 1213, 1213, -1000, -1000, 17530, 48942, 48942, -1000, - -1000, -1000, -1000, 3511, 9304, -1000, 37010, -1000, -1000, 43290, - -1000, 42662, 3511, 1969, -1000, 14998, 2192, 199, -1000, 265, - -305, 198, 2167, 194, 2237, -1000, -1000, 2766, 2762, 1790, - -1000, 1786, 2758, 1778, 1771, 2263, -1000, 68, -1000, 3313, - 1224, -1000, 2947, -1000, 1768, 3255, -1000, 1405, -1000, 1936, - 1752, -1000, -1000, -1000, 11203, 42034, 11203, 1224, 1748, 3249, - 1405, 3318, 2607, -1000, 1403, -1000, 2273, 1561, 200, -1000, - -1000, -1000, 48942, 900, 2605, 1746, 41406, 1263, -1000, 927, - 1556, 1544, -1000, 39522, 348, 39522, -1000, 39522, -1000, -1000, - 415, -1000, 48942, 3316, -1000, -1000, -1000, 2512, 1932, -336, - 48942, -1000, -1000, -1000, -1000, -1000, 1734, -1000, 975, 975, - 3669, 3005, -1000, 13727, -1000, 13727, 3496, -1000, 1967, -1000, - 11203, 2187, 4986, 11203, 4986, 1615, 26334, 33870, -106, 3334, - 3491, 48942, -1000, -1000, 11203, 11203, -1000, 3437, -1000, -1000, - -1000, -1000, 11203, 11203, 2322, -1000, 48942, -1000, -1000, -1000, - -1000, 26334, -1000, 13727, -1000, -1000, -1000, -1000, 11203, 1462, - 1462, 3266, 1727, 116, 116, 116, 3233, 3173, 3167, 1719, - 116, 3135, 3087, 3051, 3010, 2967, 2885, 2830, 2780, 2763, - 2705, 1704, -1000, 2946, -1000, -1000, 2113, 12465, 8673, -1000, - -1000, 322, 1393, 2262, 2601, 139, -1000, 1926, -1000, 2596, - 48942, 48942, 1176, -1000, 48942, 3529, -1000, -1000, -1000, -1000, - 1081, 2594, -1000, 486, 2092, 217, 323, 2757, 1374, -1000, - -1000, 48942, -1000, -1000, -1000, 17530, 2933, 2945, 2933, 156, - 1762, 630, 39522, 757, -1000, 48942, 2076, 1923, 3179, 793, - 3309, 48942, 2943, 396, 2942, 2941, 3349, 534, 5219, 48942, - 1345, -1000, 1541, 4054, -1000, 48942, -1000, 2327, -1000, 1717, - -1000, -1000, 3511, -1000, -1000, 11203, 11203, 3511, 1717, 1717, - -1000, 1873, -1000, 48942, -1000, -337, 534, 5219, 3341, 4700, - 700, 2611, -1000, 48942, -1000, -1000, -1000, 780, -1000, 1075, - 897, 48942, 2028, 1075, 2025, 2940, -1000, -1000, 48942, 48942, - 48942, 48942, -1000, -1000, 48942, -1000, 48942, 48942, 48942, 48942, - 48942, 40778, -1000, 48942, 48942, -1000, 48942, 2021, 48942, 2016, - 3546, -1000, 1762, 1762, 1037, -1000, -1000, 654, -1000, 40778, - 2259, 2256, 2241, 2238, 2589, 2588, 2582, 1762, 1762, 2235, - 2571, 40150, 2570, 1236, 2234, 2231, 2230, 2277, 2554, 1313, - -1000, 2550, 2255, 2245, 2194, 48942, 2939, 2463, 536, -1000, - -1000, 2092, 156, 1762, 424, 48942, 1922, 1920, 630, 590, - 556, -24, 23822, -1000, -1000, -1000, 48942, 37010, 37010, 37010, - 37010, 37010, -1000, 3221, 3198, 3215, -1000, 3202, 3201, 3224, - 3056, 48942, 37010, 2933, -1000, 40150, -1000, -1000, -1000, 1909, - 1702, 3568, 1084, 11203, 6140, -1000, -1000, 21, 26, -1000, - -1000, -1000, -1000, 39522, 2540, 575, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, 3438, 48942, 48942, 822, 2751, 1373, -1000, - -1000, -1000, 5219, 2928, 2928, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, 2928, 2928, 2932, -1000, -1000, 2927, - 2927, 2927, 2918, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, 2929, 2929, 2931, 2931, 2929, -1000, -1000, - 3498, -1000, 1364, -1000, -1000, 1540, -1000, 3498, 1982, -324, - 14998, 1971, 1961, -1000, 11203, 14998, 11203, -237, 407, -244, - -1000, -1000, -1000, 2539, -1000, -1000, -1000, 2229, -1000, 2217, - -1000, 183, 187, 2015, -209, 8673, 427, 48942, -209, 48942, - 8673, -1000, 48942, 169, -350, -352, 163, 420, -209, 3438, - 68, 11203, 3278, -1000, -1000, 48942, 2188, -1000, -1000, -1000, - 3528, 39522, 2327, 1693, 38894, -1000, 338, -1000, 267, 602, - 2534, -1000, 949, 137, 2533, 2512, -1000, -1000, -1000, -1000, - 13727, 1663, -1000, -1000, -1000, 2237, 11203, 2750, -1000, 1066, - 1066, 2524, 2749, 2746, -1000, 2928, 2928, -1000, 2918, 2927, - 2918, 1066, 1066, 2744, -1000, 2917, -1000, 3334, -1000, 2473, - 2699, -1000, 2656, 2651, 11203, -1000, 2740, 2963, 1401, -52, - -175, 116, 116, -1000, -1000, -1000, -1000, 116, 116, 116, - 116, -1000, 116, 116, 116, 116, 116, 116, 116, 116, - 116, 116, 116, 844, -87, -266, -88, -272, -1000, 2739, - 1337, -1000, -1000, -1000, -1000, -1000, 3741, 1317, 569, 569, - 2512, 2511, -1000, 922, 2509, 1136, 48942, 2506, -292, -1000, - 2504, -1000, -1000, 48942, 2503, -1000, 621, 48942, 48942, 2494, - 2493, 1274, 5219, 2738, 3340, 18158, 3339, 2363, -1000, -1000, - -1000, 29474, 590, -1000, -1000, -1000, 756, 365, 2185, 574, - -1000, 48942, 582, 3268, 1919, 2492, 48942, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, 3309, -1000, 993, 504, 35754, 15636, - -1000, 416, 48942, -1000, 18158, 18158, 416, 508, 1911, -1000, - 839, 1311, 152, 37010, 48942, -1000, 36382, 2737, -1000, 1222, - 3511, -1000, 2237, 2237, -337, 3511, 3511, 1717, -1000, -1000, - 508, -1000, 416, -1000, 1595, 19414, 583, 532, 523, -1000, - 733, -1000, -1000, 835, 3287, 5219, -1000, 48942, -1000, 48942, - -1000, 48942, 48942, 897, 11203, 3287, 48942, 919, -1000, 1193, - 494, 487, 855, 855, 1307, -1000, 3334, -1000, -1000, 1271, - -1000, -1000, -1000, -1000, 48942, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, 26334, 26334, 3392, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 2490, 2489, + -1000, 2447, 2443, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 48942, - 1683, -1000, 1915, 2183, 2488, 2363, 29474, 1910, 1914, 2487, - 2480, 590, -1000, 2478, 2477, 2076, 1901, 948, 48942, -1000, - 1215, 48942, 48942, -1000, 1351, -1000, 1900, 3151, 3178, 3151, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, 3204, -1000, 3186, - -1000, -1000, -1000, 1351, -1000, -1000, -1000, -1000, -1000, 1084, - -1000, 3429, 1075, 1075, 1075, 2730, -1000, -1000, -1000, -1000, - 1263, 2722, -1000, -1000, -1000, 3577, -1000, -1000, -1000, -1000, - -1000, -1000, 17530, 3302, 3492, 3505, 38266, 3492, -1000, -324, - 1963, -1000, 2044, 191, 2097, 48942, -1000, -1000, -1000, 2721, - 2718, -214, 193, 3504, 3502, 1128, -1000, 2713, 1250, -209, - -1000, -1000, 1224, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -209, -1000, 1224, -1000, 183, -1000, -1000, 3291, -1000, -1000, - 2327, -1000, 264, -1000, -1000, -1000, -1000, -1000, -1000, 242, - -1000, 48942, -1000, 1243, 134, -1000, 2237, -1000, -1000, -1000, - -1000, -1000, 4986, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, 2476, -1000, -1000, 11203, -1000, -1000, -1000, - 2562, -1000, -1000, 11203, 2708, 2474, 2706, 2471, -1000, -1000, + -1000, -1000, 49197, 1619, -1000, 1830, 2167, 2442, 2241, 29698, + 1811, 1749, 2439, 2438, 615, -1000, 2437, 2431, 2069, 1808, + 924, 49197, -1000, 1250, 49197, 49197, -1000, 1356, -1000, 1803, + 3132, 3125, 3132, -1000, 3132, -1000, -1000, -1000, -1000, -1000, + -1000, 3057, -1000, 2991, -1000, -1000, -1000, -1000, 1356, -1000, + -1000, -1000, -1000, -1000, 1044, -1000, 3374, 1050, 1050, 1050, + 2630, -1000, -1000, -1000, -1000, 1317, 2628, -1000, -1000, -1000, + 3453, -1000, -1000, -1000, -1000, -1000, -1000, 17735, 3272, 3421, + 3414, 38504, 3421, -1000, -328, 1794, -1000, 2111, 186, 1943, + 49197, -1000, -1000, -1000, 2627, 2625, -222, 203, 3413, 3409, + 1107, -1000, 2624, 1315, -211, -1000, -1000, 1297, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -211, -1000, 1297, -1000, 187, + -1000, -1000, 3276, -1000, -1000, 2326, -1000, 251, -1000, -1000, + -1000, -1000, -1000, -1000, 258, -1000, 49197, -1000, 1310, 129, + -1000, 2388, -1000, -1000, -1000, -1000, -1000, 4646, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 2429, -1000, + -1000, 11398, -1000, -1000, -1000, 2467, -1000, -1000, 11398, 2622, + 2421, 2620, 2420, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, 3572, -1000, 3499, 1672, 2702, - 2697, 1644, 2696, 2693, -1000, 11203, 2692, 3741, 1047, 2470, - 1047, -1000, -1000, 445, 28846, 48942, 3522, -1000, 48942, 2467, - -1000, -1000, 2092, 620, 676, -1000, -1000, -1000, -1000, 969, - 416, 2691, 1228, -1000, -1000, -1000, -1000, 416, -1000, 2466, - 255, -1000, -1000, -1000, -1000, 2177, -1000, -1000, 2193, 1640, - 289, -1000, -1000, -1000, -1000, -1000, -1000, 2325, 48942, 37638, - 2345, 1893, -339, -1000, 2911, -1000, 1762, 1762, 1762, 48942, - 1614, -1000, 1762, 1762, 2689, -1000, -1000, 2688, 2687, -123, - 853, 1913, 1912, -1000, 2156, 28218, 37010, 36382, 1350, -1000, - 1539, -1000, -1000, -1000, -1000, -1000, -1000, 3511, 853, -1000, - 588, 2155, 13727, 2907, 13727, 2906, 593, 2905, 1600, -1000, - 48942, -1000, -1000, 48942, 4190, 2904, -1000, 2901, 3157, 565, - 2900, 2897, 48942, 2543, -1000, 3287, 48942, 811, 3299, -1000, - -1000, -1000, 476, -1000, -1000, 631, -1000, 48942, -1000, 48942, - -1000, 1658, -1000, 26334, -1000, -1000, 1558, -1000, 2463, 2462, - 3741, -1000, -1000, 255, 2459, 6140, -1000, -1000, -1000, -1000, - -1000, 3268, 2458, 2325, 48942, -1000, 48942, 1215, 1215, 3572, - 48942, 8673, -1000, -1000, 11203, 2896, -1000, 11203, -1000, -1000, - -1000, -1000, -1000, 2869, 3257, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, 2903, -1000, 11203, 11834, -1000, -1000, 893, 14998, - -246, 400, -1000, -1000, -1000, -216, 2457, -1000, -1000, 3494, - 2455, 2374, 48942, -1000, -1000, 1224, 1224, -214, -1000, -1000, - 1222, -1000, -1000, 1230, 722, -1000, 2674, 280, -1000, 2505, - -1000, 2399, 116, -1000, 116, -1000, 304, 11203, -1000, 2451, - -1000, -1000, -1000, 2450, -1000, -1000, 2336, -1000, 2668, -1000, - 2449, -1000, -1000, 48942, -1000, -1000, 1124, 2446, -348, 2445, - 2092, 2092, 48942, 5219, -126, -123, 18158, -126, -1000, -1000, - 409, 437, -1000, -1000, 2121, 655, -1000, -1000, 2443, 660, - -1000, 1215, -1000, 1891, 2037, 2408, 33870, 26334, 27590, 2442, - -1000, -1000, 35754, 2903, 2903, 52116, 336, 52422, -1000, 2868, - 1174, 1890, -1000, 2146, -1000, 2144, -1000, 3511, 1350, 149, - -1000, -1000, 1689, -1000, 1174, 2611, 3493, -1000, 2465, 48942, - 2057, 48942, 2867, 1821, 13727, -1000, 835, 3247, -1000, -1000, - 4190, -1000, -1000, 2040, 13727, -1000, -1000, 2440, 27590, 982, - 1811, 1804, 1017, 2863, -1000, 661, 3574, -1000, -1000, -1000, - 1015, 2856, -1000, 2013, 2012, -1000, 48942, -1000, 33870, 33870, - 1410, 1410, 33870, 33870, 2855, 855, -1000, -1000, 13727, -1000, - -1000, 1762, -1000, -1000, -1000, 1762, 1656, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, 2345, -1000, -1000, 1213, -1000, 3439, - -1000, -1000, 2237, 48942, 2237, 35126, -1000, 3490, 3486, -1000, - 2237, 260, 254, 2851, 2850, -1000, -324, 48942, 48942, -218, - 2141, -1000, 2436, 197, -1000, -1000, 1203, -216, -220, 86, - 26334, 1801, -1000, 2665, 372, -133, -1000, -1000, -1000, -1000, - 2659, -1000, 1168, -1000, -1000, -1000, 1202, 2658, 2655, -1000, - -1000, -1000, -1000, 48942, -1000, 2398, -1000, 2434, 2429, 619, - -110, -1000, -1000, 425, -1000, -1000, -1000, 613, 2370, -1000, - -1000, 431, -1000, -1000, -1000, 2325, 2419, -1000, -1000, 133, - -1000, 1782, 1557, -1000, -1000, -1000, -1000, -1000, -1000, 831, - -1000, 416, 52289, -1000, 1182, 26962, -1000, 1230, 831, 546, - 32614, 697, 311, -1000, 2138, -1000, -1000, 3572, -1000, 685, - -1000, 578, -1000, 1515, -1000, 1507, 34498, 2136, 1858, -1000, - 52154, 967, -1000, -1000, 3669, -1000, -1000, -1000, -1000, -1000, - -1000, 2418, 2417, -1000, -1000, -1000, -1000, -1000, 2108, 2844, - 30, 3373, 2416, -1000, -1000, 2838, 1467, 1465, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, 1443, 1440, 33870, - -1000, -1000, 3669, 2095, 26334, 1762, -1000, -1000, 1439, 1433, - -1000, -1000, -1000, -1000, -1000, -303, 2837, 11203, 11203, -1000, - -1000, -1000, 2832, -1000, -1000, 3484, -218, -222, 2411, 171, - 182, -1000, 2410, -1000, -113, 3239, -140, -1000, -1000, 906, - -210, 147, 146, 132, -1000, -1000, -1000, 11203, -1000, -1000, - -1000, 127, -1000, 1779, -1000, -1000, 2092, 48942, 648, -1000, - -1000, -1000, -1000, 228, -1000, -1000, -1000, -1000, -1000, -1000, - 2408, 2403, -1000, 571, 3480, -1000, 52422, -1000, 1762, -1000, - 571, 1420, -1000, 1762, 1762, -1000, 528, -1000, 1750, -1000, - 2080, -1000, 3439, -1000, 527, -1000, 580, -1000, -1000, -1000, - 1419, -1000, -1000, -1000, 52154, 589, -1000, 826, 2827, -1000, - -1000, 2650, 11203, 2823, 1762, 2638, -96, 33870, 3154, 3153, - 3152, 2969, 1412, -1000, -1000, 26334, -1000, -1000, 33242, -1000, - 2822, 1408, 1407, 48942, 2374, -1000, -1000, 2401, -1000, 908, - 177, 182, -1000, 3479, 184, 3472, 3471, 1199, 3228, -1000, - -1000, 1999, -1000, 142, 140, 138, -1000, -1000, -1000, -1000, - -1000, 2398, 2397, 2390, 576, -1000, 337, -1000, -1000, -1000, - 360, -1000, 3467, 700, -1000, 26334, -1000, -1000, 32614, 2903, - 2903, -1000, -1000, 2072, -1000, -1000, -1000, -1000, 2069, -1000, - -1000, -1000, 1402, -1000, 48942, 1024, 8042, -1000, 2258, -1000, - 48942, -1000, 3174, -1000, 253, 1375, 360, 1410, 360, 1410, - 360, 1410, 360, 1410, 335, -1000, -1000, -1000, 1371, 11203, - -1000, -1000, 1369, -1000, -1000, -1000, 2820, 2054, 193, 164, - 3460, -1000, 2374, 3457, 2374, 2374, -1000, 130, -127, 906, - -1000, -1000, -1000, -1000, -1000, -1000, 2388, -1000, -1000, -1000, - -1000, 1762, 1762, 2386, 2385, 493, -1000, -1000, 31986, 583, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, 589, 52422, -1000, - 8042, 1320, -1000, 2237, -1000, 855, -1000, -1000, 3164, 3090, - 3521, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, 2743, 2623, -1000, 48942, 3361, 25706, 160, -1000, -1000, - -1000, 2376, -1000, 2374, -1000, -1000, 1754, -137, -1000, -264, - 2048, 2047, -1000, -1000, 48942, -1000, 48942, 588, -1000, 52422, - 1306, -1000, 8042, -1000, -1000, 3565, -1000, 3547, 1061, 1061, - 360, 360, 360, 360, 11203, -1000, -1000, -1000, 48942, -1000, - 1305, -1000, -1000, -1000, 1523, -1000, -1000, -1000, -1000, 2372, - -141, -1000, -1000, 2368, -1000, -1000, -1000, 1255, 2611, -1000, - -1000, -1000, -1000, -1000, 2111, 678, -1000, 2542, 1183, -1000, - 1747, -1000, 31358, 48942, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, 48942, 7411, -1000, 1520, -1000, -1000, 2237, - 48942, -1000, + 3461, -1000, 3408, 1611, 2610, 2609, 1605, 2608, 2607, -1000, + 11398, 2601, 4859, 1025, 2418, 1025, -1000, -1000, 448, 29069, + 49197, 3435, -1000, 49197, 2406, -1000, -1000, 2077, 639, 687, + -1000, -1000, -1000, -1000, 946, 411, 2599, 1306, -1000, -1000, + -1000, -1000, 411, -1000, 2399, 271, -1000, -1000, -1000, -1000, + 2166, -1000, -1000, 2112, 1623, 285, -1000, -1000, -1000, -1000, + -1000, -1000, 2117, 49197, 37875, 2199, 1799, -344, -1000, 2886, + -1000, 1734, 1734, 1734, 49197, 1603, -1000, 1734, 1734, 2598, + -1000, -1000, 2597, 2593, -122, 840, 1801, 1800, -1000, 2161, + 28440, 37246, 36617, 1341, -1000, 1518, -1000, -1000, -1000, -1000, + -1000, -1000, 3426, 840, -1000, 588, 2150, 13926, 2882, 13926, + 2874, 609, 2873, 1598, -1000, 49197, -1000, -1000, 49197, 4106, + 2870, -1000, 2867, 3134, 577, 2864, 2818, 49197, 2462, -1000, + 3244, 49197, 788, 3264, -1000, -1000, -1000, 428, -1000, -1000, + 666, -1000, 49197, -1000, 49197, -1000, 1630, -1000, 26553, -1000, + -1000, 1595, -1000, 2397, 2393, 4859, -1000, -1000, 271, 2392, + 6327, -1000, -1000, -1000, -1000, -1000, 2996, 2387, 2117, 49197, + -1000, 49197, 1250, 1250, 3461, 49197, 8864, -1000, -1000, 11398, + 2817, -1000, 11398, -1000, -1000, -1000, -1000, -1000, -1000, 2799, + 3266, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 1425, -1000, + 11398, 12030, -1000, -1000, 869, 15199, -248, 365, -1000, -1000, + -1000, -224, 2386, -1000, -1000, 3404, 2384, 2281, 49197, -1000, + -1000, 1297, 1297, -222, -1000, -1000, 1252, -1000, -1000, 1172, + 721, -1000, 2592, 278, -1000, 2346, -1000, 2311, 169, -1000, + 169, -1000, 270, 11398, -1000, 2376, -1000, -1000, -1000, 2375, + -1000, -1000, 2266, -1000, 2591, -1000, 2374, -1000, -1000, 49197, + -1000, -1000, 1081, 2372, -345, 2365, 2077, 2077, 49197, 5064, + -131, -122, 18364, -131, -1000, -1000, 422, 434, -1000, -1000, + 2090, 665, -1000, -1000, 2364, 625, -1000, 1250, -1000, 1790, + 2011, 2337, 34101, 26553, 27811, 2362, -1000, -1000, 35988, 1425, + 1425, 52353, 330, 52429, -1000, 2797, 1142, 1771, -1000, 2149, + -1000, 2148, -1000, 3426, 1341, 138, -1000, -1000, 1661, -1000, + 1142, 2352, 3401, -1000, 4437, 49197, 4339, 49197, 2796, 1787, + 13926, -1000, 815, 3202, -1000, -1000, 4106, -1000, -1000, 1985, + 13926, -1000, -1000, 2358, 27811, 1018, 1784, 1780, 1042, 2790, + -1000, 677, 3449, -1000, -1000, -1000, 978, 2789, -1000, 1987, + 1954, -1000, 49197, -1000, 34101, 34101, 769, 769, 34101, 34101, + 2783, 814, -1000, -1000, 13926, -1000, -1000, 1734, -1000, -1000, + -1000, 1734, 1607, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + 2199, -1000, -1000, 1248, -1000, 3387, -1000, -1000, 2388, 49197, + 2388, 35359, -1000, 3400, 3399, -1000, 2388, 250, 255, 2780, + 2779, -1000, -328, 49197, 49197, -227, 2143, -1000, 2357, 199, + -1000, -1000, 1231, -224, -229, 125, 26553, 1778, -1000, 2590, + 372, -140, -1000, -1000, -1000, -1000, 2585, -1000, 622, -1000, + -1000, -1000, 1201, 2584, 2583, -1000, -1000, -1000, -1000, 49197, + -1000, 2321, -1000, 2355, 2354, 638, -100, -1000, -1000, 481, + -1000, -1000, -1000, 620, 2279, -1000, -1000, 427, -1000, -1000, + -1000, 2117, 2353, -1000, -1000, 127, -1000, 1762, 1593, -1000, + -1000, -1000, -1000, -1000, -1000, 802, -1000, 411, 52391, -1000, + 1080, 27182, -1000, 1172, 802, 548, 32843, 700, 313, -1000, + 2127, -1000, -1000, 3461, -1000, 685, -1000, 602, -1000, 1591, + -1000, 1580, 34730, 2100, 3673, -1000, 3838, 953, -1000, -1000, + 4564, -1000, -1000, -1000, -1000, -1000, -1000, 2351, 2349, -1000, + -1000, -1000, -1000, -1000, 2082, 2762, 32, 3355, 2344, -1000, + -1000, 2761, 1575, 1566, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, 1544, 1509, 34101, -1000, -1000, 4564, 2079, + 26553, 1734, -1000, -1000, 1489, 1479, -1000, -1000, -1000, -1000, + -1000, -302, 2757, 11398, 11398, -1000, -1000, -1000, 2756, -1000, + -1000, 3398, -227, -231, 2343, 184, 213, -1000, 2338, -1000, + -109, 3209, -144, -1000, -1000, 945, -213, 163, 161, 153, + -1000, -1000, -1000, 11398, -1000, -1000, -1000, 118, -1000, 1741, + -1000, -1000, 2077, 49197, 673, -1000, -1000, -1000, -1000, 252, + -1000, -1000, -1000, -1000, -1000, -1000, 2337, 2329, -1000, 581, + 3397, -1000, 52429, -1000, 1734, -1000, 581, 1435, -1000, 1734, + 1734, -1000, 511, -1000, 1763, -1000, 2073, -1000, 3387, -1000, + 507, -1000, 585, -1000, -1000, -1000, 1428, -1000, -1000, -1000, + 3838, 589, -1000, 786, 2754, -1000, -1000, 2581, 11398, 2752, + 1734, 2577, -93, 34101, 3128, 3126, 3123, 2800, 1424, -1000, + -1000, 26553, -1000, -1000, 33472, -1000, 2749, 1410, 1402, 49197, + 2281, -1000, -1000, 2325, -1000, 882, 220, 213, -1000, 3396, + 198, 3395, 3394, 1198, 3208, -1000, -1000, 1944, -1000, 175, + 171, 166, -1000, -1000, -1000, -1000, -1000, 2321, 2304, 2303, + 595, -1000, 336, -1000, -1000, -1000, 338, -1000, 3393, 596, + -1000, 26553, -1000, -1000, 32843, 1425, 1425, -1000, -1000, 2072, + -1000, -1000, -1000, -1000, 2071, -1000, -1000, -1000, 1375, -1000, + 49197, 1014, 8232, -1000, 2235, -1000, 49197, -1000, 2887, -1000, + 289, 1368, 338, 769, 338, 769, 338, 769, 338, 769, + 333, -1000, -1000, -1000, 1335, 11398, -1000, -1000, 1333, -1000, + -1000, -1000, 2478, 2059, 203, 190, 3391, -1000, 2281, 3389, + 2281, 2281, -1000, 172, -136, 945, -1000, -1000, -1000, -1000, + -1000, -1000, 2287, -1000, -1000, -1000, -1000, 1734, 1734, 2286, + 2285, 495, -1000, -1000, 32214, 591, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, 589, 52429, -1000, 8232, 1301, -1000, 2388, + -1000, 814, -1000, -1000, 2883, 2865, 3431, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, 2407, 2576, -1000, + 49197, 3346, 25924, 193, -1000, -1000, -1000, 2283, -1000, 2281, + -1000, -1000, 1729, -141, -1000, -272, 2049, 2045, -1000, -1000, + 49197, -1000, 49197, 588, -1000, 52429, 1266, -1000, 8232, -1000, + -1000, 3447, -1000, 3432, 979, 979, 338, 338, 338, 338, + 11398, -1000, -1000, -1000, 49197, -1000, 1265, -1000, -1000, -1000, + 1511, -1000, -1000, -1000, -1000, 2270, -145, -1000, -1000, 2131, + -1000, -1000, -1000, 1221, 2352, -1000, -1000, -1000, -1000, -1000, + 2099, 688, -1000, 2549, 1191, -1000, 1699, -1000, 31585, 49197, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 49197, + 7600, -1000, 1426, -1000, -1000, 2388, 49197, -1000, } var yyPgo = [...]int{ - 0, 185, 3597, 252, 181, 4210, 83, 262, 272, 251, - 261, 4209, 4208, 4207, 4206, 3307, 3303, 4203, 4200, 4199, - 4198, 4197, 4196, 4195, 4194, 4193, 4192, 4191, 4190, 4189, - 4188, 4187, 4186, 4185, 4184, 4183, 4182, 4181, 4180, 4179, - 4178, 4173, 4172, 4171, 4170, 4169, 4168, 254, 4167, 4166, - 4165, 4164, 4163, 4162, 4161, 4160, 4159, 4158, 4157, 4156, - 4155, 4154, 4153, 4152, 4134, 4133, 4132, 4131, 4129, 4128, - 4127, 4126, 4125, 4123, 4122, 4121, 4120, 4109, 4108, 4107, - 4105, 4104, 247, 4103, 4102, 242, 4100, 3299, 4098, 4097, - 4093, 4091, 4090, 4089, 4088, 4087, 329, 4086, 4084, 4083, - 4081, 4080, 4079, 4077, 4076, 4075, 4074, 4073, 4072, 4066, - 286, 4065, 4064, 4063, 4062, 214, 4061, 279, 4060, 179, - 147, 4059, 4058, 4057, 4052, 4051, 4050, 4049, 4048, 4047, - 4046, 4045, 4044, 4043, 4042, 250, 201, 74, 4040, 43, - 4038, 4037, 229, 4034, 154, 4033, 156, 4032, 4030, 4029, - 4028, 4025, 4022, 4021, 4020, 4019, 4018, 4015, 4013, 4012, - 4011, 4010, 4006, 4005, 4004, 4002, 4001, 4000, 3997, 3996, - 50, 3995, 263, 3994, 72, 3992, 184, 3991, 78, 3988, - 84, 134, 258, 1901, 264, 257, 192, 177, 3986, 3985, - 294, 3984, 163, 234, 160, 97, 124, 3982, 142, 3980, - 268, 47, 48, 236, 144, 53, 162, 129, 3978, 216, - 101, 113, 3976, 3975, 146, 3974, 245, 187, 3970, 111, - 3969, 3966, 3962, 3961, 3960, 204, 200, 3959, 3958, 132, - 3957, 3951, 75, 133, 3948, 77, 126, 173, 122, 3946, - 1659, 128, 94, 3943, 121, 109, 3937, 166, 3936, 3935, - 3934, 3929, 194, 3928, 3927, 137, 66, 3925, 3924, 3923, - 71, 3922, 79, 3921, 61, 3920, 65, 3919, 3918, 3917, - 3916, 3915, 3914, 3912, 3911, 3908, 3907, 3906, 3903, 57, - 3902, 3901, 3898, 3896, 9, 11, 15, 3895, 29, 3891, - 174, 3889, 3884, 170, 3882, 198, 3881, 3880, 100, 95, - 3879, 96, 169, 3877, 7, 25, 59, 3876, 3875, 224, - 239, 110, 152, 3873, 273, 3872, 3863, 3862, 161, 3859, - 3858, 2361, 3857, 3856, 3853, 3852, 3850, 3849, 64, 3848, - 1, 218, 40, 3847, 131, 139, 3846, 37, 27, 3844, - 44, 114, 202, 136, 105, 3842, 3838, 3836, 274, 223, - 104, 67, 0, 103, 226, 159, 3835, 3834, 3832, 267, - 3831, 241, 199, 238, 178, 259, 203, 3830, 3828, 58, - 3823, 167, 30, 52, 148, 93, 20, 243, 3822, 1738, - 10, 212, 3821, 207, 3820, 8, 18, 172, 150, 3806, - 3805, 33, 265, 3804, 3803, 3800, 130, 3798, 3796, 188, - 82, 3795, 3794, 3793, 3792, 39, 3785, 190, 17, 3783, - 125, 3781, 255, 3778, 220, 138, 183, 176, 158, 230, - 235, 85, 81, 3777, 1847, 157, 108, 16, 3775, 231, - 3774, 186, 119, 3773, 86, 3772, 253, 271, 210, 3771, - 191, 12, 45, 36, 31, 42, 13, 266, 205, 3770, - 3769, 23, 51, 3766, 54, 3765, 21, 3764, 3763, 38, - 3761, 56, 5, 3760, 3759, 19, 22, 3757, 35, 209, - 175, 127, 102, 76, 3756, 3755, 143, 189, 3754, 141, - 155, 165, 3753, 80, 3752, 3751, 3749, 3748, 724, 3747, - 256, 3744, 3743, 3742, 3741, 3739, 3738, 3737, 3736, 3735, - 225, 3733, 107, 41, 3732, 3731, 3730, 3729, 89, 151, - 3728, 3726, 3725, 3724, 32, 140, 3723, 14, 3721, 26, - 24, 34, 3720, 106, 3719, 3, 195, 3718, 3715, 4, - 3714, 3713, 2, 3712, 3711, 145, 3709, 98, 28, 171, - 112, 3707, 3706, 92, 213, 149, 3705, 3688, 88, 246, - 206, 3683, 99, 237, 269, 3682, 211, 3679, 3678, 3677, - 3676, 3675, 3674, 1258, 3673, 3670, 248, 55, 91, 3669, - 227, 120, 3668, 3667, 90, 168, 116, 117, 49, 87, - 3665, 115, 215, 3664, 197, 3663, 260, 3662, 3661, 118, - 3660, 3659, 3658, 3657, 208, 3656, 3654, 196, 244, 3653, - 3652, 270, 3651, 3649, 3635, 3633, 3630, 3628, 3627, 3626, - 3625, 3602, 240, 249, 3600, + 0, 181, 3478, 260, 179, 4121, 81, 259, 253, 242, + 255, 4117, 4116, 4113, 4112, 3267, 3257, 4111, 4109, 4108, + 4107, 4106, 4105, 4104, 4103, 4102, 4099, 4098, 4097, 4096, + 4095, 4090, 4087, 4086, 4085, 4084, 4082, 4081, 4080, 4079, + 4078, 4077, 4076, 4075, 4074, 4072, 4071, 249, 4070, 4067, + 4066, 4065, 4064, 4062, 4061, 4058, 4056, 4055, 4054, 4053, + 4052, 4051, 4050, 4048, 4043, 4040, 4038, 4037, 4036, 4035, + 4034, 4031, 4028, 4027, 4026, 4025, 4024, 4023, 4022, 4021, + 4019, 4018, 245, 4017, 4016, 211, 4015, 3246, 4014, 4013, + 4012, 4010, 4009, 4008, 4007, 4005, 334, 3998, 3997, 3995, + 3994, 3992, 3991, 3990, 3988, 3983, 3982, 3981, 3980, 3979, + 328, 3978, 3976, 3975, 3972, 265, 3970, 294, 3969, 176, + 171, 3965, 3963, 3962, 3955, 3952, 3948, 3947, 3946, 3938, + 3937, 3936, 3932, 3931, 3930, 238, 190, 74, 3929, 47, + 3928, 3927, 224, 3926, 146, 3924, 149, 3922, 3921, 3920, + 3918, 3917, 3914, 3913, 3909, 3908, 3904, 3902, 3901, 3900, + 3899, 3897, 3892, 3890, 3888, 3887, 3886, 3885, 3883, 3882, + 51, 3881, 267, 3878, 75, 3875, 178, 3874, 71, 3873, + 88, 129, 261, 2430, 248, 246, 187, 186, 3872, 3870, + 278, 3866, 262, 231, 165, 104, 124, 3865, 131, 3864, + 270, 49, 53, 258, 142, 57, 205, 150, 3862, 214, + 97, 112, 3858, 3856, 141, 3855, 244, 183, 3854, 109, + 3853, 3852, 3850, 3848, 3847, 202, 197, 3846, 3845, 126, + 3843, 3842, 72, 162, 3841, 77, 125, 173, 122, 3840, + 1126, 140, 94, 3839, 120, 103, 3838, 139, 3837, 3836, + 3834, 3829, 192, 3828, 3827, 148, 64, 3810, 3809, 3808, + 67, 3807, 79, 3806, 48, 3803, 61, 3801, 3800, 3799, + 3797, 3795, 3789, 3788, 3787, 3786, 3784, 3783, 3781, 54, + 3780, 3779, 3778, 3777, 7, 13, 15, 3775, 27, 3773, + 174, 3772, 3771, 169, 3769, 195, 3768, 3767, 98, 92, + 3766, 95, 168, 3765, 9, 28, 76, 3761, 3760, 220, + 348, 105, 153, 3758, 276, 3757, 3756, 3755, 158, 3754, + 3752, 3751, 266, 3750, 3749, 3748, 3747, 3746, 3745, 254, + 3743, 1, 218, 41, 3741, 138, 133, 3740, 37, 31, + 3739, 44, 114, 223, 137, 96, 3738, 3720, 3719, 136, + 199, 111, 29, 0, 93, 216, 159, 3718, 3717, 3715, + 250, 3713, 235, 200, 240, 1708, 264, 251, 3712, 3711, + 59, 3710, 170, 32, 52, 130, 80, 20, 206, 3708, + 1021, 10, 184, 3707, 209, 3706, 8, 17, 151, 147, + 3704, 3702, 34, 263, 3701, 3700, 3698, 127, 3697, 3696, + 279, 83, 3695, 3693, 3692, 3691, 38, 3689, 185, 16, + 3688, 101, 3687, 241, 3686, 232, 143, 188, 182, 161, + 229, 234, 85, 84, 3685, 1825, 155, 108, 22, 3684, + 227, 3683, 166, 118, 3681, 106, 3678, 247, 268, 208, + 3677, 189, 11, 45, 39, 30, 43, 12, 225, 203, + 3675, 3674, 23, 50, 3673, 55, 3672, 19, 3670, 3669, + 40, 3668, 58, 5, 3667, 3666, 18, 21, 3665, 35, + 204, 175, 128, 100, 66, 3664, 3663, 132, 152, 3662, + 160, 154, 163, 3660, 78, 3659, 3658, 3657, 3655, 935, + 3654, 252, 3652, 3650, 3649, 3648, 3647, 3646, 3642, 3639, + 3637, 215, 3634, 107, 42, 3633, 3632, 3631, 3630, 82, + 145, 3629, 3628, 3622, 3621, 33, 134, 3619, 14, 3602, + 26, 24, 36, 3596, 99, 3595, 3, 194, 3594, 3593, + 4, 3590, 3588, 2, 3586, 3585, 121, 3584, 102, 25, + 172, 113, 3583, 3582, 91, 210, 144, 3579, 3577, 90, + 239, 201, 3576, 156, 236, 257, 3574, 213, 3572, 3571, + 3570, 3569, 3567, 3565, 1199, 3563, 3562, 243, 65, 86, + 3561, 226, 119, 3560, 3554, 89, 167, 117, 116, 56, + 87, 3553, 115, 207, 3552, 198, 3548, 256, 3546, 3545, + 110, 3544, 3540, 3539, 3529, 196, 3522, 3518, 191, 230, + 3517, 3516, 275, 3513, 3510, 3505, 3499, 3498, 3492, 3489, + 3485, 3484, 3483, 237, 269, 3482, } -//line mysql_sql.y:12684 +//line mysql_sql.y:12701 type yySymType struct { union interface{} id int @@ -8567,233 +8570,233 @@ func (st *yySymType) zeroFillOptUnion() bool { } var yyR1 = [...]int{ - 0, 607, 610, 610, 5, 5, 2, 6, 6, 3, + 0, 608, 611, 611, 5, 5, 2, 6, 6, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 1, 1, 1, 1, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 125, - 125, 343, 343, 344, 344, 127, 339, 339, 338, 338, - 128, 129, 130, 586, 586, 131, 132, 162, 585, 585, - 164, 164, 164, 164, 164, 459, 126, 126, 126, 126, - 126, 168, 168, 168, 168, 168, 124, 592, 592, 592, - 593, 593, 121, 151, 150, 153, 153, 152, 152, 149, + 125, 344, 344, 345, 345, 127, 340, 340, 339, 339, + 128, 129, 130, 587, 587, 131, 132, 162, 586, 586, + 164, 164, 164, 164, 164, 460, 126, 126, 126, 126, + 126, 168, 168, 168, 168, 168, 124, 593, 593, 593, + 594, 594, 121, 151, 150, 153, 153, 152, 152, 149, 149, 145, 148, 148, 147, 147, 146, 141, 143, 143, - 142, 144, 144, 122, 110, 123, 534, 534, 533, 533, - 532, 532, 484, 484, 485, 485, 330, 330, 330, 531, - 531, 531, 530, 530, 529, 529, 528, 528, 526, 526, - 527, 525, 524, 524, 524, 522, 522, 522, 518, 518, - 520, 519, 519, 521, 513, 513, 516, 516, 514, 514, - 514, 514, 517, 512, 512, 512, 511, 511, 109, 109, - 109, 426, 426, 108, 108, 440, 440, 440, 440, 440, - 438, 438, 438, 438, 438, 438, 437, 437, 436, 436, - 441, 441, 439, 439, 439, 439, 439, 439, 439, 439, - 439, 439, 439, 439, 439, 439, 439, 439, 439, 439, - 439, 439, 439, 439, 439, 439, 439, 439, 439, 439, - 439, 439, 439, 439, 439, 439, 439, 439, 439, 439, - 439, 439, 439, 439, 439, 439, 439, 439, 439, 439, - 439, 439, 439, 97, 97, 97, 97, 97, 97, 97, - 104, 102, 102, 102, 103, 598, 598, 597, 597, 599, - 599, 599, 599, 600, 600, 100, 100, 100, 101, 435, - 435, 435, 98, 99, 99, 425, 425, 430, 430, 429, - 429, 429, 429, 429, 429, 429, 429, 429, 429, 429, - 429, 429, 434, 434, 434, 432, 432, 431, 431, 433, - 433, 91, 91, 91, 94, 93, 424, 424, 424, 424, - 424, 424, 424, 424, 424, 92, 92, 92, 92, 92, + 142, 144, 144, 122, 110, 123, 535, 535, 534, 534, + 533, 533, 485, 485, 486, 486, 331, 331, 331, 532, + 532, 532, 531, 531, 530, 530, 529, 529, 527, 527, + 528, 526, 525, 525, 525, 523, 523, 523, 519, 519, + 521, 520, 520, 522, 514, 514, 517, 517, 515, 515, + 515, 515, 518, 513, 513, 513, 512, 512, 109, 109, + 109, 427, 427, 108, 108, 441, 441, 441, 441, 441, + 439, 439, 439, 439, 439, 439, 438, 438, 437, 437, + 442, 442, 440, 440, 440, 440, 440, 440, 440, 440, + 440, 440, 440, 440, 440, 440, 440, 440, 440, 440, + 440, 440, 440, 440, 440, 440, 440, 440, 440, 440, + 440, 440, 440, 440, 440, 440, 440, 440, 440, 440, + 440, 440, 440, 440, 440, 440, 440, 440, 440, 440, + 440, 440, 440, 97, 97, 97, 97, 97, 97, 97, + 104, 102, 102, 102, 103, 599, 599, 598, 598, 600, + 600, 600, 600, 601, 601, 100, 100, 100, 101, 436, + 436, 436, 98, 99, 99, 426, 426, 431, 431, 430, + 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, + 430, 430, 435, 435, 435, 433, 433, 432, 432, 434, + 434, 91, 91, 91, 94, 93, 425, 425, 425, 425, + 425, 425, 425, 425, 425, 92, 92, 92, 92, 92, 92, 86, 86, 86, 86, 86, 85, 85, 87, 87, - 422, 422, 421, 105, 105, 106, 595, 595, 594, 596, - 596, 596, 596, 107, 113, 113, 113, 113, 113, 113, + 423, 423, 422, 105, 105, 106, 596, 596, 595, 597, + 597, 597, 597, 107, 113, 113, 113, 113, 113, 113, 113, 113, 112, 112, 115, 115, 114, 116, 96, 96, 96, 96, 96, 96, 95, 95, 95, 95, 95, 95, 95, 95, 95, 95, 95, 95, 95, 95, 95, 95, - 559, 559, 559, 559, 561, 561, 357, 358, 611, 360, - 356, 356, 356, 555, 555, 556, 557, 558, 558, 558, - 111, 14, 215, 215, 458, 458, 11, 11, 11, 11, + 560, 560, 560, 560, 562, 562, 358, 359, 612, 361, + 357, 357, 357, 556, 556, 557, 558, 559, 559, 559, + 111, 14, 215, 215, 459, 459, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 13, 84, 89, 89, 90, 294, 294, 295, 289, 289, 296, 296, 167, 297, 297, 297, 302, 302, 303, 303, 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, 274, 274, 274, 269, 269, 269, 269, 270, 270, 271, 271, 272, - 272, 272, 272, 273, 273, 349, 349, 298, 298, 298, + 272, 272, 272, 273, 273, 350, 350, 298, 298, 298, 300, 300, 299, 293, 291, 291, 291, 291, 291, 291, 291, 292, 292, 292, 292, 301, 301, 82, 88, 88, - 88, 88, 573, 573, 83, 83, 584, 584, 488, 488, - 371, 371, 370, 370, 370, 370, 370, 370, 370, 370, - 370, 370, 370, 370, 370, 370, 370, 370, 494, 495, - 367, 47, 47, 47, 47, 47, 47, 47, 47, 47, + 88, 88, 574, 574, 83, 83, 585, 585, 489, 489, + 372, 372, 371, 371, 371, 371, 371, 371, 371, 371, + 371, 371, 371, 371, 371, 371, 371, 371, 495, 496, + 368, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 79, 80, - 81, 54, 57, 58, 166, 53, 53, 53, 411, 411, - 52, 612, 612, 342, 342, 67, 66, 56, 68, 69, + 81, 54, 57, 58, 166, 53, 53, 53, 412, 412, + 52, 613, 613, 343, 343, 67, 66, 56, 68, 69, 70, 71, 72, 73, 51, 65, 65, 65, 65, 65, - 65, 65, 65, 76, 507, 507, 614, 614, 614, 74, - 75, 487, 487, 487, 64, 63, 62, 61, 60, 60, - 50, 50, 49, 49, 55, 157, 59, 158, 158, 364, - 364, 364, 366, 366, 362, 613, 613, 454, 454, 365, - 365, 48, 48, 48, 48, 77, 363, 363, 341, 361, - 361, 361, 12, 12, 10, 17, 17, 17, 17, 17, + 65, 65, 65, 76, 508, 508, 615, 615, 615, 74, + 75, 488, 488, 488, 64, 63, 62, 61, 60, 60, + 50, 50, 49, 49, 55, 157, 59, 158, 158, 365, + 365, 365, 367, 367, 363, 614, 614, 455, 455, 366, + 366, 48, 48, 48, 48, 77, 364, 364, 342, 362, + 362, 362, 12, 12, 10, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, - 17, 17, 26, 27, 29, 419, 419, 416, 28, 20, + 17, 17, 26, 27, 29, 420, 420, 417, 28, 20, 19, 19, 23, 22, 18, 18, 21, 24, 25, 25, 9, 9, 9, 9, 15, 15, 16, 187, 187, 241, - 241, 567, 567, 563, 563, 564, 564, 564, 565, 565, - 566, 566, 117, 501, 501, 501, 501, 501, 501, 8, - 8, 210, 210, 500, 500, 500, 500, 500, 500, 423, - 423, 423, 544, 544, 544, 545, 209, 209, 204, 204, - 502, 502, 388, 546, 546, 510, 510, 509, 509, 508, - 508, 207, 207, 208, 208, 190, 190, 136, 136, 515, - 515, 515, 515, 523, 523, 483, 483, 279, 279, 332, - 332, 333, 333, 180, 180, 181, 181, 181, 181, 181, - 181, 601, 601, 602, 603, 604, 604, 605, 605, 605, - 606, 606, 606, 606, 606, 552, 552, 554, 554, 553, + 241, 568, 568, 564, 564, 565, 565, 565, 566, 566, + 567, 567, 117, 502, 502, 502, 502, 502, 502, 8, + 8, 210, 210, 501, 501, 501, 501, 501, 501, 424, + 424, 424, 545, 545, 545, 546, 209, 209, 204, 204, + 503, 503, 389, 547, 547, 511, 511, 510, 510, 509, + 509, 207, 207, 208, 208, 190, 190, 136, 136, 516, + 516, 516, 516, 524, 524, 484, 484, 279, 279, 333, + 333, 334, 334, 180, 180, 181, 181, 181, 181, 181, + 181, 602, 602, 603, 604, 605, 605, 606, 606, 606, + 607, 607, 607, 607, 607, 553, 553, 555, 555, 554, 206, 206, 202, 202, 203, 203, 203, 201, 201, 200, 199, 199, 198, 196, 196, 196, 197, 197, 197, 214, 214, 183, 183, 183, 182, 182, 182, 182, 182, 314, 314, 314, 314, 314, 314, 314, 314, 314, 314, 314, - 314, 184, 184, 560, 560, 560, 489, 489, 489, 497, - 497, 311, 311, 312, 312, 312, 312, 312, 134, 134, - 493, 493, 310, 310, 185, 185, 186, 186, 186, 186, + 314, 184, 184, 561, 561, 561, 490, 490, 490, 498, + 498, 311, 311, 312, 312, 312, 312, 312, 134, 134, + 494, 494, 310, 310, 185, 185, 186, 186, 186, 186, 309, 309, 308, 195, 195, 194, 193, 193, 193, 188, - 188, 188, 188, 189, 320, 320, 319, 319, 318, 318, - 318, 318, 120, 133, 133, 135, 213, 213, 317, 316, - 316, 316, 316, 212, 212, 211, 211, 205, 205, 192, - 192, 192, 192, 315, 191, 313, 591, 591, 590, 590, - 589, 587, 587, 587, 588, 588, 588, 588, 536, 536, - 536, 536, 536, 350, 350, 350, 355, 355, 353, 353, - 353, 353, 353, 359, 7, 7, 7, 7, 7, 7, - 7, 7, 7, 33, 33, 33, 33, 33, 33, 33, - 33, 33, 33, 33, 33, 33, 39, 224, 225, 40, - 226, 226, 227, 227, 228, 228, 229, 230, 231, 231, - 231, 231, 38, 216, 216, 217, 217, 218, 218, 219, - 220, 220, 220, 223, 221, 222, 222, 609, 609, 608, - 37, 37, 30, 171, 171, 172, 172, 172, 174, 174, - 275, 275, 275, 173, 173, 175, 175, 175, 568, 570, - 570, 572, 571, 571, 571, 574, 574, 574, 574, 574, - 575, 575, 575, 575, 576, 576, 31, 154, 154, 178, - 178, 159, 579, 579, 579, 578, 578, 580, 580, 581, - 581, 336, 336, 337, 337, 169, 170, 170, 161, 156, - 177, 177, 177, 177, 177, 179, 179, 243, 243, 155, - 160, 163, 165, 569, 577, 577, 577, 420, 420, 417, - 418, 418, 415, 414, 414, 414, 583, 583, 582, 582, - 582, 351, 351, 32, 410, 410, 412, 413, 413, 413, - 404, 404, 404, 404, 36, 408, 408, 409, 409, 409, - 409, 409, 409, 409, 405, 405, 407, 407, 403, 403, - 403, 403, 403, 403, 403, 35, 176, 176, 402, 402, - 399, 399, 397, 397, 398, 398, 396, 396, 396, 400, - 400, 43, 78, 44, 45, 46, 42, 401, 401, 34, - 34, 34, 34, 34, 34, 34, 34, 34, 34, 138, - 137, 137, 137, 137, 137, 140, 140, 335, 335, 334, - 334, 139, 276, 276, 41, 254, 254, 475, 475, 470, - 470, 470, 470, 470, 491, 491, 491, 471, 471, 471, - 472, 472, 472, 474, 474, 474, 473, 473, 473, 473, - 473, 490, 490, 492, 492, 492, 442, 442, 443, 443, - 443, 446, 446, 462, 462, 463, 463, 461, 461, 468, - 468, 467, 467, 466, 466, 465, 465, 464, 464, 464, - 464, 457, 457, 456, 456, 444, 444, 444, 444, 444, - 445, 445, 445, 455, 455, 460, 460, 307, 307, 306, - 306, 262, 262, 263, 263, 305, 305, 260, 260, 261, - 261, 261, 304, 304, 304, 304, 304, 304, 304, 304, + 188, 188, 188, 188, 189, 320, 320, 319, 319, 318, + 318, 318, 318, 321, 120, 133, 133, 135, 213, 213, + 317, 316, 316, 316, 316, 212, 212, 211, 211, 205, + 205, 192, 192, 192, 192, 315, 191, 313, 592, 592, + 591, 591, 590, 588, 588, 588, 589, 589, 589, 589, + 537, 537, 537, 537, 537, 351, 351, 351, 356, 356, + 354, 354, 354, 354, 354, 360, 7, 7, 7, 7, + 7, 7, 7, 7, 7, 33, 33, 33, 33, 33, + 33, 33, 33, 33, 33, 33, 33, 33, 39, 224, + 225, 40, 226, 226, 227, 227, 228, 228, 229, 230, + 231, 231, 231, 231, 38, 216, 216, 217, 217, 218, + 218, 219, 220, 220, 220, 223, 221, 222, 222, 610, + 610, 609, 37, 37, 30, 171, 171, 172, 172, 172, + 174, 174, 275, 275, 275, 173, 173, 175, 175, 175, + 569, 571, 571, 573, 572, 572, 572, 575, 575, 575, + 575, 575, 576, 576, 576, 576, 577, 577, 31, 154, + 154, 178, 178, 159, 580, 580, 580, 579, 579, 581, + 581, 582, 582, 337, 337, 338, 338, 169, 170, 170, + 161, 156, 177, 177, 177, 177, 177, 179, 179, 243, + 243, 155, 160, 163, 165, 570, 578, 578, 578, 421, + 421, 418, 419, 419, 416, 415, 415, 415, 584, 584, + 583, 583, 583, 352, 352, 32, 411, 411, 413, 414, + 414, 414, 405, 405, 405, 405, 36, 409, 409, 410, + 410, 410, 410, 410, 410, 410, 406, 406, 408, 408, + 404, 404, 404, 404, 404, 404, 404, 35, 176, 176, + 403, 403, 400, 400, 398, 398, 399, 399, 397, 397, + 397, 401, 401, 43, 78, 44, 45, 46, 42, 402, + 402, 34, 34, 34, 34, 34, 34, 34, 34, 34, + 34, 138, 137, 137, 137, 137, 137, 140, 140, 336, + 336, 335, 335, 139, 276, 276, 41, 254, 254, 476, + 476, 471, 471, 471, 471, 471, 492, 492, 492, 472, + 472, 472, 473, 473, 473, 475, 475, 475, 474, 474, + 474, 474, 474, 491, 491, 493, 493, 493, 443, 443, + 444, 444, 444, 447, 447, 463, 463, 464, 464, 462, + 462, 469, 469, 468, 468, 467, 467, 466, 466, 465, + 465, 465, 465, 458, 458, 457, 457, 445, 445, 445, + 445, 445, 446, 446, 446, 456, 456, 461, 461, 307, + 307, 306, 306, 262, 262, 263, 263, 305, 305, 260, + 260, 261, 261, 261, 304, 304, 304, 304, 304, 304, 304, 304, 304, 304, 304, 304, 304, 304, 304, 304, 304, 304, 304, 304, 304, 304, 304, 304, 304, 304, - 304, 304, 304, 304, 304, 304, 304, 304, 542, 542, - 543, 265, 265, 277, 277, 277, 277, 277, 277, 264, - 264, 266, 266, 242, 242, 240, 240, 232, 232, 232, - 232, 232, 233, 233, 234, 234, 235, 235, 235, 239, - 239, 238, 238, 238, 238, 236, 236, 237, 237, 237, - 237, 237, 237, 428, 428, 539, 539, 540, 540, 535, - 535, 535, 538, 538, 538, 538, 538, 538, 538, 541, - 541, 541, 537, 537, 244, 329, 329, 329, 352, 352, - 352, 352, 354, 328, 328, 328, 259, 259, 258, 258, - 256, 256, 256, 256, 256, 256, 256, 256, 256, 256, + 304, 304, 304, 304, 304, 304, 304, 304, 304, 304, + 543, 543, 544, 265, 265, 277, 277, 277, 277, 277, + 277, 264, 264, 266, 266, 242, 242, 240, 240, 232, + 232, 232, 232, 232, 233, 233, 234, 234, 235, 235, + 235, 239, 239, 238, 238, 238, 238, 236, 236, 237, + 237, 237, 237, 237, 237, 429, 429, 540, 540, 541, + 541, 536, 536, 536, 539, 539, 539, 539, 539, 539, + 539, 542, 542, 542, 538, 538, 244, 330, 330, 330, + 353, 353, 353, 353, 355, 329, 329, 329, 259, 259, + 258, 258, 256, 256, 256, 256, 256, 256, 256, 256, 256, 256, 256, 256, 256, 256, 256, 256, 256, 256, - 256, 256, 427, 427, 368, 368, 369, 369, 288, 287, - 287, 287, 287, 287, 285, 286, 284, 284, 284, 284, - 284, 281, 281, 280, 280, 280, 282, 282, 282, 282, - 282, 406, 406, 278, 278, 268, 268, 268, 267, 267, - 267, 469, 375, 375, 375, 375, 375, 375, 375, 375, - 375, 375, 375, 375, 375, 377, 377, 377, 377, 377, - 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, - 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, - 377, 377, 283, 326, 326, 326, 327, 327, 327, 327, - 327, 327, 327, 327, 378, 378, 384, 384, 551, 551, - 550, 245, 245, 245, 246, 246, 246, 246, 246, 246, - 246, 246, 246, 255, 255, 255, 451, 451, 451, 451, - 452, 452, 452, 452, 453, 453, 453, 449, 449, 450, - 450, 389, 390, 390, 498, 498, 499, 499, 447, 447, - 448, 325, 325, 325, 325, 325, 325, 325, 325, 325, - 325, 325, 325, 325, 325, 325, 325, 325, 325, 325, - 325, 325, 325, 325, 506, 506, 506, 322, 322, 322, - 322, 322, 322, 322, 322, 322, 322, 322, 322, 322, - 322, 322, 322, 562, 562, 562, 547, 547, 547, 548, - 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, + 256, 256, 256, 256, 428, 428, 369, 369, 370, 370, + 288, 287, 287, 287, 287, 287, 285, 286, 284, 284, + 284, 284, 284, 281, 281, 280, 280, 280, 282, 282, + 282, 282, 282, 407, 407, 278, 278, 268, 268, 268, + 267, 267, 267, 470, 376, 376, 376, 376, 376, 376, + 376, 376, 376, 376, 376, 376, 376, 378, 378, 378, + 378, 378, 378, 378, 378, 378, 378, 378, 378, 378, + 378, 378, 378, 378, 378, 378, 378, 378, 378, 378, + 378, 378, 378, 378, 283, 327, 327, 327, 328, 328, + 328, 328, 328, 328, 328, 328, 379, 379, 385, 385, + 552, 552, 551, 245, 245, 245, 246, 246, 246, 246, + 246, 246, 246, 246, 246, 255, 255, 255, 452, 452, + 452, 452, 453, 453, 453, 453, 454, 454, 454, 450, + 450, 451, 451, 390, 391, 391, 499, 499, 500, 500, + 448, 448, 449, 326, 326, 326, 326, 326, 326, 326, + 326, 326, 326, 326, 326, 326, 326, 326, 326, 326, + 326, 326, 326, 326, 326, 326, 507, 507, 507, 323, + 323, 323, 323, 323, 323, 323, 323, 323, 323, 323, + 323, 323, 323, 323, 323, 563, 563, 563, 548, 548, 548, 549, 549, 549, 549, 549, 549, 549, 549, 549, - 549, 549, 549, 549, 549, 549, 549, 549, 324, 324, - 324, 323, 323, 323, 323, 323, 323, 323, 323, 323, - 323, 323, 323, 323, 323, 323, 323, 323, 323, 391, - 391, 392, 392, 503, 503, 503, 503, 503, 503, 504, - 504, 505, 505, 505, 505, 496, 496, 496, 496, 496, - 496, 496, 496, 496, 496, 496, 496, 496, 496, 496, - 496, 496, 496, 496, 496, 496, 496, 496, 496, 496, - 496, 496, 496, 496, 376, 321, 321, 321, 393, 385, - 385, 386, 386, 387, 387, 379, 379, 379, 379, 379, - 379, 380, 380, 382, 382, 382, 382, 382, 382, 382, - 382, 382, 382, 382, 374, 374, 374, 374, 374, 374, - 374, 374, 374, 374, 374, 381, 381, 383, 383, 395, - 395, 395, 394, 394, 394, 394, 394, 394, 394, 257, - 257, 257, 257, 373, 373, 373, 372, 372, 372, 372, - 372, 372, 372, 372, 372, 372, 372, 372, 247, 247, - 247, 247, 251, 251, 253, 253, 253, 253, 253, 253, - 253, 253, 253, 253, 253, 253, 253, 253, 252, 252, - 252, 252, 252, 250, 250, 250, 250, 250, 248, 248, + 549, 549, 549, 550, 550, 550, 550, 550, 550, 550, + 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, + 325, 325, 325, 324, 324, 324, 324, 324, 324, 324, + 324, 324, 324, 324, 324, 324, 324, 324, 324, 324, + 324, 392, 392, 393, 393, 504, 504, 504, 504, 504, + 504, 505, 505, 506, 506, 506, 506, 497, 497, 497, + 497, 497, 497, 497, 497, 497, 497, 497, 497, 497, + 497, 497, 497, 497, 497, 497, 497, 497, 497, 497, + 497, 497, 497, 497, 497, 497, 497, 377, 322, 322, + 322, 394, 386, 386, 387, 387, 388, 388, 380, 380, + 380, 380, 380, 380, 381, 381, 383, 383, 383, 383, + 383, 383, 383, 383, 383, 383, 383, 375, 375, 375, + 375, 375, 375, 375, 375, 375, 375, 375, 382, 382, + 384, 384, 396, 396, 396, 395, 395, 395, 395, 395, + 395, 395, 257, 257, 257, 257, 374, 374, 374, 373, + 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, + 373, 247, 247, 247, 247, 251, 251, 253, 253, 253, + 253, 253, 253, 253, 253, 253, 253, 253, 253, 253, + 253, 252, 252, 252, 252, 252, 250, 250, 250, 250, + 250, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, - 248, 248, 248, 248, 248, 248, 248, 118, 119, 119, - 249, 331, 331, 476, 476, 479, 479, 477, 477, 478, - 480, 480, 480, 481, 481, 481, 482, 482, 482, 486, - 486, 340, 340, 340, 348, 348, 347, 347, 347, 347, - 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, - 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, - 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, - 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, - 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, - 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, - 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, - 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, - 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, - 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, - 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, - 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, - 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, - 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, - 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, - 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, - 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, - 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, - 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, - 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, - 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, - 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, - 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, - 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, - 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, - 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, - 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, - 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, - 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, - 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, - 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, - 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, - 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, - 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, - 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, - 347, 346, 346, 346, 346, 346, 346, 346, 346, 346, - 346, 345, 345, 345, 345, 345, 345, 345, 345, 345, - 345, 345, 345, 345, 345, 345, 345, 345, 345, 345, - 345, 345, 345, 345, 345, 345, 345, 345, 345, 345, - 345, 345, 345, 345, 345, 345, 345, 345, 345, 345, - 345, 345, 345, 345, 345, 345, 345, 345, 345, 345, - 345, + 118, 119, 119, 249, 332, 332, 477, 477, 480, 480, + 478, 478, 479, 481, 481, 481, 482, 482, 482, 483, + 483, 483, 487, 487, 341, 341, 341, 349, 349, 348, + 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, + 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, + 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, + 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, + 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, + 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, + 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, + 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, + 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, + 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, + 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, + 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, + 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, + 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, + 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, + 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, + 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, + 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, + 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, + 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, + 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, + 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, + 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, + 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, + 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, + 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, + 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, + 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, + 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, + 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, + 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, + 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, + 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, + 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, + 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, + 348, 348, 348, 348, 347, 347, 347, 347, 347, 347, + 347, 347, 347, 347, 346, 346, 346, 346, 346, 346, + 346, 346, 346, 346, 346, 346, 346, 346, 346, 346, + 346, 346, 346, 346, 346, 346, 346, 346, 346, 346, + 346, 346, 346, 346, 346, 346, 346, 346, 346, 346, + 346, 346, 346, 346, 346, 346, 346, 346, 346, 346, + 346, 346, 346, 346, } var yyR2 = [...]int{ @@ -8886,103 +8889,102 @@ var yyR2 = [...]int{ 1, 0, 2, 0, 4, 7, 6, 6, 3, 5, 0, 2, 0, 2, 1, 3, 1, 2, 3, 5, 0, 1, 2, 1, 3, 1, 1, 1, 1, 4, - 4, 4, 3, 3, 2, 2, 2, 2, 2, 3, - 2, 3, 4, 1, 3, 4, 0, 2, 1, 1, - 2, 2, 2, 0, 1, 2, 4, 1, 3, 1, - 3, 2, 3, 1, 4, 3, 0, 1, 1, 2, - 5, 2, 2, 2, 0, 2, 3, 3, 0, 1, - 3, 1, 3, 0, 1, 2, 1, 1, 0, 1, - 2, 1, 2, 1, 1, 1, 1, 1, 1, 1, + 4, 4, 3, 4, 3, 2, 2, 2, 2, 2, + 3, 2, 3, 2, 4, 1, 3, 4, 0, 2, + 1, 1, 2, 2, 2, 0, 1, 2, 4, 1, + 3, 1, 3, 2, 3, 1, 4, 3, 0, 1, + 1, 2, 5, 2, 2, 2, 0, 2, 3, 3, + 0, 1, 3, 1, 3, 0, 1, 2, 1, 1, + 0, 1, 2, 1, 2, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, 7, 1, + 1, 7, 1, 3, 0, 1, 1, 3, 1, 3, + 0, 1, 1, 1, 14, 1, 3, 0, 1, 1, + 3, 1, 1, 2, 4, 1, 1, 1, 1, 0, + 1, 2, 9, 9, 7, 1, 2, 3, 3, 3, + 0, 4, 1, 1, 1, 1, 1, 0, 1, 1, + 1, 1, 1, 4, 1, 1, 1, 3, 3, 4, + 3, 3, 0, 1, 1, 1, 0, 2, 7, 8, + 10, 2, 2, 8, 0, 3, 3, 0, 3, 0, + 3, 0, 5, 1, 3, 0, 3, 3, 0, 2, + 9, 8, 0, 2, 2, 3, 3, 0, 2, 0, + 2, 4, 4, 4, 4, 1, 0, 2, 2, 1, + 3, 2, 1, 3, 2, 1, 3, 2, 0, 1, + 3, 4, 3, 1, 1, 4, 1, 3, 1, 1, + 1, 1, 0, 1, 1, 1, 11, 0, 2, 3, + 3, 2, 2, 3, 1, 1, 1, 3, 3, 4, + 0, 2, 2, 2, 2, 2, 2, 6, 0, 4, + 1, 1, 0, 3, 0, 1, 1, 2, 4, 4, + 4, 0, 1, 8, 2, 4, 4, 4, 9, 0, + 2, 11, 9, 11, 8, 6, 9, 7, 10, 7, + 6, 2, 2, 9, 4, 5, 3, 0, 4, 1, + 3, 0, 3, 6, 0, 2, 10, 0, 2, 0, + 2, 0, 3, 2, 4, 3, 0, 2, 1, 0, + 2, 3, 0, 2, 3, 0, 2, 1, 0, 3, + 2, 4, 3, 0, 1, 0, 1, 1, 0, 6, + 0, 3, 5, 0, 4, 0, 3, 1, 3, 4, + 5, 0, 3, 1, 3, 2, 3, 1, 2, 0, + 4, 6, 5, 0, 2, 0, 2, 4, 5, 4, + 5, 1, 5, 6, 5, 0, 3, 0, 1, 1, + 3, 3, 3, 0, 4, 1, 3, 3, 3, 0, + 1, 1, 3, 2, 3, 3, 3, 4, 4, 3, + 3, 3, 3, 4, 4, 3, 3, 3, 3, 3, + 3, 3, 3, 3, 3, 3, 3, 2, 3, 3, + 3, 3, 3, 3, 3, 3, 1, 5, 4, 5, + 1, 3, 3, 2, 2, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 3, 2, 4, 0, + 5, 5, 5, 5, 0, 1, 1, 3, 1, 1, + 1, 1, 1, 7, 9, 7, 9, 2, 1, 7, + 9, 7, 9, 8, 5, 0, 1, 0, 1, 1, + 1, 1, 3, 3, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 0, 1, 3, 1, 3, 5, + 1, 1, 1, 1, 1, 1, 3, 5, 0, 1, + 1, 2, 1, 2, 2, 1, 1, 2, 2, 2, + 3, 3, 2, 2, 1, 5, 6, 4, 1, 1, + 1, 5, 4, 1, 1, 2, 0, 1, 1, 2, + 5, 0, 1, 1, 2, 2, 3, 3, 1, 1, + 2, 2, 2, 0, 1, 2, 2, 2, 0, 4, + 7, 3, 3, 0, 3, 0, 3, 1, 1, 1, + 1, 1, 1, 1, 3, 3, 3, 3, 3, 3, + 3, 3, 3, 3, 3, 3, 1, 1, 1, 1, + 3, 5, 2, 2, 2, 2, 4, 1, 1, 2, + 5, 6, 8, 6, 6, 6, 1, 1, 1, 1, + 1, 1, 3, 9, 1, 4, 4, 4, 7, 9, + 7, 7, 7, 9, 7, 7, 0, 2, 0, 1, + 1, 2, 4, 1, 2, 2, 1, 2, 2, 1, + 2, 2, 2, 2, 2, 0, 1, 1, 1, 2, + 2, 2, 2, 2, 2, 2, 1, 1, 1, 2, + 5, 0, 1, 3, 0, 1, 0, 2, 0, 2, + 0, 1, 6, 8, 8, 6, 6, 5, 5, 5, + 6, 6, 6, 6, 5, 6, 6, 6, 6, 6, + 6, 6, 6, 6, 6, 6, 1, 1, 1, 4, + 4, 6, 8, 6, 4, 5, 4, 4, 4, 3, + 4, 6, 6, 7, 4, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 7, 1, 1, 7, - 1, 3, 0, 1, 1, 3, 1, 3, 0, 1, - 1, 1, 14, 1, 3, 0, 1, 1, 3, 1, - 1, 2, 4, 1, 1, 1, 1, 0, 1, 2, - 9, 9, 7, 1, 2, 3, 3, 3, 0, 4, - 1, 1, 1, 1, 1, 0, 1, 1, 1, 1, - 1, 4, 1, 1, 1, 3, 3, 4, 3, 3, - 0, 1, 1, 1, 0, 2, 7, 8, 10, 2, - 2, 8, 0, 3, 3, 0, 3, 0, 3, 0, - 5, 1, 3, 0, 3, 3, 0, 2, 9, 8, - 0, 2, 2, 3, 3, 0, 2, 0, 2, 4, - 4, 4, 4, 1, 0, 2, 2, 1, 3, 2, - 1, 3, 2, 1, 3, 2, 0, 1, 3, 4, - 3, 1, 1, 4, 1, 3, 1, 1, 1, 1, - 0, 1, 1, 1, 11, 0, 2, 3, 3, 2, - 2, 3, 1, 1, 1, 3, 3, 4, 0, 2, - 2, 2, 2, 2, 2, 6, 0, 4, 1, 1, - 0, 3, 0, 1, 1, 2, 4, 4, 4, 0, - 1, 8, 2, 4, 4, 4, 9, 0, 2, 11, - 9, 11, 8, 6, 9, 7, 10, 7, 6, 2, - 2, 9, 4, 5, 3, 0, 4, 1, 3, 0, - 3, 6, 0, 2, 10, 0, 2, 0, 2, 0, - 3, 2, 4, 3, 0, 2, 1, 0, 2, 3, - 0, 2, 3, 0, 2, 1, 0, 3, 2, 4, - 3, 0, 1, 0, 1, 1, 0, 6, 0, 3, - 5, 0, 4, 0, 3, 1, 3, 4, 5, 0, - 3, 1, 3, 2, 3, 1, 2, 0, 4, 6, - 5, 0, 2, 0, 2, 4, 5, 4, 5, 1, - 5, 6, 5, 0, 3, 0, 1, 1, 3, 3, - 3, 0, 4, 1, 3, 3, 3, 0, 1, 1, - 3, 2, 3, 3, 3, 4, 4, 3, 3, 3, - 3, 4, 4, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 2, 3, 3, 3, 3, - 3, 3, 3, 3, 1, 5, 4, 5, 1, 3, - 3, 2, 2, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 3, 2, 4, 0, 5, 5, - 5, 5, 0, 1, 1, 3, 1, 1, 1, 1, - 1, 7, 9, 7, 9, 2, 1, 7, 9, 7, - 9, 8, 5, 0, 1, 0, 1, 1, 1, 1, - 3, 3, 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 0, 1, 3, 1, 3, 5, 1, 1, - 1, 1, 1, 1, 3, 5, 0, 1, 1, 2, - 1, 2, 2, 1, 1, 2, 2, 2, 3, 3, - 2, 2, 1, 5, 6, 4, 1, 1, 1, 5, - 4, 1, 1, 2, 0, 1, 1, 2, 5, 0, - 1, 1, 2, 2, 3, 3, 1, 1, 2, 2, - 2, 0, 1, 2, 2, 2, 0, 4, 7, 3, - 3, 0, 3, 0, 3, 1, 1, 1, 1, 1, - 1, 1, 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 1, 1, 1, 1, 3, 5, - 2, 2, 2, 2, 4, 1, 1, 2, 5, 6, - 8, 6, 6, 6, 1, 1, 1, 1, 1, 1, - 3, 9, 1, 4, 4, 4, 7, 9, 7, 7, - 7, 9, 7, 7, 0, 2, 0, 1, 1, 2, - 4, 1, 2, 2, 1, 2, 2, 1, 2, 2, - 2, 2, 2, 0, 1, 1, 1, 2, 2, 2, - 2, 2, 2, 2, 1, 1, 1, 2, 5, 0, - 1, 3, 0, 1, 0, 2, 0, 2, 0, 1, - 6, 8, 8, 6, 6, 5, 5, 5, 6, 6, - 6, 6, 5, 6, 6, 6, 6, 6, 6, 6, - 6, 6, 6, 6, 1, 1, 1, 4, 4, 6, - 8, 6, 4, 5, 4, 4, 4, 3, 4, 6, - 6, 7, 4, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, + 2, 2, 8, 4, 2, 3, 2, 4, 2, 2, + 4, 6, 2, 2, 4, 6, 4, 2, 4, 4, + 4, 0, 1, 2, 3, 1, 1, 1, 1, 1, + 1, 0, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, 2, 2, - 8, 4, 2, 3, 2, 4, 2, 2, 4, 6, - 2, 2, 4, 6, 4, 2, 4, 4, 4, 0, - 1, 2, 3, 1, 1, 1, 1, 1, 1, 0, - 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 3, 0, 1, + 1, 3, 0, 1, 1, 3, 1, 3, 3, 3, + 3, 3, 2, 1, 1, 1, 3, 4, 3, 4, + 3, 4, 3, 4, 3, 4, 1, 3, 4, 4, + 5, 4, 5, 3, 4, 5, 6, 1, 0, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 3, 0, 1, 1, 3, 0, - 1, 1, 3, 1, 3, 3, 3, 3, 3, 2, - 1, 1, 1, 3, 4, 3, 4, 3, 4, 3, - 4, 3, 4, 1, 3, 4, 4, 5, 4, 5, - 3, 4, 5, 6, 1, 0, 2, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, 1, 2, - 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 2, 1, 1, 1, 2, 3, 1, - 1, 1, 2, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, 2, 2, - 2, 2, 2, 1, 2, 2, 2, 2, 2, 2, - 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 2, 2, 4, 4, 1, 2, 3, 5, - 1, 1, 3, 0, 1, 0, 3, 0, 3, 3, - 0, 3, 5, 0, 3, 5, 0, 1, 1, 0, - 1, 1, 2, 2, 0, 1, 1, 1, 1, 1, + 1, 1, 2, 2, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 2, 1, 1, 1, + 2, 3, 1, 1, 1, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, + 1, 2, 2, 2, 2, 2, 1, 2, 2, 2, + 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 2, 2, 4, 4, 1, + 2, 3, 5, 1, 1, 3, 0, 1, 0, 3, + 0, 3, 3, 0, 3, 5, 0, 3, 5, 0, + 1, 1, 0, 1, 1, 2, 2, 0, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, @@ -9023,415 +9025,416 @@ var yyR2 = [...]int{ 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, - 1, + 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, } var yyChk = [...]int{ - -1000, -607, -610, -2, -5, 628, -1, -4, -119, -91, + -1000, -608, -611, -2, -5, 629, -1, -4, -119, -91, -7, -14, -121, -122, -8, -117, -9, -10, -12, -95, -112, -114, -116, -115, -47, -11, -111, -85, -86, -97, -105, -108, -109, -110, -123, -118, -120, -180, -124, -125, - -126, -168, -129, -131, -132, 618, -92, -93, -94, -33, - -32, -31, -30, -154, -159, -162, -164, -127, 551, 624, - 454, 14, 503, -15, -16, -552, -17, 263, -356, -357, - -358, -360, -611, -48, -49, -50, -60, -61, -62, -63, + -126, -168, -129, -131, -132, 619, -92, -93, -94, -33, + -32, -31, -30, -154, -159, -162, -164, -127, 552, 625, + 455, 14, 504, -15, -16, -553, -17, 264, -357, -358, + -359, -361, -612, -48, -49, -50, -60, -61, -62, -63, -64, -74, -75, -76, -51, -52, -53, -56, -54, -67, -66, -68, -69, -70, -71, -72, -73, -55, -59, -157, -158, -77, -57, -78, -58, -166, -128, -79, -80, -81, -83, -82, -88, -84, -89, -156, -161, -13, -167, -90, - 237, -87, 76, -98, -99, -100, -101, -102, -103, -104, - -106, -107, 378, 384, 441, 617, 61, -181, -183, 645, - 646, 649, 538, 541, 161, 162, 164, 165, -34, -35, + 238, -87, 76, -98, -99, -100, -101, -102, -103, -104, + -106, -107, 379, 385, 442, 618, 61, -181, -183, 646, + 647, 650, 539, 542, 162, 163, 165, 166, -34, -35, -36, -37, -38, -39, -41, -40, -42, -43, -44, -45, - -46, 233, 16, 546, -18, -21, -19, -22, -20, -28, + -46, 234, 16, 547, -18, -21, -19, -22, -20, -28, -29, -27, -24, -26, -155, -25, -160, -23, -163, -165, - -130, 258, 257, 38, 324, 325, 326, 382, 256, 234, - 236, 15, 31, 42, 357, -182, 84, 539, 235, -184, - 13, 651, -6, -3, -2, -141, -145, -149, -152, -153, - -150, -151, -4, -119, 119, 248, 619, -352, 374, 620, - 622, 621, 87, 95, -345, -347, 454, 263, 378, 384, - 617, 646, 649, 538, 541, 553, 554, 555, 556, 557, - 558, 559, 560, 562, 563, 564, 565, 566, 567, 568, - 578, 579, 569, 570, 571, 572, 573, 574, 575, 576, - 580, 581, 582, 583, 584, 585, 586, 587, 588, 589, - 590, 591, 592, 593, 506, 603, 604, 605, 606, 534, - 561, 597, 598, 599, 600, 355, 356, 543, 275, 299, - 409, 305, 312, 161, 180, 174, 203, 194, 539, 169, - 279, 317, 280, 94, 164, 489, 109, 466, 438, 167, - 294, 296, 298, 535, 536, 368, 301, 533, 300, 302, - 304, 537, 281, 358, 190, 185, 293, 277, 183, 282, - 40, 283, 208, 284, 285, 548, 462, 354, 468, 309, - 52, 436, 184, 463, 297, 465, 212, 216, 480, 481, - 178, 179, 470, 483, 207, 210, 211, 255, 351, 352, - 43, 545, 267, 484, 214, 641, 206, 201, 492, 313, - 311, 205, 177, 200, 278, 65, 218, 217, 219, 432, - 433, 434, 435, 286, 287, 372, 479, 197, 186, 359, - 170, 23, 487, 262, 467, 385, 288, 306, 314, 213, - 215, 269, 274, 329, 547, 440, 273, 310, 485, 182, - 266, 295, 261, 488, 642, 171, 387, 289, 165, 303, - 482, 644, 491, 64, 400, 176, 168, 635, 252, 162, - 271, 276, 643, 290, 291, 292, 532, 316, 315, 307, - 540, 198, 268, 204, 188, 175, 199, 163, 270, 490, - 401, 615, 357, 419, 196, 193, 272, 245, 486, 469, - 166, 423, 402, 191, 318, 610, 611, 612, 373, 350, - 319, 320, 189, 259, 460, 461, 323, 429, 341, 403, - 439, 410, 404, 225, 226, 327, 472, 474, 209, 613, - 330, 331, 332, 464, 333, 334, 335, 336, 377, 56, - 58, 96, 99, 98, 647, 648, 63, 29, 363, 366, - 398, 405, 343, 616, 544, 340, 344, 345, 367, 25, - 421, 389, 425, 424, 48, 49, 50, 53, 54, 55, - 57, 59, 60, 51, 531, 382, 395, 493, 45, 47, - 392, 27, 369, 420, 442, 339, 422, 453, 46, 451, - 452, 473, 26, 371, 370, 62, 44, 428, 430, 431, - 321, 337, 380, 625, 494, 375, 391, 394, 376, 342, - 365, 396, 67, 66, 388, 626, 383, 381, 338, 549, - 550, 346, 577, 360, 437, 528, 527, 526, 525, 524, - 523, 522, 521, 324, 325, 326, 406, 407, 408, 418, - 411, 412, 413, 414, 415, 416, 417, 456, 457, 627, - 475, 477, 478, 476, 240, 650, 361, 362, 243, 629, - 630, 97, 631, 633, 632, 28, 634, 640, 637, 638, - 639, -433, -431, -352, 539, 617, 384, 538, 541, 378, - 357, 646, 649, 382, 263, 324, 325, 326, 454, 355, - -226, -352, 650, -190, 247, 39, -240, -352, -190, -87, - -16, -15, -182, -183, -240, 242, -361, 24, 436, -96, - 437, 237, 238, 84, 77, -352, -9, -110, -8, -117, - -85, -180, 441, -359, -352, 324, 324, -359, 242, -354, - 273, 417, -352, -488, 248, -437, -410, 274, -436, -412, - -439, -413, 32, 233, 235, 234, 551, 270, 16, 382, - 244, 14, 13, 383, 256, 25, 26, 28, 15, 384, - 386, 29, 387, 390, 391, 392, 42, 395, 396, 263, - 87, 95, 90, 281, -225, -352, -387, -379, 116, -382, - -374, -375, -377, -330, -526, -372, 84, 143, 144, 151, - 117, 652, -376, -469, 36, 119, 557, 561, 597, 504, - -322, -323, -324, -325, -326, -327, 542, -352, -527, -525, - 90, 100, 102, 106, 107, 105, 103, 155, 187, 104, - 91, 156, -183, 87, -547, 567, -346, 590, 603, 604, - 605, 606, 589, 61, -496, -505, 241, -503, 154, 192, - 259, 188, 14, 149, 429, 189, 598, 599, 600, 564, - 586, 506, 568, 578, 593, 559, 560, 562, 554, 555, - 556, 558, 569, 571, 585, -506, 581, 591, 592, 577, - 601, 602, 637, 594, 595, 596, 633, 89, 88, 584, - 583, 570, 565, 566, 572, 553, 563, 573, 574, 582, - 587, 588, 366, 109, 367, 368, 496, 358, 369, 248, - 436, 70, 370, 371, 372, 373, 374, 503, 375, 71, - 376, 365, 263, 419, 377, 191, 209, 508, 507, 509, - 500, 497, 495, 498, 499, 501, 502, 575, 576, 580, - -133, -135, 608, -601, -314, -602, 6, 7, 8, 9, - -603, 156, -592, 438, 547, 90, 317, 355, 17, 496, - 636, 537, 636, 537, 166, 163, -424, 166, 115, 171, - 170, -424, 636, 168, 635, 327, 269, -401, -171, 355, - 419, 333, 96, 273, -404, -402, 535, -490, 321, 317, - 293, 243, 112, -172, 253, 252, 110, 496, 241, 393, - 312, 56, 58, -563, -564, 230, 231, 232, -554, 529, - -553, -352, 636, 641, 368, 98, 99, 635, 27, 242, - 379, 269, 474, 472, 473, 475, 476, 477, 478, -65, - -507, -487, 469, 468, -365, 461, 467, 459, 471, 462, - 356, 335, 551, 334, 233, 629, 536, 530, -340, 403, - 439, 493, 494, 380, 440, 480, 482, 463, 109, 195, - 192, 243, 245, 242, 635, 355, 496, 419, 96, 333, - 242, -563, 641, 163, 480, 482, 438, 273, 417, 41, - -430, 429, -429, -431, 481, 492, 88, 89, 479, -340, - 109, 460, 460, -601, -314, -181, -183, -120, -552, 537, - 636, 243, 355, 419, 273, 244, 242, 532, 535, 245, - 496, 241, 324, 379, 269, 333, 96, 168, 635, -489, - -560, 32, -497, 227, 228, 229, 33, 34, -1, 123, - 651, -379, -379, -6, 654, -6, -379, -352, -352, 158, - -247, -251, -248, -250, -249, -253, -252, 192, 193, 154, - 196, 202, 198, 199, 200, 201, 203, 204, 205, 206, - 207, 210, 211, 208, 31, 209, 259, 188, 189, 190, - 191, 212, 174, 194, 544, 220, 175, 221, 176, 222, - 177, 223, 178, 179, 224, 180, 183, 184, 185, 186, - 182, 157, -215, 90, 32, 84, 157, 90, -207, 265, - -190, -240, -232, 157, 652, -207, -601, -200, -201, 11, - -240, -328, -352, 438, 126, -96, 77, -96, 437, 77, - -96, 437, 237, -555, -556, -557, -559, 237, 437, 436, - 238, -115, 157, 281, 17, -359, -359, 82, -240, -412, - 273, -437, -410, 36, 81, 158, 246, 158, 81, 84, - 380, 355, 419, 381, 496, 242, 393, 245, 273, 394, - 355, 419, 242, 245, 496, 273, 355, 242, 245, 419, - 273, 394, 355, 459, 460, 245, 27, 385, 388, 389, - 460, -511, 492, 158, 115, 112, 113, 114, -379, 133, - -394, 126, 127, 128, 129, 130, 131, 132, 140, 139, - 150, 143, 144, 145, 146, 147, 148, 149, 141, 142, - 136, 116, 134, 138, 135, 118, 153, -183, -379, -387, - 61, -377, -377, -377, -377, -352, -469, -384, -379, 84, - 84, 84, 84, 84, 157, 103, 90, -379, 84, 84, - 84, 84, 84, 84, 84, 84, 84, 84, 84, 84, - -504, 84, 84, -391, -392, 84, 84, -372, -328, 84, - 90, 90, 84, 84, 84, 90, 84, 84, 84, -392, - -392, 84, 84, 84, 84, 84, 84, 84, 84, 84, - 84, 84, 84, 84, 84, 84, 84, 84, 84, 84, - 84, 84, 84, 84, 84, 84, 84, -201, 158, -200, - 84, -200, -201, -184, -183, 32, 33, 32, 33, 32, - 33, 32, 33, -604, 626, 84, 100, 647, 225, 17, - -352, 635, -352, -586, 32, 540, 540, 540, 540, 485, - 546, 169, 170, 171, -399, 248, -399, -399, -224, -352, - -226, 379, 245, 532, 245, -172, -399, -399, -399, -399, - -399, 244, -399, 24, 242, 242, 242, 242, -399, 503, - 126, 126, 59, -565, 173, 158, -554, -206, 84, -586, - 642, 643, 644, -364, 134, 138, -364, -310, 18, -310, - 24, 271, 271, 271, -364, 311, -612, -613, 17, 136, - -362, -613, -362, -362, -364, -614, 244, 470, 43, 272, - 271, -202, -203, 22, -202, 464, 460, -454, 465, 466, - -366, -613, -365, -364, -364, -365, -364, -364, -364, 32, - 242, 245, 496, 333, 630, -612, -612, 31, 31, -488, - -488, -240, -488, -488, 530, -341, -352, -488, -488, -488, - -294, -295, -240, -566, 247, 644, -598, -597, 483, -600, - 485, 163, -431, 163, -431, 87, -412, 273, 273, 158, - 126, 24, -432, 126, 137, -431, -431, -432, -432, -264, - 41, -351, 154, -352, 90, -264, 41, -595, -594, -240, - -201, -184, -183, 85, 85, 85, 540, -586, -488, -488, - -488, -488, -488, -490, -488, -488, -488, -488, -488, -359, - -216, -352, -226, 248, -488, -488, -488, -488, -185, -186, - 145, -379, -352, -185, -3, -143, -142, 120, 121, 123, - 620, 374, 619, 623, 617, -431, 41, -482, 401, 400, - -476, -478, 84, -477, 84, -477, -477, -477, -477, -477, - 84, 84, -479, 84, -479, -479, -476, -480, 84, -480, - -481, 84, -481, -480, -352, -458, 546, -385, -387, -352, - 39, -500, 61, -180, 84, 31, 84, -207, -352, 189, - 168, 634, -501, 61, -180, 84, 31, -201, -136, 39, - -203, 21, 157, 100, 90, -115, -96, 77, -115, -96, - -96, 85, 158, -558, 106, 107, -561, 207, 198, -352, - -113, 90, -7, -8, -9, -10, -47, -85, -82, -180, - 234, 538, 541, -528, -526, 84, 32, 428, 81, 17, - -438, 242, 496, 379, 269, 245, 355, -436, -419, -416, - -414, -351, -412, -415, -414, -441, -328, 460, -137, 443, - 442, 323, -379, -379, -379, -379, -379, 105, 116, 350, - 106, 107, -374, -395, 32, 319, 320, -375, -375, -375, - -375, -375, -375, -375, -375, -375, -375, -375, -375, -383, - -393, -469, 84, 136, 134, 138, 135, 118, -377, -377, - -375, -375, -266, -351, 154, 85, 158, -379, -551, -550, - 120, -379, -379, -379, -379, -405, -407, -328, 84, -352, - -548, -549, 510, 511, 512, 513, 514, 515, 516, 517, - 518, 519, 520, 370, 365, 371, 369, 358, 377, 372, - 373, 191, 527, 528, 521, 522, 523, 524, 525, 526, - -385, -385, -379, -548, -385, -321, 33, 32, -387, -387, - -387, 85, -379, -562, 348, 347, 349, -204, -352, -385, - 85, 85, 85, 100, -387, -387, -385, -375, -385, -385, - -385, -385, -549, -321, -321, -321, -321, 145, -387, -387, - -321, -321, -321, -321, 145, -321, -321, -321, -321, -321, - -321, -321, -321, -321, -321, -321, 85, 85, 85, 145, - -387, -202, -135, -509, -508, -379, 41, -136, -203, -605, - 627, 84, -328, -593, 90, 90, 652, 168, 635, 17, - 496, -352, 17, 242, -352, 100, -352, 100, 486, 487, - 167, 170, 169, -352, 116, -352, -352, 35, 84, -216, - -399, -399, -399, -570, -352, 91, -420, -417, -414, -352, - -352, -410, -352, -341, -240, -399, -399, -399, -399, -240, - -275, 53, 54, 55, -414, -173, 56, 57, -566, -553, - 35, -205, -352, -310, -377, -377, -379, -414, 273, -612, - -364, -364, -342, -341, -366, -361, -366, -366, -310, -362, - -364, -364, -379, -366, -362, -310, -352, 460, -310, -310, - -454, -364, -363, -352, -363, -399, -341, -342, -342, -240, - -240, -289, -296, -290, -297, 265, 239, 363, 364, 236, - 234, 11, 235, -304, 312, -400, 504, -270, -271, 77, - 42, -273, 263, 405, 398, 275, 279, 94, 280, 438, - 281, 244, 283, 284, 285, 300, 302, 255, 286, 287, - 288, 429, 289, 162, 301, 290, 291, 292, 381, -265, - 6, 336, 546, 41, 51, 52, 452, 451, 549, 276, - -352, -570, -568, 31, -352, 31, -420, -414, -352, -352, - 158, 246, -193, -195, -192, -188, -189, -194, -313, -315, - -191, 84, -240, -183, -352, -431, 158, 484, 486, 487, - -598, -432, -598, -432, 246, 32, 428, -435, 428, 32, - -410, -429, 480, 482, -425, 90, 429, -415, -434, 81, - 154, -508, -432, -432, -434, -434, 153, 158, -596, 485, - 486, 230, -202, 100, -242, -240, -570, -419, -410, -352, - -488, -242, -242, -242, -354, -354, 84, 157, 36, -352, - -352, -352, -352, -309, 158, -308, 17, -353, -352, 35, - 90, 157, -309, -144, -142, 122, -379, -6, 619, -379, - -6, -6, -379, -6, -379, -486, 402, 100, 100, -331, - 90, -331, 100, 100, 100, 552, 85, 90, -423, 81, - -502, -388, -546, 608, -209, 85, -204, -544, -545, -204, - -208, -352, -500, -232, 126, 126, 126, -502, -209, 85, - -544, -202, 609, -136, -199, -198, -379, -352, 24, -115, - -96, -556, 157, 355, 158, -205, -438, -418, -415, -440, - 145, -352, -426, 158, 546, 655, 88, 246, -583, -582, - 420, 85, 158, -512, 247, 503, 90, 652, 436, 225, - 226, 105, 350, 106, 107, -469, -387, -383, -377, -377, - -375, -375, -381, 260, -381, 115, -379, 653, -378, -550, - 122, -379, 35, 158, 35, 158, 82, 158, 85, -476, - -379, 157, 85, 85, 17, 17, 85, -379, 85, 85, - 85, 85, 17, 17, -379, 85, 157, 85, 85, 85, - 85, 82, 85, 158, 85, 85, 85, 85, 158, -387, - -387, -379, -387, 85, 85, 85, -379, -379, -379, -387, - 85, -379, -379, -379, -379, -379, -379, -379, -379, -379, - -379, -205, -448, 455, -448, -448, 158, 158, 158, 85, - -136, 84, 100, 158, 648, -335, -334, 90, -352, -352, - 168, 635, -352, 90, 635, -352, 90, 168, 171, 171, - 170, 90, 36, 24, 310, 24, -225, -227, -228, -229, - -230, -231, 136, 159, 160, 84, -240, -240, -240, -572, - 406, -584, 158, 41, -582, 496, -169, 323, -403, 82, - -176, 17, 546, -240, -240, -240, -240, -254, 35, 17, - -187, -241, -352, 84, 85, 158, -411, 82, -352, -342, - -310, -310, -366, -310, -310, 158, 23, -364, -366, -366, - -232, -362, -232, 157, -232, -341, -475, 35, -206, 158, - 21, 265, -239, -349, -236, -238, 250, -369, -237, 253, - -540, 251, 249, 110, 254, 308, 111, 244, -349, -349, - 250, -274, 246, 35, -349, -292, 244, 353, 308, 251, - 21, 265, -291, 244, 111, -352, 250, 254, 251, 249, - -348, 126, -340, 153, 246, 43, 381, -348, 550, 265, - -348, -348, -348, -348, -348, -348, -348, 282, 282, -348, - -348, -348, -348, -348, -348, -348, -348, -348, -348, -348, - 163, -348, -348, -348, -348, -348, -348, 84, 328, 277, - 278, 310, -573, 406, 31, 361, 361, 362, -584, 357, - 31, -177, 355, -295, -293, -363, 31, -316, -317, -318, - -319, -320, 68, 72, 74, 78, 69, 70, 71, 75, - 73, 31, 158, -350, -355, 35, -352, 90, -350, -183, - -195, -193, -350, 84, -432, -597, -599, 488, 485, 491, - -434, -434, 100, 246, 84, 126, -434, -434, 41, -351, - -594, 492, 486, -136, 158, 81, -242, -217, -218, -219, - -220, -247, -328, 193, 196, 198, 199, 200, 201, 203, - 204, 205, 206, 207, 210, 211, 208, 209, 259, 188, - 189, 190, 191, 212, 174, 194, 544, 175, 176, 177, - 178, 179, 180, 183, 184, 185, 186, 182, -352, -226, - -310, -186, -195, -352, 90, -352, 145, -310, 123, -6, - 121, -148, -147, -146, 124, 617, 623, 123, 123, 123, - 85, 85, 85, 158, 85, 85, 85, 158, 85, 158, - 100, -515, 465, 40, 158, 84, 85, 158, 61, 158, - 126, 85, 158, -379, -352, 90, -379, 85, 61, -136, - 90, 158, -196, 37, 38, 157, 438, -352, -526, 85, - -440, 158, 246, 157, 157, -416, 384, -351, -418, 21, - 546, -328, 39, -335, 126, 652, -352, 85, -381, -381, - 115, -377, -374, 85, 123, -379, 121, -245, -247, 400, - 401, -379, -245, -246, -252, 154, 192, 259, 191, 190, - 188, 400, 401, -264, -407, 543, -196, 85, -352, -379, - -379, 85, -379, -379, 17, -352, -264, -375, -379, -201, - -201, 85, 85, -447, -448, -447, -447, 85, 85, 85, - 85, -447, 85, 85, 85, 85, 85, 85, 85, 85, - 85, 85, 85, 84, 100, 102, 100, 102, -508, -606, - 63, 625, 62, 428, 105, 313, 158, 100, 90, 653, - 158, 126, 90, -352, -352, 17, 242, -352, 17, 171, - 90, -585, 317, 355, -459, 100, 317, 355, 496, 392, - 85, 158, -328, -217, -206, 84, -206, -575, 467, 408, - 418, -348, -371, -370, 357, 42, -494, 429, 414, 415, - -417, 273, -341, -581, 97, 126, 81, 340, 344, 345, - 341, 342, 343, -397, -398, -396, -400, -568, 84, 84, - -180, 35, 134, -176, 84, 84, 35, -470, 330, -247, - -240, -187, -352, 17, 158, -567, 157, -1, -352, -410, - -364, -310, -379, -379, -310, -364, -364, -366, -352, -232, - -470, -247, 35, -290, 239, 235, -444, 310, 311, -445, - -460, 313, -462, 84, -244, -328, -237, -539, -540, -399, - -352, 111, -539, 111, 84, -244, -328, -328, -293, -328, - -352, -352, -352, -352, -300, -299, -328, -302, 32, -303, - -352, -352, -352, -352, 111, -352, 111, -269, 41, 48, - 49, 50, -348, -348, 195, -272, 41, 428, 430, 431, - -302, 100, 100, 100, 100, 90, 90, 90, -348, -348, - 100, 90, -355, 90, -541, 170, 45, 46, 100, 100, - 100, 100, 41, 90, -277, 41, 293, 297, 294, 295, - 296, 90, 100, 41, 100, 41, 100, 41, -352, 84, - -542, -543, 90, 329, -459, -575, -348, 361, -431, 126, - 126, -371, -577, 94, 409, -580, 323, -179, 496, 32, - -210, 239, 235, -568, -422, -421, -328, -192, -192, -192, - -192, -192, 68, 79, 68, 68, 68, 73, 68, 73, - 68, -318, 79, -422, -194, -206, -355, 85, -591, -590, - -589, -587, 76, 247, 77, -385, -434, 485, 489, 490, - -418, -367, 90, -425, -214, 24, -240, -240, -492, 303, - 304, 85, 158, -247, -312, 19, 157, -312, 119, -6, - -144, -146, -379, -6, -379, 619, 374, 620, 90, 100, - 100, -523, 449, 444, 446, 111, -388, -510, -509, 61, - -180, -204, -502, -545, -508, -352, 653, 653, 653, 653, - 61, -180, -502, -214, -515, -198, -197, 44, -352, 100, - 17, -415, -410, 145, 145, -352, 385, -426, 90, 407, - 90, 242, 653, 90, -335, -374, -379, 85, -255, 179, - 178, -255, 35, 85, 85, -477, -477, -476, -479, -476, - -255, -255, 85, 84, -196, 85, 24, 85, 85, 85, - -379, 85, 85, 158, -498, 505, -499, 579, -447, -447, - -447, -447, -447, -447, -447, -447, -447, -447, -447, -447, - -447, -447, -447, -447, -447, -390, -389, 265, 450, 632, - 632, 450, 632, 632, 85, 158, -548, 158, -343, 318, - -343, -334, 90, 246, 90, 168, -352, 90, 635, 90, - -352, 90, 310, -352, -352, 90, 90, -229, -247, 85, - 35, -233, -234, -235, -244, -236, -238, 35, -576, 94, - -571, 90, -352, 91, -577, 156, 359, 41, 410, 411, - 426, 354, 100, 100, 416, -569, -352, -178, 242, 355, - -579, 52, 126, 90, -240, -396, -340, 153, 284, 333, - -307, -306, -352, 90, -233, -180, -240, -233, -233, -180, - -471, 332, 21, 100, 144, -207, 82, 157, -195, -241, - -352, 145, 85, -310, -232, -310, -310, -364, -471, -180, - -456, 314, 84, -454, 84, -454, 111, 341, -463, -461, - 265, -298, 45, 47, -247, -537, -352, -535, -537, -352, - -535, -535, -399, -379, -298, -244, 246, 31, 235, -301, - 338, 339, 344, -427, 309, 116, -427, 158, -196, 158, - -352, -264, -264, 31, 90, 90, -242, 85, 158, 126, - 100, 90, -576, -571, 126, -432, 90, 90, -577, 90, - 90, -581, 126, -243, 242, -341, 158, -210, -210, -310, - 158, 126, -212, -211, 81, 82, -213, 81, -211, 68, - 68, -310, -589, -588, 24, -540, -540, -540, 85, 85, - 15, -219, 41, -311, 20, 21, 145, -311, 123, 121, - 123, 123, -352, 85, 85, -483, 610, -519, -521, 444, - 21, 21, 15, 247, 85, -502, -502, -523, 45, 46, - -410, -426, 429, -240, 158, 653, -245, -283, 90, -379, - 85, -379, 85, 90, 85, 90, -201, 21, 85, 158, - 85, 85, 85, 158, 85, 85, -379, 85, -548, -344, - 189, 90, -344, 355, -353, -352, 17, -352, 90, -459, - 310, 310, 242, 240, -180, 85, 158, -180, 90, -574, - 420, 100, 41, 100, 156, 412, -495, -170, 94, -242, - 32, -210, -578, 94, 126, 652, 84, -348, -348, -348, - -352, 85, 158, -348, -348, 85, 85, 85, -262, 546, - -472, 264, 100, 144, 100, 144, 100, -350, -195, -352, - -310, -567, 157, -310, -472, -446, 315, 100, -375, 84, - -375, 84, -455, 312, 84, 85, 158, -352, -328, -259, - -258, -256, 105, 116, 41, 398, -257, 94, 153, 298, - 301, 300, 276, 299, -288, -368, 81, 404, 338, 339, - -400, 610, 534, 249, 110, 111, 386, -369, 84, 84, - 82, 318, 84, 84, -537, 85, -298, -328, 41, -301, - 41, 351, 309, -299, -352, 153, -264, 85, -543, 90, - -548, -574, 90, -434, -579, 90, -170, -242, -568, -201, - -421, -508, -379, 84, -379, 84, 68, 11, 19, -372, - -379, -387, 637, 639, 640, 248, -6, 620, 374, -279, - 611, 90, 21, 90, -517, 90, -422, -483, -139, -276, - -340, 281, 85, -282, 136, 546, 85, 85, -447, -447, - -450, -449, -453, 450, 310, 458, -387, 90, 90, 85, - 85, 90, -352, 168, 90, 652, 90, -459, -459, -352, - -221, -247, -174, 546, -262, -235, -174, 21, 546, 358, - 41, 100, 41, 413, 90, -178, 126, 106, 107, -336, - -337, 90, -405, -264, -266, 90, -306, -372, -372, -260, - -180, 35, -261, -304, -400, 546, -138, -137, -260, 546, - 84, -473, 162, 100, 144, 100, 100, -310, -310, -473, - -462, 21, 85, -441, 85, -441, 84, 126, -375, -461, - -464, 61, -256, 105, -375, 90, -266, -267, 41, 297, - 293, 126, 126, -268, 41, 277, 278, -278, 84, 308, - 15, 195, 84, 111, 111, -240, -405, -405, -538, 340, - 341, 342, 346, 344, 345, 343, -538, -405, -405, 84, - -428, -427, -375, -348, -348, 153, -578, -202, -205, -536, - -352, 249, 21, 21, -493, 546, 638, 84, 84, -352, - -352, -332, 612, 100, 90, 446, -279, -484, 613, -513, - -454, -264, 126, 85, 75, 544, 547, 85, -452, 118, - 412, 416, -373, -376, 100, 102, 187, 156, 85, 85, - -352, -339, -338, 90, 90, 90, 310, 545, -175, 60, - 492, 90, 91, 407, 90, 91, 358, -170, 90, 653, - 158, 126, 85, -442, 265, -180, 158, -304, -340, -139, - -442, -263, -305, -352, 90, -491, 170, 331, 546, 100, - 144, 100, -201, -474, 170, 331, -445, 85, 85, 85, - -441, 100, 85, -468, -465, 84, -304, 267, 136, 90, - 90, 100, 84, -503, 31, 90, -406, 84, 85, 85, - 85, 85, -405, 100, -264, -348, 85, 85, 158, 640, - 84, -387, -387, 84, 21, -332, -485, 614, 90, -522, - 449, -516, -514, 444, 445, 446, 447, 90, 545, 65, - 548, -451, -452, 416, -373, -376, 608, 456, 456, 456, - 653, 158, 126, -459, -223, -352, 308, 429, -337, 90, - -408, -443, 317, 21, -304, -348, -443, 85, 158, -348, - -348, 331, 100, 144, 100, -202, 331, -457, 316, 85, - -468, -304, -467, -466, 315, 268, 84, 85, -379, -391, - -348, 85, -281, -280, 542, -405, -408, 82, -408, 82, - -408, 82, -408, 82, 85, -264, -352, 249, -134, 84, - 85, 85, -333, -352, -517, 90, -524, 247, -520, -521, - 448, -514, 21, 446, 21, 21, -140, 158, 65, 115, - 457, 457, 457, -338, 90, 90, -222, 35, 451, 385, - -409, 255, 351, 352, 94, 546, 338, 339, 21, -444, - -264, -305, -372, -372, 100, 100, 85, 158, -352, 264, - 84, -386, -380, -379, 264, 85, -352, -287, -285, -286, - 81, 463, 306, 307, 85, -538, -538, -538, -538, -288, - 85, 158, -385, 85, 158, -531, 84, 100, -519, -518, - -520, 21, -517, 21, -517, -517, 453, 546, -451, 90, - -348, -348, 90, 90, 337, -328, 84, -456, -466, -465, - -386, 85, 158, -427, -286, 81, -285, 81, 16, 15, - -408, -408, -408, -408, 84, 85, -352, -534, 31, 85, - -530, -529, -329, -525, -352, 449, 450, 90, -517, 126, - 547, -609, -608, 631, 100, 100, -352, -441, -446, 85, - -380, -284, 303, 304, 31, 170, -284, -385, -533, -532, - -330, 85, 158, 157, 90, 548, 90, 85, -462, 105, - 41, 305, 85, 158, 126, -529, -352, -532, 41, -379, - 157, -352, + -130, 259, 258, 38, 325, 326, 327, 383, 257, 235, + 237, 15, 31, 42, 358, -182, 85, 540, 236, -184, + 13, 652, -6, -3, -2, -141, -145, -149, -152, -153, + -150, -151, -4, -119, 120, 249, 620, -353, 375, 621, + 623, 622, 88, 96, -346, -348, 455, 264, 379, 385, + 618, 647, 650, 539, 542, 554, 555, 556, 557, 558, + 559, 560, 561, 563, 564, 565, 566, 567, 568, 569, + 579, 580, 570, 571, 572, 573, 574, 575, 576, 577, + 581, 582, 583, 584, 585, 586, 587, 588, 589, 590, + 591, 592, 593, 594, 507, 604, 605, 606, 607, 535, + 562, 598, 599, 600, 601, 356, 357, 544, 276, 300, + 410, 306, 313, 162, 181, 175, 204, 195, 540, 170, + 280, 318, 281, 95, 165, 490, 110, 467, 439, 168, + 295, 297, 299, 536, 537, 369, 302, 534, 301, 303, + 305, 538, 282, 359, 191, 186, 294, 278, 184, 283, + 40, 284, 209, 285, 286, 549, 463, 355, 469, 310, + 52, 437, 185, 464, 298, 466, 213, 217, 481, 482, + 179, 180, 471, 484, 208, 211, 212, 256, 352, 353, + 43, 546, 268, 485, 215, 642, 207, 202, 493, 314, + 312, 206, 178, 201, 279, 65, 219, 218, 220, 433, + 434, 435, 436, 287, 288, 373, 480, 198, 187, 360, + 171, 23, 488, 263, 468, 386, 289, 307, 315, 214, + 216, 270, 275, 330, 548, 441, 274, 311, 486, 183, + 267, 296, 262, 489, 643, 172, 388, 290, 166, 304, + 483, 645, 492, 64, 401, 177, 169, 636, 253, 163, + 272, 277, 644, 291, 292, 293, 533, 317, 316, 308, + 541, 199, 269, 205, 189, 176, 200, 164, 271, 491, + 402, 616, 358, 420, 197, 194, 273, 246, 487, 470, + 167, 424, 403, 192, 319, 611, 612, 613, 374, 351, + 320, 321, 190, 260, 461, 462, 324, 430, 342, 404, + 440, 411, 405, 226, 227, 328, 473, 475, 210, 614, + 331, 332, 333, 465, 334, 335, 336, 337, 378, 56, + 58, 97, 100, 99, 648, 649, 63, 29, 364, 367, + 399, 406, 344, 617, 545, 341, 345, 346, 368, 25, + 422, 390, 426, 425, 48, 49, 50, 53, 54, 55, + 57, 59, 60, 51, 532, 383, 396, 494, 45, 47, + 393, 27, 370, 421, 443, 340, 423, 454, 46, 452, + 453, 474, 26, 372, 371, 62, 44, 429, 431, 432, + 322, 338, 381, 626, 495, 376, 392, 395, 377, 343, + 366, 397, 67, 66, 389, 627, 384, 382, 339, 550, + 551, 347, 578, 361, 438, 529, 528, 527, 526, 525, + 524, 523, 522, 325, 326, 327, 407, 408, 409, 419, + 412, 413, 414, 415, 416, 417, 418, 457, 458, 628, + 476, 478, 479, 477, 241, 651, 362, 363, 244, 630, + 631, 98, 632, 634, 633, 28, 635, 641, 638, 639, + 640, -434, -432, -353, 540, 618, 385, 539, 542, 379, + 358, 647, 650, 383, 264, 325, 326, 327, 455, 356, + -226, -353, 651, -190, 248, 39, -240, -353, -190, -87, + -16, -15, -182, -183, -240, 243, -362, 24, 437, -96, + 438, 238, 239, 85, 77, -353, -9, -110, -8, -117, + -85, -180, 442, -360, -353, 325, 325, -360, 243, -355, + 274, 418, -353, -489, 249, -438, -411, 275, -437, -413, + -440, -414, 32, 234, 236, 235, 552, 271, 16, 383, + 245, 14, 13, 384, 257, 25, 26, 28, 15, 385, + 387, 29, 388, 391, 392, 393, 42, 396, 397, 264, + 88, 96, 91, 282, -225, -353, -388, -380, 117, -383, + -375, -376, -378, -331, -527, -373, 85, 144, 145, 152, + 118, 653, -377, -470, 36, 120, 558, 562, 598, 505, + -323, -324, -325, -326, -327, -328, 543, -353, -528, -526, + 91, 101, 103, 107, 108, 106, 104, 156, 188, 105, + 92, 157, -183, 88, -548, 568, -347, 591, 604, 605, + 606, 607, 590, 61, -497, -506, 242, -504, 155, 193, + 260, 189, 14, 150, 430, 190, 599, 600, 601, 565, + 587, 507, 569, 579, 594, 560, 561, 563, 555, 556, + 557, 559, 570, 572, 586, -507, 582, 592, 593, 578, + 602, 603, 638, 595, 596, 597, 634, 90, 89, 585, + 584, 571, 566, 567, 573, 554, 564, 574, 575, 583, + 588, 589, 367, 110, 368, 369, 497, 359, 80, 370, + 249, 437, 70, 371, 372, 373, 374, 375, 504, 376, + 71, 377, 366, 264, 420, 378, 192, 210, 509, 508, + 510, 501, 498, 496, 499, 500, 502, 503, 576, 577, + 581, -133, -135, 609, -602, -314, -603, 6, 7, 8, + 9, -604, 157, -593, 439, 548, 91, 318, 356, 17, + 497, 637, 538, 637, 538, 167, 164, -425, 167, 116, + 172, 171, -425, 637, 169, 636, 328, 270, -402, -171, + 356, 420, 334, 97, 274, -405, -403, 536, -491, 322, + 318, 294, 244, 113, -172, 254, 253, 111, 497, 242, + 394, 313, 56, 58, -564, -565, 231, 232, 233, -555, + 530, -554, -353, 637, 642, 369, 99, 100, 636, 27, + 243, 380, 270, 475, 473, 474, 476, 477, 478, 479, + -65, -508, -488, 470, 469, -366, 462, 468, 460, 472, + 463, 357, 336, 552, 335, 234, 630, 537, 531, -341, + 404, 440, 494, 495, 381, 441, 481, 483, 464, 110, + 196, 193, 244, 246, 243, 636, 356, 497, 420, 97, + 334, 243, -564, 642, 164, 481, 483, 439, 274, 418, + 41, -431, 430, -430, -432, 482, 493, 89, 90, 480, + -341, 110, 461, 461, -602, -314, -181, -183, -120, -553, + 538, 637, 244, 356, 420, 274, 245, 243, 533, 536, + 246, 497, 242, 325, 380, 270, 334, 97, 169, 636, + -490, -561, 32, -498, 228, 229, 230, 33, 34, -1, + 124, 652, -380, -380, -6, 655, -6, -380, -353, -353, + 159, -247, -251, -248, -250, -249, -253, -252, 193, 194, + 155, 197, 203, 199, 200, 201, 202, 204, 205, 206, + 207, 208, 211, 212, 209, 31, 210, 260, 189, 190, + 191, 192, 213, 175, 195, 545, 221, 176, 222, 177, + 223, 178, 224, 179, 180, 225, 181, 184, 185, 186, + 187, 183, 158, -215, 91, 32, 85, 158, 91, -207, + 266, -190, -240, -232, 158, 653, -207, -602, -200, -201, + 11, -240, -329, -353, 439, 127, -96, 77, -96, 438, + 77, -96, 438, 238, -556, -557, -558, -560, 238, 438, + 437, 239, -115, 158, 282, 17, -360, -360, 83, -240, + -413, 274, -438, -411, 36, 82, 159, 247, 159, 82, + 85, 381, 356, 420, 382, 497, 243, 394, 246, 274, + 395, 356, 420, 243, 246, 497, 274, 356, 243, 246, + 420, 274, 395, 356, 460, 461, 246, 27, 386, 389, + 390, 461, -512, 493, 159, 116, 113, 114, 115, -380, + 134, -395, 127, 128, 129, 130, 131, 132, 133, 141, + 140, 151, 144, 145, 146, 147, 148, 149, 150, 142, + 143, 137, 117, 135, 139, 136, 119, 154, -183, -380, + -388, 61, -378, -378, -378, -378, -353, -470, -385, -380, + 85, 85, 85, 85, 85, 158, 104, 91, -380, 85, + 85, 85, 85, 85, 85, 85, 85, 85, 85, 85, + 85, -505, 85, 85, -392, -393, 85, 85, -373, -329, + 85, 91, 91, 85, 85, 85, 91, 85, 85, 85, + -393, -393, 85, 85, 85, 85, 85, 85, 85, 85, + 85, 85, 85, 85, 85, 85, 85, 85, 85, 85, + 85, 85, 85, 85, 85, 85, 85, 85, -201, 159, + -200, 85, -200, -201, -184, -183, 32, 33, 32, 33, + 32, 33, 32, 33, -605, 627, 85, 101, 648, 226, + 17, -353, 636, -353, -587, 32, 541, 541, 541, 541, + 486, 547, 170, 171, 172, -400, 249, -400, -400, -224, + -353, -226, 380, 246, 533, 246, -172, -400, -400, -400, + -400, -400, 245, -400, 24, 243, 243, 243, 243, -400, + 504, 127, 127, 59, -566, 174, 159, -555, -206, 85, + -587, 643, 644, 645, -365, 135, 139, -365, -310, 18, + -310, 24, 272, 272, 272, -365, 312, -613, -614, 17, + 137, -363, -614, -363, -363, -365, -615, 245, 471, 43, + 273, 272, -202, -203, 22, -202, 465, 461, -455, 466, + 467, -367, -614, -366, -365, -365, -366, -365, -365, -365, + 32, 243, 246, 497, 334, 631, -613, -613, 31, 31, + -489, -489, -240, -489, -489, 531, -342, -353, -489, -489, + -489, -294, -295, -240, -567, 248, 645, -599, -598, 484, + -601, 486, 164, -432, 164, -432, 88, -413, 274, 274, + 159, 127, 24, -433, 127, 138, -432, -432, -433, -433, + -264, 41, -352, 155, -353, 91, -264, 41, -596, -595, + -240, -201, -184, -183, 86, 86, 86, 541, -587, -489, + -489, -489, -489, -489, -491, -489, -489, -489, -489, -489, + -360, -216, -353, -226, 249, -489, -489, -489, -489, -185, + -186, 146, -380, -353, -185, -3, -143, -142, 121, 122, + 124, 621, 375, 620, 624, 618, -432, 41, -483, 402, + 401, -477, -479, 85, -478, 85, -478, -478, -478, -478, + -478, 85, 85, -480, 85, -480, -480, -477, -481, 85, + -481, -482, 85, -482, -481, -353, -459, 547, -386, -388, + -353, 39, -501, 61, -180, 85, 31, 85, -207, -353, + 190, 169, 635, -502, 61, -180, 85, 31, -201, -136, + 39, -203, 21, 158, 101, 91, -115, -96, 77, -115, + -96, -96, 86, 159, -559, 107, 108, -562, 208, 199, + -353, -113, 91, -7, -8, -9, -10, -47, -85, -82, + -180, 235, 539, 542, -529, -527, 85, 32, 429, 82, + 17, -439, 243, 497, 380, 270, 246, 356, -437, -420, + -417, -415, -352, -413, -416, -415, -442, -329, 461, -137, + 444, 443, 324, -380, -380, -380, -380, -380, 106, 117, + 351, 107, 108, -375, -396, 32, 320, 321, -376, -376, + -376, -376, -376, -376, -376, -376, -376, -376, -376, -376, + -384, -394, -470, 85, 137, 135, 139, 136, 119, -378, + -378, -376, -376, -266, -352, 155, 86, 159, -380, -552, + -551, 121, -380, -380, -380, -380, -406, -408, -329, 85, + -353, -549, -550, 511, 512, 513, 514, 515, 516, 517, + 518, 519, 520, 521, 371, 366, 372, 370, 359, 378, + 373, 374, 192, 528, 529, 522, 523, 524, 525, 526, + 527, -386, -386, -380, -549, -386, -322, 33, 32, -388, + -388, -388, 86, -380, -563, 349, 348, 350, -204, -353, + -386, 86, 86, 86, 101, -388, -388, -386, -376, -386, + -386, -386, -386, -550, -322, -322, -322, -322, 146, -388, + -388, -322, -322, -322, -322, 146, -322, -322, -322, -322, + -322, -322, -322, -322, -322, -322, -322, 86, 86, 86, + 146, -388, -202, -135, -510, -509, -380, 41, -136, -203, + -606, 628, 85, -329, -594, 91, 91, 653, 169, 636, + 17, 497, -353, 17, 243, -353, 101, -353, 101, 487, + 488, 168, 171, 170, -353, 117, -353, -353, 35, 85, + -216, -400, -400, -400, -571, -353, 92, -421, -418, -415, + -353, -353, -411, -353, -342, -240, -400, -400, -400, -400, + -240, -275, 53, 54, 55, -415, -173, 56, 57, -567, + -554, 35, -205, -353, -310, -378, -378, -380, -415, 274, + -613, -365, -365, -343, -342, -367, -362, -367, -367, -310, + -363, -365, -365, -380, -367, -363, -310, -353, 461, -310, + -310, -455, -365, -364, -353, -364, -400, -342, -343, -343, + -240, -240, -289, -296, -290, -297, 266, 240, 364, 365, + 237, 235, 11, 236, -304, 313, -401, 505, -270, -271, + 77, 42, -273, 264, 406, 399, 276, 280, 95, 281, + 439, 282, 245, 284, 285, 286, 301, 303, 256, 287, + 288, 289, 430, 290, 163, 302, 291, 292, 293, 382, + -265, 6, 337, 547, 41, 51, 52, 453, 452, 550, + 277, -353, -571, -569, 31, -353, 31, -421, -415, -353, + -353, 159, 247, -193, -195, -192, -188, -189, -194, -313, + -315, -191, 85, -240, -183, -353, -432, 159, 485, 487, + 488, -599, -433, -599, -433, 247, 32, 429, -436, 429, + 32, -411, -430, 481, 483, -426, 91, 430, -416, -435, + 82, 155, -509, -433, -433, -435, -435, 154, 159, -597, + 486, 487, 231, -202, 101, -242, -240, -571, -420, -411, + -353, -489, -242, -242, -242, -355, -355, 85, 158, 36, + -353, -353, -353, -353, -309, 159, -308, 17, -354, -353, + 35, 91, 158, -309, -144, -142, 123, -380, -6, 620, + -380, -6, -6, -380, -6, -380, -487, 403, 101, 101, + -332, 91, -332, 101, 101, 101, 553, 86, 91, -424, + 82, -503, -389, -547, 609, -209, 86, -204, -545, -546, + -204, -208, -353, -501, -232, 127, 127, 127, -503, -209, + 86, -545, -202, 610, -136, -199, -198, -380, -353, 24, + -115, -96, -557, 158, 356, 159, -205, -439, -419, -416, + -441, 146, -353, -427, 159, 547, 656, 89, 247, -584, + -583, 421, 86, 159, -513, 248, 504, 91, 653, 437, + 226, 227, 106, 351, 107, 108, -470, -388, -384, -378, + -378, -376, -376, -382, 261, -382, 116, -380, 654, -379, + -551, 123, -380, 35, 159, 35, 159, 83, 159, 86, + -477, -380, 158, 86, 86, 17, 17, 86, -380, 86, + 86, 86, 86, 17, 17, -380, 86, 158, 86, 86, + 86, 86, 83, 86, 159, 86, 86, 86, 86, 159, + -388, -388, -380, -388, 86, 86, 86, -380, -380, -380, + -388, 86, -380, -380, -380, -380, -380, -380, -380, -380, + -380, -380, -205, -449, 456, -449, -449, 159, 159, 159, + 86, -136, 85, 101, 159, 649, -336, -335, 91, -353, + -353, 169, 636, -353, 91, 636, -353, 91, 169, 172, + 172, 171, 91, 36, 24, 311, 24, -225, -227, -228, + -229, -230, -231, 137, 160, 161, 85, -240, -240, -240, + -573, 407, -585, 159, 41, -583, 497, -169, 324, -404, + 83, -176, 17, 547, -240, -240, -240, -240, -254, 35, + 17, -187, -241, -353, 85, 86, 159, -412, 83, -353, + -343, -310, -310, -367, -310, -310, 159, 23, -365, -367, + -367, -232, -363, -232, 158, -232, -342, -476, 35, -206, + 159, 21, 266, -239, -350, -236, -238, 251, -370, -237, + 254, -541, 252, 250, 111, 255, 309, 112, 245, -350, + -350, 251, -274, 247, 35, -350, -292, 245, 354, 309, + 252, 21, 266, -291, 245, 112, -353, 251, 255, 252, + 250, -349, 127, -341, 154, 247, 43, 382, -349, 551, + 266, -349, -349, -349, -349, -349, -349, -349, 283, 283, + -349, -349, -349, -349, -349, -349, -349, -349, -349, -349, + -349, 164, -349, -349, -349, -349, -349, -349, 85, 329, + 278, 279, 311, -574, 407, 31, 362, 362, 363, -585, + 358, 31, -177, 356, -295, -293, -364, 31, -316, -317, + -318, -319, -321, -320, 68, 72, 74, 78, 69, 70, + 71, 75, 80, 73, 31, 159, -351, -356, 35, -353, + 91, -351, -183, -195, -193, -351, 85, -433, -598, -600, + 489, 486, 492, -435, -435, 101, 247, 85, 127, -435, + -435, 41, -352, -595, 493, 487, -136, 159, 82, -242, + -217, -218, -219, -220, -247, -329, 194, 197, 199, 200, + 201, 202, 204, 205, 206, 207, 208, 211, 212, 209, + 210, 260, 189, 190, 191, 192, 213, 175, 195, 545, + 176, 177, 178, 179, 180, 181, 184, 185, 186, 187, + 183, -353, -226, -310, -186, -195, -353, 91, -353, 146, + -310, 124, -6, 122, -148, -147, -146, 125, 618, 624, + 124, 124, 124, 86, 86, 86, 159, 86, 86, 86, + 159, 86, 159, 101, -516, 466, 40, 159, 85, 86, + 159, 61, 159, 127, 86, 159, -380, -353, 91, -380, + 86, 61, -136, 91, 159, -196, 37, 38, 158, 439, + -353, -527, 86, -441, 159, 247, 158, 158, -417, 385, + -352, -419, 21, 547, -329, 39, -336, 127, 653, -353, + 86, -382, -382, 116, -378, -375, 86, 124, -380, 122, + -245, -247, 401, 402, -380, -245, -246, -252, 155, 193, + 260, 192, 191, 189, 401, 402, -264, -408, 544, -196, + 86, -353, -380, -380, 86, -380, -380, 17, -353, -264, + -376, -380, -201, -201, 86, 86, -448, -449, -448, -448, + 86, 86, 86, 86, -448, 86, 86, 86, 86, 86, + 86, 86, 86, 86, 86, 86, 85, 101, 103, 101, + 103, -509, -607, 63, 626, 62, 429, 106, 314, 159, + 101, 91, 654, 159, 127, 91, -353, -353, 17, 243, + -353, 17, 172, 91, -586, 318, 356, -460, 101, 318, + 356, 497, 393, 86, 159, -329, -217, -206, 85, -206, + -576, 468, 409, 419, -349, -372, -371, 358, 42, -495, + 430, 415, 416, -418, 274, -342, -582, 98, 127, 82, + 341, 345, 346, 342, 343, 344, -398, -399, -397, -401, + -569, 85, 85, -180, 35, 135, -176, 85, 85, 35, + -471, 331, -247, -240, -187, -353, 17, 159, -568, 158, + -1, -353, -411, -365, -310, -380, -380, -310, -365, -365, + -367, -353, -232, -471, -247, 35, -290, 240, 236, -445, + 311, 312, -446, -461, 314, -463, 85, -244, -329, -237, + -540, -541, -400, -353, 112, -540, 112, 85, -244, -329, + -329, -293, -329, -353, -353, -353, -353, -300, -299, -329, + -302, 32, -303, -353, -353, -353, -353, 112, -353, 112, + -269, 41, 48, 49, 50, -349, -349, 196, -272, 41, + 429, 431, 432, -302, 101, 101, 101, 101, 91, 91, + 91, -349, -349, 101, 91, -356, 91, -542, 171, 45, + 46, 101, 101, 101, 101, 41, 91, -277, 41, 294, + 298, 295, 296, 297, 91, 101, 41, 101, 41, 101, + 41, -353, 85, -543, -544, 91, 330, -460, -576, -349, + 362, -432, 127, 127, -372, -578, 95, 410, -581, 324, + -179, 497, 32, -210, 240, 236, -569, -423, -422, -329, + -192, -192, -192, -192, -192, -192, 68, 79, 68, 68, + 68, 73, 68, 73, 68, -318, 68, 79, -423, -194, + -206, -356, 86, -592, -591, -590, -588, 76, 248, 77, + -386, -435, 486, 490, 491, -419, -368, 91, -426, -214, + 24, -240, -240, -493, 304, 305, 86, 159, -247, -312, + 19, 158, -312, 120, -6, -144, -146, -380, -6, -380, + 620, 375, 621, 91, 101, 101, -524, 450, 445, 447, + 112, -389, -511, -510, 61, -180, -204, -503, -546, -509, + -353, 654, 654, 654, 654, 61, -180, -503, -214, -516, + -198, -197, 44, -353, 101, 17, -416, -411, 146, 146, + -353, 386, -427, 91, 408, 91, 243, 654, 91, -336, + -375, -380, 86, -255, 180, 179, -255, 35, 86, 86, + -478, -478, -477, -480, -477, -255, -255, 86, 85, -196, + 86, 24, 86, 86, 86, -380, 86, 86, 159, -499, + 506, -500, 580, -448, -448, -448, -448, -448, -448, -448, + -448, -448, -448, -448, -448, -448, -448, -448, -448, -448, + -391, -390, 266, 451, 633, 633, 451, 633, 633, 86, + 159, -549, 159, -344, 319, -344, -335, 91, 247, 91, + 169, -353, 91, 636, 91, -353, 91, 311, -353, -353, + 91, 91, -229, -247, 86, 35, -233, -234, -235, -244, + -236, -238, 35, -577, 95, -572, 91, -353, 92, -578, + 157, 360, 41, 411, 412, 427, 355, 101, 101, 417, + -570, -353, -178, 243, 356, -580, 52, 127, 91, -240, + -397, -341, 154, 285, 334, -307, -306, -353, 91, -233, + -180, -240, -233, -233, -180, -472, 333, 21, 101, 145, + -207, 83, 158, -195, -241, -353, 146, 86, -310, -232, + -310, -310, -365, -472, -180, -457, 315, 85, -455, 85, + -455, 112, 342, -464, -462, 266, -298, 45, 47, -247, + -538, -353, -536, -538, -353, -536, -536, -400, -380, -298, + -244, 247, 31, 236, -301, 339, 340, 345, -428, 310, + 117, -428, 159, -196, 159, -353, -264, -264, 31, 91, + 91, -242, 86, 159, 127, 101, 91, -577, -572, 127, + -433, 91, 91, -578, 91, 91, -582, 127, -243, 243, + -342, 159, -210, -210, -310, 159, 127, -212, -211, 82, + 83, -213, 82, -211, -211, 68, 68, -310, -590, -589, + 24, -541, -541, -541, 86, 86, 15, -219, 41, -311, + 20, 21, 146, -311, 124, 122, 124, 124, -353, 86, + 86, -484, 611, -520, -522, 445, 21, 21, 15, 248, + 86, -503, -503, -524, 45, 46, -411, -427, 430, -240, + 159, 654, -245, -283, 91, -380, 86, -380, 86, 91, + 86, 91, -201, 21, 86, 159, 86, 86, 86, 159, + 86, 86, -380, 86, -549, -345, 190, 91, -345, 356, + -354, -353, 17, -353, 91, -460, 311, 311, 243, 241, + -180, 86, 159, -180, 91, -575, 421, 101, 41, 101, + 157, 413, -496, -170, 95, -242, 32, -210, -579, 95, + 127, 653, 85, -349, -349, -349, -353, 86, 159, -349, + -349, 86, 86, 86, -262, 547, -473, 265, 101, 145, + 101, 145, 101, -351, -195, -353, -310, -568, 158, -310, + -473, -447, 316, 101, -376, 85, -376, 85, -456, 313, + 85, 86, 159, -353, -329, -259, -258, -256, 106, 117, + 41, 399, -257, 95, 154, 299, 302, 301, 277, 300, + -288, -369, 82, 405, 339, 340, -401, 611, 535, 250, + 111, 112, 387, -370, 85, 85, 83, 319, 85, 85, + -538, 86, -298, -329, 41, -301, 41, 352, 310, -299, + -353, 154, -264, 86, -544, 91, -549, -575, 91, -435, + -580, 91, -170, -242, -569, -201, -422, -509, -380, 85, + -380, 85, 68, 11, 19, -373, -380, -388, 638, 640, + 641, 249, -6, 621, 375, -279, 612, 91, 21, 91, + -518, 91, -423, -484, -139, -276, -341, 282, 86, -282, + 137, 547, 86, 86, -448, -448, -451, -450, -454, 451, + 311, 459, -388, 91, 91, 86, 86, 91, -353, 169, + 91, 653, 91, -460, -460, -353, -221, -247, -174, 547, + -262, -235, -174, 21, 547, 359, 41, 101, 41, 414, + 91, -178, 127, 107, 108, -337, -338, 91, -406, -264, + -266, 91, -306, -373, -373, -260, -180, 35, -261, -304, + -401, 547, -138, -137, -260, 547, 85, -474, 163, 101, + 145, 101, 101, -310, -310, -474, -463, 21, 86, -442, + 86, -442, 85, 127, -376, -462, -465, 61, -256, 106, + -376, 91, -266, -267, 41, 298, 294, 127, 127, -268, + 41, 278, 279, -278, 85, 309, 15, 196, 85, 112, + 112, -240, -406, -406, -539, 341, 342, 343, 347, 345, + 346, 344, -539, -406, -406, 85, -429, -428, -376, -349, + -349, 154, -579, -202, -205, -537, -353, 250, 21, 21, + -494, 547, 639, 85, 85, -353, -353, -333, 613, 101, + 91, 447, -279, -485, 614, -514, -455, -264, 127, 86, + 75, 545, 548, 86, -453, 119, 413, 417, -374, -377, + 101, 103, 188, 157, 86, 86, -353, -340, -339, 91, + 91, 91, 311, 546, -175, 60, 493, 91, 92, 408, + 91, 92, 359, -170, 91, 654, 159, 127, 86, -443, + 266, -180, 159, -304, -341, -139, -443, -263, -305, -353, + 91, -492, 171, 332, 547, 101, 145, 101, -201, -475, + 171, 332, -446, 86, 86, 86, -442, 101, 86, -469, + -466, 85, -304, 268, 137, 91, 91, 101, 85, -504, + 31, 91, -407, 85, 86, 86, 86, 86, -406, 101, + -264, -349, 86, 86, 159, 641, 85, -388, -388, 85, + 21, -333, -486, 615, 91, -523, 450, -517, -515, 445, + 446, 447, 448, 91, 546, 65, 549, -452, -453, 417, + -374, -377, 609, 457, 457, 457, 654, 159, 127, -460, + -223, -353, 309, 430, -338, 91, -409, -444, 318, 21, + -304, -349, -444, 86, 159, -349, -349, 332, 101, 145, + 101, -202, 332, -458, 317, 86, -469, -304, -468, -467, + 316, 269, 85, 86, -380, -392, -349, 86, -281, -280, + 543, -406, -409, 83, -409, 83, -409, 83, -409, 83, + 86, -264, -353, 250, -134, 85, 86, 86, -334, -353, + -518, 91, -525, 248, -521, -522, 449, -515, 21, 447, + 21, 21, -140, 159, 65, 116, 458, 458, 458, -339, + 91, 91, -222, 35, 452, 386, -410, 256, 352, 353, + 95, 547, 339, 340, 21, -445, -264, -305, -373, -373, + 101, 101, 86, 159, -353, 265, 85, -387, -381, -380, + 265, 86, -353, -287, -285, -286, 82, 464, 307, 308, + 86, -539, -539, -539, -539, -288, 86, 159, -386, 86, + 159, -532, 85, 101, -520, -519, -521, 21, -518, 21, + -518, -518, 454, 547, -452, 91, -349, -349, 91, 91, + 338, -329, 85, -457, -467, -466, -387, 86, 159, -428, + -286, 82, -285, 82, 16, 15, -409, -409, -409, -409, + 85, 86, -353, -535, 31, 86, -531, -530, -330, -526, + -353, 450, 451, 91, -518, 127, 548, -610, -609, 632, + 101, 101, -353, -442, -447, 86, -381, -284, 304, 305, + 31, 171, -284, -386, -534, -533, -331, 86, 159, 158, + 91, 549, 91, 86, -463, 106, 41, 306, 86, 159, + 127, -530, -353, -533, 41, -380, 158, -353, } var yyDef = [...]int{ @@ -9439,8 +9442,8 @@ var yyDef = [...]int{ 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, - 54, 55, 56, 57, 58, 0, 311, 312, 313, 954, - 955, 956, 957, 958, 959, 960, 961, 962, 0, 0, + 54, 55, 56, 57, 58, 0, 311, 312, 313, 956, + 957, 958, 959, 960, 961, 962, 963, 964, 0, 0, 0, 0, 0, 700, 701, 0, 664, 0, 0, 0, 0, 0, 0, 541, 542, 543, 544, 545, 546, 547, 548, 549, 550, 551, 552, 553, 554, 555, 556, 557, @@ -9449,413 +9452,413 @@ var yyDef = [...]int{ 416, 417, 418, 419, 420, 421, 422, 423, 424, 425, 0, 336, 332, 253, 254, 255, 256, 257, 258, 259, 343, 344, 518, 0, 0, 0, 0, 783, -2, 97, - 0, 0, 0, 0, 325, 0, 316, 316, 963, 964, - 965, 966, 967, 968, 969, 970, 971, 972, 973, 974, - 975, -2, 713, 0, 665, 666, 667, 668, 669, 670, + 0, 0, 0, 0, 325, 0, 316, 316, 965, 966, + 967, 968, 969, 970, 971, 972, 973, 974, 975, 976, + 977, -2, 713, 0, 665, 666, 667, 668, 669, 670, 671, 672, 673, 674, 675, 676, 677, 678, 679, 680, 681, 400, 401, 402, 396, 397, 399, 398, -2, 0, 0, 713, 0, 0, 0, 791, 0, 0, 0, 834, 856, 23, 0, 7, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 0, 0, 19, 0, 19, 0, - 0, 0, 1378, 1379, 1380, 1381, 2176, 2146, -2, 1912, - 1889, 2070, 2071, 1968, 1980, 2211, 2212, 2213, 2214, 2215, - 2216, 2217, 2218, 2219, 2220, 2221, 2222, 2223, 2224, 2225, - 2226, 2227, 2228, 2229, 2230, 2231, 2232, 2233, 2234, 2235, - 2236, 2237, 2238, 2239, 2240, 2241, 2242, 2243, 2244, 2245, - 2246, 2247, 2248, 2249, 2250, 2251, 2252, 2253, 2254, 2255, - 2256, 2257, 2258, 2259, 2260, 1846, 1847, 1848, 1849, 1850, - 1851, 1852, 1853, 1854, 1855, 1856, 1857, 1858, 1859, 1860, - 1861, 1862, 1863, 1864, 1865, 1866, 1867, 1868, 1869, 1870, - 1871, 1872, 1873, 1874, 1875, 1876, 1877, 1878, 1879, 1880, - 1881, 1882, 1883, 1884, 1885, 1886, 1887, 1888, 1890, 1891, - 1892, 1893, 1894, 1895, 1896, 1897, 1898, 1899, 1900, 1901, - 1902, 1903, 1904, 1905, 1906, 1907, 1908, 1909, 1910, 1911, - 1913, 1914, 1915, 1916, 1917, 1918, 1919, 1920, 1921, 1922, - 1923, 1924, 1925, 1926, 1927, 1928, 1929, 1930, 1931, 1932, - 1933, 1934, 1935, 1936, 1937, 1938, 1939, 1940, 1941, 1942, - 1943, 1944, 1945, 1946, 1947, 1948, 1949, 1950, 1951, 1952, - 1953, 1954, 1955, 1956, 1957, 1958, 1959, 1960, 1961, 1962, - 1963, 1964, 1965, 1966, 1967, 1969, 1970, 1971, 1972, 1973, - 1974, 1975, 1976, 1977, 1978, 1979, 1982, 1983, 1984, 1985, - 1986, 1987, 1988, 1989, 1990, 1991, 1992, 1993, 1994, 1995, - 1996, 1997, 1998, 1999, 2000, 2001, 2002, 2003, 2004, 2005, - 2006, 2007, 2008, 2009, 2010, 2011, 2012, 2013, 2014, 2015, - 2016, 2017, 2018, 2019, 2020, 2021, 2022, 2023, 2024, 2025, - 2026, 2027, 2028, 2029, 2030, 2031, 2032, 2033, 2034, 2035, - 2036, 2037, 2038, 2039, 2040, 2041, 2042, 2043, 2044, 2045, - 2046, 2047, 2048, 2049, 2050, 2051, 2052, 2053, 2054, 2055, - 2056, 2057, 2058, 2059, 2060, 2061, 2062, 2063, 2064, 2065, - 2066, 2067, 2068, 2069, 2072, 2073, 2074, 2075, 2076, 2077, - 2078, 2079, 2080, 2081, 2082, 2083, 2084, 2085, 2086, 2087, - 2088, 2089, 2090, 2091, 2092, 2093, 2094, 2095, 2096, 2097, - 2098, 2099, 2100, 2101, 2102, -2, 2104, 2105, 2106, 2107, - 2108, 2109, 2110, 2111, 2112, 2113, 2114, 2115, 2116, 2117, - 2118, 2119, 2120, 2121, 2122, 2123, 2124, 2125, 2126, 2127, - 2128, 2129, 2130, 2131, 2132, 2133, 2134, 2135, 2136, 2137, - 2138, 2139, 2140, 2141, 2142, 2143, 2144, 2145, 2147, 2148, - 2149, 2150, 2151, 2152, 2153, 2154, 2155, 2156, 2157, 2158, - 2159, 2160, 2161, -2, -2, -2, 2165, 2166, 2167, 2168, - 2169, 2170, 2171, 2172, 2173, 2174, 2175, 2177, 2178, 2179, - 2180, 2181, 2182, 2183, 2184, 2185, 2186, 2187, 2188, 2189, - 2190, 2191, 2192, 2193, 2194, 2195, 2196, 2197, 2198, 2199, - 2200, 0, 309, 307, 1859, 1889, 1912, 1968, 1980, 1981, - 2018, 2070, 2071, 2103, 2146, 2162, 2163, 2164, 2176, 0, - 0, 980, 0, 761, 0, 0, 766, 1327, 761, 337, - 702, 703, 791, 817, 662, 0, 374, 0, 1903, 378, - 2153, 0, 0, 0, 0, 659, 368, 369, 370, 371, - 372, 373, 0, 0, 953, 0, 0, 364, 0, 331, - 1970, 2175, 1382, 0, 0, 0, 0, 0, 196, 1104, - 198, 1106, 202, 210, 0, 0, 0, 215, 216, 219, + 0, 0, 1380, 1381, 1382, 1383, 2179, 2149, -2, 1915, + 1892, 2073, 2074, 1971, 1983, 2214, 2215, 2216, 2217, 2218, + 2219, 2220, 2221, 2222, 2223, 2224, 2225, 2226, 2227, 2228, + 2229, 2230, 2231, 2232, 2233, 2234, 2235, 2236, 2237, 2238, + 2239, 2240, 2241, 2242, 2243, 2244, 2245, 2246, 2247, 2248, + 2249, 2250, 2251, 2252, 2253, 2254, 2255, 2256, 2257, 2258, + 2259, 2260, 2261, 2262, 2263, 1849, 1850, 1851, 1852, 1853, + 1854, 1855, 1856, 1857, 1858, 1859, 1860, 1861, 1862, 1863, + 1864, 1865, 1866, 1867, 1868, 1869, 1870, 1871, 1872, 1873, + 1874, 1875, 1876, 1877, 1878, 1879, 1880, 1881, 1882, 1883, + 1884, 1885, 1886, 1887, 1888, 1889, 1890, 1891, 1893, 1894, + 1895, 1896, 1897, 1898, 1899, 1900, 1901, 1902, 1903, 1904, + 1905, 1906, 1907, 1908, 1909, 1910, 1911, 1912, 1913, 1914, + 1916, 1917, 1918, 1919, 1920, 1921, 1922, 1923, 1924, 1925, + 1926, 1927, 1928, 1929, 1930, 1931, 1932, 1933, 1934, 1935, + 1936, 1937, 1938, 1939, 1940, 1941, 1942, 1943, 1944, 1945, + 1946, 1947, 1948, 1949, 1950, 1951, 1952, 1953, 1954, 1955, + 1956, 1957, 1958, 1959, 1960, 1961, 1962, 1963, 1964, 1965, + 1966, 1967, 1968, 1969, 1970, 1972, 1973, 1974, 1975, 1976, + 1977, 1978, 1979, 1980, 1981, 1982, 1985, 1986, 1987, 1988, + 1989, 1990, 1991, 1992, 1993, 1994, 1995, 1996, 1997, 1998, + 1999, 2000, 2001, 2002, 2003, 2004, 2005, 2006, 2007, 2008, + 2009, 2010, 2011, 2012, 2013, 2014, 2015, 2016, 2017, 2018, + 2019, 2020, 2021, 2022, 2023, 2024, 2025, 2026, 2027, 2028, + 2029, 2030, 2031, 2032, 2033, 2034, 2035, 2036, 2037, 2038, + 2039, 2040, 2041, 2042, 2043, 2044, 2045, 2046, 2047, 2048, + 2049, 2050, 2051, 2052, 2053, 2054, 2055, 2056, 2057, 2058, + 2059, 2060, 2061, 2062, 2063, 2064, 2065, 2066, 2067, 2068, + 2069, 2070, 2071, 2072, 2075, 2076, 2077, 2078, 2079, 2080, + 2081, 2082, 2083, 2084, 2085, 2086, 2087, 2088, 2089, 2090, + 2091, 2092, 2093, 2094, 2095, 2096, 2097, 2098, 2099, 2100, + 2101, 2102, 2103, 2104, 2105, -2, 2107, 2108, 2109, 2110, + 2111, 2112, 2113, 2114, 2115, 2116, 2117, 2118, 2119, 2120, + 2121, 2122, 2123, 2124, 2125, 2126, 2127, 2128, 2129, 2130, + 2131, 2132, 2133, 2134, 2135, 2136, 2137, 2138, 2139, 2140, + 2141, 2142, 2143, 2144, 2145, 2146, 2147, 2148, 2150, 2151, + 2152, 2153, 2154, 2155, 2156, 2157, 2158, 2159, 2160, 2161, + 2162, 2163, 2164, -2, -2, -2, 2168, 2169, 2170, 2171, + 2172, 2173, 2174, 2175, 2176, 2177, 2178, 2180, 2181, 2182, + 2183, 2184, 2185, 2186, 2187, 2188, 2189, 2190, 2191, 2192, + 2193, 2194, 2195, 2196, 2197, 2198, 2199, 2200, 2201, 2202, + 2203, 0, 309, 307, 1862, 1892, 1915, 1971, 1983, 1984, + 2021, 2073, 2074, 2106, 2149, 2165, 2166, 2167, 2179, 0, + 0, 982, 0, 761, 0, 0, 766, 1329, 761, 337, + 702, 703, 791, 817, 662, 0, 374, 0, 1906, 378, + 2156, 0, 0, 0, 0, 659, 368, 369, 370, 371, + 372, 373, 0, 0, 955, 0, 0, 364, 0, 331, + 1973, 2178, 1384, 0, 0, 0, 0, 0, 196, 1106, + 198, 1108, 202, 210, 0, 0, 0, 215, 216, 219, 220, 221, 222, 223, 0, 227, 0, 229, 232, 0, 234, 235, 0, 238, 239, 240, 0, 250, 251, 252, - 1107, 1108, 1109, -2, 125, 978, 1817, 1703, 0, 1710, - 1723, 1734, 1464, 1465, 1466, 1467, 0, 0, 0, 0, - 0, 0, 1475, 1476, 0, 1506, 2215, 2256, 2257, 0, - 1484, 1485, 1486, 1487, 1488, 1489, 0, 136, 148, 149, - 1756, 1757, 1758, 1759, 1760, 1761, 1762, 0, 1764, 1765, - 1766, 1674, 1451, 1378, 0, 2224, 0, 2246, 2251, 2252, - 2253, 2254, 2245, 0, 0, 1659, 0, 1649, 0, 0, - -2, -2, 0, 0, 2043, -2, 2258, 2259, 2260, 2221, - 2242, 2250, 2225, 2226, 2249, 2217, 2218, 2219, 2212, 2213, - 2214, 2216, 2228, 2230, 2241, 0, 2237, 2247, 2248, 2151, - 0, 0, 2198, 0, 0, 0, 2193, 150, 151, -2, + 1109, 1110, 1111, -2, 125, 980, 1820, 1706, 0, 1713, + 1726, 1737, 1466, 1467, 1468, 1469, 0, 0, 0, 0, + 0, 0, 1477, 1478, 0, 1508, 2218, 2259, 2260, 0, + 1486, 1487, 1488, 1489, 1490, 1491, 0, 136, 148, 149, + 1759, 1760, 1761, 1762, 1763, 1764, 1765, 0, 1767, 1768, + 1769, 1677, 1453, 1380, 0, 2227, 0, 2249, 2254, 2255, + 2256, 2257, 2248, 0, 0, 1661, 0, 1651, 0, 0, + -2, -2, 0, 0, 2046, -2, 2261, 2262, 2263, 2224, + 2245, 2253, 2228, 2229, 2252, 2220, 2221, 2222, 2215, 2216, + 2217, 2219, 2231, 2233, 2244, 0, 2240, 2250, 2251, 2154, + 0, 0, 2201, 0, 0, 0, 2196, 150, 151, -2, + -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, + -2, -2, -2, -2, -2, -2, 1672, -2, 1674, -2, + 1676, -2, 1679, -2, -2, -2, -2, 1684, 1685, -2, + 1687, -2, -2, -2, -2, -2, -2, -2, 1663, 1664, + 1665, 1666, 1655, 1656, 1657, 1658, 1659, 1660, -2, -2, + -2, 817, 905, 0, 817, 0, 792, 839, 842, 845, + 848, 795, 0, 0, 98, 99, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 326, 327, 315, 317, 0, + 321, 0, 314, 1142, 1142, 1142, 0, 0, 0, 0, + 1142, 1142, 1142, 1142, 1142, 0, 1142, 0, 0, 0, + 0, 0, 1142, 0, 1015, 1113, 1114, 1115, 1140, 1141, + 1214, 0, 0, 0, 718, 714, 715, 716, 717, 805, + 0, 807, 810, 0, 0, 639, 639, 872, 872, 585, + 0, 0, 0, 639, 0, 599, 591, 0, 0, 0, + 639, 0, 0, 812, 812, 0, 642, 649, 639, 639, + -2, 639, 639, 636, 639, 0, 0, 1154, 605, 606, + 607, 591, 591, 610, 611, 612, 622, 623, 650, 1844, + 0, 0, 518, 518, 0, 518, 518, 0, 518, 518, + 518, 0, 720, 1931, 2016, 1913, 1989, 1872, 1973, 2178, + 0, 282, 2046, 287, 0, 1914, 1934, 0, 0, 1952, + 0, -2, 0, 353, 817, 0, 0, 791, 0, 0, + 0, 0, 518, 518, 518, 518, 518, 1213, 518, 518, + 518, 518, 518, 0, 0, 0, 518, 518, 518, 518, + 0, 0, 857, 858, 853, 854, 855, 859, 860, 5, + 6, 19, 0, 0, 0, 0, 0, 0, 104, 103, + 0, 1821, 1839, 1772, 1773, 1774, 1826, 1776, 1830, 1830, + 1830, 1830, 1805, 1806, 1807, 1808, 1809, 1810, 1811, 1812, + 1813, 1814, 1830, 1830, 0, 0, 1819, 1796, 1828, 1828, + 1828, 1826, 1823, 1777, 1778, 1779, 1780, 1781, 1782, 1783, + 1784, 1785, 1786, 1787, 1788, 1789, 1790, 1833, 1833, 1836, + 1836, 1833, 0, 414, 412, 413, 1702, 0, 0, 0, + 0, 761, 765, 1327, 0, 0, 0, 817, -2, 0, + 0, 663, 375, 1385, 0, 0, 379, 0, 380, 0, + 0, 382, 0, 0, 0, 403, 0, 406, 390, 391, + 392, 393, 386, 0, 176, 0, 366, 367, 0, 0, + 333, 0, 0, 0, 519, 0, 0, 0, 0, 0, + 0, 207, 203, 211, 214, 224, 231, 0, 243, 245, + 248, 204, 212, 217, 218, 225, 246, 205, 208, 209, + 213, 247, 249, 206, 226, 230, 244, 228, 233, 236, + 237, 242, 0, 177, 0, 0, 0, 0, 0, 1712, + 0, 0, 1745, 1746, 1747, 1748, 1749, 1750, 1751, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, -2, 1706, + 0, 0, 1472, 1473, 1474, 1475, 0, 1479, 0, 1509, + 0, 0, 0, 0, 0, 0, 1766, 1770, 0, 1702, + 1702, 0, 1702, 1698, 0, 0, 0, 0, 0, 0, + 1702, 1634, 0, 0, 1636, 1652, 0, 0, 1638, 1639, + 0, 1642, 1643, 1702, 0, 1702, 1647, 1702, 1702, 1702, + 1630, 1631, 0, 1698, 1698, 1698, 1698, 0, 0, 1698, + 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, + 1698, 1698, 1698, 1698, 0, 0, 0, 0, 812, 0, + 818, 0, -2, 0, 836, 838, 840, 841, 843, 844, + 846, 847, 849, 850, 797, 0, 0, 100, 0, 0, + 0, 0, 0, 0, 71, 73, 0, 0, 0, 0, + 0, 0, 319, 0, 324, 0, 0, 0, 0, 0, + 979, 0, 0, 1142, 1142, 1142, 1016, 0, 0, 0, + 0, 0, 0, 0, 0, 1142, 1142, 1142, 1142, 0, + 1160, 0, 0, 0, 720, 719, 0, 806, 0, 0, + 70, 578, 579, 580, 872, 0, 0, 582, 583, 0, + 584, 0, 591, 639, 639, 597, 598, 593, 592, 645, + 646, 642, 0, 642, 642, 872, 0, 616, 617, 618, + 639, 639, 624, 813, 0, 625, 626, 642, 0, 647, + 648, 872, 0, 0, 872, 872, 0, 634, 635, 637, + 639, 0, 0, 1142, 0, 655, 593, 593, 1845, 1846, + 0, 0, 1151, 0, 0, 0, 0, 658, 0, 0, + 0, 430, 431, 0, 0, 721, 0, 261, 265, 0, + 268, 0, 2016, 0, 2016, 0, 0, 275, 0, 0, + 0, 0, 0, 0, 305, 306, 0, 0, 0, 0, + 296, 299, 1321, 1322, 1103, 1104, 300, 301, 345, 346, + 0, 812, 835, 837, 831, 832, 833, 0, 72, 0, + 0, 0, 0, 0, 518, 0, 0, 0, 0, 0, + 696, 0, 995, 698, 0, 0, 0, 0, 0, 880, + 874, 876, 950, 136, 880, 8, 121, 118, 0, 19, + 0, 0, 19, 19, 0, 19, 310, 0, 1842, 1840, + 1841, 1775, 1827, 0, 1801, 0, 1802, 1803, 1804, 1815, + 1816, 0, 0, 1797, 0, 1798, 1799, 1800, 1791, 0, + 1792, 1793, 0, 1794, 1795, 308, 411, 0, 0, 1703, + 983, 0, 739, 753, 734, 0, 742, 0, 0, 1329, + 0, 0, 0, 722, 753, 724, 0, 742, 812, 789, + 0, 767, 0, 0, 376, 0, 387, 381, 0, 388, + 383, 384, 0, 0, 405, 407, 408, 409, 394, 395, + 660, 362, 363, 354, 355, 356, 357, 358, 359, 360, + 361, 0, 0, 0, 365, 146, 0, 334, 335, 0, + 0, 0, 190, 191, 192, 193, 194, 195, 197, 181, + 685, 687, 1095, 1107, 0, 1098, 0, 200, 241, 173, + 0, 0, 0, 1707, 1708, 1709, 1710, 1711, 1716, 0, + 1718, 1720, 1722, 1724, 0, 1742, -2, -2, 1454, 1455, + 1456, 1457, 1458, 1459, 1460, 1461, 1462, 1463, 1464, 1465, + 1727, 1740, 1741, 0, 0, 0, 0, 0, 0, 1738, + 1738, 1733, 0, 1492, 1323, 1324, 1470, 0, 0, 1506, + 1510, 0, 0, 0, 0, 0, 0, 1126, 1826, 0, + 137, 1697, 1601, 1602, 1603, 1604, 1605, 1606, 1607, 1608, + 1609, 1610, 1611, 1612, 1613, 1614, 1615, 1616, 1617, 1618, + 1619, 1620, 1621, 1622, 1623, 1624, 1625, 1626, 1627, 1628, + 1629, 0, 0, 1706, 0, 0, 0, 1699, 1700, 0, + 0, 0, 1589, 0, 0, 1595, 1596, 1597, 0, 748, + 0, 1662, 1635, 1653, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 904, 906, 0, 757, 759, 760, 786, 767, + 793, 0, 0, 0, 96, 101, 0, 1181, 0, 0, + 0, 0, 0, 0, 0, 74, 1155, 75, 1157, 328, + 329, 0, 0, 323, 0, 0, 0, 0, 0, -2, + 0, 0, 0, 0, 0, 1031, 1032, 516, 1089, 0, + 0, 0, 1105, 1130, 1138, 0, 0, 0, 0, 0, + 1187, 1017, 1022, 1023, 1024, 1018, 1019, 1025, 1026, 0, + 808, 0, 0, 919, 581, 640, 641, 873, 588, 1973, + 593, 872, 872, 600, 594, 601, 644, 602, 603, 604, + 642, 872, 872, 814, 639, 642, 627, 643, 642, 1329, + 631, 0, 638, 1329, 656, 1329, 0, 654, 608, 609, + 1189, 810, 428, 429, 434, 436, 0, 485, 485, 485, + 468, 485, 0, 0, 456, 1847, 0, 0, 0, 0, + 465, 1847, 0, 0, 1847, 1847, 1847, 1847, 1847, 1847, + 1847, 0, 0, 1847, 1847, 1847, 1847, 1847, 1847, 1847, + 1847, 1847, 1847, 1847, 0, 1847, 1847, 1847, 1847, 1847, + 1306, 1847, 0, 484, 1152, 475, 476, 477, 478, 483, + 0, 0, 512, 0, 0, 1030, 0, 516, 0, 0, + 1072, 0, 0, 885, 0, 886, 887, 888, 883, 921, + 945, 945, 0, 945, 925, 1329, 0, 0, 0, 273, + 274, 262, 0, 263, 0, 0, 276, 277, 0, 279, + 280, 281, 288, 1913, 1989, 283, 285, 0, 0, 289, + 302, 303, 304, 0, 0, 294, 295, 0, 0, 348, + 349, 351, 0, 767, 1156, 682, 1325, 683, 684, 688, + 0, 0, 691, 692, 693, 694, 695, 997, 0, 0, + 1081, 1082, 1083, 1084, 872, 0, 881, 0, 877, 951, + 0, 953, 0, 872, 0, 119, 19, 0, 112, 109, + 0, 0, 0, 0, 0, 1822, 1771, 1843, 0, 0, + 0, 1824, 0, 0, 0, 0, 0, 102, 769, 729, + 0, 733, 750, 0, 754, 0, 0, 746, 738, 743, + 0, 0, 763, 730, 1328, 0, 0, 0, 723, 0, + 0, 728, 767, 0, 790, 819, 820, 823, 1386, 0, + 389, 385, 404, 0, 518, 0, 0, 0, 184, 1092, + 0, 185, 189, 179, 0, 0, 0, 1097, 0, 1094, + 1099, 0, 199, 0, 0, 174, 175, 1172, 1181, 0, + 0, 0, 1717, 1719, 1721, 1723, 1725, 0, 1728, 1738, + 1738, 1734, 0, 1729, 0, 1731, 0, 1707, 1476, 0, + 1511, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 823, 0, 0, 1579, 1580, 0, 0, 1584, 0, 1586, + 1587, 1588, 1590, 0, 0, 0, 1594, 0, 1633, 1654, + 1637, 1640, 0, 1644, 0, 1646, 1648, 1649, 1650, 0, + 817, 817, 0, 0, 1550, 1550, 1550, 0, 0, 0, + 0, 1550, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1495, 0, 1496, 1497, 0, 0, 0, + 907, 787, 0, 0, 0, 0, 0, 1179, 0, 86, + 0, 0, 0, 0, 91, 0, 0, 76, 330, 318, + 320, 0, 0, 1143, 0, 0, 0, 0, 0, 985, + 986, 988, 0, 991, 992, 993, 997, 810, 0, 810, + 1042, 1847, 520, 0, 0, 1091, 0, 1061, 0, 0, + 0, -2, 0, 0, 1138, 0, 0, 0, 1191, 0, + 0, 0, 707, 711, 23, 811, 0, 586, 0, 587, + 639, 595, 596, 872, 619, 620, 0, 0, 872, 639, + 639, 630, 642, 651, 0, 652, 1329, 1191, 0, 0, + 1151, 1257, 1225, 446, 0, 1341, 1342, 486, 0, 1348, + 1357, 1142, 1418, 0, 1357, 0, 0, 1359, 1360, 0, + 0, 0, 0, 469, 470, 0, 455, 0, 0, 0, + 0, 0, 0, 454, 0, 0, 496, 0, 0, 0, + 0, 0, 1848, 1847, 1847, 0, 463, 464, 0, 467, + 0, 0, 0, 0, 0, 0, 0, 0, 1847, 1847, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 1297, 0, 0, 0, 0, 0, 0, 0, 0, + 1313, 1314, 0, 1042, 1847, 0, 0, 0, 0, 520, + 1086, 1059, 1077, 0, 432, 433, 493, 0, 0, 0, + 0, 0, 0, 0, 911, 0, 0, 0, 910, 0, + 0, 0, 0, 0, 0, 0, 810, 946, 0, 948, + 949, 923, -2, 0, 885, 928, 1702, 0, 266, 267, + 0, 0, 272, 290, 292, 264, 0, 0, 0, 291, + 293, 297, 298, 347, 350, 352, 829, 0, 0, 1215, + 0, 998, 999, 1001, 1002, 0, -2, -2, -2, -2, + -2, -2, -2, -2, -2, -2, -2, -2, -2, 1897, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, - -2, -2, -2, -2, -2, -2, 1670, -2, -2, 1673, - -2, 1676, -2, -2, -2, -2, 1681, 1682, -2, 1684, - -2, -2, -2, -2, -2, -2, -2, 1661, 1662, 1663, - 1664, 1653, 1654, 1655, 1656, 1657, 1658, -2, -2, -2, - 817, 903, 0, 817, 0, 792, 839, 842, 845, 848, - 795, 0, 0, 98, 99, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 326, 327, 315, 317, 0, 321, - 0, 314, 1140, 1140, 1140, 0, 0, 0, 0, 1140, - 1140, 1140, 1140, 1140, 0, 1140, 0, 0, 0, 0, - 0, 1140, 0, 1013, 1111, 1112, 1113, 1138, 1139, 1212, - 0, 0, 0, 718, 714, 715, 716, 717, 805, 0, - 807, 810, 0, 0, 639, 639, 872, 872, 585, 0, - 0, 0, 639, 0, 599, 591, 0, 0, 0, 639, - 0, 0, 812, 812, 0, 642, 649, 639, 639, -2, - 639, 639, 636, 639, 0, 0, 1152, 605, 606, 607, - 591, 591, 610, 611, 612, 622, 623, 650, 1841, 0, - 0, 518, 518, 0, 518, 518, 0, 518, 518, 518, - 0, 720, 1928, 2013, 1910, 1986, 1869, 1970, 2175, 0, - 282, 2043, 287, 0, 1911, 1931, 0, 0, 1949, 0, - -2, 0, 353, 817, 0, 0, 791, 0, 0, 0, - 0, 518, 518, 518, 518, 518, 1211, 518, 518, 518, - 518, 518, 0, 0, 0, 518, 518, 518, 518, 0, - 0, 857, 858, 853, 854, 855, 859, 860, 5, 6, - 19, 0, 0, 0, 0, 0, 0, 104, 103, 0, - 1818, 1836, 1769, 1770, 1771, 1823, 1773, 1827, 1827, 1827, - 1827, 1802, 1803, 1804, 1805, 1806, 1807, 1808, 1809, 1810, - 1811, 1827, 1827, 0, 0, 1816, 1793, 1825, 1825, 1825, - 1823, 1820, 1774, 1775, 1776, 1777, 1778, 1779, 1780, 1781, - 1782, 1783, 1784, 1785, 1786, 1787, 1830, 1830, 1833, 1833, - 1830, 0, 414, 412, 413, 1699, 0, 0, 0, 0, - 761, 765, 1325, 0, 0, 0, 817, -2, 0, 0, - 663, 375, 1383, 0, 0, 379, 0, 380, 0, 0, - 382, 0, 0, 0, 403, 0, 406, 390, 391, 392, - 393, 386, 0, 176, 0, 366, 367, 0, 0, 333, - 0, 0, 0, 519, 0, 0, 0, 0, 0, 0, - 207, 203, 211, 214, 224, 231, 0, 243, 245, 248, - 204, 212, 217, 218, 225, 246, 205, 208, 209, 213, - 247, 249, 206, 226, 230, 244, 228, 233, 236, 237, - 242, 0, 177, 0, 0, 0, 0, 0, 1709, 0, - 0, 1742, 1743, 1744, 1745, 1746, 1747, 1748, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, -2, 1703, 0, - 0, 1470, 1471, 1472, 1473, 0, 1477, 0, 1507, 0, - 0, 0, 0, 0, 0, 1763, 1767, 0, 1699, 1699, - 0, 1699, 1695, 0, 0, 0, 0, 0, 0, 1699, - 1632, 0, 0, 1634, 1650, 0, 0, 1636, 1637, 0, - 1640, 1641, 1699, 0, 1699, 1645, 1699, 1699, 1699, 1628, - 1629, 0, 1695, 1695, 1695, 1695, 0, 0, 1695, 1695, - 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, - 1695, 1695, 1695, 0, 0, 0, 0, 812, 0, 818, - 0, -2, 0, 836, 838, 840, 841, 843, 844, 846, - 847, 849, 850, 797, 0, 0, 100, 0, 0, 0, - 0, 0, 0, 71, 73, 0, 0, 0, 0, 0, - 0, 319, 0, 324, 0, 0, 0, 0, 0, 977, - 0, 0, 1140, 1140, 1140, 1014, 0, 0, 0, 0, - 0, 0, 0, 0, 1140, 1140, 1140, 1140, 0, 1158, - 0, 0, 0, 720, 719, 0, 806, 0, 0, 70, - 578, 579, 580, 872, 0, 0, 582, 583, 0, 584, - 0, 591, 639, 639, 597, 598, 593, 592, 645, 646, - 642, 0, 642, 642, 872, 0, 616, 617, 618, 639, - 639, 624, 813, 0, 625, 626, 642, 0, 647, 648, - 872, 0, 0, 872, 872, 0, 634, 635, 637, 639, - 0, 0, 1140, 0, 655, 593, 593, 1842, 1843, 0, - 0, 1149, 0, 0, 0, 0, 658, 0, 0, 0, - 430, 431, 0, 0, 721, 0, 261, 265, 0, 268, - 0, 2013, 0, 2013, 0, 0, 275, 0, 0, 0, - 0, 0, 0, 305, 306, 0, 0, 0, 0, 296, - 299, 1319, 1320, 1101, 1102, 300, 301, 345, 346, 0, - 812, 835, 837, 831, 832, 833, 0, 72, 0, 0, - 0, 0, 0, 518, 0, 0, 0, 0, 0, 696, - 0, 993, 698, 0, 0, 0, 0, 0, 880, 874, - 876, 948, 136, 880, 8, 121, 118, 0, 19, 0, - 0, 19, 19, 0, 19, 310, 0, 1839, 1837, 1838, - 1772, 1824, 0, 1798, 0, 1799, 1800, 1801, 1812, 1813, - 0, 0, 1794, 0, 1795, 1796, 1797, 1788, 0, 1789, - 1790, 0, 1791, 1792, 308, 411, 0, 0, 1700, 981, - 0, 739, 753, 734, 0, 742, 0, 0, 1327, 0, - 0, 0, 722, 753, 724, 0, 742, 812, 789, 0, - 767, 0, 0, 376, 0, 387, 381, 0, 388, 383, - 384, 0, 0, 405, 407, 408, 409, 394, 395, 660, - 362, 363, 354, 355, 356, 357, 358, 359, 360, 361, - 0, 0, 0, 365, 146, 0, 334, 335, 0, 0, - 0, 190, 191, 192, 193, 194, 195, 197, 181, 685, - 687, 1093, 1105, 0, 1096, 0, 200, 241, 173, 0, - 0, 0, 1704, 1705, 1706, 1707, 1708, 1713, 0, 1715, - 1717, 1719, 1721, 0, 1739, -2, -2, 1452, 1453, 1454, - 1455, 1456, 1457, 1458, 1459, 1460, 1461, 1462, 1463, 1724, - 1737, 1738, 0, 0, 0, 0, 0, 0, 1735, 1735, - 1730, 0, 1490, 1321, 1322, 1468, 0, 0, 1504, 1508, - 0, 0, 0, 0, 0, 0, 1124, 1823, 0, 137, - 1694, 1599, 1600, 1601, 1602, 1603, 1604, 1605, 1606, 1607, - 1608, 1609, 1610, 1611, 1612, 1613, 1614, 1615, 1616, 1617, - 1618, 1619, 1620, 1621, 1622, 1623, 1624, 1625, 1626, 1627, - 0, 0, 1703, 0, 0, 0, 1696, 1697, 0, 0, - 0, 1587, 0, 0, 1593, 1594, 1595, 0, 748, 0, - 1660, 1633, 1651, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 902, 904, 0, 757, 759, 760, 786, 767, 793, - 0, 0, 0, 96, 101, 0, 1179, 0, 0, 0, - 0, 0, 0, 0, 74, 1153, 75, 1155, 328, 329, - 0, 0, 323, 0, 0, 0, 0, 0, -2, 0, - 0, 0, 0, 0, 1029, 1030, 516, 1087, 0, 0, - 0, 1103, 1128, 1136, 0, 0, 0, 0, 0, 1185, - 1015, 1020, 1021, 1022, 1016, 1017, 1023, 1024, 0, 808, - 0, 0, 917, 581, 640, 641, 873, 588, 1970, 593, - 872, 872, 600, 594, 601, 644, 602, 603, 604, 642, - 872, 872, 814, 639, 642, 627, 643, 642, 1327, 631, - 0, 638, 1327, 656, 1327, 0, 654, 608, 609, 1187, - 810, 428, 429, 434, 436, 0, 485, 485, 485, 468, - 485, 0, 0, 456, 1844, 0, 0, 0, 0, 465, - 1844, 0, 0, 1844, 1844, 1844, 1844, 1844, 1844, 1844, - 0, 0, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, - 1844, 1844, 1844, 0, 1844, 1844, 1844, 1844, 1844, 1304, - 1844, 0, 484, 1150, 475, 476, 477, 478, 483, 0, - 0, 512, 0, 0, 1028, 0, 516, 0, 0, 1070, - 0, 0, 885, 0, 886, 887, 888, 883, 919, 943, - 943, 0, 943, 923, 1327, 0, 0, 0, 273, 274, - 262, 0, 263, 0, 0, 276, 277, 0, 279, 280, - 281, 288, 1910, 1986, 283, 285, 0, 0, 289, 302, - 303, 304, 0, 0, 294, 295, 0, 0, 348, 349, - 351, 0, 767, 1154, 682, 1323, 683, 684, 688, 0, - 0, 691, 692, 693, 694, 695, 995, 0, 0, 1079, - 1080, 1081, 1082, 872, 0, 881, 0, 877, 949, 0, - 951, 0, 872, 0, 119, 19, 0, 112, 109, 0, - 0, 0, 0, 0, 1819, 1768, 1840, 0, 0, 0, - 1821, 0, 0, 0, 0, 0, 102, 769, 729, 0, - 733, 750, 0, 754, 0, 0, 746, 738, 743, 0, - 0, 763, 730, 1326, 0, 0, 0, 723, 0, 0, - 728, 767, 0, 790, 819, 820, 823, 1384, 0, 389, - 385, 404, 0, 518, 0, 0, 0, 184, 1090, 0, - 185, 189, 179, 0, 0, 0, 1095, 0, 1092, 1097, - 0, 199, 0, 0, 174, 175, 1170, 1179, 0, 0, - 0, 1714, 1716, 1718, 1720, 1722, 0, 1725, 1735, 1735, - 1731, 0, 1726, 0, 1728, 0, 1704, 1474, 0, 1509, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 823, - 0, 0, 1577, 1578, 0, 0, 1582, 0, 1584, 1585, - 1586, 1588, 0, 0, 0, 1592, 0, 1631, 1652, 1635, - 1638, 0, 1642, 0, 1644, 1646, 1647, 1648, 0, 817, - 817, 0, 0, 1548, 1548, 1548, 0, 0, 0, 0, - 1548, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 1493, 0, 1494, 1495, 0, 0, 0, 905, - 787, 0, 0, 0, 0, 0, 1177, 0, 86, 0, - 0, 0, 0, 91, 0, 0, 76, 330, 318, 320, - 0, 0, 1141, 0, 0, 0, 0, 0, 983, 984, - 986, 0, 989, 990, 991, 995, 810, 0, 810, 1040, - 1844, 520, 0, 0, 1089, 0, 1059, 0, 0, 0, - -2, 0, 0, 1136, 0, 0, 0, 1189, 0, 0, - 0, 707, 711, 23, 811, 0, 586, 0, 587, 639, - 595, 596, 872, 619, 620, 0, 0, 872, 639, 639, - 630, 642, 651, 0, 652, 1327, 1189, 0, 0, 1149, - 1255, 1223, 446, 0, 1339, 1340, 486, 0, 1346, 1355, - 1140, 1416, 0, 1355, 0, 0, 1357, 1358, 0, 0, - 0, 0, 469, 470, 0, 455, 0, 0, 0, 0, - 0, 0, 454, 0, 0, 496, 0, 0, 0, 0, - 0, 1845, 1844, 1844, 0, 463, 464, 0, 467, 0, - 0, 0, 0, 0, 0, 0, 0, 1844, 1844, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1295, 0, 0, 0, 0, 0, 0, 0, 0, 1311, - 1312, 0, 1040, 1844, 0, 0, 0, 0, 520, 1084, - 1057, 1075, 0, 432, 433, 493, 0, 0, 0, 0, - 0, 0, 909, 0, 0, 0, 908, 0, 0, 0, - 0, 0, 0, 810, 944, 0, 946, 947, 921, -2, - 0, 885, 926, 1699, 0, 266, 267, 0, 0, 272, - 290, 292, 264, 0, 0, 0, 291, 293, 297, 298, - 347, 350, 352, 829, 0, 0, 1213, 0, 996, 997, - 999, 1000, 0, -2, -2, -2, -2, -2, -2, -2, - -2, -2, -2, -2, -2, -2, 1894, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, - -2, -2, -2, -2, -2, -2, -2, -2, 994, 699, - 863, 875, 882, 950, 952, 137, 878, 863, 0, 122, - 19, 121, 113, 114, 0, 19, 0, 0, 0, 0, - 1829, 1828, 1814, 0, 1815, 1826, 1831, 0, 1834, 0, - 415, 773, 0, 0, 753, 755, 0, 0, 753, 0, - 0, 762, 0, 0, 0, 0, 0, 0, 753, 829, - 769, 0, 826, 824, 825, 0, 0, 661, 147, 410, - 0, 0, 0, 0, 0, 686, 0, 1094, 181, 0, - 0, 201, 0, 0, 0, 1179, 1174, 1698, 1727, 1729, - 0, 1736, 1732, 1469, 1478, 1505, 0, 0, 1511, 1523, - 1523, 0, 0, 0, 1514, 1827, 1827, 1517, 1823, 1825, - 1823, 1523, 1523, 0, 1125, 0, 1126, 823, 138, 0, - 0, 1583, 0, 0, 0, 749, 0, 0, 0, 1544, - 1546, 1548, 1548, 1555, 1549, 1556, 1557, 1548, 1548, 1548, - 1548, 1562, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, - 1548, 1548, 1548, 1542, 0, 0, 1757, 1758, 758, 0, - 0, 800, 801, 802, 803, 804, 0, 0, 61, 61, - 1179, 0, 95, 87, 0, 0, 0, 0, 0, 322, - 0, 77, 78, 0, 0, 85, 0, 0, 0, 0, - 0, 988, 0, 0, 0, 1332, 0, 1044, 1041, 1042, - 1043, 0, 1084, 521, 522, 523, 524, 0, 0, 0, - 1088, 0, 0, 1052, 0, 0, 0, 1129, 1130, 1131, - 1132, 1133, 1134, 1135, -2, 1144, 0, 0, 0, 1332, - 1163, 0, 0, 1168, 1332, 1332, 0, 1197, 0, 1186, - 761, 0, -2, 0, 0, 709, 0, 0, 918, 589, - 872, 613, 815, 816, 1327, 872, 872, 639, 657, 653, - 1197, 1188, 0, 435, 485, 0, 1243, 0, 0, 1249, - 0, 1256, 439, 0, 487, 0, 1345, 1372, 1356, 1372, - 1417, 1372, 1372, 1140, 0, 487, 0, 0, 457, 0, - 0, 0, 0, 0, 453, 490, 823, 440, 442, 443, - 444, 494, 495, 497, 0, 499, 500, 459, 471, 472, - 473, 474, 0, 0, 0, 466, 479, 480, 481, 482, - 441, 1272, 1273, 1274, 1277, 1278, 1279, 1280, 0, 0, - 1283, 1284, 1285, 1286, 1287, 1369, 1370, 1371, 1288, 1289, - 1290, 1291, 1292, 1293, 1294, 1313, 1314, 1315, 1316, 1317, - 1318, 1296, 1297, 1298, 1299, 1300, 1301, 1302, 1303, 0, - 0, 1308, 0, 0, 0, 1044, 0, 0, 0, 0, - 0, 1084, 515, 0, 0, 1059, 0, 1077, 0, 1071, - 1072, 0, 0, 731, 872, 340, 0, 913, 906, 0, - 892, 893, 910, 894, 911, 912, 898, 0, 900, 0, - 896, 897, 895, 872, 884, 920, 945, 922, 925, 927, - 928, 934, 0, 0, 0, 0, 260, 269, 270, 271, - 278, 0, 540, 284, 785, 0, 1324, 689, 690, 1214, - 1215, 697, 0, 1001, 861, 0, 0, 861, 117, 120, - 0, 115, 0, 0, 0, 0, 107, 105, 1822, 0, - 0, 775, 161, 0, 0, 0, 751, 0, 756, 753, - 737, 747, 736, 744, 745, 764, 1328, 1329, 1330, 1331, - 753, 727, 726, 788, 773, 821, 822, 0, 1385, 377, - 0, 1091, 181, 186, 187, 188, 182, 180, 1098, 0, - 1100, 0, 1172, 0, 0, 1733, 1510, 1479, 1512, 1524, - 1525, 1513, 0, 1481, 1482, 1515, 1516, 1518, 1519, 1520, - 1521, 1522, 1483, 0, 1127, 1579, 0, 1581, 1589, 1590, - 0, 1639, 1643, 0, 0, 0, 0, 0, 1553, 1554, - 1558, 1559, 1560, 1561, 1563, 1564, 1565, 1566, 1567, 1568, - 1569, 1570, 1571, 1572, 1573, 817, 1543, 0, 0, 0, - 0, 0, 0, 0, 798, 0, 0, 0, 63, 0, - 63, 1178, 1180, 0, 948, 0, 0, 92, 0, 0, - 79, 80, 0, 0, 0, 976, 979, 985, 987, 0, - 0, 0, 1333, 1334, 1336, 1337, 1338, 0, 1012, 0, - 0, 1032, 1033, 1034, 1046, 0, 526, 527, 0, 0, - 0, 539, 535, 536, 537, 517, 1083, 1066, 0, 0, - 1055, 0, 0, 1065, 0, 1145, 1844, 1844, 1844, 0, - 0, 1257, 1844, 1844, 0, 1165, 1167, 0, 0, 1261, - 1200, 0, 0, 1191, 0, 943, 0, 0, 872, 708, - 711, 712, 809, 590, 628, 632, 629, 872, 1200, 427, - 1221, 0, 0, 0, 0, 0, 1253, 0, 0, 1225, - 0, 458, 488, 0, -2, 0, 1373, 0, 1359, 1373, - 0, 0, 1372, 0, 447, 487, 0, 0, 0, 501, - 505, 506, 0, 503, 1412, 0, 504, 0, 492, 0, - 498, 1275, 1276, 0, 1281, 1282, 0, 1306, 0, 0, - 0, 438, 507, 0, 0, 0, 508, 509, 514, 1085, - 1086, 1052, 0, 1066, 0, 1076, 0, 1073, 1074, 817, - 0, 0, 889, 914, 0, 0, 890, 0, 891, 899, - 901, 339, 929, 0, 0, 931, 932, 933, 924, 286, - 830, 998, 0, 851, 0, 0, 879, 852, 0, 19, - 0, 0, 110, 1832, 1835, 777, 0, 774, 162, 0, - 0, 0, 0, 741, 752, 735, 725, 775, 827, 828, - 183, 178, 1099, 1182, 0, 1173, 0, 1436, 1492, 0, - 1591, 0, 1548, 1545, 1548, 1547, 1539, 0, 1496, 0, - 1498, 1499, 1500, 0, 1502, 1503, 0, 796, 0, 59, - 0, 62, 60, 0, 94, 88, 0, 0, 0, 0, - 0, 0, 0, 0, 1018, 1261, 0, 1018, 1045, 1031, - 0, 0, 528, 529, 0, 532, 538, 1047, 0, 0, - 1049, 1050, 1051, 0, 0, 1063, 0, 0, 0, 0, - 1137, 1151, 0, 0, 0, -2, 0, -2, 1162, 0, - 1206, 0, 1198, 0, 1190, 0, 1193, 872, 872, -2, - 705, 710, 0, 633, 1206, 1223, 0, 1244, 0, 0, - 0, 0, 0, 0, 0, 1224, 0, 1237, 489, 1374, - -2, 1388, 1390, 0, 1150, 1393, 1394, 0, 0, 0, - 0, 0, 0, 1443, 1402, 0, 0, 1406, 1407, 1408, - 0, 0, 1411, 0, 1751, 1752, 0, 1415, 0, 0, - 0, 0, 0, 0, 0, 1353, 448, 449, 0, 451, - 452, 1844, 1413, 491, 445, 1844, 461, 1305, 1309, 1310, - 1307, 513, 510, 511, 1055, 1058, 1069, 1078, 732, 812, - 341, 342, 915, 0, 907, 938, 935, 0, 0, 1002, - 862, 870, 2198, 2200, 2197, 111, 116, 0, 0, 779, - 0, 776, 0, 770, 772, 172, 740, 777, 132, 164, - 0, 0, 1480, 0, 0, 0, 1580, 1630, 1551, 1552, - 0, 1540, 0, 1534, 1535, 1536, 1541, 0, 0, 799, - 794, 64, 90, 0, 93, 68, 81, 0, 0, 0, - 0, 1004, 1011, 1025, 1156, 1335, 1010, 0, 0, 525, - 530, 0, 533, 534, 1067, 1066, 0, 1053, 1054, 0, - 1061, 0, 0, 1146, 1147, 1148, 1258, 1259, 1260, 1216, - 1164, 0, -2, 1269, 0, 0, 1160, 1182, 1216, 0, - 0, 1194, 0, 1201, 0, 1199, 1192, 817, 706, 1203, - 437, 1255, 1245, 0, 1247, 0, 0, 0, 0, 1226, - -2, 0, 1389, 1391, 1392, 1395, 1396, 1397, 1448, 1449, - 1450, 0, 0, 1400, 1445, 1446, 1447, 1401, 0, 0, - 0, 0, 0, 1749, 1750, 1441, 0, 0, 1360, 1362, - 1363, 1364, 1365, 1366, 1367, 1368, 1361, 0, 0, 0, - 1352, 1354, 450, 0, 0, 1844, 1068, 338, 0, 0, - 939, 941, 936, 937, 864, 0, 0, 0, 0, 106, - 108, 123, 0, 778, 163, 0, 779, 134, 0, 155, - 0, 1183, 0, 1491, 0, 0, 0, 1550, 1537, 0, - 0, 0, 0, 0, 1753, 1754, 1755, 0, 1497, 1501, - 89, 0, 66, 0, 82, 83, 0, 0, 0, 1026, - 1027, 1035, 1036, 0, 1038, 1039, 531, 1048, 1056, 1060, - 1063, 0, 1115, 1218, 0, 1166, 1149, 1271, 1844, 1169, - 1218, 0, 1263, 1844, 1844, 1184, 0, 1196, 0, 1208, - 0, 1202, 812, 426, 0, 1205, 1241, 1246, 1248, 1250, - 0, 1254, 1252, 1227, -2, 0, 1235, 0, 0, 1398, - 1399, 0, 0, 1649, 1844, 0, 1431, 0, 1115, 1115, - 1115, 1115, 0, 502, 460, 0, 916, 930, 0, 871, - 0, 0, 0, 0, 0, 768, 124, 0, 133, 152, - 0, 165, 166, 0, 0, 0, 0, 1175, 0, 1439, - 1440, 0, 1526, 0, 0, 0, 1530, 1531, 1532, 1533, - 65, 68, 0, 0, 0, 1003, 0, 1037, 1062, 1064, - 1114, 1159, 0, 1255, 1270, 0, 1161, 1262, 0, 0, - 0, 1195, 1207, 0, 1210, 704, 1204, 1222, 0, 1251, - 1228, 1236, 0, 1231, 0, 0, 0, 1444, 0, 1405, - 0, 1410, 1419, 1432, 0, 0, 1341, 0, 1343, 0, - 1347, 0, 1349, 0, 0, 462, 940, 942, 0, 1699, - 866, 867, 0, 781, 771, 135, 139, 0, 161, 158, - 0, 167, 0, 0, 0, 0, 1171, 0, 1437, 0, - 1527, 1528, 1529, 67, 69, 84, 0, 1005, 1006, 1019, - 1116, 1844, 1844, 0, 0, 0, 1122, 1123, 0, 1243, - 1275, 1264, 1265, 1266, 1209, 1242, 1230, 0, -2, 1238, - 0, 0, 1701, 1711, 1712, 1403, 1409, 1418, 1420, 1421, - 0, 1433, 1434, 1435, 1442, 1115, 1115, 1115, 1115, 1351, - 865, 0, 0, 780, 0, 126, 0, 0, 156, 157, - 159, 0, 168, 0, 170, 171, 0, 0, 1538, 1007, - 0, 0, 1119, 1120, 0, 1219, 0, 1221, 1232, -2, - 0, 1240, 0, 1404, 1422, 0, 1423, 0, 0, 0, - 1342, 1344, 1348, 1350, 1699, 868, 782, 1181, 0, 140, - 0, 142, 144, 145, 1375, 153, 154, 160, 169, 0, - 0, 992, 1008, 0, 1117, 1118, 1121, 0, 1223, 1239, - 1702, 1424, 1426, 1427, 0, 0, 1425, 0, 127, 128, - 0, 141, 0, 0, 1176, 1438, 1009, 1220, 1217, 1428, - 1430, 1429, 869, 0, 0, 143, 1376, 129, 130, 131, - 0, 1377, + -2, 996, 699, 863, 875, 882, 952, 954, 137, 878, + 863, 0, 122, 19, 121, 113, 114, 0, 19, 0, + 0, 0, 0, 1832, 1831, 1817, 0, 1818, 1829, 1834, + 0, 1837, 0, 415, 773, 0, 0, 753, 755, 0, + 0, 753, 0, 0, 762, 0, 0, 0, 0, 0, + 0, 753, 829, 769, 0, 826, 824, 825, 0, 0, + 661, 147, 410, 0, 0, 0, 0, 0, 686, 0, + 1096, 181, 0, 0, 201, 0, 0, 0, 1181, 1176, + 1701, 1730, 1732, 0, 1739, 1735, 1471, 1480, 1507, 0, + 0, 1513, 1525, 1525, 0, 0, 0, 1516, 1830, 1830, + 1519, 1826, 1828, 1826, 1525, 1525, 0, 1127, 0, 1128, + 823, 138, 0, 0, 1585, 0, 0, 0, 749, 0, + 0, 0, 1546, 1548, 1550, 1550, 1557, 1551, 1558, 1559, + 1550, 1550, 1550, 1550, 1564, 1550, 1550, 1550, 1550, 1550, + 1550, 1550, 1550, 1550, 1550, 1550, 1544, 0, 0, 1760, + 1761, 758, 0, 0, 800, 801, 802, 803, 804, 0, + 0, 61, 61, 1181, 0, 95, 87, 0, 0, 0, + 0, 0, 322, 0, 77, 78, 0, 0, 85, 0, + 0, 0, 0, 0, 990, 0, 0, 0, 1334, 0, + 1046, 1043, 1044, 1045, 0, 1086, 521, 522, 523, 524, + 0, 0, 0, 1090, 0, 0, 1054, 0, 0, 0, + 1131, 1132, 1133, 1134, 1135, 1136, 1137, -2, 1146, 0, + 0, 0, 1334, 1165, 0, 0, 1170, 1334, 1334, 0, + 1199, 0, 1188, 761, 0, -2, 0, 0, 709, 0, + 0, 920, 589, 872, 613, 815, 816, 1329, 872, 872, + 639, 657, 653, 1199, 1190, 0, 435, 485, 0, 1245, + 0, 0, 1251, 0, 1258, 439, 0, 487, 0, 1347, + 1374, 1358, 1374, 1419, 1374, 1374, 1142, 0, 487, 0, + 0, 457, 0, 0, 0, 0, 0, 453, 490, 823, + 440, 442, 443, 444, 494, 495, 497, 0, 499, 500, + 459, 471, 472, 473, 474, 0, 0, 0, 466, 479, + 480, 481, 482, 441, 1274, 1275, 1276, 1279, 1280, 1281, + 1282, 0, 0, 1285, 1286, 1287, 1288, 1289, 1371, 1372, + 1373, 1290, 1291, 1292, 1293, 1294, 1295, 1296, 1315, 1316, + 1317, 1318, 1319, 1320, 1298, 1299, 1300, 1301, 1302, 1303, + 1304, 1305, 0, 0, 1310, 0, 0, 0, 1046, 0, + 0, 0, 0, 0, 1086, 515, 0, 0, 1061, 0, + 1079, 0, 1073, 1074, 0, 0, 731, 872, 340, 0, + 915, 908, 0, 892, 0, 894, 912, 895, 913, 914, + 899, 0, 901, 0, 897, 898, 903, 896, 872, 884, + 922, 947, 924, 927, 929, 930, 936, 0, 0, 0, + 0, 260, 269, 270, 271, 278, 0, 540, 284, 785, + 0, 1326, 689, 690, 1216, 1217, 697, 0, 1003, 861, + 0, 0, 861, 117, 120, 0, 115, 0, 0, 0, + 0, 107, 105, 1825, 0, 0, 775, 161, 0, 0, + 0, 751, 0, 756, 753, 737, 747, 736, 744, 745, + 764, 1330, 1331, 1332, 1333, 753, 727, 726, 788, 773, + 821, 822, 0, 1387, 377, 0, 1093, 181, 186, 187, + 188, 182, 180, 1100, 0, 1102, 0, 1174, 0, 0, + 1736, 1512, 1481, 1514, 1526, 1527, 1515, 0, 1483, 1484, + 1517, 1518, 1520, 1521, 1522, 1523, 1524, 1485, 0, 1129, + 1581, 0, 1583, 1591, 1592, 0, 1641, 1645, 0, 0, + 0, 0, 0, 1555, 1556, 1560, 1561, 1562, 1563, 1565, + 1566, 1567, 1568, 1569, 1570, 1571, 1572, 1573, 1574, 1575, + 817, 1545, 0, 0, 0, 0, 0, 0, 0, 798, + 0, 0, 0, 63, 0, 63, 1180, 1182, 0, 950, + 0, 0, 92, 0, 0, 79, 80, 0, 0, 0, + 978, 981, 987, 989, 0, 0, 0, 1335, 1336, 1338, + 1339, 1340, 0, 1014, 0, 0, 1034, 1035, 1036, 1048, + 0, 526, 527, 0, 0, 0, 539, 535, 536, 537, + 517, 1085, 1068, 0, 0, 1057, 0, 0, 1067, 0, + 1147, 1847, 1847, 1847, 0, 0, 1259, 1847, 1847, 0, + 1167, 1169, 0, 0, 1263, 1202, 0, 0, 1193, 0, + 945, 0, 0, 872, 708, 711, 712, 809, 590, 628, + 632, 629, 872, 1202, 427, 1223, 0, 0, 0, 0, + 0, 1255, 0, 0, 1227, 0, 458, 488, 0, -2, + 0, 1375, 0, 1361, 1375, 0, 0, 1374, 0, 447, + 487, 0, 0, 0, 501, 505, 506, 0, 503, 1414, + 0, 504, 0, 492, 0, 498, 1277, 1278, 0, 1283, + 1284, 0, 1308, 0, 0, 0, 438, 507, 0, 0, + 0, 508, 509, 514, 1087, 1088, 1054, 0, 1068, 0, + 1078, 0, 1075, 1076, 817, 0, 0, 889, 916, 0, + 0, 890, 0, 891, 893, 900, 902, 339, 931, 0, + 0, 933, 934, 935, 926, 286, 830, 1000, 0, 851, + 0, 0, 879, 852, 0, 19, 0, 0, 110, 1835, + 1838, 777, 0, 774, 162, 0, 0, 0, 0, 741, + 752, 735, 725, 775, 827, 828, 183, 178, 1101, 1184, + 0, 1175, 0, 1438, 1494, 0, 1593, 0, 1550, 1547, + 1550, 1549, 1541, 0, 1498, 0, 1500, 1501, 1502, 0, + 1504, 1505, 0, 796, 0, 59, 0, 62, 60, 0, + 94, 88, 0, 0, 0, 0, 0, 0, 0, 0, + 1020, 1263, 0, 1020, 1047, 1033, 0, 0, 528, 529, + 0, 532, 538, 1049, 0, 0, 1051, 1052, 1053, 0, + 0, 1065, 0, 0, 0, 0, 1139, 1153, 0, 0, + 0, -2, 0, -2, 1164, 0, 1208, 0, 1200, 0, + 1192, 0, 1195, 872, 872, -2, 705, 710, 0, 633, + 1208, 1225, 0, 1246, 0, 0, 0, 0, 0, 0, + 0, 1226, 0, 1239, 489, 1376, -2, 1390, 1392, 0, + 1152, 1395, 1396, 0, 0, 0, 0, 0, 0, 1445, + 1404, 0, 0, 1408, 1409, 1410, 0, 0, 1413, 0, + 1754, 1755, 0, 1417, 0, 0, 0, 0, 0, 0, + 0, 1355, 448, 449, 0, 451, 452, 1847, 1415, 491, + 445, 1847, 461, 1307, 1311, 1312, 1309, 513, 510, 511, + 1057, 1060, 1071, 1080, 732, 812, 341, 342, 917, 0, + 909, 940, 937, 0, 0, 1004, 862, 870, 2201, 2203, + 2200, 111, 116, 0, 0, 779, 0, 776, 0, 770, + 772, 172, 740, 777, 132, 164, 0, 0, 1482, 0, + 0, 0, 1582, 1632, 1553, 1554, 0, 1542, 0, 1536, + 1537, 1538, 1543, 0, 0, 799, 794, 64, 90, 0, + 93, 68, 81, 0, 0, 0, 0, 1006, 1013, 1027, + 1158, 1337, 1012, 0, 0, 525, 530, 0, 533, 534, + 1069, 1068, 0, 1055, 1056, 0, 1063, 0, 0, 1148, + 1149, 1150, 1260, 1261, 1262, 1218, 1166, 0, -2, 1271, + 0, 0, 1162, 1184, 1218, 0, 0, 1196, 0, 1203, + 0, 1201, 1194, 817, 706, 1205, 437, 1257, 1247, 0, + 1249, 0, 0, 0, 0, 1228, -2, 0, 1391, 1393, + 1394, 1397, 1398, 1399, 1450, 1451, 1452, 0, 0, 1402, + 1447, 1448, 1449, 1403, 0, 0, 0, 0, 0, 1752, + 1753, 1443, 0, 0, 1362, 1364, 1365, 1366, 1367, 1368, + 1369, 1370, 1363, 0, 0, 0, 1354, 1356, 450, 0, + 0, 1847, 1070, 338, 0, 0, 941, 943, 938, 939, + 864, 0, 0, 0, 0, 106, 108, 123, 0, 778, + 163, 0, 779, 134, 0, 155, 0, 1185, 0, 1493, + 0, 0, 0, 1552, 1539, 0, 0, 0, 0, 0, + 1756, 1757, 1758, 0, 1499, 1503, 89, 0, 66, 0, + 82, 83, 0, 0, 0, 1028, 1029, 1037, 1038, 0, + 1040, 1041, 531, 1050, 1058, 1062, 1065, 0, 1117, 1220, + 0, 1168, 1151, 1273, 1847, 1171, 1220, 0, 1265, 1847, + 1847, 1186, 0, 1198, 0, 1210, 0, 1204, 812, 426, + 0, 1207, 1243, 1248, 1250, 1252, 0, 1256, 1254, 1229, + -2, 0, 1237, 0, 0, 1400, 1401, 0, 0, 1651, + 1847, 0, 1433, 0, 1117, 1117, 1117, 1117, 0, 502, + 460, 0, 918, 932, 0, 871, 0, 0, 0, 0, + 0, 768, 124, 0, 133, 152, 0, 165, 166, 0, + 0, 0, 0, 1177, 0, 1441, 1442, 0, 1528, 0, + 0, 0, 1532, 1533, 1534, 1535, 65, 68, 0, 0, + 0, 1005, 0, 1039, 1064, 1066, 1116, 1161, 0, 1257, + 1272, 0, 1163, 1264, 0, 0, 0, 1197, 1209, 0, + 1212, 704, 1206, 1224, 0, 1253, 1230, 1238, 0, 1233, + 0, 0, 0, 1446, 0, 1407, 0, 1412, 1421, 1434, + 0, 0, 1343, 0, 1345, 0, 1349, 0, 1351, 0, + 0, 462, 942, 944, 0, 1702, 866, 867, 0, 781, + 771, 135, 139, 0, 161, 158, 0, 167, 0, 0, + 0, 0, 1173, 0, 1439, 0, 1529, 1530, 1531, 67, + 69, 84, 0, 1007, 1008, 1021, 1118, 1847, 1847, 0, + 0, 0, 1124, 1125, 0, 1245, 1277, 1266, 1267, 1268, + 1211, 1244, 1232, 0, -2, 1240, 0, 0, 1704, 1714, + 1715, 1405, 1411, 1420, 1422, 1423, 0, 1435, 1436, 1437, + 1444, 1117, 1117, 1117, 1117, 1353, 865, 0, 0, 780, + 0, 126, 0, 0, 156, 157, 159, 0, 168, 0, + 170, 171, 0, 0, 1540, 1009, 0, 0, 1121, 1122, + 0, 1221, 0, 1223, 1234, -2, 0, 1242, 0, 1406, + 1424, 0, 1425, 0, 0, 0, 1344, 1346, 1350, 1352, + 1702, 868, 782, 1183, 0, 140, 0, 142, 144, 145, + 1377, 153, 154, 160, 169, 0, 0, 994, 1010, 0, + 1119, 1120, 1123, 0, 1225, 1241, 1705, 1426, 1428, 1429, + 0, 0, 1427, 0, 127, 128, 0, 141, 0, 0, + 1178, 1440, 1011, 1222, 1219, 1430, 1432, 1431, 869, 0, + 0, 143, 1378, 129, 130, 131, 0, 1379, } var yyTok1 = [...]int{ 1, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 117, 3, 3, 3, 148, 140, 3, - 84, 85, 145, 143, 158, 144, 157, 146, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, 654, 651, - 127, 126, 128, 3, 655, 3, 3, 3, 3, 3, + 3, 3, 3, 118, 3, 3, 3, 149, 141, 3, + 85, 86, 146, 144, 159, 145, 158, 147, 3, 3, + 3, 3, 3, 3, 3, 3, 3, 3, 655, 652, + 128, 127, 129, 3, 656, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 150, 3, 3, 3, 3, 3, + 3, 3, 3, 3, 151, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 652, 139, 653, 151, + 3, 3, 3, 653, 140, 654, 152, } var yyTok2 = [...]int{ @@ -9867,13 +9870,13 @@ var yyTok2 = [...]int{ 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63, 64, 65, 66, 67, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 78, 79, 80, 81, - 82, 83, 86, 87, 88, 89, 90, 91, 92, 93, + 82, 83, 84, 87, 88, 89, 90, 91, 92, 93, 94, 95, 96, 97, 98, 99, 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, - 114, 115, 116, 118, 119, 120, 121, 122, 123, 124, - 125, 129, 130, 131, 132, 133, 134, 135, 136, 137, - 138, 141, 142, 147, 149, 152, 153, 154, 155, 156, - 159, 160, 161, 162, 163, 164, 165, 166, 167, 168, + 114, 115, 116, 117, 119, 120, 121, 122, 123, 124, + 125, 126, 130, 131, 132, 133, 134, 135, 136, 137, + 138, 139, 142, 143, 148, 150, 153, 154, 155, 156, + 157, 160, 161, 162, 163, 164, 165, 166, 167, 168, 169, 170, 171, 172, 173, 174, 175, 176, 177, 178, 179, 180, 181, 182, 183, 184, 185, 186, 187, 188, 189, 190, 191, 192, 193, 194, 195, 196, 197, 198, @@ -9963,7 +9966,7 @@ var yyTok3 = [...]int{ 57960, 635, 57961, 636, 57962, 637, 57963, 638, 57964, 639, 57965, 640, 57966, 641, 57967, 642, 57968, 643, 57969, 644, 57970, 645, 57971, 646, 57972, 647, 57973, 648, 57974, 649, - 57975, 650, 0, + 57975, 650, 57976, 651, 0, } var yyErrorMessages = [...]struct { @@ -17229,9 +17232,22 @@ yydefault: } yyVAL.union = yyLOCAL case 893: + yyDollar = yyS[yypt-4 : yypt+1] + var yyLOCAL *tree.JoinTableExpr +//line mysql_sql.y:5868 + { + yyLOCAL = &tree.JoinTableExpr{ + Left: yyDollar[1].tableExprUnion(), + JoinType: yyDollar[2].str, + Right: yyDollar[3].tableExprUnion(), + Cond: yyDollar[4].joinCondUnion(), + } + } + yyVAL.union = yyLOCAL + case 894: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *tree.ApplyTableExpr -//line mysql_sql.y:5870 +//line mysql_sql.y:5879 { yyLOCAL = &tree.ApplyTableExpr{ Left: yyDollar[1].tableExprUnion(), @@ -17240,27 +17256,27 @@ yydefault: } } yyVAL.union = yyLOCAL - case 894: + case 895: yyDollar = yyS[yypt-2 : yypt+1] -//line mysql_sql.y:5880 +//line mysql_sql.y:5889 { yyVAL.str = tree.APPLY_TYPE_CROSS } - case 895: + case 896: yyDollar = yyS[yypt-2 : yypt+1] -//line mysql_sql.y:5884 +//line mysql_sql.y:5893 { yyVAL.str = tree.APPLY_TYPE_OUTER } - case 896: + case 897: yyDollar = yyS[yypt-2 : yypt+1] -//line mysql_sql.y:5890 +//line mysql_sql.y:5899 { yyVAL.str = tree.JOIN_TYPE_NATURAL } - case 897: + case 898: yyDollar = yyS[yypt-2 : yypt+1] -//line mysql_sql.y:5894 +//line mysql_sql.y:5903 { if yyDollar[2].str == tree.JOIN_TYPE_LEFT { yyVAL.str = tree.JOIN_TYPE_NATURAL_LEFT @@ -17268,34 +17284,40 @@ yydefault: yyVAL.str = tree.JOIN_TYPE_NATURAL_RIGHT } } - case 898: + case 899: yyDollar = yyS[yypt-2 : yypt+1] -//line mysql_sql.y:5904 +//line mysql_sql.y:5913 { yyVAL.str = tree.JOIN_TYPE_LEFT } - case 899: + case 900: yyDollar = yyS[yypt-3 : yypt+1] -//line mysql_sql.y:5908 +//line mysql_sql.y:5917 { yyVAL.str = tree.JOIN_TYPE_LEFT } - case 900: + case 901: yyDollar = yyS[yypt-2 : yypt+1] -//line mysql_sql.y:5912 +//line mysql_sql.y:5921 { yyVAL.str = tree.JOIN_TYPE_RIGHT } - case 901: + case 902: yyDollar = yyS[yypt-3 : yypt+1] -//line mysql_sql.y:5916 +//line mysql_sql.y:5925 { yyVAL.str = tree.JOIN_TYPE_RIGHT } - case 902: + case 903: + yyDollar = yyS[yypt-2 : yypt+1] +//line mysql_sql.y:5931 + { + yyVAL.str = tree.JOIN_TYPE_DEDUP + } + case 904: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:5922 +//line mysql_sql.y:5937 { yyLOCAL = &tree.ValuesStatement{ Rows: yyDollar[2].rowsExprsUnion(), @@ -17304,136 +17326,136 @@ yydefault: } } yyVAL.union = yyLOCAL - case 903: + case 905: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []tree.Exprs -//line mysql_sql.y:5933 +//line mysql_sql.y:5948 { yyLOCAL = []tree.Exprs{yyDollar[1].exprsUnion()} } yyVAL.union = yyLOCAL - case 904: + case 906: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL []tree.Exprs -//line mysql_sql.y:5937 +//line mysql_sql.y:5952 { yyLOCAL = append(yyDollar[1].rowsExprsUnion(), yyDollar[3].exprsUnion()) } yyVAL.union = yyLOCAL - case 905: + case 907: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.Exprs -//line mysql_sql.y:5943 +//line mysql_sql.y:5958 { yyLOCAL = yyDollar[3].exprsUnion() } yyVAL.union = yyLOCAL - case 906: + case 908: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL tree.JoinCond -//line mysql_sql.y:5949 +//line mysql_sql.y:5964 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 907: + case 909: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.JoinCond -//line mysql_sql.y:5953 +//line mysql_sql.y:5968 { yyLOCAL = &tree.OnJoinCond{Expr: yyDollar[2].exprUnion()} } yyVAL.union = yyLOCAL - case 908: + case 910: yyDollar = yyS[yypt-1 : yypt+1] -//line mysql_sql.y:5959 +//line mysql_sql.y:5974 { yyVAL.str = tree.JOIN_TYPE_STRAIGHT } - case 909: + case 911: yyDollar = yyS[yypt-1 : yypt+1] -//line mysql_sql.y:5965 +//line mysql_sql.y:5980 { yyVAL.str = tree.JOIN_TYPE_INNER } - case 910: + case 912: yyDollar = yyS[yypt-2 : yypt+1] -//line mysql_sql.y:5969 +//line mysql_sql.y:5984 { yyVAL.str = tree.JOIN_TYPE_INNER } - case 911: + case 913: yyDollar = yyS[yypt-2 : yypt+1] -//line mysql_sql.y:5973 +//line mysql_sql.y:5988 { yyVAL.str = tree.JOIN_TYPE_CROSS } - case 912: + case 914: yyDollar = yyS[yypt-2 : yypt+1] -//line mysql_sql.y:5977 +//line mysql_sql.y:5992 { yyVAL.str = tree.JOIN_TYPE_CROSS_L2 } - case 913: + case 915: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL tree.JoinCond -//line mysql_sql.y:5983 +//line mysql_sql.y:5998 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 914: + case 916: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.JoinCond -//line mysql_sql.y:5987 +//line mysql_sql.y:6002 { yyLOCAL = yyDollar[1].joinCondUnion() } yyVAL.union = yyLOCAL - case 915: + case 917: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.JoinCond -//line mysql_sql.y:5993 +//line mysql_sql.y:6008 { yyLOCAL = &tree.OnJoinCond{Expr: yyDollar[2].exprUnion()} } yyVAL.union = yyLOCAL - case 916: + case 918: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.JoinCond -//line mysql_sql.y:5997 +//line mysql_sql.y:6012 { yyLOCAL = &tree.UsingJoinCond{Cols: yyDollar[3].identifierListUnion()} } yyVAL.union = yyLOCAL - case 917: + case 919: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.IdentifierList -//line mysql_sql.y:6003 +//line mysql_sql.y:6018 { yyLOCAL = tree.IdentifierList{tree.Identifier(yyDollar[1].cstrUnion().Compare())} } yyVAL.union = yyLOCAL - case 918: + case 920: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.IdentifierList -//line mysql_sql.y:6007 +//line mysql_sql.y:6022 { yyLOCAL = append(yyDollar[1].identifierListUnion(), tree.Identifier(yyDollar[3].cstrUnion().Compare())) } yyVAL.union = yyLOCAL - case 919: + case 921: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.TableExpr -//line mysql_sql.y:6013 +//line mysql_sql.y:6028 { yyLOCAL = yyDollar[1].aliasedTableExprUnion() } yyVAL.union = yyLOCAL - case 920: + case 922: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.TableExpr -//line mysql_sql.y:6017 +//line mysql_sql.y:6032 { yyLOCAL = &tree.AliasedTableExpr{ Expr: yyDollar[1].parenTableExprUnion(), @@ -17444,10 +17466,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 921: + case 923: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.TableExpr -//line mysql_sql.y:6027 +//line mysql_sql.y:6042 { if yyDollar[2].str != "" { yyLOCAL = &tree.AliasedTableExpr{ @@ -17461,26 +17483,26 @@ yydefault: } } yyVAL.union = yyLOCAL - case 922: + case 924: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.TableExpr -//line mysql_sql.y:6040 +//line mysql_sql.y:6055 { yyLOCAL = yyDollar[2].tableExprUnion() } yyVAL.union = yyLOCAL - case 923: + case 925: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.ParenTableExpr -//line mysql_sql.y:6046 +//line mysql_sql.y:6061 { yyLOCAL = &tree.ParenTableExpr{Expr: yyDollar[1].selectStatementUnion().(*tree.ParenSelect).Select} } yyVAL.union = yyLOCAL - case 924: + case 926: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.TableExpr -//line mysql_sql.y:6052 +//line mysql_sql.y:6067 { name := tree.NewUnresolvedName(yyDollar[1].cstrUnion()) yyLOCAL = &tree.TableFunction{ @@ -17493,10 +17515,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 925: + case 927: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *tree.AliasedTableExpr -//line mysql_sql.y:6066 +//line mysql_sql.y:6081 { yyLOCAL = &tree.AliasedTableExpr{ Expr: yyDollar[1].tableNameUnion(), @@ -17507,34 +17529,34 @@ yydefault: } } yyVAL.union = yyLOCAL - case 926: + case 928: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL []*tree.IndexHint -//line mysql_sql.y:6077 +//line mysql_sql.y:6092 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 928: + case 930: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []*tree.IndexHint -//line mysql_sql.y:6084 +//line mysql_sql.y:6099 { yyLOCAL = []*tree.IndexHint{yyDollar[1].indexHintUnion()} } yyVAL.union = yyLOCAL - case 929: + case 931: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL []*tree.IndexHint -//line mysql_sql.y:6088 +//line mysql_sql.y:6103 { yyLOCAL = append(yyDollar[1].indexHintListUnion(), yyDollar[2].indexHintUnion()) } yyVAL.union = yyLOCAL - case 930: + case 932: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *tree.IndexHint -//line mysql_sql.y:6094 +//line mysql_sql.y:6109 { yyLOCAL = &tree.IndexHint{ IndexNames: yyDollar[4].strsUnion(), @@ -17543,182 +17565,182 @@ yydefault: } } yyVAL.union = yyLOCAL - case 931: + case 933: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.IndexHintType -//line mysql_sql.y:6104 +//line mysql_sql.y:6119 { yyLOCAL = tree.HintUse } yyVAL.union = yyLOCAL - case 932: + case 934: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.IndexHintType -//line mysql_sql.y:6108 +//line mysql_sql.y:6123 { yyLOCAL = tree.HintIgnore } yyVAL.union = yyLOCAL - case 933: + case 935: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.IndexHintType -//line mysql_sql.y:6112 +//line mysql_sql.y:6127 { yyLOCAL = tree.HintForce } yyVAL.union = yyLOCAL - case 934: + case 936: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL tree.IndexHintScope -//line mysql_sql.y:6117 +//line mysql_sql.y:6132 { yyLOCAL = tree.HintForScan } yyVAL.union = yyLOCAL - case 935: + case 937: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.IndexHintScope -//line mysql_sql.y:6121 +//line mysql_sql.y:6136 { yyLOCAL = tree.HintForJoin } yyVAL.union = yyLOCAL - case 936: + case 938: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.IndexHintScope -//line mysql_sql.y:6125 +//line mysql_sql.y:6140 { yyLOCAL = tree.HintForOrderBy } yyVAL.union = yyLOCAL - case 937: + case 939: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.IndexHintScope -//line mysql_sql.y:6129 +//line mysql_sql.y:6144 { yyLOCAL = tree.HintForGroupBy } yyVAL.union = yyLOCAL - case 938: + case 940: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL []string -//line mysql_sql.y:6134 +//line mysql_sql.y:6149 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 939: + case 941: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []string -//line mysql_sql.y:6138 +//line mysql_sql.y:6153 { yyLOCAL = []string{yyDollar[1].cstrUnion().Compare()} } yyVAL.union = yyLOCAL - case 940: + case 942: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL []string -//line mysql_sql.y:6142 +//line mysql_sql.y:6157 { yyLOCAL = append(yyDollar[1].strsUnion(), yyDollar[3].cstrUnion().Compare()) } yyVAL.union = yyLOCAL - case 941: + case 943: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []string -//line mysql_sql.y:6146 +//line mysql_sql.y:6161 { yyLOCAL = []string{yyDollar[1].str} } yyVAL.union = yyLOCAL - case 942: + case 944: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL []string -//line mysql_sql.y:6150 +//line mysql_sql.y:6165 { yyLOCAL = append(yyDollar[1].strsUnion(), yyDollar[3].str) } yyVAL.union = yyLOCAL - case 943: + case 945: yyDollar = yyS[yypt-0 : yypt+1] -//line mysql_sql.y:6155 +//line mysql_sql.y:6170 { yyVAL.str = "" } - case 944: + case 946: yyDollar = yyS[yypt-1 : yypt+1] -//line mysql_sql.y:6159 +//line mysql_sql.y:6174 { yyVAL.str = yyDollar[1].str } - case 945: + case 947: yyDollar = yyS[yypt-2 : yypt+1] -//line mysql_sql.y:6163 +//line mysql_sql.y:6178 { yyVAL.str = yyDollar[2].str } - case 946: + case 948: yyDollar = yyS[yypt-1 : yypt+1] -//line mysql_sql.y:6169 +//line mysql_sql.y:6184 { yyVAL.str = yylex.(*Lexer).GetDbOrTblName(yyDollar[1].cstrUnion().Origin()) } - case 947: + case 949: yyDollar = yyS[yypt-1 : yypt+1] -//line mysql_sql.y:6173 +//line mysql_sql.y:6188 { yyVAL.str = yylex.(*Lexer).GetDbOrTblName(yyDollar[1].str) } - case 948: + case 950: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *tree.CStr -//line mysql_sql.y:6178 +//line mysql_sql.y:6193 { yyLOCAL = tree.NewCStr("", 1) } yyVAL.union = yyLOCAL - case 949: + case 951: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.CStr -//line mysql_sql.y:6182 +//line mysql_sql.y:6197 { yyLOCAL = yyDollar[1].cstrUnion() } yyVAL.union = yyLOCAL - case 950: + case 952: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.CStr -//line mysql_sql.y:6186 +//line mysql_sql.y:6201 { yyLOCAL = yyDollar[2].cstrUnion() } yyVAL.union = yyLOCAL - case 951: + case 953: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.CStr -//line mysql_sql.y:6190 +//line mysql_sql.y:6205 { yyLOCAL = tree.NewCStr(yyDollar[1].str, 1) } yyVAL.union = yyLOCAL - case 952: + case 954: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.CStr -//line mysql_sql.y:6194 +//line mysql_sql.y:6209 { yyLOCAL = tree.NewCStr(yyDollar[2].str, 1) } yyVAL.union = yyLOCAL - case 953: + case 955: yyDollar = yyS[yypt-1 : yypt+1] -//line mysql_sql.y:6200 +//line mysql_sql.y:6215 { yyVAL.str = yyDollar[1].cstrUnion().Compare() } - case 976: + case 978: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:6242 +//line mysql_sql.y:6257 { var Language = yyDollar[3].str var Name = tree.Identifier(yyDollar[5].str) @@ -17730,22 +17752,22 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 977: + case 979: yyDollar = yyS[yypt-1 : yypt+1] -//line mysql_sql.y:6255 +//line mysql_sql.y:6270 { yyVAL.str = yyDollar[1].cstrUnion().Compare() } - case 978: + case 980: yyDollar = yyS[yypt-1 : yypt+1] -//line mysql_sql.y:6261 +//line mysql_sql.y:6276 { yyVAL.str = yyDollar[1].cstrUnion().Compare() } - case 979: + case 981: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:6267 +//line mysql_sql.y:6282 { var Name = yyDollar[3].procNameUnion() var Args = yyDollar[5].procArgsUnion() @@ -17757,101 +17779,101 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 980: + case 982: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.ProcedureName -//line mysql_sql.y:6280 +//line mysql_sql.y:6295 { prefix := tree.ObjectNamePrefix{ExplicitSchema: false} yyLOCAL = tree.NewProcedureName(tree.Identifier(yyDollar[1].cstrUnion().Compare()), prefix) } yyVAL.union = yyLOCAL - case 981: + case 983: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *tree.ProcedureName -//line mysql_sql.y:6285 +//line mysql_sql.y:6300 { dbName := yylex.(*Lexer).GetDbOrTblName(yyDollar[1].cstrUnion().Origin()) prefix := tree.ObjectNamePrefix{SchemaName: tree.Identifier(dbName), ExplicitSchema: true} yyLOCAL = tree.NewProcedureName(tree.Identifier(yyDollar[3].cstrUnion().Compare()), prefix) } yyVAL.union = yyLOCAL - case 982: + case 984: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL tree.ProcedureArgs -//line mysql_sql.y:6292 +//line mysql_sql.y:6307 { yyLOCAL = tree.ProcedureArgs(nil) } yyVAL.union = yyLOCAL - case 984: + case 986: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.ProcedureArgs -//line mysql_sql.y:6299 +//line mysql_sql.y:6314 { yyLOCAL = tree.ProcedureArgs{yyDollar[1].procArgUnion()} } yyVAL.union = yyLOCAL - case 985: + case 987: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.ProcedureArgs -//line mysql_sql.y:6303 +//line mysql_sql.y:6318 { yyLOCAL = append(yyDollar[1].procArgsUnion(), yyDollar[3].procArgUnion()) } yyVAL.union = yyLOCAL - case 986: + case 988: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.ProcedureArg -//line mysql_sql.y:6309 +//line mysql_sql.y:6324 { yyLOCAL = tree.ProcedureArg(yyDollar[1].procArgDeclUnion()) } yyVAL.union = yyLOCAL - case 987: + case 989: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *tree.ProcedureArgDecl -//line mysql_sql.y:6315 +//line mysql_sql.y:6330 { yyLOCAL = tree.NewProcedureArgDecl(yyDollar[1].procArgTypeUnion(), yyDollar[2].unresolvedNameUnion(), yyDollar[3].columnTypeUnion()) } yyVAL.union = yyLOCAL - case 988: + case 990: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL tree.InOutArgType -//line mysql_sql.y:6320 +//line mysql_sql.y:6335 { yyLOCAL = tree.TYPE_IN } yyVAL.union = yyLOCAL - case 989: + case 991: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.InOutArgType -//line mysql_sql.y:6324 +//line mysql_sql.y:6339 { yyLOCAL = tree.TYPE_IN } yyVAL.union = yyLOCAL - case 990: + case 992: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.InOutArgType -//line mysql_sql.y:6328 +//line mysql_sql.y:6343 { yyLOCAL = tree.TYPE_OUT } yyVAL.union = yyLOCAL - case 991: + case 993: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.InOutArgType -//line mysql_sql.y:6332 +//line mysql_sql.y:6347 { yyLOCAL = tree.TYPE_INOUT } yyVAL.union = yyLOCAL - case 992: + case 994: yyDollar = yyS[yypt-14 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:6339 +//line mysql_sql.y:6354 { if yyDollar[13].str == "" { yylex.Error("no function body error") @@ -17883,127 +17905,127 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 993: + case 995: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.FunctionName -//line mysql_sql.y:6372 +//line mysql_sql.y:6387 { prefix := tree.ObjectNamePrefix{ExplicitSchema: false} yyLOCAL = tree.NewFuncName(tree.Identifier(yyDollar[1].cstrUnion().Compare()), prefix) } yyVAL.union = yyLOCAL - case 994: + case 996: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *tree.FunctionName -//line mysql_sql.y:6377 +//line mysql_sql.y:6392 { dbName := yylex.(*Lexer).GetDbOrTblName(yyDollar[1].cstrUnion().Origin()) prefix := tree.ObjectNamePrefix{SchemaName: tree.Identifier(dbName), ExplicitSchema: true} yyLOCAL = tree.NewFuncName(tree.Identifier(yyDollar[3].cstrUnion().Compare()), prefix) } yyVAL.union = yyLOCAL - case 995: + case 997: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL tree.FunctionArgs -//line mysql_sql.y:6384 +//line mysql_sql.y:6399 { yyLOCAL = tree.FunctionArgs(nil) } yyVAL.union = yyLOCAL - case 997: + case 999: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.FunctionArgs -//line mysql_sql.y:6391 +//line mysql_sql.y:6406 { yyLOCAL = tree.FunctionArgs{yyDollar[1].funcArgUnion()} } yyVAL.union = yyLOCAL - case 998: + case 1000: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.FunctionArgs -//line mysql_sql.y:6395 +//line mysql_sql.y:6410 { yyLOCAL = append(yyDollar[1].funcArgsUnion(), yyDollar[3].funcArgUnion()) } yyVAL.union = yyLOCAL - case 999: + case 1001: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.FunctionArg -//line mysql_sql.y:6401 +//line mysql_sql.y:6416 { yyLOCAL = tree.FunctionArg(yyDollar[1].funcArgDeclUnion()) } yyVAL.union = yyLOCAL - case 1000: + case 1002: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.FunctionArgDecl -//line mysql_sql.y:6407 +//line mysql_sql.y:6422 { yyLOCAL = tree.NewFunctionArgDecl(nil, yyDollar[1].columnTypeUnion(), nil) } yyVAL.union = yyLOCAL - case 1001: + case 1003: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.FunctionArgDecl -//line mysql_sql.y:6411 +//line mysql_sql.y:6426 { yyLOCAL = tree.NewFunctionArgDecl(yyDollar[1].unresolvedNameUnion(), yyDollar[2].columnTypeUnion(), nil) } yyVAL.union = yyLOCAL - case 1002: + case 1004: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *tree.FunctionArgDecl -//line mysql_sql.y:6415 +//line mysql_sql.y:6430 { yyLOCAL = tree.NewFunctionArgDecl(yyDollar[1].unresolvedNameUnion(), yyDollar[2].columnTypeUnion(), yyDollar[4].exprUnion()) } yyVAL.union = yyLOCAL - case 1003: + case 1005: yyDollar = yyS[yypt-1 : yypt+1] -//line mysql_sql.y:6421 +//line mysql_sql.y:6436 { yyVAL.str = yyDollar[1].cstrUnion().Compare() } - case 1004: + case 1006: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.ReturnType -//line mysql_sql.y:6427 +//line mysql_sql.y:6442 { yyLOCAL = tree.NewReturnType(yyDollar[1].columnTypeUnion()) } yyVAL.union = yyLOCAL - case 1005: + case 1007: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL bool -//line mysql_sql.y:6433 +//line mysql_sql.y:6448 { yyLOCAL = false } yyVAL.union = yyLOCAL - case 1006: + case 1008: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL bool -//line mysql_sql.y:6437 +//line mysql_sql.y:6452 { yyLOCAL = true } yyVAL.union = yyLOCAL - case 1007: + case 1009: yyDollar = yyS[yypt-0 : yypt+1] -//line mysql_sql.y:6443 +//line mysql_sql.y:6458 { yyVAL.str = "" } - case 1009: + case 1011: yyDollar = yyS[yypt-2 : yypt+1] -//line mysql_sql.y:6450 +//line mysql_sql.y:6465 { yyVAL.str = yyDollar[2].str } - case 1010: + case 1012: yyDollar = yyS[yypt-9 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:6456 +//line mysql_sql.y:6471 { var Replace bool var Name = yyDollar[5].tableNameUnion() @@ -18019,10 +18041,10 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1011: + case 1013: yyDollar = yyS[yypt-9 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:6471 +//line mysql_sql.y:6486 { var Replace = yyDollar[2].sourceOptionalUnion() var Name = yyDollar[5].tableNameUnion() @@ -18038,10 +18060,10 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1012: + case 1014: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:6488 +//line mysql_sql.y:6503 { var IfNotExists = yyDollar[3].ifNotExistsUnion() var Name = yyDollar[4].exprUnion() @@ -18057,81 +18079,81 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1013: + case 1015: yyDollar = yyS[yypt-1 : yypt+1] -//line mysql_sql.y:6505 +//line mysql_sql.y:6520 { yyVAL.str = yyDollar[1].str } - case 1014: + case 1016: yyDollar = yyS[yypt-2 : yypt+1] -//line mysql_sql.y:6509 +//line mysql_sql.y:6524 { yyVAL.str = yyVAL.str + yyDollar[2].str } - case 1015: + case 1017: yyDollar = yyS[yypt-3 : yypt+1] -//line mysql_sql.y:6515 +//line mysql_sql.y:6530 { yyVAL.str = "ALGORITHM = " + yyDollar[3].str } - case 1016: + case 1018: yyDollar = yyS[yypt-3 : yypt+1] -//line mysql_sql.y:6519 +//line mysql_sql.y:6534 { yyVAL.str = "DEFINER = " } - case 1017: + case 1019: yyDollar = yyS[yypt-3 : yypt+1] -//line mysql_sql.y:6523 +//line mysql_sql.y:6538 { yyVAL.str = "SQL SECURITY " + yyDollar[3].str } - case 1018: + case 1020: yyDollar = yyS[yypt-0 : yypt+1] -//line mysql_sql.y:6528 +//line mysql_sql.y:6543 { yyVAL.str = "" } - case 1019: + case 1021: yyDollar = yyS[yypt-4 : yypt+1] -//line mysql_sql.y:6532 +//line mysql_sql.y:6547 { yyVAL.str = "WITH " + yyDollar[2].str + " CHECK OPTION" } - case 1025: + case 1027: yyDollar = yyS[yypt-0 : yypt+1] -//line mysql_sql.y:6546 +//line mysql_sql.y:6561 { yyVAL.str = "" } - case 1028: + case 1030: yyDollar = yyS[yypt-1 : yypt+1] -//line mysql_sql.y:6554 +//line mysql_sql.y:6569 { yyVAL.str = yyDollar[1].cstrUnion().Compare() } - case 1029: + case 1031: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:6560 +//line mysql_sql.y:6575 { var str = yyDollar[1].cstrUnion().Compare() yyLOCAL = tree.NewNumVal(str, str, false, tree.P_char) } yyVAL.union = yyLOCAL - case 1030: + case 1032: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:6565 +//line mysql_sql.y:6580 { yyLOCAL = tree.NewParamExpr(yylex.(*Lexer).GetParamIndex()) } yyVAL.union = yyLOCAL - case 1031: + case 1033: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.AccountAuthOption -//line mysql_sql.y:6571 +//line mysql_sql.y:6586 { var Equal = yyDollar[2].str var AdminName = yyDollar[3].exprUnion() @@ -18143,36 +18165,36 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1032: + case 1034: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:6584 +//line mysql_sql.y:6599 { var str = yyDollar[1].str yyLOCAL = tree.NewNumVal(str, str, false, tree.P_char) } yyVAL.union = yyLOCAL - case 1033: + case 1035: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:6589 +//line mysql_sql.y:6604 { var str = yyDollar[1].cstrUnion().Compare() yyLOCAL = tree.NewNumVal(str, str, false, tree.P_char) } yyVAL.union = yyLOCAL - case 1034: + case 1036: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:6594 +//line mysql_sql.y:6609 { yyLOCAL = tree.NewParamExpr(yylex.(*Lexer).GetParamIndex()) } yyVAL.union = yyLOCAL - case 1035: + case 1037: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.AccountIdentified -//line mysql_sql.y:6600 +//line mysql_sql.y:6615 { yyLOCAL = *tree.NewAccountIdentified( tree.AccountIdentifiedByPassword, @@ -18180,10 +18202,10 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1036: + case 1038: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.AccountIdentified -//line mysql_sql.y:6607 +//line mysql_sql.y:6622 { yyLOCAL = *tree.NewAccountIdentified( tree.AccountIdentifiedByPassword, @@ -18191,10 +18213,10 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1037: + case 1039: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.AccountIdentified -//line mysql_sql.y:6614 +//line mysql_sql.y:6629 { yyLOCAL = *tree.NewAccountIdentified( tree.AccountIdentifiedByRandomPassword, @@ -18202,10 +18224,10 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1038: + case 1040: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.AccountIdentified -//line mysql_sql.y:6621 +//line mysql_sql.y:6636 { yyLOCAL = *tree.NewAccountIdentified( tree.AccountIdentifiedWithSSL, @@ -18213,10 +18235,10 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1039: + case 1041: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.AccountIdentified -//line mysql_sql.y:6628 +//line mysql_sql.y:6643 { yyLOCAL = *tree.NewAccountIdentified( tree.AccountIdentifiedWithSSL, @@ -18224,20 +18246,20 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1040: + case 1042: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL tree.AccountStatus -//line mysql_sql.y:6636 +//line mysql_sql.y:6651 { as := tree.NewAccountStatus() as.Exist = false yyLOCAL = *as } yyVAL.union = yyLOCAL - case 1041: + case 1043: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.AccountStatus -//line mysql_sql.y:6642 +//line mysql_sql.y:6657 { as := tree.NewAccountStatus() as.Exist = true @@ -18245,10 +18267,10 @@ yydefault: yyLOCAL = *as } yyVAL.union = yyLOCAL - case 1042: + case 1044: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.AccountStatus -//line mysql_sql.y:6649 +//line mysql_sql.y:6664 { as := tree.NewAccountStatus() as.Exist = true @@ -18256,10 +18278,10 @@ yydefault: yyLOCAL = *as } yyVAL.union = yyLOCAL - case 1043: + case 1045: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.AccountStatus -//line mysql_sql.y:6656 +//line mysql_sql.y:6671 { as := tree.NewAccountStatus() as.Exist = true @@ -18267,20 +18289,20 @@ yydefault: yyLOCAL = *as } yyVAL.union = yyLOCAL - case 1044: + case 1046: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL tree.AccountComment -//line mysql_sql.y:6664 +//line mysql_sql.y:6679 { ac := tree.NewAccountComment() ac.Exist = false yyLOCAL = *ac } yyVAL.union = yyLOCAL - case 1045: + case 1047: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.AccountComment -//line mysql_sql.y:6670 +//line mysql_sql.y:6685 { ac := tree.NewAccountComment() ac.Exist = true @@ -18288,10 +18310,10 @@ yydefault: yyLOCAL = *ac } yyVAL.union = yyLOCAL - case 1046: + case 1048: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:6679 +//line mysql_sql.y:6694 { var IfNotExists = yyDollar[3].ifNotExistsUnion() var Users = yyDollar[4].usersUnion() @@ -18307,10 +18329,10 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1047: + case 1049: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:6696 +//line mysql_sql.y:6711 { var IfNotExists = yyDollar[3].ifNotExistsUnion() var Name = tree.Identifier(yyDollar[4].cstrUnion().Compare()) @@ -18327,10 +18349,10 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1048: + case 1050: yyDollar = yyS[yypt-10 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:6712 +//line mysql_sql.y:6727 { var IfNotExists = yyDollar[3].ifNotExistsUnion() var Name = tree.Identifier(yyDollar[4].cstrUnion().Compare()) @@ -18348,30 +18370,30 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1049: + case 1051: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.AccountsSetOption -//line mysql_sql.y:6731 +//line mysql_sql.y:6746 { yyLOCAL = &tree.AccountsSetOption{ All: true, } } yyVAL.union = yyLOCAL - case 1050: + case 1052: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.AccountsSetOption -//line mysql_sql.y:6737 +//line mysql_sql.y:6752 { yyLOCAL = &tree.AccountsSetOption{ SetAccounts: yyDollar[2].identifierListUnion(), } } yyVAL.union = yyLOCAL - case 1051: + case 1053: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:6746 +//line mysql_sql.y:6761 { var IfNotExists = yyDollar[3].ifNotExistsUnion() var Name = tree.Identifier(yyDollar[4].cstrUnion().Compare()) @@ -18389,20 +18411,20 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1052: + case 1054: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL tree.StageStatus -//line mysql_sql.y:6764 +//line mysql_sql.y:6779 { yyLOCAL = tree.StageStatus{ Exist: false, } } yyVAL.union = yyLOCAL - case 1053: + case 1055: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.StageStatus -//line mysql_sql.y:6770 +//line mysql_sql.y:6785 { yyLOCAL = tree.StageStatus{ Exist: true, @@ -18410,10 +18432,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1054: + case 1056: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.StageStatus -//line mysql_sql.y:6777 +//line mysql_sql.y:6792 { yyLOCAL = tree.StageStatus{ Exist: true, @@ -18421,20 +18443,20 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1055: + case 1057: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL tree.StageComment -//line mysql_sql.y:6785 +//line mysql_sql.y:6800 { yyLOCAL = tree.StageComment{ Exist: false, } } yyVAL.union = yyLOCAL - case 1056: + case 1058: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.StageComment -//line mysql_sql.y:6791 +//line mysql_sql.y:6806 { yyLOCAL = tree.StageComment{ Exist: true, @@ -18442,20 +18464,20 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1057: + case 1059: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL tree.StageUrl -//line mysql_sql.y:6799 +//line mysql_sql.y:6814 { yyLOCAL = tree.StageUrl{ Exist: false, } } yyVAL.union = yyLOCAL - case 1058: + case 1060: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.StageUrl -//line mysql_sql.y:6805 +//line mysql_sql.y:6820 { yyLOCAL = tree.StageUrl{ Exist: true, @@ -18463,20 +18485,20 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1059: + case 1061: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL tree.StageCredentials -//line mysql_sql.y:6813 +//line mysql_sql.y:6828 { yyLOCAL = tree.StageCredentials{ Exist: false, } } yyVAL.union = yyLOCAL - case 1060: + case 1062: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL tree.StageCredentials -//line mysql_sql.y:6819 +//line mysql_sql.y:6834 { yyLOCAL = tree.StageCredentials{ Exist: true, @@ -18484,61 +18506,61 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1061: + case 1063: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []string -//line mysql_sql.y:6828 +//line mysql_sql.y:6843 { yyLOCAL = yyDollar[1].strsUnion() } yyVAL.union = yyLOCAL - case 1062: + case 1064: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL []string -//line mysql_sql.y:6832 +//line mysql_sql.y:6847 { yyLOCAL = append(yyDollar[1].strsUnion(), yyDollar[3].strsUnion()...) } yyVAL.union = yyLOCAL - case 1063: + case 1065: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL []string -//line mysql_sql.y:6837 +//line mysql_sql.y:6852 { yyLOCAL = []string{} } yyVAL.union = yyLOCAL - case 1064: + case 1066: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL []string -//line mysql_sql.y:6841 +//line mysql_sql.y:6856 { yyLOCAL = append(yyLOCAL, yyDollar[1].str) yyLOCAL = append(yyLOCAL, yyDollar[3].str) } yyVAL.union = yyLOCAL - case 1065: + case 1067: yyDollar = yyS[yypt-3 : yypt+1] -//line mysql_sql.y:6848 +//line mysql_sql.y:6863 { yyVAL.str = yyDollar[3].str } - case 1066: + case 1068: yyDollar = yyS[yypt-0 : yypt+1] -//line mysql_sql.y:6853 +//line mysql_sql.y:6868 { yyVAL.str = "" } - case 1067: + case 1069: yyDollar = yyS[yypt-2 : yypt+1] -//line mysql_sql.y:6857 +//line mysql_sql.y:6872 { yyVAL.str = yyDollar[2].str } - case 1068: + case 1070: yyDollar = yyS[yypt-9 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:6863 +//line mysql_sql.y:6878 { var ifNotExists = yyDollar[3].boolValUnion() var name = tree.Identifier(yyDollar[4].cstrUnion().Compare()) @@ -18549,10 +18571,10 @@ yydefault: yyLOCAL = tree.NewAlterStage(ifNotExists, name, urlOption, credentialsOption, statusOption, comment) } yyVAL.union = yyLOCAL - case 1069: + case 1071: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:6876 +//line mysql_sql.y:6891 { var ifExists = yyDollar[3].boolValUnion() var name = tree.Identifier(yyDollar[4].cstrUnion().Compare()) @@ -18563,132 +18585,132 @@ yydefault: yyLOCAL = tree.NewAlterPublication(ifExists, name, accountsSet, dbName, table, comment) } yyVAL.union = yyLOCAL - case 1070: + case 1072: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *tree.AccountsSetOption -//line mysql_sql.y:6887 +//line mysql_sql.y:6902 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1071: + case 1073: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.AccountsSetOption -//line mysql_sql.y:6891 +//line mysql_sql.y:6906 { yyLOCAL = &tree.AccountsSetOption{ All: true, } } yyVAL.union = yyLOCAL - case 1072: + case 1074: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.AccountsSetOption -//line mysql_sql.y:6897 +//line mysql_sql.y:6912 { yyLOCAL = &tree.AccountsSetOption{ SetAccounts: yyDollar[2].identifierListUnion(), } } yyVAL.union = yyLOCAL - case 1073: + case 1075: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *tree.AccountsSetOption -//line mysql_sql.y:6903 +//line mysql_sql.y:6918 { yyLOCAL = &tree.AccountsSetOption{ AddAccounts: yyDollar[3].identifierListUnion(), } } yyVAL.union = yyLOCAL - case 1074: + case 1076: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *tree.AccountsSetOption -//line mysql_sql.y:6909 +//line mysql_sql.y:6924 { yyLOCAL = &tree.AccountsSetOption{ DropAccounts: yyDollar[3].identifierListUnion(), } } yyVAL.union = yyLOCAL - case 1075: + case 1077: yyDollar = yyS[yypt-0 : yypt+1] -//line mysql_sql.y:6916 +//line mysql_sql.y:6931 { yyVAL.str = "" } - case 1076: + case 1078: yyDollar = yyS[yypt-2 : yypt+1] -//line mysql_sql.y:6920 +//line mysql_sql.y:6935 { yyVAL.str = yyDollar[2].str } - case 1077: + case 1079: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL tree.TableNames -//line mysql_sql.y:6925 +//line mysql_sql.y:6940 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1078: + case 1080: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.TableNames -//line mysql_sql.y:6929 +//line mysql_sql.y:6944 { yyLOCAL = yyDollar[2].tableNamesUnion() } yyVAL.union = yyLOCAL - case 1079: + case 1081: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:6935 +//line mysql_sql.y:6950 { var ifExists = yyDollar[3].boolValUnion() var name = tree.Identifier(yyDollar[4].cstrUnion().Compare()) yyLOCAL = tree.NewDropPublication(ifExists, name) } yyVAL.union = yyLOCAL - case 1080: + case 1082: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:6943 +//line mysql_sql.y:6958 { var ifNotExists = yyDollar[3].boolValUnion() var name = tree.Identifier(yyDollar[4].cstrUnion().Compare()) yyLOCAL = tree.NewDropStage(ifNotExists, name) } yyVAL.union = yyLOCAL - case 1081: + case 1083: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:6951 +//line mysql_sql.y:6966 { var ifExists = yyDollar[3].boolValUnion() var name = tree.Identifier(yyDollar[4].cstrUnion().Compare()) yyLOCAL = tree.NewDropSnapShot(ifExists, name) } yyVAL.union = yyLOCAL - case 1082: + case 1084: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:6959 +//line mysql_sql.y:6974 { var ifExists = yyDollar[3].boolValUnion() var name = tree.Identifier(yyDollar[4].cstrUnion().Compare()) yyLOCAL = tree.NewDropPitr(ifExists, name) } yyVAL.union = yyLOCAL - case 1083: + case 1085: yyDollar = yyS[yypt-1 : yypt+1] -//line mysql_sql.y:6967 +//line mysql_sql.y:6982 { yyVAL.str = yyDollar[1].cstrUnion().Compare() } - case 1084: + case 1086: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL tree.AccountCommentOrAttribute -//line mysql_sql.y:6972 +//line mysql_sql.y:6987 { var Exist = false var IsComment bool @@ -18701,10 +18723,10 @@ yydefault: } yyVAL.union = yyLOCAL - case 1085: + case 1087: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.AccountCommentOrAttribute -//line mysql_sql.y:6984 +//line mysql_sql.y:6999 { var Exist = true var IsComment = true @@ -18716,10 +18738,10 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1086: + case 1088: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.AccountCommentOrAttribute -//line mysql_sql.y:6995 +//line mysql_sql.y:7010 { var Exist = true var IsComment = false @@ -18731,26 +18753,26 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1087: + case 1089: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []*tree.User -//line mysql_sql.y:7104 +//line mysql_sql.y:7119 { yyLOCAL = []*tree.User{yyDollar[1].userUnion()} } yyVAL.union = yyLOCAL - case 1088: + case 1090: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL []*tree.User -//line mysql_sql.y:7108 +//line mysql_sql.y:7123 { yyLOCAL = append(yyDollar[1].usersUnion(), yyDollar[3].userUnion()) } yyVAL.union = yyLOCAL - case 1089: + case 1091: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.User -//line mysql_sql.y:7114 +//line mysql_sql.y:7129 { var Username = yyDollar[1].usernameRecordUnion().Username var Hostname = yyDollar[1].usernameRecordUnion().Hostname @@ -18762,26 +18784,26 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1090: + case 1092: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []*tree.User -//line mysql_sql.y:7127 +//line mysql_sql.y:7142 { yyLOCAL = []*tree.User{yyDollar[1].userUnion()} } yyVAL.union = yyLOCAL - case 1091: + case 1093: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL []*tree.User -//line mysql_sql.y:7131 +//line mysql_sql.y:7146 { yyLOCAL = append(yyDollar[1].usersUnion(), yyDollar[3].userUnion()) } yyVAL.union = yyLOCAL - case 1092: + case 1094: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.User -//line mysql_sql.y:7137 +//line mysql_sql.y:7152 { var Username = yyDollar[1].usernameRecordUnion().Username var Hostname = yyDollar[1].usernameRecordUnion().Hostname @@ -18793,50 +18815,50 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1093: + case 1095: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.UsernameRecord -//line mysql_sql.y:7150 +//line mysql_sql.y:7165 { yyLOCAL = &tree.UsernameRecord{Username: yyDollar[1].str, Hostname: "%"} } yyVAL.union = yyLOCAL - case 1094: + case 1096: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *tree.UsernameRecord -//line mysql_sql.y:7154 +//line mysql_sql.y:7169 { yyLOCAL = &tree.UsernameRecord{Username: yyDollar[1].str, Hostname: yyDollar[3].str} } yyVAL.union = yyLOCAL - case 1095: + case 1097: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.UsernameRecord -//line mysql_sql.y:7158 +//line mysql_sql.y:7173 { yyLOCAL = &tree.UsernameRecord{Username: yyDollar[1].str, Hostname: yyDollar[2].str} } yyVAL.union = yyLOCAL - case 1096: + case 1098: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *tree.AccountIdentified -//line mysql_sql.y:7163 +//line mysql_sql.y:7178 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1097: + case 1099: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.AccountIdentified -//line mysql_sql.y:7167 +//line mysql_sql.y:7182 { yyLOCAL = yyDollar[1].userIdentifiedUnion() } yyVAL.union = yyLOCAL - case 1098: + case 1100: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *tree.AccountIdentified -//line mysql_sql.y:7173 +//line mysql_sql.y:7188 { yyLOCAL = &tree.AccountIdentified{ Typ: tree.AccountIdentifiedByPassword, @@ -18844,20 +18866,20 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1099: + case 1101: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *tree.AccountIdentified -//line mysql_sql.y:7180 +//line mysql_sql.y:7195 { yyLOCAL = &tree.AccountIdentified{ Typ: tree.AccountIdentifiedByRandomPassword, } } yyVAL.union = yyLOCAL - case 1100: + case 1102: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *tree.AccountIdentified -//line mysql_sql.y:7186 +//line mysql_sql.y:7201 { yyLOCAL = &tree.AccountIdentified{ Typ: tree.AccountIdentifiedWithSSL, @@ -18865,16 +18887,16 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1101: + case 1103: yyDollar = yyS[yypt-1 : yypt+1] -//line mysql_sql.y:7195 +//line mysql_sql.y:7210 { yyVAL.str = yyDollar[1].cstrUnion().Compare() } - case 1103: + case 1105: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:7202 +//line mysql_sql.y:7217 { var IfNotExists = yyDollar[3].ifNotExistsUnion() var Roles = yyDollar[4].rolesUnion() @@ -18884,26 +18906,26 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1104: + case 1106: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []*tree.Role -//line mysql_sql.y:7213 +//line mysql_sql.y:7228 { yyLOCAL = []*tree.Role{yyDollar[1].roleUnion()} } yyVAL.union = yyLOCAL - case 1105: + case 1107: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL []*tree.Role -//line mysql_sql.y:7217 +//line mysql_sql.y:7232 { yyLOCAL = append(yyDollar[1].rolesUnion(), yyDollar[3].roleUnion()) } yyVAL.union = yyLOCAL - case 1106: + case 1108: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.Role -//line mysql_sql.y:7223 +//line mysql_sql.y:7238 { var UserName = yyDollar[1].cstrUnion().Compare() yyLOCAL = tree.NewRole( @@ -18911,66 +18933,66 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1107: + case 1109: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.CStr -//line mysql_sql.y:7232 +//line mysql_sql.y:7247 { yyLOCAL = tree.NewCStr(yyDollar[1].str, 1) } yyVAL.union = yyLOCAL - case 1108: + case 1110: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.CStr -//line mysql_sql.y:7236 +//line mysql_sql.y:7251 { yyLOCAL = tree.NewCStr(yyDollar[1].str, 1) } yyVAL.union = yyLOCAL - case 1109: + case 1111: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.CStr -//line mysql_sql.y:7240 +//line mysql_sql.y:7255 { yyLOCAL = tree.NewCStr(yyDollar[1].str, 1) } yyVAL.union = yyLOCAL - case 1110: + case 1112: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL tree.IndexCategory -//line mysql_sql.y:7245 +//line mysql_sql.y:7260 { yyLOCAL = tree.INDEX_CATEGORY_NONE } yyVAL.union = yyLOCAL - case 1111: + case 1113: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.IndexCategory -//line mysql_sql.y:7249 +//line mysql_sql.y:7264 { yyLOCAL = tree.INDEX_CATEGORY_FULLTEXT } yyVAL.union = yyLOCAL - case 1112: + case 1114: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.IndexCategory -//line mysql_sql.y:7253 +//line mysql_sql.y:7268 { yyLOCAL = tree.INDEX_CATEGORY_SPATIAL } yyVAL.union = yyLOCAL - case 1113: + case 1115: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.IndexCategory -//line mysql_sql.y:7257 +//line mysql_sql.y:7272 { yyLOCAL = tree.INDEX_CATEGORY_UNIQUE } yyVAL.union = yyLOCAL - case 1114: + case 1116: yyDollar = yyS[yypt-11 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:7263 +//line mysql_sql.y:7278 { var io *tree.IndexOption = nil if yyDollar[11].indexOptionUnion() == nil && yyDollar[5].indexTypeUnion() != tree.INDEX_TYPE_INVALID { @@ -19001,18 +19023,18 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1115: + case 1117: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *tree.IndexOption -//line mysql_sql.y:7294 +//line mysql_sql.y:7309 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1116: + case 1118: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.IndexOption -//line mysql_sql.y:7298 +//line mysql_sql.y:7313 { // Merge the options if yyDollar[1].indexOptionUnion() == nil { @@ -19037,20 +19059,20 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1117: + case 1119: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *tree.IndexOption -//line mysql_sql.y:7324 +//line mysql_sql.y:7339 { io := tree.NewIndexOption() io.KeyBlockSize = uint64(yyDollar[3].item.(int64)) yyLOCAL = io } yyVAL.union = yyLOCAL - case 1118: + case 1120: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *tree.IndexOption -//line mysql_sql.y:7330 +//line mysql_sql.y:7345 { val := int64(yyDollar[3].item.(int64)) if val <= 0 { @@ -19063,76 +19085,76 @@ yydefault: yyLOCAL = io } yyVAL.union = yyLOCAL - case 1119: + case 1121: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.IndexOption -//line mysql_sql.y:7342 +//line mysql_sql.y:7357 { io := tree.NewIndexOption() io.AlgoParamVectorOpType = yyDollar[2].str yyLOCAL = io } yyVAL.union = yyLOCAL - case 1120: + case 1122: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.IndexOption -//line mysql_sql.y:7348 +//line mysql_sql.y:7363 { io := tree.NewIndexOption() io.Comment = yyDollar[2].str yyLOCAL = io } yyVAL.union = yyLOCAL - case 1121: + case 1123: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *tree.IndexOption -//line mysql_sql.y:7354 +//line mysql_sql.y:7369 { io := tree.NewIndexOption() io.ParserName = yyDollar[3].cstrUnion().Compare() yyLOCAL = io } yyVAL.union = yyLOCAL - case 1122: + case 1124: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.IndexOption -//line mysql_sql.y:7360 +//line mysql_sql.y:7375 { io := tree.NewIndexOption() io.Visible = tree.VISIBLE_TYPE_VISIBLE yyLOCAL = io } yyVAL.union = yyLOCAL - case 1123: + case 1125: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.IndexOption -//line mysql_sql.y:7366 +//line mysql_sql.y:7381 { io := tree.NewIndexOption() io.Visible = tree.VISIBLE_TYPE_INVISIBLE yyLOCAL = io } yyVAL.union = yyLOCAL - case 1124: + case 1126: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []*tree.KeyPart -//line mysql_sql.y:7374 +//line mysql_sql.y:7389 { yyLOCAL = []*tree.KeyPart{yyDollar[1].keyPartUnion()} } yyVAL.union = yyLOCAL - case 1125: + case 1127: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL []*tree.KeyPart -//line mysql_sql.y:7378 +//line mysql_sql.y:7393 { yyLOCAL = append(yyDollar[1].keyPartsUnion(), yyDollar[3].keyPartUnion()) } yyVAL.union = yyLOCAL - case 1126: + case 1128: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *tree.KeyPart -//line mysql_sql.y:7384 +//line mysql_sql.y:7399 { // Order is parsed but just ignored as MySQL dtree. var ColName = yyDollar[1].unresolvedNameUnion() @@ -19147,10 +19169,10 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1127: + case 1129: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *tree.KeyPart -//line mysql_sql.y:7398 +//line mysql_sql.y:7413 { var ColName *tree.UnresolvedName var Length int @@ -19164,66 +19186,66 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1128: + case 1130: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL tree.IndexType -//line mysql_sql.y:7412 +//line mysql_sql.y:7427 { yyLOCAL = tree.INDEX_TYPE_INVALID } yyVAL.union = yyLOCAL - case 1129: + case 1131: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.IndexType -//line mysql_sql.y:7416 +//line mysql_sql.y:7431 { yyLOCAL = tree.INDEX_TYPE_BTREE } yyVAL.union = yyLOCAL - case 1130: + case 1132: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.IndexType -//line mysql_sql.y:7420 +//line mysql_sql.y:7435 { yyLOCAL = tree.INDEX_TYPE_IVFFLAT } yyVAL.union = yyLOCAL - case 1131: + case 1133: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.IndexType -//line mysql_sql.y:7424 +//line mysql_sql.y:7439 { yyLOCAL = tree.INDEX_TYPE_MASTER } yyVAL.union = yyLOCAL - case 1132: + case 1134: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.IndexType -//line mysql_sql.y:7428 +//line mysql_sql.y:7443 { yyLOCAL = tree.INDEX_TYPE_HASH } yyVAL.union = yyLOCAL - case 1133: + case 1135: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.IndexType -//line mysql_sql.y:7432 +//line mysql_sql.y:7447 { yyLOCAL = tree.INDEX_TYPE_RTREE } yyVAL.union = yyLOCAL - case 1134: + case 1136: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.IndexType -//line mysql_sql.y:7436 +//line mysql_sql.y:7451 { yyLOCAL = tree.INDEX_TYPE_BSI } yyVAL.union = yyLOCAL - case 1135: + case 1137: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:7442 +//line mysql_sql.y:7457 { var IfNotExists = yyDollar[3].ifNotExistsUnion() var Name = tree.Identifier(yyDollar[4].str) @@ -19237,76 +19259,76 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1136: + case 1138: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *tree.SubscriptionOption -//line mysql_sql.y:7457 +//line mysql_sql.y:7472 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1137: + case 1139: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *tree.SubscriptionOption -//line mysql_sql.y:7461 +//line mysql_sql.y:7476 { var From = tree.Identifier(yyDollar[2].str) var Publication = tree.Identifier(yyDollar[4].cstrUnion().Compare()) yyLOCAL = tree.NewSubscriptionOption(From, Publication) } yyVAL.union = yyLOCAL - case 1140: + case 1142: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL bool -//line mysql_sql.y:7472 +//line mysql_sql.y:7487 { yyLOCAL = false } yyVAL.union = yyLOCAL - case 1141: + case 1143: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL bool -//line mysql_sql.y:7476 +//line mysql_sql.y:7491 { yyLOCAL = true } yyVAL.union = yyLOCAL - case 1142: + case 1144: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL []tree.CreateOption -//line mysql_sql.y:7481 +//line mysql_sql.y:7496 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1143: + case 1145: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []tree.CreateOption -//line mysql_sql.y:7485 +//line mysql_sql.y:7500 { yyLOCAL = yyDollar[1].createOptionsUnion() } yyVAL.union = yyLOCAL - case 1144: + case 1146: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []tree.CreateOption -//line mysql_sql.y:7491 +//line mysql_sql.y:7506 { yyLOCAL = []tree.CreateOption{yyDollar[1].createOptionUnion()} } yyVAL.union = yyLOCAL - case 1145: + case 1147: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL []tree.CreateOption -//line mysql_sql.y:7495 +//line mysql_sql.y:7510 { yyLOCAL = append(yyDollar[1].createOptionsUnion(), yyDollar[2].createOptionUnion()) } yyVAL.union = yyLOCAL - case 1146: + case 1148: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.CreateOption -//line mysql_sql.y:7501 +//line mysql_sql.y:7516 { var IsDefault = yyDollar[1].defaultOptionalUnion() var Charset = yyDollar[4].str @@ -19316,10 +19338,10 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1147: + case 1149: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.CreateOption -//line mysql_sql.y:7510 +//line mysql_sql.y:7525 { var IsDefault = yyDollar[1].defaultOptionalUnion() var Collate = yyDollar[4].str @@ -19329,35 +19351,35 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1148: + case 1150: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.CreateOption -//line mysql_sql.y:7519 +//line mysql_sql.y:7534 { var Encrypt = yyDollar[4].str yyLOCAL = tree.NewCreateOptionEncryption(Encrypt) } yyVAL.union = yyLOCAL - case 1149: + case 1151: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL bool -//line mysql_sql.y:7525 +//line mysql_sql.y:7540 { yyLOCAL = false } yyVAL.union = yyLOCAL - case 1150: + case 1152: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL bool -//line mysql_sql.y:7529 +//line mysql_sql.y:7544 { yyLOCAL = true } yyVAL.union = yyLOCAL - case 1151: + case 1153: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:7535 +//line mysql_sql.y:7550 { var TableName = yyDollar[4].tableNameUnion() var Options = yyDollar[7].connectorOptionsUnion() @@ -19367,18 +19389,18 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1152: + case 1154: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:7546 +//line mysql_sql.y:7561 { yyLOCAL = &tree.ShowConnectors{} } yyVAL.union = yyLOCAL - case 1153: + case 1155: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:7552 +//line mysql_sql.y:7567 { var taskID uint64 switch v := yyDollar[4].item.(type) { @@ -19395,10 +19417,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1154: + case 1156: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:7570 +//line mysql_sql.y:7585 { var taskID uint64 switch v := yyDollar[4].item.(type) { @@ -19415,10 +19437,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1155: + case 1157: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:7588 +//line mysql_sql.y:7603 { var taskID uint64 switch v := yyDollar[4].item.(type) { @@ -19435,10 +19457,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1156: + case 1158: yyDollar = yyS[yypt-9 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:7606 +//line mysql_sql.y:7621 { var Replace = yyDollar[2].sourceOptionalUnion() var IfNotExists = yyDollar[4].ifNotExistsUnion() @@ -19454,26 +19476,26 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1157: + case 1159: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL bool -//line mysql_sql.y:7622 +//line mysql_sql.y:7637 { yyLOCAL = false } yyVAL.union = yyLOCAL - case 1158: + case 1160: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL bool -//line mysql_sql.y:7626 +//line mysql_sql.y:7641 { yyLOCAL = true } yyVAL.union = yyLOCAL - case 1159: + case 1161: yyDollar = yyS[yypt-11 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:7635 +//line mysql_sql.y:7650 { t := tree.NewCreateTable() t.Temporary = yyDollar[2].boolValUnion() @@ -19486,10 +19508,10 @@ yydefault: yyLOCAL = t } yyVAL.union = yyLOCAL - case 1160: + case 1162: yyDollar = yyS[yypt-9 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:7647 +//line mysql_sql.y:7662 { t := tree.NewCreateTable() t.IfNotExists = yyDollar[4].ifNotExistsUnion() @@ -19499,10 +19521,10 @@ yydefault: yyLOCAL = t } yyVAL.union = yyLOCAL - case 1161: + case 1163: yyDollar = yyS[yypt-11 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:7656 +//line mysql_sql.y:7671 { t := tree.NewCreateTable() t.IsClusterTable = true @@ -19515,10 +19537,10 @@ yydefault: yyLOCAL = t } yyVAL.union = yyLOCAL - case 1162: + case 1164: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:7668 +//line mysql_sql.y:7683 { t := tree.NewCreateTable() t.IsDynamicTable = true @@ -19529,10 +19551,10 @@ yydefault: yyLOCAL = t } yyVAL.union = yyLOCAL - case 1163: + case 1165: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:7678 +//line mysql_sql.y:7693 { t := tree.NewCreateTable() t.IsAsSelect = true @@ -19543,10 +19565,10 @@ yydefault: yyLOCAL = t } yyVAL.union = yyLOCAL - case 1164: + case 1166: yyDollar = yyS[yypt-9 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:7688 +//line mysql_sql.y:7703 { t := tree.NewCreateTable() t.IsAsSelect = true @@ -19558,10 +19580,10 @@ yydefault: yyLOCAL = t } yyVAL.union = yyLOCAL - case 1165: + case 1167: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:7699 +//line mysql_sql.y:7714 { t := tree.NewCreateTable() t.IsAsSelect = true @@ -19572,10 +19594,10 @@ yydefault: yyLOCAL = t } yyVAL.union = yyLOCAL - case 1166: + case 1168: yyDollar = yyS[yypt-10 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:7709 +//line mysql_sql.y:7724 { t := tree.NewCreateTable() t.IsAsSelect = true @@ -19587,10 +19609,10 @@ yydefault: yyLOCAL = t } yyVAL.union = yyLOCAL - case 1167: + case 1169: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:7720 +//line mysql_sql.y:7735 { t := tree.NewCreateTable() t.IsAsLike = true @@ -19599,10 +19621,10 @@ yydefault: yyLOCAL = t } yyVAL.union = yyLOCAL - case 1168: + case 1170: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:7728 +//line mysql_sql.y:7743 { t := tree.NewCreateTable() t.Temporary = yyDollar[2].boolValUnion() @@ -19612,19 +19634,19 @@ yydefault: yyLOCAL = t } yyVAL.union = yyLOCAL - case 1169: + case 1171: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.ExternParam -//line mysql_sql.y:7739 +//line mysql_sql.y:7754 { yyLOCAL = yyDollar[1].loadParamUnion() yyLOCAL.Tail = yyDollar[2].tailParamUnion() } yyVAL.union = yyLOCAL - case 1170: + case 1172: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.ExternParam -//line mysql_sql.y:7746 +//line mysql_sql.y:7761 { yyLOCAL = &tree.ExternParam{ ExParamConst: tree.ExParamConst{ @@ -19635,10 +19657,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1171: + case 1173: yyDollar = yyS[yypt-9 : yypt+1] var yyLOCAL *tree.ExternParam -//line mysql_sql.y:7756 +//line mysql_sql.y:7771 { yyLOCAL = &tree.ExternParam{ ExParamConst: tree.ExParamConst{ @@ -19652,10 +19674,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1172: + case 1174: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *tree.ExternParam -//line mysql_sql.y:7769 +//line mysql_sql.y:7784 { yyLOCAL = &tree.ExternParam{ ExParamConst: tree.ExParamConst{ @@ -19664,10 +19686,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1173: + case 1175: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *tree.ExternParam -//line mysql_sql.y:7777 +//line mysql_sql.y:7792 { yyLOCAL = &tree.ExternParam{ ExParamConst: tree.ExParamConst{ @@ -19677,10 +19699,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1174: + case 1176: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *tree.ExternParam -//line mysql_sql.y:7786 +//line mysql_sql.y:7801 { yyLOCAL = &tree.ExternParam{ ExParamConst: tree.ExParamConst{ @@ -19689,55 +19711,55 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1175: + case 1177: yyDollar = yyS[yypt-0 : yypt+1] -//line mysql_sql.y:7795 +//line mysql_sql.y:7810 { yyVAL.str = "" } - case 1176: + case 1178: yyDollar = yyS[yypt-4 : yypt+1] -//line mysql_sql.y:7799 +//line mysql_sql.y:7814 { yyVAL.str = yyDollar[4].str } - case 1177: + case 1179: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []string -//line mysql_sql.y:7805 +//line mysql_sql.y:7820 { yyLOCAL = yyDollar[1].strsUnion() } yyVAL.union = yyLOCAL - case 1178: + case 1180: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL []string -//line mysql_sql.y:7809 +//line mysql_sql.y:7824 { yyLOCAL = append(yyDollar[1].strsUnion(), yyDollar[3].strsUnion()...) } yyVAL.union = yyLOCAL - case 1179: + case 1181: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL []string -//line mysql_sql.y:7814 +//line mysql_sql.y:7829 { yyLOCAL = []string{} } yyVAL.union = yyLOCAL - case 1180: + case 1182: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL []string -//line mysql_sql.y:7818 +//line mysql_sql.y:7833 { yyLOCAL = append(yyLOCAL, yyDollar[1].str) yyLOCAL = append(yyLOCAL, yyDollar[3].str) } yyVAL.union = yyLOCAL - case 1181: + case 1183: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *tree.TailParameter -//line mysql_sql.y:7825 +//line mysql_sql.y:7840 { yyLOCAL = &tree.TailParameter{ Charset: yyDollar[1].str, @@ -19749,22 +19771,22 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1182: + case 1184: yyDollar = yyS[yypt-0 : yypt+1] -//line mysql_sql.y:7837 +//line mysql_sql.y:7852 { yyVAL.str = "" } - case 1183: + case 1185: yyDollar = yyS[yypt-2 : yypt+1] -//line mysql_sql.y:7841 +//line mysql_sql.y:7856 { yyVAL.str = yyDollar[2].str } - case 1184: + case 1186: yyDollar = yyS[yypt-10 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:7847 +//line mysql_sql.y:7862 { var Name = yyDollar[4].tableNameUnion() var Type = yyDollar[5].columnTypeUnion() @@ -19786,10 +19808,10 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1185: + case 1187: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:7868 +//line mysql_sql.y:7883 { locale := "" fstr := "bigint" @@ -19804,44 +19826,44 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1186: + case 1188: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:7882 +//line mysql_sql.y:7897 { yyLOCAL = yyDollar[2].columnTypeUnion() } yyVAL.union = yyLOCAL - case 1187: + case 1189: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *tree.TypeOption -//line mysql_sql.y:7886 +//line mysql_sql.y:7901 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1188: + case 1190: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.TypeOption -//line mysql_sql.y:7890 +//line mysql_sql.y:7905 { yyLOCAL = &tree.TypeOption{ Type: yyDollar[2].columnTypeUnion(), } } yyVAL.union = yyLOCAL - case 1189: + case 1191: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *tree.IncrementByOption -//line mysql_sql.y:7896 +//line mysql_sql.y:7911 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1190: + case 1192: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *tree.IncrementByOption -//line mysql_sql.y:7900 +//line mysql_sql.y:7915 { yyLOCAL = &tree.IncrementByOption{ Minus: false, @@ -19849,10 +19871,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1191: + case 1193: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.IncrementByOption -//line mysql_sql.y:7907 +//line mysql_sql.y:7922 { yyLOCAL = &tree.IncrementByOption{ Minus: false, @@ -19860,10 +19882,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1192: + case 1194: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *tree.IncrementByOption -//line mysql_sql.y:7914 +//line mysql_sql.y:7929 { yyLOCAL = &tree.IncrementByOption{ Minus: true, @@ -19871,10 +19893,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1193: + case 1195: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *tree.IncrementByOption -//line mysql_sql.y:7921 +//line mysql_sql.y:7936 { yyLOCAL = &tree.IncrementByOption{ Minus: true, @@ -19882,42 +19904,42 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1194: + case 1196: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL bool -//line mysql_sql.y:7928 +//line mysql_sql.y:7943 { yyLOCAL = false } yyVAL.union = yyLOCAL - case 1195: + case 1197: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL bool -//line mysql_sql.y:7932 +//line mysql_sql.y:7947 { yyLOCAL = false } yyVAL.union = yyLOCAL - case 1196: + case 1198: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL bool -//line mysql_sql.y:7936 +//line mysql_sql.y:7951 { yyLOCAL = true } yyVAL.union = yyLOCAL - case 1197: + case 1199: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *tree.MinValueOption -//line mysql_sql.y:7940 +//line mysql_sql.y:7955 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1198: + case 1200: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.MinValueOption -//line mysql_sql.y:7944 +//line mysql_sql.y:7959 { yyLOCAL = &tree.MinValueOption{ Minus: false, @@ -19925,10 +19947,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1199: + case 1201: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *tree.MinValueOption -//line mysql_sql.y:7951 +//line mysql_sql.y:7966 { yyLOCAL = &tree.MinValueOption{ Minus: true, @@ -19936,18 +19958,18 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1200: + case 1202: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *tree.MaxValueOption -//line mysql_sql.y:7958 +//line mysql_sql.y:7973 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1201: + case 1203: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.MaxValueOption -//line mysql_sql.y:7962 +//line mysql_sql.y:7977 { yyLOCAL = &tree.MaxValueOption{ Minus: false, @@ -19955,10 +19977,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1202: + case 1204: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *tree.MaxValueOption -//line mysql_sql.y:7969 +//line mysql_sql.y:7984 { yyLOCAL = &tree.MaxValueOption{ Minus: true, @@ -19966,46 +19988,46 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1203: + case 1205: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *tree.CycleOption -//line mysql_sql.y:7976 +//line mysql_sql.y:7991 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1204: + case 1206: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.CycleOption -//line mysql_sql.y:7980 +//line mysql_sql.y:7995 { yyLOCAL = &tree.CycleOption{ Cycle: false, } } yyVAL.union = yyLOCAL - case 1205: + case 1207: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.CycleOption -//line mysql_sql.y:7986 +//line mysql_sql.y:8001 { yyLOCAL = &tree.CycleOption{ Cycle: true, } } yyVAL.union = yyLOCAL - case 1206: + case 1208: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *tree.StartWithOption -//line mysql_sql.y:7992 +//line mysql_sql.y:8007 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1207: + case 1209: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *tree.StartWithOption -//line mysql_sql.y:7996 +//line mysql_sql.y:8011 { yyLOCAL = &tree.StartWithOption{ Minus: false, @@ -20013,10 +20035,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1208: + case 1210: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.StartWithOption -//line mysql_sql.y:8003 +//line mysql_sql.y:8018 { yyLOCAL = &tree.StartWithOption{ Minus: false, @@ -20024,10 +20046,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1209: + case 1211: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *tree.StartWithOption -//line mysql_sql.y:8010 +//line mysql_sql.y:8025 { yyLOCAL = &tree.StartWithOption{ Minus: true, @@ -20035,10 +20057,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1210: + case 1212: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *tree.StartWithOption -//line mysql_sql.y:8017 +//line mysql_sql.y:8032 { yyLOCAL = &tree.StartWithOption{ Minus: true, @@ -20046,58 +20068,58 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1211: + case 1213: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL bool -//line mysql_sql.y:8024 +//line mysql_sql.y:8039 { yyLOCAL = false } yyVAL.union = yyLOCAL - case 1212: + case 1214: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL bool -//line mysql_sql.y:8028 +//line mysql_sql.y:8043 { yyLOCAL = true } yyVAL.union = yyLOCAL - case 1213: + case 1215: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL bool -//line mysql_sql.y:8033 +//line mysql_sql.y:8048 { yyLOCAL = true } yyVAL.union = yyLOCAL - case 1214: + case 1216: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL bool -//line mysql_sql.y:8037 +//line mysql_sql.y:8052 { yyLOCAL = true } yyVAL.union = yyLOCAL - case 1215: + case 1217: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL bool -//line mysql_sql.y:8041 +//line mysql_sql.y:8056 { yyLOCAL = true } yyVAL.union = yyLOCAL - case 1216: + case 1218: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *tree.PartitionOption -//line mysql_sql.y:8046 +//line mysql_sql.y:8061 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1217: + case 1219: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *tree.PartitionOption -//line mysql_sql.y:8050 +//line mysql_sql.y:8065 { yyDollar[3].partitionByUnion().Num = uint64(yyDollar[4].int64ValUnion()) var PartBy = yyDollar[3].partitionByUnion() @@ -20110,18 +20132,18 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1218: + case 1220: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *tree.ClusterByOption -//line mysql_sql.y:8063 +//line mysql_sql.y:8078 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1219: + case 1221: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *tree.ClusterByOption -//line mysql_sql.y:8067 +//line mysql_sql.y:8082 { var ColumnList = []*tree.UnresolvedName{yyDollar[3].unresolvedNameUnion()} yyLOCAL = tree.NewClusterByOption( @@ -20130,10 +20152,10 @@ yydefault: } yyVAL.union = yyLOCAL - case 1220: + case 1222: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *tree.ClusterByOption -//line mysql_sql.y:8075 +//line mysql_sql.y:8090 { var ColumnList = yyDollar[4].unresolveNamesUnion() yyLOCAL = tree.NewClusterByOption( @@ -20141,18 +20163,18 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1221: + case 1223: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *tree.PartitionBy -//line mysql_sql.y:8083 +//line mysql_sql.y:8098 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1222: + case 1224: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *tree.PartitionBy -//line mysql_sql.y:8087 +//line mysql_sql.y:8102 { var IsSubPartition = true var PType = yyDollar[3].partitionByUnion() @@ -20164,42 +20186,42 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1223: + case 1225: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL []*tree.Partition -//line mysql_sql.y:8099 +//line mysql_sql.y:8114 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1224: + case 1226: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL []*tree.Partition -//line mysql_sql.y:8103 +//line mysql_sql.y:8118 { yyLOCAL = yyDollar[2].partitionsUnion() } yyVAL.union = yyLOCAL - case 1225: + case 1227: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []*tree.Partition -//line mysql_sql.y:8109 +//line mysql_sql.y:8124 { yyLOCAL = []*tree.Partition{yyDollar[1].partitionUnion()} } yyVAL.union = yyLOCAL - case 1226: + case 1228: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL []*tree.Partition -//line mysql_sql.y:8113 +//line mysql_sql.y:8128 { yyLOCAL = append(yyDollar[1].partitionsUnion(), yyDollar[3].partitionUnion()) } yyVAL.union = yyLOCAL - case 1227: + case 1229: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *tree.Partition -//line mysql_sql.y:8119 +//line mysql_sql.y:8134 { var Name = tree.Identifier(yyDollar[2].cstrUnion().Compare()) var Values = yyDollar[3].valuesUnion() @@ -20213,10 +20235,10 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1228: + case 1230: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *tree.Partition -//line mysql_sql.y:8132 +//line mysql_sql.y:8147 { var Name = tree.Identifier(yyDollar[2].cstrUnion().Compare()) var Values = yyDollar[3].valuesUnion() @@ -20230,42 +20252,42 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1229: + case 1231: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL []*tree.SubPartition -//line mysql_sql.y:8146 +//line mysql_sql.y:8161 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1230: + case 1232: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL []*tree.SubPartition -//line mysql_sql.y:8150 +//line mysql_sql.y:8165 { yyLOCAL = yyDollar[2].subPartitionsUnion() } yyVAL.union = yyLOCAL - case 1231: + case 1233: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []*tree.SubPartition -//line mysql_sql.y:8156 +//line mysql_sql.y:8171 { yyLOCAL = []*tree.SubPartition{yyDollar[1].subPartitionUnion()} } yyVAL.union = yyLOCAL - case 1232: + case 1234: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL []*tree.SubPartition -//line mysql_sql.y:8160 +//line mysql_sql.y:8175 { yyLOCAL = append(yyDollar[1].subPartitionsUnion(), yyDollar[3].subPartitionUnion()) } yyVAL.union = yyLOCAL - case 1233: + case 1235: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.SubPartition -//line mysql_sql.y:8166 +//line mysql_sql.y:8181 { var Name = tree.Identifier(yyDollar[2].cstrUnion().Compare()) var Options []tree.TableOption @@ -20275,10 +20297,10 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1234: + case 1236: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *tree.SubPartition -//line mysql_sql.y:8175 +//line mysql_sql.y:8190 { var Name = tree.Identifier(yyDollar[2].cstrUnion().Compare()) var Options = yyDollar[3].tableOptionsUnion() @@ -20288,53 +20310,53 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1235: + case 1237: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []tree.TableOption -//line mysql_sql.y:8186 +//line mysql_sql.y:8201 { yyLOCAL = []tree.TableOption{yyDollar[1].tableOptionUnion()} } yyVAL.union = yyLOCAL - case 1236: + case 1238: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL []tree.TableOption -//line mysql_sql.y:8190 +//line mysql_sql.y:8205 { yyLOCAL = append(yyDollar[1].tableOptionsUnion(), yyDollar[2].tableOptionUnion()) } yyVAL.union = yyLOCAL - case 1237: + case 1239: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL tree.Values -//line mysql_sql.y:8195 +//line mysql_sql.y:8210 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1238: + case 1240: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.Values -//line mysql_sql.y:8199 +//line mysql_sql.y:8214 { expr := tree.NewMaxValue() var valueList = tree.Exprs{expr} yyLOCAL = tree.NewValuesLessThan(valueList) } yyVAL.union = yyLOCAL - case 1239: + case 1241: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL tree.Values -//line mysql_sql.y:8205 +//line mysql_sql.y:8220 { var valueList = yyDollar[5].exprsUnion() yyLOCAL = tree.NewValuesLessThan(valueList) } yyVAL.union = yyLOCAL - case 1240: + case 1242: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL tree.Values -//line mysql_sql.y:8210 +//line mysql_sql.y:8225 { var valueList = yyDollar[4].exprsUnion() yyLOCAL = tree.NewValuesIn( @@ -20342,18 +20364,18 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1241: + case 1243: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL int64 -//line mysql_sql.y:8218 +//line mysql_sql.y:8233 { yyLOCAL = 0 } yyVAL.union = yyLOCAL - case 1242: + case 1244: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL int64 -//line mysql_sql.y:8222 +//line mysql_sql.y:8237 { res := yyDollar[2].item.(int64) if res == 0 { @@ -20363,18 +20385,18 @@ yydefault: yyLOCAL = res } yyVAL.union = yyLOCAL - case 1243: + case 1245: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL int64 -//line mysql_sql.y:8232 +//line mysql_sql.y:8247 { yyLOCAL = 0 } yyVAL.union = yyLOCAL - case 1244: + case 1246: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL int64 -//line mysql_sql.y:8236 +//line mysql_sql.y:8251 { res := yyDollar[2].item.(int64) if res == 0 { @@ -20384,10 +20406,10 @@ yydefault: yyLOCAL = res } yyVAL.union = yyLOCAL - case 1245: + case 1247: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *tree.PartitionBy -//line mysql_sql.y:8247 +//line mysql_sql.y:8262 { rangeTyp := tree.NewRangeType() rangeTyp.Expr = yyDollar[3].exprUnion() @@ -20396,10 +20418,10 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1246: + case 1248: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *tree.PartitionBy -//line mysql_sql.y:8255 +//line mysql_sql.y:8270 { rangeTyp := tree.NewRangeType() rangeTyp.ColumnList = yyDollar[4].unresolveNamesUnion() @@ -20408,10 +20430,10 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1247: + case 1249: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *tree.PartitionBy -//line mysql_sql.y:8263 +//line mysql_sql.y:8278 { listTyp := tree.NewListType() listTyp.Expr = yyDollar[3].exprUnion() @@ -20420,10 +20442,10 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1248: + case 1250: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *tree.PartitionBy -//line mysql_sql.y:8271 +//line mysql_sql.y:8286 { listTyp := tree.NewListType() listTyp.ColumnList = yyDollar[4].unresolveNamesUnion() @@ -20432,10 +20454,10 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1250: + case 1252: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *tree.PartitionBy -//line mysql_sql.y:8282 +//line mysql_sql.y:8297 { keyTyp := tree.NewKeyType() keyTyp.Linear = yyDollar[1].boolValUnion() @@ -20445,10 +20467,10 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1251: + case 1253: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *tree.PartitionBy -//line mysql_sql.y:8291 +//line mysql_sql.y:8306 { keyTyp := tree.NewKeyType() keyTyp.Linear = yyDollar[1].boolValUnion() @@ -20459,10 +20481,10 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1252: + case 1254: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *tree.PartitionBy -//line mysql_sql.y:8301 +//line mysql_sql.y:8316 { Linear := yyDollar[1].boolValUnion() Expr := yyDollar[4].exprUnion() @@ -20472,58 +20494,58 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1253: + case 1255: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL int64 -//line mysql_sql.y:8311 +//line mysql_sql.y:8326 { yyLOCAL = 2 } yyVAL.union = yyLOCAL - case 1254: + case 1256: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL int64 -//line mysql_sql.y:8315 +//line mysql_sql.y:8330 { yyLOCAL = yyDollar[3].item.(int64) } yyVAL.union = yyLOCAL - case 1255: + case 1257: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL bool -//line mysql_sql.y:8320 +//line mysql_sql.y:8335 { yyLOCAL = false } yyVAL.union = yyLOCAL - case 1256: + case 1258: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL bool -//line mysql_sql.y:8324 +//line mysql_sql.y:8339 { yyLOCAL = true } yyVAL.union = yyLOCAL - case 1257: + case 1259: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []*tree.ConnectorOption -//line mysql_sql.y:8330 +//line mysql_sql.y:8345 { yyLOCAL = []*tree.ConnectorOption{yyDollar[1].connectorOptionUnion()} } yyVAL.union = yyLOCAL - case 1258: + case 1260: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL []*tree.ConnectorOption -//line mysql_sql.y:8334 +//line mysql_sql.y:8349 { yyLOCAL = append(yyDollar[1].connectorOptionsUnion(), yyDollar[3].connectorOptionUnion()) } yyVAL.union = yyLOCAL - case 1259: + case 1261: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *tree.ConnectorOption -//line mysql_sql.y:8340 +//line mysql_sql.y:8355 { var Key = tree.Identifier(yyDollar[1].cstrUnion().Compare()) var Val = yyDollar[3].exprUnion() @@ -20533,10 +20555,10 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1260: + case 1262: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *tree.ConnectorOption -//line mysql_sql.y:8349 +//line mysql_sql.y:8364 { var Key = tree.Identifier(yyDollar[1].str) var Val = yyDollar[3].exprUnion() @@ -20546,42 +20568,42 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1261: + case 1263: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL []tree.TableOption -//line mysql_sql.y:8359 +//line mysql_sql.y:8374 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1262: + case 1264: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL []tree.TableOption -//line mysql_sql.y:8363 +//line mysql_sql.y:8378 { yyLOCAL = yyDollar[3].tableOptionsUnion() } yyVAL.union = yyLOCAL - case 1263: + case 1265: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []tree.TableOption -//line mysql_sql.y:8369 +//line mysql_sql.y:8384 { yyLOCAL = []tree.TableOption{yyDollar[1].tableOptionUnion()} } yyVAL.union = yyLOCAL - case 1264: + case 1266: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL []tree.TableOption -//line mysql_sql.y:8373 +//line mysql_sql.y:8388 { yyLOCAL = append(yyDollar[1].tableOptionsUnion(), yyDollar[3].tableOptionUnion()) } yyVAL.union = yyLOCAL - case 1265: + case 1267: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8379 +//line mysql_sql.y:8394 { var Key = tree.Identifier(yyDollar[1].cstrUnion().Compare()) var Val = yyDollar[3].exprUnion() @@ -20591,10 +20613,10 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1266: + case 1268: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8388 +//line mysql_sql.y:8403 { var Key = tree.Identifier(yyDollar[1].str) var Val = yyDollar[3].exprUnion() @@ -20604,348 +20626,348 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1267: + case 1269: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL []tree.TableOption -//line mysql_sql.y:8398 +//line mysql_sql.y:8413 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1268: + case 1270: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []tree.TableOption -//line mysql_sql.y:8402 +//line mysql_sql.y:8417 { yyLOCAL = yyDollar[1].tableOptionsUnion() } yyVAL.union = yyLOCAL - case 1269: + case 1271: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []tree.TableOption -//line mysql_sql.y:8408 +//line mysql_sql.y:8423 { yyLOCAL = []tree.TableOption{yyDollar[1].tableOptionUnion()} } yyVAL.union = yyLOCAL - case 1270: + case 1272: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL []tree.TableOption -//line mysql_sql.y:8412 +//line mysql_sql.y:8427 { yyLOCAL = append(yyDollar[1].tableOptionsUnion(), yyDollar[3].tableOptionUnion()) } yyVAL.union = yyLOCAL - case 1271: + case 1273: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL []tree.TableOption -//line mysql_sql.y:8416 +//line mysql_sql.y:8431 { yyLOCAL = append(yyDollar[1].tableOptionsUnion(), yyDollar[2].tableOptionUnion()) } yyVAL.union = yyLOCAL - case 1272: + case 1274: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8422 +//line mysql_sql.y:8437 { yyLOCAL = tree.NewTableOptionAUTOEXTEND_SIZE(uint64(yyDollar[3].item.(int64))) } yyVAL.union = yyLOCAL - case 1273: + case 1275: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8426 +//line mysql_sql.y:8441 { yyLOCAL = tree.NewTableOptionAutoIncrement(uint64(yyDollar[3].item.(int64))) } yyVAL.union = yyLOCAL - case 1274: + case 1276: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8430 +//line mysql_sql.y:8445 { yyLOCAL = tree.NewTableOptionAvgRowLength(uint64(yyDollar[3].item.(int64))) } yyVAL.union = yyLOCAL - case 1275: + case 1277: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8434 +//line mysql_sql.y:8449 { yyLOCAL = tree.NewTableOptionCharset(yyDollar[4].str) } yyVAL.union = yyLOCAL - case 1276: + case 1278: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8438 +//line mysql_sql.y:8453 { yyLOCAL = tree.NewTableOptionCollate(yyDollar[4].str) } yyVAL.union = yyLOCAL - case 1277: + case 1279: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8442 +//line mysql_sql.y:8457 { yyLOCAL = tree.NewTableOptionChecksum(uint64(yyDollar[3].item.(int64))) } yyVAL.union = yyLOCAL - case 1278: + case 1280: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8446 +//line mysql_sql.y:8461 { str := util.DealCommentString(yyDollar[3].str) yyLOCAL = tree.NewTableOptionComment(str) } yyVAL.union = yyLOCAL - case 1279: + case 1281: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8451 +//line mysql_sql.y:8466 { yyLOCAL = tree.NewTableOptionCompression(yyDollar[3].str) } yyVAL.union = yyLOCAL - case 1280: + case 1282: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8455 +//line mysql_sql.y:8470 { yyLOCAL = tree.NewTableOptionConnection(yyDollar[3].str) } yyVAL.union = yyLOCAL - case 1281: + case 1283: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8459 +//line mysql_sql.y:8474 { yyLOCAL = tree.NewTableOptionDataDirectory(yyDollar[4].str) } yyVAL.union = yyLOCAL - case 1282: + case 1284: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8463 +//line mysql_sql.y:8478 { yyLOCAL = tree.NewTableOptionIndexDirectory(yyDollar[4].str) } yyVAL.union = yyLOCAL - case 1283: + case 1285: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8467 +//line mysql_sql.y:8482 { yyLOCAL = tree.NewTableOptionDelayKeyWrite(uint64(yyDollar[3].item.(int64))) } yyVAL.union = yyLOCAL - case 1284: + case 1286: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8471 +//line mysql_sql.y:8486 { yyLOCAL = tree.NewTableOptionEncryption(yyDollar[3].str) } yyVAL.union = yyLOCAL - case 1285: + case 1287: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8475 +//line mysql_sql.y:8490 { yyLOCAL = tree.NewTableOptionEngine(yyDollar[3].str) } yyVAL.union = yyLOCAL - case 1286: + case 1288: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8479 +//line mysql_sql.y:8494 { yyLOCAL = tree.NewTableOptionEngineAttr(yyDollar[3].str) } yyVAL.union = yyLOCAL - case 1287: + case 1289: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8483 +//line mysql_sql.y:8498 { yyLOCAL = tree.NewTableOptionInsertMethod(yyDollar[3].str) } yyVAL.union = yyLOCAL - case 1288: + case 1290: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8487 +//line mysql_sql.y:8502 { yyLOCAL = tree.NewTableOptionKeyBlockSize(uint64(yyDollar[3].item.(int64))) } yyVAL.union = yyLOCAL - case 1289: + case 1291: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8491 +//line mysql_sql.y:8506 { yyLOCAL = tree.NewTableOptionMaxRows(uint64(yyDollar[3].item.(int64))) } yyVAL.union = yyLOCAL - case 1290: + case 1292: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8495 +//line mysql_sql.y:8510 { yyLOCAL = tree.NewTableOptionMinRows(uint64(yyDollar[3].item.(int64))) } yyVAL.union = yyLOCAL - case 1291: + case 1293: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8499 +//line mysql_sql.y:8514 { t := tree.NewTableOptionPackKeys() t.Value = yyDollar[3].item.(int64) yyLOCAL = t } yyVAL.union = yyLOCAL - case 1292: + case 1294: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8505 +//line mysql_sql.y:8520 { t := tree.NewTableOptionPackKeys() t.Default = true yyLOCAL = t } yyVAL.union = yyLOCAL - case 1293: + case 1295: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8511 +//line mysql_sql.y:8526 { yyLOCAL = tree.NewTableOptionPassword(yyDollar[3].str) } yyVAL.union = yyLOCAL - case 1294: + case 1296: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8515 +//line mysql_sql.y:8530 { yyLOCAL = tree.NewTableOptionRowFormat(yyDollar[3].rowFormatTypeUnion()) } yyVAL.union = yyLOCAL - case 1295: + case 1297: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8519 +//line mysql_sql.y:8534 { yyLOCAL = tree.NewTTableOptionStartTrans(true) } yyVAL.union = yyLOCAL - case 1296: + case 1298: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8523 +//line mysql_sql.y:8538 { yyLOCAL = tree.NewTTableOptionSecondaryEngineAttr(yyDollar[3].str) } yyVAL.union = yyLOCAL - case 1297: + case 1299: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8527 +//line mysql_sql.y:8542 { t := tree.NewTableOptionStatsAutoRecalc() t.Value = uint64(yyDollar[3].item.(int64)) yyLOCAL = t } yyVAL.union = yyLOCAL - case 1298: + case 1300: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8533 +//line mysql_sql.y:8548 { t := tree.NewTableOptionStatsAutoRecalc() t.Default = true yyLOCAL = t } yyVAL.union = yyLOCAL - case 1299: + case 1301: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8539 +//line mysql_sql.y:8554 { t := tree.NewTableOptionStatsPersistent() t.Value = uint64(yyDollar[3].item.(int64)) yyLOCAL = t } yyVAL.union = yyLOCAL - case 1300: + case 1302: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8545 +//line mysql_sql.y:8560 { t := tree.NewTableOptionStatsPersistent() t.Default = true yyLOCAL = t } yyVAL.union = yyLOCAL - case 1301: + case 1303: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8551 +//line mysql_sql.y:8566 { t := tree.NewTableOptionStatsSamplePages() t.Value = uint64(yyDollar[3].item.(int64)) yyLOCAL = t } yyVAL.union = yyLOCAL - case 1302: + case 1304: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8557 +//line mysql_sql.y:8572 { t := tree.NewTableOptionStatsSamplePages() t.Default = true yyLOCAL = t } yyVAL.union = yyLOCAL - case 1303: + case 1305: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8563 +//line mysql_sql.y:8578 { yyLOCAL = tree.NewTableOptionTablespace(yyDollar[3].cstrUnion().Compare(), "") } yyVAL.union = yyLOCAL - case 1304: + case 1306: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8567 +//line mysql_sql.y:8582 { yyLOCAL = tree.NewTableOptionTablespace("", yyDollar[1].str) } yyVAL.union = yyLOCAL - case 1305: + case 1307: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8571 +//line mysql_sql.y:8586 { yyLOCAL = tree.NewTableOptionUnion(yyDollar[4].tableNamesUnion()) } yyVAL.union = yyLOCAL - case 1306: + case 1308: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8575 +//line mysql_sql.y:8590 { var Preperties = yyDollar[3].propertiesUnion() yyLOCAL = tree.NewTableOptionProperties(Preperties) } yyVAL.union = yyLOCAL - case 1307: + case 1309: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL tree.TableOption -//line mysql_sql.y:8580 +//line mysql_sql.y:8595 { var retentionPeriod = uint64(yyDollar[4].item.(int64)) var retentionUnit = strings.ToLower(yyDollar[5].str) @@ -20955,26 +20977,26 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1308: + case 1310: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []tree.Property -//line mysql_sql.y:8591 +//line mysql_sql.y:8606 { yyLOCAL = []tree.Property{yyDollar[1].propertyUnion()} } yyVAL.union = yyLOCAL - case 1309: + case 1311: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL []tree.Property -//line mysql_sql.y:8595 +//line mysql_sql.y:8610 { yyLOCAL = append(yyDollar[1].propertiesUnion(), yyDollar[3].propertyUnion()) } yyVAL.union = yyLOCAL - case 1310: + case 1312: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.Property -//line mysql_sql.y:8601 +//line mysql_sql.y:8616 { var Key = yyDollar[1].str var Value = yyDollar[3].str @@ -20984,96 +21006,96 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1311: + case 1313: yyDollar = yyS[yypt-2 : yypt+1] -//line mysql_sql.y:8612 +//line mysql_sql.y:8627 { yyVAL.str = " " + yyDollar[1].str + " " + yyDollar[2].str } - case 1312: + case 1314: yyDollar = yyS[yypt-2 : yypt+1] -//line mysql_sql.y:8616 +//line mysql_sql.y:8631 { yyVAL.str = " " + yyDollar[1].str + " " + yyDollar[2].str } - case 1313: + case 1315: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.RowFormatType -//line mysql_sql.y:8622 +//line mysql_sql.y:8637 { yyLOCAL = tree.ROW_FORMAT_DEFAULT } yyVAL.union = yyLOCAL - case 1314: + case 1316: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.RowFormatType -//line mysql_sql.y:8626 +//line mysql_sql.y:8641 { yyLOCAL = tree.ROW_FORMAT_DYNAMIC } yyVAL.union = yyLOCAL - case 1315: + case 1317: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.RowFormatType -//line mysql_sql.y:8630 +//line mysql_sql.y:8645 { yyLOCAL = tree.ROW_FORMAT_FIXED } yyVAL.union = yyLOCAL - case 1316: + case 1318: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.RowFormatType -//line mysql_sql.y:8634 +//line mysql_sql.y:8649 { yyLOCAL = tree.ROW_FORMAT_COMPRESSED } yyVAL.union = yyLOCAL - case 1317: + case 1319: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.RowFormatType -//line mysql_sql.y:8638 +//line mysql_sql.y:8653 { yyLOCAL = tree.ROW_FORMAT_REDUNDANT } yyVAL.union = yyLOCAL - case 1318: + case 1320: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.RowFormatType -//line mysql_sql.y:8642 +//line mysql_sql.y:8657 { yyLOCAL = tree.ROW_FORMAT_COMPACT } yyVAL.union = yyLOCAL - case 1323: + case 1325: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.TableNames -//line mysql_sql.y:8656 +//line mysql_sql.y:8671 { yyLOCAL = tree.TableNames{yyDollar[1].tableNameUnion()} } yyVAL.union = yyLOCAL - case 1324: + case 1326: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.TableNames -//line mysql_sql.y:8660 +//line mysql_sql.y:8675 { yyLOCAL = append(yyDollar[1].tableNamesUnion(), yyDollar[3].tableNameUnion()) } yyVAL.union = yyLOCAL - case 1325: + case 1327: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.TableName -//line mysql_sql.y:8669 +//line mysql_sql.y:8684 { tblName := yylex.(*Lexer).GetDbOrTblName(yyDollar[1].cstrUnion().Origin()) prefix := tree.ObjectNamePrefix{ExplicitSchema: false} yyLOCAL = tree.NewTableName(tree.Identifier(tblName), prefix, yyDollar[2].atTimeStampUnion()) } yyVAL.union = yyLOCAL - case 1326: + case 1328: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *tree.TableName -//line mysql_sql.y:8675 +//line mysql_sql.y:8690 { dbName := yylex.(*Lexer).GetDbOrTblName(yyDollar[1].cstrUnion().Origin()) tblName := yylex.(*Lexer).GetDbOrTblName(yyDollar[3].cstrUnion().Origin()) @@ -21081,18 +21103,18 @@ yydefault: yyLOCAL = tree.NewTableName(tree.Identifier(tblName), prefix, yyDollar[4].atTimeStampUnion()) } yyVAL.union = yyLOCAL - case 1327: + case 1329: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *tree.AtTimeStamp -//line mysql_sql.y:8683 +//line mysql_sql.y:8698 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1328: + case 1330: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *tree.AtTimeStamp -//line mysql_sql.y:8687 +//line mysql_sql.y:8702 { yyLOCAL = &tree.AtTimeStamp{ Type: tree.ATTIMESTAMPTIME, @@ -21100,10 +21122,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1329: + case 1331: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *tree.AtTimeStamp -//line mysql_sql.y:8694 +//line mysql_sql.y:8709 { var str = yyDollar[4].cstrUnion().Compare() yyLOCAL = &tree.AtTimeStamp{ @@ -21113,10 +21135,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1330: + case 1332: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *tree.AtTimeStamp -//line mysql_sql.y:8703 +//line mysql_sql.y:8718 { yyLOCAL = &tree.AtTimeStamp{ Type: tree.ATTIMESTAMPSNAPSHOT, @@ -21125,10 +21147,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1331: + case 1333: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *tree.AtTimeStamp -//line mysql_sql.y:8711 +//line mysql_sql.y:8726 { yyLOCAL = &tree.AtTimeStamp{ Type: tree.ATMOTIMESTAMP, @@ -21136,74 +21158,74 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1332: + case 1334: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL tree.TableDefs -//line mysql_sql.y:8719 +//line mysql_sql.y:8734 { yyLOCAL = tree.TableDefs(nil) } yyVAL.union = yyLOCAL - case 1334: + case 1336: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.TableDefs -//line mysql_sql.y:8726 +//line mysql_sql.y:8741 { yyLOCAL = tree.TableDefs{yyDollar[1].tableDefUnion()} } yyVAL.union = yyLOCAL - case 1335: + case 1337: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.TableDefs -//line mysql_sql.y:8730 +//line mysql_sql.y:8745 { yyLOCAL = append(yyDollar[1].tableDefsUnion(), yyDollar[3].tableDefUnion()) } yyVAL.union = yyLOCAL - case 1336: + case 1338: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.TableDef -//line mysql_sql.y:8736 +//line mysql_sql.y:8751 { yyLOCAL = tree.TableDef(yyDollar[1].columnTableDefUnion()) } yyVAL.union = yyLOCAL - case 1337: + case 1339: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.TableDef -//line mysql_sql.y:8740 +//line mysql_sql.y:8755 { yyLOCAL = yyDollar[1].tableDefUnion() } yyVAL.union = yyLOCAL - case 1338: + case 1340: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.TableDef -//line mysql_sql.y:8744 +//line mysql_sql.y:8759 { yyLOCAL = yyDollar[1].tableDefUnion() } yyVAL.union = yyLOCAL - case 1339: + case 1341: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.TableDef -//line mysql_sql.y:8750 +//line mysql_sql.y:8765 { yyLOCAL = yyDollar[1].tableDefUnion() } yyVAL.union = yyLOCAL - case 1340: + case 1342: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.TableDef -//line mysql_sql.y:8754 +//line mysql_sql.y:8769 { yyLOCAL = yyDollar[1].tableDefUnion() } yyVAL.union = yyLOCAL - case 1341: + case 1343: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL tree.TableDef -//line mysql_sql.y:8760 +//line mysql_sql.y:8775 { var KeyParts = yyDollar[5].keyPartsUnion() var Name = yyDollar[3].str @@ -21217,10 +21239,10 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1342: + case 1344: yyDollar = yyS[yypt-9 : yypt+1] var yyLOCAL tree.TableDef -//line mysql_sql.y:8773 +//line mysql_sql.y:8788 { var KeyParts = yyDollar[5].keyPartsUnion() var Name = yyDollar[3].str @@ -21234,10 +21256,10 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1343: + case 1345: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL tree.TableDef -//line mysql_sql.y:8786 +//line mysql_sql.y:8801 { keyTyp := tree.INDEX_TYPE_INVALID if yyDollar[3].strsUnion()[1] != "" { @@ -21277,10 +21299,10 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1344: + case 1346: yyDollar = yyS[yypt-9 : yypt+1] var yyLOCAL tree.TableDef -//line mysql_sql.y:8825 +//line mysql_sql.y:8840 { keyTyp := tree.INDEX_TYPE_INVALID if yyDollar[3].strsUnion()[1] != "" { @@ -21319,10 +21341,10 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1345: + case 1347: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.TableDef -//line mysql_sql.y:8865 +//line mysql_sql.y:8880 { if yyDollar[1].str != "" { switch v := yyDollar[2].tableDefUnion().(type) { @@ -21337,18 +21359,18 @@ yydefault: yyLOCAL = yyDollar[2].tableDefUnion() } yyVAL.union = yyLOCAL - case 1346: + case 1348: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.TableDef -//line mysql_sql.y:8879 +//line mysql_sql.y:8894 { yyLOCAL = yyDollar[1].tableDefUnion() } yyVAL.union = yyLOCAL - case 1347: + case 1349: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL tree.TableDef -//line mysql_sql.y:8885 +//line mysql_sql.y:8900 { var KeyParts = yyDollar[5].keyPartsUnion() var Name = yyDollar[3].strsUnion()[0] @@ -21362,10 +21384,10 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1348: + case 1350: yyDollar = yyS[yypt-9 : yypt+1] var yyLOCAL tree.TableDef -//line mysql_sql.y:8898 +//line mysql_sql.y:8913 { var KeyParts = yyDollar[5].keyPartsUnion() var Name = yyDollar[3].strsUnion()[0] @@ -21379,10 +21401,10 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1349: + case 1351: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL tree.TableDef -//line mysql_sql.y:8911 +//line mysql_sql.y:8926 { var KeyParts = yyDollar[5].keyPartsUnion() var Name = yyDollar[3].strsUnion()[0] @@ -21396,10 +21418,10 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1350: + case 1352: yyDollar = yyS[yypt-9 : yypt+1] var yyLOCAL tree.TableDef -//line mysql_sql.y:8924 +//line mysql_sql.y:8939 { var KeyParts = yyDollar[5].keyPartsUnion() var Name = yyDollar[3].strsUnion()[0] @@ -21413,10 +21435,10 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1351: + case 1353: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL tree.TableDef -//line mysql_sql.y:8937 +//line mysql_sql.y:8952 { var IfNotExists = yyDollar[3].ifNotExistsUnion() var KeyParts = yyDollar[6].keyPartsUnion() @@ -21432,10 +21454,10 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1352: + case 1354: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL tree.TableDef -//line mysql_sql.y:8952 +//line mysql_sql.y:8967 { var Expr = yyDollar[3].exprUnion() var Enforced = yyDollar[5].boolValUnion() @@ -21445,327 +21467,327 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1353: + case 1355: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL bool -//line mysql_sql.y:8962 +//line mysql_sql.y:8977 { yyLOCAL = false } yyVAL.union = yyLOCAL - case 1355: + case 1357: yyDollar = yyS[yypt-0 : yypt+1] -//line mysql_sql.y:8968 +//line mysql_sql.y:8983 { yyVAL.str = "" } - case 1356: + case 1358: yyDollar = yyS[yypt-1 : yypt+1] -//line mysql_sql.y:8972 +//line mysql_sql.y:8987 { yyVAL.str = yyDollar[1].str } - case 1359: + case 1361: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []string -//line mysql_sql.y:8982 +//line mysql_sql.y:8997 { yyLOCAL = make([]string, 2) yyLOCAL[0] = yyDollar[1].str yyLOCAL[1] = "" } yyVAL.union = yyLOCAL - case 1360: + case 1362: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL []string -//line mysql_sql.y:8988 +//line mysql_sql.y:9003 { yyLOCAL = make([]string, 2) yyLOCAL[0] = yyDollar[1].str yyLOCAL[1] = yyDollar[3].str } yyVAL.union = yyLOCAL - case 1361: + case 1363: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL []string -//line mysql_sql.y:8994 +//line mysql_sql.y:9009 { yyLOCAL = make([]string, 2) yyLOCAL[0] = yyDollar[1].cstrUnion().Compare() yyLOCAL[1] = yyDollar[3].str } yyVAL.union = yyLOCAL - case 1372: + case 1374: yyDollar = yyS[yypt-0 : yypt+1] -//line mysql_sql.y:9015 +//line mysql_sql.y:9030 { yyVAL.str = "" } - case 1373: + case 1375: yyDollar = yyS[yypt-1 : yypt+1] -//line mysql_sql.y:9019 +//line mysql_sql.y:9034 { yyVAL.str = yyDollar[1].cstrUnion().Compare() } - case 1374: + case 1376: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *tree.ColumnTableDef -//line mysql_sql.y:9025 +//line mysql_sql.y:9040 { yyLOCAL = tree.NewColumnTableDef(yyDollar[1].unresolvedNameUnion(), yyDollar[2].columnTypeUnion(), yyDollar[3].columnAttributesUnion()) } yyVAL.union = yyLOCAL - case 1375: + case 1377: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.UnresolvedName -//line mysql_sql.y:9031 +//line mysql_sql.y:9046 { yyLOCAL = tree.NewUnresolvedName(yyDollar[1].cstrUnion()) } yyVAL.union = yyLOCAL - case 1376: + case 1378: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *tree.UnresolvedName -//line mysql_sql.y:9035 +//line mysql_sql.y:9050 { tblNameCStr := yylex.(*Lexer).GetDbOrTblNameCStr(yyDollar[1].cstrUnion().Origin()) yyLOCAL = tree.NewUnresolvedName(tblNameCStr, yyDollar[3].cstrUnion()) } yyVAL.union = yyLOCAL - case 1377: + case 1379: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *tree.UnresolvedName -//line mysql_sql.y:9040 +//line mysql_sql.y:9055 { dbNameCStr := yylex.(*Lexer).GetDbOrTblNameCStr(yyDollar[1].cstrUnion().Origin()) tblNameCStr := yylex.(*Lexer).GetDbOrTblNameCStr(yyDollar[3].cstrUnion().Origin()) yyLOCAL = tree.NewUnresolvedName(dbNameCStr, tblNameCStr, yyDollar[5].cstrUnion()) } yyVAL.union = yyLOCAL - case 1378: + case 1380: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.CStr -//line mysql_sql.y:9048 +//line mysql_sql.y:9063 { yyLOCAL = tree.NewCStr(yyDollar[1].str, 1) } yyVAL.union = yyLOCAL - case 1379: + case 1381: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.CStr -//line mysql_sql.y:9052 +//line mysql_sql.y:9067 { yyLOCAL = tree.NewCStr(yyDollar[1].str, 1) } yyVAL.union = yyLOCAL - case 1380: + case 1382: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.CStr -//line mysql_sql.y:9056 +//line mysql_sql.y:9071 { yyLOCAL = tree.NewCStr(yyDollar[1].str, 1) } yyVAL.union = yyLOCAL - case 1381: + case 1383: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.CStr -//line mysql_sql.y:9060 +//line mysql_sql.y:9075 { yyLOCAL = tree.NewCStr(yyDollar[1].str, 1) } yyVAL.union = yyLOCAL - case 1382: + case 1384: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.CStr -//line mysql_sql.y:9066 +//line mysql_sql.y:9081 { yyLOCAL = yylex.(*Lexer).GetDbOrTblNameCStr(yyDollar[1].cstrUnion().Origin()) } yyVAL.union = yyLOCAL - case 1383: + case 1385: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.UnresolvedName -//line mysql_sql.y:9072 +//line mysql_sql.y:9087 { yyLOCAL = tree.NewUnresolvedName(yyDollar[1].cstrUnion()) } yyVAL.union = yyLOCAL - case 1384: + case 1386: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *tree.UnresolvedName -//line mysql_sql.y:9076 +//line mysql_sql.y:9091 { tblNameCStr := yylex.(*Lexer).GetDbOrTblNameCStr(yyDollar[1].cstrUnion().Origin()) yyLOCAL = tree.NewUnresolvedName(tblNameCStr, yyDollar[3].cstrUnion()) } yyVAL.union = yyLOCAL - case 1385: + case 1387: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *tree.UnresolvedName -//line mysql_sql.y:9081 +//line mysql_sql.y:9096 { dbNameCStr := yylex.(*Lexer).GetDbOrTblNameCStr(yyDollar[1].cstrUnion().Origin()) tblNameCStr := yylex.(*Lexer).GetDbOrTblNameCStr(yyDollar[3].cstrUnion().Origin()) yyLOCAL = tree.NewUnresolvedName(dbNameCStr, tblNameCStr, yyDollar[5].cstrUnion()) } yyVAL.union = yyLOCAL - case 1386: + case 1388: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL []tree.ColumnAttribute -//line mysql_sql.y:9088 +//line mysql_sql.y:9103 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1387: + case 1389: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []tree.ColumnAttribute -//line mysql_sql.y:9092 +//line mysql_sql.y:9107 { yyLOCAL = yyDollar[1].columnAttributesUnion() } yyVAL.union = yyLOCAL - case 1388: + case 1390: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []tree.ColumnAttribute -//line mysql_sql.y:9098 +//line mysql_sql.y:9113 { yyLOCAL = []tree.ColumnAttribute{yyDollar[1].columnAttributeUnion()} } yyVAL.union = yyLOCAL - case 1389: + case 1391: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL []tree.ColumnAttribute -//line mysql_sql.y:9102 +//line mysql_sql.y:9117 { yyLOCAL = append(yyDollar[1].columnAttributesUnion(), yyDollar[2].columnAttributeUnion()) } yyVAL.union = yyLOCAL - case 1390: + case 1392: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.ColumnAttribute -//line mysql_sql.y:9108 +//line mysql_sql.y:9123 { yyLOCAL = tree.NewAttributeNull(true) } yyVAL.union = yyLOCAL - case 1391: + case 1393: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.ColumnAttribute -//line mysql_sql.y:9112 +//line mysql_sql.y:9127 { yyLOCAL = tree.NewAttributeNull(false) } yyVAL.union = yyLOCAL - case 1392: + case 1394: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.ColumnAttribute -//line mysql_sql.y:9116 +//line mysql_sql.y:9131 { yyLOCAL = tree.NewAttributeDefault(yyDollar[2].exprUnion()) } yyVAL.union = yyLOCAL - case 1393: + case 1395: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.ColumnAttribute -//line mysql_sql.y:9120 +//line mysql_sql.y:9135 { yyLOCAL = tree.NewAttributeAutoIncrement() } yyVAL.union = yyLOCAL - case 1394: + case 1396: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.ColumnAttribute -//line mysql_sql.y:9124 +//line mysql_sql.y:9139 { yyLOCAL = yyDollar[1].columnAttributeUnion() } yyVAL.union = yyLOCAL - case 1395: + case 1397: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.ColumnAttribute -//line mysql_sql.y:9128 +//line mysql_sql.y:9143 { str := util.DealCommentString(yyDollar[2].str) yyLOCAL = tree.NewAttributeComment(tree.NewNumVal(str, str, false, tree.P_char)) } yyVAL.union = yyLOCAL - case 1396: + case 1398: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.ColumnAttribute -//line mysql_sql.y:9133 +//line mysql_sql.y:9148 { yyLOCAL = tree.NewAttributeCollate(yyDollar[2].str) } yyVAL.union = yyLOCAL - case 1397: + case 1399: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.ColumnAttribute -//line mysql_sql.y:9137 +//line mysql_sql.y:9152 { yyLOCAL = tree.NewAttributeColumnFormat(yyDollar[2].str) } yyVAL.union = yyLOCAL - case 1398: + case 1400: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.ColumnAttribute -//line mysql_sql.y:9141 +//line mysql_sql.y:9156 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1399: + case 1401: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.ColumnAttribute -//line mysql_sql.y:9145 +//line mysql_sql.y:9160 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1400: + case 1402: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.ColumnAttribute -//line mysql_sql.y:9149 +//line mysql_sql.y:9164 { yyLOCAL = tree.NewAttributeStorage(yyDollar[2].str) } yyVAL.union = yyLOCAL - case 1401: + case 1403: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.ColumnAttribute -//line mysql_sql.y:9153 +//line mysql_sql.y:9168 { yyLOCAL = tree.NewAttributeAutoRandom(int(yyDollar[2].int64ValUnion())) } yyVAL.union = yyLOCAL - case 1402: + case 1404: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.ColumnAttribute -//line mysql_sql.y:9157 +//line mysql_sql.y:9172 { yyLOCAL = yyDollar[1].attributeReferenceUnion() } yyVAL.union = yyLOCAL - case 1403: + case 1405: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL tree.ColumnAttribute -//line mysql_sql.y:9161 +//line mysql_sql.y:9176 { yyLOCAL = tree.NewAttributeCheckConstraint(yyDollar[4].exprUnion(), false, yyDollar[1].str) } yyVAL.union = yyLOCAL - case 1404: + case 1406: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL tree.ColumnAttribute -//line mysql_sql.y:9165 +//line mysql_sql.y:9180 { yyLOCAL = tree.NewAttributeCheckConstraint(yyDollar[4].exprUnion(), yyDollar[6].boolValUnion(), yyDollar[1].str) } yyVAL.union = yyLOCAL - case 1405: + case 1407: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.ColumnAttribute -//line mysql_sql.y:9169 +//line mysql_sql.y:9184 { name := tree.NewUnresolvedColName(yyDollar[3].str) var es tree.Exprs = nil @@ -21780,98 +21802,98 @@ yydefault: yyLOCAL = tree.NewAttributeOnUpdate(expr) } yyVAL.union = yyLOCAL - case 1406: + case 1408: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.ColumnAttribute -//line mysql_sql.y:9183 +//line mysql_sql.y:9198 { yyLOCAL = tree.NewAttributeLowCardinality() } yyVAL.union = yyLOCAL - case 1407: + case 1409: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.ColumnAttribute -//line mysql_sql.y:9187 +//line mysql_sql.y:9202 { yyLOCAL = tree.NewAttributeVisable(true) } yyVAL.union = yyLOCAL - case 1408: + case 1410: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.ColumnAttribute -//line mysql_sql.y:9191 +//line mysql_sql.y:9206 { yyLOCAL = tree.NewAttributeVisable(false) } yyVAL.union = yyLOCAL - case 1409: + case 1411: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL tree.ColumnAttribute -//line mysql_sql.y:9195 +//line mysql_sql.y:9210 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1410: + case 1412: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.ColumnAttribute -//line mysql_sql.y:9199 +//line mysql_sql.y:9214 { yyLOCAL = tree.NewAttributeHeader(yyDollar[3].str) } yyVAL.union = yyLOCAL - case 1411: + case 1413: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.ColumnAttribute -//line mysql_sql.y:9203 +//line mysql_sql.y:9218 { yyLOCAL = tree.NewAttributeHeaders() } yyVAL.union = yyLOCAL - case 1412: + case 1414: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL bool -//line mysql_sql.y:9209 +//line mysql_sql.y:9224 { yyLOCAL = true } yyVAL.union = yyLOCAL - case 1413: + case 1415: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL bool -//line mysql_sql.y:9213 +//line mysql_sql.y:9228 { yyLOCAL = false } yyVAL.union = yyLOCAL - case 1414: + case 1416: yyDollar = yyS[yypt-0 : yypt+1] -//line mysql_sql.y:9218 +//line mysql_sql.y:9233 { yyVAL.str = "" } - case 1415: + case 1417: yyDollar = yyS[yypt-1 : yypt+1] -//line mysql_sql.y:9222 +//line mysql_sql.y:9237 { yyVAL.str = yyDollar[1].str } - case 1416: + case 1418: yyDollar = yyS[yypt-1 : yypt+1] -//line mysql_sql.y:9228 +//line mysql_sql.y:9243 { yyVAL.str = "" } - case 1417: + case 1419: yyDollar = yyS[yypt-2 : yypt+1] -//line mysql_sql.y:9232 +//line mysql_sql.y:9247 { yyVAL.str = yyDollar[2].cstrUnion().Compare() } - case 1418: + case 1420: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *tree.AttributeReference -//line mysql_sql.y:9238 +//line mysql_sql.y:9253 { var TableName = yyDollar[2].tableNameUnion() var KeyParts = yyDollar[3].keyPartsUnion() @@ -21887,10 +21909,10 @@ yydefault: ) } yyVAL.union = yyLOCAL - case 1419: + case 1421: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *tree.ReferenceOnRecord -//line mysql_sql.y:9255 +//line mysql_sql.y:9270 { yyLOCAL = &tree.ReferenceOnRecord{ OnDelete: tree.REFERENCE_OPTION_INVALID, @@ -21898,10 +21920,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1420: + case 1422: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.ReferenceOnRecord -//line mysql_sql.y:9262 +//line mysql_sql.y:9277 { yyLOCAL = &tree.ReferenceOnRecord{ OnDelete: yyDollar[1].referenceOptionTypeUnion(), @@ -21909,10 +21931,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1421: + case 1423: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.ReferenceOnRecord -//line mysql_sql.y:9269 +//line mysql_sql.y:9284 { yyLOCAL = &tree.ReferenceOnRecord{ OnDelete: tree.REFERENCE_OPTION_INVALID, @@ -21920,10 +21942,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1422: + case 1424: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.ReferenceOnRecord -//line mysql_sql.y:9276 +//line mysql_sql.y:9291 { yyLOCAL = &tree.ReferenceOnRecord{ OnDelete: yyDollar[1].referenceOptionTypeUnion(), @@ -21931,10 +21953,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1423: + case 1425: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.ReferenceOnRecord -//line mysql_sql.y:9283 +//line mysql_sql.y:9298 { yyLOCAL = &tree.ReferenceOnRecord{ OnDelete: yyDollar[2].referenceOptionTypeUnion(), @@ -21942,354 +21964,354 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1424: + case 1426: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.ReferenceOptionType -//line mysql_sql.y:9292 +//line mysql_sql.y:9307 { yyLOCAL = yyDollar[3].referenceOptionTypeUnion() } yyVAL.union = yyLOCAL - case 1425: + case 1427: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.ReferenceOptionType -//line mysql_sql.y:9298 +//line mysql_sql.y:9313 { yyLOCAL = yyDollar[3].referenceOptionTypeUnion() } yyVAL.union = yyLOCAL - case 1426: + case 1428: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.ReferenceOptionType -//line mysql_sql.y:9304 +//line mysql_sql.y:9319 { yyLOCAL = tree.REFERENCE_OPTION_RESTRICT } yyVAL.union = yyLOCAL - case 1427: + case 1429: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.ReferenceOptionType -//line mysql_sql.y:9308 +//line mysql_sql.y:9323 { yyLOCAL = tree.REFERENCE_OPTION_CASCADE } yyVAL.union = yyLOCAL - case 1428: + case 1430: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.ReferenceOptionType -//line mysql_sql.y:9312 +//line mysql_sql.y:9327 { yyLOCAL = tree.REFERENCE_OPTION_SET_NULL } yyVAL.union = yyLOCAL - case 1429: + case 1431: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.ReferenceOptionType -//line mysql_sql.y:9316 +//line mysql_sql.y:9331 { yyLOCAL = tree.REFERENCE_OPTION_NO_ACTION } yyVAL.union = yyLOCAL - case 1430: + case 1432: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.ReferenceOptionType -//line mysql_sql.y:9320 +//line mysql_sql.y:9335 { yyLOCAL = tree.REFERENCE_OPTION_SET_DEFAULT } yyVAL.union = yyLOCAL - case 1431: + case 1433: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL tree.MatchType -//line mysql_sql.y:9325 +//line mysql_sql.y:9340 { yyLOCAL = tree.MATCH_INVALID } yyVAL.union = yyLOCAL - case 1433: + case 1435: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.MatchType -//line mysql_sql.y:9332 +//line mysql_sql.y:9347 { yyLOCAL = tree.MATCH_FULL } yyVAL.union = yyLOCAL - case 1434: + case 1436: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.MatchType -//line mysql_sql.y:9336 +//line mysql_sql.y:9351 { yyLOCAL = tree.MATCH_PARTIAL } yyVAL.union = yyLOCAL - case 1435: + case 1437: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.MatchType -//line mysql_sql.y:9340 +//line mysql_sql.y:9355 { yyLOCAL = tree.MATCH_SIMPLE } yyVAL.union = yyLOCAL - case 1436: + case 1438: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL tree.FullTextSearchType -//line mysql_sql.y:9345 +//line mysql_sql.y:9360 { yyLOCAL = tree.FULLTEXT_DEFAULT } yyVAL.union = yyLOCAL - case 1437: + case 1439: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.FullTextSearchType -//line mysql_sql.y:9349 +//line mysql_sql.y:9364 { yyLOCAL = tree.FULLTEXT_NL } yyVAL.union = yyLOCAL - case 1438: + case 1440: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL tree.FullTextSearchType -//line mysql_sql.y:9353 +//line mysql_sql.y:9368 { yyLOCAL = tree.FULLTEXT_NL_QUERY_EXPANSION } yyVAL.union = yyLOCAL - case 1439: + case 1441: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.FullTextSearchType -//line mysql_sql.y:9357 +//line mysql_sql.y:9372 { yyLOCAL = tree.FULLTEXT_BOOLEAN } yyVAL.union = yyLOCAL - case 1440: + case 1442: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.FullTextSearchType -//line mysql_sql.y:9361 +//line mysql_sql.y:9376 { yyLOCAL = tree.FULLTEXT_QUERY_EXPANSION } yyVAL.union = yyLOCAL - case 1441: + case 1443: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL []*tree.KeyPart -//line mysql_sql.y:9366 +//line mysql_sql.y:9381 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1442: + case 1444: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL []*tree.KeyPart -//line mysql_sql.y:9370 +//line mysql_sql.y:9385 { yyLOCAL = yyDollar[2].keyPartsUnion() } yyVAL.union = yyLOCAL - case 1443: + case 1445: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL int64 -//line mysql_sql.y:9375 +//line mysql_sql.y:9390 { yyLOCAL = -1 } yyVAL.union = yyLOCAL - case 1444: + case 1446: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL int64 -//line mysql_sql.y:9379 +//line mysql_sql.y:9394 { yyLOCAL = yyDollar[2].item.(int64) } yyVAL.union = yyLOCAL - case 1451: + case 1453: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.Subquery -//line mysql_sql.y:9395 +//line mysql_sql.y:9410 { yyLOCAL = &tree.Subquery{Select: yyDollar[1].selectStatementUnion(), Exists: false} } yyVAL.union = yyLOCAL - case 1452: + case 1454: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9401 +//line mysql_sql.y:9416 { yyLOCAL = tree.NewBinaryExpr(tree.BIT_AND, yyDollar[1].exprUnion(), yyDollar[3].exprUnion()) } yyVAL.union = yyLOCAL - case 1453: + case 1455: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9405 +//line mysql_sql.y:9420 { yyLOCAL = tree.NewBinaryExpr(tree.BIT_OR, yyDollar[1].exprUnion(), yyDollar[3].exprUnion()) } yyVAL.union = yyLOCAL - case 1454: + case 1456: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9409 +//line mysql_sql.y:9424 { yyLOCAL = tree.NewBinaryExpr(tree.BIT_XOR, yyDollar[1].exprUnion(), yyDollar[3].exprUnion()) } yyVAL.union = yyLOCAL - case 1455: + case 1457: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9413 +//line mysql_sql.y:9428 { yyLOCAL = tree.NewBinaryExpr(tree.PLUS, yyDollar[1].exprUnion(), yyDollar[3].exprUnion()) } yyVAL.union = yyLOCAL - case 1456: + case 1458: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9417 +//line mysql_sql.y:9432 { yyLOCAL = tree.NewBinaryExpr(tree.MINUS, yyDollar[1].exprUnion(), yyDollar[3].exprUnion()) } yyVAL.union = yyLOCAL - case 1457: + case 1459: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9421 +//line mysql_sql.y:9436 { yyLOCAL = tree.NewBinaryExpr(tree.MULTI, yyDollar[1].exprUnion(), yyDollar[3].exprUnion()) } yyVAL.union = yyLOCAL - case 1458: + case 1460: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9425 +//line mysql_sql.y:9440 { yyLOCAL = tree.NewBinaryExpr(tree.DIV, yyDollar[1].exprUnion(), yyDollar[3].exprUnion()) } yyVAL.union = yyLOCAL - case 1459: + case 1461: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9429 +//line mysql_sql.y:9444 { yyLOCAL = tree.NewBinaryExpr(tree.INTEGER_DIV, yyDollar[1].exprUnion(), yyDollar[3].exprUnion()) } yyVAL.union = yyLOCAL - case 1460: + case 1462: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9433 +//line mysql_sql.y:9448 { yyLOCAL = tree.NewBinaryExpr(tree.MOD, yyDollar[1].exprUnion(), yyDollar[3].exprUnion()) } yyVAL.union = yyLOCAL - case 1461: + case 1463: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9437 +//line mysql_sql.y:9452 { yyLOCAL = tree.NewBinaryExpr(tree.MOD, yyDollar[1].exprUnion(), yyDollar[3].exprUnion()) } yyVAL.union = yyLOCAL - case 1462: + case 1464: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9441 +//line mysql_sql.y:9456 { yyLOCAL = tree.NewBinaryExpr(tree.LEFT_SHIFT, yyDollar[1].exprUnion(), yyDollar[3].exprUnion()) } yyVAL.union = yyLOCAL - case 1463: + case 1465: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9445 +//line mysql_sql.y:9460 { yyLOCAL = tree.NewBinaryExpr(tree.RIGHT_SHIFT, yyDollar[1].exprUnion(), yyDollar[3].exprUnion()) } yyVAL.union = yyLOCAL - case 1464: + case 1466: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9449 +//line mysql_sql.y:9464 { yyLOCAL = yyDollar[1].exprUnion() } yyVAL.union = yyLOCAL - case 1465: + case 1467: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9455 +//line mysql_sql.y:9470 { yyLOCAL = yyDollar[1].unresolvedNameUnion() } yyVAL.union = yyLOCAL - case 1466: + case 1468: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9459 +//line mysql_sql.y:9474 { yyLOCAL = yyDollar[1].varExprUnion() } yyVAL.union = yyLOCAL - case 1467: + case 1469: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9463 +//line mysql_sql.y:9478 { yyLOCAL = yyDollar[1].exprUnion() } yyVAL.union = yyLOCAL - case 1468: + case 1470: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9467 +//line mysql_sql.y:9482 { yyLOCAL = tree.NewParentExpr(yyDollar[2].exprUnion()) } yyVAL.union = yyLOCAL - case 1469: + case 1471: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9471 +//line mysql_sql.y:9486 { yyLOCAL = tree.NewTuple(append(yyDollar[2].exprsUnion(), yyDollar[4].exprUnion())) } yyVAL.union = yyLOCAL - case 1470: + case 1472: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9475 +//line mysql_sql.y:9490 { yyLOCAL = tree.NewUnaryExpr(tree.UNARY_PLUS, yyDollar[2].exprUnion()) } yyVAL.union = yyLOCAL - case 1471: + case 1473: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9479 +//line mysql_sql.y:9494 { yyLOCAL = tree.NewUnaryExpr(tree.UNARY_MINUS, yyDollar[2].exprUnion()) } yyVAL.union = yyLOCAL - case 1472: + case 1474: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9483 +//line mysql_sql.y:9498 { yyLOCAL = tree.NewUnaryExpr(tree.UNARY_TILDE, yyDollar[2].exprUnion()) } yyVAL.union = yyLOCAL - case 1473: + case 1475: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9487 +//line mysql_sql.y:9502 { yyLOCAL = tree.NewUnaryExpr(tree.UNARY_MARK, yyDollar[2].exprUnion()) } yyVAL.union = yyLOCAL - case 1474: + case 1476: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9491 +//line mysql_sql.y:9506 { hint := strings.ToLower(yyDollar[2].cstrUnion().Compare()) switch hint { @@ -22332,35 +22354,35 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1475: + case 1477: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9533 +//line mysql_sql.y:9548 { yyLOCAL = yyDollar[1].exprUnion() } yyVAL.union = yyLOCAL - case 1476: + case 1478: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9537 +//line mysql_sql.y:9552 { yyLOCAL = yyDollar[1].subqueryUnion() } yyVAL.union = yyLOCAL - case 1477: + case 1479: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9541 +//line mysql_sql.y:9556 { yyDollar[2].subqueryUnion().Exists = true yyLOCAL = yyDollar[2].subqueryUnion() } yyVAL.union = yyLOCAL - case 1478: + case 1480: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9546 +//line mysql_sql.y:9561 { yyLOCAL = &tree.CaseExpr{ Expr: yyDollar[2].exprUnion(), @@ -22369,42 +22391,42 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1479: + case 1481: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9554 +//line mysql_sql.y:9569 { yyLOCAL = tree.NewCastExpr(yyDollar[3].exprUnion(), yyDollar[5].columnTypeUnion()) } yyVAL.union = yyLOCAL - case 1480: + case 1482: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9558 +//line mysql_sql.y:9573 { yyLOCAL = tree.NewSerialExtractExpr(yyDollar[3].exprUnion(), yyDollar[5].exprUnion(), yyDollar[7].columnTypeUnion()) } yyVAL.union = yyLOCAL - case 1481: + case 1483: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9562 +//line mysql_sql.y:9577 { yyLOCAL = tree.NewBitCastExpr(yyDollar[3].exprUnion(), yyDollar[5].columnTypeUnion()) } yyVAL.union = yyLOCAL - case 1482: + case 1484: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9566 +//line mysql_sql.y:9581 { yyLOCAL = tree.NewCastExpr(yyDollar[3].exprUnion(), yyDollar[5].columnTypeUnion()) } yyVAL.union = yyLOCAL - case 1483: + case 1485: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9570 +//line mysql_sql.y:9585 { name := tree.NewUnresolvedColName(yyDollar[1].str) es := tree.NewNumVal(yyDollar[5].str, yyDollar[5].str, false, tree.P_char) @@ -22415,66 +22437,66 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1484: + case 1486: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9580 +//line mysql_sql.y:9595 { yyLOCAL = yyDollar[1].funcExprUnion() } yyVAL.union = yyLOCAL - case 1485: + case 1487: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9584 +//line mysql_sql.y:9599 { yyLOCAL = yyDollar[1].funcExprUnion() } yyVAL.union = yyLOCAL - case 1486: + case 1488: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9588 +//line mysql_sql.y:9603 { yyLOCAL = yyDollar[1].funcExprUnion() } yyVAL.union = yyLOCAL - case 1487: + case 1489: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9592 +//line mysql_sql.y:9607 { yyLOCAL = yyDollar[1].funcExprUnion() } yyVAL.union = yyLOCAL - case 1488: + case 1490: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9596 +//line mysql_sql.y:9611 { yyLOCAL = yyDollar[1].funcExprUnion() } yyVAL.union = yyLOCAL - case 1489: + case 1491: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9600 +//line mysql_sql.y:9615 { yyLOCAL = yyDollar[1].exprUnion() } yyVAL.union = yyLOCAL - case 1490: + case 1492: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9604 +//line mysql_sql.y:9619 { yyLOCAL = yyDollar[1].exprUnion() } yyVAL.union = yyLOCAL - case 1491: + case 1493: yyDollar = yyS[yypt-9 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9608 +//line mysql_sql.y:9623 { val, err := tree.NewFullTextMatchFuncExpression(yyDollar[3].keyPartsUnion(), yyDollar[7].str, yyDollar[8].fullTextSearchTypeUnion()) if err != nil { @@ -22484,16 +22506,16 @@ yydefault: yyLOCAL = val } yyVAL.union = yyLOCAL - case 1492: + case 1494: yyDollar = yyS[yypt-1 : yypt+1] -//line mysql_sql.y:9621 +//line mysql_sql.y:9636 { yyVAL.str = yyDollar[1].str } - case 1493: + case 1495: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:9627 +//line mysql_sql.y:9642 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -22503,10 +22525,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1494: + case 1496: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:9636 +//line mysql_sql.y:9651 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -22516,10 +22538,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1495: + case 1497: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:9645 +//line mysql_sql.y:9660 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -22529,10 +22551,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1496: + case 1498: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9656 +//line mysql_sql.y:9671 { v := int(yyDollar[5].item.(int64)) val, err := tree.NewSampleRowsFuncExpression(v, true, nil, "block") @@ -22543,10 +22565,10 @@ yydefault: yyLOCAL = val } yyVAL.union = yyLOCAL - case 1497: + case 1499: yyDollar = yyS[yypt-9 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9666 +//line mysql_sql.y:9681 { v := int(yyDollar[5].item.(int64)) val, err := tree.NewSampleRowsFuncExpression(v, true, nil, yyDollar[8].str) @@ -22557,10 +22579,10 @@ yydefault: yyLOCAL = val } yyVAL.union = yyLOCAL - case 1498: + case 1500: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9676 +//line mysql_sql.y:9691 { val, err := tree.NewSamplePercentFuncExpression1(yyDollar[5].item.(int64), true, nil) if err != nil { @@ -22570,10 +22592,10 @@ yydefault: yyLOCAL = val } yyVAL.union = yyLOCAL - case 1499: + case 1501: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9685 +//line mysql_sql.y:9700 { val, err := tree.NewSamplePercentFuncExpression2(yyDollar[5].item.(float64), true, nil) if err != nil { @@ -22583,10 +22605,10 @@ yydefault: yyLOCAL = val } yyVAL.union = yyLOCAL - case 1500: + case 1502: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9695 +//line mysql_sql.y:9710 { v := int(yyDollar[5].item.(int64)) val, err := tree.NewSampleRowsFuncExpression(v, false, yyDollar[3].exprsUnion(), "block") @@ -22597,10 +22619,10 @@ yydefault: yyLOCAL = val } yyVAL.union = yyLOCAL - case 1501: + case 1503: yyDollar = yyS[yypt-9 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9705 +//line mysql_sql.y:9720 { v := int(yyDollar[5].item.(int64)) val, err := tree.NewSampleRowsFuncExpression(v, false, yyDollar[3].exprsUnion(), yyDollar[8].str) @@ -22611,10 +22633,10 @@ yydefault: yyLOCAL = val } yyVAL.union = yyLOCAL - case 1502: + case 1504: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9715 +//line mysql_sql.y:9730 { val, err := tree.NewSamplePercentFuncExpression1(yyDollar[5].item.(int64), false, yyDollar[3].exprsUnion()) if err != nil { @@ -22624,10 +22646,10 @@ yydefault: yyLOCAL = val } yyVAL.union = yyLOCAL - case 1503: + case 1505: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9724 +//line mysql_sql.y:9739 { val, err := tree.NewSamplePercentFuncExpression2(yyDollar[5].item.(float64), false, yyDollar[3].exprsUnion()) if err != nil { @@ -22637,58 +22659,58 @@ yydefault: yyLOCAL = val } yyVAL.union = yyLOCAL - case 1504: + case 1506: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9734 +//line mysql_sql.y:9749 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1505: + case 1507: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9738 +//line mysql_sql.y:9753 { yyLOCAL = yyDollar[2].exprUnion() } yyVAL.union = yyLOCAL - case 1506: + case 1508: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9743 +//line mysql_sql.y:9758 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1507: + case 1509: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:9747 +//line mysql_sql.y:9762 { yyLOCAL = yyDollar[1].exprUnion() } yyVAL.union = yyLOCAL - case 1508: + case 1510: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []*tree.When -//line mysql_sql.y:9753 +//line mysql_sql.y:9768 { yyLOCAL = []*tree.When{yyDollar[1].whenClauseUnion()} } yyVAL.union = yyLOCAL - case 1509: + case 1511: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL []*tree.When -//line mysql_sql.y:9757 +//line mysql_sql.y:9772 { yyLOCAL = append(yyDollar[1].whenClauseListUnion(), yyDollar[2].whenClauseUnion()) } yyVAL.union = yyLOCAL - case 1510: + case 1512: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *tree.When -//line mysql_sql.y:9763 +//line mysql_sql.y:9778 { yyLOCAL = &tree.When{ Cond: yyDollar[2].exprUnion(), @@ -22696,9 +22718,9 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1511: + case 1513: yyDollar = yyS[yypt-1 : yypt+1] -//line mysql_sql.y:9772 +//line mysql_sql.y:9787 { t := yyVAL.columnTypeUnion() str := strings.ToLower(t.InternalType.FamilyString) @@ -22711,10 +22733,10 @@ yydefault: } } } - case 1512: + case 1514: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:9784 +//line mysql_sql.y:9799 { name := yyDollar[1].str if yyDollar[2].str != "" { @@ -22732,10 +22754,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1513: + case 1515: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:9801 +//line mysql_sql.y:9816 { locale := "" yyLOCAL = &tree.T{ @@ -22750,10 +22772,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1515: + case 1517: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:9818 +//line mysql_sql.y:9833 { locale := "" yyLOCAL = &tree.T{ @@ -22767,10 +22789,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1516: + case 1518: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:9831 +//line mysql_sql.y:9846 { locale := "" yyLOCAL = &tree.T{ @@ -22784,10 +22806,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1517: + case 1519: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:9844 +//line mysql_sql.y:9859 { locale := "" yyLOCAL = &tree.T{ @@ -22800,10 +22822,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1518: + case 1520: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:9856 +//line mysql_sql.y:9871 { locale := "" yyLOCAL = &tree.T{ @@ -22818,10 +22840,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1519: + case 1521: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:9870 +//line mysql_sql.y:9885 { locale := "" yyLOCAL = &tree.T{ @@ -22837,10 +22859,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1520: + case 1522: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:9885 +//line mysql_sql.y:9900 { locale := "" yyLOCAL = &tree.T{ @@ -22856,10 +22878,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1521: + case 1523: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:9900 +//line mysql_sql.y:9915 { name := yyDollar[1].str if yyDollar[2].str != "" { @@ -22877,10 +22899,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1522: + case 1524: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:9917 +//line mysql_sql.y:9932 { locale := "" yyLOCAL = &tree.T{ @@ -22895,95 +22917,95 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1523: + case 1525: yyDollar = yyS[yypt-0 : yypt+1] -//line mysql_sql.y:9932 +//line mysql_sql.y:9947 { } - case 1527: + case 1529: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.FrameBound -//line mysql_sql.y:9939 +//line mysql_sql.y:9954 { yyLOCAL = &tree.FrameBound{Type: tree.Following, UnBounded: true} } yyVAL.union = yyLOCAL - case 1528: + case 1530: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.FrameBound -//line mysql_sql.y:9943 +//line mysql_sql.y:9958 { yyLOCAL = &tree.FrameBound{Type: tree.Following, Expr: yyDollar[1].exprUnion()} } yyVAL.union = yyLOCAL - case 1529: + case 1531: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.FrameBound -//line mysql_sql.y:9947 +//line mysql_sql.y:9962 { yyLOCAL = &tree.FrameBound{Type: tree.Following, Expr: yyDollar[1].exprUnion()} } yyVAL.union = yyLOCAL - case 1530: + case 1532: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.FrameBound -//line mysql_sql.y:9953 +//line mysql_sql.y:9968 { yyLOCAL = &tree.FrameBound{Type: tree.CurrentRow} } yyVAL.union = yyLOCAL - case 1531: + case 1533: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.FrameBound -//line mysql_sql.y:9957 +//line mysql_sql.y:9972 { yyLOCAL = &tree.FrameBound{Type: tree.Preceding, UnBounded: true} } yyVAL.union = yyLOCAL - case 1532: + case 1534: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.FrameBound -//line mysql_sql.y:9961 +//line mysql_sql.y:9976 { yyLOCAL = &tree.FrameBound{Type: tree.Preceding, Expr: yyDollar[1].exprUnion()} } yyVAL.union = yyLOCAL - case 1533: + case 1535: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.FrameBound -//line mysql_sql.y:9965 +//line mysql_sql.y:9980 { yyLOCAL = &tree.FrameBound{Type: tree.Preceding, Expr: yyDollar[1].exprUnion()} } yyVAL.union = yyLOCAL - case 1534: + case 1536: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.FrameType -//line mysql_sql.y:9971 +//line mysql_sql.y:9986 { yyLOCAL = tree.Rows } yyVAL.union = yyLOCAL - case 1535: + case 1537: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.FrameType -//line mysql_sql.y:9975 +//line mysql_sql.y:9990 { yyLOCAL = tree.Range } yyVAL.union = yyLOCAL - case 1536: + case 1538: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.FrameType -//line mysql_sql.y:9979 +//line mysql_sql.y:9994 { yyLOCAL = tree.Groups } yyVAL.union = yyLOCAL - case 1537: + case 1539: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.FrameClause -//line mysql_sql.y:9985 +//line mysql_sql.y:10000 { yyLOCAL = &tree.FrameClause{ Type: yyDollar[1].frameTypeUnion(), @@ -22992,10 +23014,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1538: + case 1540: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *tree.FrameClause -//line mysql_sql.y:9993 +//line mysql_sql.y:10008 { yyLOCAL = &tree.FrameClause{ Type: yyDollar[1].frameTypeUnion(), @@ -23005,82 +23027,82 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1539: + case 1541: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *tree.FrameClause -//line mysql_sql.y:10003 +//line mysql_sql.y:10018 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1540: + case 1542: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.FrameClause -//line mysql_sql.y:10007 +//line mysql_sql.y:10022 { yyLOCAL = yyDollar[1].frameClauseUnion() } yyVAL.union = yyLOCAL - case 1541: + case 1543: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.Exprs -//line mysql_sql.y:10014 +//line mysql_sql.y:10029 { yyLOCAL = yyDollar[3].exprsUnion() } yyVAL.union = yyLOCAL - case 1542: + case 1544: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL tree.Exprs -//line mysql_sql.y:10019 +//line mysql_sql.y:10034 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1543: + case 1545: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Exprs -//line mysql_sql.y:10023 +//line mysql_sql.y:10038 { yyLOCAL = yyDollar[1].exprsUnion() } yyVAL.union = yyLOCAL - case 1544: + case 1546: yyDollar = yyS[yypt-0 : yypt+1] -//line mysql_sql.y:10028 +//line mysql_sql.y:10043 { yyVAL.str = "," } - case 1545: + case 1547: yyDollar = yyS[yypt-2 : yypt+1] -//line mysql_sql.y:10032 +//line mysql_sql.y:10047 { yyVAL.str = yyDollar[2].str } - case 1546: + case 1548: yyDollar = yyS[yypt-0 : yypt+1] -//line mysql_sql.y:10037 +//line mysql_sql.y:10052 { yyVAL.str = "1,vector_l2_ops,random,false" } - case 1547: + case 1549: yyDollar = yyS[yypt-2 : yypt+1] -//line mysql_sql.y:10041 +//line mysql_sql.y:10056 { yyVAL.str = yyDollar[2].str } - case 1548: + case 1550: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *tree.WindowSpec -//line mysql_sql.y:10046 +//line mysql_sql.y:10061 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1550: + case 1552: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *tree.WindowSpec -//line mysql_sql.y:10053 +//line mysql_sql.y:10068 { hasFrame := true var f *tree.FrameClause @@ -23105,10 +23127,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1551: + case 1553: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10079 +//line mysql_sql.y:10094 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23121,10 +23143,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1552: + case 1554: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10091 +//line mysql_sql.y:10106 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23137,10 +23159,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1553: + case 1555: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10103 +//line mysql_sql.y:10118 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23152,10 +23174,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1554: + case 1556: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10114 +//line mysql_sql.y:10129 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23167,10 +23189,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1555: + case 1557: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10125 +//line mysql_sql.y:10140 { name := tree.NewUnresolvedColName(yyDollar[1].str) es := tree.NewNumVal("*", "*", false, tree.P_char) @@ -23182,10 +23204,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1556: + case 1558: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10136 +//line mysql_sql.y:10151 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23196,10 +23218,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1557: + case 1559: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10146 +//line mysql_sql.y:10161 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23210,10 +23232,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1558: + case 1560: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10156 +//line mysql_sql.y:10171 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23225,10 +23247,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1559: + case 1561: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10167 +//line mysql_sql.y:10182 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23240,10 +23262,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1560: + case 1562: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10178 +//line mysql_sql.y:10193 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23255,10 +23277,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1561: + case 1563: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10189 +//line mysql_sql.y:10204 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23270,10 +23292,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1562: + case 1564: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10200 +//line mysql_sql.y:10215 { name := tree.NewUnresolvedColName(yyDollar[1].str) es := tree.NewNumVal("*", "*", false, tree.P_char) @@ -23285,10 +23307,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1563: + case 1565: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10211 +//line mysql_sql.y:10226 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23300,10 +23322,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1564: + case 1566: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10222 +//line mysql_sql.y:10237 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23315,10 +23337,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1565: + case 1567: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10233 +//line mysql_sql.y:10248 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23330,10 +23352,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1566: + case 1568: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10244 +//line mysql_sql.y:10259 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23345,10 +23367,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1567: + case 1569: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10255 +//line mysql_sql.y:10270 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23360,10 +23382,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1568: + case 1570: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10266 +//line mysql_sql.y:10281 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23375,10 +23397,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1569: + case 1571: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10277 +//line mysql_sql.y:10292 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23390,10 +23412,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1570: + case 1572: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10288 +//line mysql_sql.y:10303 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23405,10 +23427,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1571: + case 1573: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10299 +//line mysql_sql.y:10314 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23420,10 +23442,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1572: + case 1574: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10310 +//line mysql_sql.y:10325 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23435,10 +23457,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1573: + case 1575: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10321 +//line mysql_sql.y:10336 { name := tree.NewUnresolvedColName(yyDollar[1].str) var columnList tree.Exprs @@ -23456,10 +23478,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1577: + case 1579: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10345 +//line mysql_sql.y:10360 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23469,10 +23491,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1578: + case 1580: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10354 +//line mysql_sql.y:10369 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23482,10 +23504,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1579: + case 1581: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10363 +//line mysql_sql.y:10378 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23495,10 +23517,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1580: + case 1582: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10372 +//line mysql_sql.y:10387 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23508,10 +23530,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1581: + case 1583: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10381 +//line mysql_sql.y:10396 { name := tree.NewUnresolvedColName(yyDollar[1].str) str := strings.ToLower(yyDollar[3].str) @@ -23523,10 +23545,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1582: + case 1584: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10392 +//line mysql_sql.y:10407 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23536,10 +23558,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1583: + case 1585: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10401 +//line mysql_sql.y:10416 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23550,10 +23572,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1584: + case 1586: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10411 +//line mysql_sql.y:10426 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23563,10 +23585,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1585: + case 1587: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10420 +//line mysql_sql.y:10435 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23576,10 +23598,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1586: + case 1588: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10429 +//line mysql_sql.y:10444 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23589,10 +23611,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1587: + case 1589: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10438 +//line mysql_sql.y:10453 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23602,10 +23624,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1588: + case 1590: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10447 +//line mysql_sql.y:10462 { name := tree.NewUnresolvedColName(yyDollar[1].str) arg0 := tree.NewNumVal(int64(0), "0", false, tree.P_int64) @@ -23618,10 +23640,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1589: + case 1591: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10459 +//line mysql_sql.y:10474 { name := tree.NewUnresolvedColName(yyDollar[1].str) arg0 := tree.NewNumVal(int64(1), "1", false, tree.P_int64) @@ -23633,10 +23655,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1590: + case 1592: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10470 +//line mysql_sql.y:10485 { name := tree.NewUnresolvedColName(yyDollar[1].str) arg0 := tree.NewNumVal(int64(2), "2", false, tree.P_int64) @@ -23650,10 +23672,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1591: + case 1593: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10483 +//line mysql_sql.y:10498 { name := tree.NewUnresolvedColName(yyDollar[1].str) arg0 := tree.NewNumVal(int64(3), "3", false, tree.P_int64) @@ -23666,10 +23688,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1592: + case 1594: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10495 +//line mysql_sql.y:10510 { column := tree.NewUnresolvedColName(yyDollar[3].str) name := tree.NewUnresolvedColName(yyDollar[1].str) @@ -23680,16 +23702,16 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1599: + case 1601: yyDollar = yyS[yypt-1 : yypt+1] -//line mysql_sql.y:10518 +//line mysql_sql.y:10533 { yyVAL.str = yyDollar[1].str } - case 1628: + case 1630: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10554 +//line mysql_sql.y:10569 { name := tree.NewUnresolvedColName(yyDollar[1].str) var es tree.Exprs = nil @@ -23703,10 +23725,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1629: + case 1631: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10567 +//line mysql_sql.y:10582 { name := tree.NewUnresolvedColName(yyDollar[1].str) var es tree.Exprs = nil @@ -23720,10 +23742,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1630: + case 1632: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10580 +//line mysql_sql.y:10595 { name := tree.NewUnresolvedColName(yyDollar[1].str) str := strings.ToLower(yyDollar[3].str) @@ -23735,10 +23757,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1631: + case 1633: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10592 +//line mysql_sql.y:10607 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23748,10 +23770,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1632: + case 1634: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10601 +//line mysql_sql.y:10616 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23760,10 +23782,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1633: + case 1635: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10609 +//line mysql_sql.y:10624 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23772,10 +23794,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1634: + case 1636: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10617 +//line mysql_sql.y:10632 { name := tree.NewUnresolvedColName(yyDollar[1].str) var es tree.Exprs = nil @@ -23789,10 +23811,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1635: + case 1637: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10630 +//line mysql_sql.y:10645 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23802,10 +23824,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1636: + case 1638: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10639 +//line mysql_sql.y:10654 { name := tree.NewUnresolvedColName(yyDollar[1].str) exprs := make([]tree.Expr, 1) @@ -23817,10 +23839,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1637: + case 1639: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10650 +//line mysql_sql.y:10665 { name := tree.NewUnresolvedColName(yyDollar[1].str) exprs := make([]tree.Expr, 1) @@ -23832,10 +23854,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1638: + case 1640: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10661 +//line mysql_sql.y:10676 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23845,10 +23867,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1639: + case 1641: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10670 +//line mysql_sql.y:10685 { cn := tree.NewNumVal(yyDollar[5].str, yyDollar[5].str, false, tree.P_char) es := yyDollar[3].exprsUnion() @@ -23861,10 +23883,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1640: + case 1642: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10682 +//line mysql_sql.y:10697 { val := tree.NewNumVal(yyDollar[2].str, yyDollar[2].str, false, tree.P_char) name := tree.NewUnresolvedColName(yyDollar[1].str) @@ -23875,10 +23897,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1641: + case 1643: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10692 +//line mysql_sql.y:10707 { val := tree.NewNumVal(yyDollar[2].str, yyDollar[2].str, false, tree.P_char) name := tree.NewUnresolvedColName(yyDollar[1].str) @@ -23889,10 +23911,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1642: + case 1644: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10702 +//line mysql_sql.y:10717 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23902,10 +23924,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1643: + case 1645: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10711 +//line mysql_sql.y:10726 { es := tree.Exprs{yyDollar[3].exprUnion()} es = append(es, yyDollar[5].exprUnion()) @@ -23917,10 +23939,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1644: + case 1646: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10722 +//line mysql_sql.y:10737 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23930,10 +23952,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1645: + case 1647: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10731 +//line mysql_sql.y:10746 { val := tree.NewNumVal(yyDollar[2].str, yyDollar[2].str, false, tree.P_char) name := tree.NewUnresolvedColName(yyDollar[1].str) @@ -23944,10 +23966,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1646: + case 1648: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10741 +//line mysql_sql.y:10756 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23957,10 +23979,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1647: + case 1649: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10750 +//line mysql_sql.y:10765 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23970,10 +23992,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1648: + case 1650: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *tree.FuncExpr -//line mysql_sql.y:10759 +//line mysql_sql.y:10774 { name := tree.NewUnresolvedColName(yyDollar[1].str) yyLOCAL = &tree.FuncExpr{ @@ -23983,34 +24005,34 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1649: + case 1651: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:10769 +//line mysql_sql.y:10784 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1650: + case 1652: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:10773 +//line mysql_sql.y:10788 { yyLOCAL = yyDollar[1].exprUnion() } yyVAL.union = yyLOCAL - case 1651: + case 1653: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:10779 +//line mysql_sql.y:10794 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1652: + case 1654: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:10783 +//line mysql_sql.y:10798 { ival, errStr := util.GetInt64(yyDollar[2].item) if errStr != "" { @@ -24021,20 +24043,20 @@ yydefault: yyLOCAL = tree.NewNumVal(ival, str, false, tree.P_int64) } yyVAL.union = yyLOCAL - case 1659: + case 1661: yyDollar = yyS[yypt-0 : yypt+1] -//line mysql_sql.y:10802 +//line mysql_sql.y:10817 { } - case 1660: + case 1662: yyDollar = yyS[yypt-2 : yypt+1] -//line mysql_sql.y:10804 +//line mysql_sql.y:10819 { } - case 1694: + case 1697: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:10845 +//line mysql_sql.y:10861 { name := tree.NewUnresolvedColName(yyDollar[1].str) str := strings.ToLower(yyDollar[3].str) @@ -24046,106 +24068,106 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1695: + case 1698: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL tree.FuncType -//line mysql_sql.y:10857 +//line mysql_sql.y:10873 { yyLOCAL = tree.FUNC_TYPE_DEFAULT } yyVAL.union = yyLOCAL - case 1696: + case 1699: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.FuncType -//line mysql_sql.y:10861 +//line mysql_sql.y:10877 { yyLOCAL = tree.FUNC_TYPE_DISTINCT } yyVAL.union = yyLOCAL - case 1697: + case 1700: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.FuncType -//line mysql_sql.y:10865 +//line mysql_sql.y:10881 { yyLOCAL = tree.FUNC_TYPE_ALL } yyVAL.union = yyLOCAL - case 1698: + case 1701: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *tree.Tuple -//line mysql_sql.y:10871 +//line mysql_sql.y:10887 { yyLOCAL = tree.NewTuple(yyDollar[2].exprsUnion()) } yyVAL.union = yyLOCAL - case 1699: + case 1702: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL tree.Exprs -//line mysql_sql.y:10876 +//line mysql_sql.y:10892 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1700: + case 1703: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Exprs -//line mysql_sql.y:10880 +//line mysql_sql.y:10896 { yyLOCAL = yyDollar[1].exprsUnion() } yyVAL.union = yyLOCAL - case 1701: + case 1704: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Exprs -//line mysql_sql.y:10886 +//line mysql_sql.y:10902 { yyLOCAL = tree.Exprs{yyDollar[1].exprUnion()} } yyVAL.union = yyLOCAL - case 1702: + case 1705: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.Exprs -//line mysql_sql.y:10890 +//line mysql_sql.y:10906 { yyLOCAL = append(yyDollar[1].exprsUnion(), yyDollar[3].exprUnion()) } yyVAL.union = yyLOCAL - case 1703: + case 1706: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Exprs -//line mysql_sql.y:10896 +//line mysql_sql.y:10912 { yyLOCAL = tree.Exprs{yyDollar[1].exprUnion()} } yyVAL.union = yyLOCAL - case 1704: + case 1707: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.Exprs -//line mysql_sql.y:10900 +//line mysql_sql.y:10916 { yyLOCAL = append(yyDollar[1].exprsUnion(), yyDollar[3].exprUnion()) } yyVAL.union = yyLOCAL - case 1705: + case 1708: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:10907 +//line mysql_sql.y:10923 { yyLOCAL = tree.NewAndExpr(yyDollar[1].exprUnion(), yyDollar[3].exprUnion()) } yyVAL.union = yyLOCAL - case 1706: + case 1709: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:10911 +//line mysql_sql.y:10927 { yyLOCAL = tree.NewOrExpr(yyDollar[1].exprUnion(), yyDollar[3].exprUnion()) } yyVAL.union = yyLOCAL - case 1707: + case 1710: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:10915 +//line mysql_sql.y:10931 { name := tree.NewUnresolvedColName("concat") yyLOCAL = &tree.FuncExpr{ @@ -24155,355 +24177,355 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1708: + case 1711: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:10924 +//line mysql_sql.y:10940 { yyLOCAL = tree.NewXorExpr(yyDollar[1].exprUnion(), yyDollar[3].exprUnion()) } yyVAL.union = yyLOCAL - case 1709: + case 1712: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:10928 +//line mysql_sql.y:10944 { yyLOCAL = tree.NewNotExpr(yyDollar[2].exprUnion()) } yyVAL.union = yyLOCAL - case 1710: + case 1713: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:10932 +//line mysql_sql.y:10948 { yyLOCAL = yyDollar[1].exprUnion() } yyVAL.union = yyLOCAL - case 1711: + case 1714: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:10937 +//line mysql_sql.y:10953 { yyLOCAL = yyDollar[1].exprUnion() } yyVAL.union = yyLOCAL - case 1712: + case 1715: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:10941 +//line mysql_sql.y:10957 { yyLOCAL = tree.NewMaxValue() } yyVAL.union = yyLOCAL - case 1713: + case 1716: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:10947 +//line mysql_sql.y:10963 { yyLOCAL = tree.NewIsNullExpr(yyDollar[1].exprUnion()) } yyVAL.union = yyLOCAL - case 1714: + case 1717: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:10951 +//line mysql_sql.y:10967 { yyLOCAL = tree.NewIsNotNullExpr(yyDollar[1].exprUnion()) } yyVAL.union = yyLOCAL - case 1715: + case 1718: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:10955 +//line mysql_sql.y:10971 { yyLOCAL = tree.NewIsUnknownExpr(yyDollar[1].exprUnion()) } yyVAL.union = yyLOCAL - case 1716: + case 1719: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:10959 +//line mysql_sql.y:10975 { yyLOCAL = tree.NewIsNotUnknownExpr(yyDollar[1].exprUnion()) } yyVAL.union = yyLOCAL - case 1717: + case 1720: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:10963 +//line mysql_sql.y:10979 { yyLOCAL = tree.NewIsTrueExpr(yyDollar[1].exprUnion()) } yyVAL.union = yyLOCAL - case 1718: + case 1721: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:10967 +//line mysql_sql.y:10983 { yyLOCAL = tree.NewIsNotTrueExpr(yyDollar[1].exprUnion()) } yyVAL.union = yyLOCAL - case 1719: + case 1722: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:10971 +//line mysql_sql.y:10987 { yyLOCAL = tree.NewIsFalseExpr(yyDollar[1].exprUnion()) } yyVAL.union = yyLOCAL - case 1720: + case 1723: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:10975 +//line mysql_sql.y:10991 { yyLOCAL = tree.NewIsNotFalseExpr(yyDollar[1].exprUnion()) } yyVAL.union = yyLOCAL - case 1721: + case 1724: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:10979 +//line mysql_sql.y:10995 { yyLOCAL = tree.NewComparisonExpr(yyDollar[2].comparisonOpUnion(), yyDollar[1].exprUnion(), yyDollar[3].exprUnion()) } yyVAL.union = yyLOCAL - case 1722: + case 1725: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:10983 +//line mysql_sql.y:10999 { yyLOCAL = tree.NewSubqueryComparisonExpr(yyDollar[2].comparisonOpUnion(), yyDollar[3].comparisonOpUnion(), yyDollar[1].exprUnion(), yyDollar[4].subqueryUnion()) yyLOCAL = tree.NewSubqueryComparisonExpr(yyDollar[2].comparisonOpUnion(), yyDollar[3].comparisonOpUnion(), yyDollar[1].exprUnion(), yyDollar[4].subqueryUnion()) } yyVAL.union = yyLOCAL - case 1724: + case 1727: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:10991 +//line mysql_sql.y:11007 { yyLOCAL = tree.NewComparisonExpr(tree.IN, yyDollar[1].exprUnion(), yyDollar[3].exprUnion()) } yyVAL.union = yyLOCAL - case 1725: + case 1728: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:10995 +//line mysql_sql.y:11011 { yyLOCAL = tree.NewComparisonExpr(tree.NOT_IN, yyDollar[1].exprUnion(), yyDollar[4].exprUnion()) } yyVAL.union = yyLOCAL - case 1726: + case 1729: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:10999 +//line mysql_sql.y:11015 { yyLOCAL = tree.NewComparisonExprWithEscape(tree.LIKE, yyDollar[1].exprUnion(), yyDollar[3].exprUnion(), yyDollar[4].exprUnion()) } yyVAL.union = yyLOCAL - case 1727: + case 1730: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:11003 +//line mysql_sql.y:11019 { yyLOCAL = tree.NewComparisonExprWithEscape(tree.NOT_LIKE, yyDollar[1].exprUnion(), yyDollar[4].exprUnion(), yyDollar[5].exprUnion()) } yyVAL.union = yyLOCAL - case 1728: + case 1731: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:11007 +//line mysql_sql.y:11023 { yyLOCAL = tree.NewComparisonExprWithEscape(tree.ILIKE, yyDollar[1].exprUnion(), yyDollar[3].exprUnion(), yyDollar[4].exprUnion()) } yyVAL.union = yyLOCAL - case 1729: + case 1732: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:11011 +//line mysql_sql.y:11027 { yyLOCAL = tree.NewComparisonExprWithEscape(tree.NOT_ILIKE, yyDollar[1].exprUnion(), yyDollar[4].exprUnion(), yyDollar[5].exprUnion()) } yyVAL.union = yyLOCAL - case 1730: + case 1733: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:11015 +//line mysql_sql.y:11031 { yyLOCAL = tree.NewComparisonExpr(tree.REG_MATCH, yyDollar[1].exprUnion(), yyDollar[3].exprUnion()) } yyVAL.union = yyLOCAL - case 1731: + case 1734: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:11019 +//line mysql_sql.y:11035 { yyLOCAL = tree.NewComparisonExpr(tree.NOT_REG_MATCH, yyDollar[1].exprUnion(), yyDollar[4].exprUnion()) } yyVAL.union = yyLOCAL - case 1732: + case 1735: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:11023 +//line mysql_sql.y:11039 { yyLOCAL = tree.NewRangeCond(false, yyDollar[1].exprUnion(), yyDollar[3].exprUnion(), yyDollar[5].exprUnion()) } yyVAL.union = yyLOCAL - case 1733: + case 1736: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:11027 +//line mysql_sql.y:11043 { yyLOCAL = tree.NewRangeCond(true, yyDollar[1].exprUnion(), yyDollar[4].exprUnion(), yyDollar[6].exprUnion()) } yyVAL.union = yyLOCAL - case 1735: + case 1738: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:11033 +//line mysql_sql.y:11049 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1736: + case 1739: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:11037 +//line mysql_sql.y:11053 { yyLOCAL = yyDollar[2].exprUnion() } yyVAL.union = yyLOCAL - case 1737: + case 1740: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:11043 +//line mysql_sql.y:11059 { yyLOCAL = yyDollar[1].tupleUnion() } yyVAL.union = yyLOCAL - case 1738: + case 1741: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:11047 +//line mysql_sql.y:11063 { yyLOCAL = yyDollar[1].subqueryUnion() } yyVAL.union = yyLOCAL - case 1739: + case 1742: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.ComparisonOp -//line mysql_sql.y:11054 +//line mysql_sql.y:11070 { yyLOCAL = tree.ALL } yyVAL.union = yyLOCAL - case 1740: + case 1743: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.ComparisonOp -//line mysql_sql.y:11058 +//line mysql_sql.y:11074 { yyLOCAL = tree.ANY } yyVAL.union = yyLOCAL - case 1741: + case 1744: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.ComparisonOp -//line mysql_sql.y:11062 +//line mysql_sql.y:11078 { yyLOCAL = tree.SOME } yyVAL.union = yyLOCAL - case 1742: + case 1745: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.ComparisonOp -//line mysql_sql.y:11068 +//line mysql_sql.y:11084 { yyLOCAL = tree.EQUAL } yyVAL.union = yyLOCAL - case 1743: + case 1746: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.ComparisonOp -//line mysql_sql.y:11072 +//line mysql_sql.y:11088 { yyLOCAL = tree.LESS_THAN } yyVAL.union = yyLOCAL - case 1744: + case 1747: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.ComparisonOp -//line mysql_sql.y:11076 +//line mysql_sql.y:11092 { yyLOCAL = tree.GREAT_THAN } yyVAL.union = yyLOCAL - case 1745: + case 1748: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.ComparisonOp -//line mysql_sql.y:11080 +//line mysql_sql.y:11096 { yyLOCAL = tree.LESS_THAN_EQUAL } yyVAL.union = yyLOCAL - case 1746: + case 1749: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.ComparisonOp -//line mysql_sql.y:11084 +//line mysql_sql.y:11100 { yyLOCAL = tree.GREAT_THAN_EQUAL } yyVAL.union = yyLOCAL - case 1747: + case 1750: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.ComparisonOp -//line mysql_sql.y:11088 +//line mysql_sql.y:11104 { yyLOCAL = tree.NOT_EQUAL } yyVAL.union = yyLOCAL - case 1748: + case 1751: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.ComparisonOp -//line mysql_sql.y:11092 +//line mysql_sql.y:11108 { yyLOCAL = tree.NULL_SAFE_EQUAL } yyVAL.union = yyLOCAL - case 1749: + case 1752: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.ColumnAttribute -//line mysql_sql.y:11098 +//line mysql_sql.y:11114 { yyLOCAL = tree.NewAttributePrimaryKey() } yyVAL.union = yyLOCAL - case 1750: + case 1753: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.ColumnAttribute -//line mysql_sql.y:11102 +//line mysql_sql.y:11118 { yyLOCAL = tree.NewAttributeUniqueKey() } yyVAL.union = yyLOCAL - case 1751: + case 1754: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.ColumnAttribute -//line mysql_sql.y:11106 +//line mysql_sql.y:11122 { yyLOCAL = tree.NewAttributeUnique() } yyVAL.union = yyLOCAL - case 1752: + case 1755: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.ColumnAttribute -//line mysql_sql.y:11110 +//line mysql_sql.y:11126 { yyLOCAL = tree.NewAttributeKey() } yyVAL.union = yyLOCAL - case 1753: + case 1756: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:11116 +//line mysql_sql.y:11132 { str := fmt.Sprintf("%v", yyDollar[1].item) switch v := yyDollar[1].item.(type) { @@ -24517,35 +24539,35 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1754: + case 1757: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:11129 +//line mysql_sql.y:11145 { fval := yyDollar[1].item.(float64) yyLOCAL = tree.NewNumVal(fval, yylex.(*Lexer).scanner.LastToken, false, tree.P_float64) } yyVAL.union = yyLOCAL - case 1755: + case 1758: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:11134 +//line mysql_sql.y:11150 { yyLOCAL = tree.NewNumVal(yyDollar[1].str, yyDollar[1].str, false, tree.P_decimal) } yyVAL.union = yyLOCAL - case 1756: + case 1759: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:11140 +//line mysql_sql.y:11156 { yyLOCAL = tree.NewNumVal(yyDollar[1].str, yyDollar[1].str, false, tree.P_char) } yyVAL.union = yyLOCAL - case 1757: + case 1760: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:11144 +//line mysql_sql.y:11160 { str := fmt.Sprintf("%v", yyDollar[1].item) switch v := yyDollar[1].item.(type) { @@ -24559,51 +24581,51 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1758: + case 1761: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:11157 +//line mysql_sql.y:11173 { fval := yyDollar[1].item.(float64) yyLOCAL = tree.NewNumVal(fval, yylex.(*Lexer).scanner.LastToken, false, tree.P_float64) } yyVAL.union = yyLOCAL - case 1759: + case 1762: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:11162 +//line mysql_sql.y:11178 { yyLOCAL = tree.NewNumVal(true, "true", false, tree.P_bool) } yyVAL.union = yyLOCAL - case 1760: + case 1763: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:11166 +//line mysql_sql.y:11182 { yyLOCAL = tree.NewNumVal(false, "false", false, tree.P_bool) } yyVAL.union = yyLOCAL - case 1761: + case 1764: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:11170 +//line mysql_sql.y:11186 { yyLOCAL = tree.NewNumVal("null", "null", false, tree.P_null) } yyVAL.union = yyLOCAL - case 1762: + case 1765: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:11174 +//line mysql_sql.y:11190 { yyLOCAL = tree.NewNumVal(yyDollar[1].str, yyDollar[1].str, false, tree.P_hexnum) } yyVAL.union = yyLOCAL - case 1763: + case 1766: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:11178 +//line mysql_sql.y:11194 { if strings.HasPrefix(yyDollar[2].str, "0x") { yyDollar[2].str = yyDollar[2].str[2:] @@ -24611,69 +24633,69 @@ yydefault: yyLOCAL = tree.NewNumVal(yyDollar[2].str, yyDollar[2].str, false, tree.P_bit) } yyVAL.union = yyLOCAL - case 1764: + case 1767: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:11185 +//line mysql_sql.y:11201 { yyLOCAL = tree.NewNumVal(yyDollar[1].str, yyDollar[1].str, false, tree.P_decimal) } yyVAL.union = yyLOCAL - case 1765: + case 1768: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:11189 +//line mysql_sql.y:11205 { yyLOCAL = tree.NewNumVal(yyDollar[1].str, yyDollar[1].str, false, tree.P_bit) } yyVAL.union = yyLOCAL - case 1766: + case 1769: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:11193 +//line mysql_sql.y:11209 { yyLOCAL = tree.NewParamExpr(yylex.(*Lexer).GetParamIndex()) } yyVAL.union = yyLOCAL - case 1767: + case 1770: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.Expr -//line mysql_sql.y:11197 +//line mysql_sql.y:11213 { yyLOCAL = tree.NewNumVal(yyDollar[2].str, yyDollar[2].str, false, tree.P_ScoreBinary) } yyVAL.union = yyLOCAL - case 1768: + case 1771: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11204 +//line mysql_sql.y:11220 { yyLOCAL = yyDollar[1].columnTypeUnion() yyLOCAL.InternalType.Unsigned = yyDollar[2].unsignedOptUnion() yyLOCAL.InternalType.Zerofill = yyDollar[3].zeroFillOptUnion() } yyVAL.union = yyLOCAL - case 1772: + case 1775: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11215 +//line mysql_sql.y:11231 { yyLOCAL = yyDollar[1].columnTypeUnion() yyLOCAL.InternalType.DisplayWith = yyDollar[2].lengthOptUnion() } yyVAL.union = yyLOCAL - case 1773: + case 1776: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11220 +//line mysql_sql.y:11236 { yyLOCAL = yyDollar[1].columnTypeUnion() } yyVAL.union = yyLOCAL - case 1774: + case 1777: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11226 +//line mysql_sql.y:11242 { locale := "" yyLOCAL = &tree.T{ @@ -24686,10 +24708,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1775: + case 1778: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11238 +//line mysql_sql.y:11254 { locale := "" yyLOCAL = &tree.T{ @@ -24702,10 +24724,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1776: + case 1779: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11250 +//line mysql_sql.y:11266 { locale := "" yyLOCAL = &tree.T{ @@ -24718,10 +24740,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1777: + case 1780: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11262 +//line mysql_sql.y:11278 { locale := "" yyLOCAL = &tree.T{ @@ -24735,10 +24757,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1778: + case 1781: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11275 +//line mysql_sql.y:11291 { locale := "" yyLOCAL = &tree.T{ @@ -24752,10 +24774,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1779: + case 1782: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11288 +//line mysql_sql.y:11304 { locale := "" yyLOCAL = &tree.T{ @@ -24769,10 +24791,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1780: + case 1783: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11301 +//line mysql_sql.y:11317 { locale := "" yyLOCAL = &tree.T{ @@ -24786,10 +24808,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1781: + case 1784: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11314 +//line mysql_sql.y:11330 { locale := "" yyLOCAL = &tree.T{ @@ -24803,10 +24825,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1782: + case 1785: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11327 +//line mysql_sql.y:11343 { locale := "" yyLOCAL = &tree.T{ @@ -24820,10 +24842,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1783: + case 1786: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11340 +//line mysql_sql.y:11356 { locale := "" yyLOCAL = &tree.T{ @@ -24837,10 +24859,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1784: + case 1787: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11353 +//line mysql_sql.y:11369 { locale := "" yyLOCAL = &tree.T{ @@ -24854,10 +24876,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1785: + case 1788: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11366 +//line mysql_sql.y:11382 { locale := "" yyLOCAL = &tree.T{ @@ -24871,10 +24893,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1786: + case 1789: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11379 +//line mysql_sql.y:11395 { locale := "" yyLOCAL = &tree.T{ @@ -24888,10 +24910,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1787: + case 1790: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11392 +//line mysql_sql.y:11408 { locale := "" yyLOCAL = &tree.T{ @@ -24905,10 +24927,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1788: + case 1791: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11407 +//line mysql_sql.y:11423 { locale := "" if yyDollar[2].lengthScaleOptUnion().DisplayWith > 255 { @@ -24936,10 +24958,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1789: + case 1792: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11434 +//line mysql_sql.y:11450 { locale := "" if yyDollar[2].lengthScaleOptUnion().DisplayWith > 255 { @@ -24981,10 +25003,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1790: + case 1793: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11476 +//line mysql_sql.y:11492 { locale := "" if yyDollar[2].lengthScaleOptUnion().Scale != tree.NotDefineDec && yyDollar[2].lengthScaleOptUnion().Scale > yyDollar[2].lengthScaleOptUnion().DisplayWith { @@ -25021,10 +25043,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1791: + case 1794: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11512 +//line mysql_sql.y:11528 { locale := "" if yyDollar[2].lengthScaleOptUnion().Scale != tree.NotDefineDec && yyDollar[2].lengthScaleOptUnion().Scale > yyDollar[2].lengthScaleOptUnion().DisplayWith { @@ -25061,10 +25083,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1792: + case 1795: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11548 +//line mysql_sql.y:11564 { locale := "" yyLOCAL = &tree.T{ @@ -25080,10 +25102,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1793: + case 1796: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11565 +//line mysql_sql.y:11581 { locale := "" yyLOCAL = &tree.T{ @@ -25096,10 +25118,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1794: + case 1797: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11577 +//line mysql_sql.y:11593 { locale := "" if yyDollar[2].lengthOptUnion() < 0 || yyDollar[2].lengthOptUnion() > 6 { @@ -25120,10 +25142,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1795: + case 1798: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11597 +//line mysql_sql.y:11613 { locale := "" if yyDollar[2].lengthOptUnion() < 0 || yyDollar[2].lengthOptUnion() > 6 { @@ -25144,10 +25166,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1796: + case 1799: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11617 +//line mysql_sql.y:11633 { locale := "" if yyDollar[2].lengthOptUnion() < 0 || yyDollar[2].lengthOptUnion() > 6 { @@ -25168,10 +25190,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1797: + case 1800: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11637 +//line mysql_sql.y:11653 { locale := "" yyLOCAL = &tree.T{ @@ -25186,10 +25208,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1798: + case 1801: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11653 +//line mysql_sql.y:11669 { locale := "" yyLOCAL = &tree.T{ @@ -25203,10 +25225,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1799: + case 1802: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11666 +//line mysql_sql.y:11682 { locale := "" yyLOCAL = &tree.T{ @@ -25220,10 +25242,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1800: + case 1803: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11679 +//line mysql_sql.y:11695 { locale := "" yyLOCAL = &tree.T{ @@ -25237,10 +25259,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1801: + case 1804: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11692 +//line mysql_sql.y:11708 { locale := "" yyLOCAL = &tree.T{ @@ -25254,10 +25276,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1802: + case 1805: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11705 +//line mysql_sql.y:11721 { locale := "" yyLOCAL = &tree.T{ @@ -25270,10 +25292,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1803: + case 1806: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11717 +//line mysql_sql.y:11733 { locale := "" yyLOCAL = &tree.T{ @@ -25286,10 +25308,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1804: + case 1807: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11729 +//line mysql_sql.y:11745 { locale := "" yyLOCAL = &tree.T{ @@ -25302,10 +25324,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1805: + case 1808: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11741 +//line mysql_sql.y:11757 { locale := "" yyLOCAL = &tree.T{ @@ -25318,10 +25340,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1806: + case 1809: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11753 +//line mysql_sql.y:11769 { locale := "" yyLOCAL = &tree.T{ @@ -25334,10 +25356,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1807: + case 1810: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11765 +//line mysql_sql.y:11781 { locale := "" yyLOCAL = &tree.T{ @@ -25350,10 +25372,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1808: + case 1811: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11777 +//line mysql_sql.y:11793 { locale := "" yyLOCAL = &tree.T{ @@ -25366,10 +25388,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1809: + case 1812: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11789 +//line mysql_sql.y:11805 { locale := "" yyLOCAL = &tree.T{ @@ -25382,10 +25404,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1810: + case 1813: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11801 +//line mysql_sql.y:11817 { locale := "" yyLOCAL = &tree.T{ @@ -25398,10 +25420,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1811: + case 1814: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11813 +//line mysql_sql.y:11829 { locale := "" yyLOCAL = &tree.T{ @@ -25414,10 +25436,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1812: + case 1815: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11825 +//line mysql_sql.y:11841 { locale := "" yyLOCAL = &tree.T{ @@ -25431,10 +25453,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1813: + case 1816: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11838 +//line mysql_sql.y:11854 { locale := "" yyLOCAL = &tree.T{ @@ -25448,10 +25470,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1814: + case 1817: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11851 +//line mysql_sql.y:11867 { locale := "" yyLOCAL = &tree.T{ @@ -25465,10 +25487,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1815: + case 1818: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11864 +//line mysql_sql.y:11880 { locale := "" yyLOCAL = &tree.T{ @@ -25482,10 +25504,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1816: + case 1819: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11877 +//line mysql_sql.y:11893 { locale := "" yyLOCAL = &tree.T{ @@ -25499,20 +25521,20 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1817: + case 1820: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:11892 +//line mysql_sql.y:11908 { yyLOCAL = &tree.Do{ Exprs: yyDollar[2].exprsUnion(), } } yyVAL.union = yyLOCAL - case 1818: + case 1821: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:11900 +//line mysql_sql.y:11916 { yyLOCAL = &tree.Declare{ Variables: yyDollar[2].strsUnion(), @@ -25521,10 +25543,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1819: + case 1822: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL tree.Statement -//line mysql_sql.y:11909 +//line mysql_sql.y:11925 { yyLOCAL = &tree.Declare{ Variables: yyDollar[2].strsUnion(), @@ -25533,10 +25555,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1820: + case 1823: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *tree.T -//line mysql_sql.y:11919 +//line mysql_sql.y:11935 { locale := "" yyLOCAL = &tree.T{ @@ -25549,75 +25571,75 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1821: + case 1824: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []string -//line mysql_sql.y:11942 +//line mysql_sql.y:11958 { yyLOCAL = make([]string, 0, 4) yyLOCAL = append(yyLOCAL, yyDollar[1].str) } yyVAL.union = yyLOCAL - case 1822: + case 1825: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL []string -//line mysql_sql.y:11947 +//line mysql_sql.y:11963 { yyLOCAL = append(yyDollar[1].strsUnion(), yyDollar[3].str) } yyVAL.union = yyLOCAL - case 1823: + case 1826: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL int32 -//line mysql_sql.y:11953 +//line mysql_sql.y:11969 { yyLOCAL = 0 } yyVAL.union = yyLOCAL - case 1825: + case 1828: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL int32 -//line mysql_sql.y:11960 +//line mysql_sql.y:11976 { yyLOCAL = 0 } yyVAL.union = yyLOCAL - case 1826: + case 1829: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL int32 -//line mysql_sql.y:11964 +//line mysql_sql.y:11980 { yyLOCAL = int32(yyDollar[2].item.(int64)) } yyVAL.union = yyLOCAL - case 1827: + case 1830: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL int32 -//line mysql_sql.y:11969 +//line mysql_sql.y:11985 { yyLOCAL = int32(-1) } yyVAL.union = yyLOCAL - case 1828: + case 1831: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL int32 -//line mysql_sql.y:11973 +//line mysql_sql.y:11989 { yyLOCAL = int32(yyDollar[2].item.(int64)) } yyVAL.union = yyLOCAL - case 1829: + case 1832: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL int32 -//line mysql_sql.y:11979 +//line mysql_sql.y:11995 { yyLOCAL = tree.GetDisplayWith(int32(yyDollar[2].item.(int64))) } yyVAL.union = yyLOCAL - case 1830: + case 1833: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL tree.LengthScaleOpt -//line mysql_sql.y:11985 +//line mysql_sql.y:12001 { yyLOCAL = tree.LengthScaleOpt{ DisplayWith: tree.NotDefineDisplayWidth, @@ -25625,10 +25647,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1831: + case 1834: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.LengthScaleOpt -//line mysql_sql.y:11992 +//line mysql_sql.y:12008 { yyLOCAL = tree.LengthScaleOpt{ DisplayWith: tree.GetDisplayWith(int32(yyDollar[2].item.(int64))), @@ -25636,10 +25658,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1832: + case 1835: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL tree.LengthScaleOpt -//line mysql_sql.y:11999 +//line mysql_sql.y:12015 { yyLOCAL = tree.LengthScaleOpt{ DisplayWith: tree.GetDisplayWith(int32(yyDollar[2].item.(int64))), @@ -25647,10 +25669,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1833: + case 1836: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL tree.LengthScaleOpt -//line mysql_sql.y:12008 +//line mysql_sql.y:12024 { yyLOCAL = tree.LengthScaleOpt{ DisplayWith: 38, // this is the default precision for decimal @@ -25658,10 +25680,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1834: + case 1837: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL tree.LengthScaleOpt -//line mysql_sql.y:12015 +//line mysql_sql.y:12031 { yyLOCAL = tree.LengthScaleOpt{ DisplayWith: tree.GetDisplayWith(int32(yyDollar[2].item.(int64))), @@ -25669,10 +25691,10 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1835: + case 1838: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL tree.LengthScaleOpt -//line mysql_sql.y:12022 +//line mysql_sql.y:12038 { yyLOCAL = tree.LengthScaleOpt{ DisplayWith: tree.GetDisplayWith(int32(yyDollar[2].item.(int64))), @@ -25680,52 +25702,52 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1836: + case 1839: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL bool -//line mysql_sql.y:12031 +//line mysql_sql.y:12047 { yyLOCAL = false } yyVAL.union = yyLOCAL - case 1837: + case 1840: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL bool -//line mysql_sql.y:12035 +//line mysql_sql.y:12051 { yyLOCAL = true } yyVAL.union = yyLOCAL - case 1838: + case 1841: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL bool -//line mysql_sql.y:12039 +//line mysql_sql.y:12055 { yyLOCAL = false } yyVAL.union = yyLOCAL - case 1839: + case 1842: yyDollar = yyS[yypt-0 : yypt+1] -//line mysql_sql.y:12045 +//line mysql_sql.y:12061 { } - case 1840: + case 1843: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL bool -//line mysql_sql.y:12047 +//line mysql_sql.y:12063 { yyLOCAL = true } yyVAL.union = yyLOCAL - case 1844: + case 1847: yyDollar = yyS[yypt-0 : yypt+1] -//line mysql_sql.y:12057 +//line mysql_sql.y:12073 { yyVAL.str = "" } - case 1845: + case 1848: yyDollar = yyS[yypt-1 : yypt+1] -//line mysql_sql.y:12061 +//line mysql_sql.y:12077 { yyVAL.str = string(yyDollar[1].str) } diff --git a/pkg/sql/parsers/dialect/mysql/mysql_sql.y b/pkg/sql/parsers/dialect/mysql/mysql_sql.y index 45cafdfceae7..ff68c318b932 100644 --- a/pkg/sql/parsers/dialect/mysql/mysql_sql.y +++ b/pkg/sql/parsers/dialect/mysql/mysql_sql.y @@ -280,7 +280,7 @@ import ( %token VALUES %token NEXT VALUE SHARE MODE %token SQL_NO_CACHE SQL_CACHE -%left JOIN STRAIGHT_JOIN LEFT RIGHT INNER OUTER CROSS NATURAL USE FORCE CROSS_L2 APPLY +%left JOIN STRAIGHT_JOIN LEFT RIGHT INNER OUTER CROSS NATURAL USE FORCE CROSS_L2 APPLY DEDUP %nonassoc LOWER_THAN_ON %nonassoc ON USING %left SUBQUERY_AS_EXPR @@ -632,7 +632,7 @@ import ( %type aliased_table_name %type union_op %type table_subquery -%type inner_join straight_join outer_join natural_join apply_type +%type inner_join straight_join outer_join natural_join apply_type dedup_join %type func_type_opt %type function_call_generic %type function_call_keyword @@ -5864,6 +5864,15 @@ join_table: Right: $3, } } +| table_reference dedup_join table_factor join_condition + { + $$ = &tree.JoinTableExpr{ + Left: $1, + JoinType: $2, + Right: $3, + Cond: $4, + } + } apply_table: table_reference apply_type table_factor @@ -5917,6 +5926,12 @@ outer_join: $$ = tree.JOIN_TYPE_RIGHT } +dedup_join: + DEDUP JOIN + { + $$ = tree.JOIN_TYPE_DEDUP + } + values_stmt: VALUES row_constructor_list order_by_opt limit_opt { @@ -10818,6 +10833,7 @@ name_confict: | DATE | DATABASE | DAY +| DEDUP | HOUR | IF | INTERVAL @@ -12105,6 +12121,7 @@ equal_opt: //| SUBJECT //| DATABASE //| DATABASES +//| DEDUP //| DEFAULT //| DELETE //| DESC @@ -12144,7 +12161,7 @@ equal_opt: //| LAST //| LEFT //| LIKE -//| ILIKE +//| ILIKE //| LIMIT //| LOCALTIME //| LOCALTIMESTAMP @@ -12172,7 +12189,7 @@ equal_opt: //| REQUIRE //| REPEAT //| ROW_COUNT -//| REFERENCES +//| REFERENCES //| RECURSIVE //| REVERSE //| SCHEMA diff --git a/pkg/sql/parsers/tree/select.go b/pkg/sql/parsers/tree/select.go index dc30dd19138c..75cd17ba0da8 100644 --- a/pkg/sql/parsers/tree/select.go +++ b/pkg/sql/parsers/tree/select.go @@ -433,6 +433,7 @@ const ( JOIN_TYPE_NATURAL_LEFT = "NATURAL LEFT" JOIN_TYPE_NATURAL_RIGHT = "NATURAL RIGHT" JOIN_TYPE_CROSS_L2 = "CROSS_L2" + JOIN_TYPE_DEDUP = "DEDUP" ) // the table expression diff --git a/pkg/sql/plan/base_binder.go b/pkg/sql/plan/base_binder.go index 88df6c60058d..99cb85f7c6c7 100644 --- a/pkg/sql/plan/base_binder.go +++ b/pkg/sql/plan/base_binder.go @@ -439,12 +439,12 @@ func (b *baseBinder) baseBindSubquery(astExpr *tree.Subquery, isRoot bool) (*Exp var err error switch subquery := astExpr.Select.(type) { case *tree.ParenSelect: - nodeID, err = b.builder.buildSelect(subquery.Select, subCtx, false) + nodeID, err = b.builder.bindSelect(subquery.Select, subCtx, false) if err != nil { return nil, err } case *tree.Select: - nodeID, err = b.builder.buildSelect(subquery, subCtx, false) + nodeID, err = b.builder.bindSelect(subquery, subCtx, false) if err != nil { return nil, err } diff --git a/pkg/sql/plan/bind_delete.go b/pkg/sql/plan/bind_delete.go new file mode 100644 index 000000000000..34260ce54cad --- /dev/null +++ b/pkg/sql/plan/bind_delete.go @@ -0,0 +1,451 @@ +// Copyright 2021 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package plan + +import ( + "fmt" + + "github.com/matrixorigin/matrixone/pkg/catalog" + "github.com/matrixorigin/matrixone/pkg/common/moerr" + "github.com/matrixorigin/matrixone/pkg/logutil" + "github.com/matrixorigin/matrixone/pkg/pb/plan" + "github.com/matrixorigin/matrixone/pkg/sql/parsers/tree" +) + +func (builder *QueryBuilder) bindDelete(stmt *tree.Delete, bindCtx *BindContext) (int32, error) { + if len(stmt.Tables) != 1 { + return 0, moerr.NewUnsupportedDML(builder.GetContext(), "delete from multiple tables") + } + + //FIXME: optimize truncate table? + if stmt.Where == nil && stmt.Limit == nil { + return 0, moerr.NewUnsupportedDML(builder.GetContext(), "rewrite to truncate table") + } + + aliasMap := make(map[string][2]string) + for _, tbl := range stmt.TableRefs { + getAliasToName(builder.compCtx, tbl, "", aliasMap) + } + + dmlCtx := NewDMLContext() + err := dmlCtx.ResolveTables(builder.compCtx, stmt.Tables, stmt.With, aliasMap, false) + if err != nil { + return 0, err + } + + var selectList []tree.SelectExpr + colName2Idx := make([]map[string]int32, len(stmt.Tables)) + + getResolveExpr := func(alias string) { + defIdx := dmlCtx.aliasMap[alias] + colName2Idx[defIdx] = make(map[string]int32) + for _, col := range dmlCtx.tableDefs[defIdx].Cols { + colName2Idx[defIdx][col.Name] = int32(len(selectList)) + selectExpr := tree.NewUnresolvedName(tree.NewCStr(alias, bindCtx.lower), tree.NewCStr(col.Name, 1)) + selectList = append(selectList, tree.SelectExpr{ + Expr: selectExpr, + }) + } + } + + for _, tbl := range stmt.Tables { + if aliasTbl, ok := tbl.(*tree.AliasedTableExpr); ok { + alias := string(aliasTbl.As.Alias) + if alias != "" { + getResolveExpr(alias) + } else { + astTbl := aliasTbl.Expr.(*tree.TableName) + getResolveExpr(string(astTbl.ObjectName)) + } + } else if astTbl, ok := tbl.(*tree.TableName); ok { + getResolveExpr(string(astTbl.ObjectName)) + } + } + + fromTables := &tree.From{} + if stmt.TableRefs != nil { + fromTables.Tables = stmt.TableRefs + } else { + fromTables.Tables = stmt.Tables + } + + astSelect := &tree.Select{ + Select: &tree.SelectClause{ + Distinct: false, + Exprs: selectList, + From: fromTables, + Where: stmt.Where, + }, + OrderBy: stmt.OrderBy, + Limit: stmt.Limit, + With: stmt.With, + } + + selectCtx := NewBindContext(builder, bindCtx) + lastNodeID, err := builder.bindSelect(astSelect, selectCtx, false) + if err != nil { + return 0, err + } + + selectNode := builder.qry.Nodes[lastNodeID] + if selectNode.NodeType != plan.Node_PROJECT { + return 0, moerr.NewUnsupportedDML(builder.GetContext(), "malformed select node") + } + + if dmlCtx.tableDefs[0].Partition != nil { + selectNodeTag := selectNode.BindingTags[0] + var partitionExpr *Expr + partitionExpr, err = getRemapParitionExpr(dmlCtx.tableDefs[0], selectNodeTag, colName2Idx[0], false) + if err != nil { + return -1, err + } + + projectList := make([]*plan.Expr, len(selectNode.ProjectList)+1) + for i := range selectNode.ProjectList { + projectList[i] = &plan.Expr{ + Typ: selectNode.ProjectList[i].Typ, + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + RelPos: selectNodeTag, + ColPos: int32(i), + }, + }, + } + } + projectList[len(selectNode.ProjectList)] = partitionExpr + lastNodeID = builder.appendNode(&plan.Node{ + NodeType: plan.Node_PROJECT, + Children: []int32{lastNodeID}, + BindingTags: []int32{builder.genNewTag()}, + ProjectList: projectList, + }, bindCtx) + selectNode = builder.qry.Nodes[lastNodeID] + } + + idxScanNodes := make([][]*plan.Node, len(dmlCtx.tableDefs)) + + for i, tableDef := range dmlCtx.tableDefs { + idxDefs := tableDef.Indexes + idxScanNodes[i] = make([]*plan.Node, len(idxDefs)) + + for j, idxDef := range idxDefs { + if !idxDef.TableExist { + continue + } + + if !catalog.IsRegularIndexAlgo(idxDef.IndexAlgo) { + return 0, moerr.NewUnsupportedDML(builder.GetContext(), "have vector index table") + } + + idxObjRef, idxTableDef := builder.compCtx.Resolve(dmlCtx.objRefs[0].SchemaName, idxDef.IndexTableName, bindCtx.snapshot) + if len(idxTableDef.Name2ColIndex) == 0 { + idxTableDef.Name2ColIndex = make(map[string]int32) + for colIdx, col := range idxTableDef.Cols { + idxTableDef.Name2ColIndex[col.Name] = int32(colIdx) + } + } + idxTag := builder.genNewTag() + builder.addNameByColRef(idxTag, idxTableDef) + + idxScanNodes[i][j] = &plan.Node{ + NodeType: plan.Node_TABLE_SCAN, + TableDef: idxTableDef, + ObjRef: idxObjRef, + BindingTags: []int32{idxTag}, + ScanSnapshot: bindCtx.snapshot, + } + idxTableNodeID := builder.appendNode(idxScanNodes[i][j], bindCtx) + + rightPkPos := idxTableDef.Name2ColIndex[catalog.IndexTableIndexColName] + pkTyp := idxTableDef.Cols[rightPkPos].Typ + + rightExpr := &plan.Expr{ + Typ: pkTyp, + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + RelPos: idxTag, + ColPos: rightPkPos, + }, + }, + } + + var leftExpr *plan.Expr + + argsLen := len(idxDef.Parts) + if argsLen == 1 { + leftExpr = &plan.Expr{ + Typ: pkTyp, + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + RelPos: selectNode.BindingTags[0], + ColPos: int32(colName2Idx[i][idxDef.Parts[0]]), + }, + }, + } + } else { + args := make([]*plan.Expr, argsLen) + var colPos int32 + var ok bool + for k, colName := range idxDef.Parts { + if colPos, ok = colName2Idx[i][catalog.ResolveAlias(colName)]; !ok { + errMsg := fmt.Sprintf("bind delete err, can not find colName = %s", colName) + return 0, moerr.NewInternalError(builder.GetContext(), errMsg) + } + args[k] = &plan.Expr{ + Typ: selectNode.ProjectList[colPos].Typ, + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + RelPos: selectNode.BindingTags[0], + ColPos: colPos, + }, + }, + } + } + + fnName := "serial" + if !idxDef.Unique { + fnName = "serial_full" + } + leftExpr, _ = BindFuncExprImplByPlanExpr(builder.GetContext(), fnName, args) + } + + joinCond, _ := BindFuncExprImplByPlanExpr(builder.GetContext(), "=", []*plan.Expr{ + leftExpr, + rightExpr, + }) + + joinType := plan.Node_INNER + if idxDef.Unique { + joinType = plan.Node_LEFT + } + + lastNodeID = builder.appendNode(&plan.Node{ + NodeType: plan.Node_JOIN, + Children: []int32{lastNodeID, idxTableNodeID}, + JoinType: joinType, + OnList: []*plan.Expr{joinCond}, + }, bindCtx) + } + } + + dmlNode := &plan.Node{ + NodeType: plan.Node_MULTI_UPDATE, + BindingTags: []int32{builder.genNewTag()}, + } + selectNodeTag := selectNode.BindingTags[0] + var lockTargets []*plan.LockTarget + + for i, tableDef := range dmlCtx.tableDefs { + pkPos := colName2Idx[i][tableDef.Pkey.PkeyColName] + rowIDPos := colName2Idx[i][catalog.Row_ID] + updateCtx := &plan.UpdateCtx{ + TableDef: DeepCopyTableDef(tableDef, true), + ObjRef: DeepCopyObjectRef(dmlCtx.objRefs[i]), + OldPartitionIdx: -1, + NewPartitionIdx: -1, + } + + if tableDef.Partition != nil { + partitionTableIDs, partitionTableNames := getPartitionInfos(builder.compCtx, dmlCtx.objRefs[i], tableDef) + updateCtx.OldPartitionIdx = int32(len(selectNode.ProjectList) - 1) + updateCtx.PartitionTableIds = partitionTableIDs + updateCtx.PartitionTableNames = partitionTableNames + dmlNode.BindingTags = append(dmlNode.BindingTags, selectNodeTag) + } + + for _, col := range tableDef.Cols { + if col.Name == tableDef.Pkey.PkeyColName { + lockTarget := &plan.LockTarget{ + TableId: tableDef.TblId, + PrimaryColIdxInBat: int32(pkPos), + PrimaryColRelPos: selectNodeTag, + PrimaryColTyp: col.Typ, + } + if tableDef.Partition != nil { + lockTarget.IsPartitionTable = true + lockTarget.PartitionTableIds = updateCtx.PartitionTableIds + lockTarget.FilterColIdxInBat = updateCtx.OldPartitionIdx + lockTarget.FilterColRelPos = selectNodeTag + } + lockTargets = append(lockTargets, lockTarget) + break + } + } + + updateCtx.DeleteCols = []plan.ColRef{ + { + RelPos: selectNodeTag, + ColPos: rowIDPos, + }, + { + RelPos: selectNodeTag, + ColPos: pkPos, + }, + } + + dmlNode.UpdateCtxList = append(dmlNode.UpdateCtxList, updateCtx) + + //lastNodeID = builder.appendNode(&plan.Node{ + // NodeType: plan.Node_DELETE, + // Children: []int32{lastNodeID}, + // DeleteCtx: &plan.DeleteCtx{ + // TableDef: DeepCopyTableDef(tableDef, true), + // Ref: DeepCopyObjectRef(tblInfo.objRef[i]), + // AddAffectedRows: delNodeInfo.addAffectedRows, + // IsClusterTable: delNodeInfo.IsClusterTable, + // PartitionTableIds: delNodeInfo.partTableIDs, + // PartitionTableNames: delNodeInfo.partTableNames, + // PartitionIdx: int32(delNodeInfo.partitionIdx), + // }, + // InsertDeleteCols: []*plan.Expr{ + // { + // Typ: selectNode.ProjectList[pkPos].Typ, + // Expr: &plan.Expr_Col{ + // Col: &plan.ColRef{ + // RelPos: selectNode.BindingTags[0], + // ColPos: pkPos, + // }, + // }, + // }, + // { + // Typ: selectNode.ProjectList[rowIDPos].Typ, + // Expr: &plan.Expr_Col{ + // Col: &plan.ColRef{ + // RelPos: selectNode.BindingTags[0], + // ColPos: rowIDPos, + // }, + // }, + // }, + // }, + //}, ctx) + + for j, idxNode := range idxScanNodes[i] { + if idxNode == nil { + continue + } + + pkPos := idxNode.TableDef.Name2ColIndex[idxNode.TableDef.Pkey.PkeyColName] + rowIDPos := idxNode.TableDef.Name2ColIndex[catalog.Row_ID] + + if tableDef.Indexes[j].Unique { + for _, col := range idxNode.TableDef.Cols { + if col.Name == idxNode.TableDef.Pkey.PkeyColName { + lockTargets = append(lockTargets, &plan.LockTarget{ + TableId: idxNode.TableDef.TblId, + PrimaryColIdxInBat: int32(pkPos), + PrimaryColRelPos: idxNode.BindingTags[0], + PrimaryColTyp: col.Typ, + }) + break + } + } + } + + dmlNode.UpdateCtxList = append(dmlNode.UpdateCtxList, &plan.UpdateCtx{ + TableDef: DeepCopyTableDef(idxNode.TableDef, true), + ObjRef: DeepCopyObjectRef(idxNode.ObjRef), + OldPartitionIdx: -1, + NewPartitionIdx: -1, + DeleteCols: []plan.ColRef{ + { + RelPos: idxNode.BindingTags[0], + ColPos: rowIDPos, + }, + { + RelPos: idxNode.BindingTags[0], + ColPos: pkPos, + }, + }, + }) + + //lastNodeID = builder.appendNode(&plan.Node{ + // NodeType: plan.Node_DELETE, + // Children: []int32{lastNodeID}, + // DeleteCtx: &plan.DeleteCtx{ + // TableDef: DeepCopyTableDef(idxNode.TableDef, true), + // Ref: DeepCopyObjectRef(idxNode.ObjRef), + // PrimaryKeyIdx: int32(delNodeInfo.pkPos), + // CanTruncate: canTruncate, + // AddAffectedRows: delNodeInfo.addAffectedRows, + // IsClusterTable: delNodeInfo.IsClusterTable, + // PartitionTableIds: delNodeInfo.partTableIDs, + // PartitionTableNames: delNodeInfo.partTableNames, + // PartitionIdx: int32(delNodeInfo.partitionIdx), + // }, + // InsertDeleteCols: []*plan.Expr{ + // { + // Typ: idxNode.TableDef.Cols[pkPos].Typ, + // Expr: &plan.Expr_Col{ + // Col: &plan.ColRef{ + // RelPos: idxNode.BindingTags[0], + // ColPos: pkPos, + // }, + // }, + // }, + // { + // Typ: idxNode.TableDef.Cols[rowIDPos].Typ, + // Expr: &plan.Expr_Col{ + // Col: &plan.ColRef{ + // RelPos: idxNode.BindingTags[0], + // ColPos: rowIDPos, + // }, + // }, + // }, + // }, + //}, ctx) + } + } + + if len(lockTargets) > 0 { + lastNodeID = builder.appendNode(&plan.Node{ + NodeType: plan.Node_LOCK_OP, + Children: []int32{lastNodeID}, + TableDef: dmlCtx.tableDefs[0], + BindingTags: []int32{builder.genNewTag()}, + LockTargets: lockTargets, + }, bindCtx) + + reCheckifNeedLockWholeTable(builder) + } + + dmlNode.Children = append(dmlNode.Children, lastNodeID) + lastNodeID = builder.appendNode(dmlNode, bindCtx) + + return lastNodeID, err +} + +func (builder *QueryBuilder) updateLocksOnDemand(nodeID int32) { + lockService := builder.compCtx.GetProcess().Base.LockService + if lockService == nil { + // MockCompilerContext + return + } + lockconfig := lockService.GetConfig() + + node := builder.qry.Nodes[nodeID] + if node.NodeType != plan.Node_LOCK_OP { + for _, childID := range node.Children { + builder.updateLocksOnDemand(childID) + } + } else if !node.LockTargets[0].LockTable && node.Stats.Outcnt > float64(lockconfig.MaxLockRowCount) { + node.LockTargets[0].LockTable = true + logutil.Infof("Row lock upgraded to table lock for SQL : %s", builder.compCtx.GetRootSql()) + logutil.Infof("the outcnt stats is %f", node.Stats.Outcnt) + + if len(node.LockTargets) > 1 && node.LockTargets[1].IsPartitionTable { + node.LockTargets[1].LockTable = true + } + } +} diff --git a/pkg/sql/plan/bind_insert.go b/pkg/sql/plan/bind_insert.go new file mode 100644 index 000000000000..1d556b567567 --- /dev/null +++ b/pkg/sql/plan/bind_insert.go @@ -0,0 +1,917 @@ +// Copyright 2021 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package plan + +import ( + "fmt" + "strings" + + "github.com/google/uuid" + "github.com/matrixorigin/matrixone/pkg/catalog" + "github.com/matrixorigin/matrixone/pkg/common/moerr" + "github.com/matrixorigin/matrixone/pkg/container/batch" + "github.com/matrixorigin/matrixone/pkg/container/types" + "github.com/matrixorigin/matrixone/pkg/container/vector" + "github.com/matrixorigin/matrixone/pkg/pb/plan" + "github.com/matrixorigin/matrixone/pkg/sql/parsers/tree" + "github.com/matrixorigin/matrixone/pkg/sql/util" +) + +func (builder *QueryBuilder) bindInsert(stmt *tree.Insert, bindCtx *BindContext) (int32, error) { + var onDupAction plan.Node_OnDuplicateAction + if len(stmt.OnDuplicateUpdate) == 0 { + onDupAction = plan.Node_ERROR + } else if len(stmt.OnDuplicateUpdate) == 1 && stmt.OnDuplicateUpdate[0] == nil { + onDupAction = plan.Node_IGNORE + } else { + //onDupAction = plan.Node_UPDATE + return 0, moerr.NewUnsupportedDML(builder.GetContext(), "on duplicate key update") + } + + dmlCtx := NewDMLContext() + err := dmlCtx.ResolveTables(builder.compCtx, tree.TableExprs{stmt.Table}, nil, nil, true) + if err != nil { + return 0, err + } + + // clusterTable, err := getAccountInfoOfClusterTable(ctx, stmt.Accounts, tableDef, tblInfo.isClusterTable[0]) + // if err != nil { + // return 0, err + // } + // if len(stmt.OnDuplicateUpdate) > 0 && clusterTable.IsClusterTable { + // return 0, moerr.NewNotSupported(builder.compCtx.GetContext(), "INSERT ... ON DUPLICATE KEY UPDATE ... for cluster table") + // } + + if stmt.IsRestore { + builder.isRestore = true + oldSnapshot := builder.compCtx.GetSnapshot() + builder.compCtx.SetSnapshot(&Snapshot{ + Tenant: &plan.SnapshotTenant{ + TenantName: "xxx", + TenantID: stmt.FromDataTenantID, + }, + }) + defer func() { + builder.compCtx.SetSnapshot(oldSnapshot) + }() + } + + lastNodeID, colName2Idx, skipUniqueIdx, err := builder.initInsertStmt(bindCtx, stmt, dmlCtx.objRefs[0], dmlCtx.tableDefs[0]) + if err != nil { + return 0, err + } + + return builder.appendDedupAndMultiUpdateNodesForBindInsert(bindCtx, dmlCtx, lastNodeID, colName2Idx, skipUniqueIdx, onDupAction) +} + +func (builder *QueryBuilder) appendDedupAndMultiUpdateNodesForBindInsert( + bindCtx *BindContext, + dmlCtx *DMLContext, + lastNodeID int32, + colName2Idx map[string]int32, + skipUniqueIdx []bool, + onDupAction plan.Node_OnDuplicateAction, +) (int32, error) { + var err error + selectNode := builder.qry.Nodes[lastNodeID] + selectNodeTag := selectNode.BindingTags[0] + partitionExprIdx := int32(len(selectNode.ProjectList) - 1) + idxObjRefs := make([][]*plan.ObjectRef, len(dmlCtx.tableDefs)) + idxTableDefs := make([][]*plan.TableDef, len(dmlCtx.tableDefs)) + + for _, tableDef := range dmlCtx.tableDefs { + if tableDef.TableType != catalog.SystemOrdinaryRel && + tableDef.TableType != catalog.SystemIndexRel { + return 0, moerr.NewUnsupportedDML(builder.GetContext(), "insert into vector/text index table") + } + + for _, idxDef := range tableDef.Indexes { + if !catalog.IsRegularIndexAlgo(idxDef.IndexAlgo) { + return 0, moerr.NewUnsupportedDML(builder.GetContext(), "have vector index table") + } + } + } + + //lock main table + lockTargets := make([]*plan.LockTarget, 0, len(dmlCtx.tableDefs[0].Indexes)+1) + mainTableDef := dmlCtx.tableDefs[0] + for _, col := range mainTableDef.Cols { + if col.Name == mainTableDef.Pkey.PkeyColName && mainTableDef.Pkey.PkeyColName != catalog.FakePrimaryKeyColName { + lockTarget := &plan.LockTarget{ + TableId: mainTableDef.TblId, + PrimaryColIdxInBat: int32(colName2Idx[mainTableDef.Name+"."+col.Name]), + PrimaryColRelPos: selectNodeTag, + PrimaryColTyp: col.Typ, + } + if mainTableDef.Partition != nil { + partitionTableIDs, _ := getPartitionInfos(builder.compCtx, dmlCtx.objRefs[0], mainTableDef) + lockTarget.IsPartitionTable = true + lockTarget.PartitionTableIds = partitionTableIDs + lockTarget.FilterColIdxInBat = partitionExprIdx + lockTarget.FilterColRelPos = selectNodeTag + } + lockTargets = append(lockTargets, lockTarget) + break + } + } + // lock unique key table + for j, idxDef := range mainTableDef.Indexes { + if !idxDef.TableExist || skipUniqueIdx[j] || !idxDef.Unique { + continue + } + _, idxTableDef := builder.compCtx.Resolve(dmlCtx.objRefs[0].SchemaName, idxDef.IndexTableName, bindCtx.snapshot) + var pkIdxInBat int32 + + if len(idxDef.Parts) == 1 { + pkIdxInBat = colName2Idx[mainTableDef.Name+"."+idxDef.Parts[0]] + } else { + pkIdxInBat = colName2Idx[idxTableDef.Name+"."+catalog.IndexTableIndexColName] + } + lockTarget := &plan.LockTarget{ + TableId: idxTableDef.TblId, + PrimaryColIdxInBat: pkIdxInBat, + PrimaryColRelPos: selectNodeTag, + PrimaryColTyp: selectNode.ProjectList[int(pkIdxInBat)].Typ, + } + lockTargets = append(lockTargets, lockTarget) + } + if len(lockTargets) > 0 { + lastNodeID = builder.appendNode(&plan.Node{ + NodeType: plan.Node_LOCK_OP, + Children: []int32{lastNodeID}, + TableDef: dmlCtx.tableDefs[0], + BindingTags: []int32{builder.genNewTag()}, + LockTargets: lockTargets, + }, bindCtx) + reCheckifNeedLockWholeTable(builder) + } + + // handle primary/unique key confliction + if builder.qry.LoadTag { + // load do not handle primary/unique key confliction + for i, tableDef := range dmlCtx.tableDefs { + idxObjRefs[i] = make([]*plan.ObjectRef, len(tableDef.Indexes)) + idxTableDefs[i] = make([]*plan.TableDef, len(tableDef.Indexes)) + + for j, idxDef := range tableDef.Indexes { + if !idxDef.TableExist || skipUniqueIdx[j] { + continue + } + + idxObjRefs[i][j], idxTableDefs[i][j] = builder.compCtx.Resolve(dmlCtx.objRefs[i].SchemaName, idxDef.IndexTableName, bindCtx.snapshot) + } + } + } else { + for i, tableDef := range dmlCtx.tableDefs { + pkName := tableDef.Pkey.PkeyColName + pkPos := tableDef.Name2ColIndex[pkName] + if pkName != catalog.FakePrimaryKeyColName { + scanTag := builder.genNewTag() + builder.addNameByColRef(scanTag, tableDef) + + scanNodeID := builder.appendNode(&plan.Node{ + NodeType: plan.Node_TABLE_SCAN, + TableDef: DeepCopyTableDef(tableDef, true), + ObjRef: dmlCtx.objRefs[i], + BindingTags: []int32{scanTag}, + ScanSnapshot: bindCtx.snapshot, + }, bindCtx) + + pkTyp := tableDef.Cols[pkPos].Typ + leftExpr := &plan.Expr{ + Typ: pkTyp, + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + RelPos: scanTag, + ColPos: pkPos, + }, + }, + } + + rightExpr := &plan.Expr{ + Typ: pkTyp, + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + RelPos: selectNode.BindingTags[0], + ColPos: colName2Idx[tableDef.Name+"."+pkName], + }, + }, + } + + joinCond, _ := BindFuncExprImplByPlanExpr(builder.GetContext(), "=", []*plan.Expr{ + leftExpr, + rightExpr, + }) + + var dedupColName string + dedupColTypes := make([]plan.Type, len(tableDef.Pkey.Names)) + + if len(tableDef.Pkey.Names) == 1 { + dedupColName = tableDef.Pkey.Names[0] + } else { + dedupColName = "(" + strings.Join(tableDef.Pkey.Names, ",") + ")" + } + + for j, part := range tableDef.Pkey.Names { + dedupColTypes[j] = tableDef.Cols[tableDef.Name2ColIndex[part]].Typ + } + + lastNodeID = builder.appendNode(&plan.Node{ + NodeType: plan.Node_JOIN, + Children: []int32{scanNodeID, lastNodeID}, + JoinType: plan.Node_DEDUP, + OnList: []*plan.Expr{joinCond}, + OnDuplicateAction: onDupAction, + DedupColName: dedupColName, + DedupColTypes: dedupColTypes, + }, bindCtx) + } + + idxObjRefs[i] = make([]*plan.ObjectRef, len(tableDef.Indexes)) + idxTableDefs[i] = make([]*plan.TableDef, len(tableDef.Indexes)) + + for j, idxDef := range tableDef.Indexes { + if !idxDef.TableExist || skipUniqueIdx[j] { + continue + } + + idxObjRefs[i][j], idxTableDefs[i][j] = builder.compCtx.Resolve(dmlCtx.objRefs[i].SchemaName, idxDef.IndexTableName, bindCtx.snapshot) + + if !idxDef.Unique { + continue + } + + idxTag := builder.genNewTag() + builder.addNameByColRef(idxTag, idxTableDefs[i][j]) + + idxScanNode := &plan.Node{ + NodeType: plan.Node_TABLE_SCAN, + TableDef: idxTableDefs[i][j], + ObjRef: idxObjRefs[i][j], + BindingTags: []int32{idxTag}, + ScanSnapshot: bindCtx.snapshot, + } + idxTableNodeID := builder.appendNode(idxScanNode, bindCtx) + + idxPkPos := idxTableDefs[i][j].Name2ColIndex[catalog.IndexTableIndexColName] + pkTyp := idxTableDefs[i][j].Cols[idxPkPos].Typ + + leftExpr := &plan.Expr{ + Typ: pkTyp, + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + RelPos: idxTag, + ColPos: idxPkPos, + }, + }, + } + + rightExpr := &plan.Expr{ + Typ: pkTyp, + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + RelPos: selectNode.BindingTags[0], + ColPos: colName2Idx[idxTableDefs[i][j].Name+"."+catalog.IndexTableIndexColName], + }, + }, + } + + joinCond, _ := BindFuncExprImplByPlanExpr(builder.GetContext(), "=", []*plan.Expr{ + leftExpr, + rightExpr, + }) + + var dedupColName string + dedupColTypes := make([]plan.Type, len(idxDef.Parts)) + + if len(idxDef.Parts) == 1 { + dedupColName = idxDef.Parts[0] + } else { + dedupColName = "(" + for j, part := range idxDef.Parts { + if j == 0 { + dedupColName += catalog.ResolveAlias(part) + } else { + dedupColName += "," + catalog.ResolveAlias(part) + } + } + dedupColName += ")" + } + + for j, part := range idxDef.Parts { + dedupColTypes[j] = tableDef.Cols[tableDef.Name2ColIndex[catalog.ResolveAlias(part)]].Typ + } + + lastNodeID = builder.appendNode(&plan.Node{ + NodeType: plan.Node_JOIN, + Children: []int32{idxTableNodeID, lastNodeID}, + JoinType: plan.Node_DEDUP, + OnList: []*plan.Expr{joinCond}, + OnDuplicateAction: onDupAction, + DedupColName: dedupColName, + DedupColTypes: dedupColTypes, + }, bindCtx) + } + } + } + + dmlNode := &plan.Node{ + NodeType: plan.Node_MULTI_UPDATE, + BindingTags: []int32{builder.genNewTag()}, + } + + for i, tableDef := range dmlCtx.tableDefs { + insertCols := make([]plan.ColRef, len(tableDef.Cols)-1) + updateCtx := &plan.UpdateCtx{ + ObjRef: dmlCtx.objRefs[i], + TableDef: tableDef, + InsertCols: insertCols, + OldPartitionIdx: -1, + NewPartitionIdx: -1, + } + if tableDef.Partition != nil { + partitionTableIDs, partitionTableNames := getPartitionInfos(builder.compCtx, dmlCtx.objRefs[i], tableDef) + updateCtx.NewPartitionIdx = partitionExprIdx + updateCtx.PartitionTableIds = partitionTableIDs + updateCtx.PartitionTableNames = partitionTableNames + dmlNode.BindingTags = append(dmlNode.BindingTags, selectNodeTag) + } + + for k, col := range tableDef.Cols { + if col.Name == catalog.Row_ID { + continue + } + insertCols[k].RelPos = selectNodeTag + insertCols[k].ColPos = colName2Idx[tableDef.Name+"."+col.Name] + } + + dmlNode.UpdateCtxList = append(dmlNode.UpdateCtxList, updateCtx) + + for j, idxTableDef := range idxTableDefs[i] { + if idxTableDef == nil { + continue + } + + idxInsertCols := make([]plan.ColRef, len(idxTableDef.Cols)-1) + for k, col := range idxTableDef.Cols { + if col.Name == catalog.Row_ID { + continue + } + idxInsertCols[k].RelPos = selectNodeTag + idxInsertCols[k].ColPos = int32(colName2Idx[idxTableDef.Name+"."+col.Name]) + } + + dmlNode.UpdateCtxList = append(dmlNode.UpdateCtxList, &plan.UpdateCtx{ + ObjRef: idxObjRefs[i][j], + TableDef: idxTableDef, + InsertCols: idxInsertCols, + OldPartitionIdx: -1, + NewPartitionIdx: -1, + }) + } + + } + + dmlNode.Children = append(dmlNode.Children, lastNodeID) + lastNodeID = builder.appendNode(dmlNode, bindCtx) + + return lastNodeID, err +} + +// getInsertColsFromStmt retrieves the list of column names to be inserted into a table +// based on the given INSERT statement and table definition. +// If the INSERT statement does not specify the columns, all columns except the fake primary key column +// will be included in the list. +// If the INSERT statement specifies the columns, it validates the column names against the table definition +// and returns an error if any of the column names are invalid. +// The function returns the list of insert columns and an error, if any. +func (builder *QueryBuilder) getInsertColsFromStmt(stmt *tree.Insert, tableDef *TableDef) ([]string, error) { + var insertColNames []string + colToIdx := make(map[string]int) + for i, col := range tableDef.Cols { + colToIdx[strings.ToLower(col.Name)] = i + } + if stmt.Columns == nil { + for _, col := range tableDef.Cols { + if !col.Hidden { + insertColNames = append(insertColNames, col.Name) + } + } + } else { + for _, column := range stmt.Columns { + colName := strings.ToLower(string(column)) + idx, ok := colToIdx[colName] + if !ok { + return nil, moerr.NewBadFieldError(builder.GetContext(), colName, tableDef.Name) + } + insertColNames = append(insertColNames, tableDef.Cols[idx].Name) + } + } + return insertColNames, nil +} + +func (builder *QueryBuilder) initInsertStmt(bindCtx *BindContext, stmt *tree.Insert, objRef *plan.ObjectRef, tableDef *plan.TableDef) (int32, map[string]int32, []bool, error) { + var ( + lastNodeID int32 + err error + ) + + // var uniqueCheckOnAutoIncr string + var insertColumns []string + + //var ifInsertFromUniqueColMap map[string]bool + if insertColumns, err = builder.getInsertColsFromStmt(stmt, tableDef); err != nil { + return 0, nil, nil, err + } + + var astSelect *tree.Select + switch selectImpl := stmt.Rows.Select.(type) { + // rewrite 'insert into tbl values (1,1)' to 'insert into tbl select * from (values row(1,1))' + case *tree.ValuesClause: + isAllDefault := false + if selectImpl.Rows[0] == nil { + isAllDefault = true + } + if isAllDefault { + for j, row := range selectImpl.Rows { + if row != nil { + return 0, nil, nil, moerr.NewWrongValueCountOnRow(builder.GetContext(), j+1) + } + } + } else { + colCount := len(insertColumns) + for j, row := range selectImpl.Rows { + if len(row) != colCount { + return 0, nil, nil, moerr.NewWrongValueCountOnRow(builder.GetContext(), j+1) + } + } + } + + // example1:insert into a values (); + // but it does not work at the case: + // insert into a(a) values (); insert into a values (0),(); + if isAllDefault && stmt.Columns != nil { + return 0, nil, nil, moerr.NewInvalidInput(builder.GetContext(), "insert values does not match the number of columns") + } + lastNodeID, err = builder.buildValueScan(isAllDefault, bindCtx, tableDef, selectImpl, insertColumns) + if err != nil { + return 0, nil, nil, err + } + + case *tree.SelectClause: + astSelect = stmt.Rows + + subCtx := NewBindContext(builder, bindCtx) + lastNodeID, err = builder.bindSelect(astSelect, subCtx, false) + if err != nil { + return 0, nil, nil, err + } + //ifInsertFromUniqueColMap = make(map[string]bool) + + case *tree.ParenSelect: + astSelect = selectImpl.Select + + subCtx := NewBindContext(builder, bindCtx) + lastNodeID, err = builder.bindSelect(astSelect, subCtx, false) + if err != nil { + return 0, nil, nil, err + } + // ifInsertFromUniqueColMap = make(map[string]bool) + + default: + return 0, nil, nil, moerr.NewInvalidInput(builder.GetContext(), "insert has unknown select statement") + } + + if err = builder.addBinding(lastNodeID, tree.AliasClause{Alias: derivedTableName}, bindCtx); err != nil { + return 0, nil, nil, err + } + + lastNode := builder.qry.Nodes[lastNodeID] + if len(insertColumns) != len(lastNode.ProjectList) { + return 0, nil, nil, moerr.NewInvalidInput(builder.GetContext(), "insert values does not match the number of columns") + } + + selectTag := lastNode.BindingTags[0] + + insertColToExpr := make(map[string]*plan.Expr) + for i, column := range insertColumns { + colIdx := tableDef.Name2ColIndex[column] + projExpr := &plan.Expr{ + Typ: lastNode.ProjectList[i].Typ, + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + RelPos: selectTag, + ColPos: int32(i), + }, + }, + } + if tableDef.Cols[colIdx].Typ.Id == int32(types.T_enum) { + projExpr, err = funcCastForEnumType(builder.GetContext(), projExpr, tableDef.Cols[colIdx].Typ) + if err != nil { + return 0, nil, nil, err + } + } else { + projExpr, err = forceCastExpr(builder.GetContext(), projExpr, tableDef.Cols[colIdx].Typ) + if err != nil { + return 0, nil, nil, err + } + } + insertColToExpr[column] = projExpr + } + + return builder.appendNodesForInsertStmt(bindCtx, lastNodeID, tableDef, objRef, insertColToExpr) +} + +func (builder *QueryBuilder) appendNodesForInsertStmt( + bindCtx *BindContext, + lastNodeID int32, + tableDef *TableDef, + objRef *ObjectRef, + insertColToExpr map[string]*Expr, +) (int32, map[string]int32, []bool, error) { + colName2Idx := make(map[string]int32) + hasAutoCol := false + for _, col := range tableDef.Cols { + if col.Typ.AutoIncr { + hasAutoCol = true + break + } + } + + projList1 := make([]*plan.Expr, 0, len(tableDef.Cols)-1) + projList2 := make([]*plan.Expr, 0, len(tableDef.Cols)-1) + projTag1 := builder.genNewTag() + preInsertTag := builder.genNewTag() + + var ( + compPkeyExpr *plan.Expr + clusterByExpr *plan.Expr + ) + + columnIsNull := make(map[string]bool) + + for i, col := range tableDef.Cols { + if oldExpr, exists := insertColToExpr[col.Name]; exists { + projList2 = append(projList2, &plan.Expr{ + Typ: oldExpr.Typ, + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + RelPos: projTag1, + ColPos: int32(len(projList1)), + }, + }, + }) + projList1 = append(projList1, oldExpr) + } else if col.Name == catalog.Row_ID { + continue + } else if col.Name == catalog.CPrimaryKeyColName { + //args := make([]*plan.Expr, len(tableDef.Pkey.Names)) + // + //for k, part := range tableDef.Pkey.Names { + // args[k] = DeepCopyExpr(insertColToExpr[part]) + //} + // + //compPkeyExpr, _ = BindFuncExprImplByPlanExpr(builder.GetContext(), "serial", args) + compPkeyExpr = makeCompPkeyExpr(tableDef, tableDef.Name2ColIndex) + projList2 = append(projList2, &plan.Expr{ + Typ: compPkeyExpr.Typ, + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + RelPos: preInsertTag, + ColPos: 0, + }, + }, + }) + } else if tableDef.ClusterBy != nil && col.Name == tableDef.ClusterBy.Name { + //names := util.SplitCompositeClusterByColumnName(tableDef.ClusterBy.Name) + //args := make([]*plan.Expr, len(names)) + // + //for k, part := range names { + // args[k] = DeepCopyExpr(insertColToExpr[part]) + //} + // + //clusterByExpr, _ = BindFuncExprImplByPlanExpr(builder.GetContext(), "serial_full", args) + clusterByExpr = makeClusterByExpr(tableDef, tableDef.Name2ColIndex) + projList2 = append(projList2, &plan.Expr{ + Typ: clusterByExpr.Typ, + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + RelPos: preInsertTag, + ColPos: 0, + }, + }, + }) + } else { + defExpr, err := getDefaultExpr(builder.GetContext(), col) + if err != nil { + return 0, nil, nil, err + } + + if !col.Typ.AutoIncr { + if lit := defExpr.GetLit(); lit != nil { + if lit.Isnull { + columnIsNull[col.Name] = true + } + } + } + + projList2 = append(projList2, &plan.Expr{ + Typ: defExpr.Typ, + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + RelPos: projTag1, + ColPos: int32(len(projList1)), + }, + }, + }) + projList1 = append(projList1, defExpr) + } + + colName2Idx[tableDef.Name+"."+col.Name] = int32(i) + } + + skipUniqueIdx := make([]bool, len(tableDef.Indexes)) + pkName := tableDef.Pkey.PkeyColName + pkPos := tableDef.Name2ColIndex[pkName] + for i, idxDef := range tableDef.Indexes { + if !idxDef.TableExist { + continue + } + + if idxDef.Unique { + skipUniqueIdx[i] = true + for _, part := range idxDef.Parts { + if !columnIsNull[part] { + skipUniqueIdx[i] = false + break + } + } + } + + idxTableName := idxDef.IndexTableName + colName2Idx[idxTableName+"."+catalog.IndexTablePrimaryColName] = pkPos + argsLen := len(idxDef.Parts) + if argsLen == 1 { + colName2Idx[idxTableName+"."+catalog.IndexTableIndexColName] = colName2Idx[tableDef.Name+"."+idxDef.Parts[0]] + } else { + args := make([]*plan.Expr, argsLen) + + var colPos int32 + var ok bool + for k := 0; k < argsLen; k++ { + if colPos, ok = colName2Idx[tableDef.Name+"."+catalog.ResolveAlias(idxDef.Parts[k])]; !ok { + errMsg := fmt.Sprintf("bind insert err, can not find colName = %s", idxDef.Parts[k]) + return 0, nil, nil, moerr.NewInternalError(builder.GetContext(), errMsg) + } + args[k] = DeepCopyExpr(projList2[colPos]) + } + + fnName := "serial" + if !idxDef.Unique { + fnName = "serial_full" + } + idxExpr, _ := BindFuncExprImplByPlanExpr(builder.GetContext(), fnName, args) + colName2Idx[idxTableName+"."+catalog.IndexTableIndexColName] = int32(len(projList2)) + projList2 = append(projList2, idxExpr) + } + } + + tmpCtx := NewBindContext(builder, bindCtx) + lastNodeID = builder.appendNode(&plan.Node{ + NodeType: plan.Node_PROJECT, + ProjectList: projList1, + Children: []int32{lastNodeID}, + BindingTags: []int32{projTag1}, + }, tmpCtx) + + if hasAutoCol || compPkeyExpr != nil || clusterByExpr != nil { + lastNodeID = builder.appendNode(&plan.Node{ + NodeType: plan.Node_PRE_INSERT, + Children: []int32{lastNodeID}, + PreInsertCtx: &plan.PreInsertCtx{ + Ref: objRef, + TableDef: tableDef, + HasAutoCol: hasAutoCol, + CompPkeyExpr: compPkeyExpr, + ClusterByExpr: clusterByExpr, + }, + BindingTags: []int32{preInsertTag}, + }, tmpCtx) + } + + if tableDef.Partition != nil { + partitionExpr, err := getRemapParitionExpr(tableDef, projTag1, colName2Idx, true) + if err != nil { + return 0, nil, nil, err + } + projList2 = append(projList2, partitionExpr) + } + + lastNodeID = builder.appendNode(&plan.Node{ + NodeType: plan.Node_PROJECT, + ProjectList: projList2, + Children: []int32{lastNodeID}, + BindingTags: []int32{builder.genNewTag()}, + }, tmpCtx) + + return lastNodeID, colName2Idx, skipUniqueIdx, nil +} + +func (builder *QueryBuilder) buildValueScan( + isAllDefault bool, + bindCtx *BindContext, + tableDef *TableDef, + stmt *tree.ValuesClause, + colNames []string, +) (int32, error) { + var err error + + proc := builder.compCtx.GetProcess() + lastTag := builder.genNewTag() + colCount := len(colNames) + rowsetData := &plan.RowsetData{ + Cols: make([]*plan.ColData, colCount), + } + for i := 0; i < colCount; i++ { + rowsetData.Cols[i] = new(plan.ColData) + } + valueScanTableDef := &plan.TableDef{ + TblId: 0, + Name: "", + Cols: make([]*plan.ColDef, colCount), + } + projectList := make([]*plan.Expr, colCount) + bat := batch.NewWithSize(len(colNames)) + + for i, colName := range colNames { + col := tableDef.Cols[tableDef.Name2ColIndex[colName]] + colTyp := makeTypeByPlan2Type(col.Typ) + vec := vector.NewVec(colTyp) + bat.Vecs[i] = vec + targetTyp := &plan.Expr{ + Typ: col.Typ, + Expr: &plan.Expr_T{ + T: &plan.TargetType{}, + }, + } + var defExpr *plan.Expr + if isAllDefault { + if err := vector.AppendMultiBytes(vec, nil, true, len(stmt.Rows), proc.Mp()); err != nil { + bat.Clean(proc.Mp()) + return 0, err + } + defExpr, err := getDefaultExpr(builder.GetContext(), col) + if err != nil { + return 0, err + } + defExpr, err = forceCastExpr2(builder.GetContext(), defExpr, colTyp, targetTyp) + if err != nil { + return 0, err + } + for j := range stmt.Rows { + rowsetData.Cols[i].Data = append(rowsetData.Cols[i].Data, &plan.RowsetExpr{ + Pos: -1, + RowPos: int32(j), + Expr: defExpr, + }) + } + } else { + binder := NewDefaultBinder(builder.GetContext(), nil, nil, col.Typ, nil) + binder.builder = builder + for j, r := range stmt.Rows { + if nv, ok := r[i].(*tree.NumVal); ok { + canInsert, err := util.SetInsertValue(proc, nv, vec) + if err != nil { + bat.Clean(proc.Mp()) + return 0, err + } + if canInsert { + continue + } + } + + if err := vector.AppendBytes(vec, nil, true, proc.Mp()); err != nil { + bat.Clean(proc.Mp()) + return 0, err + } + if _, ok := r[i].(*tree.DefaultVal); ok { + defExpr, err = getDefaultExpr(builder.GetContext(), col) + if err != nil { + bat.Clean(proc.Mp()) + return 0, err + } + } else if nv, ok := r[i].(*tree.ParamExpr); ok { + if !builder.isPrepareStatement { + bat.Clean(proc.Mp()) + return 0, moerr.NewInvalidInput(builder.GetContext(), "only prepare statement can use ? expr") + } + rowsetData.Cols[i].Data = append(rowsetData.Cols[i].Data, &plan.RowsetExpr{ + RowPos: int32(j), + Pos: int32(nv.Offset), + Expr: &plan.Expr{ + Typ: constTextType, + Expr: &plan.Expr_P{ + P: &plan.ParamRef{ + Pos: int32(nv.Offset), + }, + }, + }, + }) + continue + } else { + defExpr, err = binder.BindExpr(r[i], 0, true) + if err != nil { + bat.Clean(proc.Mp()) + return 0, err + } + if col.Typ.Id == int32(types.T_enum) { + defExpr, err = funcCastForEnumType(builder.GetContext(), defExpr, col.Typ) + if err != nil { + bat.Clean(proc.Mp()) + return 0, err + } + } + } + defExpr, err = forceCastExpr2(builder.GetContext(), defExpr, colTyp, targetTyp) + if err != nil { + return 0, err + } + if nv, ok := r[i].(*tree.ParamExpr); ok { + if !builder.isPrepareStatement { + bat.Clean(proc.Mp()) + return 0, moerr.NewInvalidInput(builder.GetContext(), "only prepare statement can use ? expr") + } + rowsetData.Cols[i].Data = append(rowsetData.Cols[i].Data, &plan.RowsetExpr{ + RowPos: int32(j), + Pos: int32(nv.Offset), + Expr: defExpr, + }) + continue + } + rowsetData.Cols[i].Data = append(rowsetData.Cols[i].Data, &plan.RowsetExpr{ + Pos: -1, + RowPos: int32(j), + Expr: defExpr, + }) + } + } + colName := fmt.Sprintf("column_%d", i) // like MySQL + valueScanTableDef.Cols[i] = &plan.ColDef{ + ColId: 0, + Name: colName, + Typ: col.Typ, + } + expr := &plan.Expr{ + Typ: col.Typ, + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + RelPos: lastTag, + ColPos: int32(i), + }, + }, + } + projectList[i] = expr + } + + bat.SetRowCount(len(stmt.Rows)) + rowsetData.RowCount = int32(len(stmt.Rows)) + nodeId, _ := uuid.NewV7() + scanNode := &plan.Node{ + NodeType: plan.Node_VALUE_SCAN, + RowsetData: rowsetData, + TableDef: valueScanTableDef, + BindingTags: []int32{lastTag}, + Uuid: nodeId[:], + } + if builder.isPrepareStatement { + proc.SetPrepareBatch(bat) + } else { + proc.SetValueScanBatch(nodeId, bat) + } + nodeID := builder.appendNode(scanNode, bindCtx) + if err = builder.addBinding(nodeID, tree.AliasClause{Alias: "_valuescan"}, bindCtx); err != nil { + return 0, err + } + + lastTag = builder.genNewTag() + nodeID = builder.appendNode(&plan.Node{ + NodeType: plan.Node_PROJECT, + ProjectList: projectList, + Children: []int32{nodeID}, + BindingTags: []int32{lastTag}, + }, bindCtx) + + return nodeID, nil +} diff --git a/pkg/sql/plan/bind_load.go b/pkg/sql/plan/bind_load.go new file mode 100644 index 000000000000..e8c81e1c701c --- /dev/null +++ b/pkg/sql/plan/bind_load.go @@ -0,0 +1,199 @@ +// Copyright 2021 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package plan + +import ( + "encoding/json" + + "github.com/matrixorigin/matrixone/pkg/common/moerr" + "github.com/matrixorigin/matrixone/pkg/pb/plan" + "github.com/matrixorigin/matrixone/pkg/sql/parsers/tree" +) + +func (builder *QueryBuilder) bindLoad(stmt *tree.Load, bindCtx *BindContext) (int32, error) { + dmlCtx := NewDMLContext() + onDupAction := plan.Node_ERROR + builder.qry.LoadTag = true + lastNodeID, insertColToExpr, err := builder.bindExternalScan(stmt, bindCtx, dmlCtx) + if err != nil { + return -1, err + } + + lastNodeID, colName2Idx, skipUniqueIdx, err := builder.appendNodesForInsertStmt(bindCtx, lastNodeID, dmlCtx.tableDefs[0], dmlCtx.objRefs[0], insertColToExpr) + if err != nil { + return -1, err + } + + return builder.appendDedupAndMultiUpdateNodesForBindInsert(bindCtx, dmlCtx, lastNodeID, colName2Idx, skipUniqueIdx, onDupAction) +} + +func (builder *QueryBuilder) bindExternalScan( + stmt *tree.Load, + bindCtx *BindContext, + dmlCtx *DMLContext) (int32, map[string]*plan.Expr, error) { + externalScanTag := builder.genNewTag() + err := dmlCtx.ResolveTables(builder.compCtx, tree.TableExprs{stmt.Table}, nil, nil, true) + if err != nil { + return -1, nil, err + } + + ctx := builder.compCtx + stmt.Param.Local = stmt.Local + fileName, err := checkFileExist(stmt.Param, ctx) + if err != nil { + return -1, nil, err + } + + if err := InitNullMap(stmt.Param, ctx); err != nil { + return -1, nil, err + } + + tableDef := DeepCopyTableDef(dmlCtx.tableDefs[0], true) + objRef := dmlCtx.objRefs[0] + + // load with columnlist will copy a new tableDef + tbColToDataCol := make(map[string]int32) + insertColToExpr := make(map[string]*plan.Expr) + if stmt.Param.ScanType != tree.INLINE || len(stmt.Param.Tail.ColumnList) > 0 { + if len(stmt.Param.Tail.ColumnList) > 0 { + colToIndex := make(map[string]int32, 0) + var newCols []*ColDef + colPos := 0 + + for i, col := range stmt.Param.Tail.ColumnList { + switch realCol := col.(type) { + case *tree.UnresolvedName: + colName := realCol.ColName() + if _, ok := tableDef.Name2ColIndex[colName]; !ok { + return -1, nil, moerr.NewInternalErrorf(ctx.GetContext(), "column '%s' does not exist", colName) + } + tbColIdx := tableDef.Name2ColIndex[colName] + colExpr := &plan.Expr{ + Typ: tableDef.Cols[tbColIdx].Typ, + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + RelPos: externalScanTag, + ColPos: int32(colPos), + Name: tableDef.Name + "." + colName, + }, + }, + } + insertColToExpr[colName] = colExpr + colToIndex[colName] = int32(colPos) + colPos++ + tbColToDataCol[colName] = int32(i) + newCols = append(newCols, tableDef.Cols[tbColIdx]) + case *tree.VarExpr: + //NOTE:variable like '@abc' will be passed by. + name := realCol.Name + tbColToDataCol[name] = -1 // when in external call, can use len of the map to check load data row whether valid + default: + return -1, nil, moerr.NewInternalErrorf(ctx.GetContext(), "unsupported column type %v", realCol) + } + } + tableDef.Cols = newCols + tableDef.Name2ColIndex = colToIndex + } + } + + if len(tbColToDataCol) == 0 { + idx := 0 + for _, col := range tableDef.Cols { + if !col.Hidden { + tbColToDataCol[col.Name] = int32(len(insertColToExpr)) + + insertColToExpr[col.Name] = &plan.Expr{ + Typ: col.Typ, + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + RelPos: externalScanTag, + ColPos: int32(idx), + }, + }, + } + idx++ + } + } + } + + if err := checkNullMap(stmt, tableDef.Cols, ctx); err != nil { + return -1, nil, err + } + + noCompress := getCompressType(stmt.Param, fileName) == tree.NOCOMPRESS + var offset int64 = 0 + if stmt.Param.Tail.IgnoredLines > 0 && stmt.Param.Parallel && noCompress && !stmt.Param.Local { + offset, err = IgnoredLines(stmt.Param, ctx) + if err != nil { + return -1, nil, err + } + stmt.Param.FileStartOff = offset + } + + if stmt.Param.FileSize-offset < int64(LoadParallelMinSize) { + stmt.Param.Parallel = false + } + + stmt.Param.Tail.ColumnList = nil + if stmt.Param.ScanType != tree.INLINE { + json_byte, err := json.Marshal(stmt.Param) + if err != nil { + return -1, nil, err + } + tableDef.Createsql = string(json_byte) + } + + terminated := "," + enclosedBy := []byte("\"") + escapedBy := []byte{0} + if stmt.Param.Tail.Fields != nil { + if stmt.Param.Tail.Fields.EnclosedBy != nil { + if stmt.Param.Tail.Fields.EnclosedBy.Value != 0 { + enclosedBy = []byte{stmt.Param.Tail.Fields.EnclosedBy.Value} + } + } + if stmt.Param.Tail.Fields.EscapedBy != nil { + if stmt.Param.Tail.Fields.EscapedBy.Value != 0 { + escapedBy = []byte{stmt.Param.Tail.Fields.EscapedBy.Value} + } + } + if stmt.Param.Tail.Fields.Terminated != nil { + terminated = stmt.Param.Tail.Fields.Terminated.Value + } + } + + externalScanNode := &plan.Node{ + NodeType: plan.Node_EXTERNAL_SCAN, + Stats: &plan.Stats{}, + ObjRef: objRef, + TableDef: tableDef, + ExternScan: &plan.ExternScan{ + Type: int32(plan.ExternType_LOAD), + LoadType: int32(stmt.Param.ScanType), + Data: stmt.Param.Data, + Format: stmt.Param.Format, + IgnoredLines: uint64(stmt.Param.Tail.IgnoredLines), + EnclosedBy: enclosedBy, + Terminated: terminated, + EscapedBy: escapedBy, + JsonType: stmt.Param.JsonData, + TbColToDataCol: tbColToDataCol, + }, + BindingTags: []int32{externalScanTag}, + } + lastNodeId := builder.appendNode(externalScanNode, bindCtx) + + return lastNodeId, insertColToExpr, nil +} diff --git a/pkg/sql/plan/bind_update.go b/pkg/sql/plan/bind_update.go new file mode 100644 index 000000000000..1c972d03d987 --- /dev/null +++ b/pkg/sql/plan/bind_update.go @@ -0,0 +1,536 @@ +// Copyright 2021 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package plan + +import ( + "fmt" + + "github.com/matrixorigin/matrixone/pkg/catalog" + "github.com/matrixorigin/matrixone/pkg/common/moerr" + "github.com/matrixorigin/matrixone/pkg/container/types" + "github.com/matrixorigin/matrixone/pkg/pb/plan" + "github.com/matrixorigin/matrixone/pkg/sql/parsers/tree" +) + +func (builder *QueryBuilder) bindUpdate(stmt *tree.Update, bindCtx *BindContext) (int32, error) { + dmlCtx := NewDMLContext() + err := dmlCtx.ResolveUpdateTables(builder.compCtx, stmt) + if err != nil { + return 0, err + } + + var selectList []tree.SelectExpr + colName2Idx := make(map[string]int32) + updateColName2Idx := make(map[string]int32) + + for i, alias := range dmlCtx.aliases { + if len(dmlCtx.updateCol2Expr[i]) == 0 { + continue + } + + tableDef := dmlCtx.tableDefs[i] + useColInPartExpr := make(map[string]bool) + if tableDef.Partition != nil { + getPartColsFromExpr(tableDef.Partition.PartitionExpression, useColInPartExpr) + } + + // append table.* to project list + for _, col := range tableDef.Cols { + colName2Idx[alias+"."+col.Name] = int32(len(selectList)) + e := tree.NewUnresolvedName(tree.NewCStr(alias, bindCtx.lower), tree.NewCStr(col.Name, 1)) + selectList = append(selectList, tree.SelectExpr{ + Expr: e, + }) + } + + // TODO: support update primary key or unique key or secondary key or master index or ivfflat index + var pkAndUkCols = make(map[string]bool) + if tableDef.Pkey != nil { + for _, colName := range tableDef.Pkey.Names { + pkAndUkCols[colName] = true + } + } + for _, idxDef := range tableDef.Indexes { + if !idxDef.TableExist || !idxDef.Unique { + if catalog.IsRegularIndexAlgo(idxDef.IndexAlgo) { + continue + } + } + + for _, colName := range idxDef.Parts { + pkAndUkCols[colName] = true + } + } + + for colName, updateExpr := range dmlCtx.updateCol2Expr[i] { + if pkAndUkCols[colName] { + return 0, moerr.NewUnsupportedDML(builder.compCtx.GetContext(), "update primary key or unique key or master index or ivfflat index") + } + + if !dmlCtx.updatePartCol[i] { + if _, ok := useColInPartExpr[colName]; ok { + dmlCtx.updatePartCol[i] = true + } + } + + for _, colDef := range tableDef.Cols { + if colDef.Name == colName { + if colDef.Typ.Id == int32(types.T_enum) { + if colDef.Typ.AutoIncr { + return 0, moerr.NewUnsupportedDML(builder.compCtx.GetContext(), "auto_increment default value") + } + + binder := NewDefaultBinder(builder.GetContext(), nil, nil, colDef.Typ, nil) + updateKeyExpr, err := binder.BindExpr(updateExpr, 0, false) + if err != nil { + return 0, err + } + + exprs := []tree.Expr{ + tree.NewNumVal(colDef.Typ.Enumvalues, colDef.Typ.Enumvalues, false, tree.P_char), + updateExpr, + } + + if updateKeyExpr.Typ.Id >= 20 && updateKeyExpr.Typ.Id <= 29 { + updateExpr = &tree.FuncExpr{ + Func: tree.FuncName2ResolvableFunctionReference(tree.NewUnresolvedColName(moEnumCastIndexValueToIndexFun)), + Type: tree.FUNC_TYPE_DEFAULT, + Exprs: exprs, + } + } else { + updateExpr = &tree.FuncExpr{ + Func: tree.FuncName2ResolvableFunctionReference(tree.NewUnresolvedColName(moEnumCastValueToIndexFun)), + Type: tree.FUNC_TYPE_DEFAULT, + Exprs: exprs, + } + } + } + + if colDef.Typ.AutoIncr { + if constExpr, ok := updateExpr.(*tree.NumVal); ok { + if constExpr.ValType == tree.P_null { + return 0, moerr.NewConstraintViolation(builder.compCtx.GetContext(), fmt.Sprintf("Column '%s' cannot be null", colName)) + } + } + } + } + } + + updateColName2Idx[alias+"."+colName] = int32(len(selectList)) + selectList = append(selectList, tree.SelectExpr{ + Expr: updateExpr, + }) + } + } + + selectAst := &tree.Select{ + Select: &tree.SelectClause{ + Exprs: selectList, + From: &tree.From{ + Tables: stmt.Tables, + }, + Where: stmt.Where, + }, + OrderBy: stmt.OrderBy, + Limit: stmt.Limit, + With: stmt.With, + } + + lastNodeID, err := builder.bindSelect(selectAst, bindCtx, false) + if err != nil { + return 0, err + } + + selectNode := builder.qry.Nodes[lastNodeID] + + for i, alias := range dmlCtx.aliases { + if len(dmlCtx.updateCol2Expr[i]) == 0 { + continue + } + + tableDef := dmlCtx.tableDefs[i] + + for originPos, col := range tableDef.Cols { + if colPos, ok := updateColName2Idx[alias+"."+col.Name]; ok { + updateExpr := selectNode.ProjectList[colPos] + if isDefaultValExpr(updateExpr) { // set col = default + updateExpr, err = getDefaultExpr(builder.GetContext(), col) + if err != nil { + return 0, err + } + } + err = checkNotNull(builder.GetContext(), updateExpr, tableDef, col) + if err != nil { + return 0, err + } + if col != nil && col.Typ.Id == int32(types.T_enum) { + selectNode.ProjectList[colPos], err = funcCastForEnumType(builder.GetContext(), updateExpr, col.Typ) + if err != nil { + return 0, err + } + } else { + selectNode.ProjectList[colPos], err = forceCastExpr(builder.GetContext(), updateExpr, col.Typ) + if err != nil { + return 0, err + } + } + } else { + if col.OnUpdate != nil && col.OnUpdate.Expr != nil { + //pos := colName2Idx[alias+"."+col.Name] + //selectNode.ProjectList[pos] = col.OnUpdate.Expr + // + //if col.Typ.Id == int32(types.T_enum) { + // selectNode.ProjectList[pos], err = funcCastForEnumType(builder.GetContext(), selectNode.ProjectList[pos], col.Typ) + // if err != nil { + // return 0, err + // } + //} else { + // selectNode.ProjectList[pos], err = forceCastExpr(builder.GetContext(), selectNode.ProjectList[pos], col.Typ) + // if err != nil { + // return 0, err + // } + //} + return 0, moerr.NewUnsupportedDML(builder.compCtx.GetContext(), "update column with on update") + } + if col.Typ.Id == int32(types.T_enum) { + selectNode.ProjectList[originPos], err = funcCastForEnumType(builder.GetContext(), selectNode.ProjectList[originPos], col.Typ) + if err != nil { + return 0, err + } + } + } + } + } + + selectNodeTag := selectNode.BindingTags[0] + idxScanNodes := make([][]*plan.Node, len(dmlCtx.tableDefs)) + idxNeedUpdate := make([][]bool, len(dmlCtx.tableDefs)) + + for i, tableDef := range dmlCtx.tableDefs { + if len(dmlCtx.updateCol2Expr[i]) == 0 { + continue + } + + alias := dmlCtx.aliases[i] + idxScanNodes[i] = make([]*plan.Node, len(tableDef.Indexes)) + idxNeedUpdate[i] = make([]bool, len(tableDef.Indexes)) + + for j, idxDef := range tableDef.Indexes { + if !idxDef.TableExist || idxDef.Unique { + continue + } + + for _, colName := range idxDef.Parts { + if _, ok := updateColName2Idx[alias+"."+colName]; ok { + idxNeedUpdate[i][j] = true + break + } + } + if !idxNeedUpdate[i][j] { + continue + } + + idxObjRef, idxTableDef := builder.compCtx.Resolve(dmlCtx.objRefs[i].SchemaName, idxDef.IndexTableName, bindCtx.snapshot) + idxTag := builder.genNewTag() + builder.addNameByColRef(idxTag, idxTableDef) + + idxScanNodes[i][j] = &plan.Node{ + NodeType: plan.Node_TABLE_SCAN, + TableDef: idxTableDef, + ObjRef: idxObjRef, + BindingTags: []int32{idxTag}, + ScanSnapshot: bindCtx.snapshot, + } + idxTableNodeID := builder.appendNode(idxScanNodes[i][j], bindCtx) + + rightPkPos := idxTableDef.Name2ColIndex[catalog.IndexTableIndexColName] + pkTyp := idxTableDef.Cols[rightPkPos].Typ + + rightExpr := &plan.Expr{ + Typ: pkTyp, + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + RelPos: idxTag, + ColPos: rightPkPos, + }, + }, + } + + args := make([]*plan.Expr, len(idxDef.Parts)) + + var colPos int32 + var ok bool + for k, colName := range idxDef.Parts { + if colPos, ok = colName2Idx[alias+"."+catalog.ResolveAlias(colName)]; !ok { + errMsg := fmt.Sprintf("bind update err, can not find colName = %s", colName) + return 0, moerr.NewInternalError(builder.GetContext(), errMsg) + } + args[k] = &plan.Expr{ + Typ: selectNode.ProjectList[colPos].Typ, + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + RelPos: selectNodeTag, + ColPos: colPos, + }, + }, + } + } + + leftExpr, _ := BindFuncExprImplByPlanExpr(builder.GetContext(), "serial_full", args) + + joinCond, _ := BindFuncExprImplByPlanExpr(builder.GetContext(), "=", []*plan.Expr{ + leftExpr, + rightExpr, + }) + + lastNodeID = builder.appendNode(&plan.Node{ + NodeType: plan.Node_JOIN, + Children: []int32{lastNodeID, idxTableNodeID}, + JoinType: plan.Node_INNER, + OnList: []*plan.Expr{joinCond}, + }, bindCtx) + } + } + + lockTargets := make([]*plan.LockTarget, 0) + updateCtxList := make([]*plan.UpdateCtx, 0) + + finalProjTag := builder.genNewTag() + finalColName2Idx := make(map[string]int32) + var finalProjList []*plan.Expr + + for i, tableDef := range dmlCtx.tableDefs { + if len(dmlCtx.updateCol2Expr[i]) == 0 { + continue + } + + alias := dmlCtx.aliases[i] + insertCols := make([]plan.ColRef, len(tableDef.Cols)-1) + + for j, col := range tableDef.Cols { + finalColIdx := len(finalProjList) + if col.Name == tableDef.Pkey.PkeyColName { + lockTarget := &plan.LockTarget{ + TableId: tableDef.TblId, + PrimaryColIdxInBat: int32(finalColIdx), + PrimaryColRelPos: finalProjTag, + PrimaryColTyp: col.Typ, + } + lockTargets = append(lockTargets, lockTarget) + + if dmlCtx.updatePartCol[i] { + // if update col which partition expr used, + // need lock oldPk by old partition idx, lock new pk by new partition idx + lockTarget := &plan.LockTarget{ + TableId: tableDef.TblId, + PrimaryColIdxInBat: int32(finalColIdx), + PrimaryColRelPos: finalProjTag, + PrimaryColTyp: col.Typ, + } + lockTargets = append(lockTargets, lockTarget) + } + } + + if col.Name != catalog.Row_ID { + insertCols[j].RelPos = finalProjTag + insertCols[j].ColPos = int32(finalColIdx) + } + + colIdx := colName2Idx[alias+"."+col.Name] + if updateIdx, ok := updateColName2Idx[alias+"."+col.Name]; ok { + colIdx = updateIdx + } + + finalColName2Idx[alias+"."+col.Name] = int32(finalColIdx) + finalProjList = append(finalProjList, &plan.Expr{ + Typ: selectNode.ProjectList[colIdx].Typ, + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + RelPos: selectNodeTag, + ColPos: int32(colIdx), + }, + }, + }) + } + + updateCtxList = append(updateCtxList, &plan.UpdateCtx{ + ObjRef: dmlCtx.objRefs[i], + TableDef: tableDef, + InsertCols: insertCols, + OldPartitionIdx: -1, + NewPartitionIdx: -1, + DeleteCols: []plan.ColRef{ + { + RelPos: finalProjTag, + ColPos: finalColName2Idx[alias+"."+catalog.Row_ID], + }, + { + RelPos: finalProjTag, + ColPos: finalColName2Idx[alias+"."+tableDef.Pkey.PkeyColName], + }, + }, + }) + + for j, idxNode := range idxScanNodes[i] { + if !idxNeedUpdate[i][j] { + continue + } + + insertCols := make([]plan.ColRef, 2) + deleteCols := make([]plan.ColRef, 2) + + idxNodeTag := idxNode.BindingTags[0] + + oldIdx := len(finalProjList) + rowIDIdx := idxNode.TableDef.Name2ColIndex[catalog.Row_ID] + finalProjList = append(finalProjList, &plan.Expr{ + Typ: idxNode.TableDef.Cols[rowIDIdx].Typ, + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + RelPos: idxNodeTag, + ColPos: rowIDIdx, + }, + }, + }) + deleteCols[0].RelPos = finalProjTag + deleteCols[0].ColPos = int32(oldIdx) + + oldIdx = len(finalProjList) + idxColIdx := idxNode.TableDef.Name2ColIndex[catalog.IndexTableIndexColName] + finalProjList = append(finalProjList, &plan.Expr{ + Typ: idxNode.TableDef.Cols[idxColIdx].Typ, + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + RelPos: idxNodeTag, + ColPos: idxColIdx, + }, + }, + }) + deleteCols[1].RelPos = finalProjTag + deleteCols[1].ColPos = int32(oldIdx) + + oldIdx = len(finalProjList) + idxDef := tableDef.Indexes[j] + args := make([]*plan.Expr, len(idxDef.Parts)) + + for k, colName := range idxDef.Parts { + colPos := int32(colName2Idx[alias+"."+colName]) + if updateIdx, ok := updateColName2Idx[alias+"."+colName]; ok { + colPos = int32(updateIdx) + } + args[k] = &plan.Expr{ + Typ: selectNode.ProjectList[colPos].Typ, + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + RelPos: selectNodeTag, + ColPos: colPos, + }, + }, + } + } + + newIdxExpr, _ := BindFuncExprImplByPlanExpr(builder.GetContext(), "serial_full", args) + finalProjList = append(finalProjList, newIdxExpr) + insertCols[0].RelPos = finalProjTag + insertCols[0].ColPos = int32(oldIdx) + + insertCols[1].RelPos = finalProjTag + insertCols[1].ColPos = finalColName2Idx[alias+"."+tableDef.Pkey.PkeyColName] + + updateCtxList = append(updateCtxList, &plan.UpdateCtx{ + ObjRef: idxNode.ObjRef, + TableDef: idxNode.TableDef, + InsertCols: insertCols, + DeleteCols: deleteCols, + }) + } + } + + dmlNode := &plan.Node{ + NodeType: plan.Node_MULTI_UPDATE, + BindingTags: []int32{builder.genNewTag()}, + UpdateCtxList: updateCtxList, + } + + if dmlCtx.tableDefs[0].Partition != nil { + partitionTableIDs, partitionTableNames := getPartitionInfos(builder.compCtx, dmlCtx.objRefs[0], dmlCtx.tableDefs[0]) + updateCtxList[0].PartitionTableIds = partitionTableIDs + updateCtxList[0].PartitionTableNames = partitionTableNames + var oldPartExpr *Expr + oldPartExpr, err = getRemapParitionExpr(dmlCtx.tableDefs[0], selectNodeTag, colName2Idx, true) + if err != nil { + return -1, err + } + updateCtxList[0].OldPartitionIdx = int32(len(finalProjList)) + finalProjList = append(finalProjList, oldPartExpr) + lockTargets[0].IsPartitionTable = true + lockTargets[0].PartitionTableIds = partitionTableIDs + lockTargets[0].FilterColIdxInBat = updateCtxList[0].OldPartitionIdx + lockTargets[0].FilterColRelPos = finalProjTag + + if dmlCtx.updatePartCol[0] { + // if update col which partition expr used, + // need lock oldPk by old partition idx, lock new pk by new partition idx + var newPartExpr *Expr + partName2Idx := make(map[string]int32) + for k, v := range colName2Idx { + partName2Idx[k] = v + } + for k, v := range updateColName2Idx { + partName2Idx[k] = v + } + newPartExpr, err = getRemapParitionExpr(dmlCtx.tableDefs[0], selectNodeTag, partName2Idx, true) + if err != nil { + return -1, err + } + updateCtxList[0].NewPartitionIdx = int32(len(finalProjList)) + finalProjList = append(finalProjList, newPartExpr) + + lockTargets[1].IsPartitionTable = true + lockTargets[1].PartitionTableIds = partitionTableIDs + lockTargets[1].FilterColIdxInBat = updateCtxList[0].NewPartitionIdx + lockTargets[1].FilterColRelPos = finalProjTag + } else { + // if do not update col which partition expr used, + // just use old partition idx. + // @todo if update pk, we need another lockTarget too(not support now) + updateCtxList[0].NewPartitionIdx = updateCtxList[0].OldPartitionIdx + } + + dmlNode.BindingTags = append(dmlNode.BindingTags, finalProjTag) + + } + + lastNodeID = builder.appendNode(&plan.Node{ + NodeType: plan.Node_PROJECT, + ProjectList: finalProjList, + Children: []int32{lastNodeID}, + BindingTags: []int32{finalProjTag}, + }, bindCtx) + + lastNodeID = builder.appendNode(&plan.Node{ + NodeType: plan.Node_LOCK_OP, + Children: []int32{lastNodeID}, + TableDef: dmlCtx.tableDefs[0], + BindingTags: []int32{builder.genNewTag()}, + LockTargets: lockTargets, + }, bindCtx) + reCheckifNeedLockWholeTable(builder) + + dmlNode.Children = append(dmlNode.Children, lastNodeID) + lastNodeID = builder.appendNode(dmlNode, bindCtx) + + return lastNodeID, err +} diff --git a/pkg/sql/plan/build.go b/pkg/sql/plan/build.go index 22f87a811448..2fc4df298121 100644 --- a/pkg/sql/plan/build.go +++ b/pkg/sql/plan/build.go @@ -27,7 +27,7 @@ import ( v2 "github.com/matrixorigin/matrixone/pkg/util/metric/v2" ) -func runBuildSelectByBinder(stmtType plan.Query_StatementType, ctx CompilerContext, stmt *tree.Select, isPrepareStmt bool, skipStats bool) (*Plan, error) { +func bindAndOptimizeSelectQuery(stmtType plan.Query_StatementType, ctx CompilerContext, stmt *tree.Select, isPrepareStmt bool, skipStats bool) (*Plan, error) { start := time.Now() defer func() { v2.TxnStatementBuildSelectHistogram.Observe(time.Since(start).Seconds()) @@ -39,7 +39,7 @@ func runBuildSelectByBinder(stmtType plan.Query_StatementType, ctx CompilerConte bindCtx.snapshot = ctx.GetSnapshot() } - rootId, err := builder.buildSelect(stmt, bindCtx, true) + rootId, err := builder.bindSelect(stmt, bindCtx, true) if err != nil { return nil, err } @@ -58,6 +58,147 @@ func runBuildSelectByBinder(stmtType plan.Query_StatementType, ctx CompilerConte }, err } +func bindAndOptimizeInsertQuery(ctx CompilerContext, stmt *tree.Insert, isPrepareStmt bool, skipStats bool) (*Plan, error) { + start := time.Now() + defer func() { + v2.TxnStatementBuildInsertHistogram.Observe(time.Since(start).Seconds()) + }() + + builder := NewQueryBuilder(plan.Query_INSERT, ctx, isPrepareStmt, true) + bindCtx := NewBindContext(builder, nil) + if IsSnapshotValid(ctx.GetSnapshot()) { + bindCtx.snapshot = ctx.GetSnapshot() + } + + rootId, err := builder.bindInsert(stmt, bindCtx) + if err != nil { + if err.(*moerr.Error).ErrorCode() == moerr.ErrUnsupportedDML { + return buildInsert(stmt, ctx, false, isPrepareStmt) + } + return nil, err + } + ctx.SetViews(bindCtx.views) + + builder.qry.Steps = append(builder.qry.Steps, rootId) + builder.skipStats = skipStats + query, err := builder.createQuery() + if err != nil { + return nil, err + } + return &Plan{ + Plan: &plan.Plan_Query{ + Query: query, + }, + }, err +} + +func bindAndOptimizeLoadQuery(ctx CompilerContext, stmt *tree.Load, isPrepareStmt bool, skipStats bool) (*Plan, error) { + // return buildLoad(stmt, ctx, isPrepareStmt) + start := time.Now() + defer func() { + v2.TxnStatementBuildInsertHistogram.Observe(time.Since(start).Seconds()) + }() + + builder := NewQueryBuilder(plan.Query_INSERT, ctx, isPrepareStmt, true) + bindCtx := NewBindContext(builder, nil) + if IsSnapshotValid(ctx.GetSnapshot()) { + bindCtx.snapshot = ctx.GetSnapshot() + } + + rootId, err := builder.bindLoad(stmt, bindCtx) + if err != nil { + if err.(*moerr.Error).ErrorCode() == moerr.ErrUnsupportedDML { + return buildLoad(stmt, ctx, isPrepareStmt) + } + return nil, err + } + ctx.SetViews(bindCtx.views) + + builder.qry.Steps = append(builder.qry.Steps, rootId) + builder.skipStats = skipStats + query, err := builder.createQuery() + if err != nil { + return nil, err + } + return &Plan{ + Plan: &plan.Plan_Query{ + Query: query, + }, + }, err +} + +func bindAndOptimizeDeleteQuery(ctx CompilerContext, stmt *tree.Delete, isPrepareStmt bool, skipStats bool) (*Plan, error) { + start := time.Now() + defer func() { + v2.TxnStatementBuildDeleteHistogram.Observe(time.Since(start).Seconds()) + }() + + builder := NewQueryBuilder(plan.Query_DELETE, ctx, isPrepareStmt, true) + bindCtx := NewBindContext(builder, nil) + if IsSnapshotValid(ctx.GetSnapshot()) { + bindCtx.snapshot = ctx.GetSnapshot() + } + + rootId, err := builder.bindDelete(stmt, bindCtx) + if err != nil { + if err.(*moerr.Error).ErrorCode() == moerr.ErrUnsupportedDML { + return buildDelete(stmt, ctx, isPrepareStmt) + } + return nil, err + } + ctx.SetViews(bindCtx.views) + + builder.qry.Steps = append(builder.qry.Steps, rootId) + builder.skipStats = skipStats + query, err := builder.createQuery() + if err != nil { + return nil, err + } + return &Plan{ + Plan: &plan.Plan_Query{ + Query: query, + }, + }, err +} + +func bindAndOptimizeUpdateQuery(ctx CompilerContext, stmt *tree.Update, isPrepareStmt bool, skipStats bool) (*Plan, error) { + // if !isExplain { + // return buildTableUpdate(stmt, ctx, isPrepareStmt) + // } + + start := time.Now() + defer func() { + v2.TxnStatementBuildDeleteHistogram.Observe(time.Since(start).Seconds()) + }() + + builder := NewQueryBuilder(plan.Query_UPDATE, ctx, isPrepareStmt, true) + bindCtx := NewBindContext(builder, nil) + if IsSnapshotValid(ctx.GetSnapshot()) { + bindCtx.snapshot = ctx.GetSnapshot() + } + + rootId, err := builder.bindUpdate(stmt, bindCtx) + if err != nil { + if err.(*moerr.Error).ErrorCode() == moerr.ErrUnsupportedDML { + return buildTableUpdate(stmt, ctx, isPrepareStmt) + } + return nil, err + } + ctx.SetViews(bindCtx.views) + + builder.qry.Steps = append(builder.qry.Steps, rootId) + builder.skipStats = skipStats + query, err := builder.createQuery() + if err != nil { + return nil, err + } + return &Plan{ + Plan: &plan.Plan_Query{ + Query: query, + }, + }, err +} + func buildExplainPlan(ctx CompilerContext, stmt tree.Statement, isPrepareStmt bool) (*Plan, error) { start := time.Now() defer func() { @@ -106,21 +247,21 @@ func BuildPlan(ctx CompilerContext, stmt tree.Statement, isPrepareStmt bool) (*P defer task.End() switch stmt := stmt.(type) { case *tree.Select: - return runBuildSelectByBinder(plan.Query_SELECT, ctx, stmt, isPrepareStmt, false) + return bindAndOptimizeSelectQuery(plan.Query_SELECT, ctx, stmt, isPrepareStmt, false) case *tree.ParenSelect: - return runBuildSelectByBinder(plan.Query_SELECT, ctx, stmt.Select, isPrepareStmt, false) + return bindAndOptimizeSelectQuery(plan.Query_SELECT, ctx, stmt.Select, isPrepareStmt, false) case *tree.ExplainAnalyze: return buildExplainAnalyze(ctx, stmt, isPrepareStmt) case *tree.ExplainPhyPlan: return buildExplainPhyPlan(ctx, stmt, isPrepareStmt) case *tree.Insert: - return buildInsert(stmt, ctx, false, isPrepareStmt) + return bindAndOptimizeInsertQuery(ctx, stmt, isPrepareStmt, false) case *tree.Replace: return buildReplace(stmt, ctx, isPrepareStmt, false) case *tree.Update: - return buildTableUpdate(stmt, ctx, isPrepareStmt) + return bindAndOptimizeUpdateQuery(ctx, stmt, isPrepareStmt, false) case *tree.Delete: - return buildDelete(stmt, ctx, isPrepareStmt) + return bindAndOptimizeDeleteQuery(ctx, stmt, isPrepareStmt, false) case *tree.BeginTransaction: return buildBeginTransaction(stmt, ctx) case *tree.CommitTransaction: @@ -208,7 +349,7 @@ func BuildPlan(ctx CompilerContext, stmt tree.Statement, isPrepareStmt bool) (*P case *tree.Deallocate: return buildDeallocate(stmt, ctx) case *tree.Load: - return buildLoad(stmt, ctx, isPrepareStmt) + return bindAndOptimizeLoadQuery(ctx, stmt, isPrepareStmt, false) case *tree.PrepareStmt, *tree.PrepareString: return buildPrepare(stmt, ctx) case *tree.Do, *tree.Declare: diff --git a/pkg/sql/plan/build_constraint_util.go b/pkg/sql/plan/build_constraint_util.go index 484060a6edff..7b04f9321636 100644 --- a/pkg/sql/plan/build_constraint_util.go +++ b/pkg/sql/plan/build_constraint_util.go @@ -429,7 +429,7 @@ func initInsertStmt(builder *QueryBuilder, bindCtx *BindContext, stmt *tree.Inse astSlt = stmt.Rows subCtx := NewBindContext(builder, bindCtx) - info.rootId, err = builder.buildSelect(astSlt, subCtx, false) + info.rootId, err = builder.bindSelect(astSlt, subCtx, false) if err != nil { return false, nil, nil, err } @@ -439,7 +439,7 @@ func initInsertStmt(builder *QueryBuilder, bindCtx *BindContext, stmt *tree.Inse astSlt = slt.Select subCtx := NewBindContext(builder, bindCtx) - info.rootId, err = builder.buildSelect(astSlt, subCtx, false) + info.rootId, err = builder.bindSelect(astSlt, subCtx, false) if err != nil { return false, nil, nil, err } @@ -866,7 +866,7 @@ func deleteToSelect(builder *QueryBuilder, bindCtx *BindContext, node *tree.Dele // sql := ftCtx.String() // fmt.Print(sql) - return builder.buildSelect(astSelect, bindCtx, false) + return builder.bindSelect(astSelect, bindCtx, false) } func checkNotNull(ctx context.Context, expr *Expr, tableDef *TableDef, col *ColDef) error { diff --git a/pkg/sql/plan/build_ddl.go b/pkg/sql/plan/build_ddl.go index 8e7e4542e672..58b9476ac1aa 100644 --- a/pkg/sql/plan/build_ddl.go +++ b/pkg/sql/plan/build_ddl.go @@ -43,12 +43,12 @@ func genDynamicTableDef(ctx CompilerContext, stmt *tree.Select) (*plan.TableDef, var err error switch s := stmt.Select.(type) { case *tree.ParenSelect: - stmtPlan, err = runBuildSelectByBinder(plan.Query_SELECT, ctx, s.Select, false, true) + stmtPlan, err = bindAndOptimizeSelectQuery(plan.Query_SELECT, ctx, s.Select, false, true) if err != nil { return nil, err } default: - stmtPlan, err = runBuildSelectByBinder(plan.Query_SELECT, ctx, stmt, false, true) + stmtPlan, err = bindAndOptimizeSelectQuery(plan.Query_SELECT, ctx, stmt, false, true) if err != nil { return nil, err } @@ -105,12 +105,12 @@ func genViewTableDef(ctx CompilerContext, stmt *tree.Select) (*plan.TableDef, er var err error switch s := stmt.Select.(type) { case *tree.ParenSelect: - stmtPlan, err = runBuildSelectByBinder(plan.Query_SELECT, ctx, s.Select, false, true) + stmtPlan, err = bindAndOptimizeSelectQuery(plan.Query_SELECT, ctx, s.Select, false, true) if err != nil { return nil, err } default: - stmtPlan, err = runBuildSelectByBinder(plan.Query_SELECT, ctx, stmt, false, true) + stmtPlan, err = bindAndOptimizeSelectQuery(plan.Query_SELECT, ctx, stmt, false, true) if err != nil { return nil, err } @@ -193,7 +193,7 @@ func genAsSelectCols(ctx CompilerContext, stmt *tree.Select) ([]*ColDef, error) if s, ok := stmt.Select.(*tree.ParenSelect); ok { stmt = s.Select } - if rootId, err = builder.buildSelect(stmt, bindCtx, true); err != nil { + if rootId, err = builder.bindSelect(stmt, bindCtx, true); err != nil { return nil, err } rootNode := builder.qry.Nodes[rootId] diff --git a/pkg/sql/plan/build_test.go b/pkg/sql/plan/build_test.go index 6d73fbc7fd66..441bf93b439c 100644 --- a/pkg/sql/plan/build_test.go +++ b/pkg/sql/plan/build_test.go @@ -535,6 +535,7 @@ func TestJoinTableSqlBuilder(t *testing.T) { "SELECT N_NAME, R_REGIONKEY FROM NATION2 join REGION using(R_REGIONKEY)", "select nation.n_name from nation join nation2 on nation.n_name !='a' join region on nation.n_regionkey = region.r_regionkey", "select * from nation, nation2, region", + "select n_name from nation dedup join region on n_regionkey = r_regionkey", } runTestShouldPass(mock, t, sqls, false, false) diff --git a/pkg/sql/plan/build_update.go b/pkg/sql/plan/build_update.go index f6fe003f4d46..caeb9cb18e9b 100644 --- a/pkg/sql/plan/build_update.go +++ b/pkg/sql/plan/build_update.go @@ -291,7 +291,7 @@ func selectUpdateTables(builder *QueryBuilder, bindCtx *BindContext, stmt *tree. //selectAst.Format(ftCtx) //sql := ftCtx.String() //fmt.Print(sql) - lastNodeId, err := builder.buildSelect(selectAst, bindCtx, false) + lastNodeId, err := builder.bindSelect(selectAst, bindCtx, false) if err != nil { return -1, nil, err } diff --git a/pkg/sql/plan/build_util.go b/pkg/sql/plan/build_util.go index 85869145fd71..97d386bbf9ad 100644 --- a/pkg/sql/plan/build_util.go +++ b/pkg/sql/plan/build_util.go @@ -60,6 +60,10 @@ func reCheckifNeedLockWholeTable(builder *QueryBuilder) { logutil.Infof("Row lock upgraded to table lock for SQL : %s", builder.compCtx.GetRootSql()) logutil.Infof("the outcnt stats is %f", n.Stats.Outcnt) n.LockTargets[0].LockTable = reCheckIfNeed + + if len(n.LockTargets) > 1 && n.LockTargets[1].IsPartitionTable { + n.LockTargets[1].LockTable = true + } } } } @@ -661,3 +665,83 @@ func genSqlsForCheckFKSelfRefer(ctx context.Context, } return ret, nil } + +func getPartitionInfos(ctx CompilerContext, objRef *ObjectRef, tableDef *TableDef) ([]uint64, []string) { + partTableIds := make([]uint64, tableDef.Partition.PartitionNum) + partTableNames := make([]string, tableDef.Partition.PartitionNum) + for i, partition := range tableDef.Partition.Partitions { + _, partTableDef := ctx.Resolve(objRef.SchemaName, partition.PartitionTableName, nil) + partTableIds[i] = partTableDef.TblId + partTableNames[i] = partition.PartitionTableName + } + return partTableIds, partTableNames +} + +func getRemapParitionExpr(tableDef *TableDef, relPos int32, colPosMap map[string]int32, containTableName bool) (retExpr *Expr, err error) { + retExpr = DeepCopyExpr(tableDef.Partition.PartitionExpression) + err = remapPartitionExprColRef(retExpr, tableDef.Name, relPos, colPosMap, containTableName) + return +} + +func remapPartitionExprColRef(expr *Expr, tableName string, relPos int32, colPosMap map[string]int32, containTableName bool) (err error) { + switch ne := expr.Expr.(type) { + case *plan.Expr_Col: + colName := ne.Col.Name + if containTableName { + colName = tableName + "." + colName + } + if colPos, ok := colPosMap[colName]; ok { + ne.Col.RelPos = relPos + ne.Col.ColPos = colPos + } else { + err = moerr.NewInternalErrorNoCtx("can not find col by name in partition expr") + return + } + + case *plan.Expr_F: + for _, arg := range ne.F.GetArgs() { + if err = remapPartitionExprColRef(arg, tableName, relPos, colPosMap, containTableName); err != nil { + return + } + } + + case *plan.Expr_W: + if err = remapPartitionExprColRef(ne.W.WindowFunc, tableName, relPos, colPosMap, containTableName); err != nil { + return + } + + for _, arg := range ne.W.PartitionBy { + if err = remapPartitionExprColRef(arg, tableName, relPos, colPosMap, containTableName); err != nil { + return + } + } + for _, order := range ne.W.OrderBy { + if err = remapPartitionExprColRef(order.Expr, tableName, relPos, colPosMap, containTableName); err != nil { + return + } + } + } + return +} + +func getPartColsFromExpr(expr *Expr, colNameMap map[string]bool) { + switch ne := expr.Expr.(type) { + case *plan.Expr_Col: + colNameMap[ne.Col.Name] = true + + case *plan.Expr_F: + for _, arg := range ne.F.GetArgs() { + getPartColsFromExpr(arg, colNameMap) + } + + case *plan.Expr_W: + getPartColsFromExpr(ne.W.WindowFunc, colNameMap) + + for _, arg := range ne.W.PartitionBy { + getPartColsFromExpr(arg, colNameMap) + } + for _, order := range ne.W.OrderBy { + getPartColsFromExpr(order.Expr, colNameMap) + } + } +} diff --git a/pkg/sql/plan/deepcopy.go b/pkg/sql/plan/deepcopy.go index 740c2c5139f5..324ad1dd195c 100644 --- a/pkg/sql/plan/deepcopy.go +++ b/pkg/sql/plan/deepcopy.go @@ -16,8 +16,7 @@ package plan import ( "bytes" - - "golang.org/x/exp/constraints" + "slices" "github.com/matrixorigin/matrixone/pkg/pb/plan" ) @@ -61,16 +60,32 @@ func DeepCopyObjectRef(ref *plan.ObjectRef) *plan.ObjectRef { } } -func DeepCopyOnDupliateKeyCtx(ctx *plan.OnDuplicateKeyCtx) *plan.OnDuplicateKeyCtx { +func DeepCopyUpdateCtxList(updateCtxList []*plan.UpdateCtx) []*plan.UpdateCtx { + result := make([]*plan.UpdateCtx, len(updateCtxList)) + for i, ctx := range updateCtxList { + result[i] = &plan.UpdateCtx{ + ObjRef: DeepCopyObjectRef(ctx.ObjRef), + TableDef: DeepCopyTableDef(ctx.TableDef, true), + PartitionTableIds: slices.Clone(ctx.PartitionTableIds), + PartitionTableNames: slices.Clone(ctx.PartitionTableNames), + OldPartitionIdx: ctx.OldPartitionIdx, + NewPartitionIdx: ctx.NewPartitionIdx, + InsertCols: slices.Clone(ctx.InsertCols), + DeleteCols: slices.Clone(ctx.DeleteCols), + } + } + + return result +} + +func DeepCopyOnDuplicateKeyCtx(ctx *plan.OnDuplicateKeyCtx) *plan.OnDuplicateKeyCtx { if ctx == nil { return nil } newCtx := &plan.OnDuplicateKeyCtx{ - OnDuplicateIdx: make([]int32, len(ctx.OnDuplicateIdx)), + OnDuplicateIdx: slices.Clone(ctx.OnDuplicateIdx), } - copy(newCtx.OnDuplicateIdx, ctx.OnDuplicateIdx) - if ctx.OnDuplicateExpr != nil { newCtx.OnDuplicateExpr = make(map[string]*Expr) for k, v := range ctx.OnDuplicateExpr { @@ -90,12 +105,11 @@ func DeepCopyInsertCtx(ctx *plan.InsertCtx) *plan.InsertCtx { AddAffectedRows: ctx.AddAffectedRows, IsClusterTable: ctx.IsClusterTable, TableDef: DeepCopyTableDef(ctx.TableDef, true), - PartitionTableIds: make([]uint64, len(ctx.PartitionTableIds)), - PartitionTableNames: make([]string, len(ctx.PartitionTableNames)), + PartitionTableIds: slices.Clone(ctx.PartitionTableIds), + PartitionTableNames: slices.Clone(ctx.PartitionTableNames), PartitionIdx: ctx.PartitionIdx, } - copy(newCtx.PartitionTableIds, ctx.PartitionTableIds) - copy(newCtx.PartitionTableNames, ctx.PartitionTableNames) + return newCtx } @@ -110,13 +124,12 @@ func DeepCopyDeleteCtx(ctx *plan.DeleteCtx) *plan.DeleteCtx { Ref: DeepCopyObjectRef(ctx.Ref), IsClusterTable: ctx.IsClusterTable, TableDef: DeepCopyTableDef(ctx.TableDef, true), - PartitionTableIds: make([]uint64, len(ctx.PartitionTableIds)), - PartitionTableNames: make([]string, len(ctx.PartitionTableNames)), + PartitionTableIds: slices.Clone(ctx.PartitionTableIds), + PartitionTableNames: slices.Clone(ctx.PartitionTableNames), PartitionIdx: ctx.PartitionIdx, PrimaryKeyIdx: ctx.PrimaryKeyIdx, } - copy(newCtx.PartitionTableIds, ctx.PartitionTableIds) - copy(newCtx.PartitionTableNames, ctx.PartitionTableNames) + return newCtx } @@ -139,12 +152,11 @@ func DeepCopyPreInsertUkCtx(ctx *plan.PreInsertUkCtx) *plan.PreInsertUkCtx { return nil } newCtx := &plan.PreInsertUkCtx{ - Columns: make([]int32, len(ctx.Columns)), + Columns: slices.Clone(ctx.Columns), PkColumn: ctx.PkColumn, PkType: ctx.PkType, UkType: ctx.UkType, } - copy(newCtx.Columns, ctx.Columns) return newCtx } @@ -154,9 +166,8 @@ func DeepCopyPreDeleteCtx(ctx *plan.PreDeleteCtx) *plan.PreDeleteCtx { return nil } newCtx := &plan.PreDeleteCtx{ - Idx: make([]int32, len(ctx.Idx)), + Idx: slices.Clone(ctx.Idx), } - copy(newCtx.Idx, ctx.Idx) return newCtx } @@ -178,84 +189,55 @@ func DeepCopyLockTarget(target *plan.LockTarget) *plan.LockTarget { func DeepCopyNode(node *plan.Node) *plan.Node { newNode := &Node{ - NodeType: node.NodeType, - NodeId: node.NodeId, - ExtraOptions: node.ExtraOptions, - Children: make([]int32, len(node.Children)), - JoinType: node.JoinType, - BuildOnLeft: node.BuildOnLeft, - BindingTags: make([]int32, len(node.BindingTags)), - Limit: DeepCopyExpr(node.Limit), - Offset: DeepCopyExpr(node.Offset), - ProjectList: make([]*plan.Expr, len(node.ProjectList)), - OnList: make([]*plan.Expr, len(node.OnList)), - FilterList: make([]*plan.Expr, len(node.FilterList)), - BlockFilterList: make([]*plan.Expr, len(node.BlockFilterList)), - GroupBy: make([]*plan.Expr, len(node.GroupBy)), - AggList: make([]*plan.Expr, len(node.AggList)), - OrderBy: make([]*plan.OrderBySpec, len(node.OrderBy)), - DeleteCtx: DeepCopyDeleteCtx(node.DeleteCtx), - TblFuncExprList: make([]*plan.Expr, len(node.TblFuncExprList)), - ClusterTable: DeepCopyClusterTable(node.GetClusterTable()), - InsertCtx: DeepCopyInsertCtx(node.InsertCtx), - ReplaceCtx: DeepCopyReplaceCtx(node.ReplaceCtx), - NotCacheable: node.NotCacheable, - SourceStep: node.SourceStep, - PreInsertCtx: DeepCopyPreInsertCtx(node.PreInsertCtx), - PreInsertUkCtx: DeepCopyPreInsertUkCtx(node.PreInsertUkCtx), - PreDeleteCtx: DeepCopyPreDeleteCtx(node.PreDeleteCtx), - OnDuplicateKey: DeepCopyOnDupliateKeyCtx(node.OnDuplicateKey), - LockTargets: make([]*plan.LockTarget, len(node.LockTargets)), - AnalyzeInfo: DeepCopyAnalyzeInfo(node.AnalyzeInfo), - IsEnd: node.IsEnd, - ExternScan: node.ExternScan, - PartitionPrune: DeepCopyPartitionPrune(node.PartitionPrune), - SampleFunc: DeepCopySampleFuncSpec(node.SampleFunc), - OnUpdateExprs: make([]*plan.Expr, len(node.OnUpdateExprs)), + NodeType: node.NodeType, + NodeId: node.NodeId, + ExtraOptions: node.ExtraOptions, + Children: slices.Clone(node.Children), + JoinType: node.JoinType, + BuildOnLeft: node.BuildOnLeft, + BindingTags: slices.Clone(node.BindingTags), + Limit: DeepCopyExpr(node.Limit), + Offset: DeepCopyExpr(node.Offset), + ProjectList: DeepCopyExprList(node.ProjectList), + OnList: DeepCopyExprList(node.OnList), + FilterList: DeepCopyExprList(node.FilterList), + BlockFilterList: DeepCopyExprList(node.BlockFilterList), + GroupBy: DeepCopyExprList(node.GroupBy), + GroupingFlag: slices.Clone(node.GroupingFlag), + AggList: DeepCopyExprList(node.AggList), + OrderBy: make([]*plan.OrderBySpec, len(node.OrderBy)), + DeleteCtx: DeepCopyDeleteCtx(node.DeleteCtx), + TblFuncExprList: DeepCopyExprList(node.TblFuncExprList), + ClusterTable: DeepCopyClusterTable(node.GetClusterTable()), + InsertCtx: DeepCopyInsertCtx(node.InsertCtx), + ReplaceCtx: DeepCopyReplaceCtx(node.ReplaceCtx), + NotCacheable: node.NotCacheable, + SourceStep: node.SourceStep, + PreInsertCtx: DeepCopyPreInsertCtx(node.PreInsertCtx), + PreInsertUkCtx: DeepCopyPreInsertUkCtx(node.PreInsertUkCtx), + PreDeleteCtx: DeepCopyPreDeleteCtx(node.PreDeleteCtx), + OnDuplicateKey: DeepCopyOnDuplicateKeyCtx(node.OnDuplicateKey), + LockTargets: make([]*plan.LockTarget, len(node.LockTargets)), + AnalyzeInfo: DeepCopyAnalyzeInfo(node.AnalyzeInfo), + IsEnd: node.IsEnd, + ExternScan: node.ExternScan, + PartitionPrune: DeepCopyPartitionPrune(node.PartitionPrune), + SampleFunc: DeepCopySampleFuncSpec(node.SampleFunc), + OnUpdateExprs: DeepCopyExprList(node.OnUpdateExprs), + InsertDeleteCols: DeepCopyExprList(node.InsertDeleteCols), + DedupColName: node.DedupColName, + DedupColTypes: slices.Clone(node.DedupColTypes), } newNode.Uuid = append(newNode.Uuid, node.Uuid...) - copy(newNode.Children, node.Children) - copy(newNode.BindingTags, node.BindingTags) - for idx, target := range node.LockTargets { newNode.LockTargets[idx] = DeepCopyLockTarget(target) } - for idx, expr := range node.ProjectList { - newNode.ProjectList[idx] = DeepCopyExpr(expr) - } - - for idx, expr := range node.OnList { - newNode.OnList[idx] = DeepCopyExpr(expr) - } - - for idx, expr := range node.FilterList { - newNode.FilterList[idx] = DeepCopyExpr(expr) - } - - for idx, expr := range node.BlockFilterList { - newNode.BlockFilterList[idx] = DeepCopyExpr(expr) - } - - for idx, expr := range node.GroupBy { - newNode.GroupBy[idx] = DeepCopyExpr(expr) - } - - copy(newNode.GroupingFlag, node.GroupingFlag) - - for idx, expr := range node.AggList { - newNode.AggList[idx] = DeepCopyExpr(expr) - } - for idx, orderBy := range node.OrderBy { newNode.OrderBy[idx] = DeepCopyOrderBy(orderBy) } - for idx, expr := range node.OnUpdateExprs { - newNode.OnUpdateExprs[idx] = DeepCopyExpr(expr) - } - newNode.Stats = DeepCopyStats(node.Stats) newNode.ObjRef = DeepCopyObjectRef(node.ObjRef) @@ -282,9 +264,6 @@ func DeepCopyNode(node *plan.Node) *plan.Node { newNode.RowsetData.Cols[idx] = DeepCopyColData(col) } } - for idx, expr := range node.TblFuncExprList { - newNode.TblFuncExprList[idx] = DeepCopyExpr(expr) - } return newNode } @@ -299,13 +278,11 @@ func DeepCopyReplaceCtx(oldCtx *plan.ReplaceCtx) *plan.ReplaceCtx { IsClusterTable: oldCtx.IsClusterTable, TableDef: DeepCopyTableDef(oldCtx.TableDef, true), DeleteCond: oldCtx.DeleteCond, - PartitionTableIds: make([]uint64, len(oldCtx.PartitionTableIds)), - PartitionTableNames: make([]string, len(oldCtx.PartitionTableNames)), + PartitionTableIds: slices.Clone(oldCtx.PartitionTableIds), + PartitionTableNames: slices.Clone(oldCtx.PartitionTableNames), PartitionIdx: oldCtx.PartitionIdx, RewriteFromOnDuplicateKey: oldCtx.RewriteFromOnDuplicateKey, } - copy(ctx.PartitionTableIds, oldCtx.PartitionTableIds) - copy(ctx.PartitionTableNames, oldCtx.PartitionTableNames) return ctx } @@ -363,9 +340,8 @@ func DeepCopyPrimaryKeyDef(pkeyDef *plan.PrimaryKeyDef) *plan.PrimaryKeyDef { } def := &plan.PrimaryKeyDef{ PkeyColName: pkeyDef.PkeyColName, - Names: make([]string, len(pkeyDef.Names)), + Names: slices.Clone(pkeyDef.Names), } - copy(def.Names, pkeyDef.Names) // Check whether the composite primary key column is included if pkeyDef.CompPkeyCol != nil { def.CompPkeyCol = DeepCopyColDef(pkeyDef.CompPkeyCol) @@ -388,12 +364,9 @@ func DeepCopyIndexDef(indexDef *plan.IndexDef) *plan.IndexDef { IndexAlgo: indexDef.IndexAlgo, IndexAlgoTableType: indexDef.IndexAlgoTableType, IndexAlgoParams: indexDef.IndexAlgoParams, + Parts: slices.Clone(indexDef.Parts), } newindexDef.Option = DeepCopyIndexOption(indexDef.Option) - - newParts := make([]string, len(indexDef.Parts)) - copy(newParts, indexDef.Parts) - newindexDef.Parts = newParts return newindexDef } @@ -475,7 +448,7 @@ func DeepCopyTableDef(table *plan.TableDef, withCols bool) *plan.TableDef { Pkey: DeepCopyPrimaryKeyDef(table.Pkey), Indexes: make([]*IndexDef, len(table.Indexes)), Fkeys: make([]*plan.ForeignKeyDef, len(table.Fkeys)), - RefChildTbls: make([]uint64, len(table.RefChildTbls)), + RefChildTbls: slices.Clone(table.RefChildTbls), Checks: make([]*plan.CheckDef, len(table.Checks)), Props: make([]*plan.PropertyDef, len(table.Props)), Defs: make([]*plan.TableDef_DefType, len(table.Defs)), @@ -488,8 +461,6 @@ func DeepCopyTableDef(table *plan.TableDef, withCols bool) *plan.TableDef { DbId: table.DbId, } - copy(newTable.RefChildTbls, table.RefChildTbls) - if withCols { newTable.Cols = make([]*plan.ColDef, len(table.Cols)) for idx, col := range table.Cols { @@ -518,9 +489,8 @@ func DeepCopyTableDef(table *plan.TableDef, withCols bool) *plan.TableDef { if table.TblFunc != nil { newTable.TblFunc = &plan.TableFunction{ Name: table.TblFunc.Name, - Param: make([]byte, len(table.TblFunc.Param)), + Param: slices.Clone(table.TblFunc.Param), } - copy(newTable.TblFunc.Param, table.TblFunc.Param) } if table.ClusterBy != nil { @@ -593,12 +563,9 @@ func DeepCopyQuery(qry *plan.Query) *plan.Query { StmtType: qry.StmtType, Steps: qry.Steps, Nodes: make([]*plan.Node, len(qry.Nodes)), - Params: make([]*plan.Expr, len(qry.Params)), + Params: DeepCopyExprList(qry.Params), Headings: qry.Headings, } - for idx, param := range qry.Params { - newQry.Params[idx] = DeepCopyExpr(param) - } for idx, node := range qry.Nodes { newQry.Nodes[idx] = DeepCopyNode(node) } @@ -673,17 +640,13 @@ func DeepCopyDataDefinition(old *plan.DataDefinition) *plan.DataDefinition { Database: df.CreateTable.Database, TableDef: DeepCopyTableDef(df.CreateTable.TableDef, true), IndexTables: DeepCopyTableDefList(df.CreateTable.GetIndexTables()), - FkDbs: make([]string, len(df.CreateTable.FkDbs)), - FkTables: make([]string, len(df.CreateTable.FkTables)), + FkDbs: slices.Clone(df.CreateTable.FkDbs), + FkTables: slices.Clone(df.CreateTable.FkTables), FkCols: make([]*plan.FkColName, len(df.CreateTable.FkCols)), PartitionTables: DeepCopyTableDefList(df.CreateTable.GetPartitionTables()), } - copy(CreateTable.FkDbs, df.CreateTable.FkDbs) - copy(CreateTable.FkTables, df.CreateTable.FkTables) for i, val := range df.CreateTable.FkCols { - cols := &plan.FkColName{Cols: make([]string, len(val.Cols))} - copy(cols.Cols, val.Cols) - CreateTable.FkCols[i] = cols + CreateTable.FkCols[i] = &plan.FkColName{Cols: slices.Clone(val.Cols)} } newDf.Definition = &plan.DataDefinition_CreateTable{ CreateTable: CreateTable, @@ -716,11 +679,10 @@ func DeepCopyDataDefinition(old *plan.DataDefinition) *plan.DataDefinition { AddFk: &plan.AlterTableAddFk{ DbName: act.AddFk.DbName, TableName: act.AddFk.TableName, - Cols: make([]string, len(act.AddFk.Cols)), + Cols: slices.Clone(act.AddFk.Cols), Fkey: DeepCopyFkey(act.AddFk.Fkey), }, } - copy(AddFk.AddFk.Cols, act.AddFk.Cols) AlterTable.Actions[i] = &plan.AlterTable_Action{ Action: AddFk, } @@ -737,11 +699,11 @@ func DeepCopyDataDefinition(old *plan.DataDefinition) *plan.DataDefinition { IfExists: df.DropTable.IfExists, Database: df.DropTable.Database, Table: df.DropTable.Table, - IndexTableNames: DeepCopyStringList(df.DropTable.GetIndexTableNames()), + IndexTableNames: slices.Clone(df.DropTable.GetIndexTableNames()), ClusterTable: DeepCopyClusterTable(df.DropTable.GetClusterTable()), TableId: df.DropTable.GetTableId(), - ForeignTbl: DeepCopyNumberList(df.DropTable.GetForeignTbl()), - PartitionTableNames: DeepCopyStringList(df.DropTable.GetPartitionTableNames()), + ForeignTbl: slices.Clone(df.DropTable.GetForeignTbl()), + PartitionTableNames: slices.Clone(df.DropTable.GetPartitionTableNames()), IsView: df.DropTable.IsView, TableDef: DeepCopyTableDef(df.DropTable.GetTableDef(), true), }, @@ -784,9 +746,8 @@ func DeepCopyDataDefinition(old *plan.DataDefinition) *plan.DataDefinition { Database: df.TruncateTable.Database, Table: df.TruncateTable.Table, ClusterTable: DeepCopyClusterTable(df.TruncateTable.GetClusterTable()), - IndexTableNames: make([]string, len(df.TruncateTable.IndexTableNames)), + IndexTableNames: slices.Clone(df.TruncateTable.IndexTableNames), } - copy(truncateTable.IndexTableNames, df.TruncateTable.IndexTableNames) newDf.Definition = &plan.DataDefinition_TruncateTable{ TruncateTable: truncateTable, } @@ -794,10 +755,7 @@ func DeepCopyDataDefinition(old *plan.DataDefinition) *plan.DataDefinition { case *plan.DataDefinition_ShowVariables: showVariables := &plan.ShowVariables{ Global: df.ShowVariables.Global, - Where: make([]*plan.Expr, len(df.ShowVariables.Where)), - } - for i, e := range df.ShowVariables.Where { - showVariables.Where[i] = DeepCopyExpr(e) + Where: DeepCopyExprList(df.ShowVariables.Where), } newDf.Definition = &plan.DataDefinition_ShowVariables{ @@ -833,14 +791,12 @@ func DeepCopyDataDefinition(old *plan.DataDefinition) *plan.DataDefinition { func DeepCopyFkey(fkey *ForeignKeyDef) *ForeignKeyDef { def := &ForeignKeyDef{ Name: fkey.Name, - Cols: make([]uint64, len(fkey.Cols)), + Cols: slices.Clone(fkey.Cols), ForeignTbl: fkey.ForeignTbl, - ForeignCols: make([]uint64, len(fkey.ForeignCols)), + ForeignCols: slices.Clone(fkey.ForeignCols), OnDelete: fkey.OnDelete, OnUpdate: fkey.OnUpdate, } - copy(def.Cols, fkey.Cols) - copy(def.ForeignCols, fkey.ForeignCols) return def } @@ -962,10 +918,6 @@ func DeepCopyExpr(expr *Expr) *Expr { } case *plan.Expr_W: - ps := make([]*Expr, len(item.W.PartitionBy)) - for i, p := range item.W.PartitionBy { - ps[i] = DeepCopyExpr(p) - } os := make([]*OrderBySpec, len(item.W.OrderBy)) for i, o := range item.W.OrderBy { os[i] = DeepCopyOrderBy(o) @@ -974,7 +926,7 @@ func DeepCopyExpr(expr *Expr) *Expr { newExpr.Expr = &plan.Expr_W{ W: &plan.WindowSpec{ WindowFunc: DeepCopyExpr(item.W.WindowFunc), - PartitionBy: ps, + PartitionBy: DeepCopyExprList(item.W.PartitionBy), OrderBy: os, Name: item.W.Name, Frame: &plan.FrameClause{ @@ -1026,14 +978,10 @@ func DeepCopyExpr(expr *Expr) *Expr { } case *plan.Expr_List: - e := &plan.ExprList{ - List: make([]*plan.Expr, len(item.List.List)), - } - for i, ie := range item.List.List { - e.List[i] = DeepCopyExpr(ie) - } newExpr.Expr = &plan.Expr_List{ - List: e, + List: &plan.ExprList{ + List: DeepCopyExprList(item.List.List), + }, } case *plan.Expr_Vec: @@ -1053,25 +1001,14 @@ func DeepCopyClusterTable(cluster *plan.ClusterTable) *plan.ClusterTable { return nil } - accountIds := make([]uint32, len(cluster.GetAccountIDs())) - copy(accountIds, cluster.GetAccountIDs()) newClusterTable := &plan.ClusterTable{ IsClusterTable: cluster.GetIsClusterTable(), - AccountIDs: accountIds, + AccountIDs: slices.Clone(cluster.GetAccountIDs()), ColumnIndexOfAccountId: cluster.GetColumnIndexOfAccountId(), } return newClusterTable } -func DeepCopySliceInt64(s []int64) []int64 { - if s == nil { - return nil - } - result := make([]int64, 0, len(s)) - result = append(result, s...) - return result -} - func DeepCopyAnalyzeInfo(analyzeinfo *plan.AnalyzeInfo) *plan.AnalyzeInfo { if analyzeinfo == nil { return nil @@ -1084,8 +1021,8 @@ func DeepCopyAnalyzeInfo(analyzeinfo *plan.AnalyzeInfo) *plan.AnalyzeInfo { InputSize: analyzeinfo.GetInputSize(), OutputSize: analyzeinfo.GetOutputSize(), TimeConsumed: analyzeinfo.GetTimeConsumed(), - TimeConsumedArrayMajor: DeepCopySliceInt64(analyzeinfo.GetTimeConsumedArrayMajor()), - TimeConsumedArrayMinor: DeepCopySliceInt64(analyzeinfo.GetTimeConsumedArrayMinor()), + TimeConsumedArrayMajor: slices.Clone(analyzeinfo.GetTimeConsumedArrayMajor()), + TimeConsumedArrayMinor: slices.Clone(analyzeinfo.GetTimeConsumedArrayMinor()), MemorySize: analyzeinfo.GetMemorySize(), WaitTimeConsumed: analyzeinfo.GetWaitTimeConsumed(), DiskIO: analyzeinfo.GetDiskIO(), @@ -1103,24 +1040,6 @@ func DeepCopyAnalyzeInfo(analyzeinfo *plan.AnalyzeInfo) *plan.AnalyzeInfo { } } -func DeepCopyStringList(src []string) []string { - if src == nil { - return nil - } - ret := make([]string, len(src)) - copy(ret, src) - return ret -} - -func DeepCopyNumberList[T constraints.Integer](src []T) []T { - if src == nil { - return nil - } - ret := make([]T, len(src)) - copy(ret, src) - return ret -} - func DeepCopyPartitionByDef(partiiondef *PartitionByDef) *PartitionByDef { partitionDef := &plan.PartitionByDef{ Type: partiiondef.GetType(), @@ -1130,7 +1049,7 @@ func DeepCopyPartitionByDef(partiiondef *PartitionByDef) *PartitionByDef { Algorithm: partiiondef.GetAlgorithm(), IsSubPartition: partiiondef.GetIsSubPartition(), PartitionMsg: partiiondef.GetPartitionMsg(), - PartitionTableNames: DeepCopyStringList(partiiondef.GetPartitionTableNames()), + PartitionTableNames: slices.Clone(partiiondef.GetPartitionTableNames()), } if partiiondef.PartitionExpr != nil { partitionDef.PartitionExpr = &plan.PartitionExpr{ @@ -1142,7 +1061,7 @@ func DeepCopyPartitionByDef(partiiondef *PartitionByDef) *PartitionByDef { if partiiondef.PartitionColumns != nil { partitionDef.PartitionColumns = &plan.PartitionColumns{ Columns: DeepCopyExprList(partiiondef.PartitionColumns.Columns), - PartitionColumns: DeepCopyStringList(partiiondef.PartitionColumns.PartitionColumns), + PartitionColumns: slices.Clone(partiiondef.PartitionColumns.PartitionColumns), } } diff --git a/pkg/sql/plan/dml_context.go b/pkg/sql/plan/dml_context.go new file mode 100644 index 000000000000..7742a946fe1d --- /dev/null +++ b/pkg/sql/plan/dml_context.go @@ -0,0 +1,276 @@ +// Copyright 2021 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package plan + +import ( + "github.com/matrixorigin/matrixone/pkg/catalog" + "github.com/matrixorigin/matrixone/pkg/common/moerr" + "github.com/matrixorigin/matrixone/pkg/container/types" + "github.com/matrixorigin/matrixone/pkg/defines" + "github.com/matrixorigin/matrixone/pkg/pb/plan" + "github.com/matrixorigin/matrixone/pkg/sql/parsers/tree" + "github.com/matrixorigin/matrixone/pkg/sql/util" +) + +type DMLContext struct { + objRefs []*plan.ObjectRef + tableDefs []*plan.TableDef + aliases []string + isClusterTable []bool + + updateCol2Expr []map[string]tree.Expr // This slice index correspond to tableDefs + updatePartCol []bool //If update cols contains col that Partition expr used + //oldColPosMap []map[string]int // origin table values to their position in derived table + //newColPosMap []map[string]int // insert/update values to their position in derived table + //nameToIdx map[string]int // Mapping of table full path name to tableDefs index,such as: 'tpch.nation -> 0' + //idToName map[uint64]string // Mapping of tableId to full path name of table + aliasMap map[string]int // Mapping of table aliases to tableDefs array index,If there is no alias, replace it with the original name of the table +} + +func NewDMLContext() *DMLContext { + return &DMLContext{ + //nameToIdx: make(map[string]int), + //idToName: make(map[uint64]string), + aliasMap: make(map[string]int), + } +} + +func (dmlCtx *DMLContext) ResolveUpdateTables(ctx CompilerContext, stmt *tree.Update) error { + err := dmlCtx.ResolveTables(ctx, stmt.Tables, stmt.With, nil, false) + if err != nil { + return err + } + + // check update field and set updateKeys + usedTbl := make(map[string]map[string]tree.Expr) + allColumns := make(map[string]map[string]bool) + for alias, idx := range dmlCtx.aliasMap { + allColumns[alias] = make(map[string]bool) + for _, col := range dmlCtx.tableDefs[idx].Cols { + allColumns[alias][col.Name] = true + } + } + + appendToTbl := func(table, column string, expr tree.Expr) { + if _, exists := usedTbl[table]; !exists { + usedTbl[table] = make(map[string]tree.Expr) + } + usedTbl[table][column] = expr + } + + for _, updateExpr := range stmt.Exprs { + if len(updateExpr.Names) > 1 { + return moerr.NewNYI(ctx.GetContext(), "unsupport expr") + } + + parts := updateExpr.Names[0] + expr := updateExpr.Expr + if parts.NumParts > 1 { + colName := parts.ColName() + tblName := parts.TblName() + if _, tblExists := dmlCtx.aliasMap[tblName]; tblExists { + if allColumns[tblName][colName] { + appendToTbl(tblName, colName, expr) + } else { + return moerr.NewInternalErrorf(ctx.GetContext(), "column '%v' not found in table %s", parts.ColNameOrigin(), parts.TblNameOrigin()) + } + } else { + return moerr.NewNoSuchTable(ctx.GetContext(), "", parts.TblNameOrigin()) + } + } else { + colName := parts.ColName() + found := false + for alias, columns := range allColumns { + if columns[colName] { + if found { + return moerr.NewInternalErrorf(ctx.GetContext(), "Column '%v' in field list is ambiguous", parts.ColNameOrigin()) + } + found = true + appendToTbl(alias, colName, expr) + } + } + if !found && stmt.With != nil { + var str string + for i, c := range stmt.With.CTEs { + if i > 0 { + str += ", " + } + str += string(c.Name.Alias) + } + return moerr.NewInternalErrorf(ctx.GetContext(), "column '%v' not found in table or the target table %s of the UPDATE is not updatable", parts.ColNameOrigin(), str) + } else if !found { + return moerr.NewInternalErrorf(ctx.GetContext(), "column '%v' not found in table", parts.ColNameOrigin()) + } + } + } + + if len(usedTbl) > 1 { + return moerr.NewUnsupportedDML(ctx.GetContext(), "multi-table update") + } + + dmlCtx.updateCol2Expr = make([]map[string]tree.Expr, len(dmlCtx.tableDefs)) + for alias, columnMap := range usedTbl { + idx := dmlCtx.aliasMap[alias] + dmlCtx.updateCol2Expr[idx] = columnMap + } + + dmlCtx.updatePartCol = make([]bool, len(dmlCtx.tableDefs)) + + return nil +} + +func (dmlCtx *DMLContext) ResolveTables(ctx CompilerContext, tableExprs tree.TableExprs, with *tree.With, aliasMap map[string][2]string, respectFKCheck bool) error { + cteMap := make(map[string]bool) + if with != nil { + for _, cte := range with.CTEs { + cteMap[string(cte.Name.Alias)] = true + } + } + + for _, tbl := range tableExprs { + err := dmlCtx.ResolveSingleTable(ctx, tbl, aliasMap, cteMap, respectFKCheck) + if err != nil { + return err + } + } + + return nil +} + +func (dmlCtx *DMLContext) ResolveSingleTable(ctx CompilerContext, tbl tree.TableExpr, aliasMap map[string][2]string, withMap map[string]bool, respectFKCheck bool) error { + var tblName, dbName, alias string + + if aliasTbl, ok := tbl.(*tree.AliasedTableExpr); ok { + alias = string(aliasTbl.As.Alias) + tbl = aliasTbl.Expr + } + + for { + if baseTbl, ok := tbl.(*tree.ParenTableExpr); ok { + tbl = baseTbl.Expr + } else { + break + } + } + + //if joinTbl, ok := tbl.(*tree.JoinTableExpr); ok { + // dmlCtx.needAggFilter = true + // err := setTableExprToDmlTableInfo(ctx, joinTbl.Left, dmlCtx, aliasMap, withMap) + // if err != nil { + // return err + // } + // if joinTbl.Right != nil { + // return setTableExprToDmlTableInfo(ctx, joinTbl.Right, dmlCtx, aliasMap, withMap) + // } + // return nil + //} + + if baseTbl, ok := tbl.(*tree.TableName); ok { + dbName = string(baseTbl.SchemaName) + tblName = string(baseTbl.ObjectName) + } else { + return moerr.NewUnsupportedDML(ctx.GetContext(), "unsupported table type") + } + + if withMap[tblName] { + return nil + } + + if aliasNames, exist := aliasMap[tblName]; exist { + alias = tblName // work in delete statement + dbName = aliasNames[0] + tblName = aliasNames[1] + } + + if len(tblName) == 0 { + return moerr.NewUnsupportedDML(ctx.GetContext(), "empty table name") + } + + if len(dbName) == 0 { + dbName = ctx.DefaultDatabase() + } + + objRef, tableDef := ctx.Resolve(dbName, tblName, nil) + if tableDef == nil { + return moerr.NewNoSuchTable(ctx.GetContext(), dbName, tblName) + } + + if tableDef.TableType == catalog.SystemSourceRel { + return moerr.NewInvalidInput(ctx.GetContext(), "cannot insert/update/delete from source") + } else if tableDef.TableType == catalog.SystemExternalRel { + return moerr.NewInvalidInput(ctx.GetContext(), "cannot insert/update/delete from external table") + } else if tableDef.TableType == catalog.SystemViewRel { + return moerr.NewInvalidInput(ctx.GetContext(), "cannot insert/update/delete from view") + } else if tableDef.TableType == catalog.SystemSequenceRel && ctx.GetContext().Value(defines.BgKey{}) == nil { + return moerr.NewInvalidInput(ctx.GetContext(), "Cannot insert/update/delete from sequence") + } + + var err error + checkFK := true + if respectFKCheck { + checkFK, err = IsForeignKeyChecksEnabled(ctx) + if err != nil { + return err + } + } + + if checkFK && (len(tableDef.Fkeys) > 0 || len(tableDef.RefChildTbls) > 0) { + return moerr.NewUnsupportedDML(ctx.GetContext(), "foreign key constraint") + } + + for _, col := range tableDef.Cols { + if types.T(col.Typ.Id).IsArrayRelate() { + return moerr.NewUnsupportedDML(ctx.GetContext(), "vector column") + } + } + + isClusterTable := util.TableIsClusterTable(tableDef.GetTableType()) + accountId, err := ctx.GetAccountId() + if err != nil { + return err + } + if isClusterTable && accountId != catalog.System_Account { + return moerr.NewInternalError(ctx.GetContext(), "only the sys account can insert/update/delete the cluster table") + } + + if util.TableIsClusterTable(tableDef.GetTableType()) && accountId != catalog.System_Account { + return moerr.NewInternalErrorf(ctx.GetContext(), "only the sys account can insert/update/delete the cluster table %s", tableDef.GetName()) + } + if objRef.PubInfo != nil { + return moerr.NewInternalError(ctx.GetContext(), "cannot insert/update/delete from public table") + } + + if len(tableDef.Name2ColIndex) == 0 { + tableDef.Name2ColIndex = make(map[string]int32) + for colIdx, col := range tableDef.Cols { + tableDef.Name2ColIndex[col.Name] = int32(colIdx) + } + } + + nowIdx := len(dmlCtx.tableDefs) + dmlCtx.isClusterTable = append(dmlCtx.isClusterTable, isClusterTable) + dmlCtx.objRefs = append(dmlCtx.objRefs, objRef) + dmlCtx.tableDefs = append(dmlCtx.tableDefs, tableDef) + //key := dbName + "." + tblName + //dmlCtx.nameToIdx[key] = nowIdx + //dmlCtx.idToName[tableDef.TblId] = key + if alias == "" { + alias = tblName + } + dmlCtx.aliases = append(dmlCtx.aliases, alias) + dmlCtx.aliasMap[alias] = nowIdx + + return nil +} diff --git a/pkg/sql/plan/explain/explain_expr.go b/pkg/sql/plan/explain/explain_expr.go index 9583b34e8f16..0cf0a7882516 100644 --- a/pkg/sql/plan/explain/explain_expr.go +++ b/pkg/sql/plan/explain/explain_expr.go @@ -22,14 +22,12 @@ import ( "strings" "github.com/matrixorigin/matrixone/pkg/catalog" - - "github.com/matrixorigin/matrixone/pkg/vm/message" - "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/container/types" "github.com/matrixorigin/matrixone/pkg/container/vector" "github.com/matrixorigin/matrixone/pkg/pb/plan" "github.com/matrixorigin/matrixone/pkg/sql/plan/function" + "github.com/matrixorigin/matrixone/pkg/vm/message" ) func describeMessage(m *plan.MsgHeader, buf *bytes.Buffer) { @@ -41,18 +39,22 @@ func describeMessage(m *plan.MsgHeader, buf *bytes.Buffer) { buf.WriteString("]") } +func describeColRef(col *plan.ColRef, buf *bytes.Buffer) { + if len(col.Name) > 0 && !strings.HasPrefix(col.Name, catalog.PrefixIndexTableName) { + buf.WriteString(col.Name) + } else { + buf.WriteString("#[") + buf.WriteString(strconv.Itoa(int(col.RelPos))) + buf.WriteString(",") + buf.WriteString(strconv.Itoa(int(col.ColPos))) + buf.WriteString("]") + } +} + func describeExpr(ctx context.Context, expr *plan.Expr, options *ExplainOptions, buf *bytes.Buffer) error { switch exprImpl := expr.Expr.(type) { case *plan.Expr_Col: - if len(exprImpl.Col.Name) > 0 && !strings.HasPrefix(exprImpl.Col.Name, catalog.PrefixIndexTableName) { - buf.WriteString(exprImpl.Col.Name) - } else { - buf.WriteString("#[") - buf.WriteString(strconv.Itoa(int(exprImpl.Col.RelPos))) - buf.WriteString(",") - buf.WriteString(strconv.Itoa(int(exprImpl.Col.ColPos))) - buf.WriteString("]") - } + describeColRef(exprImpl.Col, buf) case *plan.Expr_Lit: if exprImpl.Lit.Isnull { diff --git a/pkg/sql/plan/explain/explain_node.go b/pkg/sql/plan/explain/explain_node.go index 7c8b31b5bd2b..3918814230e2 100644 --- a/pkg/sql/plan/explain/explain_node.go +++ b/pkg/sql/plan/explain/explain_node.go @@ -21,12 +21,10 @@ import ( "sort" "strconv" - "github.com/matrixorigin/matrixone/pkg/vm/message" - - plan2 "github.com/matrixorigin/matrixone/pkg/sql/plan" - "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/pb/plan" + plan2 "github.com/matrixorigin/matrixone/pkg/sql/plan" + "github.com/matrixorigin/matrixone/pkg/vm/message" ) var _ NodeDescribe = &NodeDescribeImpl{} @@ -144,10 +142,10 @@ func (ndesc *NodeDescribeImpl) GetNodeBasicInfo(ctx context.Context, options *Ex pname = "Lock" case plan.Node_APPLY: pname = "CROSS APPLY" + case plan.Node_MULTI_UPDATE: + pname = "Multi Update" case plan.Node_POSTDML: pname = "Post DML" - //case plan.Node_MULTI_UPDATE: - // pname = "Multi Update" default: panic("error node type") } @@ -363,6 +361,14 @@ func (ndesc *NodeDescribeImpl) GetExtraInfo(ctx context.Context, options *Explai lines = append(lines, windowSpecListInfo) } + if len(ndesc.Node.InsertDeleteCols) > 0 { + colsInfo, err := ndesc.GetInsertDeleteColsInfo(ctx, options) + if err != nil { + return nil, err + } + lines = append(lines, colsInfo) + } + // Get Filter list info if len(ndesc.Node.FilterList) > 0 { filterInfo, err := ndesc.GetFilterConditionInfo(ctx, options) @@ -449,6 +455,7 @@ func (ndesc *NodeDescribeImpl) GetExtraInfo(ctx context.Context, options *Explai lines = append(lines, msgInfo) } } + return lines, nil } @@ -546,6 +553,71 @@ func (ndesc *NodeDescribeImpl) GetPartitionPruneInfo(ctx context.Context, option return buf.String(), nil } +func (ndesc *NodeDescribeImpl) GetInsertDeleteColsInfo(ctx context.Context, options *ExplainOptions) (string, error) { + buf := bytes.NewBuffer(make([]byte, 0, 512)) + if ndesc.Node.NodeType == plan.Node_INSERT { + buf.WriteString("Insert Columns: ") + } else { + buf.WriteString("Delete Columns: ") + } + if options.Format == EXPLAIN_FORMAT_TEXT { + first := true + for _, v := range ndesc.Node.InsertDeleteCols { + if !first { + buf.WriteString(", ") + } + first = false + err := describeExpr(ctx, v, options, buf) + if err != nil { + return "", err + } + } + } else if options.Format == EXPLAIN_FORMAT_JSON { + return "", moerr.NewNYI(ctx, "explain format json") + } else if options.Format == EXPLAIN_FORMAT_DOT { + return "", moerr.NewNYI(ctx, "explain format dot") + } + return buf.String(), nil +} + +func (ndesc *NodeDescribeImpl) GetUpdateCtxInfo(ctx context.Context, options *ExplainOptions) ([]string, error) { + var lines []string + if options.Format == EXPLAIN_FORMAT_TEXT { + for _, updateCtx := range ndesc.Node.UpdateCtxList { + buf := bytes.NewBuffer(make([]byte, 0, 512)) + buf.WriteString("Table: " + updateCtx.TableDef.Name) + if len(updateCtx.InsertCols) > 0 { + buf.WriteString(" Insert Columns: ") + first := true + for i := range updateCtx.InsertCols { + if !first { + buf.WriteString(", ") + } + first = false + describeColRef(&updateCtx.InsertCols[i], buf) + } + } + if len(updateCtx.DeleteCols) > 0 { + buf.WriteString(" Delete Columns: ") + first := true + for i := range updateCtx.DeleteCols { + if !first { + buf.WriteString(", ") + } + first = false + describeColRef(&updateCtx.DeleteCols[i], buf) + } + } + lines = append(lines, buf.String()) + } + } else if options.Format == EXPLAIN_FORMAT_JSON { + return nil, moerr.NewNYI(ctx, "explain format json") + } else if options.Format == EXPLAIN_FORMAT_DOT { + return nil, moerr.NewNYI(ctx, "explain format dot") + } + return lines, nil +} + func (ndesc *NodeDescribeImpl) GetFilterConditionInfo(ctx context.Context, options *ExplainOptions) (string, error) { buf := bytes.NewBuffer(make([]byte, 0, 512)) buf.WriteString("Filter Cond: ") diff --git a/pkg/sql/plan/explain/explain_query.go b/pkg/sql/plan/explain/explain_query.go index 1c8f038b5ac9..3718d2c5f389 100644 --- a/pkg/sql/plan/explain/explain_query.go +++ b/pkg/sql/plan/explain/explain_query.go @@ -20,7 +20,6 @@ import ( "fmt" "github.com/google/uuid" - "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/logutil" "github.com/matrixorigin/matrixone/pkg/pb/plan" @@ -78,7 +77,8 @@ const ( Label_Unique = "Unique" Label_Replace = "Replace" Label_Unknown = "Unknown" - Label_Meterial = "Meterial" + Label_Material = "Material" + Label_Dedup_Join = "Dedup join" Label_Apply = "Apply" Label_PostDml = "PostDml" ) @@ -280,6 +280,16 @@ func explainStep(ctx context.Context, step *plan.Node, nodes []*plan.Node, setti } } + if nodedescImpl.Node.NodeType == plan.Node_MULTI_UPDATE { + msgInfo, err := nodedescImpl.GetUpdateCtxInfo(ctx, options) + if err != nil { + return err + } + for _, line := range msgInfo { + settings.buffer.PushNewLine(line, false, settings.level) + } + } + if nodedescImpl.Node.NodeType == plan.Node_FUZZY_FILTER { buf := bytes.NewBuffer(make([]byte, 0, 360)) buf.WriteString("Build on: ") diff --git a/pkg/sql/plan/explain/marshal_query.go b/pkg/sql/plan/explain/marshal_query.go index 192ff6679425..ab057f17e2d9 100644 --- a/pkg/sql/plan/explain/marshal_query.go +++ b/pkg/sql/plan/explain/marshal_query.go @@ -638,7 +638,7 @@ func (m MarshalNodeImpl) GetNodeLabels(ctx context.Context, options *ExplainOpti }) case plan.Node_MATERIAL: labels = append(labels, models.Label{ - Name: Label_Meterial, + Name: Label_Material, Value: []string{}, }) case plan.Node_APPLY: diff --git a/pkg/sql/plan/function/func_builtin.go b/pkg/sql/plan/function/func_builtin.go index c771d0f50c36..8fdda6e337c9 100644 --- a/pkg/sql/plan/function/func_builtin.go +++ b/pkg/sql/plan/function/func_builtin.go @@ -1056,46 +1056,77 @@ func builtInUnixTimestampVarcharToDecimal128(parameters []*vector.Vector, result // XXX I just copy this function. func builtInHash(parameters []*vector.Vector, result vector.FunctionResultWrapper, proc *process.Process, length int, selectList *FunctionSelectList) error { fillStringGroupStr := func(keys [][]byte, vec *vector.Vector, n int, start int) { - area := vec.GetArea() - vs := vector.MustFixedColWithTypeCheck[types.Varlena](vec) - if !vec.GetNulls().Any() { - for i := 0; i < n; i++ { - keys[i] = append(keys[i], byte(0)) - keys[i] = append(keys[i], vs[i+start].GetByteSlice(area)...) + if vec.IsConst() { + area := vec.GetArea() + vs := vector.MustFixedColWithTypeCheck[types.Varlena](vec) + data := vs[0].GetByteSlice(area) + if vec.IsConstNull() { + for i := 0; i < n; i++ { + keys[i] = append(keys[i], byte(1)) + } + } else { + for i := 0; i < n; i++ { + keys[i] = append(keys[i], byte(0)) + keys[i] = append(keys[i], data...) + } } } else { - nsp := vec.GetNulls() - for i := 0; i < n; i++ { - hasNull := nsp.Contains(uint64(i + start)) - if hasNull { - keys[i] = append(keys[i], byte(1)) - } else { + area := vec.GetArea() + vs := vector.MustFixedColWithTypeCheck[types.Varlena](vec) + if !vec.GetNulls().Any() { + for i := 0; i < n; i++ { keys[i] = append(keys[i], byte(0)) keys[i] = append(keys[i], vs[i+start].GetByteSlice(area)...) } + } else { + nsp := vec.GetNulls() + for i := 0; i < n; i++ { + hasNull := nsp.Contains(uint64(i + start)) + if hasNull { + keys[i] = append(keys[i], byte(1)) + } else { + keys[i] = append(keys[i], byte(0)) + keys[i] = append(keys[i], vs[i+start].GetByteSlice(area)...) + } + } } } } fillGroupStr := func(keys [][]byte, vec *vector.Vector, n int, sz int, start int) { - data := unsafe.Slice(vector.GetPtrAt[byte](vec, 0), (n+start)*sz) - if !vec.GetNulls().Any() { - for i := 0; i < n; i++ { - keys[i] = append(keys[i], byte(0)) - keys[i] = append(keys[i], data[(i+start)*sz:(i+start+1)*sz]...) + if vec.IsConst() { + data := unsafe.Slice(vector.GetPtrAt[byte](vec, 0), vec.GetType().Size) + if vec.IsConstNull() { + for i := 0; i < n; i++ { + keys[i] = append(keys[i], byte(1)) + } + } else { + for i := 0; i < n; i++ { + keys[i] = append(keys[i], byte(0)) + keys[i] = append(keys[i], data...) + } } } else { - nsp := vec.GetNulls() - for i := 0; i < n; i++ { - isNull := nsp.Contains(uint64(i + start)) - if isNull { - keys[i] = append(keys[i], byte(1)) - } else { + data := unsafe.Slice(vector.GetPtrAt[byte](vec, 0), (n+start)*sz) + if !vec.GetNulls().Any() { + for i := 0; i < n; i++ { keys[i] = append(keys[i], byte(0)) keys[i] = append(keys[i], data[(i+start)*sz:(i+start+1)*sz]...) } + } else { + nsp := vec.GetNulls() + for i := 0; i < n; i++ { + isNull := nsp.Contains(uint64(i + start)) + if isNull { + keys[i] = append(keys[i], byte(1)) + } else { + keys[i] = append(keys[i], byte(0)) + keys[i] = append(keys[i], data[(i+start)*sz:(i+start+1)*sz]...) + } + } } } + } encodeHashKeys := func(keys [][]byte, vecs []*vector.Vector, start, count int) { @@ -1144,15 +1175,16 @@ func builtInHash(parameters []*vector.Vector, result vector.FunctionResultWrappe // result vec is [serial(1, 2, 3), serial(1, 2, 3), null] func (op *opSerial) BuiltInSerial(parameters []*vector.Vector, result vector.FunctionResultWrapper, proc *process.Process, length int, selectList *FunctionSelectList) error { rs := vector.MustFunctionResult[types.Varlena](result) - op.tryExpand(length, proc.Mp()) - - bitMap := new(nulls.Nulls) - for _, v := range parameters { - if v.IsConstNull() { - nulls.AddRange(rs.GetResultVector().GetNulls(), 0, uint64(length)) + if v.AllNull() { + rs.SetNullResult(uint64(length)) return nil } + } + + op.tryExpand(length, proc.Mp()) + bitMap := new(nulls.Nulls) + for _, v := range parameters { SerialHelper(v, bitMap, op.ps, false) } diff --git a/pkg/sql/plan/index_table_dml_test.go b/pkg/sql/plan/index_table_dml_test.go index b693172e6d96..98380b5e6139 100644 --- a/pkg/sql/plan/index_table_dml_test.go +++ b/pkg/sql/plan/index_table_dml_test.go @@ -73,6 +73,19 @@ func TestSingleTableDeleteSQL(t *testing.T) { } // Composite unique index +func TestCompositeUniqueIndexTableInsertSQL(t *testing.T) { + mock := NewMockOptimizer(true) + + sqls := []string{ + "insert into dept values(), (), ()", + "insert into dept select * from dept where deptno = 10", + "insert into dept select * from dept where dname = 'RESEARCH'", + "insert into dept select * from dept where deptno = 10 order by dname limit 1", + } + + runTestShouldPass(mock, t, sqls, false, false) +} + func TestCompositeUniqueIndexTableDeleteSQL(t *testing.T) { mock := NewMockOptimizer(true) diff --git a/pkg/sql/plan/join_order.go b/pkg/sql/plan/join_order.go index 0b24dfb2d719..8cd3b62565ee 100644 --- a/pkg/sql/plan/join_order.go +++ b/pkg/sql/plan/join_order.go @@ -557,9 +557,9 @@ func (builder *QueryBuilder) enumerateTags(nodeID int32) []int32 { node := builder.qry.Nodes[nodeID] if len(node.BindingTags) > 0 { tags = append(tags, node.BindingTags...) - if node.NodeType != plan.Node_JOIN { - return tags - } + //if node.NodeType != plan.Node_JOIN { + // return tags + //} } for _, childID := range builder.qry.Nodes[nodeID].Children { diff --git a/pkg/sql/plan/mock.go b/pkg/sql/plan/mock.go index 7f5a4056b466..95a7c4ff1314 100644 --- a/pkg/sql/plan/mock.go +++ b/pkg/sql/plan/mock.go @@ -601,6 +601,16 @@ func NewMockCompilerContext(isDml bool) *MockCompilerContext { tableExist: true, unique: true, }, + { + indexName: "", + tableName: catalog.UniqueIndexTableNamePrefix + "35fd5c5f-ab54-4873-85e4-3d5ab0ae20a2", + parts: []string{"loc", "dname"}, + cols: []col{ + {catalog.IndexTableIndexColName, types.T_varchar, true, 255, 0}, + }, + tableExist: true, + unique: false, + }, }, outcnt: 4, } @@ -615,6 +625,15 @@ func NewMockCompilerContext(isDml bool) *MockCompilerContext { pks: []int{0}, outcnt: 4, } + constraintTestSchema[catalog.UniqueIndexTableNamePrefix+"35fd5c5f-ab54-4873-85e4-3d5ab0ae20a2"] = &Schema{ + cols: []col{ + {catalog.IndexTableIndexColName, types.T_varchar, false, 255, 0}, + {catalog.IndexTablePrimaryColName, types.T_uint32, false, 32, 0}, + {catalog.Row_ID, types.T_Rowid, true, 0, 0}, + }, + pks: []int{0}, + outcnt: 4, + } /* create table products ( pid int not null, diff --git a/pkg/sql/plan/opt_misc.go b/pkg/sql/plan/opt_misc.go index a18e5adbb1c1..dab411e30649 100644 --- a/pkg/sql/plan/opt_misc.go +++ b/pkg/sql/plan/opt_misc.go @@ -33,9 +33,47 @@ func (builder *QueryBuilder) countColRefs(nodeID int32, colRefCnt map[[2]int32]i increaseRefCntForExprList(node.GroupBy, 1, colRefCnt) increaseRefCntForExprList(node.AggList, 1, colRefCnt) increaseRefCntForExprList(node.WinSpecList, 1, colRefCnt) + increaseRefCntForExprList(node.InsertDeleteCols, 2, colRefCnt) for i := range node.OrderBy { increaseRefCnt(node.OrderBy[i].Expr, 1, colRefCnt) } + for _, updateCtx := range node.UpdateCtxList { + increaseRefCntForColRefList(updateCtx.InsertCols, 2, colRefCnt) + increaseRefCntForColRefList(updateCtx.DeleteCols, 2, colRefCnt) + if len(updateCtx.PartitionTableIds) > 0 { + colRefs := make([]ColRef, 0, 2) + if updateCtx.OldPartitionIdx > -1 { + colRefs = append(colRefs, ColRef{ + RelPos: node.BindingTags[1], + ColPos: updateCtx.OldPartitionIdx, + }) + } + if updateCtx.NewPartitionIdx > -1 { + colRefs = append(colRefs, ColRef{ + RelPos: node.BindingTags[1], + ColPos: updateCtx.NewPartitionIdx, + }) + } + increaseRefCntForColRefList(colRefs, 1, colRefCnt) + } + } + + if node.NodeType == plan.Node_LOCK_OP { + var colRefs []ColRef + for _, lockTarget := range node.LockTargets { + colRefs = append(colRefs, ColRef{ + RelPos: lockTarget.PrimaryColRelPos, + ColPos: lockTarget.PrimaryColIdxInBat, + }) + if lockTarget.IsPartitionTable { + colRefs = append(colRefs, ColRef{ + RelPos: lockTarget.FilterColRelPos, + ColPos: lockTarget.FilterColIdxInBat, + }) + } + } + increaseRefCntForColRefList(colRefs, 1, colRefCnt) + } for _, childID := range node.Children { builder.countColRefs(childID, colRefCnt) @@ -76,6 +114,15 @@ func (builder *QueryBuilder) removeSimpleProjections(nodeID int32, parentType pl } } + case plan.Node_MULTI_UPDATE: + for i, childID := range node.Children { + newChildID, childProjMap := builder.removeSimpleProjections(childID, node.NodeType, true, colRefCnt) + node.Children[i] = newChildID + for ref, expr := range childProjMap { + projMap[ref] = expr + } + } + default: for i, childID := range node.Children { newChildID, childProjMap := builder.removeSimpleProjections(childID, node.NodeType, flag, colRefCnt) @@ -122,6 +169,12 @@ func increaseRefCntForExprList(exprs []*plan.Expr, inc int, colRefCnt map[[2]int } } +func increaseRefCntForColRefList(cols []plan.ColRef, inc int, colRefCnt map[[2]int32]int) { + for _, col := range cols { + colRefCnt[[2]int32{col.RelPos, col.ColPos}] += inc + } +} + // FIXME: We should remove PROJECT node for more cases, but keep them now to avoid intricate issues. func (builder *QueryBuilder) canRemoveProject(parentType plan.Node_NodeType, node *plan.Node) bool { if node.NodeType != plan.Node_PROJECT || node.Limit != nil || node.Offset != nil { @@ -149,18 +202,43 @@ func (builder *QueryBuilder) canRemoveProject(parentType plan.Node_NodeType, nod if parentType == plan.Node_INSERT || parentType == plan.Node_PRE_INSERT || parentType == plan.Node_PRE_INSERT_UK || parentType == plan.Node_PRE_INSERT_SK { return false } + if parentType == plan.Node_PRE_INSERT || parentType == plan.Node_MULTI_UPDATE || parentType == plan.Node_LOCK_OP { + return false + } + + for _, e := range node.ProjectList { + if !exprCanRemoveProject(e) { + return false + } + } childType := builder.qry.Nodes[node.Children[0]].NodeType if childType == plan.Node_VALUE_SCAN || childType == plan.Node_EXTERNAL_SCAN { - return false + return parentType == plan.Node_PROJECT } if childType == plan.Node_FUNCTION_SCAN || childType == plan.Node_EXTERNAL_FUNCTION { - return false + return parentType == plan.Node_PROJECT } return true } +func exprCanRemoveProject(expr *Expr) bool { + switch ne := expr.Expr.(type) { + case *plan.Expr_F: + if ne.F.Func.ObjName == "sleep" { + return false + } + for _, arg := range ne.F.GetArgs() { + canRemove := exprCanRemoveProject(arg) + if !canRemove { + return canRemove + } + } + } + return true +} + func replaceColumnsForNode(node *plan.Node, projMap map[[2]int32]*plan.Expr) { replaceColumnsForExprList(node.ProjectList, projMap) replaceColumnsForExprList(node.OnList, projMap) @@ -168,9 +246,57 @@ func replaceColumnsForNode(node *plan.Node, projMap map[[2]int32]*plan.Expr) { replaceColumnsForExprList(node.GroupBy, projMap) replaceColumnsForExprList(node.AggList, projMap) replaceColumnsForExprList(node.WinSpecList, projMap) + replaceColumnsForExprList(node.InsertDeleteCols, projMap) for i := range node.OrderBy { node.OrderBy[i].Expr = replaceColumnsForExpr(node.OrderBy[i].Expr, projMap) } + for _, updateCtx := range node.UpdateCtxList { + replaceColumnsForColRefList(updateCtx.InsertCols, projMap) + replaceColumnsForColRefList(updateCtx.DeleteCols, projMap) + if len(updateCtx.PartitionTableIds) > 0 { + oldPartRef := [2]int32{node.BindingTags[1], updateCtx.OldPartitionIdx} + newPartRef := [2]int32{node.BindingTags[1], updateCtx.NewPartitionIdx} + if updateCtx.OldPartitionIdx > -1 { + if expr, ok := projMap[oldPartRef]; ok { + if e, ok := expr.Expr.(*plan.Expr_Col); ok { + node.BindingTags[1] = e.Col.RelPos + updateCtx.OldPartitionIdx = e.Col.ColPos + } + } + } + if updateCtx.NewPartitionIdx > -1 { + if expr, ok := projMap[newPartRef]; ok { + if e, ok := expr.Expr.(*plan.Expr_Col); ok { + node.BindingTags[1] = e.Col.RelPos + updateCtx.NewPartitionIdx = e.Col.ColPos + } + } + } + } + + } + + if node.NodeType == plan.Node_LOCK_OP { + for _, lockTarget := range node.LockTargets { + colRef := [2]int32{lockTarget.PrimaryColRelPos, lockTarget.PrimaryColIdxInBat} + if expr, ok := projMap[colRef]; ok { + if e, ok := expr.Expr.(*plan.Expr_Col); ok { + lockTarget.PrimaryColRelPos = e.Col.RelPos + lockTarget.PrimaryColIdxInBat = e.Col.ColPos + } + } + + if lockTarget.IsPartitionTable { + colRef = [2]int32{lockTarget.FilterColRelPos, lockTarget.FilterColIdxInBat} + if expr, ok := projMap[colRef]; ok { + if e, ok := expr.Expr.(*plan.Expr_Col); ok { + lockTarget.FilterColRelPos = e.Col.RelPos + lockTarget.FilterColIdxInBat = e.Col.ColPos + } + } + } + } + } } func replaceColumnsForExprList(exprList []*plan.Expr, projMap map[[2]int32]*plan.Expr) { @@ -182,6 +308,17 @@ func replaceColumnsForExprList(exprList []*plan.Expr, projMap map[[2]int32]*plan } } +func replaceColumnsForColRefList(cols []plan.ColRef, projMap map[[2]int32]*plan.Expr) { + for i := range cols { + mapID := [2]int32{cols[i].RelPos, cols[i].ColPos} + if projExpr, ok := projMap[mapID]; ok { + newCol := projExpr.Expr.(*plan.Expr_Col).Col + cols[i].RelPos = newCol.RelPos + cols[i].ColPos = newCol.ColPos + } + } +} + func replaceColumnsForExpr(expr *plan.Expr, projMap map[[2]int32]*plan.Expr) *plan.Expr { if expr == nil { return nil @@ -334,9 +471,15 @@ func (builder *QueryBuilder) removeEffectlessLeftJoins(nodeID int32, tagCnt map[ increaseTagCntForExprList(node.GroupBy, 1, tagCnt) increaseTagCntForExprList(node.AggList, 1, tagCnt) increaseTagCntForExprList(node.WinSpecList, 1, tagCnt) + increaseTagCntForExprList(node.InsertDeleteCols, 2, tagCnt) for i := range node.OrderBy { increaseTagCnt(node.OrderBy[i].Expr, 1, tagCnt) } + for _, updateCtx := range node.UpdateCtxList { + increaseTagCntForColRefList(updateCtx.InsertCols, 2, tagCnt) + increaseTagCntForColRefList(updateCtx.DeleteCols, 2, tagCnt) + } + for i, childID := range node.Children { node.Children[i] = builder.removeEffectlessLeftJoins(childID, tagCnt) } @@ -366,9 +509,14 @@ END: increaseTagCntForExprList(node.GroupBy, -1, tagCnt) increaseTagCntForExprList(node.AggList, -1, tagCnt) increaseTagCntForExprList(node.WinSpecList, -1, tagCnt) + increaseTagCntForExprList(node.InsertDeleteCols, -2, tagCnt) for i := range node.OrderBy { increaseTagCnt(node.OrderBy[i].Expr, -1, tagCnt) } + for _, updateCtx := range node.UpdateCtxList { + increaseTagCntForColRefList(updateCtx.InsertCols, -2, tagCnt) + increaseTagCntForColRefList(updateCtx.DeleteCols, -2, tagCnt) + } return nodeID } @@ -382,6 +530,12 @@ func increaseTagCntForExprList(exprs []*plan.Expr, inc int, tagCnt map[int32]int } } +func increaseTagCntForColRefList(cols []plan.ColRef, inc int, tagCnt map[int32]int) { + for _, col := range cols { + tagCnt[col.RelPos] += inc + } +} + func increaseTagCnt(expr *plan.Expr, inc int, tagCnt map[int32]int) { switch exprImpl := expr.Expr.(type) { case *plan.Expr_Col: @@ -846,6 +1000,10 @@ func (builder *QueryBuilder) forceJoinOnOneCN(nodeID int32, force bool) { if node.NodeType == plan.Node_TABLE_SCAN { node.Stats.ForceOneCN = force } else if node.NodeType == plan.Node_JOIN { + if node.JoinType == plan.Node_DEDUP && !node.Stats.HashmapStats.Shuffle { + force = true + } + if len(node.RuntimeFilterBuildList) > 0 { switch node.JoinType { case plan.Node_RIGHT: diff --git a/pkg/sql/plan/pushdown.go b/pkg/sql/plan/pushdown.go index 5ff6a05f4580..4632a210fe19 100644 --- a/pkg/sql/plan/pushdown.go +++ b/pkg/sql/plan/pushdown.go @@ -269,7 +269,7 @@ func (builder *QueryBuilder) pushdownFilters(nodeID int32, filters []*plan.Expr, } case JoinSideRight: - if node.JoinType == plan.Node_INNER { + if node.JoinType == plan.Node_INNER || node.JoinType == plan.Node_DEDUP { rightPushdown = append(rightPushdown, filter) } else { cantPushdown = append(cantPushdown, filter) diff --git a/pkg/sql/plan/query_builder.go b/pkg/sql/plan/query_builder.go index 64b7519d9247..d33aa3472b86 100644 --- a/pkg/sql/plan/query_builder.go +++ b/pkg/sql/plan/query_builder.go @@ -62,28 +62,34 @@ func NewQueryBuilder(queryType plan.Query_StatementType, ctx CompilerContext, is nextTag: 0, mysqlCompatible: mysqlCompatible, tag2Table: make(map[int32]*TableDef), + tag2NodeID: make(map[int32]int32), isPrepareStatement: isPrepareStatement, deleteNode: make(map[uint64]int32), skipStats: skipStats, } } +func (builder *QueryBuilder) remapSingleColRef(col *plan.ColRef, colMap map[[2]int32][2]int32, remapInfo *RemapInfo) error { + mapID := [2]int32{col.RelPos, col.ColPos} + if ids, ok := colMap[mapID]; ok { + col.RelPos = ids[0] + col.ColPos = ids[1] + col.Name = builder.nameByColRef[mapID] + } else { + var keys []string + for k := range colMap { + keys = append(keys, fmt.Sprintf("%v", k)) + } + mapKeys := fmt.Sprintf("{ %s }", strings.Join(keys, ", ")) + return moerr.NewParseErrorf(builder.GetContext(), "remapInfo %s ; can't find column %v in context's map %s", remapInfo.String(), mapID, mapKeys) + } + return nil +} + func (builder *QueryBuilder) remapColRefForExpr(expr *Expr, colMap map[[2]int32][2]int32, remapInfo *RemapInfo) error { switch ne := expr.Expr.(type) { case *plan.Expr_Col: - mapID := [2]int32{ne.Col.RelPos, ne.Col.ColPos} - if ids, ok := colMap[mapID]; ok { - ne.Col.RelPos = ids[0] - ne.Col.ColPos = ids[1] - ne.Col.Name = builder.nameByColRef[mapID] - } else { - var keys []string - for k := range colMap { - keys = append(keys, fmt.Sprintf("%v", k)) - } - mapKeys := fmt.Sprintf("{ %s }", strings.Join(keys, ", ")) - return moerr.NewParseErrorf(builder.GetContext(), "remapInfo %s ; can't find column %v in context's map %s", remapInfo.String(), mapID, mapKeys) - } + return builder.remapSingleColRef(ne.Col, colMap, remapInfo) case *plan.Expr_F: for _, arg := range ne.F.GetArgs() { @@ -494,6 +500,10 @@ func (builder *QueryBuilder) remapAllColRefs(nodeID int32, step int32, colRefCnt } } + if node.JoinType == plan.Node_DEDUP && len(node.DedupColTypes) == 0 { + node.DedupColTypes = []plan.Type{node.OnList[0].GetF().Args[0].Typ} + } + childProjList := builder.qry.Nodes[leftID].ProjectList for i, globalRef := range leftRemapping.localToGlobal { if colRefCnt[globalRef] == 0 { @@ -1343,38 +1353,62 @@ func (builder *QueryBuilder) remapAllColRefs(nodeID int32, step int32, colRefCnt case plan.Node_LOCK_OP: preNode := builder.qry.Nodes[node.Children[0]] - pkexpr := &plan.Expr{ - Typ: node.LockTargets[0].GetPrimaryColTyp(), - Expr: &plan.Expr_Col{ - Col: &plan.ColRef{ - RelPos: node.BindingTags[1], - ColPos: node.LockTargets[0].PrimaryColIdxInBat, + + var pkExprs, partExprs []*plan.Expr + var oldPkPos, oldPartPos [][2]int32 + for _, lockTarget := range node.LockTargets { + pkExpr := &plan.Expr{ + // Typ: node.LockTargets[0].GetPrimaryColTyp(), + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + RelPos: lockTarget.PrimaryColRelPos, + ColPos: lockTarget.PrimaryColIdxInBat, + }, }, - }, + } + increaseRefCnt(pkExpr, 1, colRefCnt) + pkExprs = append(pkExprs, pkExpr) + oldPkPos = append(oldPkPos, [2]int32{lockTarget.PrimaryColRelPos, lockTarget.PrimaryColIdxInBat}) + + if lockTarget.IsPartitionTable { + partExpr := &Expr{ + // Typ: node.ProjectList[len(node.ProjectList)-1].Typ, + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + RelPos: lockTarget.FilterColRelPos, + ColPos: lockTarget.FilterColIdxInBat, + }, + }, + } + increaseRefCnt(partExpr, 1, colRefCnt) + partExprs = append(partExprs, partExpr) + oldPartPos = append(oldPartPos, [2]int32{lockTarget.FilterColRelPos, lockTarget.FilterColIdxInBat}) + } else { + partExprs = append(partExprs, nil) + oldPartPos = append(oldPartPos, [2]int32{-1, -1}) + } } - oldPos := [2]int32{node.BindingTags[1], node.LockTargets[0].PrimaryColIdxInBat} - increaseRefCnt(pkexpr, 1, colRefCnt) + childRemapping, err := builder.remapAllColRefs(node.Children[0], step, colRefCnt, colRefBool, sinkColRef) if err != nil { return nil, err } - tableDef := node.GetTableDef() - if tableDef.Partition != nil { - partitionIdx := len(preNode.ProjectList) - partitionExpr := DeepCopyExpr(tableDef.Partition.PartitionExpression) - preNode.ProjectList = append(preNode.ProjectList, partitionExpr) - - partTableIDs, _ := getPartTableIdsAndNames(builder.compCtx, preNode.GetObjRef(), tableDef) - node.LockTargets[0].IsPartitionTable = true - node.LockTargets[0].PartitionTableIds = partTableIDs - node.LockTargets[0].FilterColIdxInBat = int32(partitionIdx) - } + for pkIdx, pkExpr := range pkExprs { + if newPos, ok := childRemapping.globalToLocal[oldPkPos[pkIdx]]; ok { + node.LockTargets[pkIdx].PrimaryColRelPos = newPos[0] + node.LockTargets[pkIdx].PrimaryColIdxInBat = newPos[1] + } + increaseRefCnt(pkExpr, -1, colRefCnt) - if newPos, ok := childRemapping.globalToLocal[oldPos]; ok { - node.LockTargets[0].PrimaryColIdxInBat = newPos[1] + if partExprs[pkIdx] != nil { + if newPos, ok := childRemapping.globalToLocal[oldPartPos[pkIdx]]; ok { + node.LockTargets[pkIdx].FilterColRelPos = newPos[0] + node.LockTargets[pkIdx].FilterColIdxInBat = newPos[1] + } + increaseRefCnt(partExprs[pkIdx], -1, colRefCnt) + } } - increaseRefCnt(pkexpr, -1, colRefCnt) for i, globalRef := range childRemapping.localToGlobal { if colRefCnt[globalRef] == 0 { @@ -1394,22 +1428,6 @@ func (builder *QueryBuilder) remapAllColRefs(nodeID int32, step int32, colRefCnt }) } - if len(node.ProjectList) == 0 { - if len(childRemapping.localToGlobal) > 0 { - remapping.addColRef(childRemapping.localToGlobal[0]) - } - - node.ProjectList = append(node.ProjectList, &plan.Expr{ - Typ: preNode.ProjectList[0].Typ, - Expr: &plan.Expr_Col{ - Col: &plan.ColRef{ - RelPos: 0, - ColPos: 0, - }, - }, - }) - } - case plan.Node_APPLY: internalMap := make(map[[2]int32][2]int32) @@ -1477,6 +1495,234 @@ func (builder *QueryBuilder) remapAllColRefs(nodeID int32, step int32, colRefCnt } } + case plan.Node_INSERT, plan.Node_DELETE: + for _, expr := range node.InsertDeleteCols { + increaseRefCnt(expr, 1, colRefCnt) + } + + childRemapping, err := builder.remapAllColRefs(node.Children[0], step, colRefCnt, colRefBool, sinkColRef) + if err != nil { + return nil, err + } + + remapInfo.tip = "InsertDeleteCols" + for idx, expr := range node.InsertDeleteCols { + increaseRefCnt(expr, -1, colRefCnt) + remapInfo.srcExprIdx = idx + err := builder.remapColRefForExpr(expr, childRemapping.globalToLocal, &remapInfo) + if err != nil { + return nil, err + } + } + + childProjList := builder.qry.Nodes[node.Children[0]].ProjectList + for i, globalRef := range childRemapping.localToGlobal { + if colRefCnt[globalRef] == 0 { + continue + } + + remapping.addColRef(globalRef) + + node.ProjectList = append(node.ProjectList, &plan.Expr{ + Typ: childProjList[i].Typ, + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + RelPos: 0, + ColPos: int32(i), + Name: builder.nameByColRef[globalRef], + }, + }, + }) + } + + if len(node.ProjectList) == 0 { + if len(childRemapping.localToGlobal) > 0 { + remapping.addColRef(childRemapping.localToGlobal[0]) + } + + node.ProjectList = append(node.ProjectList, &plan.Expr{ + Typ: childProjList[0].Typ, + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + RelPos: 0, + ColPos: 0, + }, + }, + }) + } + + case plan.Node_MULTI_UPDATE: + for _, updateCtx := range node.UpdateCtxList { + for _, col := range updateCtx.InsertCols { + colRefCnt[[2]int32{col.RelPos, col.ColPos}]++ + } + + for _, col := range updateCtx.DeleteCols { + colRefCnt[[2]int32{col.RelPos, col.ColPos}]++ + } + } + + var oldPartRel, newPartRel [2]int32 + var oldPartExpr, newPartExpr *Expr + mainTableCtx := node.UpdateCtxList[0] + if mainTableCtx.TableDef.Partition != nil { + if mainTableCtx.OldPartitionIdx > -1 { + oldPartExpr = &Expr{ + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + RelPos: node.BindingTags[1], + ColPos: mainTableCtx.OldPartitionIdx, + }, + }, + } + increaseRefCnt(oldPartExpr, 1, colRefCnt) + oldPartRel = [2]int32{node.BindingTags[1], mainTableCtx.OldPartitionIdx} + } + + if mainTableCtx.NewPartitionIdx > -1 { + newPartExpr = &Expr{ + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + RelPos: node.BindingTags[1], + ColPos: mainTableCtx.NewPartitionIdx, + }, + }, + } + increaseRefCnt(newPartExpr, 1, colRefCnt) + newPartRel = [2]int32{node.BindingTags[1], mainTableCtx.NewPartitionIdx} + } + } + + childRemapping, err := builder.remapAllColRefs(node.Children[0], step, colRefCnt, colRefBool, sinkColRef) + if err != nil { + return nil, err + } + + if mainTableCtx.TableDef.Partition != nil { + if mainTableCtx.OldPartitionIdx > -1 { + if newPos, ok := childRemapping.globalToLocal[oldPartRel]; ok { + mainTableCtx.OldPartitionIdx = newPos[1] + } + increaseRefCnt(oldPartExpr, -1, colRefCnt) + } + if mainTableCtx.NewPartitionIdx > -1 { + if newPos, ok := childRemapping.globalToLocal[newPartRel]; ok { + mainTableCtx.NewPartitionIdx = newPos[1] + } + increaseRefCnt(newPartExpr, -1, colRefCnt) + } + } + + remapInfo.tip = "UpdateCtxList" + for idx, updateCtx := range node.UpdateCtxList { + remapInfo.srcExprIdx = idx + for i, col := range updateCtx.InsertCols { + colRefCnt[[2]int32{col.RelPos, col.ColPos}]-- + err := builder.remapSingleColRef(&updateCtx.InsertCols[i], childRemapping.globalToLocal, &remapInfo) + if err != nil { + return nil, err + } + } + + for i, col := range updateCtx.DeleteCols { + colRefCnt[[2]int32{col.RelPos, col.ColPos}]-- + err := builder.remapSingleColRef(&updateCtx.DeleteCols[i], childRemapping.globalToLocal, &remapInfo) + if err != nil { + return nil, err + } + } + } + + childProjList := builder.qry.Nodes[node.Children[0]].ProjectList + for i, globalRef := range childRemapping.localToGlobal { + if colRefCnt[globalRef] == 0 { + continue + } + + remapping.addColRef(globalRef) + + node.ProjectList = append(node.ProjectList, &plan.Expr{ + Typ: childProjList[i].Typ, + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + RelPos: 0, + ColPos: int32(i), + Name: builder.nameByColRef[globalRef], + }, + }, + }) + } + + if len(node.ProjectList) == 0 { + if len(childRemapping.localToGlobal) > 0 { + remapping.addColRef(childRemapping.localToGlobal[0]) + } + + node.ProjectList = append(node.ProjectList, &plan.Expr{ + Typ: childProjList[0].Typ, + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + RelPos: 0, + ColPos: 0, + }, + }, + }) + } + + case plan.Node_PRE_INSERT: + childRemapping, err := builder.remapAllColRefs(node.Children[0], step, colRefCnt, colRefBool, sinkColRef) + if err != nil { + return nil, err + } + + childProjList := builder.qry.Nodes[node.Children[0]].ProjectList + for i, globalRef := range childRemapping.localToGlobal { + if colRefCnt[globalRef] == 0 { + continue + } + + remapping.addColRef(globalRef) + + node.ProjectList = append(node.ProjectList, &plan.Expr{ + Typ: childProjList[i].Typ, + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + RelPos: 0, + ColPos: int32(i), + Name: builder.nameByColRef[globalRef], + }, + }, + }) + } + + if node.PreInsertCtx.CompPkeyExpr != nil { + globalRef := [2]int32{node.BindingTags[0], 0} + remapping.addColRef(globalRef) + + node.ProjectList = append(node.ProjectList, &plan.Expr{ + Typ: node.PreInsertCtx.CompPkeyExpr.Typ, + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + RelPos: -1, + ColPos: 0, + }, + }, + }) + } else if node.PreInsertCtx.ClusterByExpr != nil { + globalRef := [2]int32{node.BindingTags[0], 0} + remapping.addColRef(globalRef) + + node.ProjectList = append(node.ProjectList, &plan.Expr{ + Typ: node.PreInsertCtx.ClusterByExpr.Typ, + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + RelPos: -1, + ColPos: 0, + }, + }, + }) + } + default: return nil, moerr.NewInternalError(builder.GetContext(), "unsupport node type") } @@ -1651,6 +1897,9 @@ func (builder *QueryBuilder) createQuery() (*Query, error) { builder.rewriteStarApproxCount(rootID) + if builder.qry.StmtType != plan.Query_SELECT { + builder.updateLocksOnDemand(rootID) + } rootNode := builder.qry.Nodes[rootID] for j := range rootNode.ProjectList { @@ -1669,10 +1918,12 @@ func (builder *QueryBuilder) createQuery() (*Query, error) { for i := len(builder.qry.Steps) - 1; i >= 0; i-- { rootID := builder.qry.Steps[i] rootNode := builder.qry.Nodes[rootID] - resultTag := rootNode.BindingTags[0] colRefCnt := make(map[[2]int32]int) - for j := range rootNode.ProjectList { - colRefCnt[[2]int32{resultTag, int32(j)}] = 1 + if len(rootNode.BindingTags) > 0 { + resultTag := rootNode.BindingTags[0] + for j := range rootNode.ProjectList { + colRefCnt[[2]int32{resultTag, int32(j)}] = 1 + } } _, err := builder.remapAllColRefs(rootID, int32(i), colRefCnt, colRefBool, sinkColRef) if err != nil { @@ -1703,7 +1954,7 @@ func (builder *QueryBuilder) buildUnion(stmt *tree.UnionClause, astOrderBy tree. case *tree.Select: if sltClause, ok := sltStmt.Select.(*tree.SelectClause); ok { sltClause.Distinct = true - return builder.buildSelect(sltStmt, ctx, isRoot) + return builder.bindSelect(sltStmt, ctx, isRoot) } else { // rewrite sltStmt to select distinct * from (sltStmt) a tmpSltStmt := &tree.Select{ @@ -1729,14 +1980,14 @@ func (builder *QueryBuilder) buildUnion(stmt *tree.UnionClause, astOrderBy tree. Limit: astLimit, OrderBy: astOrderBy, } - return builder.buildSelect(tmpSltStmt, ctx, isRoot) + return builder.bindSelect(tmpSltStmt, ctx, isRoot) } case *tree.SelectClause: if !sltStmt.Distinct { sltStmt.Distinct = true } - return builder.buildSelect(&tree.Select{Select: sltStmt, Limit: astLimit, OrderBy: astOrderBy}, ctx, isRoot) + return builder.bindSelect(&tree.Select{Select: sltStmt, Limit: astLimit, OrderBy: astOrderBy}, ctx, isRoot) } } @@ -1751,9 +2002,9 @@ func (builder *QueryBuilder) buildUnion(stmt *tree.UnionClause, astOrderBy tree. subCtx := NewBindContext(builder, ctx) subCtx.unionSelect = subCtx.initSelect if slt, ok := sltStmt.(*tree.Select); ok { - nodeID, err = builder.buildSelect(slt, subCtx, isRoot) + nodeID, err = builder.bindSelect(slt, subCtx, isRoot) } else { - nodeID, err = builder.buildSelect(&tree.Select{Select: sltStmt}, subCtx, isRoot) + nodeID, err = builder.bindSelect(&tree.Select{Select: sltStmt}, subCtx, isRoot) } if err != nil { return 0, err @@ -2069,7 +2320,7 @@ func (bc *BindContext) generateForceWinSpecList() ([]*plan.Expr, error) { return windowsSpecList, nil } -func (builder *QueryBuilder) buildSelect(stmt *tree.Select, ctx *BindContext, isRoot bool) (int32, error) { +func (builder *QueryBuilder) bindSelect(stmt *tree.Select, ctx *BindContext, isRoot bool) (int32, error) { // preprocess CTEs if stmt.With != nil { ctx.cteByName = make(map[string]*CTERef) @@ -2139,7 +2390,7 @@ func (builder *QueryBuilder) buildSelect(stmt *tree.Select, ctx *BindContext, is oldSnapshot := builder.compCtx.GetSnapshot() builder.compCtx.SetSnapshot(subCtx.snapshot) - nodeID, err := builder.buildSelect(s, subCtx, false) + nodeID, err := builder.bindSelect(s, subCtx, false) builder.compCtx.SetSnapshot(oldSnapshot) if err != nil { return 0, err @@ -2153,7 +2404,7 @@ func (builder *QueryBuilder) buildSelect(stmt *tree.Select, ctx *BindContext, is initCtx.initSelect = true initCtx.sinkTag = builder.genNewTag() initCtx.isTryBindingCTE = true - initLastNodeID, err := builder.buildSelect(&tree.Select{Select: *left}, initCtx, false) + initLastNodeID, err := builder.bindSelect(&tree.Select{Select: *left}, initCtx, false) if err != nil { return 0, err } @@ -2178,7 +2429,7 @@ func (builder *QueryBuilder) buildSelect(stmt *tree.Select, ctx *BindContext, is sourceStep := builder.appendStep(recursiveNodeId) nodeID := appendRecursiveScanNode(builder, subCtx, sourceStep, subCtx.sinkTag) subCtx.recRecursiveScanNodeId = nodeID - recursiveNodeId, err = builder.buildSelect(&tree.Select{Select: r}, subCtx, false) + recursiveNodeId, err = builder.bindSelect(&tree.Select{Select: r}, subCtx, false) if err != nil { return 0, err } @@ -2425,7 +2676,7 @@ func (builder *QueryBuilder) buildSelect(stmt *tree.Select, ctx *BindContext, is } // build FROM clause - nodeID, err = builder.buildFrom(clause.From.Tables, ctx) + nodeID, err = builder.buildFrom(clause.From.Tables, ctx, isRoot) if err != nil { return 0, err } @@ -2450,21 +2701,69 @@ func (builder *QueryBuilder) buildSelect(stmt *tree.Select, ctx *BindContext, is if builder.isForUpdate { tableDef := builder.qry.Nodes[nodeID].GetTableDef() + objRef := builder.qry.Nodes[nodeID].GetObjRef() pkPos, pkTyp := getPkPos(tableDef, false) + lastTag := builder.qry.Nodes[nodeID].BindingTags[0] lockTarget := &plan.LockTarget{ TableId: tableDef.TblId, PrimaryColIdxInBat: int32(pkPos), + PrimaryColRelPos: lastTag, PrimaryColTyp: pkTyp, Block: true, RefreshTsIdxInBat: -1, //unsupport now - FilterColIdxInBat: -1, //unsupport now } + if tableDef.Partition != nil { + partTableIDs, _ := getPartTableIdsAndNames(builder.compCtx, objRef, tableDef) + lockTarget.IsPartitionTable = true + lockTarget.PartitionTableIds = partTableIDs + + colPosMap := make(map[string]int32) + for idx, col := range tableDef.Cols { + colPosMap[col.Name] = int32(idx) + } + partitionExpr, err := getRemapParitionExpr(tableDef, lastTag, colPosMap, false) + if err != nil { + return -1, err + } + projectList := make([]*Expr, 0, len(tableDef.Cols)+1) + for i, col := range tableDef.Cols { + if !col.Hidden { + projectList = append(projectList, &plan.Expr{ + Typ: col.Typ, + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + TblName: tableDef.Name, + Name: col.Name, + RelPos: lastTag, + ColPos: int32(i), + }, + }, + }) + } + } + lockTarget.FilterColIdxInBat = int32(len(projectList)) + projectList = append(projectList, partitionExpr) + newBindingTag := builder.genNewTag() + lockTarget.FilterColRelPos = newBindingTag + if binding, ok := ctx.bindingByTable[tableDef.Name]; ok { + //@xxx not a good choice + binding.tag = newBindingTag + } + nodeID = builder.appendNode(&plan.Node{ + NodeType: plan.Node_PROJECT, + Children: []int32{nodeID}, + BindingTags: []int32{newBindingTag}, + ProjectList: projectList, + }, ctx) + + } + lockNode = &Node{ NodeType: plan.Node_LOCK_OP, Children: []int32{nodeID}, TableDef: tableDef, LockTargets: []*plan.LockTarget{lockTarget}, - BindingTags: []int32{builder.genNewTag(), builder.qry.Nodes[nodeID].BindingTags[0]}, + BindingTags: []int32{builder.genNewTag()}, } if astLimit == nil { @@ -3408,6 +3707,9 @@ func (builder *QueryBuilder) appendNode(node *plan.Node, ctx *BindContext) int32 builder.qry.Nodes = append(builder.qry.Nodes, node) builder.ctxByNode = append(builder.ctxByNode, ctx) ReCalcNodeStats(nodeID, builder, false, true, true) + for _, tag := range node.BindingTags { + builder.tag2NodeID[tag] = nodeID + } return nodeID } @@ -3424,7 +3726,7 @@ func (builder *QueryBuilder) rewriteRightJoinToLeftJoin(nodeID int32) { } } -func (builder *QueryBuilder) buildFrom(stmt tree.TableExprs, ctx *BindContext) (int32, error) { +func (builder *QueryBuilder) buildFrom(stmt tree.TableExprs, ctx *BindContext, isRoot bool) (int32, error) { if len(stmt) == 1 { return builder.buildTable(stmt[0], ctx, -1, nil) } @@ -3563,7 +3865,7 @@ func (builder *QueryBuilder) buildTable(stmt tree.TableExpr, ctx *BindContext, p return 0, moerr.NewInternalError(builder.GetContext(), "not support select from derived table for update") } subCtx := NewBindContext(builder, ctx) - nodeID, err = builder.buildSelect(tbl, subCtx, false) + nodeID, err = builder.bindSelect(tbl, subCtx, false) if subCtx.isCorrelated { return 0, moerr.NewNYI(builder.GetContext(), "correlated subquery in FROM clause") } @@ -3652,7 +3954,7 @@ func (builder *QueryBuilder) buildTable(stmt tree.TableExpr, ctx *BindContext, p oldSnapshot := builder.compCtx.GetSnapshot() builder.compCtx.SetSnapshot(subCtx.snapshot) - nodeID, err = builder.buildSelect(s, subCtx, false) + nodeID, err = builder.bindSelect(s, subCtx, false) builder.compCtx.SetSnapshot(oldSnapshot) if err != nil { return @@ -3680,7 +3982,7 @@ func (builder *QueryBuilder) buildTable(stmt tree.TableExpr, ctx *BindContext, p initCtx := NewBindContext(builder, ctx) initCtx.initSelect = true initCtx.sinkTag = builder.genNewTag() - initLastNodeID, err1 := builder.buildSelect(&tree.Select{Select: *left}, initCtx, false) + initLastNodeID, err1 := builder.bindSelect(&tree.Select{Select: *left}, initCtx, false) if err1 != nil { err = err1 return @@ -3714,7 +4016,7 @@ func (builder *QueryBuilder) buildTable(stmt tree.TableExpr, ctx *BindContext, p subCtx.recRecursiveScanNodeId = appendRecursiveScanNode(builder, subCtx, initSourceStep, subCtx.sinkTag) recursiveNodeIDs[i] = subCtx.recRecursiveScanNodeId recursiveSteps[i] = int32(len(builder.qry.Steps)) - recursiveLastNodeID, err = builder.buildSelect(&tree.Select{Select: r}, subCtx, false) + recursiveLastNodeID, err = builder.bindSelect(&tree.Select{Select: r}, subCtx, false) if err != nil { return } @@ -4255,7 +4557,7 @@ func (builder *QueryBuilder) addBinding(nodeID int32, alias tree.AliasClause, ct } func (builder *QueryBuilder) buildJoinTable(tbl *tree.JoinTableExpr, ctx *BindContext) (int32, error) { - var joinType plan.Node_JoinType + joinType := plan.Node_INNER switch tbl.JoinType { case tree.JOIN_TYPE_CROSS, tree.JOIN_TYPE_INNER, tree.JOIN_TYPE_NATURAL: @@ -4268,6 +4570,8 @@ func (builder *QueryBuilder) buildJoinTable(tbl *tree.JoinTableExpr, ctx *BindCo joinType = plan.Node_RIGHT case tree.JOIN_TYPE_FULL: joinType = plan.Node_OUTER + case tree.JOIN_TYPE_DEDUP: + joinType = plan.Node_DEDUP } leftCtx := NewBindContext(builder, ctx) diff --git a/pkg/sql/plan/runtime_filter.go b/pkg/sql/plan/runtime_filter.go index f9186c38dc02..69747a6b9ece 100644 --- a/pkg/sql/plan/runtime_filter.go +++ b/pkg/sql/plan/runtime_filter.go @@ -52,6 +52,14 @@ func GetInFilterCardLimitOnPK( return int32(upper) } +func mustRuntimeFilter(n *plan.Node) bool { + switch n.JoinType { + case plan.Node_INDEX, plan.Node_DEDUP: + return true + } + return false +} + func (builder *QueryBuilder) generateRuntimeFilters(nodeID int32) { node := builder.qry.Nodes[nodeID] sid := builder.compCtx.GetProcess().GetService() @@ -97,7 +105,7 @@ func (builder *QueryBuilder) generateRuntimeFilters(nodeID int32) { } rightChild := builder.qry.Nodes[node.Children[1]] - if node.JoinType != plan.Node_INDEX && rightChild.Stats.Outcnt > 5000000 { + if !mustRuntimeFilter(node) && rightChild.Stats.Outcnt > 5000000 { return } if node.Stats.HashmapStats.HashOnPK && rightChild.Stats.Outcnt > 320000 { @@ -173,7 +181,7 @@ func (builder *QueryBuilder) generateRuntimeFilters(nodeID int32) { //} } - if builder.optimizerHints != nil && builder.optimizerHints.runtimeFilter != 0 && node.JoinType != plan.Node_INDEX { + if builder.optimizerHints != nil && builder.optimizerHints.runtimeFilter != 0 && !mustRuntimeFilter(node) { return } diff --git a/pkg/sql/plan/shuffle.go b/pkg/sql/plan/shuffle.go index 1e529157c2a5..c0e44b5a75aa 100644 --- a/pkg/sql/plan/shuffle.go +++ b/pkg/sql/plan/shuffle.go @@ -326,6 +326,16 @@ func determinShuffleForJoin(n *plan.Node, builder *QueryBuilder) { return } switch n.JoinType { + case plan.Node_DEDUP: + rightchild := builder.qry.Nodes[n.Children[1]] + if rightchild.Stats.Outcnt > 320000 { + //dedup join always go hash shuffle, optimize this in the future + n.Stats.HashmapStats.Shuffle = true + n.Stats.HashmapStats.ShuffleColIdx = 0 + n.Stats.HashmapStats.ShuffleType = plan.ShuffleType_Hash + } + return + case plan.Node_INNER, plan.Node_ANTI, plan.Node_SEMI, plan.Node_LEFT, plan.Node_RIGHT: default: return diff --git a/pkg/sql/plan/stats.go b/pkg/sql/plan/stats.go index d13b23b80d56..9906b2bd3b3c 100644 --- a/pkg/sql/plan/stats.go +++ b/pkg/sql/plan/stats.go @@ -884,7 +884,13 @@ func ReCalcNodeStats(nodeID int32, builder *QueryBuilder, recursive bool, leafNo node.Stats.Outcnt = rightStats.Outcnt node.Stats.Cost = leftStats.Cost + rightStats.Cost node.Stats.HashmapStats.HashmapSize = rightStats.Outcnt - node.Stats.Selectivity = selectivity_out + node.Stats.Selectivity = selectivity + + case plan.Node_DEDUP: + node.Stats.Outcnt = rightStats.Outcnt + node.Stats.Cost = leftStats.Cost + rightStats.Cost + node.Stats.HashmapStats.HashmapSize = rightStats.Outcnt + node.Stats.Selectivity = selectivity case plan.Node_OUTER: node.Stats.Outcnt = leftStats.Outcnt + rightStats.Outcnt @@ -1111,7 +1117,9 @@ func computeFunctionScan(name string, exprs []*Expr, nodeStat *Stats) bool { } var cost float64 var canGetCost bool - if len(exprs) == 2 { + if len(exprs) == 1 { + cost, canGetCost = getCost(nil, exprs[0], nil) + } else if len(exprs) == 2 { if exprs[0].Typ.Id != exprs[1].Typ.Id { return false } @@ -1154,9 +1162,13 @@ func getCost(start *Expr, end *Expr, step *Expr) (float64, bool) { return 0, false } - switch start.Typ.Id { + switch end.Typ.Id { case int32(types.T_int32): - startNum, flag1 = getInt32Val(start) + if start == nil { + startNum, flag1 = 0, true + } else { + startNum, flag1 = getInt32Val(start) + } endNum, flag2 = getInt32Val(end) flag3 = true if step != nil { @@ -1166,7 +1178,11 @@ func getCost(start *Expr, end *Expr, step *Expr) (float64, bool) { return 0, false } case int32(types.T_int64): - startNum, flag1 = getInt64Val(start) + if start == nil { + startNum, flag1 = 0, true + } else { + startNum, flag1 = getInt64Val(start) + } endNum, flag2 = getInt64Val(end) flag3 = true if step != nil { @@ -1213,7 +1229,7 @@ func recalcStatsByRuntimeFilter(scanNode *plan.Node, joinNode *plan.Node, builde return } - if joinNode.JoinType == plan.Node_INDEX || joinNode.NodeType == plan.Node_FUZZY_FILTER { + if joinNode.JoinType == plan.Node_INDEX || joinNode.JoinType == plan.Node_DEDUP || joinNode.NodeType == plan.Node_FUZZY_FILTER { scanNode.Stats.Outcnt = builder.qry.Nodes[joinNode.Children[1]].Stats.Outcnt if scanNode.Stats.Outcnt > scanNode.Stats.TableCnt { scanNode.Stats.Outcnt = scanNode.Stats.TableCnt diff --git a/pkg/sql/plan/types.go b/pkg/sql/plan/types.go index 65eb4fac48a4..1a637622df9b 100644 --- a/pkg/sql/plan/types.go +++ b/pkg/sql/plan/types.go @@ -169,7 +169,8 @@ type QueryBuilder struct { ctxByNode []*BindContext nameByColRef map[[2]int32]string - tag2Table map[int32]*TableDef + tag2Table map[int32]*TableDef + tag2NodeID map[int32]int32 nextTag int32 nextMsgTag int32 diff --git a/pkg/sql/plan/visit_plan_rule.go b/pkg/sql/plan/visit_plan_rule.go index 901c04ba39cd..b7f44145ca62 100644 --- a/pkg/sql/plan/visit_plan_rule.go +++ b/pkg/sql/plan/visit_plan_rule.go @@ -43,7 +43,7 @@ func NewGetParamRule() *GetParamRule { } func (rule *GetParamRule) MatchNode(node *Node) bool { - if node.NodeType == plan.Node_TABLE_SCAN || node.NodeType == plan.Node_INSERT || node.NodeType == plan.Node_MULTI_UPDATE { + if node.NodeType == plan.Node_TABLE_SCAN || node.NodeType == plan.Node_INSERT { rule.schemas = append(rule.schemas, &plan.ObjectRef{ Server: int64(node.TableDef.Version), //we use this unused field to store table's version Db: int64(node.TableDef.DbId), @@ -54,6 +54,19 @@ func (rule *GetParamRule) MatchNode(node *Node) bool { SchemaName: node.ObjRef.SchemaName, ObjName: node.ObjRef.ObjName, }) + } else if node.NodeType == plan.Node_MULTI_UPDATE { + for _, updateCtx := range node.UpdateCtxList { + rule.schemas = append(rule.schemas, &plan.ObjectRef{ + Server: int64(updateCtx.TableDef.Version), //we use this unused field to store table's version + Db: int64(updateCtx.TableDef.DbId), + Schema: int64(updateCtx.TableDef.DbId), + Obj: updateCtx.ObjRef.Obj, + ServerName: updateCtx.ObjRef.ServerName, + DbName: updateCtx.ObjRef.DbName, + SchemaName: updateCtx.ObjRef.SchemaName, + ObjName: updateCtx.ObjRef.ObjName, + }) + } } return false } diff --git a/pkg/txn/client/operator.go b/pkg/txn/client/operator.go index fe44fff9b99e..15bbc26451db 100644 --- a/pkg/txn/client/operator.go +++ b/pkg/txn/client/operator.go @@ -611,15 +611,15 @@ func (tc *txnOperator) Commit(ctx context.Context) (err error) { tc.reset.commitCounter.addEnter() defer tc.reset.commitCounter.addExit() - txn := tc.getTxnMeta(false) - util.LogTxnCommit(tc.logger, txn) + txnMeta := tc.getTxnMeta(false) + util.LogTxnCommit(tc.logger, txnMeta) readonly := tc.reset.workspace != nil && tc.reset.workspace.Readonly() if !readonly { tc.reset.commitSeq = tc.NextSequence() tc.reset.commitAt = time.Now() - tc.triggerEvent(newEvent(CommitEvent, txn, tc.reset.commitSeq, nil)) + tc.triggerEvent(newEvent(CommitEvent, txnMeta, tc.reset.commitSeq, nil)) defer func() { cost := time.Since(tc.reset.commitAt) v2.TxnCNCommitDurationHistogram.Observe(cost.Seconds()) @@ -630,6 +630,7 @@ func (tc *txnOperator) Commit(ctx context.Context) (err error) { if tc.opts.options.ReadOnly() { tc.mu.Lock() defer tc.mu.Unlock() + tc.mu.txn.Status = txn.TxnStatus_Committed tc.closeLocked() return } @@ -1257,6 +1258,9 @@ func (tc *txnOperator) needUnlockLocked() bool { func (tc *txnOperator) closeLocked() { if !tc.mu.closed { tc.mu.closed = true + if tc.reset.commitErr != nil { + tc.mu.txn.Status = txn.TxnStatus_Aborted + } tc.triggerEventLocked( TxnEvent{ Event: ClosedEvent, diff --git a/pkg/txn/client/operator_test.go b/pkg/txn/client/operator_test.go index 545989d597ff..0e2b2482c7fa 100644 --- a/pkg/txn/client/operator_test.go +++ b/pkg/txn/client/operator_test.go @@ -129,6 +129,7 @@ func TestCommitWithNoWrite(t *testing.T) { err := tc.Commit(ctx) assert.NoError(t, err) assert.Empty(t, ts.getLastRequests()) + assert.Equal(t, txn.TxnStatus_Committed, tc.mu.txn.Status) }) } @@ -137,6 +138,7 @@ func TestCommitReadOnly(t *testing.T) { err := tc.Commit(ctx) assert.NoError(t, err) assert.Empty(t, ts.getLastRequests()) + assert.Equal(t, txn.TxnStatus_Committed, tc.mu.txn.Status) }, WithTxnReadyOnly()) } @@ -209,6 +211,7 @@ func TestCommitWithLockTablesChanged(t *testing.T) { tc.mu.txn.TNShards = append(tc.mu.txn.TNShards, metadata.TNShard{TNShardRecord: metadata.TNShardRecord{ShardID: 1}}) err = tc.Commit(ctx) assert.Error(t, err) + assert.Equal(t, txn.TxnStatus_Aborted, tc.mu.txn.Status) // table 1 will be removed bind, err := s.GetLockTableBind(0, tableID1) diff --git a/pkg/util/metric/mometric/cron_task.go b/pkg/util/metric/mometric/cron_task.go index 1714c337c292..d863412c9458 100644 --- a/pkg/util/metric/mometric/cron_task.go +++ b/pkg/util/metric/mometric/cron_task.go @@ -413,12 +413,12 @@ func checkNewAccountSize(ctx context.Context, logger *log.MOLogger, sqlExecutor continue } - if result.RowCount() == 0 { + if showRet.RowCount() == 0 { logger.Warn("failed to fetch new account size, not exist.") continue } - sizeMB, err = result.GetFloat64(ctx, 0, sizeIdx) + sizeMB, err = showRet.GetFloat64(ctx, 0, sizeIdx) if err != nil { logger.Error("failed to fetch new account size", zap.Error(err), zap.String("account", account)) continue @@ -427,7 +427,7 @@ func checkNewAccountSize(ctx context.Context, logger *log.MOLogger, sqlExecutor if snapshotSizeIdx == math.MaxUint64 { snapshotSizeMB = 0.0 } else { - snapshotSizeMB, err = result.GetFloat64(ctx, 0, snapshotSizeIdx) + snapshotSizeMB, err = showRet.GetFloat64(ctx, 0, snapshotSizeIdx) if err != nil { logger.Error("failed to fetch new account size", zap.Error(err), zap.String("account", account)) continue diff --git a/pkg/vm/message/joinMapMsg.go b/pkg/vm/message/joinMapMsg.go index b01bc38b388d..d771465517f3 100644 --- a/pkg/vm/message/joinMapMsg.go +++ b/pkg/vm/message/joinMapMsg.go @@ -20,10 +20,9 @@ import ( "strconv" "sync/atomic" + "github.com/matrixorigin/matrixone/pkg/common/hashmap" "github.com/matrixorigin/matrixone/pkg/common/mpool" "github.com/matrixorigin/matrixone/pkg/container/batch" - - "github.com/matrixorigin/matrixone/pkg/common/hashmap" ) var _ Message = new(JoinMapMsg) @@ -74,6 +73,7 @@ type JoinMap struct { mpool *mpool.MPool multiSels JoinSels batches []*batch.Batch + //ignoreRows *bitmap.Bitmap } func NewJoinMap(sels JoinSels, ihm *hashmap.IntHashMap, shm *hashmap.StrHashMap, batches []*batch.Batch, m *mpool.MPool) *JoinMap { @@ -128,6 +128,14 @@ func (jm *JoinMap) GetSels(k uint64) []int32 { return jm.multiSels.GetSels(int32(k)) } +//func (jm *JoinMap) GetIgnoreRows() *bitmap.Bitmap { +// return jm.ignoreRows +//} + +//func (jm *JoinMap) SetIgnoreRows(ignoreRows *bitmap.Bitmap) { +// jm.ignoreRows = ignoreRows +//} + func (jm *JoinMap) NewIterator() hashmap.Iterator { if jm.shm != nil { return jm.shm.NewIterator() diff --git a/pkg/vm/types.go b/pkg/vm/types.go index a6120f54c7b2..a2437c37c69d 100644 --- a/pkg/vm/types.go +++ b/pkg/vm/types.go @@ -52,6 +52,7 @@ const ( Mark IndexJoin IndexBuild + DedupJoin Merge MergeTop @@ -555,7 +556,7 @@ func CannotRemote(op Operator) bool { } type ModificationArgument interface { - AffectedRows() uint64 + GetAffectedRows() uint64 } // doHandleAllOp function uses post traversal to recursively process nodes in the operand tree. diff --git a/proto/pipeline.proto b/proto/pipeline.proto index ae596b8f67a7..7ec5273f94be 100644 --- a/proto/pipeline.proto +++ b/proto/pipeline.proto @@ -133,11 +133,11 @@ message Insert{ message MultiUpdate { uint64 affected_rows = 1; - bool ToWriteS3 = 2; + uint32 Action = 2; uint32 IBucket = 3; uint32 NBucket = 4; map SegmentMap = 5; - repeated plan.UpdateCtx update_ctx = 6; + repeated plan.UpdateCtx update_ctx_list = 6; } message Array{ @@ -352,6 +352,18 @@ message MarkJoin { int32 shuffle_idx = 9; } +message DedupJoin { + repeated plan.Expr left_cond = 1; + repeated plan.Expr right_cond = 2; + repeated plan.RuntimeFilterSpec runtime_filter_build_list = 3; + bool is_shuffle = 4; + int32 join_map_tag = 5; + int32 shuffle_idx = 6; + plan.Node.OnDuplicateAction on_duplicate_action = 7; + string dedup_col_name = 8; + repeated plan.Type dedup_col_types = 9 [(gogoproto.nullable) = false]; +} + message Product { repeated int32 rel_list = 1; repeated int32 col_list = 2; @@ -429,6 +441,10 @@ message HashBuild { int32 JoinMapTag = 6; int32 JoinMapRefCnt = 7; plan.RuntimeFilterSpec RuntimeFilterSpec = 8; + bool is_dedup = 9; + plan.Node.OnDuplicateAction on_duplicate_action = 10; + string dedup_col_name = 11; + repeated plan.Type dedup_col_types = 12 [(gogoproto.nullable) = false]; } message Shufflebuild { @@ -439,6 +455,10 @@ message Shufflebuild { plan.RuntimeFilterSpec RuntimeFilterSpec = 5; int32 JoinMapTag = 6; int32 ShuffleIdx = 7; + bool is_dedup = 8; + plan.Node.OnDuplicateAction on_duplicate_action = 9; + string dedup_col_name = 10; + repeated plan.Type dedup_col_types = 11 [(gogoproto.nullable) = false]; } message Indexbuild { @@ -466,11 +486,12 @@ message Instruction{ Connector connect = 4; Dispatch dispatch = 5; Group agg = 6; + Join join = 7; LeftJoin left_join = 8; SemiJoin semi_join = 9; SingleJoin single_join = 10; MarkJoin mark_join = 11; - Join join = 12; + DedupJoin dedup_join = 12; Product product = 13; TableFunction table_function = 14; ExternalScan external_scan = 16; @@ -514,7 +535,8 @@ message Instruction{ Shufflebuild shuffle_build = 49; Indexbuild index_build = 50; Apply apply = 51; - PostDml post_dml = 52; + MultiUpdate multi_update = 52; + PostDml post_dml = 53; } message AnalysisList { diff --git a/proto/plan.proto b/proto/plan.proto index 58f2959e4ab9..4204532a9f6f 100644 --- a/proto/plan.proto +++ b/proto/plan.proto @@ -614,13 +614,16 @@ message OnDuplicateKeyCtx { } message UpdateCtx { - ObjectRef obj_ref = 1; - TableDef table_def = 2; - repeated Expr insert_cols = 3; - repeated Expr delete_cols = 4; - repeated uint64 partition_table_ids = 5; - repeated string partition_table_names = 6; - int32 partition_idx = 7; + ObjectRef obj_ref = 1; + TableDef table_def = 2; + + repeated uint64 partition_table_ids = 3; + repeated string partition_table_names = 4; + int32 old_partition_idx = 5; + int32 new_partition_idx = 6; + + repeated ColRef insert_cols = 7 [(gogoproto.nullable) = false]; + repeated ColRef delete_cols = 8 [(gogoproto.nullable) = false]; } message InsertCtx { @@ -685,7 +688,7 @@ message OriginTableMessageForFuzzy { message Node { enum NodeType { UNKNOWN = 0; - // Node Types + ASSERT = 100; // Scans VALUE_SCAN = 1; @@ -694,66 +697,54 @@ message Node { EXTERNAL_SCAN = 4; MATERIAL_SCAN = 5; SOURCE_SCAN = 6; + EXTERNAL_FUNCTION = 7; - // Proj, for convenience + // Proper Relational Operators PROJECT = 10; - // External function call (UDF) - EXTERNAL_FUNCTION = 11; + AGG = 11; + DISTINCT = 12; + FILTER = 13; + JOIN = 14; + SAMPLE = 15; + SORT = 16; + WINDOW = 17; + UNION = 18; + UNION_ALL = 19; + UNIQUE = 20; + INTERSECT = 21; + INTERSECT_ALL = 22; + MINUS = 23; + MINUS_ALL = 24; + + // DML + INSERT = 30; + DELETE = 31; + REPLACE = 32; + LOCK_OP = 33; + ON_DUPLICATE_KEY = 34; + PRE_INSERT = 35; + PRE_DELETE = 36; + FUZZY_FILTER = 37; + // the node which build insert batch for hidden table(unique key) + PRE_INSERT_UK = 38; + PRE_INSERT_SK = 39; // Material, CTE, etc. - MATERIAL = 20; - RECURSIVE_CTE = 21; - SINK = 22; - SINK_SCAN = 23; - RECURSIVE_SCAN = 24; - - // Proper Relational Operators - AGG = 30; - DISTINCT = 31; - FILTER = 32; - JOIN = 33; - SAMPLE = 34; - SORT = 35; - UNION = 36; - UNION_ALL = 37; - UNIQUE = 38; - WINDOW = 39; - - // Physical tuple mover - BROADCAST = 40; - SPLIT = 41; - GATHER = 42; - - // Misc - ASSERT = 50; - - // - INSERT = 51; - DELETE = 52; - REPLACE = 53; - ON_DUPLICATE_KEY = 54; - PRE_INSERT = 55; - PRE_DELETE = 56; - // the node which build insert batch for hidden table(unique key) - PRE_INSERT_UK = 57; - PRE_INSERT_SK = 58; - MULTI_UPDATE = 59; - - LOCK_OP = 60; - // - INTERSECT = 61; - INTERSECT_ALL = 62; - MINUS = 63; - MINUS_ALL = 64; - - // - // - TIME_WINDOW = 65; - FILL = 66; - PARTITION = 67; - FUZZY_FILTER = 68; - APPLY = 69; - POSTDML = 70; + MATERIAL = 40; + RECURSIVE_CTE = 41; + SINK = 42; + SINK_SCAN = 43; + RECURSIVE_SCAN = 44; + BROADCAST = 45; + SPLIT = 46; + GATHER = 47; + + TIME_WINDOW = 50; + FILL = 51; + PARTITION = 52; + APPLY = 53; + MULTI_UPDATE = 54; + POSTDML = 55; } enum JoinType { @@ -765,8 +756,9 @@ message Node { ANTI = 5; SINGLE = 6; MARK = 7; - INDEX = 8; - L2 = 9; + INDEX = 8; + L2 = 9; + DEDUP = 10; } enum AggMode { @@ -784,6 +776,12 @@ message Node { LINEAR = 5; } + enum OnDuplicateAction { + ERROR = 0; + IGNORE = 1; + UPDATE = 2; + } + enum ApplyType { CROSSAPPLY = 0; OUTERAPPLY = 1; @@ -825,6 +823,8 @@ message Node { ObjectRef parent_obj_ref = 19; RowsetData rowset_data = 20; + repeated Expr insert_delete_cols = 21; + string extra_options = 22; DeleteCtx delete_ctx = 23; @@ -888,6 +888,9 @@ message Node { ApplyType apply_type = 64; PostDmlCtx post_dml_ctx = 65; + OnDuplicateAction on_duplicate_action = 66; + string dedup_col_name = 67; + repeated Type dedup_col_types = 68 [(gogoproto.nullable) = false]; } // Snapshot Represents a snapshot of the database @@ -935,6 +938,8 @@ message LockTarget { repeated uint64 partition_table_ids = 8; bool block = 9; lock.LockMode Mode = 10; + int32 primary_col_rel_pos = 11; + int32 filter_col_rel_pos = 12; } message PreInsertUkCtx { diff --git a/test/distributed/cases/array/array_index.sql b/test/distributed/cases/array/array_index.sql index 8923be9e1b82..babc10815526 100644 --- a/test/distributed/cases/array/array_index.sql +++ b/test/distributed/cases/array/array_index.sql @@ -714,4 +714,4 @@ select * from vector_index_08; -- post SET experimental_ivf_index = 0; -drop database vecdb2; \ No newline at end of file +drop database vecdb2; diff --git a/test/distributed/cases/auto_increment/auto_increment.result b/test/distributed/cases/auto_increment/auto_increment.result index 6ed92cb62639..c805168d7397 100644 --- a/test/distributed/cases/auto_increment/auto_increment.result +++ b/test/distributed/cases/auto_increment/auto_increment.result @@ -22,7 +22,7 @@ Select * from auto_increment02; col1 10 Insert into auto_increment02 values(10); -Duplicate entry '10' for key '__mo_index_idx_col' +Duplicate entry '10' for key '(.*)' insert into auto_increment02 values(100); select last_insert_id(); last_insert_id() @@ -83,9 +83,9 @@ col1 10001 10002 Insert into auto_increment05 values(10001); -Duplicate entry '10001' for key '__mo_index_idx_col' +Duplicate entry '10001' for key '(.*)' Insert into auto_increment05 values(10002); -Duplicate entry '10002' for key '__mo_index_idx_col' +Duplicate entry '10002' for key '(.*)' Select * from auto_increment05; col1 10000 diff --git a/test/distributed/cases/auto_increment/auto_increment.sql b/test/distributed/cases/auto_increment/auto_increment.sql index a875c19d21c5..c3262f0ca3de 100644 --- a/test/distributed/cases/auto_increment/auto_increment.sql +++ b/test/distributed/cases/auto_increment/auto_increment.sql @@ -20,6 +20,7 @@ Drop table if exists auto_increment02; Create table auto_increment02(col1 int auto_increment unique key)auto_increment = 10; Insert into auto_increment02 values(); Select * from auto_increment02; +-- @pattern Insert into auto_increment02 values(10); insert into auto_increment02 values(100); select last_insert_id(); @@ -62,7 +63,9 @@ Insert into auto_increment05 values(); Insert into auto_increment05 values(); select last_insert_id(); Select * from auto_increment05; +-- @pattern Insert into auto_increment05 values(10001); +-- @pattern Insert into auto_increment05 values(10002); Select * from auto_increment05; Drop table auto_increment05; diff --git a/test/distributed/cases/auto_increment/auto_increment_columns.result b/test/distributed/cases/auto_increment/auto_increment_columns.result index 42d1e0e83421..972eb0dacba8 100644 --- a/test/distributed/cases/auto_increment/auto_increment_columns.result +++ b/test/distributed/cases/auto_increment/auto_increment_columns.result @@ -821,7 +821,7 @@ select last_insert_id(); last_insert_id() 10 Insert into auto_increment02 values(10); -Duplicate entry '10' for key '__mo_index_idx_col' +Duplicate entry '10' for key '(.*)' Select * from auto_increment02; col1 10 diff --git a/test/distributed/cases/auto_increment/auto_increment_columns.sql b/test/distributed/cases/auto_increment/auto_increment_columns.sql index 7073f9486247..96c1b29d9a32 100644 --- a/test/distributed/cases/auto_increment/auto_increment_columns.sql +++ b/test/distributed/cases/auto_increment/auto_increment_columns.sql @@ -480,6 +480,7 @@ select last_insert_id(); Insert into auto_increment02 values(); Select * from auto_increment02; select last_insert_id(); +-- @pattern Insert into auto_increment02 values(10); Select * from auto_increment02; select last_insert_id(); diff --git a/test/distributed/cases/ddl/alter_table.result b/test/distributed/cases/ddl/alter_table.result index f84dc3a9ddc0..f6dabbe7157a 100644 --- a/test/distributed/cases/ddl/alter_table.result +++ b/test/distributed/cases/ddl/alter_table.result @@ -54,7 +54,7 @@ insert into t1 values(5, '1981-09-28','bcvdf', 25); insert into t1 values(6, '1981-05-01','green', 26); ALTER TABLE t1 ADD UNIQUE idx1 (col2, col3); insert into t1 values(7, '1981-05-01','green', 26); -Duplicate entry ('\(\d{4}-\d{2}-\d{2},\w{5}\)'|'\d{5}\w\d{17}\w\d{2}') for key '__mo_index_idx_col' +Duplicate entry ('\(\d{4}-\d{2}-\d{2},\w{5}\)'|'\d{5}\w\d{17}\w\d{2}') for key '(.*)' show index from t1; Table Non_unique Key_name Seq_in_index Column_name Collation Cardinality Sub_part Packed Null Index_type Comment Index_comment Index_params Visible Expression t1 0 PRIMARY 1 col1 A 0 NULL NULL YES NULL diff --git a/test/distributed/cases/ddl/alter_table_add_drop_primary_key.result b/test/distributed/cases/ddl/alter_table_add_drop_primary_key.result index 9d99c8831abb..696697a485a3 100644 --- a/test/distributed/cases/ddl/alter_table_add_drop_primary_key.result +++ b/test/distributed/cases/ddl/alter_table_add_drop_primary_key.result @@ -234,7 +234,7 @@ insert into pri10 (col1, col2) values (1, null); constraint violation: Column 'col2' cannot be null insert into pri10 values (-2, 'p'); insert into pri10 (col1, col2) values (1, 'a'); -Duplicate entry ('\(1\,a\)'|'\d\w\d{12}') for key '__mo_cpkey_col' +Duplicate entry ('\(1\,a\)'|'\d\w\d{12}') for key '(.*)' select * from pri10; col1 col2 1 a diff --git a/test/distributed/cases/ddl/alter_table_change_column.result b/test/distributed/cases/ddl/alter_table_change_column.result index 7f2bb3ac58ce..4d2b0718ca15 100644 --- a/test/distributed/cases/ddl/alter_table_change_column.result +++ b/test/distributed/cases/ddl/alter_table_change_column.result @@ -1421,7 +1421,7 @@ show create table index02; Table Create Table index02 CREATE TABLE `index02` (\n `a` int NOT NULL,\n `bnewNew` varchar(20) DEFAULT NULL,\n `c` date DEFAULT NULL,\n `d` decimal(7,2) DEFAULT NULL,\n PRIMARY KEY (`a`),\n UNIQUE KEY `bnewnew` (`bnewNew`),\n KEY `a` (`a`,`bnewNew`),\n KEY `c` (`c`)\n) insert into index02 values (4, 'ab', '2000-10-10', 10000); -Duplicate entry 'ab' for key '__mo_index_idx_col' +Duplicate entry 'ab' for key '(.*)' insert into index02 values (5, 'gh', '1999-12-31', 20000); delete from index02 where bnewnew = 'ab'; update index02 set bnewnew = 'database' where bnewnEW = 'ad'; diff --git a/test/distributed/cases/ddl/alter_table_change_column.sql b/test/distributed/cases/ddl/alter_table_change_column.sql index 0a68aa963ca8..4a200b0d1252 100644 --- a/test/distributed/cases/ddl/alter_table_change_column.sql +++ b/test/distributed/cases/ddl/alter_table_change_column.sql @@ -750,6 +750,7 @@ show create table index02; alter table index02 change b bnewNew VARCHAR(20) UNIQUE KEY; show index from index02; show create table index02; +-- @pattern insert into index02 values (4, 'ab', '2000-10-10', 10000); insert into index02 values (5, 'gh', '1999-12-31', 20000); delete from index02 where bnewnew = 'ab'; diff --git a/test/distributed/cases/ddl/comprimary_key.result b/test/distributed/cases/ddl/comprimary_key.result index b3868bf8a330..d80d5c1f440f 100644 --- a/test/distributed/cases/ddl/comprimary_key.result +++ b/test/distributed/cases/ddl/comprimary_key.result @@ -24,7 +24,7 @@ col3 col18 5 OPQR.STU-_+=VWXYZa 3 L/MN?OPQR.STU-_+=VWXYZabcdefghigklmnopqrstuvwxyz012 insert into cpk_table_1 select * from ex_table_cpk; -Duplicate entry ('\(\d\,\w{7}\)'|'\d\w\d{9}\w\d{14}') for key '__mo_cpkey_col' +Duplicate entry ('\(\d\,.*\)'|'\w{10,300}') for key '(.*)' create table cpk_table_2(col1 tinyint,col2 smallint,col3 int,col4 bigint,col5 tinyint unsigned,col6 smallint unsigned,col7 int unsigned,col8 bigint unsigned,col9 float,col10 double,col11 varchar(255),col12 Date,col13 DateTime,col14 timestamp,col15 bool,col16 decimal(5,2),col17 text,col18 varchar(255),col19 varchar(255),col20 varchar(255),primary key(col1,col12,col3)); insert into cpk_table_2 select * from ex_table_cpk; select col1,col12,col3 from cpk_table_2; @@ -34,7 +34,7 @@ col1 col12 col3 3 2020-02-16 5 3 2020-05-08 3 insert into cpk_table_2 select * from ex_table_cpk; -Duplicate entry ('\(\d\,\d{4}-\d{2}-\d{2}\,\d\)'|'\d{11}\w\d{5}\w\d{4}') for key '__mo_cpkey_col' +Duplicate entry ('\(\d\,\d{4}-\d{2}-\d{2}\,\d\)'|'\w{10,100}') for key '(.*)' create table cpk_table_3(col1 tinyint,col2 smallint,col3 int,col4 bigint,col5 tinyint unsigned,col6 smallint unsigned,col7 int unsigned,col8 bigint unsigned,col9 float,col10 double,col11 varchar(255),col12 Date,col13 DateTime,col14 timestamp,col15 bool,col16 decimal(5,2),col17 text,col18 varchar(255),col19 varchar(255),col20 varchar(255),primary key(col2,col9,col14,col20)); insert into cpk_table_3 select * from ex_table_cpk; select col2,col9,col14,col20 from cpk_table_3; @@ -158,4 +158,4 @@ PRIMARY KEY (account_id, task_id) insert into test values(3,"019126ce-64a8-78cf-1234-be2626281abd","task3"); insert into test values(3,"019126ce-64a8-78cf-5678-be2626281abd","task4"); insert into test values(3,"019126ce-64a8-78cf-5678-be2626281abd","task4"); -Duplicate entry ('(\(\d,\w{8}-\w{4}-\w{4}-\w{4}-\w{12}\))'|'(\d{8}\w{32})'|'40150346017461736b3400') for key '__mo_cpkey_col' +Duplicate entry ('(\(\d,\w{8}-\w{4}-\w{4}-\w{4}-\w{12}\))'|'(\d{8}\w{32})'|'40150346017461736b3400') for key '(.*)' diff --git a/test/distributed/cases/ddl/create_index.result b/test/distributed/cases/ddl/create_index.result index 6141f6c00335..8ae6c026d868 100644 --- a/test/distributed/cases/ddl/create_index.result +++ b/test/distributed/cases/ddl/create_index.result @@ -50,7 +50,7 @@ col1 col2 col3 col4 insert into t3 values(4,"Dora", 28,'zbcvdf'); Duplicate entry '4' for key 'col1' insert into t3 values(5,"Dora", 29,'zbcvdf'); -Duplicate entry ('\(\w{4}\,\d{2}\)'|'\d{7}\w\d{8}\w\d\w\d{5}') for key '__mo_index_idx_col' +Duplicate entry ('\(\w{4}\,\d{2}\)'|'\d{7}\w\d{8}\w\d\w\d{5}') for key '(.*)' drop table t3; create table t4(a int, b int, key(c)); invalid input: column 'c' is not exist diff --git a/test/distributed/cases/ddl/create_table_as_select.result b/test/distributed/cases/ddl/create_table_as_select.result index 8cc6f46fd583..0188e8c77d2b 100644 --- a/test/distributed/cases/ddl/create_table_as_select.result +++ b/test/distributed/cases/ddl/create_table_as_select.result @@ -61,7 +61,7 @@ a b 2147483647 max 1 1_1 CREATE table t6 (a int unique) as select * from t1; -Duplicate entry '1' for key '__mo_index_idx_col' +Duplicate entry '1' for key 'a' drop table t6; CREATE table t6 as select max(a) from t1; desc t6; @@ -1193,7 +1193,7 @@ desc test03; Field Type Null Key Default Extra Comment col2 DECIMAL(38,0) YES UNI null insert into test03 values (372.324); -Duplicate entry '372' for key '__mo_index_idx_col' +Duplicate entry '372' for key '(.*)' drop table if exists test04; create table test04 (col1 int, col2 varchar(50), key(col1, col2)) as select col1, col3 from test01; show create table test04; @@ -1215,7 +1215,7 @@ col1 col2 2 39304 3 372 insert into test05 values (2, 39304.3424); -Duplicate entry ('\d\w\d{38}'|'\(2\,39304\)') for key '__mo_cpkey_col' +Duplicate entry ('\d\w\d{38}'|'\(2\,39304\)') for key '(.*)' alter table test01 rename column col1 to newCol; show create table test01; Table Create Table @@ -1353,7 +1353,7 @@ drop table if exists abnormal05; create table abnormal04 (col1 int primary key ) as select col1 from abnormal03; Duplicate entry '1' for key 'col1' create table abnormal05 (col2 bigint unique key) as select col2 from abnormal03; -Duplicate entry '8324824234' for key '__mo_index_idx_col' +Duplicate entry '8324824234' for key 'col2' drop table abnormal03; drop table if exists abnormal06; create table abnormal06 (col1 bigint, col2 decimal); diff --git a/test/distributed/cases/ddl/create_table_as_select.sql b/test/distributed/cases/ddl/create_table_as_select.sql index 216fa577ca0f..dd6fe09758ba 100644 --- a/test/distributed/cases/ddl/create_table_as_select.sql +++ b/test/distributed/cases/ddl/create_table_as_select.sql @@ -822,6 +822,7 @@ drop table if exists test03; create table test03 (col2 decimal unique key) as select col2 from test01; show create table test03; desc test03; +-- @pattern insert into test03 values (372.324); drop table if exists test04; diff --git a/test/distributed/cases/ddl/hash_key_partition.result b/test/distributed/cases/ddl/hash_key_partition.result index b415b082a6e8..fe82ac86e14b 100644 --- a/test/distributed/cases/ddl/hash_key_partition.result +++ b/test/distributed/cases/ddl/hash_key_partition.result @@ -587,7 +587,7 @@ select * from `%!%p3%!%p_table_15`; col1 col2 col3 col4 create table p_table_16(col1 bigint,col2 date,col3 varchar(25),col4 decimal(6,4),primary key(col1,col2,col3))partition by key(col2)partitions 8; insert into p_table_16 values (1000,'1999-09-29','res1',0.12),(6000,'1999-10-01','opt1',0.89),(729,'1999-10-02','res2',0.32),(6000,'1999-10-01','opt1',0.64), (6000,'1999-10-01','opt1',0.55),(206,'1999-10-30','oop1',0.87); -Duplicate entry '(6000,1999-10-01,opt1)' for key '__mo_cpkey_col' +Duplicate entry '(6000,1999-10-01,opt1)' for key '(col1,col2,col3)' insert into p_table_16 values (1000,'0001-09-29','res1',0.12),(6000,'1999-10-11','opt1',0.89); select * from p_table_16; col1 col2 col3 col4 diff --git a/test/distributed/cases/dml/delete/delete_index.result b/test/distributed/cases/dml/delete/delete_index.result index 08028b896bf8..72a35052f4bc 100644 --- a/test/distributed/cases/dml/delete/delete_index.result +++ b/test/distributed/cases/dml/delete/delete_index.result @@ -241,7 +241,7 @@ insert into t1 values(2, 2, 2); insert into t1 values(3, 3, 3); insert into t1 values(4, 4, 4); insert into t1 values(NULL, 5, 5); -Duplicate entry '5' for key '__mo_index_idx_col' +Duplicate entry '5' for key '(.*)' drop table if exists t1; drop table if exists t2; create table t1(a int, b int, c int, unique key(a, b)); @@ -322,7 +322,7 @@ insert into t1 values(2, 2, 2); insert into t1 values(3, 3, 3); insert into t1 values(4, 4, 4); insert into t1 values(NULL, 5, 5); -Duplicate entry ('\(\d\,\d\)'|'\d\w\d{5}\w\d{4}') for key '__mo_index_idx_col' +Duplicate entry ('\(\d\,\d\)'|'\d\w\d{5}\w\d{4}') for key '(.*)' drop table if exists t1; drop table if exists t2; create table t1(a int, b int, c int, unique key(a)); @@ -403,7 +403,7 @@ insert into t1 values(2, 2, 2); insert into t1 values(3, 3, 3); insert into t1 values(4, 4, 4); insert into t1 values(NULL, 5, 5); -Duplicate entry '5' for key '__mo_index_idx_col' +Duplicate entry '5' for key '(.*)' drop table if exists t1; drop table if exists t2; create table t1(a int, b int, c int, unique key(a, b)); @@ -484,7 +484,7 @@ insert into t1 values(2, 2, 2); insert into t1 values(3, 3, 3); insert into t1 values(4, 4, 4); insert into t1 values(NULL, 5, 5); -Duplicate entry ('\(\d\,\d\)'|'\d\w\d{5}\w\d{4}') for key '__mo_index_idx_col' +Duplicate entry ('\(\d\,\d\)'|'\d\w\d{5}\w\d{4}') for key '(.*)' drop table if exists t1; drop table if exists t2; create table t1(a int, b int, c int, unique key(a, b), unique key(c)); @@ -525,7 +525,7 @@ insert into t1 values(2, 2, 2); insert into t1 values(3, 3, 3); insert into t1 values(4, 4, 4); insert into t1 values(NULL, 5, 5); -Duplicate entry '5' for key '__mo_index_idx_col' +Duplicate entry '5' for key '(.*)' drop table if exists t1; create table t1(a int, b int, c int, primary key(c), unique key(a)); insert into t1 values(1, 1, 1); diff --git a/test/distributed/cases/dml/delete/delete_index.test b/test/distributed/cases/dml/delete/delete_index.test index 3eb54a96d949..1e8d0e396630 100644 --- a/test/distributed/cases/dml/delete/delete_index.test +++ b/test/distributed/cases/dml/delete/delete_index.test @@ -130,6 +130,7 @@ insert into t1 values(1, 1, 1); insert into t1 values(2, 2, 2); insert into t1 values(3, 3, 3); insert into t1 values(4, 4, 4); +-- @pattern insert into t1 values(NULL, 5, 5); drop table if exists t1; @@ -231,6 +232,7 @@ insert into t1 values(1, 1, 1); insert into t1 values(2, 2, 2); insert into t1 values(3, 3, 3); insert into t1 values(4, 4, 4); +-- @pattern insert into t1 values(NULL, 5, 5); drop table if exists t1; @@ -307,6 +309,7 @@ insert into t1 values(1, 1, 1); insert into t1 values(2, 2, 2); insert into t1 values(3, 3, 3); insert into t1 values(4, 4, 4); +-- @pattern insert into t1 values(NULL, 5, 5); drop table if exists t1; diff --git a/test/distributed/cases/dml/insert/insert_duplicate.result b/test/distributed/cases/dml/insert/insert_duplicate.result index 94719ed04659..3dcfc36df4a5 100644 --- a/test/distributed/cases/dml/insert/insert_duplicate.result +++ b/test/distributed/cases/dml/insert/insert_duplicate.result @@ -365,7 +365,7 @@ Duplicate entry 'matrixone'' for key 'a' drop table if exists indup_09; create table indup_09(a varchar, b varchar, primary key (a, b)); insert into indup_09 values ('matrixone\'', 'mo-tester\''), ('matrixone\'', 'mo-tester\''); -Duplicate entry '(matrixone',mo-tester')' for key '__mo_cpkey_col' +Duplicate entry '(matrixone',mo-tester')' for key '(a,b)' drop table if exists indup_10; create table indup_10(a varchar(256), b int); insert into indup_10 (a, b) select "matrixone " || " some space " || result, 1 from generate_series (1, 500000)g; diff --git a/test/distributed/cases/dml/insert/insert_ignore.result b/test/distributed/cases/dml/insert/insert_ignore.result index 00dd50504417..71d282317e25 100644 --- a/test/distributed/cases/dml/insert/insert_ignore.result +++ b/test/distributed/cases/dml/insert/insert_ignore.result @@ -116,10 +116,15 @@ count(*) 100000 select * from insert_ignore_08 where c2 in (45,55,22,1,222,19); c1 c2 -1 45 -5 1 -6 22 20 45 21 55 +1 45 +6 22 +5 1 1000 222 99999 19 +19 19 +22 22 +45 45 +55 55 +222 222 diff --git a/test/distributed/cases/dml/insert/insert_ignore.sql b/test/distributed/cases/dml/insert/insert_ignore.sql index 5feac942dade..467860ddff5f 100644 --- a/test/distributed/cases/dml/insert/insert_ignore.sql +++ b/test/distributed/cases/dml/insert/insert_ignore.sql @@ -1,5 +1,4 @@ -- insert ignore autoincrement primary key --- @bvt:issue#15365 create table insert_ignore_01(c1 int not null auto_increment primary key,c2 varchar(10)); insert into insert_ignore_01(c2) values("a"),("b"),("c"),("d"); insert ignore into insert_ignore_01 values(3,"e"),(6,"f"),(1,"g"); @@ -7,7 +6,6 @@ insert ignore into insert_ignore_01(c2) values("h"),("g"),("k"); insert ignore into insert_ignore_01 values(NULL,NULL); select * from insert_ignore_01; drop table insert_ignore_01; --- @bvt:issue -- insert ignore multi primary key create table insert_ignore_01 (part_id INT NOT NULL,color VARCHAR(20) NOT NULL,quantity INT,PRIMARY KEY (part_id, color)); insert ignore into insert_ignore_01 (part_id, color, quantity)values(1, 'Red', 10),(1, 'Blue', 20),(2, 'Green', 15),(1, 'Red', 5); @@ -66,7 +64,6 @@ create table insert_ignore_06 (sale_id INT AUTO_INCREMENT,product_id INT,sale_am insert ignore into insert_ignore_06 (product_id, sale_amount, sale_date) VALUES(1, 1000.00, '1990-04-01'),(2, 1500.00, '1992-05-01'),(3, 500.00, '1995-06-01'),(1, 2000.00, '1991-07-01'); select * from insert_ignore_06; -- insert ignore select from table --- @bvt:issue#15349 create table insert_ignore_07(c1 int primary key auto_increment, c2 int); insert into insert_ignore_07(c2) select result from generate_series(1,100000) g; create table insert_ignore_08(c1 int primary key, c2 int); @@ -74,4 +71,3 @@ insert into insert_ignore_08 values(20,45),(21,55),(1,45),(6,22),(5,1),(1000,222 insert ignore into insert_ignore_08 select * from insert_ignore_07; select count(*) from insert_ignore_08; select * from insert_ignore_08 where c2 in (45,55,22,1,222,19); --- @bvt:issue diff --git a/test/distributed/cases/dml/insert/not_null_check.result b/test/distributed/cases/dml/insert/not_null_check.result index eff87a3f2f82..a667638e6701 100644 --- a/test/distributed/cases/dml/insert/not_null_check.result +++ b/test/distributed/cases/dml/insert/not_null_check.result @@ -27,4 +27,7 @@ insert into t values (2, null); insert into t values (3, null); update t set a=null; constraint violation: Column 'a' cannot be null +drop table if exists t1; +create table t1 (a int primary key, b int, c int, unique key(b,c)); +INSERT INTO t1 SELECT result,result,null FROM generate_series(1,1000000) g; drop database if exists test; \ No newline at end of file diff --git a/test/distributed/cases/dml/insert/not_null_check.sql b/test/distributed/cases/dml/insert/not_null_check.sql index c6b54b614e8a..48c7c32dfda7 100644 --- a/test/distributed/cases/dml/insert/not_null_check.sql +++ b/test/distributed/cases/dml/insert/not_null_check.sql @@ -20,4 +20,7 @@ insert into t values (1, null); insert into t values (2, null); insert into t values (3, null); update t set a=null; +drop table if exists t1; +create table t1 (a int primary key, b int, c int, unique key(b,c)); +INSERT INTO t1 SELECT result,result,null FROM generate_series(1,1000000) g; drop database if exists test; \ No newline at end of file diff --git a/test/distributed/cases/dml/update/update.result b/test/distributed/cases/dml/update/update.result index fe4a0d942a0d..a9a20a84a188 100644 --- a/test/distributed/cases/dml/update/update.result +++ b/test/distributed/cases/dml/update/update.result @@ -215,19 +215,21 @@ a b 100 200 101 201 update t1 set a=null; +constraint violation: Column 'a' cannot be null select * from t1; a b -102 1 -103 2 -104 200 -105 201 +1 1 +2 2 +100 200 +101 201 update t1 set b=null; +constraint violation: Column 'b' cannot be null select * from t1; a b -102 202 -103 203 -104 204 -105 205 +1 1 +2 2 +100 200 +101 201 drop table if exists t1; create table t1( id int, diff --git a/test/distributed/cases/dml/update/update_index.result b/test/distributed/cases/dml/update/update_index.result index 92ddd595daab..2a97bbdf76a6 100644 --- a/test/distributed/cases/dml/update/update_index.result +++ b/test/distributed/cases/dml/update/update_index.result @@ -13,7 +13,7 @@ update t1 set a = 6 where a = 1; insert into t1 values(1, 1, 1); update t1 set a = 7 where a is null; insert into t1 values(7, 7, 7); -Duplicate entry '7' for key '__mo_index_idx_col' +Duplicate entry '7' for key '(.*)' update t1 set a = null where a = 7; insert into t1 values(7, 7, 7); select * from t1; @@ -38,7 +38,7 @@ update t1 set a = 6 where a = 1; insert into t1 values(1, 1, 1); update t1 set a = 7 where a is null; insert into t1 values(7, 7, 7); -Duplicate entry ('\(\d\,\d\)'|'\d\w\d{5}\w\d{4}') for key '__mo_index_idx_col' +Duplicate entry ('\(\d\,\d\)'|'\d\w\d{5}\w\d{4}') for key '(.*)' update t1 set a = null where a = 7; insert into t1 values(7, 7, 7); insert into t1 values(null, 8, 8); @@ -77,9 +77,9 @@ insert into t1 values(1, 1, 1); insert into t2 values(1, 1, 1); update t1, t2 set t1.a = 7, t2.a = 7 where t1.a is null and t2.a is null; insert into t1 values(7, 7, 7); -Duplicate entry ('\(\d\,\d\)'|'\d\w\d{5}\w\d{4}') for key '__mo_index_idx_col' +Duplicate entry ('\(\d\,\d\)'|'\d\w\d{5}\w\d{4}') for key '(.*)' insert into t2 values(7, 7, 7); -Duplicate entry '7' for key '__mo_index_idx_col' +Duplicate entry '7' for key '(.*)' update t1, t2 set t1.a = null, t2.a = null where t1.a = 7 and t1.a = t2.a; insert into t1 values(7, 7, 7); insert into t1 values(null, 8, 8); @@ -88,7 +88,7 @@ insert into t2 values(null, 8, 8); update t1, t2 set t1.a = 8, t1.b = null, t2.a = 8, t2.b = null where t1.b = 8 and t1.b = t2.b; insert into t1 values(8, 8, 8); insert into t2 values(8, 8, 8); -Duplicate entry '8' for key '__mo_index_idx_col' +Duplicate entry '8' for key '(.*)' select * from t1; a b c 2 2 2 @@ -123,7 +123,7 @@ insert into t1 values(null, 7, 7); update t1 set a = 6 where a = 1; update t1 set a = 7 where a is null; insert into t1 values(7, 7, 8); -Duplicate entry '7' for key '__mo_index_idx_col' +Duplicate entry '7' for key '(.*)' update t1 set a = null where a = 7; insert into t1 values(7, 7, 9); select * from t1; @@ -146,7 +146,7 @@ insert into t1 values(null, 7, 7); update t1 set a = 6 where a = 1; update t1 set a = 7 where a is null; insert into t1 values(7, 7, 8); -Duplicate entry ('\(\d\,\d\)'|'\d\w\d{5}\w\d{4}') for key '__mo_index_idx_col' +Duplicate entry ('\(\d\,\d\)'|'\d\w\d{5}\w\d{4}') for key '(.*)' update t1 set a = null where a = 7; insert into t1 values(7, 7, 9); insert into t1 values(null, 8, 10); @@ -176,7 +176,7 @@ update t1 set a = 6 where a = 1; insert into t1 values(1, 1, 1); update t1 set a = 7 where a is null; insert into t1 values(7, 7, 7); -Duplicate entry '7' for key '__mo_index_idx_col' +Duplicate entry '7' for key '(.*)' update t1 set a = null where a = 7; insert into t1 values(7, 7, 7); select * from t1; @@ -201,7 +201,7 @@ update t1 set a = 6 where a = 1; insert into t1 values(1, 1, 1); update t1 set a = 7 where a is null; insert into t1 values(7, 7, 7); -Duplicate entry '7' for key '__mo_index_idx_col' +Duplicate entry '7' for key '(.*)' update t1 set a = null where a = 7; insert into t1 values(7, 7, 7); select * from t1; diff --git a/test/distributed/cases/dml/update/update_index.test b/test/distributed/cases/dml/update/update_index.test index b66ec9f78509..fde2352b4c0e 100644 --- a/test/distributed/cases/dml/update/update_index.test +++ b/test/distributed/cases/dml/update/update_index.test @@ -12,6 +12,7 @@ insert into t1 values(null, 7, 7); update t1 set a = 6 where a = 1; insert into t1 values(1, 1, 1); update t1 set a = 7 where a is null; +-- @pattern insert into t1 values(7, 7, 7); update t1 set a = null where a = 7; insert into t1 values(7, 7, 7); @@ -59,6 +60,7 @@ insert into t2 values(1, 1, 1); update t1, t2 set t1.a = 7, t2.a = 7 where t1.a is null and t2.a is null; -- @pattern insert into t1 values(7, 7, 7); +-- @pattern insert into t2 values(7, 7, 7); update t1, t2 set t1.a = null, t2.a = null where t1.a = 7 and t1.a = t2.a; insert into t1 values(7, 7, 7); @@ -67,6 +69,7 @@ insert into t2 values(7, 7, 7); insert into t2 values(null, 8, 8); update t1, t2 set t1.a = 8, t1.b = null, t2.a = 8, t2.b = null where t1.b = 8 and t1.b = t2.b; insert into t1 values(8, 8, 8); +-- @pattern insert into t2 values(8, 8, 8); select * from t1; select * from t2; @@ -80,6 +83,7 @@ insert into t1 values(5, 5, 5); insert into t1 values(null, 7, 7); update t1 set a = 6 where a = 1; update t1 set a = 7 where a is null; +-- @pattern insert into t1 values(7, 7, 8); update t1 set a = null where a = 7; insert into t1 values(7, 7, 9); @@ -113,6 +117,7 @@ insert into t1 values(null, 7, 7); update t1 set a = 6 where a = 1; insert into t1 values(1, 1, 1); update t1 set a = 7 where a is null; +-- @pattern insert into t1 values(7, 7, 7); update t1 set a = null where a = 7; insert into t1 values(7, 7, 7); @@ -128,6 +133,7 @@ insert into t1 values(null, 7, 7); update t1 set a = 6 where a = 1; insert into t1 values(1, 1, 1); update t1 set a = 7 where a is null; +-- @pattern insert into t1 values(7, 7, 7); update t1 set a = null where a = 7; insert into t1 values(7, 7, 7); diff --git a/test/distributed/cases/dtype/decimal.result b/test/distributed/cases/dtype/decimal.result index 086ec0eb07bf..61d7e83405a0 100644 --- a/test/distributed/cases/dtype/decimal.result +++ b/test/distributed/cases/dtype/decimal.result @@ -1138,7 +1138,7 @@ a b c -28.32320 387213.0 3.2132313E7 null 327.32892 -38922.2123 INSERT INTO decimal15 VALUES(271.212121,387213.0,3289); -Duplicate entry '271.21212' for key '__mo_index_idx_col' +Duplicate entry '271.21212' for key '(.*)' DROP TABLE decimal15; DROP TABLE IF EXISTS decimal16; CREATE TABLE decimal16(a decimal, b decimal(38,10), c varchar(20),UNIQUE INDEX(a),INDEX(b)); diff --git a/test/distributed/cases/dtype/decimal.test b/test/distributed/cases/dtype/decimal.test index 9778861a3dd4..50c909854627 100644 --- a/test/distributed/cases/dtype/decimal.test +++ b/test/distributed/cases/dtype/decimal.test @@ -589,6 +589,7 @@ INSERT INTO decimal15 VALUES(NULL,327.328932,-38922.2123); SELECT * FROM decimal15; -- unique index cannot have duplicate value insertion +-- @pattern INSERT INTO decimal15 VALUES(271.212121,387213.0,3289); DROP TABLE decimal15; diff --git a/test/distributed/cases/dtype/numeric.result b/test/distributed/cases/dtype/numeric.result index 240499a88d89..26d4aa13f7a2 100644 --- a/test/distributed/cases/dtype/numeric.result +++ b/test/distributed/cases/dtype/numeric.result @@ -1170,7 +1170,7 @@ a b c -28.32320 387213.0 3.2132313E7 null 327.32892 -38922.2123 INSERT INTO numeric15 VALUES(271.212121,387213.0,3289); -Duplicate entry '271.21212' for key '__mo_index_idx_col' +Duplicate entry '271.21212' for key '(.*)' DROP TABLE numeric15; DROP TABLE IF EXISTS numeric16; CREATE TABLE numeric16(a numeric, b numeric(38,10), c varchar(20),UNIQUE INDEX(a),INDEX(b)); diff --git a/test/distributed/cases/dtype/numeric.sql b/test/distributed/cases/dtype/numeric.sql index 8a7b8859f7b7..eb756744d26e 100644 --- a/test/distributed/cases/dtype/numeric.sql +++ b/test/distributed/cases/dtype/numeric.sql @@ -589,6 +589,7 @@ INSERT INTO numeric15 VALUES(NULL,327.328932,-38922.2123); SELECT * FROM numeric15; -- unique index cannot have duplicate value insertion + -- @pattern INSERT INTO numeric15 VALUES(271.212121,387213.0,3289); DROP TABLE numeric15; diff --git a/test/distributed/cases/dtype/time_1.result b/test/distributed/cases/dtype/time_1.result index 7bd69b859976..68fd6d0b3554 100644 --- a/test/distributed/cases/dtype/time_1.result +++ b/test/distributed/cases/dtype/time_1.result @@ -136,7 +136,7 @@ drop table time_03; create table time_03 (t1 int,t2 time,t3 varchar(25),t4 time default '110034',primary key(t1,t2)); insert into time_03(t1,t2,t3) values (30,"24:59:09.932823","yellow"); insert into time_03(t1,t2,t3) values (30,"24:59:09.932823","oooppppp"); -Duplicate entry ('\(\d{2}\,\d{11}\)'|'\d\w\d{3}\w\d{6}\w\d{7}') for key '__mo_cpkey_col' +Duplicate entry ('\(\d{2}\,\d{2}\:\d{2}:\d{2}\)'|'\w{20}') for key '(.*)' insert into time_03(t1,t2,t3) values (31,"24:59:09.932823","postttttt"); insert into time_03(t1,t2,t3) values (32,NULL,"vinda"); constraint violation: Column 't2' cannot be null diff --git a/test/distributed/cases/dtype/time_1.sql b/test/distributed/cases/dtype/time_1.sql index d331ee4cbad6..43bdb4db0296 100644 --- a/test/distributed/cases/dtype/time_1.sql +++ b/test/distributed/cases/dtype/time_1.sql @@ -56,7 +56,7 @@ select * from time_03; drop table time_03; create table time_03 (t1 int,t2 time,t3 varchar(25),t4 time default '110034',primary key(t1,t2)); insert into time_03(t1,t2,t3) values (30,"24:59:09.932823","yellow"); --- @pattern + -- @pattern insert into time_03(t1,t2,t3) values (30,"24:59:09.932823","oooppppp"); insert into time_03(t1,t2,t3) values (31,"24:59:09.932823","postttttt"); insert into time_03(t1,t2,t3) values (32,NULL,"vinda"); diff --git a/test/distributed/cases/load_data/load_data.result b/test/distributed/cases/load_data/load_data.result index dda78bdd082d..fccd10c6a2d1 100644 --- a/test/distributed/cases/load_data/load_data.result +++ b/test/distributed/cases/load_data/load_data.result @@ -9,6 +9,11 @@ col6 smallint unsigned, col7 int unsigned, col8 bigint unsigned ); +load data infile '$resources/load_data/integer_numbers_4.csv' into table t1 fields terminated by ','; +select * from t1; +col1 col2 col3 col4 col5 col6 col7 col8 +1 234 2147483642 92233720368547 254 65533 4294967294 1844674407370956 +delete from t1; load data infile '$resources/load_data/integer_numbers_1.csv' into table t1 fields terminated by ','; select * from t1; col1 col2 col3 col4 col5 col6 col7 col8 diff --git a/test/distributed/cases/load_data/load_data.sql b/test/distributed/cases/load_data/load_data.sql index 6290dac57ab3..078007b71030 100644 --- a/test/distributed/cases/load_data/load_data.sql +++ b/test/distributed/cases/load_data/load_data.sql @@ -13,6 +13,10 @@ col8 bigint unsigned ); -- load data +load data infile '$resources/load_data/integer_numbers_4.csv' into table t1 fields terminated by ','; +select * from t1; +delete from t1; + load data infile '$resources/load_data/integer_numbers_1.csv' into table t1 fields terminated by ','; select * from t1; diff --git a/test/distributed/cases/optimistic/conflict_transation.result b/test/distributed/cases/optimistic/conflict_transation.result index 0a3773d592c4..05b436aa6a14 100644 --- a/test/distributed/cases/optimistic/conflict_transation.result +++ b/test/distributed/cases/optimistic/conflict_transation.result @@ -76,9 +76,9 @@ create table ct_06(a bigint,b varchar(25),c int, d varchar(25),primary key(a),un start transaction ; insert into ct_06 select 5678,'high',487,'comment test'; insert into ct_06 select 5679,'lower',487,'define'; -Duplicate entry '487' for key '__mo_index_idx_col' +Duplicate entry '487' for key 'c' insert into ct_06 values (897,'number',908,'run tools'),(898,'string',908,'ffff'),(899,'string',918,'while'),(900,'string',948,'word file'),(901,'string',902,'oooo'),(902,'string',87,'dddd'),(903,'string',87,'eeee'); -Duplicate entry '908' for key '__mo_index_idx_col' +Duplicate entry '908' for key 'c' select * from ct_06; a b c d 5678 high 487 comment test @@ -106,7 +106,7 @@ insert into ct_07 values (1,'901','2011-09-29',0.01),(2,'187','2011-09-29',1.31) begin; insert into ct_07 values (3,'90','2111-02-09',10.01); insert into ct_07 values (4,'11','2011-09-29',7.00),(2,'567','2011-09-29',1.31),(4,'90','2011-09-29',89.3); -Duplicate entry '(4,2011-09-29)' for key '__mo_cpkey_col' +Duplicate entry '(4,2011-09-29)' for key '(a,c)' select * from ct_07; a b c d 1 901 2011-09-29 0.01 diff --git a/test/distributed/cases/optimistic/insert.result b/test/distributed/cases/optimistic/insert.result index 3f567bdc735d..e3f519b5bb48 100644 --- a/test/distributed/cases/optimistic/insert.result +++ b/test/distributed/cases/optimistic/insert.result @@ -161,7 +161,7 @@ a b 2 2 2 3 insert into t1 values(2, '3'); -Duplicate entry ('\(\d\,\d\)'|'\d\w\d{12}') for key '__mo_cpkey_col' +Duplicate entry '3a150246013300' for key '__mo_cpkey_col' drop table if exists t1; create table t1(a int, b int, c int); insert into t1 values(1, 2, 3) on duplicate key update a=5; @@ -248,7 +248,7 @@ insert into t1 values("这是一个字节数超过五十的字符串,但是utf drop table if exists t1; create table t1 (col1 bigint primary key,col2 varchar(25),col3 float,col4 varchar(50),unique key num_phone(col2),key num_id(col4)); insert into t1 values (67834,'13456789872',20.23,'4090'),(56473,'13456789872',100.00,'5678'); -Duplicate entry '13456789872' for key '__mo_index_idx_col' +Duplicate entry '13456789872' for key 'col2' drop table if exists t; create table t(a int); insert into t values(1); diff --git a/test/distributed/cases/optimistic/special.result b/test/distributed/cases/optimistic/special.result index 367f9ac5717e..670f6041f2be 100644 --- a/test/distributed/cases/optimistic/special.result +++ b/test/distributed/cases/optimistic/special.result @@ -6,7 +6,7 @@ insert into ct_07 values (1,'901','2011-09-29',0.01),(2,'187','2011-09-29',1.31) begin; insert into ct_07 values (3,'90','2111-02-09',10.01); insert into ct_07 values (4,'11','2011-09-29',7.00),(2,'567','2011-09-29',1.31),(4,'90','2011-09-29',89.3); -(Duplicate entry) '\((\d+),(\d{4}-\d{2}-\d{2})\)' (for key '__mo_cpkey_col') +(Duplicate entry) '\((\d+),(\d{4}-\d{2}-\d{2})\)' (for key '\(a,c\)') select * from ct_07; a b c d 3 90 2111-02-09 10.01 diff --git a/test/distributed/cases/optimistic/unique_secondary_index.result b/test/distributed/cases/optimistic/unique_secondary_index.result index 9e2fc1ccdd63..698d7d1c4c61 100644 --- a/test/distributed/cases/optimistic/unique_secondary_index.result +++ b/test/distributed/cases/optimistic/unique_secondary_index.result @@ -1,7 +1,7 @@ create table index_01 (col1 bigint primary key,col2 varchar(25),col3 float,col4 varchar(50),unique key num_phone(col2),key num_id(col4)); insert into index_01 values (1,'13456789872',20.23,'5678'),(2,'13873458290',100.00,'23'),(3,NULL,100.00,'23'); insert into index_01 values (67834,'13456789872',20.23,'4090'),(56473,'13456789872',100.00,'5678'); -Duplicate entry '13456789872' for key '__mo_index_idx_col' +Duplicate entry '13456789872' for key 'col2' insert into index_01 values (4,'13866666666',20.23,'5678'),(5,'13873458290',100.00,'23'),(6,'13777777777',100.00,'23'); Duplicate entry '13873458290' for key '__mo_index_idx_col' select col2,col4 from index_01; @@ -28,7 +28,7 @@ col1 col2 col3 col4 5 MMEabc$%^123 0.2 create table index_03 (col1 bigint auto_increment primary key,col2 int,col3 float,col4 int,unique key id1(col2),key id2(col4)); insert into index_03(col2,col3,col4) values (10,20.23,4090),(10,100.00,5678); -Duplicate entry '10' for key '__mo_index_idx_col' +Duplicate entry '10' for key 'col2' insert into index_03(col2,col3,col4) values (10,20.23,4090),(11,100.00,4090); insert into index_03(col2,col3,col4) values (67834,20.23,4090),(56473,100.00,5678),(NULL,0.01,NULL); insert into index_03(col2,col3,col4) values (-2147483648,1.2,100),(2147483647,2.0,5); @@ -46,7 +46,7 @@ col1 col2 col3 col4 8 -2147483648 1.2 100 create table index_04 (col1 bigint,col2 int primary key,col3 float,col4 bigint,unique key id1(col1),key id2(col4)); insert into index_04 values (67834,2,20.23,4090),(67834,4,100.00,4091); -Duplicate entry '67834' for key '__mo_index_idx_col' +Duplicate entry '67834' for key 'col1' insert into index_04 values (1,2,20.23,4090),(2,4,100.00,4091),(NULL,3,0.01,NULL); insert into index_04 values (3,2,20.23,4090),(2,4,100.00,4091),(4,4,100.00,4090); (Duplicate entry ')([\d\D]*)(' for key 'col2') @@ -59,7 +59,7 @@ select * from index_04 where col1 between 10 and 1000000; col1 col2 col3 col4 create table index_05 (col1 smallint unique key,col2 int primary key,col3 float,col4 smallint,key id2(col4)); insert into index_05 values (1,2,20.23,4090),(1,4,100.00,4091); -Duplicate entry '1' for key '__mo_index_idx_col' +Duplicate entry '1' for key 'col1' insert into index_05 values (1,2,20.23,4090),(2,4,100.00,4091),(NULL,3,0.01,NULL); select * from index_05; col1 col2 col3 col4 @@ -68,7 +68,7 @@ col1 col2 col3 col4 null 3 0.01 null create table index_06 (col1 tinyint,col2 int primary key,col3 float,col4 tinyint,unique key id1(col1),key id2(col4)); insert into index_06 values (1,2,20.23,56),(1,4,100.00,90); -Duplicate entry '1' for key '__mo_index_idx_col' +Duplicate entry '\d' for key '(.*)' insert into index_06 values (1,2,20.23,56),(2,4,100.00,41),(NULL,3,0.01,NULL); select * from index_06; col1 col2 col3 col4 @@ -86,7 +86,7 @@ col1 col2 col3 col4 null 3 0.01 null create table index_08 (col1 bigint unsigned,col2 int primary key,col3 float,col4 bigint unsigned,unique key id1(col1),key id2(col4)); insert into index_08 values (1,2,20.23,56),(1,4,100.00,90); -Duplicate entry '1' for key '__mo_index_idx_col' +Duplicate entry '1' for key 'col1' insert into index_08 values (1,2,20.23,56),(2,4,100.00,41),(NULL,3,0.01,NULL); select * from index_08; col1 col2 col3 col4 @@ -95,7 +95,7 @@ col1 col2 col3 col4 null 3 0.01 null create table index_09 (col1 bigint primary key,col2 decimal(4,2),col3 decimal(4,2),unique key d1(col2),key d2(col3)); insert into index_09 values (1000,20.23,20.00),(1200,20.23,0.10); -Duplicate entry '20.23' for key '__mo_index_idx_col' +Duplicate entry '20.23' for key 'col2' insert into index_09 values (1000,20.23,20.00),(1200,0.23,20.10),(1100,NULL,NULL); select * from index_09; col1 col2 col3 @@ -104,7 +104,7 @@ col1 col2 col3 1100 null null create table index_10 (col1 bigint primary key,col2 float,col3 float,unique key d1(col2),key d2(col3)); insert into index_10 values (1000,20.23,20.00),(1200,20.23,0.10); -Duplicate entry '20.23' for key '__mo_index_idx_col' +Duplicate entry '20.23' for key 'col2' insert into index_10 values (1000,20.23,20.00),(1200,0.23,20.10),(1100,NULL,NULL); select * from index_10; col1 col2 col3 @@ -113,7 +113,7 @@ col1 col2 col3 1100 null null create table index_11 (col1 bigint primary key,col2 double,col3 double,unique key d1(col2),key d2(col3)); insert into index_11 values (1000,20.23,20.00),(1200,20.23,0.10); -Duplicate entry '20.23' for key '__mo_index_idx_col' +Duplicate entry '20.23' for key 'col2' insert into index_11 values (1000,20.23,20.00),(1200,0.23,20.10),(1100,NULL,NULL); select * from index_11; col1 col2 col3 @@ -122,7 +122,7 @@ col1 col2 col3 1100 null null create table index_12 (col1 bigint auto_increment primary key,col2 date,col3 date,unique key d1(col2),key d2(col3)); insert into index_12(col2,col3) values ('2013-01-01','2014-02-01'),('2013-01-01','2014-02-20'); -Duplicate entry '2013-01-01' for key '__mo_index_idx_col' +Duplicate entry '2013-01-01' for key 'col2' insert into index_12(col2,col3) values (NULL,'2014-02-01'),(NULL,NULL); select col2 from index_12; col2 @@ -130,15 +130,15 @@ null null create table index_13 (col1 bigint auto_increment primary key,col2 datetime,col3 datetime,unique key d1(col2),key d2(col3)); insert into index_13(col2,col3) values ('2013-01-01 12:00:00','2014-02-01 10:00:00'),('2013-01-01 12:00:00','2014-02-20 05:00:00'); -Duplicate entry '2013-01-01 12:00:00' for key '__mo_index_idx_col' +Duplicate entry '2013-01-01 12:00:00' for key 'col2' insert into index_13(col2,col3) values (NULL,'2014-02-01 12:00:0'),(NULL,NULL); create table index_14 (col1 bigint auto_increment primary key,col2 timestamp,col3 timestamp,unique key d1(col2),key d2(col3)); insert into index_14(col2,col3) values ('2013-01-01 12:00:00','2014-02-01 10:00:00'),('2013-01-01 12:00:00','2014-02-20 05:00:00'); -Duplicate entry '2013-01-01 12:00:00' for key '__mo_index_idx_col' +Duplicate entry '2013-01-01 12:00:00' for key 'col2' insert into index_14(col2,col3) values (NULL,'2014-02-01 12:00:0'),(NULL,NULL); create table index_15 (col1 bigint primary key,col2 bool,unique key c2(col2)); insert into index_15 values (1,TRUE),(2,FALSE),(3,TRUE); -(Duplicate entry ')([\d\D][true|false]*)(' for key '__mo_index_idx_col') +(Duplicate entry ')([\d\D][true|false]*)(' for key 'col2') insert into index_15 values (1,TRUE),(2,FALSE),(3,NULL); select * from index_15; col1 col2 @@ -181,7 +181,7 @@ drop table index_table_02; create table index_table_02 (col1 bigint primary key,col2 int default 0,unique key col2(col2)); insert into index_table_02(col1) values (1); insert into index_table_02(col1) values (2),(3); -Duplicate entry '0' for key '__mo_index_idx_col' +Duplicate entry '0' for key 'col2' drop table index_table_02; create table index_table_03 (col1 bigint,col2 int,unique key col2(col2)); insert into index_table_03 values (1,20),(2,NULL),(3,90); @@ -233,7 +233,7 @@ code_id code_type code_no code_rule_no org_no 1009 39 8744 0000243 yy10 truncate table t_code_rule; load data infile '$resources/load_data/unique_index_duplicate.csv' into table t_code_rule fields terminated by ','; -Duplicate entry '30' for key '__mo_index_idx_col' +Duplicate entry '30' for key 'code_type' select code_id,code_type,code_no,code_rule_no,org_no from t_code_rule; code_id code_type code_no code_rule_no org_no create table index_temp( col1 bigint(20) NOT NULL ,col2 varchar(50) NOT NULL,col3 varchar(50) NOT NULL,col4 varchar(50) NOT NULL,col5 varchar(255) NOT NULL,col6 varchar(50) NOT NULL,col7 varchar(50) NOT NULL,col8 varchar(50) NOT NULL,col9 int(11) ,col10 varchar(50) DEFAULT NULL,col11 varchar(255),col12 datetime NOT NULL,col13 varchar(50) DEFAULT NULL,col14 datetime DEFAULT NULL,col15 varchar(50) DEFAULT NULL,col16 varchar(20) NOT NULL DEFAULT 'N'); @@ -305,7 +305,7 @@ col1 col2 col3 col4 2 chart 2 11 truncate table index_table_05; insert into index_table_05(col2,col3,col4) values ('apple',1,'10'),('store',2,'11'),('apple',3,'12'); -Duplicate entry 'apple' for key '__mo_index_idx_col' +Duplicate entry 'apple' for key 'col2' select * from index_table_05; col1 col2 col3 col4 drop table index_table_05; @@ -341,7 +341,7 @@ create table create_index_01 (col1 bigint primary key,col2 varchar(25),col3 floa create unique index varchar_index on create_index_01(col2) comment 'create varchar index'; insert into create_index_01 values (1,'13456789872',20.23,'5678'),(2,'13873458290',100.00,'23'),(3,NULL,100.00,'23'); insert into create_index_01 values (67834,'13456789872',20.23,'4090'),(56473,'13456789872',100.00,'5678'); -Duplicate entry '13456789872' for key '__mo_index_idx_col' +Duplicate entry '13456789872' for key 'col2' select col2,col4 from create_index_01; col2 col4 13456789872 5678 @@ -372,7 +372,7 @@ drop index char_index on create_index_02; create table create_index_03 (col1 bigint auto_increment,col2 int,col3 float,col4 int); create unique index int_index on create_index_03(col2); insert into create_index_03(col2,col3,col4) values (10,20.23,4090),(10,100.00,5678); -Duplicate entry '10' for key '__mo_index_idx_col' +Duplicate entry '10' for key 'col2' insert into create_index_03(col2,col3,col4) values (10,20.23,4090),(11,100.00,4090); insert into create_index_03(col2,col3,col4) values (67834,20.23,4090),(56473,100.00,5678),(NULL,0.01,NULL); select * from create_index_03; @@ -386,7 +386,7 @@ drop index int_index on create_index_03; create table create_index_04 (col1 bigint,col2 int primary key,col3 float,col4 bigint); create unique index bigint_index on create_index_04(col1); insert into create_index_04 values (67834,2,20.23,4090),(67834,4,100.00,4091); -Duplicate entry '67834' for key '__mo_index_idx_col' +Duplicate entry '67834' for key 'col1' insert into create_index_04 values (1,2,20.23,4090),(2,4,100.00,4091),(NULL,3,0.01,NULL); insert into create_index_04 values (-9223372036854775808,5,20.23,4090),(9223372036854775807,6,100.00,4091); select * from create_index_04; @@ -403,7 +403,7 @@ col1 col2 col3 col4 create table create_index_05 (col1 smallint,col2 int primary key,col3 float,col4 smallint); create unique index smallint_index on create_index_05(col1); insert into create_index_05 values (1,2,20.23,4090),(1,4,100.00,4091); -Duplicate entry '1' for key '__mo_index_idx_col' +Duplicate entry '1' for key 'col1' insert into create_index_05 values (1,2,20.23,4090),(2,4,100.00,4091),(NULL,3,0.01,NULL); select * from create_index_05; col1 col2 col3 col4 @@ -413,7 +413,7 @@ null 3 0.01 null create table create_index_06 (col1 tinyint,col2 int primary key,col3 float,col4 tinyint); create unique index tinyint_index on create_index_06(col1); insert into create_index_06 values (1,2,20.23,56),(1,4,100.00,90); -Duplicate entry '1' for key '__mo_index_idx_col' +Duplicate entry '\d' for key '(.*)' insert into create_index_06 values (1,2,20.23,56),(2,4,100.00,41),(NULL,3,0.01,NULL); select * from create_index_06; col1 col2 col3 col4 @@ -424,7 +424,7 @@ create table create_index_07 (col1 int unsigned,col2 int primary key,col3 float, create unique index int_unsigned_index on create_index_07(col1); insert into create_index_07 values (1,2,20.23,56),(2,4,100.00,41),(NULL,3,0.01,NULL); insert into create_index_07 values (1,2,20.23,56),(1,4,100.00,90); -(Duplicate entry ')([\d\D]*)(' for key ')([col2|__mo_index_idx_col]*)(') +Duplicate entry '\d' for key '(.*)' select * from create_index_07; col1 col2 col3 col4 1 2 20.23 56 @@ -433,7 +433,7 @@ null 3 0.01 null create table create_index_08 (col1 bigint unsigned,col2 int primary key,col3 float,col4 bigint unsigned); create unique index bigint_unsigned_index on create_index_08(col1); insert into create_index_08 values (1,2,20.23,56),(1,4,100.00,90); -Duplicate entry '1' for key '__mo_index_idx_col' +Duplicate entry '\d' for key '(.*)' insert into create_index_08 values (1,2,20.23,56),(2,4,100.00,41),(NULL,3,0.01,NULL); select * from create_index_08; col1 col2 col3 col4 @@ -443,7 +443,7 @@ null 3 0.01 null create table create_index_09 (col1 bigint primary key,col2 decimal(16,8),col3 decimal(16,8)); create unique index decimal_index on create_index_09(col2); insert into create_index_09 values (1000,20.23,20.00),(1200,20.23,0.10); -Duplicate entry '20.23000000' for key '__mo_index_idx_col' +Duplicate entry '20.23000000' for key 'col2' insert into create_index_09 values (1000,20.23,20.00),(1200,0.23,20.10),(1100,NULL,NULL); select * from create_index_09; col1 col2 col3 @@ -453,7 +453,7 @@ col1 col2 col3 create table create_index_10 (col1 bigint primary key,col2 float,col3 float); create unique index float_index on create_index_10(col2); insert into create_index_10 values (1000,20.23,20.00),(1200,20.23,0.10); -Duplicate entry '20.23' for key '__mo_index_idx_col' +Duplicate entry '20.23' for key 'col2' insert into create_index_10 values (1000,20.23,20.00),(1200,0.23,20.10),(1100,NULL,NULL); select * from create_index_10; col1 col2 col3 @@ -463,7 +463,7 @@ col1 col2 col3 create table create_index_11 (col1 bigint primary key,col2 double,col3 double); create unique index double_index on create_index_11(col2); insert into create_index_11 values (1000,20.23,20.00),(1200,20.23,0.10); -Duplicate entry '20.23' for key '__mo_index_idx_col' +Duplicate entry '20.23' for key 'col2' insert into create_index_11 values (1000,20.23,20.00),(1200,0.23,20.10),(1100,NULL,NULL); select * from create_index_11; col1 col2 col3 @@ -473,22 +473,22 @@ col1 col2 col3 create table create_index_12(col1 bigint auto_increment primary key,col2 date,col3 date); create unique index date_index on create_index_12(col2); insert into create_index_12(col2,col3) values ('2013-01-01','2014-02-01'),('2013-01-01','2014-02-20'); -Duplicate entry '2013-01-01' for key '__mo_index_idx_col' +Duplicate entry '2013-01-01' for key 'col2' insert into create_index_12(col2,col3) values (NULL,'2014-02-01'),(NULL,NULL); create table create_index_13 (col1 bigint auto_increment primary key,col2 datetime,col3 datetime); create unique index datetime_index on create_index_13(col2); insert into create_index_13(col2,col3) values ('2013-01-01 12:00:00','2014-02-01 10:00:00'),('2013-01-01 12:00:00','2014-02-20 05:00:00'); -Duplicate entry '2013-01-01 12:00:00' for key '__mo_index_idx_col' +Duplicate entry '2013-01-01 12:00:00' for key 'col2' insert into create_index_13(col2,col3) values (NULL,'2014-02-01 12:00:0'),(NULL,NULL); create table create_index_14 (col1 bigint auto_increment primary key,col2 timestamp,col3 timestamp); create unique index timestamp_index on create_index_14(col2); insert into create_index_14(col2,col3) values ('2013-01-01 12:00:00','2014-02-01 10:00:00'),('2013-01-01 12:00:00','2014-02-20 05:00:00'); -Duplicate entry '2013-01-01 12:00:00' for key '__mo_index_idx_col' +Duplicate entry '2013-01-01 12:00:00' for key 'col2' insert into create_index_14(col2,col3) values (NULL,'2014-02-01 12:00:0'),(NULL,NULL); create table create_index_15 (col1 bigint primary key,col2 bool); create unique index bool_index on create_index_15(col2); insert into create_index_15 values (1,TRUE),(2,FALSE),(3,TRUE); -(Duplicate entry ')([\d\D][true|false]*)(' for key '__mo_index_idx_col') +(Duplicate entry ')([\d\D][true|false]*)(' for key 'col2') insert into create_index_15 values (1,TRUE),(2,FALSE),(3,NULL); select * from create_index_15; col1 col2 @@ -603,7 +603,7 @@ col1 col2 col3 col4 2 chart 2 11 truncate table create_index_19; insert into create_index_19(col2,col3,col4) values ('apple',1,'10'),('store',2,'11'),('apple',3,'12'); -Duplicate entry 'apple' for key '__mo_index_idx_col' +Duplicate entry 'apple' for key 'col2' select * from create_index_19; col1 col2 col3 col4 drop table create_index_19; @@ -664,7 +664,7 @@ use unique_secondary_index; drop index varchar_index on trans_index_01; insert into trans_index_01 values (1,'13456789872',20.23,'5678'),(2,'13456789872',100.00,'23'),(3,NULL,100.00,'23'); insert into trans_index_01 values (1,'13456789872',20.23,'5678'),(2,'13456789872',100.00,'23'),(3,NULL,100.00,'23'); -Duplicate entry '13456789872' for key '__mo_index_idx_col' +Duplicate entry '13456789872' for key 'col2' select * from trans_index_01; col1 col2 col3 col4 commit; diff --git a/test/distributed/cases/optimistic/unique_secondary_index.sql b/test/distributed/cases/optimistic/unique_secondary_index.sql index 86d505ee221a..db877a4b193f 100644 --- a/test/distributed/cases/optimistic/unique_secondary_index.sql +++ b/test/distributed/cases/optimistic/unique_secondary_index.sql @@ -30,6 +30,7 @@ insert into index_05 values (1,2,20.23,4090),(1,4,100.00,4091); insert into index_05 values (1,2,20.23,4090),(2,4,100.00,4091),(NULL,3,0.01,NULL); select * from index_05; create table index_06 (col1 tinyint,col2 int primary key,col3 float,col4 tinyint,unique key id1(col1),key id2(col4)); +-- @pattern insert into index_06 values (1,2,20.23,56),(1,4,100.00,90); insert into index_06 values (1,2,20.23,56),(2,4,100.00,41),(NULL,3,0.01,NULL); select * from index_06; @@ -240,6 +241,7 @@ insert into create_index_05 values (1,2,20.23,4090),(2,4,100.00,4091),(NULL,3,0. select * from create_index_05; create table create_index_06 (col1 tinyint,col2 int primary key,col3 float,col4 tinyint); create unique index tinyint_index on create_index_06(col1); +-- @pattern insert into create_index_06 values (1,2,20.23,56),(1,4,100.00,90); insert into create_index_06 values (1,2,20.23,56),(2,4,100.00,41),(NULL,3,0.01,NULL); select * from create_index_06; @@ -251,6 +253,7 @@ insert into create_index_07 values (1,2,20.23,56),(1,4,100.00,90); select * from create_index_07; create table create_index_08 (col1 bigint unsigned,col2 int primary key,col3 float,col4 bigint unsigned); create unique index bigint_unsigned_index on create_index_08(col1); +-- @pattern insert into create_index_08 values (1,2,20.23,56),(1,4,100.00,90); insert into create_index_08 values (1,2,20.23,56),(2,4,100.00,41),(NULL,3,0.01,NULL); select * from create_index_08; diff --git a/test/distributed/cases/optimizer/insert.result b/test/distributed/cases/optimizer/insert.result index 0bfff4b1393a..4064b6db465d 100644 --- a/test/distributed/cases/optimizer/insert.result +++ b/test/distributed/cases/optimizer/insert.result @@ -1,23 +1,23 @@ create table t1(a int, b int unique key); explain insert into t1(a) values (1); -QUERY PLAN -Insert on insert.t1 - -> PreInsert on insert.t1 - -> Project +TP QUERY PLAN +Multi Update + -> Project + -> PreInsert on insert.t1 -> Project -> Values Scan "*VALUES*" create table t2(a int, b int, c int, unique key(b, c)); explain insert into t2(a) values (1); -QUERY PLAN -Insert on insert.t2 - -> PreInsert on insert.t2 - -> Project +TP QUERY PLAN +Multi Update + -> Project + -> PreInsert on insert.t2 -> Project -> Values Scan "*VALUES*" create table t3(a int auto_increment, b int, unique key(a)); insert into t3(b) values (1); insert into t3(a) values (1); -Duplicate entry '1' for key '__mo_index_idx_col' +Duplicate entry '1' for key '(.*)' create table t4(a int auto_increment, b int auto_increment, unique key(a, b)); insert into t4(b) values (1); insert into t4(a) values (2); diff --git a/test/distributed/cases/optimizer/insert.test b/test/distributed/cases/optimizer/insert.test index 383d9b9fddd0..40f64cf32604 100644 --- a/test/distributed/cases/optimizer/insert.test +++ b/test/distributed/cases/optimizer/insert.test @@ -6,99 +6,8 @@ create table t2(a int, b int, c int, unique key(b, c)); explain insert into t2(a) values (1); create table t3(a int auto_increment, b int, unique key(a)); insert into t3(b) values (1); ---+--------------------------------------------------------------------+ ---| QUERY PLAN | ---+--------------------------------------------------------------------+ ---| Plan 0: | ---| Sink | ---| -> PreInsert on t.t3 | ---| -> Project | ---| -> Project | ---| -> Values Scan "*VALUES*" | ---| Plan 1: | ---| Insert on t.__mo_index_unique_018e35ac-0098-72aa-ba7c-081ab2e7d49e | should not contain plan that do dup check ---| -> Lock | ---| -> PreInsert UniqueKey | ---| -> Sink Scan | ---| DataSource: Plan 0 | ---| Plan 2: | ---| Insert on t.t3 | ---| -> Sink Scan | ---| DataSource: Plan 0 | ---+--------------------------------------------------------------------+ +-- @pattern insert into t3(a) values (1); ---+------------------------------------------------------------------------------+ ---| QUERY PLAN | ---+------------------------------------------------------------------------------+ ---| Plan 0: | ---| Sink | ---| -> PreInsert on t.t3 | ---| -> Project | ---| -> Project | ---| -> Values Scan "*VALUES*" | ---| Plan 1: | ---| Sink | ---| -> Lock | ---| -> PreInsert UniqueKey | ---| -> Sink Scan | ---| DataSource: Plan 0 | ---| Plan 2: | ---| Insert on t.__mo_index_unique_018e35ae-c21b-790f-8d5c-938bd2f430a3 | ---| -> Sink Scan | ---| DataSource: Plan 1 | ---| Plan 3: | <- should contain plan that do dup check ---| Fuzzy Filter for duplicate key | ---| -> Table Scan on t.__mo_index_unique_018e35ae-c21b-790f-8d5c-938bd2f430a3 | ---| Filter Cond: (__mo_index_idx_col = 1) | ---| Block Filter Cond: (__mo_index_idx_col = 1) | ---| -> Sink Scan | ---| DataSource: Plan 1 | ---| Plan 4: | ---| Insert on t.t3 | ---| -> Sink Scan | ---| DataSource: Plan 0 | ---+------------------------------------------------------------------------------+ create table t4(a int auto_increment, b int auto_increment, unique key(a, b)); insert into t4(b) values (1); ---mysql> explain insert into t4(b) values (1); ---+--------------------------------------------------------------------+ ---| QUERY PLAN | ---+--------------------------------------------------------------------+ ---| Plan 0: | ---| Sink | ---| -> PreInsert on t.t4 | ---| -> Project | ---| -> Project | ---| -> Values Scan "*VALUES*" | ---| Plan 1: | ---| Insert on t.__mo_index_unique_018e35aa-b3cd-7335-8b14-0427179f7e2d | ---| -> Lock | ---| -> PreInsert UniqueKey | ---| -> Sink Scan | ---| DataSource: Plan 0 | ---| Plan 2: | should not contain plan that do dup check ---| Insert on t.t4 | ---| -> Sink Scan | ---| DataSource: Plan 0 | ---+--------------------------------------------------------------------+ insert into t4(a) values (2); ---+--------------------------------------------------------------------+ ---| QUERY PLAN | ---+--------------------------------------------------------------------+ ---| Plan 0: | ---| Sink | ---| -> PreInsert on t.t4 | ---| -> Project | ---| -> Project | ---| -> Values Scan "*VALUES*" | ---| Plan 1: | ---| Insert on t.__mo_index_unique_018e35aa-b3cd-7335-8b14-0427179f7e2d | should not contain plan that do dup check ---| -> Lock | ---| -> PreInsert UniqueKey | ---| -> Sink Scan | ---| DataSource: Plan 0 | ---| Plan 2: | ---| Insert on t.t4 | ---| -> Sink Scan | ---| DataSource: Plan 0 | ---+--------------------------------------------------------------------+ diff --git a/test/distributed/cases/optimizer/joinorder.test b/test/distributed/cases/optimizer/joinorder.test index 7974b2929dcd..818acda2f6d7 100644 --- a/test/distributed/cases/optimizer/joinorder.test +++ b/test/distributed/cases/optimizer/joinorder.test @@ -22,11 +22,15 @@ select count(*) from t1,t2,t3 where t1.c1=t2.c1 and t2.c3=t3.c3 and t3.c2=1; -- @separator:table explain select * from t1,t2,t3 where t1.c1=t2.c1 and t1.c1=t3.c1 and t3.c2<10; select count(*) from t1,t2,t3 where t1.c1=t2.c1 and t1.c1=t3.c1 and t3.c2<10; +-- @bvt:issue#19733 -- @separator:table explain select * from t1,t2,t3 where t1.c1=t2.c1 and t1.c2=t3.c2 and t3.c3<10; +-- @bvt:issue select count(*) from t1,t2,t3 where t1.c1=t2.c1 and t1.c2=t3.c2 and t3.c3<10; +-- @bvt:issue#19733 -- @separator:table explain select * from t1,t2,t3 where t1.c1=t2.c1 and t2.c2=t3.c2 and t3.c3<10; +-- @bvt:issue select count(*) from t1,t2,t3 where t1.c1=t2.c1 and t2.c2=t3.c2 and t3.c3<10; -- @separator:table explain select * from t1,t2,t3 where t1.c1 Join @@ -107,8 +107,8 @@ Project -> Table Scan on d1.t1 Runtime Filter Probe: t1.__mo_cpkey_col Match Prefix -> Table Scan on d1.t2 - Filter Cond: (t2.c3 < 10000) - Block Filter Cond: (t2.c3 < 10000) + Filter Cond: (t2.c3 < 1000) + Block Filter Cond: (t2.c3 < 1000) select count(*) from t1,t2 where t1.c1=t2.c2 and t2.c3<10000; count(*) 9999 diff --git a/test/distributed/cases/optimizer/runtimefilter.test b/test/distributed/cases/optimizer/runtimefilter.test index d16e03d231c9..6d2f942b145a 100644 --- a/test/distributed/cases/optimizer/runtimefilter.test +++ b/test/distributed/cases/optimizer/runtimefilter.test @@ -38,7 +38,7 @@ select mo_ctl('dn', 'flush', 'd1.t1'); select mo_ctl('dn', 'flush', 'd1.t2'); select Sleep(1); -- @separator:table -explain select * from t1,t2 where t1.c1=t2.c2 and t2.c3<10000; +explain select * from t1,t2 where t1.c1=t2.c2 and t2.c3<1000; select count(*) from t1,t2 where t1.c1=t2.c2 and t2.c3<10000; -- @separator:table explain select * from t1,t2 where t1.c2=t2.c1 and t2.c3<5; diff --git a/test/distributed/cases/optimizer/shuffle.result b/test/distributed/cases/optimizer/shuffle.result index ed43937348d0..9721209e4f04 100644 --- a/test/distributed/cases/optimizer/shuffle.result +++ b/test/distributed/cases/optimizer/shuffle.result @@ -238,6 +238,9 @@ count(*) 10000000 create table t3(c1 int not null, c2 int not null)cluster by c1; insert into t3 select *,* from generate_series(1,1000000)g; +select mo_ctl('dn', 'flush', 'd1.t3'); +mo_ctl(dn, flush, d1.t3) +{\n "method": "Flush",\n "result": [\n {\n "returnStr": "OK"\n }\n ]\n}\n explain select count(*) from t3 where t3.c2 in (select c3 from t1 where t1.c2!=20000 and t1.c2 not in ( select c2 from t2 where t2.c3>150000 )); AP QUERY PLAN ON MULTICN(4 core) Project diff --git a/test/distributed/cases/optimizer/shuffle.test b/test/distributed/cases/optimizer/shuffle.test index 3e63f0fb0c70..eb6053dd6005 100644 --- a/test/distributed/cases/optimizer/shuffle.test +++ b/test/distributed/cases/optimizer/shuffle.test @@ -27,11 +27,15 @@ select count(*) from t1,t2 where t1.c2=t2.c1; -- @separator:table explain select count(*) from t1,t2 where t1.c2=t2.c2; select count(*) from t1,t2 where t1.c2=t2.c2; +-- @bvt:issue#19733 -- @separator:table explain select count(*) from t1,t2 where t1.c2=t2.c2 and t2.c3<500000; +-- @bvt:issue select count(*) from t1,t2 where t1.c2=t2.c2 and t2.c3<500000; +-- @bvt:issue#19733 -- @separator:table explain select count(*) from t1,t2 where t1.c2=t2.c2 and t2.c3<5000000; +-- @bvt:issue select count(*) from t1,t2 where t1.c2=t2.c2 and t2.c3<5000000; -- @separator:table explain select count(*) from t1 group by c1 limit 5; @@ -39,20 +43,30 @@ select count(*) from t1 group by c1 limit 5; -- @separator:table explain select count(*) from t1 group by c2 limit 5; select count(*) from t1 group by c2 limit 5; +-- @bvt:issue#19733 -- @separator:table explain select count(*) from t1 where t1.c2 in ( select c2 from t2 where t2.c3>100000 ); +-- @bvt:issue select count(*) from t1 where t1.c2 in ( select c2 from t2 where t2.c3>100000 ); +-- @bvt:issue#19733 -- @separator:table explain select count(*) from t1 where t1.c2 not in ( select c3 from t2 where t2.c3 between 100 and 7000000 ); +-- @bvt:issue select count(*) from t1 where t1.c2 not in ( select c3 from t2 where t2.c3 between 100 and 7000000 ); +-- @bvt:issue#19733 -- @separator:table explain select count(*) from t1 where t1.c3<800000 and t1.c2 not in ( select c3 from t2 where t2.c3 between 10000 and 6000000 ); +-- @bvt:issue select count(*) from t1 where t1.c3<800000 and t1.c2 not in ( select c3 from t2 where t2.c3 between 10000 and 6000000 ); +-- @bvt:issue#19733 -- @separator:table explain select count(*) from t1 where t1.c1 <300000 and t1.c2 in ( select c2 from t2 where t2.c3>100000 ); +-- @bvt:issue select count(*) from t1 where t1.c1 <300000 and t1.c2 in ( select c2 from t2 where t2.c3>100000 ); +-- @bvt:issue#19733 -- @separator:table explain select count(*) from t1 left join t2 on t1.c1=t2.c1 where t1.c3 >5000000; +-- @bvt:issue select count(*) from t1 left join t2 on t1.c1=t2.c1 where t1.c3 >5000000; -- @separator:table explain select count(*) from t1 left join t2 on t1.c1=t2.c1 and t1.c3 >t2.c3; @@ -60,12 +74,18 @@ select count(*) from t1 left join t2 on t1.c1=t2.c1 and t1.c3 >t2.c3; create table t3(c1 int not null, c2 int not null)cluster by c1; insert into t3 select *,* from generate_series(1,1000000)g; -- @separator:table +select mo_ctl('dn', 'flush', 'd1.t3'); +-- @bvt:issue#19733 +-- @separator:table explain select count(*) from t3 where t3.c2 in (select c3 from t1 where t1.c2!=20000 and t1.c2 not in ( select c2 from t2 where t2.c3>150000 )); +-- @bvt:issue select count(*) from t3 where t3.c2 in (select c3 from t1 where t1.c2!=20000 and t1.c2 not in ( select c2 from t2 where t2.c3>150000 )); select count(*) from t3 where t3.c1<100000 and t3.c2 not in (select c3 from t1 where t1.c2!=30000 and t1.c2 in ( select c2 from t2 where t2.c3<850000 )); select count(*) from t1,t2,t3 where t1.c1=t2.c1 and t1.c2=t3.c2 and t2.c2<900000 and t3.c1<500000; +-- @bvt:issue#19733 -- @separator:table explain select count(*) from (select c1 from t1 group by c1) s1, t2 where s1.c1=t2.c1 and t2.c2<1000000; +-- @bvt:issue select count(*) from (select c1 from t1 group by c1) s1, t2 where s1.c1=t2.c1 and t2.c2<1000000; drop table t3; drop database if exists d1; \ No newline at end of file diff --git a/test/distributed/cases/pessimistic_transaction/conflict_transaction.result b/test/distributed/cases/pessimistic_transaction/conflict_transaction.result index 08b13d11b20e..445fcaf60bac 100644 --- a/test/distributed/cases/pessimistic_transaction/conflict_transaction.result +++ b/test/distributed/cases/pessimistic_transaction/conflict_transaction.result @@ -73,9 +73,9 @@ create table ct_06(a bigint,b varchar(25),c int, d varchar(25),primary key(a),un start transaction ; insert into ct_06 select 5678,'high',487,'comment test'; insert into ct_06 select 5679,'lower',487,'define'; -Duplicate entry '487' for key '__mo_index_idx_col' +Duplicate entry '487' for key 'c' insert into ct_06 values (897,'number',908,'run tools'),(898,'string',908,'ffff'),(899,'string',918,'while'),(900,'string',948,'word file'),(901,'string',902,'oooo'),(902,'string',87,'dddd'),(903,'string',87,'eeee'); -Duplicate entry '908' for key '__mo_index_idx_col' +Duplicate entry '908' for key 'c' select * from ct_06; a b c d 5678 high 487 comment test @@ -87,7 +87,7 @@ begin; insert into ct_06 values (500,'number1',908,'ooooffff'); start transaction ; insert into ct_06 values (501,'number2',908,'zzzztttt'); -Duplicate entry '908' for key '__mo_index_idx_col' +Duplicate entry '908' for key 'c' commit; select * from ct_06; a b c d @@ -102,9 +102,9 @@ create table ct_07(a int,b varchar(25),c date, d double,primary key(a,c)); insert into ct_07 values (1,'901','2011-09-29',0.01),(2,'187','2011-09-29',1.31),(3,'90','2111-02-09',10.01); begin; insert into ct_07 values (3,'90','2111-02-09',10.01); -Duplicate entry '(3,2111-02-09)' for key '__mo_cpkey_col' +Duplicate entry '(3,2111-02-09)' for key '(a,c)' insert into ct_07 values (4,'11','2011-09-29',7.00),(2,'567','2011-09-29',1.31),(4,'90','2011-09-29',89.3); -(Duplicate entry) '\((\d+),(\d{4}-\d{2}-\d{2})\)' (for key '__mo_cpkey_col') +Duplicate entry '(4,2011-09-29)' for key '(a,c)' select * from ct_07; a b c d 1 901 2011-09-29 0.01 @@ -115,4 +115,4 @@ select * from ct_07; a b c d 1 901 2011-09-29 0.01 2 187 2011-09-29 1.31 -3 90 2111-02-09 10.01 \ No newline at end of file +3 90 2111-02-09 10.01 diff --git a/test/distributed/cases/pessimistic_transaction/conflict_transaction.sql b/test/distributed/cases/pessimistic_transaction/conflict_transaction.sql index 9e1d978217be..8f110fb988a3 100644 --- a/test/distributed/cases/pessimistic_transaction/conflict_transaction.sql +++ b/test/distributed/cases/pessimistic_transaction/conflict_transaction.sql @@ -98,7 +98,6 @@ create table ct_07(a int,b varchar(25),c date, d double,primary key(a,c)); insert into ct_07 values (1,'901','2011-09-29',0.01),(2,'187','2011-09-29',1.31),(3,'90','2111-02-09',10.01); begin; insert into ct_07 values (3,'90','2111-02-09',10.01); --- @pattern insert into ct_07 values (4,'11','2011-09-29',7.00),(2,'567','2011-09-29',1.31),(4,'90','2011-09-29',89.3); select * from ct_07; commit; diff --git a/test/distributed/cases/pessimistic_transaction/dedupjoin.result b/test/distributed/cases/pessimistic_transaction/dedupjoin.result new file mode 100644 index 000000000000..fffefa8361c4 --- /dev/null +++ b/test/distributed/cases/pessimistic_transaction/dedupjoin.result @@ -0,0 +1,30 @@ +drop table if exists t; +create table t (a int, b int); +insert into t select *, * from generate_series(1000) tf; +select t2.a from t as t1 dedup join t as t2 on t1.a = t2.a; +Duplicate entry '1' for key '' +select t2.a from t as t1 dedup join t as t2 on t1.a + 100 = t2.a; +Duplicate entry '101' for key '' +select t2.a from t as t1 dedup join t as t2 on t1.a + 100 = t2.a where t2.a < 10; +a +1 +2 +3 +4 +5 +6 +7 +8 +9 +insert into t select *, * from generate_series(10, 100) tf; +select t2.a from t as t1 dedup join t as t2 on t1.a = t2.a; +Duplicate entry '10' for key '' +drop table t; +create table t1(a int primary key); +insert into t1 select result from generate_series(1000000) tf; +select count(*) from t1 dedup join (select cast(result+999999 as int) result from generate_series(1000000) tmp) tf on a = result; +Duplicate entry '1000000' for key '' +select count(*) from t1 dedup join (select cast(result+1000000 as int) result from generate_series(1000000) tmp) tf on a = result; +count(*) +1000000 +drop table t1; diff --git a/test/distributed/cases/pessimistic_transaction/dedupjoin.test b/test/distributed/cases/pessimistic_transaction/dedupjoin.test new file mode 100644 index 000000000000..cbc4083149c2 --- /dev/null +++ b/test/distributed/cases/pessimistic_transaction/dedupjoin.test @@ -0,0 +1,15 @@ +drop table if exists t; +create table t (a int, b int); +insert into t select *, * from generate_series(1000) tf; +select t2.a from t as t1 dedup join t as t2 on t1.a = t2.a; +select t2.a from t as t1 dedup join t as t2 on t1.a + 100 = t2.a; +select t2.a from t as t1 dedup join t as t2 on t1.a + 100 = t2.a where t2.a < 10; +insert into t select *, * from generate_series(10, 100) tf; +select t2.a from t as t1 dedup join t as t2 on t1.a = t2.a; +drop table t; + +create table t1(a int primary key); +insert into t1 select result from generate_series(1000000) tf; +select count(*) from t1 dedup join (select cast(result+999999 as int) result from generate_series(1000000) tmp) tf on a = result; +select count(*) from t1 dedup join (select cast(result+1000000 as int) result from generate_series(1000000) tmp) tf on a = result; +drop table t1; \ No newline at end of file diff --git a/test/distributed/cases/pessimistic_transaction/insert.result b/test/distributed/cases/pessimistic_transaction/insert.result index fd450b75e6b7..286647acab08 100644 --- a/test/distributed/cases/pessimistic_transaction/insert.result +++ b/test/distributed/cases/pessimistic_transaction/insert.result @@ -161,7 +161,7 @@ a b 2 2 2 3 insert into t1 values(2, '3'); -Duplicate entry '(2,3)' for key '__mo_cpkey_col' +Duplicate entry '(2,3)' for key '(a,b)' drop table if exists t1; create table t1(a int, b int, c int); insert into t1 values(1, 2, 3) on duplicate key update a=5; @@ -248,7 +248,7 @@ insert into t1 values("这是一个字节数超过五十的字符串,但是utf drop table if exists t1; create table t1 (col1 bigint primary key,col2 varchar(25),col3 float,col4 varchar(50),unique key num_phone(col2),key num_id(col4)); insert into t1 values (67834,'13456789872',20.23,'4090'),(56473,'13456789872',100.00,'5678'); -Duplicate entry '13456789872' for key '__mo_index_idx_col' +Duplicate entry '13456789872' for key 'col2' drop table if exists t; create table t(a int); insert into t values(1); @@ -503,16 +503,16 @@ drop table if exists indup_11; create table indup_11(a int, b varchar(20), primary key(a, b)); insert into indup_11(a, b) values (1, "2"); insert into indup_11(b, a) values ("2", 1); -Duplicate entry '(1,2)' for key '__mo_cpkey_col' +Duplicate entry '(1,2)' for key '(a,b)' drop table indup_11; drop table if exists t4; create table t4(col1 int, col2 decimal, col3 smallint, primary key(col1, col3)); insert into t4 (col1, col2, col3) values(6, 6, 8); insert into t4 (col2, col3, col1) values(2, 3, 1); insert into t4 (col3, col1) values (8, 6); -Duplicate entry '(6,8)' for key '__mo_cpkey_col' +Duplicate entry '(6,8)' for key '(col1,col3)' select * from t4; col1 col2 col3 6 6 8 1 2 3 -drop table t4; \ No newline at end of file +drop table t4; diff --git a/test/distributed/cases/pessimistic_transaction/isolation_2.result b/test/distributed/cases/pessimistic_transaction/isolation_2.result index 626b7100bb5d..3a9e7d7f0ec3 100644 --- a/test/distributed/cases/pessimistic_transaction/isolation_2.result +++ b/test/distributed/cases/pessimistic_transaction/isolation_2.result @@ -586,7 +586,7 @@ begin; insert into dis_table_07 values (2,'blue',10.00,'2021-01-20'); use isolation_2; insert into dis_table_07 values (2,'blue',11.00,'2021-01-20'); -Duplicate entry '(2,blue,2021-01-20 00:00:00)' for key '__mo_cpkey_col' +Duplicate entry '(2,blue,2021-01-20 00:00:00)' for key '(a,b,d)' select * from dis_table_07; a b c d 1 yellow 20.09 2020-09-27 00:00:00 diff --git a/test/distributed/cases/pessimistic_transaction/select_for_update.result b/test/distributed/cases/pessimistic_transaction/select_for_update.result index cbd00aa8f0d2..7097142fa0ea 100644 --- a/test/distributed/cases/pessimistic_transaction/select_for_update.result +++ b/test/distributed/cases/pessimistic_transaction/select_for_update.result @@ -714,7 +714,7 @@ select * from su_04 where c2='results' for update; c1 c2 c3 1 results 20 insert into su_04 values(10,'tell',96); -Duplicate entry '10' for key '__mo_index_idx_col' +Duplicate entry '10' for key 'c1' update su_04 set c2='wed'; delete from su_04 where c1=2; commit; diff --git a/test/distributed/cases/pessimistic_transaction/unique_secondary_index.result b/test/distributed/cases/pessimistic_transaction/unique_secondary_index.result index 23b16ba8bb85..61b8635537d5 100644 --- a/test/distributed/cases/pessimistic_transaction/unique_secondary_index.result +++ b/test/distributed/cases/pessimistic_transaction/unique_secondary_index.result @@ -1,9 +1,9 @@ create table index_01 (col1 bigint primary key,col2 varchar(25),col3 float,col4 varchar(50),unique key num_phone(col2),key num_id(col4)); insert into index_01 values (1,'13456789872',20.23,'5678'),(2,'13873458290',100.00,'23'),(3,NULL,100.00,'23'); insert into index_01 values (67834,'13456789872',20.23,'4090'),(56473,'13456789872',100.00,'5678'); -Duplicate entry '13456789872' for key '__mo_index_idx_col' +Duplicate entry '13456789872' for key 'col2' insert into index_01 values (4,'13866666666',20.23,'5678'),(5,'13873458290',100.00,'23'),(6,'13777777777',100.00,'23'); -Duplicate entry '13873458290' for key '__mo_index_idx_col' +Duplicate entry '13873458290' for key 'col2' select col2,col4 from index_01; col2 col4 13456789872 5678 @@ -12,9 +12,9 @@ null 23 create table index_02 (col1 bigint primary key,col2 char(25) unique key,col3 float,col4 char(50),key num_id(col4)); insert into index_02 values (67834,'13456789872',20.23,'5678'),(56473,'',100.00,'5678'); insert into index_02 values (1,'',20.23,'5678'),(2,'13873458290',100.00,'23'); -Duplicate entry '' for key '__mo_index_idx_col' +Duplicate entry '' for key 'col2' insert into index_02 values (67834,'13456799878',20.23,'4090'),(56473,NULL,100.00,''); -Duplicate entry '67834' for key 'col1' +Duplicate entry '(\d{5})' for key 'col1' insert into index_02 values (3,'excel',0.1,'4090'),(4,'中文',0.2,''),(5,'MMEabc$%^123',0.2,''); select col2,col4 from index_02; col2 col4 @@ -28,7 +28,7 @@ col1 col2 col3 col4 5 MMEabc$%^123 0.2 create table index_03 (col1 bigint auto_increment primary key,col2 int,col3 float,col4 int,unique key id1(col2),key id2(col4)); insert into index_03(col2,col3,col4) values (10,20.23,4090),(10,100.00,5678); -Duplicate entry '10' for key '__mo_index_idx_col' +Duplicate entry '10' for key 'col2' insert into index_03(col2,col3,col4) values (10,20.23,4090),(11,100.00,4090); insert into index_03(col2,col3,col4) values (67834,20.23,4090),(56473,100.00,5678),(NULL,0.01,NULL); insert into index_03(col2,col3,col4) values (-2147483648,1.2,100),(2147483647,2.0,5); @@ -46,10 +46,10 @@ col1 col2 col3 col4 8 -2147483648 1.2 100 create table index_04 (col1 bigint,col2 int primary key,col3 float,col4 bigint,unique key id1(col1),key id2(col4)); insert into index_04 values (67834,2,20.23,4090),(67834,4,100.00,4091); -Duplicate entry '67834' for key '__mo_index_idx_col' +Duplicate entry '67834' for key 'col1' insert into index_04 values (1,2,20.23,4090),(2,4,100.00,4091),(NULL,3,0.01,NULL); insert into index_04 values (3,2,20.23,4090),(2,4,100.00,4091),(4,4,100.00,4090); -Duplicate entry ('\(\d{4}\,\d\)'|'\d') for key ('__mo_index_idx_col'|'\w{3}\d') +Duplicate entry '4' for key 'col2' select * from index_04; col1 col2 col3 col4 1 2 20.23 4090 @@ -59,7 +59,7 @@ select * from index_04 where col1 between 10 and 1000000; col1 col2 col3 col4 create table index_05 (col1 smallint unique key,col2 int primary key,col3 float,col4 smallint,key id2(col4)); insert into index_05 values (1,2,20.23,4090),(1,4,100.00,4091); -Duplicate entry '1' for key '__mo_index_idx_col' +Duplicate entry '1' for key 'col1' insert into index_05 values (1,2,20.23,4090),(2,4,100.00,4091),(NULL,3,0.01,NULL); select * from index_05; col1 col2 col3 col4 @@ -68,7 +68,7 @@ col1 col2 col3 col4 null 3 0.01 null create table index_06 (col1 tinyint,col2 int primary key,col3 float,col4 tinyint,unique key id1(col1),key id2(col4)); insert into index_06 values (1,2,20.23,56),(1,4,100.00,90); -Duplicate entry '1' for key '__mo_index_idx_col' +Duplicate entry '1' for key 'col1' insert into index_06 values (1,2,20.23,56),(2,4,100.00,41),(NULL,3,0.01,NULL); select * from index_06; col1 col2 col3 col4 @@ -78,7 +78,7 @@ null 3 0.01 null create table index_07 (col1 int unsigned,col2 int primary key,col3 float,col4 int unsigned,unique key id1(col1),key id2(col4)); insert into index_07 values (1,2,20.23,56),(2,4,100.00,41),(NULL,3,0.01,NULL); insert into index_07 values (1,2,20.23,56),(1,4,100.00,90); -Duplicate entry (.*) for key (.*) +Duplicate entry '2' for key 'col2' select * from index_07; col1 col2 col3 col4 1 2 20.23 56 @@ -86,7 +86,7 @@ col1 col2 col3 col4 null 3 0.01 null create table index_08 (col1 bigint unsigned,col2 int primary key,col3 float,col4 bigint unsigned,unique key id1(col1),key id2(col4)); insert into index_08 values (1,2,20.23,56),(1,4,100.00,90); -Duplicate entry '1' for key '__mo_index_idx_col' +Duplicate entry '1' for key 'col1' insert into index_08 values (1,2,20.23,56),(2,4,100.00,41),(NULL,3,0.01,NULL); select * from index_08; col1 col2 col3 col4 @@ -95,7 +95,7 @@ col1 col2 col3 col4 null 3 0.01 null create table index_09 (col1 bigint primary key,col2 decimal(4,2),col3 decimal(4,2),unique key d1(col2),key d2(col3)); insert into index_09 values (1000,20.23,20.00),(1200,20.23,0.10); -Duplicate entry '20.23' for key '__mo_index_idx_col' +Duplicate entry '20.23' for key 'col2' insert into index_09 values (1000,20.23,20.00),(1200,0.23,20.10),(1100,NULL,NULL); select * from index_09; col1 col2 col3 @@ -104,7 +104,7 @@ col1 col2 col3 1100 null null create table index_10 (col1 bigint primary key,col2 float,col3 float,unique key d1(col2),key d2(col3)); insert into index_10 values (1000,20.23,20.00),(1200,20.23,0.10); -Duplicate entry '20.23' for key '__mo_index_idx_col' +Duplicate entry '20.23' for key 'col2' insert into index_10 values (1000,20.23,20.00),(1200,0.23,20.10),(1100,NULL,NULL); select * from index_10; col1 col2 col3 @@ -113,7 +113,7 @@ col1 col2 col3 1100 null null create table index_11 (col1 bigint primary key,col2 double,col3 double,unique key d1(col2),key d2(col3)); insert into index_11 values (1000,20.23,20.00),(1200,20.23,0.10); -Duplicate entry '20.23' for key '__mo_index_idx_col' +Duplicate entry '20.23' for key 'col2' insert into index_11 values (1000,20.23,20.00),(1200,0.23,20.10),(1100,NULL,NULL); select * from index_11; col1 col2 col3 @@ -122,7 +122,7 @@ col1 col2 col3 1100 null null create table index_12 (col1 bigint auto_increment primary key,col2 date,col3 date,unique key d1(col2),key d2(col3)); insert into index_12(col2,col3) values ('2013-01-01','2014-02-01'),('2013-01-01','2014-02-20'); -Duplicate entry '2013-01-01' for key '__mo_index_idx_col' +Duplicate entry '2013-01-01' for key 'col2' insert into index_12(col2,col3) values (NULL,'2014-02-01'),(NULL,NULL); select col2 from index_12; col2 @@ -130,15 +130,15 @@ null null create table index_13 (col1 bigint auto_increment primary key,col2 datetime,col3 datetime,unique key d1(col2),key d2(col3)); insert into index_13(col2,col3) values ('2013-01-01 12:00:00','2014-02-01 10:00:00'),('2013-01-01 12:00:00','2014-02-20 05:00:00'); -Duplicate entry '2013-01-01 12:00:00' for key '__mo_index_idx_col' +Duplicate entry '2013-01-01 12:00:00' for key 'col2' insert into index_13(col2,col3) values (NULL,'2014-02-01 12:00:0'),(NULL,NULL); create table index_14 (col1 bigint auto_increment primary key,col2 timestamp,col3 timestamp,unique key d1(col2),key d2(col3)); insert into index_14(col2,col3) values ('2013-01-01 12:00:00','2014-02-01 10:00:00'),('2013-01-01 12:00:00','2014-02-20 05:00:00'); -Duplicate entry '2013-01-01 12:00:00' for key '__mo_index_idx_col' +Duplicate entry '2013-01-01 12:00:00' for key 'col2' insert into index_14(col2,col3) values (NULL,'2014-02-01 12:00:0'),(NULL,NULL); create table index_15 (col1 bigint primary key,col2 bool,unique key c2(col2)); insert into index_15 values (1,TRUE),(2,FALSE),(3,TRUE); -(Duplicate entry ')([\d\D][true|false]*)(' for key '__mo_index_idx_col') +Duplicate entry 'true' for key 'col2' insert into index_15 values (1,TRUE),(2,FALSE),(3,NULL); select * from index_15; col1 col2 @@ -167,7 +167,7 @@ create table index_table_01 (col1 bigint auto_increment primary key,col2 varchar insert into index_table_01 values (67834,'13456789872',20.23,4090),(56473,'',100.00,5678); insert into index_table_01 values (34,NULL,4090,1); insert into index_table_01 values (56478,'',103.00,5670); -Duplicate entry '' for key '__mo_index_idx_col' +Duplicate entry '' for key 'col2' select * from index_table_01; col1 col2 col3 col4 67834 13456789872 20.23 4090 @@ -181,7 +181,7 @@ drop table index_table_02; create table index_table_02 (col1 bigint primary key,col2 int default 0,unique key col2(col2)); insert into index_table_02(col1) values (1); insert into index_table_02(col1) values (2),(3); -Duplicate entry '0' for key '__mo_index_idx_col' +Duplicate entry '0' for key 'col2' drop table index_table_02; create table index_table_03 (col1 bigint,col2 int,unique key col2(col2)); insert into index_table_03 values (1,20),(2,NULL),(3,90); @@ -233,7 +233,7 @@ code_id code_type code_no code_rule_no org_no 1009 39 8744 0000243 yy10 truncate table t_code_rule; load data infile '$resources/load_data/unique_index_duplicate.csv' into table t_code_rule fields terminated by ','; -Duplicate entry '30' for key '__mo_index_idx_col' +Duplicate entry '30' for key 'code_type' select code_id,code_type,code_no,code_rule_no,org_no from t_code_rule; code_id code_type code_no code_rule_no org_no create table index_temp( col1 bigint(20) NOT NULL ,col2 varchar(50) NOT NULL,col3 varchar(50) NOT NULL,col4 varchar(50) NOT NULL,col5 varchar(255) NOT NULL,col6 varchar(50) NOT NULL,col7 varchar(50) NOT NULL,col8 varchar(50) NOT NULL,col9 int(11) ,col10 varchar(50) DEFAULT NULL,col11 varchar(255),col12 datetime NOT NULL,col13 varchar(50) DEFAULT NULL,col14 datetime DEFAULT NULL,col15 varchar(50) DEFAULT NULL,col16 varchar(20) NOT NULL DEFAULT 'N'); @@ -254,12 +254,12 @@ code_id code_type code_no code_rule_no org_no truncate table index_temp; load data infile '$resources/load_data/unique_index_duplicate.csv' into table index_temp fields terminated by ','; insert into t_code_rule select * from index_temp; -(Duplicate entry ')([\d\D]*)(' for key '__mo_index_idx_col') +Duplicate entry ')([\d\D]*)(' for key '__mo_index_idx_col' create table index_table_04 (col1 bigint not null auto_increment,col2 varchar(25),col3 int,col4 varchar(50),primary key (col1),unique key m1(col2,col3),key num_id(col4)); insert into index_table_04(col2,col3,col4) select 'apple',1,'10'; insert into index_table_04(col2,col3,col4) select 'apple',2,'11'; insert into index_table_04(col2,col3,col4) select 'apple',2,'12'; -Duplicate entry '(apple,2)' for key '__mo_index_idx_col' +Duplicate entry '(apple,2)' for key '(col2,col3)' insert into index_table_04(col2,col3,col4) select NULL,NULL,'13'; select * from index_table_04; col1 col2 col3 col4 @@ -270,9 +270,9 @@ drop table index_table_04; create table index_table_04 (col1 bigint not null auto_increment,col2 varchar(25),col3 int,col4 varchar(50),primary key (col1),unique key m1(col2),unique key m2(col3),key num_id(col4)); insert into index_table_04(col2,col3,col4) select 'apple',1,'10'; insert into index_table_04(col2,col3,col4) select 'apple',2,'11'; -Duplicate entry 'apple' for key '__mo_index_idx_col' +Duplicate entry 'apple' for key 'col2' insert into index_table_04(col2,col3,col4) select 'apple',2,'12'; -Duplicate entry 'apple' for key '__mo_index_idx_col' +Duplicate entry 'apple' for key 'col2' insert into index_table_04(col2,col3,col4) select NULL,NULL,'13'; select * from index_table_04; col1 col2 col3 col4 @@ -305,7 +305,7 @@ col1 col2 col3 col4 2 chart 2 11 truncate table index_table_05; insert into index_table_05(col2,col3,col4) values ('apple',1,'10'),('store',2,'11'),('apple',3,'12'); -Duplicate entry 'apple' for key '__mo_index_idx_col' +Duplicate entry 'apple' for key 'col2' select * from index_table_05; col1 col2 col3 col4 drop table index_table_05; @@ -341,7 +341,7 @@ create table create_index_01 (col1 bigint primary key,col2 varchar(25),col3 floa create unique index varchar_index on create_index_01(col2) comment 'create varchar index'; insert into create_index_01 values (1,'13456789872',20.23,'5678'),(2,'13873458290',100.00,'23'),(3,NULL,100.00,'23'); insert into create_index_01 values (67834,'13456789872',20.23,'4090'),(56473,'13456789872',100.00,'5678'); -Duplicate entry '13456789872' for key '__mo_index_idx_col' +Duplicate entry '13456789872' for key 'col2' select col2,col4 from create_index_01; col2 col4 13456789872 5678 @@ -372,7 +372,7 @@ drop index char_index on create_index_02; create table create_index_03 (col1 bigint auto_increment,col2 int,col3 float,col4 int); create unique index int_index on create_index_03(col2); insert into create_index_03(col2,col3,col4) values (10,20.23,4090),(10,100.00,5678); -Duplicate entry '10' for key '__mo_index_idx_col' +Duplicate entry '10' for key 'col2' insert into create_index_03(col2,col3,col4) values (10,20.23,4090),(11,100.00,4090); insert into create_index_03(col2,col3,col4) values (67834,20.23,4090),(56473,100.00,5678),(NULL,0.01,NULL); select * from create_index_03; @@ -386,7 +386,7 @@ drop index int_index on create_index_03; create table create_index_04 (col1 bigint,col2 int primary key,col3 float,col4 bigint); create unique index bigint_index on create_index_04(col1); insert into create_index_04 values (67834,2,20.23,4090),(67834,4,100.00,4091); -Duplicate entry '67834' for key '__mo_index_idx_col' +Duplicate entry '67834' for key 'col1' insert into create_index_04 values (1,2,20.23,4090),(2,4,100.00,4091),(NULL,3,0.01,NULL); insert into create_index_04 values (-9223372036854775808,5,20.23,4090),(9223372036854775807,6,100.00,4091); select * from create_index_04; @@ -403,7 +403,7 @@ col1 col2 col3 col4 create table create_index_05 (col1 smallint,col2 int primary key,col3 float,col4 smallint); create unique index smallint_index on create_index_05(col1); insert into create_index_05 values (1,2,20.23,4090),(1,4,100.00,4091); -Duplicate entry '1' for key '__mo_index_idx_col' +Duplicate entry '1' for key 'col1' insert into create_index_05 values (1,2,20.23,4090),(2,4,100.00,4091),(NULL,3,0.01,NULL); select * from create_index_05; col1 col2 col3 col4 @@ -413,7 +413,7 @@ null 3 0.01 null create table create_index_06 (col1 tinyint,col2 int primary key,col3 float,col4 tinyint); create unique index tinyint_index on create_index_06(col1); insert into create_index_06 values (1,2,20.23,56),(1,4,100.00,90); -Duplicate entry '1' for key '__mo_index_idx_col' +Duplicate entry '1' for key 'col1' insert into create_index_06 values (1,2,20.23,56),(2,4,100.00,41),(NULL,3,0.01,NULL); select * from create_index_06; col1 col2 col3 col4 @@ -424,7 +424,7 @@ create table create_index_07 (col1 int unsigned,col2 int primary key,col3 float, create unique index int_unsigned_index on create_index_07(col1); insert into create_index_07 values (1,2,20.23,56),(2,4,100.00,41),(NULL,3,0.01,NULL); insert into create_index_07 values (1,2,20.23,56),(1,4,100.00,90); -(Duplicate entry ')([\d\D]*)(' for key ')([col2|__mo_index_idx_col]*)(') +Duplicate entry '2' for key 'col2' select * from create_index_07; col1 col2 col3 col4 1 2 20.23 56 @@ -433,7 +433,7 @@ null 3 0.01 null create table create_index_08 (col1 bigint unsigned,col2 int primary key,col3 float,col4 bigint unsigned); create unique index bigint_unsigned_index on create_index_08(col1); insert into create_index_08 values (1,2,20.23,56),(1,4,100.00,90); -Duplicate entry '1' for key '__mo_index_idx_col' +Duplicate entry '1' for key 'col1' insert into create_index_08 values (1,2,20.23,56),(2,4,100.00,41),(NULL,3,0.01,NULL); select * from create_index_08; col1 col2 col3 col4 @@ -443,7 +443,7 @@ null 3 0.01 null create table create_index_09 (col1 bigint primary key,col2 decimal(16,8),col3 decimal(16,8)); create unique index decimal_index on create_index_09(col2); insert into create_index_09 values (1000,20.23,20.00),(1200,20.23,0.10); -Duplicate entry '20.23000000' for key '__mo_index_idx_col' +Duplicate entry '20.23000000' for key 'col2' insert into create_index_09 values (1000,20.23,20.00),(1200,0.23,20.10),(1100,NULL,NULL); select * from create_index_09; col1 col2 col3 @@ -453,7 +453,7 @@ col1 col2 col3 create table create_index_10 (col1 bigint primary key,col2 float,col3 float); create unique index float_index on create_index_10(col2); insert into create_index_10 values (1000,20.23,20.00),(1200,20.23,0.10); -Duplicate entry '20.23' for key '__mo_index_idx_col' +Duplicate entry '20.23' for key 'col2' insert into create_index_10 values (1000,20.23,20.00),(1200,0.23,20.10),(1100,NULL,NULL); select * from create_index_10; col1 col2 col3 @@ -463,7 +463,7 @@ col1 col2 col3 create table create_index_11 (col1 bigint primary key,col2 double,col3 double); create unique index double_index on create_index_11(col2); insert into create_index_11 values (1000,20.23,20.00),(1200,20.23,0.10); -Duplicate entry '20.23' for key '__mo_index_idx_col' +Duplicate entry '20.23' for key 'col2' insert into create_index_11 values (1000,20.23,20.00),(1200,0.23,20.10),(1100,NULL,NULL); select * from create_index_11; col1 col2 col3 @@ -473,22 +473,22 @@ col1 col2 col3 create table create_index_12(col1 bigint auto_increment primary key,col2 date,col3 date); create unique index date_index on create_index_12(col2); insert into create_index_12(col2,col3) values ('2013-01-01','2014-02-01'),('2013-01-01','2014-02-20'); -Duplicate entry '2013-01-01' for key '__mo_index_idx_col' +Duplicate entry '2013-01-01' for key 'col2' insert into create_index_12(col2,col3) values (NULL,'2014-02-01'),(NULL,NULL); create table create_index_13 (col1 bigint auto_increment primary key,col2 datetime,col3 datetime); create unique index datetime_index on create_index_13(col2); insert into create_index_13(col2,col3) values ('2013-01-01 12:00:00','2014-02-01 10:00:00'),('2013-01-01 12:00:00','2014-02-20 05:00:00'); -Duplicate entry '2013-01-01 12:00:00' for key '__mo_index_idx_col' +Duplicate entry '2013-01-01 12:00:00' for key 'col2' insert into create_index_13(col2,col3) values (NULL,'2014-02-01 12:00:0'),(NULL,NULL); create table create_index_14 (col1 bigint auto_increment primary key,col2 timestamp,col3 timestamp); create unique index timestamp_index on create_index_14(col2); insert into create_index_14(col2,col3) values ('2013-01-01 12:00:00','2014-02-01 10:00:00'),('2013-01-01 12:00:00','2014-02-20 05:00:00'); -Duplicate entry '2013-01-01 12:00:00' for key '__mo_index_idx_col' +Duplicate entry '2013-01-01 12:00:00' for key 'col2' insert into create_index_14(col2,col3) values (NULL,'2014-02-01 12:00:0'),(NULL,NULL); create table create_index_15 (col1 bigint primary key,col2 bool); create unique index bool_index on create_index_15(col2); insert into create_index_15 values (1,TRUE),(2,FALSE),(3,TRUE); -(Duplicate entry ')([\d\D][true|false]*)(' for key '__mo_index_idx_col') +Duplicate entry 'true' for key 'col2' insert into create_index_15 values (1,TRUE),(2,FALSE),(3,NULL); select * from create_index_15; col1 col2 @@ -523,7 +523,7 @@ create unique index m1_index on create_index_18(col2,col3); insert into create_index_18(col2,col3,col4) select 'apple',1,'10'; insert into create_index_18(col2,col3,col4) select 'apple',2,'11'; insert into create_index_18(col2,col3,col4) select 'apple',2,'12'; -Duplicate entry '(apple,2)' for key '__mo_index_idx_col' +Duplicate entry '(apple,2)' for key '(col2,col3)' insert into create_index_18(col2,col3,col4) select NULL,NULL,'13'; select * from create_index_18; col1 col2 col3 col4 @@ -538,7 +538,7 @@ insert into create_index_18(col2,col3,col4) select 'apple',2,'11'; insert into create_index_18(col2,col3,col4) select 'apple',2,'12'; insert into create_index_18(col2,col3,col4) select NULL,NULL,'13'; insert into create_index_18(col2,col3,col4) select 'apple',2,'12'; -Duplicate entry '(apple,2,12)' for key '__mo_index_idx_col' +Duplicate entry '(apple,2,12)' for key '(col2,col3,col4)' select * from create_index_18; col1 col2 col3 col4 1 apple 1 10 @@ -603,7 +603,7 @@ col1 col2 col3 col4 2 chart 2 11 truncate table create_index_19; insert into create_index_19(col2,col3,col4) values ('apple',1,'10'),('store',2,'11'),('apple',3,'12'); -Duplicate entry 'apple' for key '__mo_index_idx_col' +Duplicate entry 'apple' for key 'col2' select * from create_index_19; col1 col2 col3 col4 drop table create_index_19; @@ -640,7 +640,7 @@ create unique index varchar_index on trans_index_01(col2) comment 'create varcha insert into trans_index_01 values (1,'13456789872',20.23,'5678'),(2,'13873458290',100.00,'23'),(3,NULL,100.00,'23'); use unique_secondary_index; insert into trans_index_01 values (1,'13456789872',20.23,'5678'),(2,'13873458290',100.00,'23'); -Duplicate entry (\'\d{11}\'|'\d') for key ('__mo_index_idx_col'|'\w{3}\d') +Duplicate entry '1' for key 'col1' commit; select * from trans_index_01; col1 col2 col3 col4 diff --git a/test/distributed/cases/pessimistic_transaction/unique_secondary_index.sql b/test/distributed/cases/pessimistic_transaction/unique_secondary_index.sql index 5fd51de8f5c6..f4148bec4780 100644 --- a/test/distributed/cases/pessimistic_transaction/unique_secondary_index.sql +++ b/test/distributed/cases/pessimistic_transaction/unique_secondary_index.sql @@ -7,6 +7,7 @@ select col2,col4 from index_01; create table index_02 (col1 bigint primary key,col2 char(25) unique key,col3 float,col4 char(50),key num_id(col4)); insert into index_02 values (67834,'13456789872',20.23,'5678'),(56473,'',100.00,'5678'); insert into index_02 values (1,'',20.23,'5678'),(2,'13873458290',100.00,'23'); +-- @pattern insert into index_02 values (67834,'13456799878',20.23,'4090'),(56473,NULL,100.00,''); insert into index_02 values (3,'excel',0.1,'4090'),(4,'中文',0.2,''),(5,'MMEabc$%^123',0.2,''); select col2,col4 from index_02; diff --git a/test/distributed/cases/prepare/prepare.result b/test/distributed/cases/prepare/prepare.result index 29c86990bbdd..67da2269c6a8 100644 --- a/test/distributed/cases/prepare/prepare.result +++ b/test/distributed/cases/prepare/prepare.result @@ -304,7 +304,7 @@ create table t1( a int unique key, b int, c int ); prepare stmt1 from 'INSERT INTO t1 values(1, 2, 3)'; execute stmt1; execute stmt1; -Duplicate entry '1' for key '__mo_index_idx_col' +Duplicate entry '1' for key '(.*)' select * from t1; a b c 1 2 3 diff --git a/test/distributed/cases/prepare/prepare.test b/test/distributed/cases/prepare/prepare.test index 2551940dcf7a..92994c4f96d6 100644 --- a/test/distributed/cases/prepare/prepare.test +++ b/test/distributed/cases/prepare/prepare.test @@ -207,6 +207,7 @@ drop table if exists t1; create table t1( a int unique key, b int, c int ); prepare stmt1 from 'INSERT INTO t1 values(1, 2, 3)'; execute stmt1; +-- @pattern execute stmt1; select * from t1; diff --git a/test/distributed/cases/procedure/procedure.result b/test/distributed/cases/procedure/procedure.result index 393e0c8b7eae..ab613d7b4893 100644 --- a/test/distributed/cases/procedure/procedure.result +++ b/test/distributed/cases/procedure/procedure.result @@ -24,7 +24,7 @@ id val 3 c drop procedure test_if_hit_elseif_first_elseif; drop procedure if exists test_if_hit_second_elseif; -create procedure test_if_hit_second_elseif() 'begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end' +create procedure test_if_hit_second_elseif() 'begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 order by id limit 1; ELSE select * from tbh3; END IF; end' call test_if_hit_second_elseif(); id val 1 a diff --git a/test/distributed/cases/procedure/procedure.sql b/test/distributed/cases/procedure/procedure.sql index 373ecae93307..2ebc4969460a 100644 --- a/test/distributed/cases/procedure/procedure.sql +++ b/test/distributed/cases/procedure/procedure.sql @@ -42,7 +42,7 @@ drop procedure test_if_hit_elseif_first_elseif; -- @label:bvt drop procedure if exists test_if_hit_second_elseif; -- @delimiter . -create procedure test_if_hit_second_elseif() 'begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end' +create procedure test_if_hit_second_elseif() 'begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 order by id limit 1; ELSE select * from tbh3; END IF; end' . -- @delimiter ; call test_if_hit_second_elseif(); diff --git a/test/distributed/cases/snapshot/cluster_level_snapshot_restore_system_table_to_nonsys.result b/test/distributed/cases/snapshot/cluster_level_snapshot_restore_system_table_to_nonsys.result index eadd2090be71..3768d773423a 100644 --- a/test/distributed/cases/snapshot/cluster_level_snapshot_restore_system_table_to_nonsys.result +++ b/test/distributed/cases/snapshot/cluster_level_snapshot_restore_system_table_to_nonsys.result @@ -161,7 +161,7 @@ insert into tbh1(id, val) values(1,10),(2,20),(3,30); insert into tbh2(id, val) values(1,'a'),(2,'b'),(3,'c'); insert into tbh3(id, val) values(1,1.5),(2,2.5),(3,3.5); drop procedure if exists test_if_hit_second_elseif; -create procedure test_if_hit_second_elseif() 'begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end'; +create procedure test_if_hit_second_elseif() 'begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 order by id limit 1; ELSE select * from tbh3; END IF; end'; call test_if_hit_second_elseif(); id val 1 a @@ -174,7 +174,7 @@ id val 3 3.5 select * from mo_catalog.mo_stored_procedure; proc_id name creator args body db definer modified_time created_time type security_type comment character_set_client collation_connection database_collation -3 test_if_hit_second_elseif null {} begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-08-07 09:44:51 2024-08-07 09:44:51 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +3 test_if_hit_second_elseif null {} begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 order by id limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-08-07 09:44:51 2024-08-07 09:44:51 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci 4 test_if_hit_else null {} begin DECLARE v1 INT; SET v1 = 3; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-08-07 09:44:51 2024-08-07 09:44:51 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci drop snapshot if exists sp_sp06; create snapshot sp_sp06 for cluster; @@ -195,7 +195,7 @@ id val 1 a select * from mo_catalog.mo_stored_procedure; proc_id name creator args body db definer modified_time created_time type security_type comment character_set_client collation_connection database_collation -3 test_if_hit_second_elseif null {} begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-08-07 09:44:51 2024-08-07 09:44:51 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +3 test_if_hit_second_elseif null {} begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 order by id limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-08-07 09:44:51 2024-08-07 09:44:51 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci 4 test_if_hit_else null {} begin DECLARE v1 INT; SET v1 = 3; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-08-07 09:44:51 2024-08-07 09:44:51 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci drop procedure test_if_hit_second_elseif; drop procedure test_if_hit_else; diff --git a/test/distributed/cases/snapshot/cluster_level_snapshot_restore_system_table_to_nonsys.sql b/test/distributed/cases/snapshot/cluster_level_snapshot_restore_system_table_to_nonsys.sql index b888f65f8a09..ea52b3026779 100644 --- a/test/distributed/cases/snapshot/cluster_level_snapshot_restore_system_table_to_nonsys.sql +++ b/test/distributed/cases/snapshot/cluster_level_snapshot_restore_system_table_to_nonsys.sql @@ -185,7 +185,7 @@ insert into tbh2(id, val) values(1,'a'),(2,'b'),(3,'c'); insert into tbh3(id, val) values(1,1.5),(2,2.5),(3,3.5); drop procedure if exists test_if_hit_second_elseif; -create procedure test_if_hit_second_elseif() 'begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end'; +create procedure test_if_hit_second_elseif() 'begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 order by id limit 1; ELSE select * from tbh3; END IF; end'; call test_if_hit_second_elseif(); drop procedure if exists test_if_hit_else; diff --git a/test/distributed/cases/snapshot/cluster_level_snapshot_restore_system_table_to_sys_account.result b/test/distributed/cases/snapshot/cluster_level_snapshot_restore_system_table_to_sys_account.result index 5f6b0d92f6d3..03b626dd55a6 100644 --- a/test/distributed/cases/snapshot/cluster_level_snapshot_restore_system_table_to_sys_account.result +++ b/test/distributed/cases/snapshot/cluster_level_snapshot_restore_system_table_to_sys_account.result @@ -169,7 +169,7 @@ insert into tbh1(id, val) values(1,10),(2,20),(3,30); insert into tbh2(id, val) values(1,'a'),(2,'b'),(3,'c'); insert into tbh3(id, val) values(1,1.5),(2,2.5),(3,3.5); drop procedure if exists test_if_hit_second_elseif; -create procedure test_if_hit_second_elseif() 'begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end'; +create procedure test_if_hit_second_elseif() 'begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 order by id limit 1; ELSE select * from tbh3; END IF; end'; call test_if_hit_second_elseif(); id val 1 a @@ -182,7 +182,7 @@ id val 3 3.5 select * from mo_catalog.mo_stored_procedure; proc_id name creator args body db definer modified_time created_time type security_type comment character_set_client collation_connection database_collation -3 test_if_hit_second_elseif null {} begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test dump 2024-07-08 02:40:45 2024-07-08 02:40:45 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +3 test_if_hit_second_elseif null {} begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 order by id limit 1; ELSE select * from tbh3; END IF; end procedure_test dump 2024-07-08 02:40:45 2024-07-08 02:40:45 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci 4 test_if_hit_else null {} begin DECLARE v1 INT; SET v1 = 3; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test dump 2024-07-08 02:40:45 2024-07-08 02:40:45 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci drop snapshot if exists sp_sp06; create snapshot sp_sp06 for cluster; @@ -203,7 +203,7 @@ id val 1 a select * from mo_catalog.mo_stored_procedure; proc_id name creator args body db definer modified_time created_time type security_type comment character_set_client collation_connection database_collation -3 test_if_hit_second_elseif null {} begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test dump 2024-07-08 02:40:45 2024-07-08 02:40:45 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +3 test_if_hit_second_elseif null {} begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 order by id limit 1; ELSE select * from tbh3; END IF; end procedure_test dump 2024-07-08 02:40:45 2024-07-08 02:40:45 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci 4 test_if_hit_else null {} begin DECLARE v1 INT; SET v1 = 3; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test dump 2024-07-08 02:40:45 2024-07-08 02:40:45 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci drop snapshot sp_sp06; drop procedure test_if_hit_second_elseif; diff --git a/test/distributed/cases/snapshot/cluster_level_snapshot_restore_system_table_to_sys_account.sql b/test/distributed/cases/snapshot/cluster_level_snapshot_restore_system_table_to_sys_account.sql index fb3129d41580..4bc5c2170ad4 100644 --- a/test/distributed/cases/snapshot/cluster_level_snapshot_restore_system_table_to_sys_account.sql +++ b/test/distributed/cases/snapshot/cluster_level_snapshot_restore_system_table_to_sys_account.sql @@ -157,7 +157,7 @@ insert into tbh2(id, val) values(1,'a'),(2,'b'),(3,'c'); insert into tbh3(id, val) values(1,1.5),(2,2.5),(3,3.5); drop procedure if exists test_if_hit_second_elseif; -create procedure test_if_hit_second_elseif() 'begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end'; +create procedure test_if_hit_second_elseif() 'begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 order by id limit 1; ELSE select * from tbh3; END IF; end'; call test_if_hit_second_elseif(); drop procedure if exists test_if_hit_else; diff --git a/test/distributed/cases/snapshot/nonsys_restore_system_table_to_nonsys_account.result b/test/distributed/cases/snapshot/nonsys_restore_system_table_to_nonsys_account.result index 1273c28184fa..1ea61d2a2ee2 100644 --- a/test/distributed/cases/snapshot/nonsys_restore_system_table_to_nonsys_account.result +++ b/test/distributed/cases/snapshot/nonsys_restore_system_table_to_nonsys_account.result @@ -173,7 +173,7 @@ insert into tbh1(id, val) values(1,10),(2,20),(3,30); insert into tbh2(id, val) values(1,'a'),(2,'b'),(3,'c'); insert into tbh3(id, val) values(1,1.5),(2,2.5),(3,3.5); drop procedure if exists test_if_hit_second_elseif; -create procedure test_if_hit_second_elseif() 'begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end'; +create procedure test_if_hit_second_elseif() 'begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 order by id limit 1; ELSE select * from tbh3; END IF; end'; call test_if_hit_second_elseif(); id val 1 a @@ -186,7 +186,7 @@ id val 3 3.5 select * from mo_catalog.mo_stored_procedure; proc_id name creator args body db definer modified_time created_time type security_type comment character_set_client collation_connection database_collation -3 test_if_hit_second_elseif null {} begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-08-07 09:45:14 2024-08-07 09:45:14 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +3 test_if_hit_second_elseif null {} begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 order by id limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-08-07 09:45:14 2024-08-07 09:45:14 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci 4 test_if_hit_else null {} begin DECLARE v1 INT; SET v1 = 3; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-08-07 09:45:14 2024-08-07 09:45:14 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci drop snapshot if exists sp_sp06; create snapshot sp_sp06 for account acc01; @@ -207,7 +207,7 @@ id val 1 a select * from mo_catalog.mo_stored_procedure; proc_id name creator args body db definer modified_time created_time type security_type comment character_set_client collation_connection database_collation -3 test_if_hit_second_elseif null {} begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-08-07 09:45:14 2024-08-07 09:45:14 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +3 test_if_hit_second_elseif null {} begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 order by id limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-08-07 09:45:14 2024-08-07 09:45:14 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci 4 test_if_hit_else null {} begin DECLARE v1 INT; SET v1 = 3; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-08-07 09:45:14 2024-08-07 09:45:14 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci drop snapshot sp_sp06; drop procedure test_if_hit_second_elseif; diff --git a/test/distributed/cases/snapshot/nonsys_restore_system_table_to_nonsys_account.sql b/test/distributed/cases/snapshot/nonsys_restore_system_table_to_nonsys_account.sql index 48f3090b4115..ea4dbf02f4ce 100644 --- a/test/distributed/cases/snapshot/nonsys_restore_system_table_to_nonsys_account.sql +++ b/test/distributed/cases/snapshot/nonsys_restore_system_table_to_nonsys_account.sql @@ -162,7 +162,7 @@ insert into tbh2(id, val) values(1,'a'),(2,'b'),(3,'c'); insert into tbh3(id, val) values(1,1.5),(2,2.5),(3,3.5); drop procedure if exists test_if_hit_second_elseif; -create procedure test_if_hit_second_elseif() 'begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end'; +create procedure test_if_hit_second_elseif() 'begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 order by id limit 1; ELSE select * from tbh3; END IF; end'; call test_if_hit_second_elseif(); drop procedure if exists test_if_hit_else; diff --git a/test/distributed/cases/snapshot/sys_restore_system_table_to_newnonsys_account.result b/test/distributed/cases/snapshot/sys_restore_system_table_to_newnonsys_account.result index d281328b9f93..f44bf69cc9fd 100644 --- a/test/distributed/cases/snapshot/sys_restore_system_table_to_newnonsys_account.result +++ b/test/distributed/cases/snapshot/sys_restore_system_table_to_newnonsys_account.result @@ -168,7 +168,7 @@ insert into tbh1(id, val) values(1,10),(2,20),(3,30); insert into tbh2(id, val) values(1,'a'),(2,'b'),(3,'c'); insert into tbh3(id, val) values(1,1.5),(2,2.5),(3,3.5); drop procedure if exists test_if_hit_second_elseif; -create procedure test_if_hit_second_elseif() 'begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end'; +create procedure test_if_hit_second_elseif() 'begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 order by id limit 1; ELSE select * from tbh3; END IF; end'; call test_if_hit_second_elseif(); id val 1 a @@ -181,7 +181,7 @@ id val 3 3.5 select * from mo_catalog.mo_stored_procedure; proc_id name creator args body db definer modified_time created_time type security_type comment character_set_client collation_connection database_collation -3 test_if_hit_second_elseif null {} begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-08-07 09:45:41 2024-08-07 09:45:41 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +3 test_if_hit_second_elseif null {} begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 order by id limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-08-07 09:45:41 2024-08-07 09:45:41 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci 4 test_if_hit_else null {} begin DECLARE v1 INT; SET v1 = 3; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-08-07 09:45:41 2024-08-07 09:45:41 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci drop snapshot if exists sp_sp06; create snapshot sp_sp06 for account acc01; @@ -202,7 +202,7 @@ id val 1 a select * from mo_catalog.mo_stored_procedure; proc_id name creator args body db definer modified_time created_time type security_type comment character_set_client collation_connection database_collation -3 test_if_hit_second_elseif null {} begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-08-07 09:45:41 2024-08-07 09:45:41 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +3 test_if_hit_second_elseif null {} begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 order by id limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-08-07 09:45:41 2024-08-07 09:45:41 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci 4 test_if_hit_else null {} begin DECLARE v1 INT; SET v1 = 3; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-08-07 09:45:41 2024-08-07 09:45:41 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci drop procedure procedure_test.test_if_hit_second_elseif; drop procedure procedure_test.test_if_hit_else; diff --git a/test/distributed/cases/snapshot/sys_restore_system_table_to_newnonsys_account.sql b/test/distributed/cases/snapshot/sys_restore_system_table_to_newnonsys_account.sql index faffd3b8b370..ed05a845eaf7 100644 --- a/test/distributed/cases/snapshot/sys_restore_system_table_to_newnonsys_account.sql +++ b/test/distributed/cases/snapshot/sys_restore_system_table_to_newnonsys_account.sql @@ -191,7 +191,7 @@ insert into tbh2(id, val) values(1,'a'),(2,'b'),(3,'c'); insert into tbh3(id, val) values(1,1.5),(2,2.5),(3,3.5); drop procedure if exists test_if_hit_second_elseif; -create procedure test_if_hit_second_elseif() 'begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end'; +create procedure test_if_hit_second_elseif() 'begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 order by id limit 1; ELSE select * from tbh3; END IF; end'; call test_if_hit_second_elseif(); drop procedure if exists test_if_hit_else; diff --git a/test/distributed/cases/snapshot/sys_restore_system_table_to_nonsys_account.result b/test/distributed/cases/snapshot/sys_restore_system_table_to_nonsys_account.result index 0faa79a158a6..ba108b91cac3 100644 --- a/test/distributed/cases/snapshot/sys_restore_system_table_to_nonsys_account.result +++ b/test/distributed/cases/snapshot/sys_restore_system_table_to_nonsys_account.result @@ -161,7 +161,7 @@ insert into tbh1(id, val) values(1,10),(2,20),(3,30); insert into tbh2(id, val) values(1,'a'),(2,'b'),(3,'c'); insert into tbh3(id, val) values(1,1.5),(2,2.5),(3,3.5); drop procedure if exists test_if_hit_second_elseif; -create procedure test_if_hit_second_elseif() 'begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end'; +create procedure test_if_hit_second_elseif() 'begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 order by id limit 1; ELSE select * from tbh3; END IF; end'; call test_if_hit_second_elseif(); id val 1 a @@ -174,7 +174,7 @@ id val 3 3.5 select * from mo_catalog.mo_stored_procedure; proc_id name creator args body db definer modified_time created_time type security_type comment character_set_client collation_connection database_collation -3 test_if_hit_second_elseif null {} begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-08-07 09:46:01 2024-08-07 09:46:01 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +3 test_if_hit_second_elseif null {} begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 order by id limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-08-07 09:46:01 2024-08-07 09:46:01 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci 4 test_if_hit_else null {} begin DECLARE v1 INT; SET v1 = 3; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-08-07 09:46:01 2024-08-07 09:46:01 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci drop snapshot if exists sp_sp06; create snapshot sp_sp06 for account acc01; @@ -195,7 +195,7 @@ id val 1 a select * from mo_catalog.mo_stored_procedure; proc_id name creator args body db definer modified_time created_time type security_type comment character_set_client collation_connection database_collation -3 test_if_hit_second_elseif null {} begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-08-07 09:46:01 2024-08-07 09:46:01 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +3 test_if_hit_second_elseif null {} begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 order by id limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-08-07 09:46:01 2024-08-07 09:46:01 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci 4 test_if_hit_else null {} begin DECLARE v1 INT; SET v1 = 3; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test test_account 2024-08-07 09:46:01 2024-08-07 09:46:01 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci drop procedure test_if_hit_second_elseif; drop procedure test_if_hit_else; diff --git a/test/distributed/cases/snapshot/sys_restore_system_table_to_nonsys_account.sql b/test/distributed/cases/snapshot/sys_restore_system_table_to_nonsys_account.sql index c647c0c45445..2765f994850f 100644 --- a/test/distributed/cases/snapshot/sys_restore_system_table_to_nonsys_account.sql +++ b/test/distributed/cases/snapshot/sys_restore_system_table_to_nonsys_account.sql @@ -185,7 +185,7 @@ insert into tbh2(id, val) values(1,'a'),(2,'b'),(3,'c'); insert into tbh3(id, val) values(1,1.5),(2,2.5),(3,3.5); drop procedure if exists test_if_hit_second_elseif; -create procedure test_if_hit_second_elseif() 'begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end'; +create procedure test_if_hit_second_elseif() 'begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 order by id limit 1; ELSE select * from tbh3; END IF; end'; call test_if_hit_second_elseif(); drop procedure if exists test_if_hit_else; diff --git a/test/distributed/cases/snapshot/sys_restore_system_table_to_sys_account.result b/test/distributed/cases/snapshot/sys_restore_system_table_to_sys_account.result index 1e3a59d714c1..e14242f7b387 100644 --- a/test/distributed/cases/snapshot/sys_restore_system_table_to_sys_account.result +++ b/test/distributed/cases/snapshot/sys_restore_system_table_to_sys_account.result @@ -169,7 +169,7 @@ insert into tbh1(id, val) values(1,10),(2,20),(3,30); insert into tbh2(id, val) values(1,'a'),(2,'b'),(3,'c'); insert into tbh3(id, val) values(1,1.5),(2,2.5),(3,3.5); drop procedure if exists test_if_hit_second_elseif; -create procedure test_if_hit_second_elseif() 'begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end'; +create procedure test_if_hit_second_elseif() 'begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 order by id limit 1; ELSE select * from tbh3; END IF; end'; call test_if_hit_second_elseif(); id val 1 a @@ -182,7 +182,7 @@ id val 3 3.5 select * from mo_catalog.mo_stored_procedure; proc_id name creator args body db definer modified_time created_time type security_type comment character_set_client collation_connection database_collation -3 test_if_hit_second_elseif null {} begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test dump 2024-05-31 03:23:49 2024-05-31 03:23:49 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +3 test_if_hit_second_elseif null {} begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 order by id limit 1; ELSE select * from tbh3; END IF; end procedure_test dump 2024-05-31 03:23:49 2024-05-31 03:23:49 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci 4 test_if_hit_else null {} begin DECLARE v1 INT; SET v1 = 3; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test dump 2024-05-31 03:23:49 2024-05-31 03:23:49 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci drop snapshot if exists sp_sp06; create snapshot sp_sp06 for account sys; @@ -203,7 +203,7 @@ id val 1 a select * from mo_catalog.mo_stored_procedure; proc_id name creator args body db definer modified_time created_time type security_type comment character_set_client collation_connection database_collation -3 test_if_hit_second_elseif null {} begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test dump 2024-05-31 03:23:49 2024-05-31 03:23:49 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci +3 test_if_hit_second_elseif null {} begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 order by id limit 1; ELSE select * from tbh3; END IF; end procedure_test dump 2024-05-31 03:23:49 2024-05-31 03:23:49 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci 4 test_if_hit_else null {} begin DECLARE v1 INT; SET v1 = 3; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end procedure_test dump 2024-05-31 03:23:49 2024-05-31 03:23:49 PROCEDURE DEFINER utf8mb4 utf8mb4_0900_ai_ci utf8mb4_0900_ai_ci drop snapshot sp_sp06; drop procedure test_if_hit_second_elseif; diff --git a/test/distributed/cases/snapshot/sys_restore_system_table_to_sys_account.sql b/test/distributed/cases/snapshot/sys_restore_system_table_to_sys_account.sql index 378849c03a69..87510f40d1c2 100644 --- a/test/distributed/cases/snapshot/sys_restore_system_table_to_sys_account.sql +++ b/test/distributed/cases/snapshot/sys_restore_system_table_to_sys_account.sql @@ -157,7 +157,7 @@ insert into tbh2(id, val) values(1,'a'),(2,'b'),(3,'c'); insert into tbh3(id, val) values(1,1.5),(2,2.5),(3,3.5); drop procedure if exists test_if_hit_second_elseif; -create procedure test_if_hit_second_elseif() 'begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 limit 1; ELSE select * from tbh3; END IF; end'; +create procedure test_if_hit_second_elseif() 'begin DECLARE v1 INT; SET v1 = 4; IF v1 > 5 THEN select * from tbh1; ELSEIF v1 = 5 THEN select * from tbh2; ELSEIF v1 = 4 THEN select * from tbh2 order by id limit 1; ELSE select * from tbh3; END IF; end'; call test_if_hit_second_elseif(); drop procedure if exists test_if_hit_else; diff --git a/test/distributed/resources/load_data/integer_numbers_4.csv b/test/distributed/resources/load_data/integer_numbers_4.csv new file mode 100644 index 000000000000..1c4ed1b10383 --- /dev/null +++ b/test/distributed/resources/load_data/integer_numbers_4.csv @@ -0,0 +1 @@ +"1.2","234.4","2147483642.3","92233720368547.4","254.7","65533.3","4294967294.2","1844674407370955.9" \ No newline at end of file