Skip to content

Commit

Permalink
sql, *: allow table scoping under pg_temp_<session_id> physical schema
Browse files Browse the repository at this point in the history
Previously, CRDB only supported the `public` physical schema. All table
entries in `system.namespace` assumed an implied `public` schema, so
name resolution for tables only required a databaseID and table name to
uniquely identify a table.

As highlighted in the temp tables RFC, temp tables will be scoped under
a session specific schema of the form `pg_temp_<session_id>`. This
motivated adding support for additional physical schemas.

This PR involves the following changes to `system.namespace`:

- A new `system.namespace` table is constructed for cluster versions
>= 20.1, which has an additional primary key column `publicSchemaID`.

- The older `system.namespace` table is marked deprecated. All
`system.namespace` read accesses default to the new `system.namespace`.
If a match isn't found, the deprecated `system.namespace` is checked.

- All `system.namespace` write accesses for key deletions remove entries
from both versions of the table. This ensures the fallback code doesn't
read an old key that was deleted.

- All `system.namespace` write accesses that involve creating new entries
are added to the `system.namespace` table according to the cluster version.

- Selecting from `system.namespace` in mixed version clusters actually
selects from `system.namespace_deprecated`, ensuring that the change is
invisible to users.

- The new `system.namespace` table is moved out of the SystemConfig
range. This means `system.namespace` is no longer gossiped for cluster
versions >= 20.1 .

- Every new database creation adds the `public` schema to
`system.namespace` by default.

As a result of the above changes to `system.namespace`, there is a
new interface that all accesses should go through.

- Lookups
Previously: Keys were constructed and directly used to perform KV
lookups.
Now: Use LookupObjectID, or another specialized lookup method
provided. This ensures correct fallback semantics for mixed-version
19.2/20.1 clusters.

- Removals
Previously: Keys were constructed and directly used to perform KV
deletes.
Now: Use RemoveObjectNamespaceEntry or another specialized method
provided. This ensures correct behavior for mixed-version 19.2/20.1
clusters.

- Additions
Previously: Keys were constructed and directly used to perform CPuts
with the appropriate value.
Now: Use MakeObjectNameKey or another specialized method provided to
construct the key. This ensures that the key created is for the
appropriate cluster version. These methods should only be used to
create keys for adding entries -- removals/lookups should go through
the appropriate interfaces.

The `search_path` is responsible for establishing the order in which
schemas are searched during name resolution. This PR involves the
following changes to the `search_path.go`:

- The search semantics are updated to match those described in the temp
tables RFC.

- The search path is now aware of the session specific temporary schema,
which can be used during name resolution.

- The distSQL api proto had to be updated to pass the temporary schema to
other nodes in addition to the list of paths.

Benchmarks:

TPC-C with 3 nodes/16CPUs:
- max warehouses: 1565

Microbenchmarks for system.namespace access:

|                 name                | master time/op | new approach time/op | delta |
| ----------------------------------- | -------------- | -------------------- | ----- |
| NameResolution/Cockroach-8          | 163µs ± 0%     | 252µs ± 0%           | ~     |
| NameResolution/MultinodeCockroach-8 | 419µs ± 0%     | 797µs ± 0%           | ~     |

|                        name                        | master time/op | new approach time/op | delta |
| -------------------------------------------------- | -------------- | -------------------- | ----- |
| NameResolutionTempTablesExist/Cockroach-8          | 175µs ± 0%     | 337µs ± 0%           | ~     |
| NameResolutionTempTablesExist/MultinodeCockroach-8 | 1.06ms ± 0%    | 1.07ms ± 0%          | ~     |

Follow-up work:

- The `TableCollection` cache needs to be updated to have knowledge about
schemaIDs. Once this is done, there is a TODO in the code that allows the
`CachedPhysicalAccessor` to work correctly.

- Migration for clusters upgrading to 20.1. The new `system.namespace`
table needs to be populated from the deprecated table and a `public`
schema needs to be added for every database during migration.

Release note (sql change): CREATE TABLE pg_temp.abc(a int) now creates
a temporary table. See temp tables RFC (guide-level explanation) for
more details about the search path semantics.
  • Loading branch information
