Skip to content

Commit

Permalink
Merge pull request #1174 from BenPope/rpk-start-pandaproxy
Browse files Browse the repository at this point in the history
rpk/start: Add support for Pandaproxy
  • Loading branch information
emaxerrno authored Apr 16, 2021
2 parents 9b587f4 + 3501542 commit 2ef1dfc
Show file tree
Hide file tree
Showing 8 changed files with 160 additions and 13 deletions.
3 changes: 2 additions & 1 deletion docs/www/production-deployment.md
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ For the best performance the following are strongly recommended:
- The following ports need to be opened
- `33145` - Internal RPC Port
- `9092` - Kafka API Port
- `8082` - Pandaproxy Port
- `9644` - Prometheus & HTTP Admin port

## Installation
Expand Down Expand Up @@ -99,7 +100,7 @@ cd gcp
#### Step 1: Prerequisites

You will need an existing subnet to deploy the VMs into. The subnet's attached
firewall should allow inbound traffic on ports 22, 3000, 8888, 8889, 9090,
firewall should allow inbound traffic on ports 22, 3000, 8082, 8888, 8889, 9090,
9092, 9644 and 33145. This module adds the `rp-node` tag to the deployed VMs,
which can be used as the target tag for the firewall rule.

Expand Down
9 changes: 8 additions & 1 deletion docs/www/quick-start-docker.md
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ With a 1-node cluster you can test out a simple implementation of Redpanda.
# make sure you update the version of v21.4.6 to the latest release https://github.com/vectorizedio/redpanda/releases


docker run -ti --rm -p 9092:9092 vectorized/redpanda:v21.4.6 start --overprovisioned --smp 1 --memory 1G --reserve-memory 0M --node-id 0 --check=false
docker run -ti --rm -p 8082:8082 -p 9092:9092 vectorized/redpanda:v21.4.6 start --overprovisioned --smp 1 --memory 1G --reserve-memory 0M --node-id 0 --check=false

