Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
113999: engineccl: Add benchmark for ctr_stream encryption r=sumeerbhola a=bdarnell

Start measuring performance of this code in anticipation of improving it.

Epic: none

Release note: None

115223: clusterversion: change string for upgrade versions r=RaduBerinde a=RaduBerinde

#### clusterversion: move ReleaseSeries functionality to roacphb.Version

This change moves the implementation of
`clusterversion.Key.ReleaseSeries()` to `roachpb`. We now use a
hardcoded map of sucessor versions.

Epic: none
Release note: None

#### clusterversion: change string for upgrade versions

This change concerns cluster versions during upgrade. When starting an
upgrade from e.g. 23.3 to 24.1, we start with the cluster version 23.2
and then we go through a sequence of internal versions associated with
various upgrade steps. These versions are presented as `23.2-x`, e.g.
`23.2-8`.

This formatting doesn't make it clear what this version represents. It
can also be confusing - `23.2-8` looks very close to `23.2.8` which
might be an actual CockroachDB version.

This change renames versions during upgrade:
`23.2-upgrading-to-24.1-step-008`. The internal part is always
formatted to three digits (this is intended to reduce the chance of
confusing the internal part to a patch release).

Informs: #112629
Release note (general change): Versions during upgrades are renamed,
for example `23.2-8` is now `23.2-upgrading-to-24.1-step-008`.


Co-authored-by: Ben Darnell <[email protected]>
Co-authored-by: Radu Berinde <[email protected]>
  • Loading branch information