arulajmani committed Dec 5, 2019
1 parent e1c1a9a commit 90f5b69
Show file tree
Hide file tree
Showing 94 changed files with 2,022 additions and 800 deletions.
2 changes: 1 addition & 1 deletion docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -60,6 +60,6 @@
<tr><td><code>trace.debug.enable</code></td><td>boolean</td><td><code>false</code></td><td>if set, traces for recent requests can be seen in the /debug page</td></tr>
<tr><td><code>trace.lightstep.token</code></td><td>string</td><td><code></code></td><td>if set, traces go to Lightstep using this token</td></tr>
<tr><td><code>trace.zipkin.collector</code></td><td>string</td><td><code></code></td><td>if set, traces go to the given Zipkin instance (example: '127.0.0.1:9411'); ignored if trace.lightstep.token is set</td></tr>
<tr><td><code>version</code></td><td>custom validation</td><td><code>19.2-4</code></td><td>set the active cluster version in the format '<major>.<minor>'</td></tr>
<tr><td><code>version</code></td><td>custom validation</td><td><code>19.2-5</code></td><td>set the active cluster version in the format '<major>.<minor>'</td></tr>
</tbody>
</table>
45 changes: 45 additions & 0 deletions pkg/bench/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1090,3 +1090,48 @@ func BenchmarkSortJoinAggregation(b *testing.B) {
}
})
}

func BenchmarkNameResolution(b *testing.B) {
if testing.Short() {
b.Skip("short flag")
}
defer log.Scope(b).Close(b)
ForEachDB(b, func(b *testing.B, db *sqlutils.SQLRunner) {
db.Exec(b, `CREATE TABLE namespace (k INT PRIMARY KEY, v INT)`)
db.Exec(b, `INSERT INTO namespace VALUES(1, 2)`)

b.ResetTimer()

for i := 0; i < b.N; i++ {
db.Exec(b, "SELECT * FROM namespace")
}
b.StopTimer()
})
}

// When temporary tables are present in the system, the PG search path semantics
// dictate that we try searching for tables under the temporary schema before
// the public physical schema. This test is used to microbenchmark the effects
// of this scenario.
func BenchmarkNameResolutionTempTablesExist(b *testing.B) {
if testing.Short() {
b.Skip("short flag")
}
defer log.Scope(b).Close(b)
ForEachDB(b, func(b *testing.B, db *sqlutils.SQLRunner) {
db.Exec(b, `CREATE TABLE namespace (k INT PRIMARY KEY, v INT)`)
db.Exec(b, `INSERT INTO namespace VALUES(1, 2)`)

b.ResetTimer()

for i := 0; i < b.N; i++ {
b.StopTimer()
// Setup
db.Exec(b, `SET experimental_enable_temp_tables = true`)
db.Exec(b, `CREATE TEMP TABLE IF NOT EXISTS temp_table(k INT PRIMARY KEY, v INT)`)
b.StartTimer()
db.Exec(b, "SELECT * FROM namespace")
}
b.StopTimer()
})
}
4 changes: 3 additions & 1 deletion pkg/ccl/backupccl/backup.go
Original file line number Diff line number Diff line change
Expand Up @@ -1366,7 +1366,9 @@ func (b *backupResumer) Resume(
}

// OnFailOrCancel is part of the jobs.Resumer interface.
func (b *backupResumer) OnFailOrCancel(context.Context, *client.Txn) error { return nil }
func (b *backupResumer) OnFailOrCancel(context.Context, *client.Txn) error {
return nil
}