```

Expand Down Expand Up @@ -60,6 +60,7 @@ docker run -d \
--name=redpanda-1 \
--hostname=redpanda-1 \
--net=redpandanet \
-p 8082:8082 \
-p 9092:9092 \
-v "redpanda1:/var/lib/redpanda/data" \
vectorized/redpanda start \
Expand All @@ -69,6 +70,8 @@ vectorized/redpanda start \
--overprovisioned \
--node-id 0 \
--check=false \
--pandaproxy-addr 0.0.0.0:8082 \
--advertise-pandaproxy-addr 127.0.0.1:8082 \
--kafka-addr 0.0.0.0:9092 \
--advertise-kafka-addr 127.0.0.1:9092 \
--rpc-addr 0.0.0.0:33145 \
Expand All @@ -88,6 +91,8 @@ vectorized/redpanda start \
--node-id 1 \
--seeds "redpanda-1:33145" \
--check=false \
--pandaproxy-addr 0.0.0.0:8083 \
--advertise-pandaproxy-addr 127.0.0.1:8083 \
--kafka-addr 0.0.0.0:9093 \
--advertise-kafka-addr 127.0.0.1:9093 \
--rpc-addr 0.0.0.0:33146 \
Expand All @@ -107,6 +112,8 @@ vectorized/redpanda start \
--node-id 2 \
--seeds "redpanda-1:33145" \
--check=false \
--pandaproxy-addr 0.0.0.0:8084 \
--advertise-pandaproxy-addr 127.0.0.1:8084 \
--kafka-addr 0.0.0.0:9094 \
--advertise-kafka-addr 127.0.0.1:9094 \
--rpc-addr 0.0.0.0:33147 \
Expand Down
18 changes: 10 additions & 8 deletions docs/www/rpk-commands.md
Original file line number Diff line number Diff line change
Expand Up @@ -48,18 +48,20 @@ Usage:
rpk redpanda start [flags]
Flags:
--advertise-kafka-addr strings The list of Kafka addresses to advertise (<host>:<port>)
--advertise-kafka-addr strings A comma-separated list of Kafka addresses to advertise (<name>://<host>:<port>)
--advertise-pandaproxy-addr A comma-separated list of Pandaproxy addresses to advertise (<name>://<host>:<port>)
--advertise-rpc-addr string The advertised RPC address (<host>:<port>)
--check When set to false will disable system checking before starting redpanda (default: true)
--config string Redpanda config file, if not set the file will be searched for in the default locations
--install-dir string Directory where redpanda has been installed
--kafka-addr strings The list of Kafka listener addresses to bind to (<host>:<port>)
--check When set to false will disable system checking before starting redpanda (default: true)
--config string Redpanda config file, if not set the file will be searched for in the default locations
--install-dir string Directory where redpanda has been installed
--kafka-addr strings A comma-separated list of Kafka listener addresses to bind to (<name>://<host>:<port>)
--node-id int The node ID. Must be an integer and must be unique within a cluster
--pandaproxy-addr A comma-separated list of Pandaproxy listener addresses to bind to (<name>://<host>:<port>)
--rpc-addr string The RPC address to bind to (<host>:<port>)
-s, --seeds strings A comma-separated list of seed node addresses (<host>[:<port>]) to connect to
--timeout duration The maximum time to wait for the checks and tune processes to complete. The value passed is a sequence of decimal numbers, each with optional fraction and a unit suffix, such as '300ms', '1.5s' or '2h45m'. Valid time units are 'ns', 'us' (or 'µs'), 'ms', 's', 'm', 'h' (default: 10s)
--tune When present will enable tuning before starting redpanda
--well-known-io string The cloud vendor and VM type, in the format <vendor>:<vm type>:<storage type>
--timeout duration The maximum time to wait for the checks and tune processes to complete. The value passed is a sequence of decimal numbers, each with optional fraction and a unit suffix, such as '300ms', '1.5s' or '2h45m'. Valid time units are 'ns', 'us' (or 'µs'), 'ms', 's', 'm', 'h' (default: 10s)
--tune When present will enable tuning before starting redpanda
--well-known-io string The cloud vendor and VM type, in the format <vendor>:<vm type>:<storage type>
```

