Skip to content

Commit

Permalink
Merge branch 'fix-gcs-test' of https://github.com/joccau/tidb into fi…
Browse files Browse the repository at this point in the history
…x-gcs-test
  • Loading branch information
joccau committed Apr 8, 2022
2 parents cb1bc19 + d36e6d5 commit d350efe
Show file tree
Hide file tree
Showing 167 changed files with 16,143 additions and 12,126 deletions.
4 changes: 3 additions & 1 deletion .github/pull_request_template.md
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,9 @@ Documentation

### Release note

<!-- bugfix or new feature needs a release note -->
<!-- compatibility change, improvement, bugfix, and new feature need a release note -->

Please refer to [Release Notes Language Style Guide](https://pingcap.github.io/tidb-dev-guide/contribute-to-tidb/release-notes-style-guide.html) to write a quality release note.

```release-note
None
Expand Down
5 changes: 5 additions & 0 deletions .golangci.yml
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,11 @@ linters-settings:
checks: ["S1002","S1004","S1007","S1009","S1010","S1012","S1019","S1020","S1021","S1024","S1030","SA2*","SA3*","SA4009","SA5*","SA6000","SA6001","SA6005", "-SA2002"]
stylecheck:
checks: ["-ST1003"]
gosec:
severity: "low"
confidence: "low"
excludes:
- G101
issues:
exclude-rules:
- path: _test\.go
Expand Down
6 changes: 4 additions & 2 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -145,9 +145,11 @@ gotest_unstable_in_verify_ci: tools/bin/xprog tools/bin/ut failpoint-enable
@$(CLEAN_UT_BINARY)

race: failpoint-enable
@export log_level=debug; \
$(GOTEST) -timeout 25m -race $(PACKAGES) || { $(FAILPOINT_DISABLE); exit 1; }
@mkdir -p $(TEST_COVERAGE_DIR)
@export TZ='Asia/Shanghai'; \
tools/bin/ut --race --junitfile "$(TEST_COVERAGE_DIR)/tidb-junit-report.xml" --coverprofile "$(TEST_COVERAGE_DIR)/tidb_cov.unit_test" --except unstable.txt || { $(FAILPOINT_DISABLE); exit 1; }
@$(FAILPOINT_DISABLE)
@$(CLEAN_UT_BINARY)

leak: failpoint-enable
@export log_level=debug; \
Expand Down
4 changes: 2 additions & 2 deletions README.md
Original file line number Diff line number Diff line change
Expand Up @@ -62,9 +62,9 @@ For support, please contact [PingCAP](http://bit.ly/contact_us_via_github).

### To start using TiDB Cloud

We provide TiDB Cloud - a fully-managed Database as a Service for you.
We provide TiDB Cloud - a fully-managed Database as a Service for you. You can [sign up](https://tidbcloud.com/signup) and get started with TiDB Cloud Developer Tier for free.

See [TiDB Cloud Quick Start](https://docs.pingcap.com/tidbcloud/public-preview/tidb-cloud-quickstart).
See [TiDB Cloud Quick Start](https://docs.pingcap.com/tidbcloud/tidb-cloud-quickstart).

### To start using TiDB

Expand Down
8 changes: 8 additions & 0 deletions bindinfo/bind_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -76,6 +76,14 @@ func (msm *mockSessionManager1) ServerID() uint64 {
return 1
}

func (msm *mockSessionManager1) StoreInternalSession(se interface{}) {}

func (msm *mockSessionManager1) DeleteInternalSession(se interface{}) {}

func (msm *mockSessionManager1) GetInternalSessionStartTSList() []uint64 {
return nil
}

func TestPrepareCacheWithBinding(t *testing.T) {
store, clean := testkit.CreateMockStore(t)
defer clean()
Expand Down
8 changes: 8 additions & 0 deletions bindinfo/session_handle_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -406,6 +406,14 @@ func (msm *mockSessionManager) ServerID() uint64 {
return 1
}

func (msm *mockSessionManager) StoreInternalSession(se interface{}) {}

func (msm *mockSessionManager) DeleteInternalSession(se interface{}) {}

func (msm *mockSessionManager) GetInternalSessionStartTSList() []uint64 {
return nil
}

func TestIssue19836(t *testing.T) {
store, clean := testkit.CreateMockStore(t)
defer clean()
Expand Down
16 changes: 13 additions & 3 deletions br/pkg/lightning/backend/local/engine.go
Original file line number Diff line number Diff line change
Expand Up @@ -426,8 +426,15 @@ func getSizeProperties(logger log.Logger, db *pebble.DB, keyAdapter KeyAdapter)
}

func (e *Engine) getEngineFileSize() backend.EngineFileSize {
metrics := e.db.Metrics()
total := metrics.Total()
e.mutex.RLock()
db := e.db
e.mutex.RUnlock()

var total pebble.LevelMetrics
if db != nil {
metrics := db.Metrics()
total = metrics.Total()
}
var memSize int64
e.localWriters.Range(func(k, v interface{}) bool {
w := k.(*Writer)
Expand Down Expand Up @@ -524,7 +531,6 @@ func (e *Engine) ingestSSTLoop() {
for i := 0; i < concurrency; i++ {
e.wg.Add(1)
go func() {
defer e.wg.Done()
defer func() {
if e.ingestErr.Get() != nil {
seqLock.Lock()
Expand All @@ -534,6 +540,7 @@ func (e *Engine) ingestSSTLoop() {
flushQueue = flushQueue[:0]
seqLock.Unlock()
}
e.wg.Done()
}()
for {
select {
Expand Down Expand Up @@ -1471,5 +1478,8 @@ func (i dbSSTIngester) ingest(metas []*sstMeta) error {
for _, m := range metas {
paths = append(paths, m.path)
}
if i.e.db == nil {
return errorEngineClosed
}
return i.e.db.Ingest(paths)
}
85 changes: 85 additions & 0 deletions br/pkg/lightning/backend/local/engine_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,85 @@
// Copyright 2022 PingCAP, Inc.
//
// 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 local

import (
"context"
"fmt"
"math"
"os"
"path"
"path/filepath"
"testing"

"github.com/cockroachdb/pebble"
"github.com/cockroachdb/pebble/sstable"
"github.com/google/uuid"
"github.com/stretchr/testify/require"

"github.com/pingcap/tidb/br/pkg/lightning/backend"
)

func TestIngestSSTWithClosedEngine(t *testing.T) {
dir := t.TempDir()
opt := &pebble.Options{
MemTableSize: 1024 * 1024,
MaxConcurrentCompactions: 16,
L0CompactionThreshold: math.MaxInt32, // set to max try to disable compaction
L0StopWritesThreshold: math.MaxInt32, // set to max try to disable compaction
DisableWAL: true,
ReadOnly: false,
}
db, err := pebble.Open(filepath.Join(dir, "test"), opt)
require.NoError(t, err)
tmpPath := filepath.Join(dir, "test.sst")
err = os.Mkdir(tmpPath, 0o755)
require.NoError(t, err)

_, engineUUID := backend.MakeUUID("ww", 0)
engineCtx, cancel := context.WithCancel(context.Background())
f := &Engine{
db: db,
UUID: engineUUID,
sstDir: tmpPath,
ctx: engineCtx,
cancel: cancel,
sstMetasChan: make(chan metaOrFlush, 64),
keyAdapter: noopKeyAdapter{},
}
f.sstIngester = dbSSTIngester{e: f}
sstPath := path.Join(tmpPath, uuid.New().String()+".sst")
file, err := os.Create(sstPath)
require.NoError(t, err)
w := sstable.NewWriter(file, sstable.WriterOptions{})
for i := 0; i < 10; i++ {
require.NoError(t, w.Add(sstable.InternalKey{
Trailer: uint64(sstable.InternalKeyKindSet),
UserKey: []byte(fmt.Sprintf("key%d", i)),
}, nil))
}
require.NoError(t, w.Close())

require.NoError(t, f.ingestSSTs([]*sstMeta{
{
path: sstPath,
},
}))
require.NoError(t, f.Close())
require.ErrorIs(t, f.ingestSSTs([]*sstMeta{
{
path: sstPath,
},
}), errorEngineClosed)
}
2 changes: 1 addition & 1 deletion br/pkg/lightning/restore/restore.go
Original file line number Diff line number Diff line change
Expand Up @@ -972,7 +972,7 @@ func (rc *Controller) saveStatusCheckpoint(ctx context.Context, tableName string
switch {
case err == nil:
break
case utils.IsRetryableError(err), utils.MessageIsRetryableStorageError(err.Error()), common.IsContextCanceledError(err):
case utils.MessageIsRetryableStorageError(err.Error()), common.IsContextCanceledError(err):
// recoverable error, should not be recorded in checkpoint
// which will prevent lightning from automatically recovering
return nil
Expand Down
10 changes: 7 additions & 3 deletions br/pkg/lightning/restore/table_restore_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@ import (
"context"
"encoding/json"
"fmt"
"net"
"net/http"
"net/http/httptest"
"os"
Expand Down Expand Up @@ -996,9 +995,14 @@ func (s *tableRestoreSuite) TestSaveStatusCheckpoint() {
require.NoError(s.T(), err)
require.Equal(s.T(), 0, len(rc.errorSummaries.summary))

err = rc.saveStatusCheckpoint(context.Background(), common.UniqueTable("test", "tbl"), indexEngineID, &net.DNSError{IsTimeout: true}, checkpoints.CheckpointStatusImported)
err = rc.saveStatusCheckpoint(
context.Background(),
common.UniqueTable("test", "tbl"), indexEngineID,
common.ErrChecksumMismatch.GenWithStackByArgs(0, 0, 0, 0, 0, 0),
checkpoints.CheckpointStatusImported,
)
require.NoError(s.T(), err)
require.Equal(s.T(), 0, len(rc.errorSummaries.summary))
require.Equal(s.T(), 1, len(rc.errorSummaries.summary))

start := time.Now()
err = rc.saveStatusCheckpoint(context.Background(), common.UniqueTable("test", "tbl"), indexEngineID, nil, checkpoints.CheckpointStatusImported)
Expand Down
Loading

0 comments on commit d350efe

Please sign in to comment.