3 people committed Dec 1, 2023
3 parents 146b425 + d3077b0 + 31521ef commit 19ce36d
Show file tree
Hide file tree
Showing 16 changed files with 311 additions and 94 deletions.
2 changes: 1 addition & 1 deletion docs/generated/settings/settings-for-tenants.txt
Original file line number Diff line number Diff line change
Expand Up @@ -325,4 +325,4 @@ trace.snapshot.rate duration 0s if non-zero, interval at which background trace
trace.span_registry.enabled boolean true if set, ongoing traces can be seen at https://<ui>/#/debug/tracez application
trace.zipkin.collector string the address of a Zipkin instance to receive traces, as <host>:<port>. If no port is specified, 9411 will be used. application
ui.display_timezone enumeration etc/utc the timezone used to format timestamps in the ui [etc/utc = 0, america/new_york = 1] application
version version 1000023.2-4 set the active cluster version in the format '<major>.<minor>' application
version version 1000023.2-upgrading-to-1000024.1-step-004 set the active cluster version in the format '<major>.<minor>' application
2 changes: 1 addition & 1 deletion docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -276,6 +276,6 @@
<tr><td><div id="setting-trace-span-registry-enabled" class="anchored"><code>trace.span_registry.enabled</code></div></td><td>boolean</td><td><code>true</code></td><td>if set, ongoing traces can be seen at https://&lt;ui&gt;/#/debug/tracez</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-trace-zipkin-collector" class="anchored"><code>trace.zipkin.collector</code></div></td><td>string</td><td><code></code></td><td>the address of a Zipkin instance to receive traces, as &lt;host&gt;:&lt;port&gt;. If no port is specified, 9411 will be used.</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-ui-display-timezone" class="anchored"><code>ui.display_timezone</code></div></td><td>enumeration</td><td><code>etc/utc</code></td><td>the timezone used to format timestamps in the ui [etc/utc = 0, america/new_york = 1]</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-version" class="anchored"><code>version</code></div></td><td>version</td><td><code>1000023.2-4</code></td><td>set the active cluster version in the format &#39;&lt;major&gt;.&lt;minor&gt;&#39;</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-version" class="anchored"><code>version</code></div></td><td>version</td><td><code>1000023.2-upgrading-to-1000024.1-step-004</code></td><td>set the active cluster version in the format &#39;&lt;major&gt;.&lt;minor&gt;&#39;</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
</tbody>
</table>
4 changes: 2 additions & 2 deletions pkg/ccl/backupccl/restore_planning_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -132,10 +132,10 @@ func TestBackupManifestVersionCompatibility(t *testing.T) {
},
{
name: "alpha-restore",
backupVersion: roachpb.Version{Major: 100022, Minor: 2, Internal: 14},
backupVersion: roachpb.Version{Major: 1000022, Minor: 2, Internal: 14},
clusterVersion: roachpb.Version{Major: 23, Minor: 1},
minimumSupportedVersion: roachpb.Version{Major: 22, Minor: 2},
expectedError: "backup from version 100022.2-14 is newer than current version 23.1",
expectedError: "backup from version 1000022.2-upgrading-to-1000023.1-step-014 is newer than current version 23.1",
},
{
name: "old-backup",
Expand Down
10 changes: 4 additions & 6 deletions pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant
Original file line number Diff line number Diff line change
Expand Up @@ -421,18 +421,16 @@ node_id component field value
0 UI URI /

statement error unsupported within a virtual cluster
SELECT node_id, network, regexp_replace(address, '\d+$', '<port>') as address, attrs, locality, regexp_replace(server_version, '^\d+\.\d+(-\d+)?$', '<server_version>') as server_version FROM crdb_internal.gossip_nodes WHERE node_id = 1
SELECT node_id FROM crdb_internal.gossip_nodes WHERE node_id = 1

statement error unsupported within a virtual cluster
SELECT node_id, epoch, regexp_replace(expiration, '^\d+\.\d+,\d+$', '<timestamp>') as expiration, draining, decommissioning, membership FROM crdb_internal.gossip_liveness WHERE node_id = 1
SELECT node_id FROM crdb_internal.gossip_liveness WHERE node_id = 1

statement error unsupported within a virtual cluster
SELECT node_id, network, regexp_replace(address, '\d+$', '<port>') as address, attrs, locality, regexp_replace(server_version, '^\d+\.\d+(-\d+)?$', '<server_version>') as server_version, regexp_replace(go_version, '^go.+$', '<go_version>') as go_version
FROM crdb_internal.kv_node_status WHERE node_id = 1
SELECT node_id FROM crdb_internal.kv_node_status WHERE node_id = 1

statement error unsupported within a virtual cluster
SELECT node_id, store_id, attrs, used
FROM crdb_internal.kv_store_status WHERE node_id = 1
SELECT node_id FROM crdb_internal.kv_store_status WHERE node_id = 1

query TT
SELECT * FROM crdb_internal.regions ORDER BY 1
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/storageccl/engineccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,7 @@ go_test(
deps = [
"//pkg/base",
"//pkg/ccl/baseccl",
"//pkg/ccl/securityccl/fipsccl",
"//pkg/ccl/storageccl/engineccl/enginepbccl",
"//pkg/clusterversion",
"//pkg/keys",
Expand Down
67 changes: 67 additions & 0 deletions pkg/ccl/storageccl/engineccl/ctr_stream_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,10 +11,12 @@ package engineccl
import (
"context"
"crypto/rand"
"encoding/binary"
"fmt"
"strings"
"testing"

"github.com/cockroachdb/cockroach/pkg/ccl/securityccl/fipsccl"
"github.com/cockroachdb/cockroach/pkg/ccl/storageccl/engineccl/enginepbccl"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
Expand Down Expand Up @@ -173,3 +175,68 @@ func TestFileCipherStreamCreator(t *testing.T) {
t.Fatalf("%s\n%s", strings.Join(diff, "\n"), data)
}
}

// Running non-fips mode:
// ./dev bench pkg/ccl/storageccl/engineccl -f FileCipherStream --stream-output --ignore-cache
// Running fips mode (be sure to look for fips=true in the output):
// ./dev test-binaries --cross=crosslinuxfips pkg/ccl/storageccl/engineccl && mkdir -p fipsbin && tar xf bin/test_binaries.tar.gz -C fipsbin && docker run -v $PWD/fipsbin:/fipsbin -it redhat/ubi9 /fipsbin/pkg/ccl/storageccl/engineccl/bin/engineccl_test -test.run '^$' -test.bench FileCipherStream
func BenchmarkFileCipherStream(b *testing.B) {
isFips := fipsccl.IsFIPSReady()
for _, keySize := range []int{128, 192, 256} {
for _, blockSize := range []int{16, 1024, 10240} {
b.Run(fmt.Sprintf("fips=%t/key=%d/block=%d/", isFips, keySize, blockSize), func(b *testing.B) {
keyBytes := make([]byte, keySize/8)
if _, err := rand.Read(keyBytes); err != nil {
panic(err)
}
var encType enginepbccl.EncryptionType
switch keySize {
case 128:
encType = enginepbccl.EncryptionType_AES128_CTR
case 192:
encType = enginepbccl.EncryptionType_AES192_CTR
case 256:
encType = enginepbccl.EncryptionType_AES256_CTR
default:
panic("unknown key size")
}
key := &enginepbccl.SecretKey{
Info: &enginepbccl.KeyInfo{
EncryptionType: encType,
},
Key: keyBytes,
}
nonce := make([]byte, ctrNonceSize)
if _, err := rand.Read(nonce); err != nil {
panic(err)
}
initCounterBytes := make([]byte, 4)
if _, err := rand.Read(initCounterBytes); err != nil {
panic(err)
}
// Endianness doesn't matter for converting this random number to an int.
initCounter := binary.LittleEndian.Uint32(initCounterBytes)
blockStream, err := newCTRBlockCipherStream(key, nonce, initCounter)
if err != nil {
panic(err)
}

stream := fileCipherStream{blockStream}

// Benchmarks are fun! We're just going to encrypt a bunch of zeros
// and re-encrypt over the previous output because that doesn't matter
// to the speed :)
//
// TODO(bdarnell): The offset argument to stream.Encrypt *does* matter,
// specifically whether the data is aligned to the CTR block size or not.
data := make([]byte, blockSize)
b.SetBytes(int64(blockSize))
b.ResetTimer()

for i := 0; i < b.N; i++ {
stream.Encrypt(0, data)
}
})
}
}
}
1 change: 0 additions & 1 deletion pkg/clusterversion/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@ go_library(
importpath = "github.com/cockroachdb/cockroach/pkg/clusterversion",
visibility = ["//visibility:public"],
deps = [
"//pkg/build",
"//pkg/roachpb",
"//pkg/settings",
"//pkg/testutils/skip",
Expand Down
41 changes: 16 additions & 25 deletions pkg/clusterversion/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,10 +10,7 @@

package clusterversion

import (
"github.com/cockroachdb/cockroach/pkg/build"
"github.com/cockroachdb/cockroach/pkg/roachpb"
)
import "github.com/cockroachdb/cockroach/pkg/roachpb"

// Key is a unique identifier for a version of CockroachDB.
type Key int
Expand Down Expand Up @@ -425,27 +422,21 @@ func (k Key) IsFinal() bool {
return k.Version().IsFinal()
}

// ReleaseSeries returns the final version for the release series the Key
// belongs to. Specifically:
// - if the key corresponds to a final version (e.g. 23.2), the result is the
// same version; e.g. V23_2.ReleaseSeries() is v23.2.
// - if the key corresponds to a transitional upgrade version (e.g. v23.2-8),
// the result is the next final version (e.g. v24.1).
//
// Note that the result does not have any DevOffset applied.
func (k Key) ReleaseSeries() roachpb.Version {
// Find the first key >= k that is a final version.
for k := k; k < numKeys; k++ {
if k.IsFinal() {
return removeDevOffset(k.Version())
}
}
// k is a dev version in the latest release series.
major, minor := build.BranchReleaseSeries()
return roachpb.Version{
Major: int32(major),
Minor: int32(minor),
}
// ReleaseSeries returns the release series the Key. Specifically:
// - if the key corresponds to a final version (e.g. V23_2), the result has the
// same major/minor;
// - if the key corresponds to a transitional upgrade version (e.g.
// V23_2SomeFeature with version 23.2-x), the result is the next series
// (e.g. 24.1).
//
// The key must be in the range [MinSupported, Latest].
//
// Note that the release series won't have the DevOffset applied, even if the
// version has it.
func (k Key) ReleaseSeries() roachpb.ReleaseSeries {
// Note: TestReleaseSeries ensures that this works for all valid Keys.
s, _ := removeDevOffset(k.Version()).ReleaseSeries()
return s
}

func (k Key) String() string {
Expand Down
37 changes: 24 additions & 13 deletions pkg/clusterversion/cockroach_versions_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -145,12 +145,11 @@ func TestClusterVersionPrettyPrint(t *testing.T) {
cv ClusterVersion
exp string
}{
{cv(19, 2, 1, 5), "19.2-5"},
{cv(20, 1, 0, 4), "20.1-4"},
{cv(20, 2, 0, 7), "20.2-7(fence)"},
{cv(20, 2, 0, 4), "20.2-4"},
{cv(20, 2, 1, 5), "20.2-5(fence)"},
{cv(20, 2, 1, 4), "20.2-4"},
{cv(20, 1, 0, 4), "20.1-upgrading-to-20.2-step-004"},
{cv(20, 2, 0, 7), "20.2-upgrading-to-21.1-step-007(fence)"},
{cv(20, 2, 0, 4), "20.2-upgrading-to-21.1-step-004"},
{cv(22, 2, 1, 5), "22.2-upgrading-to-23.1-step-005(fence)"},
{cv(22, 2, 1, 4), "22.2-upgrading-to-23.1-step-004"},
}
for _, test := range tests {
if actual := test.cv.PrettyPrint(); actual != test.exp {
Expand All @@ -160,13 +159,25 @@ func TestClusterVersionPrettyPrint(t *testing.T) {
}

func TestReleaseSeries(t *testing.T) {
// Verify that the ReleaseSeries call works on all keys.
for k := Latest; k > 0; k-- {
if k.Version().Major > 0 {
require.NotEqual(t, roachpb.ReleaseSeries{}, k.ReleaseSeries())
} else {
require.Equal(t, roachpb.ReleaseSeries{}, k.ReleaseSeries())
}
}

// Verify the latest version.
require.Equal(t, fmt.Sprintf("v%s", Latest.ReleaseSeries()), build.BinaryVersionPrefix())
if Latest.IsFinal() {
require.True(t, Latest.Version() == Latest.ReleaseSeries())
} else {
require.True(t, removeDevOffset(Latest.Version()).Less(Latest.ReleaseSeries()))

// Verify the ReleaseSeries results down to MinSupported.
expected := Latest.ReleaseSeries()
for k := Latest; k >= MinSupported; k-- {
if k.IsFinal() {
v := removeDevOffset(k.Version())
expected = roachpb.ReleaseSeries{Major: v.Major, Minor: v.Minor}
}
require.Equalf(t, expected, k.ReleaseSeries(), "version: %s", k)
}
require.Equal(t, PreviousRelease.ReleaseSeries(), removeDevOffset(PreviousRelease.Version()))
require.Equal(t, (PreviousRelease - 1).ReleaseSeries(), removeDevOffset(PreviousRelease.Version()))
require.Equal(t, MinSupported.ReleaseSeries(), removeDevOffset(MinSupported.Version()))
}
2 changes: 1 addition & 1 deletion pkg/clusterversion/dev_offset.go
Original file line number Diff line number Diff line change
Expand Up @@ -76,7 +76,7 @@ var devOffsetKeyStart = func() Key {

// DevOffset is the offset applied to major versions into the future if this is
// a dev branch.
const DevOffset = 1_000_000
const DevOffset = roachpb.VersionMajorDevOffset

// maybeApplyDevOffset applies DevOffset to the major version, if appropriate.
func maybeApplyDevOffset(key Key, v roachpb.Version) roachpb.Version {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ func TestClusterVersionAtLeast(t *testing.T) {
name: "invalid minVersion",
currentVersion: "23.1",
minVersion: "v23.1",
expectedErr: `invalid version v23.1: strconv.ParseInt: parsing "v23": invalid syntax`,
expectedErr: `invalid version v23.1`,
},
{
name: "cluster version is behind",
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/sql-bootstrap-data/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,7 @@ func main() {
}

// writeDataFile creates a file in pkg/sql/catalog/bootstrap/data.
func writeDataFile(version roachpb.Version, filenameSuffix, data string) {
func writeDataFile(version roachpb.ReleaseSeries, filenameSuffix, data string) {
filename := filepath.Join(
"pkg", "sql", "catalog", "bootstrap", "data",
fmt.Sprintf("%d_%d_%s", version.Major, version.Minor, filenameSuffix),
Expand Down
Loading

0 comments on commit 19ce36d

Please sign in to comment.