github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cmd/kafka-consumer/main.go (about)

     1  // Copyright 2020 PingCAP, Inc.
     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  // See the License for the specific language governing permissions and
    12  // limitations under the License.
    13  
    14  package main
    15  
    16  import (
    17  	"context"
    18  	"flag"
    19  	"fmt"
    20  	"math"
    21  	"net/url"
    22  	"os"
    23  	"os/signal"
    24  	"strconv"
    25  	"strings"
    26  	"sync"
    27  	"sync/atomic"
    28  	"syscall"
    29  	"time"
    30  
    31  	"github.com/Shopify/sarama"
    32  	"github.com/google/uuid"
    33  	"github.com/pingcap/errors"
    34  	"github.com/pingcap/log"
    35  	"github.com/pingcap/ticdc/cdc/model"
    36  	"github.com/pingcap/ticdc/cdc/sink"
    37  	"github.com/pingcap/ticdc/cdc/sink/codec"
    38  	"github.com/pingcap/ticdc/pkg/config"
    39  	cdcfilter "github.com/pingcap/ticdc/pkg/filter"
    40  	"github.com/pingcap/ticdc/pkg/logutil"
    41  	"github.com/pingcap/ticdc/pkg/quotes"
    42  	"github.com/pingcap/ticdc/pkg/security"
    43  	"github.com/pingcap/ticdc/pkg/util"
    44  	"go.uber.org/zap"
    45  )
    46  
    47  // Sarama configuration options
    48  var (
    49  	kafkaAddrs           []string
    50  	kafkaTopic           string
    51  	kafkaPartitionNum    int32
    52  	kafkaGroupID         = fmt.Sprintf("ticdc_kafka_consumer_%s", uuid.New().String())
    53  	kafkaVersion         = "2.4.0"
    54  	kafkaMaxMessageBytes = math.MaxInt64
    55  	kafkaMaxBatchSize    = math.MaxInt64
    56  
    57  	downstreamURIStr string
    58  
    59  	logPath       string
    60  	logLevel      string
    61  	timezone      string
    62  	ca, cert, key string
    63  )
    64  
    65  func init() {
    66  	var upstreamURIStr string
    67  
    68  	flag.StringVar(&upstreamURIStr, "upstream-uri", "", "Kafka uri")
    69  	flag.StringVar(&downstreamURIStr, "downstream-uri", "", "downstream sink uri")
    70  	flag.StringVar(&logPath, "log-file", "cdc_kafka_consumer.log", "log file path")
    71  	flag.StringVar(&logLevel, "log-level", "info", "log file path")
    72  	flag.StringVar(&timezone, "tz", "System", "Specify time zone of Kafka consumer")
    73  	flag.StringVar(&ca, "ca", "", "CA certificate path for Kafka SSL connection")
    74  	flag.StringVar(&cert, "cert", "", "Certificate path for Kafka SSL connection")
    75  	flag.StringVar(&key, "key", "", "Private key path for Kafka SSL connection")
    76  	flag.Parse()
    77  
    78  	err := logutil.InitLogger(&logutil.Config{
    79  		Level: logLevel,
    80  		File:  logPath,
    81  	})
    82  	if err != nil {
    83  		log.Fatal("init logger failed", zap.Error(err))
    84  	}
    85  
    86  	upstreamURI, err := url.Parse(upstreamURIStr)
    87  	if err != nil {
    88  		log.Fatal("invalid upstream-uri", zap.Error(err))
    89  	}
    90  	scheme := strings.ToLower(upstreamURI.Scheme)
    91  	if scheme != "kafka" {
    92  		log.Fatal("invalid upstream-uri scheme, the scheme of upstream-uri must be `kafka`", zap.String("upstream-uri", upstreamURIStr))
    93  	}
    94  	s := upstreamURI.Query().Get("version")
    95  	if s != "" {
    96  		kafkaVersion = s
    97  	}
    98  	s = upstreamURI.Query().Get("consumer-group-id")
    99  	if s != "" {
   100  		kafkaGroupID = s
   101  	}
   102  	kafkaTopic = strings.TrimFunc(upstreamURI.Path, func(r rune) bool {
   103  		return r == '/'
   104  	})
   105  	kafkaAddrs = strings.Split(upstreamURI.Host, ",")
   106  
   107  	config, err := newSaramaConfig()
   108  	if err != nil {
   109  		log.Fatal("Error creating sarama config", zap.Error(err))
   110  	}
   111  
   112  	s = upstreamURI.Query().Get("partition-num")
   113  	if s == "" {
   114  		partition, err := getPartitionNum(kafkaAddrs, kafkaTopic, config)
   115  		if err != nil {
   116  			log.Fatal("can not get partition number", zap.String("topic", kafkaTopic), zap.Error(err))
   117  		}
   118  		kafkaPartitionNum = partition
   119  	} else {
   120  		c, err := strconv.Atoi(s)
   121  		if err != nil {
   122  			log.Fatal("invalid partition-num of upstream-uri")
   123  		}
   124  		kafkaPartitionNum = int32(c)
   125  	}
   126  
   127  	s = upstreamURI.Query().Get("max-message-bytes")
   128  	if s != "" {
   129  		c, err := strconv.Atoi(s)
   130  		if err != nil {
   131  			log.Fatal("invalid max-message-bytes of upstream-uri")
   132  		}
   133  		log.Info("Setting max-message-bytes", zap.Int("max-message-bytes", c))
   134  		kafkaMaxMessageBytes = c
   135  	}
   136  
   137  	s = upstreamURI.Query().Get("max-batch-size")
   138  	if s != "" {
   139  		c, err := strconv.Atoi(s)
   140  		if err != nil {
   141  			log.Fatal("invalid max-batch-size of upstream-uri")
   142  		}
   143  		log.Info("Setting max-batch-size", zap.Int("max-batch-size", c))
   144  		kafkaMaxBatchSize = c
   145  	}
   146  }
   147  
   148  func getPartitionNum(address []string, topic string, cfg *sarama.Config) (int32, error) {
   149  	// get partition number or create topic automatically
   150  	admin, err := sarama.NewClusterAdmin(address, cfg)
   151  	if err != nil {
   152  		return 0, errors.Trace(err)
   153  	}
   154  	topics, err := admin.ListTopics()
   155  	if err != nil {
   156  		return 0, errors.Trace(err)
   157  	}
   158  	err = admin.Close()
   159  	if err != nil {
   160  		return 0, errors.Trace(err)
   161  	}
   162  	topicDetail, exist := topics[topic]
   163  	if !exist {
   164  		return 0, errors.Errorf("can not find topic %s", topic)
   165  	}
   166  	log.Info("get partition number of topic", zap.String("topic", topic), zap.Int32("partition_num", topicDetail.NumPartitions))
   167  	return topicDetail.NumPartitions, nil
   168  }
   169  
   170  func waitTopicCreated(address []string, topic string, cfg *sarama.Config) error {
   171  	admin, err := sarama.NewClusterAdmin(address, cfg)
   172  	if err != nil {
   173  		return errors.Trace(err)
   174  	}
   175  	defer admin.Close()
   176  	for i := 0; i <= 30; i++ {
   177  		topics, err := admin.ListTopics()
   178  		if err != nil {
   179  			return errors.Trace(err)
   180  		}
   181  		if _, ok := topics[topic]; ok {
   182  			return nil
   183  		}
   184  		log.Info("wait the topic created", zap.String("topic", topic))
   185  		time.Sleep(1 * time.Second)
   186  	}
   187  	return errors.Errorf("wait the topic(%s) created timeout", topic)
   188  }
   189  
   190  func newSaramaConfig() (*sarama.Config, error) {
   191  	config := sarama.NewConfig()
   192  
   193  	version, err := sarama.ParseKafkaVersion(kafkaVersion)
   194  	if err != nil {
   195  		return nil, errors.Trace(err)
   196  	}
   197  
   198  	config.ClientID = "ticdc_kafka_sarama_consumer"
   199  	config.Version = version
   200  
   201  	config.Metadata.Retry.Max = 10000
   202  	config.Metadata.Retry.Backoff = 500 * time.Millisecond
   203  	config.Consumer.Retry.Backoff = 500 * time.Millisecond
   204  	config.Consumer.Offsets.Initial = sarama.OffsetOldest
   205  
   206  	if len(ca) != 0 {
   207  		config.Net.TLS.Enable = true
   208  		config.Net.TLS.Config, err = (&security.Credential{
   209  			CAPath:   ca,
   210  			CertPath: cert,
   211  			KeyPath:  key,
   212  		}).ToTLSConfig()
   213  		if err != nil {
   214  			return nil, errors.Trace(err)
   215  		}
   216  	}
   217  
   218  	return config, err
   219  }
   220  
   221  func main() {
   222  	log.Info("Starting a new TiCDC open protocol consumer")
   223  
   224  	/**
   225  	 * Construct a new Sarama configuration.
   226  	 * The Kafka cluster version has to be defined before the consumer/producer is initialized.
   227  	 */
   228  	config, err := newSaramaConfig()
   229  	if err != nil {
   230  		log.Fatal("Error creating sarama config", zap.Error(err))
   231  	}
   232  	err = waitTopicCreated(kafkaAddrs, kafkaTopic, config)
   233  	if err != nil {
   234  		log.Fatal("wait topic created failed", zap.Error(err))
   235  	}
   236  	/**
   237  	 * Setup a new Sarama consumer group
   238  	 */
   239  	consumer, err := NewConsumer(context.TODO())
   240  	if err != nil {
   241  		log.Fatal("Error creating consumer", zap.Error(err))
   242  	}
   243  
   244  	ctx, cancel := context.WithCancel(context.Background())
   245  	client, err := sarama.NewConsumerGroup(kafkaAddrs, kafkaGroupID, config)
   246  	if err != nil {
   247  		log.Fatal("Error creating consumer group client", zap.Error(err))
   248  	}
   249  
   250  	wg := &sync.WaitGroup{}
   251  	wg.Add(1)
   252  	go func() {
   253  		defer wg.Done()
   254  		for {
   255  			// `Consume` should be called inside an infinite loop, when a
   256  			// server-side rebalance happens, the consumer session will need to be
   257  			// recreated to get the new claims
   258  			if err := client.Consume(ctx, strings.Split(kafkaTopic, ","), consumer); err != nil {
   259  				log.Fatal("Error from consumer: %v", zap.Error(err))
   260  			}
   261  			// check if context was cancelled, signaling that the consumer should stop
   262  			if ctx.Err() != nil {
   263  				return
   264  			}
   265  			consumer.ready = make(chan bool)
   266  		}
   267  	}()
   268  
   269  	go func() {
   270  		if err := consumer.Run(ctx); err != nil {
   271  			log.Fatal("Error running consumer: %v", zap.Error(err))
   272  		}
   273  	}()
   274  
   275  	<-consumer.ready // Await till the consumer has been set up
   276  	log.Info("TiCDC open protocol consumer up and running!...")
   277  
   278  	sigterm := make(chan os.Signal, 1)
   279  	signal.Notify(sigterm, syscall.SIGINT, syscall.SIGTERM)
   280  	select {
   281  	case <-ctx.Done():
   282  		log.Info("terminating: context cancelled")
   283  	case <-sigterm:
   284  		log.Info("terminating: via signal")
   285  	}
   286  	cancel()
   287  	wg.Wait()
   288  	if err = client.Close(); err != nil {
   289  		log.Fatal("Error closing client", zap.Error(err))
   290  	}
   291  }
   292  
   293  // Consumer represents a Sarama consumer group consumer
   294  type Consumer struct {
   295  	ready chan bool
   296  
   297  	ddlList          []*model.DDLEvent
   298  	maxDDLReceivedTs uint64
   299  	ddlListMu        sync.Mutex
   300  
   301  	sinks []*struct {
   302  		sink.Sink
   303  		resolvedTs uint64
   304  	}
   305  	sinksMu sync.Mutex
   306  
   307  	ddlSink              sink.Sink
   308  	fakeTableIDGenerator *fakeTableIDGenerator
   309  
   310  	globalResolvedTs uint64
   311  }
   312  
   313  // NewConsumer creates a new cdc kafka consumer
   314  func NewConsumer(ctx context.Context) (*Consumer, error) {
   315  	// TODO support filter in downstream sink
   316  	tz, err := util.GetTimezone(timezone)
   317  	if err != nil {
   318  		return nil, errors.Annotate(err, "can not load timezone")
   319  	}
   320  	ctx = util.PutTimezoneInCtx(ctx, tz)
   321  	filter, err := cdcfilter.NewFilter(config.GetDefaultReplicaConfig())
   322  	if err != nil {
   323  		return nil, errors.Trace(err)
   324  	}
   325  	c := new(Consumer)
   326  	c.fakeTableIDGenerator = &fakeTableIDGenerator{
   327  		tableIDs: make(map[string]int64),
   328  	}
   329  	c.sinks = make([]*struct {
   330  		sink.Sink
   331  		resolvedTs uint64
   332  	}, kafkaPartitionNum)
   333  	ctx, cancel := context.WithCancel(ctx)
   334  	errCh := make(chan error, 1)
   335  	opts := map[string]string{}
   336  	for i := 0; i < int(kafkaPartitionNum); i++ {
   337  		s, err := sink.NewSink(ctx, "kafka-consumer", downstreamURIStr, filter, config.GetDefaultReplicaConfig(), opts, errCh)
   338  		if err != nil {
   339  			cancel()
   340  			return nil, errors.Trace(err)
   341  		}
   342  		c.sinks[i] = &struct {
   343  			sink.Sink
   344  			resolvedTs uint64
   345  		}{Sink: s}
   346  	}
   347  	sink, err := sink.NewSink(ctx, "kafka-consumer", downstreamURIStr, filter, config.GetDefaultReplicaConfig(), opts, errCh)
   348  	if err != nil {
   349  		cancel()
   350  		return nil, errors.Trace(err)
   351  	}
   352  	go func() {
   353  		err := <-errCh
   354  		if errors.Cause(err) != context.Canceled {
   355  			log.Error("error on running consumer", zap.Error(err))
   356  		} else {
   357  			log.Info("consumer exited")
   358  		}
   359  		cancel()
   360  	}()
   361  	c.ddlSink = sink
   362  	c.ready = make(chan bool)
   363  	return c, nil
   364  }
   365  
   366  // Setup is run at the beginning of a new session, before ConsumeClaim
   367  func (c *Consumer) Setup(sarama.ConsumerGroupSession) error {
   368  	// Mark the c as ready
   369  	close(c.ready)
   370  	return nil
   371  }
   372  
   373  // Cleanup is run at the end of a session, once all ConsumeClaim goroutines have exited
   374  func (c *Consumer) Cleanup(sarama.ConsumerGroupSession) error {
   375  	return nil
   376  }
   377  
   378  // ConsumeClaim must start a consumer loop of ConsumerGroupClaim's Messages().
   379  func (c *Consumer) ConsumeClaim(session sarama.ConsumerGroupSession, claim sarama.ConsumerGroupClaim) error {
   380  	ctx := context.TODO()
   381  	partition := claim.Partition()
   382  	c.sinksMu.Lock()
   383  	sink := c.sinks[partition]
   384  	c.sinksMu.Unlock()
   385  	if sink == nil {
   386  		panic("sink should initialized")
   387  	}
   388  ClaimMessages:
   389  	for message := range claim.Messages() {
   390  		log.Info("Message claimed", zap.Int32("partition", message.Partition), zap.ByteString("key", message.Key), zap.ByteString("value", message.Value))
   391  		batchDecoder, err := codec.NewJSONEventBatchDecoder(message.Key, message.Value)
   392  		if err != nil {
   393  			return errors.Trace(err)
   394  		}
   395  
   396  		counter := 0
   397  		for {
   398  			tp, hasNext, err := batchDecoder.HasNext()
   399  			if err != nil {
   400  				log.Fatal("decode message key failed", zap.Error(err))
   401  			}
   402  			if !hasNext {
   403  				break
   404  			}
   405  
   406  			counter++
   407  			// If the message containing only one event exceeds the length limit, CDC will allow it and issue a warning.
   408  			if len(message.Key)+len(message.Value) > kafkaMaxMessageBytes && counter > 1 {
   409  				log.Fatal("kafka max-messages-bytes exceeded", zap.Int("max-message-bytes", kafkaMaxMessageBytes),
   410  					zap.Int("recevied-bytes", len(message.Key)+len(message.Value)))
   411  			}
   412  
   413  			switch tp {
   414  			case model.MqMessageTypeDDL:
   415  				ddl, err := batchDecoder.NextDDLEvent()
   416  				if err != nil {
   417  					log.Fatal("decode message value failed", zap.ByteString("value", message.Value))
   418  				}
   419  				c.appendDDL(ddl)
   420  			case model.MqMessageTypeRow:
   421  				row, err := batchDecoder.NextRowChangedEvent()
   422  				if err != nil {
   423  					log.Fatal("decode message value failed", zap.ByteString("value", message.Value))
   424  				}
   425  				globalResolvedTs := atomic.LoadUint64(&c.globalResolvedTs)
   426  				if row.CommitTs <= globalResolvedTs || row.CommitTs <= sink.resolvedTs {
   427  					log.Debug("filter fallback row", zap.ByteString("row", message.Key),
   428  						zap.Uint64("globalResolvedTs", globalResolvedTs),
   429  						zap.Uint64("sinkResolvedTs", sink.resolvedTs),
   430  						zap.Int32("partition", partition))
   431  					break ClaimMessages
   432  				}
   433  				// FIXME: hack to set start-ts in row changed event, as start-ts
   434  				// is not contained in TiCDC open protocol
   435  				row.StartTs = row.CommitTs
   436  				var partitionID int64
   437  				if row.Table.IsPartition {
   438  					partitionID = row.Table.TableID
   439  				}
   440  				row.Table.TableID =
   441  					c.fakeTableIDGenerator.generateFakeTableID(row.Table.Schema, row.Table.Table, partitionID)
   442  				err = sink.EmitRowChangedEvents(ctx, row)
   443  				if err != nil {
   444  					log.Fatal("emit row changed event failed", zap.Error(err))
   445  				}
   446  			case model.MqMessageTypeResolved:
   447  				ts, err := batchDecoder.NextResolvedEvent()
   448  				if err != nil {
   449  					log.Fatal("decode message value failed", zap.ByteString("value", message.Value))
   450  				}
   451  				resolvedTs := atomic.LoadUint64(&sink.resolvedTs)
   452  				if resolvedTs < ts {
   453  					log.Debug("update sink resolved ts",
   454  						zap.Uint64("ts", ts),
   455  						zap.Int32("partition", partition))
   456  					atomic.StoreUint64(&sink.resolvedTs, ts)
   457  				}
   458  			}
   459  			session.MarkMessage(message, "")
   460  		}
   461  
   462  		if counter > kafkaMaxBatchSize {
   463  			log.Fatal("Open Protocol max-batch-size exceeded", zap.Int("max-batch-size", kafkaMaxBatchSize),
   464  				zap.Int("actual-batch-size", counter))
   465  		}
   466  	}
   467  
   468  	return nil
   469  }
   470  
   471  func (c *Consumer) appendDDL(ddl *model.DDLEvent) {
   472  	c.ddlListMu.Lock()
   473  	defer c.ddlListMu.Unlock()
   474  	if ddl.CommitTs <= c.maxDDLReceivedTs {
   475  		return
   476  	}
   477  	globalResolvedTs := atomic.LoadUint64(&c.globalResolvedTs)
   478  	if ddl.CommitTs <= globalResolvedTs {
   479  		log.Error("unexpected ddl job", zap.Uint64("ddlts", ddl.CommitTs), zap.Uint64("globalResolvedTs", globalResolvedTs))
   480  		return
   481  	}
   482  	c.ddlList = append(c.ddlList, ddl)
   483  	c.maxDDLReceivedTs = ddl.CommitTs
   484  }
   485  
   486  func (c *Consumer) getFrontDDL() *model.DDLEvent {
   487  	c.ddlListMu.Lock()
   488  	defer c.ddlListMu.Unlock()
   489  	if len(c.ddlList) > 0 {
   490  		return c.ddlList[0]
   491  	}
   492  	return nil
   493  }
   494  
   495  func (c *Consumer) popDDL() *model.DDLEvent {
   496  	c.ddlListMu.Lock()
   497  	defer c.ddlListMu.Unlock()
   498  	if len(c.ddlList) > 0 {
   499  		ddl := c.ddlList[0]
   500  		c.ddlList = c.ddlList[1:]
   501  		return ddl
   502  	}
   503  	return nil
   504  }
   505  
   506  func (c *Consumer) forEachSink(fn func(sink *struct {
   507  	sink.Sink
   508  	resolvedTs uint64
   509  }) error) error {
   510  	c.sinksMu.Lock()
   511  	defer c.sinksMu.Unlock()
   512  	for _, sink := range c.sinks {
   513  		if err := fn(sink); err != nil {
   514  			return errors.Trace(err)
   515  		}
   516  	}
   517  	return nil
   518  }
   519  
   520  // Run runs the Consumer
   521  func (c *Consumer) Run(ctx context.Context) error {
   522  	var lastGlobalResolvedTs uint64
   523  	for {
   524  		select {
   525  		case <-ctx.Done():
   526  			return ctx.Err()
   527  		default:
   528  		}
   529  		time.Sleep(100 * time.Millisecond)
   530  		// handle ddl
   531  		globalResolvedTs := uint64(math.MaxUint64)
   532  		err := c.forEachSink(func(sink *struct {
   533  			sink.Sink
   534  			resolvedTs uint64
   535  		}) error {
   536  			resolvedTs := atomic.LoadUint64(&sink.resolvedTs)
   537  			if resolvedTs < globalResolvedTs {
   538  				globalResolvedTs = resolvedTs
   539  			}
   540  			return nil
   541  		})
   542  		if err != nil {
   543  			return errors.Trace(err)
   544  		}
   545  		todoDDL := c.getFrontDDL()
   546  		if todoDDL != nil && globalResolvedTs >= todoDDL.CommitTs {
   547  			// flush DMLs
   548  			err := c.forEachSink(func(sink *struct {
   549  				sink.Sink
   550  				resolvedTs uint64
   551  			}) error {
   552  				return syncFlushRowChangedEvents(ctx, sink, todoDDL.CommitTs)
   553  			})
   554  			if err != nil {
   555  				return errors.Trace(err)
   556  			}
   557  
   558  			// execute ddl
   559  			err = c.ddlSink.EmitDDLEvent(ctx, todoDDL)
   560  			if err != nil {
   561  				return errors.Trace(err)
   562  			}
   563  			c.popDDL()
   564  			continue
   565  		}
   566  
   567  		if todoDDL != nil && todoDDL.CommitTs < globalResolvedTs {
   568  			globalResolvedTs = todoDDL.CommitTs
   569  		}
   570  		if lastGlobalResolvedTs == globalResolvedTs {
   571  			continue
   572  		}
   573  		lastGlobalResolvedTs = globalResolvedTs
   574  		atomic.StoreUint64(&c.globalResolvedTs, globalResolvedTs)
   575  		log.Info("update globalResolvedTs", zap.Uint64("ts", globalResolvedTs))
   576  
   577  		err = c.forEachSink(func(sink *struct {
   578  			sink.Sink
   579  			resolvedTs uint64
   580  		}) error {
   581  			return syncFlushRowChangedEvents(ctx, sink, globalResolvedTs)
   582  		})
   583  		if err != nil {
   584  			return errors.Trace(err)
   585  		}
   586  	}
   587  }
   588  
   589  func syncFlushRowChangedEvents(ctx context.Context, sink sink.Sink, resolvedTs uint64) error {
   590  	for {
   591  		select {
   592  		case <-ctx.Done():
   593  			return ctx.Err()
   594  		default:
   595  		}
   596  		checkpointTs, err := sink.FlushRowChangedEvents(ctx, resolvedTs)
   597  		if err != nil {
   598  			return err
   599  		}
   600  		if checkpointTs >= resolvedTs {
   601  			return nil
   602  		}
   603  	}
   604  }
   605  
   606  type fakeTableIDGenerator struct {
   607  	tableIDs       map[string]int64
   608  	currentTableID int64
   609  	mu             sync.Mutex
   610  }
   611  
   612  func (g *fakeTableIDGenerator) generateFakeTableID(schema, table string, partition int64) int64 {
   613  	g.mu.Lock()
   614  	defer g.mu.Unlock()
   615  	key := quotes.QuoteSchema(schema, table)
   616  	if partition != 0 {
   617  		key = fmt.Sprintf("%s.`%d`", key, partition)
   618  	}
   619  	if tableID, ok := g.tableIDs[key]; ok {
   620  		return tableID
   621  	}
   622  	g.currentTableID++
   623  	g.tableIDs[key] = g.currentTableID
   624  	return g.currentTableID
   625  }