### redpanda mode
Expand Down
17 changes: 16 additions & 1 deletion src/go/rpk/pkg/cli/cmd/container/common/common.go
Original file line number Diff line number Diff line change
Expand Up @@ -199,7 +199,7 @@ func RemoveNetwork(c Client) error {

func CreateNode(
c Client,
nodeID, kafkaPort, rpcPort, metricsPort uint,
nodeID, kafkaPort, proxyPort, rpcPort, metricsPort uint,
netID string,
args ...string,
) (*NodeState, error) {
Expand All @@ -217,6 +217,13 @@ func CreateNode(
if err != nil {
return nil, err
}
pPort, err := nat.NewPort(
"tcp",
strconv.Itoa(config.DefaultProxyPort),
)
if err != nil {
return nil, err
}
metPort, err := nat.NewPort(
"tcp",
strconv.Itoa(config.DefaultAdminPort),
Expand All @@ -235,10 +242,14 @@ func CreateNode(
fmt.Sprintf("%d", nodeID),
"--kafka-addr",
fmt.Sprintf("%s:%d", ip, config.DefaultKafkaPort),
"--pandaproxy-addr",
fmt.Sprintf("%s:%d", ip, config.DefaultProxyPort),
"--rpc-addr",
fmt.Sprintf("%s:%d", ip, config.Default().Redpanda.RPCServer.Port),
"--advertise-kafka-addr",
HostAddr(kafkaPort),
"--advertise-pandaproxy-addr",
HostAddr(proxyPort),
"--advertise-rpc-addr",
fmt.Sprintf("%s:%d", ip, config.Default().Redpanda.RPCServer.Port),
"--smp 1 --memory 1G --reserve-memory 0M",
Expand All @@ -249,6 +260,7 @@ func CreateNode(
Cmd: append(cmd, args...),
ExposedPorts: nat.PortSet{
rPort: {},
pPort: {},
kPort: {},
},
Labels: map[string]string{
Expand All @@ -264,6 +276,9 @@ func CreateNode(
kPort: []nat.PortBinding{{
HostPort: fmt.Sprint(kafkaPort),
}},
pPort: []nat.PortBinding{{
HostPort: fmt.Sprint(proxyPort),
}},
metPort: []nat.PortBinding{{
HostPort: fmt.Sprint(metricsPort),
}},
Expand Down
10 changes: 10 additions & 0 deletions src/go/rpk/pkg/cli/cmd/container/start.go
Original file line number Diff line number Diff line change
Expand Up @@ -137,6 +137,10 @@ func startCluster(
if err != nil {
return err
}
seedProxyPort, err := net.GetFreePort()
if err != nil {
return err
}
seedRPCPort, err := net.GetFreePort()
if err != nil {
return err
Expand All @@ -149,6 +153,7 @@ func startCluster(
c,
seedID,
seedKafkaPort,
seedProxyPort,
seedRPCPort,
seedMetricsPort,
netID,
Expand Down Expand Up @@ -184,6 +189,10 @@ func startCluster(
if err != nil {
return err
}
proxyPort, err := net.GetFreePort()
if err != nil {
return err
}
rpcPort, err := net.GetFreePort()
if err != nil {
return err
Expand All @@ -204,6 +213,7 @@ func startCluster(
c,
id,
kafkaPort,
proxyPort,
rpcPort,
metricsPort,
netID,
Expand Down
63 changes: 61 additions & 2 deletions src/go/rpk/pkg/cli/cmd/redpanda/start.go
Original file line number Diff line number Diff line change
Expand Up @@ -105,8 +105,10 @@ func NewStartCommand(
nodeID int
seeds []string
kafkaAddr []string
proxyAddr []string
rpcAddr string
advertisedKafka []string
advertisedProxy []string
advertisedRPC string
installDirFlag string
timeout time.Duration
Expand Down Expand Up @@ -171,6 +173,28 @@ func NewStartCommand(
conf.Redpanda.KafkaApi = kafkaApi
}

proxyAddr = stringSliceOr(
proxyAddr,
strings.Split(
os.Getenv("REDPANDA_PANDAPROXY_ADDRESS"),
",",
),
)
proxyApi, err := parseNamedAddresses(
proxyAddr,
config.DefaultProxyPort,
)
if err != nil {
sendEnv(fs, mgr, env, conf, err)
return err
}
if proxyApi != nil && len(proxyApi) > 0 {
if conf.Pandaproxy == nil {
conf.Pandaproxy = config.Default().Pandaproxy
}
conf.Pandaproxy.PandaproxyAPI = proxyApi
}

rpcAddr = stringOr(
rpcAddr,
os.Getenv("REDPANDA_RPC_ADDRESS"),
Expand Down Expand Up @@ -205,6 +229,29 @@ func NewStartCommand(
if advKafkaApi != nil {
conf.Redpanda.AdvertisedKafkaApi = advKafkaApi
}

advertisedProxy = stringSliceOr(
advertisedProxy,
strings.Split(
os.Getenv("REDPANDA_ADVERTISE_PANDAPROXY_ADDRESS"),
",",
),
)
advProxyApi, err := parseNamedAddresses(
advertisedProxy,
config.DefaultProxyPort,
)
if err != nil {
sendEnv(fs, mgr, env, conf, err)
return err
}
if advProxyApi != nil {
if conf.Pandaproxy == nil {
conf.Pandaproxy = config.Default().Pandaproxy
}
conf.Pandaproxy.AdvertisedPandaproxyAPI = advProxyApi
}

advertisedRPC = stringOr(
advertisedRPC,
os.Getenv("REDPANDA_ADVERTISE_RPC_ADDRESS"),
Expand Down Expand Up @@ -288,7 +335,13 @@ func NewStartCommand(
&kafkaAddr,
"kafka-addr",
[]string{},
"The list of Kafka listener addresses to bind to (<host>:<port>)",
"A comma-separated list of Kafka listener addresses to bind to (<name>://<host>:<port>)",
)
command.Flags().StringSliceVar(
&proxyAddr,
"pandaproxy-addr",
[]string{},
"A comma-separated list of Pandaproxy listener addresses to bind to (<name>://<host>:<port>)",
)
command.Flags().StringVar(
&rpcAddr,
Expand All @@ -300,7 +353,13 @@ func NewStartCommand(
&advertisedKafka,
"advertise-kafka-addr",
[]string{},
"The list of Kafka addresses to advertise (<host>:<port>)",
"A comma-separated list of Kafka addresses to advertise (<name>://<host>:<port>)",
)
command.Flags().StringSliceVar(
&advertisedProxy,
"advertise-pandaproxy-addr",
[]string{},
"A comma-separated list of Pandaproxy addresses to advertise (<name>://<host>:<port>)",
)
command.Flags().StringVar(
&advertisedRPC,
Expand Down
52 changes: 52 additions & 0 deletions src/go/rpk/pkg/cli/cmd/redpanda/start_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -927,6 +927,58 @@ func TestStartCommand(t *testing.T) {
conf.Redpanda.AdvertisedKafkaApi,
)
},
}, {
name: "it should parse the --advertise-pandaproxy-addr and persist it",
args: []string{
"--install-dir", "/var/lib/redpanda",
"--advertise-pandaproxy-addr", "192.168.34.32:8083",
},
postCheck: func(fs afero.Fs, _ *rp.RedpandaArgs, st *testing.T) {
mgr := config.NewManager(fs)
conf, err := mgr.Read(config.Default().ConfigFile)
require.NoError(st, err)
expectedAddr := []config.NamedSocketAddress{{
SocketAddress: config.SocketAddress{
Address: "192.168.34.32",
Port: 8083,
},
}}
// Check that the generated config is as expected.
require.Exactly(
st,
expectedAddr,
conf.Pandaproxy.AdvertisedPandaproxyAPI,
)
},
}, {
name: "if --advertise-pandaproxy-addr, it should fall back to REDPANDA_ADVERTISE_PANDAPROXY_ADDRESS and persist it",
args: []string{
"--install-dir", "/var/lib/redpanda",
},
before: func(_ afero.Fs) error {
os.Setenv("REDPANDA_ADVERTISE_PANDAPROXY_ADDRESS", "host:3123")
return nil
},
after: func() {
os.Unsetenv("REDPANDA_ADVERTISE_PANDAPROXY_ADDRESS")
},
postCheck: func(fs afero.Fs, _ *rp.RedpandaArgs, st *testing.T) {
mgr := config.NewManager(fs)
conf, err := mgr.Read(config.Default().ConfigFile)
require.NoError(st, err)
expectedAddr := []config.NamedSocketAddress{{
SocketAddress: config.SocketAddress{
Address: "host",
Port: 3123,
},
}}
// Check that the generated config is as expected.
require.Exactly(
st,
expectedAddr,
conf.Pandaproxy.AdvertisedPandaproxyAPI,
)
},
}, {
name: "it should parse the --advertise-rpc-addr and persist it",
args: []string{
Expand Down
1 change: 1 addition & 0 deletions src/go/rpk/pkg/config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ const (
ModeProd = "prod"

DefaultKafkaPort = 9092
DefaultProxyPort = 8082
DefaultAdminPort = 9644
)

Expand Down

0 comments on commit 2ef1dfc

Please sign in to comment.