github.com/matrixorigin/matrixone@v1.2.0/pkg/sql/plan/partition_key.go (about)

     1  // Copyright 2022 Matrix Origin
     2  //
     3  // Licensed under the Apache License, Version 2.0 (the "License");
     4  // you may not use this file except in compliance with the License.
     5  // You may obtain a copy of the License at
     6  //
     7  //     http://www.apache.org/licenses/LICENSE-2.0
     8  //
     9  // Unless required by applicable law or agreed to in writing, software
    10  // distributed under the License is distributed on an "AS IS" BASIS,
    11  // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    12  // See the License for the specific language governing permissions and
    13  // limitations under the License.
    14  
    15  package plan
    16  
    17  import (
    18  	"context"
    19  	"github.com/matrixorigin/matrixone/pkg/common/moerr"
    20  	"github.com/matrixorigin/matrixone/pkg/container/types"
    21  	"github.com/matrixorigin/matrixone/pkg/pb/plan"
    22  	"github.com/matrixorigin/matrixone/pkg/sql/parsers/dialect"
    23  	"github.com/matrixorigin/matrixone/pkg/sql/parsers/tree"
    24  	"strings"
    25  )
    26  
    27  // keyPartitionBuilder processes key partition
    28  type keyPartitionBuilder struct {
    29  }
    30  
    31  func (kpb *keyPartitionBuilder) build(ctx context.Context, partitionBinder *PartitionBinder, stmt *tree.CreateTable, tableDef *TableDef) error {
    32  	partitionSyntaxDef := stmt.PartitionOption
    33  	partitionCount, err := getValidPartitionCount(ctx, false, partitionSyntaxDef)
    34  	if err != nil {
    35  		return err
    36  	}
    37  	partitionType := partitionSyntaxDef.PartBy.PType.(*tree.KeyType)
    38  	// check the algorithm option
    39  	if partitionType.Algorithm != 1 && partitionType.Algorithm != 2 {
    40  		return moerr.NewInvalidInput(ctx, "the 'ALGORITHM' option is unsupported")
    41  	}
    42  
    43  	partitionDef := &plan.PartitionByDef{
    44  		PartitionNum:   partitionCount,
    45  		Algorithm:      partitionType.Algorithm,
    46  		IsSubPartition: partitionSyntaxDef.PartBy.IsSubPartition,
    47  	}
    48  
    49  	partitionDef.Type = plan.PartitionType_KEY
    50  	if partitionType.Linear {
    51  		partitionDef.Type = plan.PartitionType_LINEAR_KEY
    52  	}
    53  
    54  	// complement the column list if there is no one in the syntax
    55  	if len(partitionType.ColumnList) == 0 {
    56  		// Any columns used as the partitioning key must comprise part or all of the table's primary key, if the table has one.
    57  		// Where no column name is specified as the partitioning key, the table's primary key is used, if there is one.
    58  		// If there is no primary key but there is a unique key, then the unique key is used for the partitioning key
    59  		primaryKeys, uniqueIndices := getPrimaryKeyAndUniqueKey(stmt.Defs)
    60  		if len(primaryKeys) != 0 {
    61  			partitionType.ColumnList = primaryKeys
    62  		} else if len(uniqueIndices) != 0 {
    63  			uniqueKeyNames, err := chooseAvailableUniqueKey(ctx, tableDef, uniqueIndices)
    64  			if err != nil {
    65  				return err
    66  			}
    67  			partitionType.ColumnList = uniqueKeyNames
    68  		}
    69  		if len(partitionType.ColumnList) == 0 {
    70  			return moerr.NewInvalidInput(ctx, "Field in list of fields for partition function not found in table")
    71  		}
    72  	}
    73  
    74  	err = buildPartitionColumns(ctx, tableDef, partitionBinder, partitionDef, partitionType.ColumnList)
    75  	//err = buildPartitionColumns(ctx, partitionBinder, partitionDef, partitionType.ColumnList)
    76  	if err != nil {
    77  		return err
    78  	}
    79  
    80  	err = kpb.buildPartitionDefs(ctx, partitionBinder, partitionDef, partitionSyntaxDef.Partitions)
    81  	if err != nil {
    82  		return err
    83  	}
    84  	err = kpb.checkPartitionIntegrity(ctx, partitionBinder, tableDef, partitionDef)
    85  	if err != nil {
    86  		return err
    87  	}
    88  	err = kpb.buildEvalPartitionExpression(ctx, partitionBinder, stmt.PartitionOption, partitionDef)
    89  	if err != nil {
    90  		return err
    91  	}
    92  
    93  	partitionDef.PartitionMsg = tree.String(partitionSyntaxDef, dialect.MYSQL)
    94  	tableDef.Partition = partitionDef
    95  	return nil
    96  }
    97  
    98  func (kpb *keyPartitionBuilder) buildPartitionDefs(ctx context.Context, partitionBinder *PartitionBinder, partitionDef *plan.PartitionByDef, syntaxDefs []*tree.Partition) (err error) {
    99  	return buildPartitionDefs(ctx, partitionDef, syntaxDefs)
   100  }
   101  
   102  func (kpb *keyPartitionBuilder) checkPartitionIntegrity(ctx context.Context, partitionBinder *PartitionBinder, tableDef *TableDef, partitionDef *plan.PartitionByDef) error {
   103  	return checkPartitionIntegrity(ctx, partitionBinder, tableDef, partitionDef)
   104  }
   105  
   106  func (kpb *keyPartitionBuilder) buildEvalPartitionExpression(ctx context.Context, partitionBinder *PartitionBinder, stmt *tree.PartitionOption, partitionDef *plan.PartitionByDef) error {
   107  	partitionType := stmt.PartBy.PType.(*tree.KeyType)
   108  	//For the Key partition, convert the partition information into the expression,
   109  	//such as : abs (hash_value (expr)) % partitionNum
   110  	var astExprs []tree.Expr
   111  	keyList := partitionType.ColumnList
   112  	astExprs = make([]tree.Expr, len(keyList))
   113  	for i, expr := range keyList {
   114  		astExprs[i] = expr
   115  	}
   116  
   117  	partitionAst := genPartitionAst(astExprs, int64(partitionDef.PartitionNum))
   118  	tempExpr, err := partitionBinder.baseBindExpr(partitionAst, 0, true)
   119  	if err != nil {
   120  		return err
   121  	}
   122  	partitionExpression, err := appendCastBeforeExpr(ctx, tempExpr, plan.Type{
   123  		Id:          int32(types.T_int32),
   124  		NotNullable: true,
   125  	})
   126  	if err != nil {
   127  		return err
   128  	}
   129  	partitionDef.PartitionExpression = partitionExpression
   130  	return nil
   131  }
   132  
   133  // checkTableColumnsNotNull check unique column is `NOT NULL`
   134  func checkTableColumnsNotNull(tableDef *TableDef, columnName string) bool {
   135  	for _, coldef := range tableDef.Cols {
   136  		if strings.EqualFold(coldef.Name, columnName) {
   137  			return !coldef.Default.NullAbility
   138  		}
   139  	}
   140  	return true
   141  }
   142  
   143  // chooseAvailableUniqueKey Select an available unique index as the partitioning key
   144  func chooseAvailableUniqueKey(ctx context.Context, tableDef *TableDef, uniqueIndexs []*tree.UniqueIndex) ([]*tree.UnresolvedName, error) {
   145  	isNotNullCheckErr := false
   146  	for _, uniqueIndex := range uniqueIndexs {
   147  		uniKeyNames := make([]*tree.UnresolvedName, len(uniqueIndex.KeyParts))
   148  		isOK := true
   149  		for i, keyPart := range uniqueIndex.KeyParts {
   150  			// if the unique key column were not defined as NOT NULL, then the previous statement would fail.
   151  			// See: https://dev.mysql.com/doc/refman/8.0/en/partitioning-key.html
   152  			if ok := checkTableColumnsNotNull(tableDef, keyPart.ColName.Parts[0]); !ok {
   153  				isNotNullCheckErr = true
   154  				isOK = false
   155  			}
   156  			uniKeyNames[i] = keyPart.ColName
   157  		}
   158  		if isOK {
   159  			return uniKeyNames, nil
   160  		}
   161  	}
   162  	if isNotNullCheckErr {
   163  		return nil, moerr.NewInvalidInput(ctx, "Field in list of fields for partition function not found in table")
   164  	}
   165  	return nil, nil
   166  }