// OnSuccess is part of the jobs.Resumer interface.
func (b *backupResumer) OnSuccess(context.Context, *client.Txn) error { return nil }
Expand Down
45 changes: 21 additions & 24 deletions pkg/ccl/backupccl/restore.go
Original file line number Diff line number Diff line change
Expand Up @@ -362,12 +362,11 @@ func allocateTableRewrites(
if err := p.ExecCfg().DB.Txn(ctx, func(ctx context.Context, txn *client.Txn) error {
// Check that any DBs being restored do _not_ exist.
for name := range restoreDBNames {
dKey := sqlbase.NewDatabaseKey(name)
existingDatabaseID, err := txn.Get(ctx, dKey.Key())
found, _, err := sqlbase.LookupDatabaseID(ctx, txn, name)
if err != nil {
return err
}
if existingDatabaseID.Value != nil {
if found {
return errors.Errorf("database %q already exists", name)
}
}
Expand All @@ -390,21 +389,15 @@ func allocateTableRewrites(
} else {
var parentID sqlbase.ID
{
dKey := sqlbase.NewDatabaseKey(targetDB)
existingDatabaseID, err := txn.Get(ctx, dKey.Key())
found, newParentID, err := sqlbase.LookupDatabaseID(ctx, txn, targetDB)
if err != nil {
return err
}
if existingDatabaseID.Value == nil {
if !found {
return errors.Errorf("a database named %q needs to exist to restore table %q",
targetDB, table.Name)
}

newParentID, err := existingDatabaseID.Value.GetInt()
if err != nil {
return err
}
parentID = sqlbase.ID(newParentID)
parentID = newParentID
}

// Check that the table name is _not_ in use.
Expand Down Expand Up @@ -481,12 +474,11 @@ func allocateTableRewrites(
func CheckTableExists(
ctx context.Context, txn *client.Txn, parentID sqlbase.ID, name string,
) error {
tKey := sqlbase.NewTableKey(parentID, name)
res, err := txn.Get(ctx, tKey.Key())
found, _, err := sqlbase.LookupPublicTableID(ctx, txn, parentID, name)
if err != nil {
return err
}
if res.Exists() {
if found {
return sqlbase.NewRelationAlreadyExistsError(name)
}
return nil
Expand Down Expand Up @@ -1036,7 +1028,11 @@ func WriteTableDescs(
if err := sql.WriteNewDescToBatch(ctx, false /* kvTrace */, settings, b, desc.ID, desc); err != nil {
return err
}
b.CPut(sqlbase.NewDatabaseKey(desc.Name).Key(), desc.ID, nil)
// Depending on which cluster version we are restoring to, we decide which
// namespace table to write the descriptor into. This may cause wrong
// behavior if the cluster version is bumped DURING a restore.
dKey := sqlbase.MakeDatabaseNameKey(ctx, settings, desc.Name)
b.CPut(dKey.Key(), desc.ID, nil)
}
for i := range tables {
if wrote, ok := wroteDBs[tables[i].ParentID]; ok {
Expand All @@ -1058,7 +1054,11 @@ func WriteTableDescs(
if err := sql.WriteNewDescToBatch(ctx, false /* kvTrace */, settings, b, tables[i].ID, tables[i]); err != nil {
return err
}
b.CPut(sqlbase.NewTableKey(tables[i].ParentID, tables[i].Name).Key(), tables[i].ID, nil)
// Depending on which cluster version we are restoring to, we decide which
// namespace table to write the descriptor into. This may cause wrong
// behavior if the cluster version is bumped DURING a restore.
tkey := sqlbase.MakePublicTableNameKey(ctx, settings, tables[i].ParentID, tables[i].Name)
b.CPut(tkey.Key(), tables[i].ID, nil)
}
for _, kv := range extra {
b.InitPut(kv.Key, &kv.Value, false)
Expand Down Expand Up @@ -1749,13 +1749,10 @@ func (r *restoreResumer) OnFailOrCancel(ctx context.Context, txn *client.Txn) er
tableDesc := *tbl
tableDesc.Version++
tableDesc.State = sqlbase.TableDescriptor_DROP
var existingIDVal roachpb.Value
existingIDVal.SetInt(int64(tableDesc.ID))
b.CPut(
sqlbase.NewTableKey(tableDesc.ParentID, tableDesc.Name).Key(),
nil,
&existingIDVal,
)
err := sqlbase.RemovePublicTableNamespaceEntry(ctx, txn, tbl.ParentID, tbl.Name)
if err != nil {
return nil
}
existingDescVal, err := sqlbase.ConditionalGetTableDescFromTxn(ctx, txn, tbl)
if err != nil {
return errors.Wrap(err, "dropping tables")
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/targets_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -108,7 +108,7 @@ func TestDescriptorsMatchingTargets(t *testing.T) {
{"", `TABLE system.offline`, []string{"system", "foo"}, nil, `table "system.public.offline" does not exist`},
{"system", `TABLE *`, []string{"system", "foo", "bar"}, nil, ``},
}
searchPath := sessiondata.MakeSearchPath([]string{"public", "pg_catalog"})
searchPath := sessiondata.MakeSearchPath([]string{"public", "pg_catalog"}, sessiondata.DefaultTemporarySchemaName)
for i, test := range tests {
t.Run(fmt.Sprintf("%d/%s/%s", i, test.sessionDatabase, test.pattern), func(t *testing.T) {
sql := fmt.Sprintf(`GRANT ALL ON %s TO ignored`, test.pattern)
Expand Down
8 changes: 4 additions & 4 deletions pkg/ccl/importccl/import_stmt.go
Original file line number Diff line number Diff line change
Expand Up @@ -922,10 +922,10 @@ func (r *importResumer) OnFailOrCancel(ctx context.Context, txn *client.Txn) err
// possible. This is safe since the table data was never visible to users,
// and so we don't need to preserve MVCC semantics.
tableDesc.DropTime = 1
var existingIDVal roachpb.Value
existingIDVal.SetInt(int64(tableDesc.ID))
tKey := sqlbase.NewTableKey(tableDesc.ParentID, tableDesc.Name)
b.CPut(tKey.Key(), nil, &existingIDVal)
err := sqlbase.RemovePublicTableNamespaceEntry(ctx, txn, tableDesc.ParentID, tableDesc.Name)
if err != nil {
return err
}
} else {
// IMPORT did not create this table, so we should not drop it.
tableDesc.State = sqlbase.TableDescriptor_PUBLIC
Expand Down
6 changes: 5 additions & 1 deletion pkg/ccl/partitionccl/partition_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1095,10 +1095,14 @@ func verifyScansOnNode(
}
traceLines = append(traceLines, traceLine.String)
if strings.Contains(traceLine.String, "read completed") {
if strings.Contains(traceLine.String, "SystemCon") {
if strings.Contains(traceLine.String, "SystemCon") || strings.Contains(traceLine.String, "NamespaceTab") {
// Ignore trace lines for the system config range (abbreviated as
// "SystemCon" in pretty printing of the range descriptor). A read might
// be performed to the system config range to update the table lease.
//
// Also ignore trace lines for the system.namespace table, which is a
// system table that resides outside the system config range. (abbreviated
// as "NamespaceTab" in pretty printing of the range descriptor).
continue
}
if !strings.Contains(traceLine.String, node) {
Expand Down
3 changes: 3 additions & 0 deletions pkg/cli/cli_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2229,6 +2229,8 @@ writing ` + os.DevNull + `
debug/nodes/1/ranges/22.json
debug/nodes/1/ranges/23.json
debug/nodes/1/ranges/24.json
debug/nodes/1/ranges/25.json
debug/nodes/1/ranges/26.json
debug/schema/[email protected]
debug/schema/[email protected]
debug/schema/[email protected]
Expand All @@ -2239,6 +2241,7 @@ writing ` + os.DevNull + `
debug/schema/system/lease.json
debug/schema/system/locations.json
debug/schema/system/namespace.json
debug/schema/system/namespace_deprecated.json
debug/schema/system/rangelog.json
debug/schema/system/replication_constraint_stats.json
debug/schema/system/replication_critical_localities.json
Expand Down
20 changes: 11 additions & 9 deletions pkg/cli/sql_util_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -150,6 +150,7 @@ SET

expectedRows := [][]string{
{`parentID`, `INT8`, `false`, `NULL`, ``, `{primary}`, `false`},
{`parentSchemaID`, `INT8`, `false`, `NULL`, ``, `{primary}`, `false`},
{`name`, `STRING`, `false`, `NULL`, ``, `{primary}`, `false`},
{`id`, `INT8`, `true`, `NULL`, ``, `{}`, `false`},
}
Expand All @@ -163,12 +164,13 @@ SET
}

expected = `
column_name | data_type | is_nullable | column_default | generation_expression | indices | is_hidden
+-------------+-----------+-------------+----------------+-----------------------+-----------+-----------+
parentID | INT8 | false | NULL | | {primary} | false
name | STRING | false | NULL | | {primary} | false
id | INT8 | true | NULL | | {} | false
(3 rows)
column_name | data_type | is_nullable | column_default | generation_expression | indices | is_hidden
+----------------+-----------+-------------+----------------+-----------------------+-----------+-----------+
parentID | INT8 | false | NULL | | {primary} | false
parentSchemaID | INT8 | false | NULL | | {primary} | false
name | STRING | false | NULL | | {primary} | false
id | INT8 | true | NULL | | {} | false
(4 rows)
`

if a, e := b.String(), expected[1:]; a != e {
Expand All @@ -183,9 +185,9 @@ SET
}

expected = `
parentID | name | id
+----------+------------+----+
1 | descriptor | 3
parentID | parentSchemaID | name | id
+----------+----------------+------------+----+
1 | 29 | descriptor | 3
(1 row)
`
if a, e := b.String(), expected[1:]; a != e {
Expand Down
17 changes: 10 additions & 7 deletions pkg/config/system_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/config"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/util/encoding"
Expand Down Expand Up @@ -163,7 +164,7 @@ func TestGetLargestID(t *testing.T) {
ms := sqlbase.MakeMetadataSchema(config.DefaultZoneConfigRef(), config.DefaultSystemZoneConfigRef())
descIDs := ms.DescriptorIDs()
maxDescID := descIDs[len(descIDs)-1]
kvs, _ /* splits */ := ms.GetInitialValues()
kvs, _ /* splits */ := ms.GetInitialValues(cluster.TestingClusterVersion)
return testCase{kvs, uint32(maxDescID), 0, ""}
}(),

Expand Down Expand Up @@ -256,7 +257,7 @@ func TestComputeSplitKeySystemRanges(t *testing.T) {
}

cfg := config.NewSystemConfig(config.DefaultZoneConfigRef())
kvs, _ /* splits */ := sqlbase.MakeMetadataSchema(cfg.DefaultZoneConfig, config.DefaultSystemZoneConfigRef()).GetInitialValues()
kvs, _ /* splits */ := sqlbase.MakeMetadataSchema(cfg.DefaultZoneConfig, config.DefaultSystemZoneConfigRef()).GetInitialValues(cluster.TestingClusterVersion)
cfg.SystemConfigEntries = config.SystemConfigEntries{
Values: kvs,
}
Expand Down Expand Up @@ -288,12 +289,14 @@ func TestComputeSplitKeyTableIDs(t *testing.T) {

schema := sqlbase.MakeMetadataSchema(config.DefaultZoneConfigRef(), config.DefaultSystemZoneConfigRef())
// Real system tables only.
baseSql, _ /* splits */ := schema.GetInitialValues()
baseSql, _ /* splits */ := schema.GetInitialValues(cluster.TestingClusterVersion)
// Real system tables plus some user stuff.
kvs, _ /* splits */ := schema.GetInitialValues()
kvs, _ /* splits */ := schema.GetInitialValues(cluster.TestingClusterVersion)
userSQL := append(kvs, descriptor(start), descriptor(start+1), descriptor(start+5))
// Real system tables and partitioned user tables.
subzoneSQL := append(userSQL,
var subzoneSQL = make([]roachpb.KeyValue, len(userSQL))
copy(subzoneSQL, userSQL)
subzoneSQL = append(subzoneSQL,
zoneConfig(start+1, subzone("a", ""), subzone("c", "e")),
zoneConfig(start+5, subzone("b", ""), subzone("c", "d"), subzone("d", "")))

Expand Down Expand Up @@ -404,13 +407,13 @@ func TestGetZoneConfigForKey(t *testing.T) {
{roachpb.RKey(keys.SystemConfigSplitKey), keys.SystemDatabaseID},

// Gossiped system tables should refer to the SystemDatabaseID.
{tkey(keys.NamespaceTableID), keys.SystemDatabaseID},
{tkey(keys.ZonesTableID), keys.SystemDatabaseID},

// Non-gossiped system tables should refer to themselves.
{tkey(keys.LeaseTableID), keys.LeaseTableID},
{tkey(keys.JobsTableID), keys.JobsTableID},
{tkey(keys.LocationsTableID), keys.LocationsTableID},
{tkey(keys.NamespaceTableID), keys.NamespaceTableID},

// Pseudo-tables should refer to the SystemDatabaseID.
{tkey(keys.MetaRangesID), keys.SystemDatabaseID},
Expand All @@ -430,7 +433,7 @@ func TestGetZoneConfigForKey(t *testing.T) {
}()
cfg := config.NewSystemConfig(config.DefaultZoneConfigRef())

kvs, _ /* splits */ := sqlbase.MakeMetadataSchema(cfg.DefaultZoneConfig, config.DefaultSystemZoneConfigRef()).GetInitialValues()
kvs, _ /* splits */ := sqlbase.MakeMetadataSchema(cfg.DefaultZoneConfig, config.DefaultSystemZoneConfigRef()).GetInitialValues(cluster.TestingClusterVersion)
cfg.SystemConfigEntries = config.SystemConfigEntries{
Values: kvs,
}
Expand Down
26 changes: 19 additions & 7 deletions pkg/keys/constants.go
Original file line number Diff line number Diff line change
Expand Up @@ -265,6 +265,12 @@ var (
MaxKey = roachpb.KeyMax
// MinKey is a minimum key value which sorts before all other keys.
MinKey = roachpb.KeyMin

// NamespaceTableMin is the start key of system.namespace, which is a system
// table that does not reside in the same range as other system tables.
NamespaceTableMin = roachpb.Key(MakeTablePrefix(NamespaceTableID))
// NamespaceTableMax is the end key of system.namespace.
NamespaceTableMax = roachpb.Key(MakeTablePrefix(NamespaceTableID + 1))
)

// Various IDs used by the structured data layer.
Expand Down Expand Up @@ -295,12 +301,14 @@ const (
// SystemDatabaseID and following are the database/table IDs for objects
// in the system span.
// NOTE: IDs must be <= MaxSystemConfigDescID.
SystemDatabaseID = 1
NamespaceTableID = 2
DescriptorTableID = 3
UsersTableID = 4
ZonesTableID = 5
SettingsTableID = 6
SystemDatabaseID = 1
// DeprecatedNamespaceTableID was the tableID for the system.namespace table
// for pre-20.1 clusters.
DeprecatedNamespaceTableID = 2
DescriptorTableID = 3
UsersTableID = 4
ZonesTableID = 5
SettingsTableID = 6

// IDs for the important columns and indexes in the zones table live here to
// avoid introducing a dependency on sql/sqlbase throughout the codebase.
Expand Down Expand Up @@ -330,6 +338,10 @@ const (
ReplicationCriticalLocalitiesTableID = 26
ReplicationStatsTableID = 27
ReportsMetaTableID = 28
PublicSchemaID = 29
// New NamespaceTableID for cluster version >= 20.1
// Ensures that NamespaceTable does not get gossiped again
NamespaceTableID = 30

// CommentType is type for system.comments
DatabaseCommentType = 0
Expand All @@ -342,4 +354,4 @@ const (
// there's no table descriptor). They're grouped here because the cluster
// bootstrap process needs to create splits for them; splits for the tables
// happen separately.
var PseudoTableIDs = []uint32{MetaRangesID, SystemRangesID, TimeseriesRangesID, LivenessRangesID}
var PseudoTableIDs = []uint32{MetaRangesID, SystemRangesID, TimeseriesRangesID, LivenessRangesID, PublicSchemaID}
3 changes: 3 additions & 0 deletions pkg/keys/printer.go
Original file line number Diff line number Diff line change
Expand Up @@ -126,6 +126,9 @@ var (
PSFunc: parseUnsupported,
},
}},
{Name: "/NamespaceTable", start: NamespaceTableMin, end: NamespaceTableMax, Entries: []DictEntry{
{Name: "", prefix: nil, ppFunc: decodeKeyPrint, PSFunc: parseUnsupported},
}},
{Name: "/Table", start: TableDataMin, end: TableDataMax, Entries: []DictEntry{
{Name: "", prefix: nil, ppFunc: decodeKeyPrint, PSFunc: tableKeyParse},
}},
Expand Down
Loading

0 comments on commit 90f5b69

Please sign in to comment.