Skip to content

Commit

Permalink
Merge #55973
Browse files Browse the repository at this point in the history
55973: backupccl: do not re-create schema change jobs in cluster restore r=pbardea a=pbardea

This commit fixes a bug where schema change jobs that were in progress
during a cluster backup would be re-created twice during a cluster
restore.

Release note (bug fix): Previously if a cluster backup that was taken
during a schema change, a cluster restore of that backup would create
duplicates of the ongoing schema changes.

Co-authored-by: Paul Bardea <pbardea@gmail.com>
  • Loading branch information
craig[bot] and pbardea committed Oct 26, 2020
2 parents a2db135 + 900cebb commit f79ebfc
Show file tree
Hide file tree
Showing 509 changed files with 207 additions and 86 deletions.
21 changes: 13 additions & 8 deletions pkg/ccl/backupccl/restore_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -1369,14 +1369,19 @@ func (r *restoreResumer) publishDescriptors(
}
allMutDescs = append(allMutDescs, mutTable)
newTables = append(newTables, mutTable.TableDesc())
// Convert any mutations that were in progress on the table descriptor
// when the backup was taken, and convert them to schema change jobs.
newJobs, err := createSchemaChangeJobsFromMutations(ctx,
r.execCfg.JobRegistry, r.execCfg.Codec, txn, r.job.Payload().Username, mutTable)
if err != nil {
return newDescriptorChangeJobs, err
// For cluster restores, all the jobs are restored directly from the jobs
// table, so there is no need to re-create ongoing schema change jobs,
// otherwise we'll create duplicate jobs.
if details.DescriptorCoverage != tree.AllDescriptors {
// Convert any mutations that were in progress on the table descriptor
// when the backup was taken, and convert them to schema change jobs.
newJobs, err := createSchemaChangeJobsFromMutations(ctx,
r.execCfg.JobRegistry, r.execCfg.Codec, txn, r.job.Payload().Username, mutTable)
if err != nil {
return newDescriptorChangeJobs, err
}
newDescriptorChangeJobs = append(newDescriptorChangeJobs, newJobs...)
}
newDescriptorChangeJobs = append(newDescriptorChangeJobs, newJobs...)
}
// For all of the newly created types, make type schema change jobs for any
// type descriptors that were backed up in the middle of a type schema change.
Expand All @@ -1390,7 +1395,7 @@ func (r *restoreResumer) publishDescriptors(
}
allMutDescs = append(allMutDescs, typ)
newTypes = append(newTypes, typ.TypeDesc())
if typ.HasPendingSchemaChanges() {
if typ.HasPendingSchemaChanges() && details.DescriptorCoverage != tree.AllDescriptors {
typJob, err := createTypeChangeJobFromDesc(ctx, r.execCfg.JobRegistry, r.execCfg.Codec, txn, r.job.Payload().Username, typ)
if err != nil {
return newDescriptorChangeJobs, err
Expand Down
234 changes: 157 additions & 77 deletions pkg/ccl/backupccl/restore_mid_schema_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,13 +14,14 @@ import (
"io/ioutil"
"os"
"path/filepath"
"strconv"
"testing"
"time"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/stretchr/testify/require"
Expand All @@ -30,8 +31,8 @@ import (
// already constructed and store in
// ccl/backupccl/testdata/restore_mid_schema_change. These backups were taken on
// tables that were in the process of performing a schema change. In particular,
// the schema changes were temporarily blocked after it completed its backfill
// stage.
// the schema changes were temporarily blocked either before of after it
// completed its backfill stage.
//
// This test ensures that these BACKUPS can be:
// 1) Restore
Expand All @@ -44,102 +45,181 @@ import (
// The test cases are organized based on cluster version of the cluster that
// took the BACKUP. Each test-case represents a BACKUP. They were created using
// the statements provided in the create.sql file in the appropriate testdata
// dir. All backups backed up defaultdb.*, which contain the relevant tables.
// Most backups contain a single table whose name matches the backup name. If
// the backup is expected to contain several tables, the table names will be
// backupName1, backupName2, ...
// dir. All backups backed up either to defaultdb.*, which contain the relevant
// tables or a cluster backup. Most backups contain a single table whose name
// matches the backup name. If the backup is expected to contain several tables,
// the table names will be backupName1, backupName2, ...
func TestRestoreMidSchemaChange(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
const (
testdataBase = "testdata/restore_mid_schema_change"
exportDirs = testdataBase + "/exports"
)
versionDirs, err := ioutil.ReadDir(exportDirs)
require.NoError(t, err)
for _, clusterVersionDir := range versionDirs {
require.True(t, clusterVersionDir.IsDir())
fullClusterVersionDir, err := filepath.Abs(filepath.Join(exportDirs, clusterVersionDir.Name()))
require.NoError(t, err)
backupDirs, err := ioutil.ReadDir(fullClusterVersionDir)
require.NoError(t, err)
// In each version folder (e.g. "19.2", "20.1"), there is a backup for each schema change.
for _, backupDir := range backupDirs {
fullBackupDir, err := filepath.Abs(filepath.Join(fullClusterVersionDir, backupDir.Name()))
require.NoError(t, err)
t.Run(clusterVersionDir.Name()+"-"+backupDir.Name(), restoreMidSchemaChange(fullBackupDir, backupDir.Name()))
for _, isClusterRestore := range []bool{true, false} {
name := "table"
if isClusterRestore {
name = "cluster"
}
t.Run(name, func(t *testing.T) {
// blockLocations indicates whether the backup taken was blocked before or
// after the backfill portion of the schema change.
for _, blockLocation := range []string{"before", "after"} {
t.Run(blockLocation, func(t *testing.T) {
versionDirs, err := ioutil.ReadDir(filepath.Join(exportDirs, blockLocation))
require.NoError(t, err)

for _, clusterVersionDir := range versionDirs {
if clusterVersionDir.Name() == "19.2" && isClusterRestore {
// 19.2 does not support cluster backups.
continue
}

t.Run(clusterVersionDir.Name(), func(t *testing.T) {
require.True(t, clusterVersionDir.IsDir())
fullClusterVersionDir, err := filepath.Abs(
filepath.Join(exportDirs, blockLocation, clusterVersionDir.Name()))
require.NoError(t, err)

// In each version folder (e.g. "19.2", "20.1"), there is a backup for
// each schema change.
backupDirs, err := ioutil.ReadDir(fullClusterVersionDir)
require.NoError(t, err)

for _, backupDir := range backupDirs {
fullBackupDir, err := filepath.Abs(filepath.Join(fullClusterVersionDir, backupDir.Name()))
require.NoError(t, err)
t.Run(backupDir.Name(), restoreMidSchemaChange(fullBackupDir, backupDir.Name(), isClusterRestore))
}
})
}
})
}
})
}
}

func restoreMidSchemaChange(backupDir, schemaChangeName string) func(t *testing.T) {
verify := func(t *testing.T, scName string, sqlDB *sqlutils.SQLRunner) {
var expectedData [][]string
tableName := fmt.Sprintf("defaultdb.%s", scName)
sqlDB.CheckQueryResultsRetry(t, "SELECT * FROM crdb_internal.jobs WHERE job_type = 'SCHEMA CHANGE' AND status <> 'succeeded'", [][]string{})
numSchemaChangeJobs := 1
// This enumerates the tests cases and specifies how each case should be
// handled.
switch scName {
case "midaddcol":
expectedData = [][]string{{"1", "1.3"}, {"2", "1.3"}, {"3", "1.3"}}
case "midaddconst":
expectedData = [][]string{{"1"}, {"2"}, {"3"}}
sqlDB.CheckQueryResults(t, "SELECT count(*) FROM [SHOW CONSTRAINTS FROM defaultdb.midaddconst] WHERE constraint_name = 'my_const'", [][]string{{"1"}})
case "midaddindex":
expectedData = [][]string{{"1"}, {"2"}, {"3"}}
sqlDB.CheckQueryResults(t, "SELECT count(*) FROM [SHOW INDEXES FROM defaultdb.midaddindex] WHERE column_name = 'a'", [][]string{{"1"}})
case "middropcol":
expectedData = [][]string{{"1"}, {"1"}, {"1"}, {"2"}, {"2"}, {"2"}, {"3"}, {"3"}, {"3"}}
case "midmany":
numSchemaChangeJobs = 3
expectedData = [][]string{{"1", "1.3"}, {"2", "1.3"}, {"3", "1.3"}}
sqlDB.CheckQueryResults(t, "SELECT count(*) FROM [SHOW CONSTRAINTS FROM defaultdb.midmany] WHERE constraint_name = 'my_const'", [][]string{{"1"}})
sqlDB.CheckQueryResults(t, "SELECT count(*) FROM [SHOW INDEXES FROM defaultdb.midmany] WHERE column_name = 'a'", [][]string{{"1"}})
case "midmultitxn":
expectedData = [][]string{{"1", "1.3"}, {"2", "1.3"}, {"3", "1.3"}}
sqlDB.CheckQueryResults(t, "SELECT count(*) FROM [SHOW CONSTRAINTS FROM defaultdb.midmultitxn] WHERE constraint_name = 'my_const'", [][]string{{"1"}})
sqlDB.CheckQueryResults(t, "SELECT count(*) FROM [SHOW INDEXES FROM defaultdb.midmultitxn] WHERE column_name = 'a'", [][]string{{"1"}})
case "midmultitable":
numSchemaChangeJobs = 2
expectedData = [][]string{{"1", "1.3"}, {"2", "1.3"}, {"3", "1.3"}}
sqlDB.CheckQueryResults(t, fmt.Sprintf("SELECT * FROM %s1", tableName), expectedData)
expectedData = [][]string{{"1"}, {"2"}, {"3"}}
sqlDB.CheckQueryResults(t, fmt.Sprintf("SELECT * FROM %s2", tableName), expectedData)
tableName += "1"
case "midprimarykeyswap":
// The primary key swap will also create a cleanup job.
numSchemaChangeJobs = 2
expectedData = [][]string{{"1"}, {"2"}, {"3"}}
case "midprimarykeyswapcleanup":
// This backup only contains the cleanup job mentioned above.
expectedData = [][]string{{"1"}, {"2"}, {"3"}}
}
if scName != "midmultitable" {
sqlDB.CheckQueryResults(t, fmt.Sprintf("SELECT * FROM %s", tableName), expectedData)
}
sqlDB.CheckQueryResults(t, "SELECT count(*) FROM crdb_internal.jobs WHERE job_type = 'SCHEMA CHANGE'", [][]string{{strconv.Itoa(numSchemaChangeJobs)}})
// Ensure that a schema change can complete on the restored table.
schemaChangeQuery := fmt.Sprintf("ALTER TABLE %s ADD CONSTRAINT post_restore_const CHECK (a > 0)", tableName)
sqlDB.Exec(t, schemaChangeQuery)
func verifyMidSchemaChange(
t *testing.T, scName string, sqlDB *sqlutils.SQLRunner, isClusterRestore bool,
) {
var expectedData [][]string
tableName := fmt.Sprintf("defaultdb.%s", scName)
// numJobsInCluster is the number of completed jobs that will be restored
// during a cluster restore.
var numJobsInCluster int
expNumSchemaChangeJobs := 1
// This enumerates the tests cases and specifies how each case should be
// handled.
switch scName {
case "midaddcol":
numJobsInCluster = 1 // the CREATE TABLE job
expectedData = [][]string{{"1", "1.3"}, {"2", "1.3"}, {"3", "1.3"}}
case "midaddconst":
numJobsInCluster = 1 // the CREATE TABLE job
expectedData = [][]string{{"1"}, {"2"}, {"3"}}
sqlDB.CheckQueryResults(t, "SELECT count(*) FROM [SHOW CONSTRAINTS FROM defaultdb.midaddconst] WHERE constraint_name = 'my_const'", [][]string{{"1"}})
case "midaddindex":
numJobsInCluster = 1 // the CREATE TABLE job
expectedData = [][]string{{"1"}, {"2"}, {"3"}}
sqlDB.CheckQueryResults(t, "SELECT count(*) FROM [SHOW INDEXES FROM defaultdb.midaddindex] WHERE column_name = 'a'", [][]string{{"1"}})
case "middropcol":
numJobsInCluster = 1 // the CREATE TABLE job
expectedData = [][]string{{"1"}, {"1"}, {"1"}, {"2"}, {"2"}, {"2"}, {"3"}, {"3"}, {"3"}}
case "midmany":
numJobsInCluster = 1 // the CREATE TABLE job
expNumSchemaChangeJobs = 3
expectedData = [][]string{{"1", "1.3"}, {"2", "1.3"}, {"3", "1.3"}}
sqlDB.CheckQueryResults(t, "SELECT count(*) FROM [SHOW CONSTRAINTS FROM defaultdb.midmany] WHERE constraint_name = 'my_const'", [][]string{{"1"}})
sqlDB.CheckQueryResults(t, "SELECT count(*) FROM [SHOW INDEXES FROM defaultdb.midmany] WHERE column_name = 'a'", [][]string{{"1"}})
case "midmultitxn":
numJobsInCluster = 1 // the CREATE TABLE job
expectedData = [][]string{{"1", "1.3"}, {"2", "1.3"}, {"3", "1.3"}}
sqlDB.CheckQueryResults(t, "SELECT count(*) FROM [SHOW CONSTRAINTS FROM defaultdb.midmultitxn] WHERE constraint_name = 'my_const'", [][]string{{"1"}})
sqlDB.CheckQueryResults(t, "SELECT count(*) FROM [SHOW INDEXES FROM defaultdb.midmultitxn] WHERE column_name = 'a'", [][]string{{"1"}})
case "midmultitable":
numJobsInCluster = 2 // the 2 CREATE TABLE jobs
expNumSchemaChangeJobs = 2
expectedData = [][]string{{"1", "1.3"}, {"2", "1.3"}, {"3", "1.3"}}
sqlDB.CheckQueryResults(t, fmt.Sprintf("SELECT * FROM %s1", tableName), expectedData)
expectedData = [][]string{{"1"}, {"2"}, {"3"}}
sqlDB.CheckQueryResults(t, fmt.Sprintf("SELECT * FROM %s2", tableName), expectedData)
tableName += "1"
case "midprimarykeyswap":
numJobsInCluster = 2 // the CREATE TABLE job and the ALTER COLUMN
// The primary key swap will also create a cleanup job.
expNumSchemaChangeJobs = 2
expectedData = [][]string{{"1"}, {"2"}, {"3"}}
case "midprimarykeyswapcleanup":
// The CREATE TABLE job, the ALTER COLUMN, and the original ALTER PRIMARY
// KEY that is being cleaned up.
numJobsInCluster = 3
// This backup only contains the cleanup job mentioned above.
expectedData = [][]string{{"1"}, {"2"}, {"3"}}
}
if scName != "midmultitable" {
sqlDB.CheckQueryResults(t, fmt.Sprintf("SELECT * FROM %s", tableName), expectedData)
}
if isClusterRestore {
// If we're performing a cluster restore, we also need to include the drop
// crdb_temp_system job.
expNumSchemaChangeJobs++
// And the create table jobs included from the backups.
expNumSchemaChangeJobs += numJobsInCluster
}
schemaChangeJobs := sqlDB.QueryStr(t, "SELECT description FROM crdb_internal.jobs WHERE job_type = 'SCHEMA CHANGE'")
require.Equal(t, expNumSchemaChangeJobs, len(schemaChangeJobs),
"Expected %d schema change jobs but found %v", expNumSchemaChangeJobs, schemaChangeJobs)
if isClusterRestore {
// Cluster restores should be restoring the exact job entries that were
// backed up, and therefore should not create jobs that contains "RESTORING"
// in the description.
schemaChangeJobs := sqlDB.QueryStr(t,
"SELECT description FROM crdb_internal.jobs WHERE job_type = 'SCHEMA CHANGE' AND description NOT LIKE '%RESTORING%'")
require.Equal(t, expNumSchemaChangeJobs, len(schemaChangeJobs),
"Expected %d schema change jobs but found %v", expNumSchemaChangeJobs, schemaChangeJobs)
} else {
// Non-cluster restores should create jobs with "RESTORE" in the job
// description.
schemaChangeJobs := sqlDB.QueryStr(t,
"SELECT description FROM crdb_internal.jobs WHERE job_type = 'SCHEMA CHANGE' AND description LIKE '%RESTORING%'")
require.Equal(t, expNumSchemaChangeJobs, len(schemaChangeJobs),
"Expected %d schema change jobs but found %v", expNumSchemaChangeJobs, schemaChangeJobs)
}
// Ensure that a schema change can complete on the restored table.
schemaChangeQuery := fmt.Sprintf("ALTER TABLE %s ADD CONSTRAINT post_restore_const CHECK (a > 0)", tableName)
sqlDB.Exec(t, schemaChangeQuery)
}

func restoreMidSchemaChange(
backupDir, schemaChangeName string, isClusterRestore bool,
) func(t *testing.T) {
return func(t *testing.T) {
params := base.TestServerArgs{}

ctx := context.Background()
defer jobs.TestingSetAdoptAndCancelIntervals(100*time.Millisecond, 100*time.Millisecond)()

const numAccounts = 1000
_, _, sqlDB, dir, cleanup := backupRestoreTestSetupWithParams(t, singleNode, numAccounts,
InitNone, base.TestClusterArgs{ServerArgs: params})
defer cleanup()
dir, dirCleanupFn := testutils.TempDir(t)
params := base.TestClusterArgs{
ServerArgs: base.TestServerArgs{ExternalIODir: dir},
}
tc := testcluster.StartTestCluster(t, singleNode, params)
defer func() {
tc.Stopper().Stop(ctx)
dirCleanupFn()
}()
sqlDB := sqlutils.MakeSQLRunner(tc.Conns[0])

symlink := filepath.Join(dir, "foo")
err := os.Symlink(backupDir, symlink)
require.NoError(t, err)

sqlDB.Exec(t, "USE defaultdb")
restoreQuery := fmt.Sprintf("RESTORE defaultdb.* from $1")
if isClusterRestore {
restoreQuery = fmt.Sprintf("RESTORE from $1")
}
log.Infof(context.Background(), "%+v", sqlDB.QueryStr(t, "SHOW BACKUP $1", LocalFoo))
sqlDB.Exec(t, restoreQuery, LocalFoo)
verify(t, schemaChangeName, sqlDB)
sqlDB.CheckQueryResultsRetry(t, "SELECT * FROM crdb_internal.jobs WHERE job_type = 'SCHEMA CHANGE' AND status <> 'succeeded'", [][]string{})
verifyMidSchemaChange(t, schemaChangeName, sqlDB, isClusterRestore)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -64,5 +64,5 @@ CREATE TABLE midprimarykeyswapcleanup AS SELECT * FROM generate_series(1,3) AS a
-- This schema change is used to enable the primary key swap. The backup is not taken during this schema change.
ALTER TABLE midprimarykeyswapcleanup ALTER COLUMN a SET NOT NULL;
ALTER TABLE midprimarykeyswapcleanup ALTER PRIMARY KEY USING COLUMNS (a);
BACKUP defaultdb.* TO 'nodelocal://1/midprimarykeyswapcleanuprestor';
BACKUP defaultdb.* TO 'nodelocal://1/midprimarykeyswapcleanup';
DROP TABLE midprimarykeyswapcleanup;
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
���O
Binary file not shown.
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
Uг
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
-�r�
Binary file not shown.
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
���7
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
�v��
Binary file not shown.
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
��%[
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
�ũ�
Binary file not shown.
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
���
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
�=LM
Binary file not shown.
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
xD�
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
!m��
Binary file not shown.
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
��A�
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
��yN
Binary file not shown.
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
t�6!
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
��ͻ
Binary file not shown.
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
ݩe�
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
���
Binary file not shown.
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
�},t
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Loading

0 comments on commit f79ebfc

Please sign in to comment.