Skip to content

Commit

Permalink
Split channel configuration from LedgerClientConfiguration (digital-a…
Browse files Browse the repository at this point in the history
…sset#12433)

* Split channel configuration from LedgerClientConfiguration

Fixes digital-asset#12391

The channel configuration now has to be provided separately from the
configuration specific to the ledger client. In this way we avoid
situations where the builder is provided with some configuration
that gets overridden.

changelog_begin
[Scala bindings] The channel configuration has been split from the
LedgerClientConfiguration class. Provide the gRPC channel specific
configuration separately or use a builder. The channel configuration
no longer overrides the builder.
changelog_end

* Fix compilation issues in //ledger-service/...
  • Loading branch information
stefanobaghino-da authored Jan 18, 2022
1 parent b988a3c commit 4af48bb
Show file tree
Hide file tree
Showing 37 changed files with 246 additions and 165 deletions.
1 change: 0 additions & 1 deletion daml-lf/engine/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -146,6 +146,5 @@ da_scala_test(
"//libs-scala/ports",
"//libs-scala/resources",
"@maven//:com_google_protobuf_protobuf_java",
"@maven//:io_netty_netty_handler",
],
)
Original file line number Diff line number Diff line change
Expand Up @@ -69,15 +69,14 @@ final class MinVersionTest
applicationId = "minversiontest",
ledgerIdRequirement = LedgerIdRequirement.none,
commandClient = CommandClientConfiguration.default,
sslContext = None,
)
// This is an integration test to make sure that the version restrictions for stable packages
// apply across the whole stack.

"MinVersionTest" - {
"can upload an LF 1.14 package and use it in a transaction" in {
for {
client <- LedgerClient.singleHost(
client <- LedgerClient.insecureSingleHost(
"localhost",
suiteResource.value.value,
ledgerClientConfig,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,6 @@ trait ReproducesTransactions
applicationId = appId,
ledgerIdRequirement = LedgerIdRequirement.none,
commandClient = CommandClientConfiguration.default,
sslContext = None,
token = None,
)
val isWindows: Boolean = sys.props("os.name").toLowerCase.contains("windows")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@ import com.daml.grpc.adapter.{AkkaExecutionSequencerPool, ExecutionSequencerFact
import com.daml.ledger.client.LedgerClient
import com.daml.ledger.client.configuration.{
CommandClientConfiguration,
LedgerClientChannelConfiguration,
LedgerClientConfiguration,
LedgerIdRequirement,
}
Expand Down Expand Up @@ -53,6 +54,7 @@ object Main {
config.ledgerHost,
config.ledgerPort,
clientConfig(config),
clientChannelConfig(config),
)
parties <- LedgerUtils.getAllParties(client, config.accessToken, config.partyConfig)
acs <- LedgerUtils.getACS(client, parties, config.start)
Expand Down Expand Up @@ -82,8 +84,12 @@ object Main {
applicationId = "script-export",
ledgerIdRequirement = LedgerIdRequirement.none,
commandClient = CommandClientConfiguration.default,
sslContext = config.tlsConfig.client(),
token = config.accessToken.flatMap(_.token),
maxInboundMessageSize = config.maxInboundMessageSize,
)

private def clientChannelConfig(config: Config): LedgerClientChannelConfiguration =
LedgerClientChannelConfiguration(
sslContext = config.tlsConfig.client(),
maxInboundMessageSize = config.maxInboundMessageSize,
)
}
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ import com.daml.ledger.api.tls.TlsConfiguration
import com.daml.ledger.client.LedgerClient
import com.daml.ledger.client.configuration.{
CommandClientConfiguration,
LedgerClientChannelConfiguration,
LedgerClientConfiguration,
LedgerIdRequirement,
}
Expand Down Expand Up @@ -230,12 +231,14 @@ object Runner {
applicationId = ApplicationId.unwrap(applicationId),
ledgerIdRequirement = LedgerIdRequirement.none,
commandClient = CommandClientConfiguration.default,
sslContext = tlsConfig.client(),
token = params.access_token,
)
val clientChannelConfig = LedgerClientChannelConfiguration(
sslContext = tlsConfig.client(),
maxInboundMessageSize = maxInboundMessageSize,
)
LedgerClient
.singleHost(params.host, params.port, clientConfig)
.singleHost(params.host, params.port, clientConfig, clientChannelConfig)
.map(new GrpcLedgerClient(_, applicationId))
}
// We might want to have one config per participant at some point but for now this should be sufficient.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -557,7 +557,6 @@ final class JsonApiIt
applicationId = "appid",
ledgerIdRequirement = LedgerIdRequirement.none,
commandClient = CommandClientConfiguration.default,
sslContext = None,
token = Some(getUserToken(UserId.assertFromString("participant_admin"))),
),
)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,11 +52,10 @@ trait SandboxFixture extends SandboxNextFixture {
seeding = Some(Seeding.Weak),
)

protected val ClientConfiguration = LedgerClientConfiguration(
protected val ClientConfiguration: LedgerClientConfiguration = LedgerClientConfiguration(
applicationId = TestUtil.LedgerID,
ledgerIdRequirement = LedgerIdRequirement.none,
commandClient = CommandClientConfiguration.default,
sslContext = None,
token = None,
)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -8,35 +8,21 @@ import java.time.Duration

import com.daml.ledger.client.binding.LedgerClientConfigurationError.MalformedTypesafeConfig
import com.daml.ledger.client.binding.config.LedgerClientConfig.ClientSslConfig
import com.daml.ledger.client.configuration.{
CommandClientConfiguration,
LedgerClientConfiguration,
LedgerIdRequirement,
}
import com.typesafe.config.{Config, ConfigFactory}
import com.daml.ledger.client.configuration.CommandClientConfiguration
import com.typesafe.config.{ConfigFactory, Config}
import io.grpc.netty.GrpcSslContexts
import io.netty.handler.ssl.SslContext
import pureconfig._
import pureconfig.generic.auto._

import scala.util.Try

case class LedgerClientConfig(
final case class LedgerClientConfig(
ledgerId: Option[String],
commandClient: CommandClientConfiguration,
maxRetryTime: Duration,
ssl: Option[ClientSslConfig],
) {
def toBindingConfig(applicationId: String) =
LedgerClientConfiguration(
applicationId,
ledgerIdRequirement,
commandClient,
ssl.map(_.sslContext),
)

private val ledgerIdRequirement = LedgerIdRequirement(ledgerId)
}
)

object LedgerClientConfig {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -90,7 +90,6 @@ class ScalaCodeGenIT
applicationId = applicationId,
ledgerIdRequirement = LedgerIdRequirement.matching(ledgerId),
commandClient = CommandClientConfiguration.default,
sslContext = None,
)

private var ledger: LedgerClient = _
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ import com.daml.ledger.client.configuration.{
CommandClientConfiguration,
LedgerClientConfiguration,
LedgerIdRequirement,
LedgerClientChannelConfiguration,
}
import com.daml.quickstart.iou.ClientUtil.workflowIdFromParty
import com.daml.quickstart.iou.DecodeUtil.decodeCreatedEvent
Expand Down Expand Up @@ -59,11 +60,12 @@ object IouMain extends App with StrictLogging {
applicationId = ApplicationId.unwrap(applicationId),
ledgerIdRequirement = LedgerIdRequirement.none,
commandClient = CommandClientConfiguration.default,
sslContext = None,
)

private val clientChannelConfig = LedgerClientChannelConfiguration.InsecureDefaults

private val clientF: Future[LedgerClient] =
LedgerClient.singleHost(ledgerHost, ledgerPort, clientConfig)(ec, aesf)
LedgerClient.singleHost(ledgerHost, ledgerPort, clientConfig, clientChannelConfig)(ec, aesf)

private val clientUtilF: Future[ClientUtil] =
clientF.map(client => new ClientUtil(client, applicationId))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ import com.daml.ledger.client.configuration.{
CommandClientConfiguration,
LedgerClientConfiguration,
LedgerIdRequirement,
LedgerClientChannelConfiguration,
}
import com.daml.quickstart.iou.ClientUtil.workflowIdFromParty
import com.daml.quickstart.iou.DecodeUtil.{decodeAllCreated, decodeArchived, decodeCreated}
Expand Down Expand Up @@ -65,13 +66,14 @@ object IouMain extends App with StrictLogging {
applicationId = ApplicationId.unwrap(applicationId),
ledgerIdRequirement = LedgerIdRequirement.none,
commandClient = CommandClientConfiguration.default,
sslContext = None,
token = None,
)
// </doc-ref:ledger-client-configuration>

private val clientChannelConfig = LedgerClientChannelConfiguration.InsecureDefaults

private val clientF: Future[LedgerClient] =
LedgerClient.singleHost(ledgerHost, ledgerPort, clientConfig)(ec, aesf)
LedgerClient.singleHost(ledgerHost, ledgerPort, clientConfig, clientChannelConfig)(ec, aesf)

private val clientUtilF: Future[ClientUtil] =
clientF.map(client => new ClientUtil(client, applicationId))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3,6 +3,7 @@

package com.daml.http

import com.daml.ledger.client.configuration.LedgerClientChannelConfiguration
import io.grpc.netty.NettyChannelBuilder

import scala.concurrent.{ExecutionContext, Future}
Expand All @@ -12,8 +13,9 @@ object LedgerClient extends LedgerClientBase {
def channelBuilder(
ledgerHost: String,
ledgerPort: Int,
clientChannelConfig: LedgerClientChannelConfiguration,
nonRepudiationConfig: nonrepudiation.Configuration.Cli,
)(implicit executionContext: ExecutionContext): Future[NettyChannelBuilder] =
Future(NettyChannelBuilder.forAddress(ledgerHost, ledgerPort))
Future(clientChannelConfig.builderFor(ledgerHost, ledgerPort))

}
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@ import java.security.{KeyFactory, PrivateKey}
import java.security.cert.{CertificateFactory, X509Certificate}
import java.security.spec.PKCS8EncodedKeySpec

import com.daml.ledger.client.configuration.LedgerClientChannelConfiguration
import com.daml.nonrepudiation.client.SigningInterceptor
import io.grpc.netty.NettyChannelBuilder

Expand Down Expand Up @@ -44,9 +45,10 @@ object LedgerClient extends LedgerClientBase {
def channelBuilder(
ledgerHost: String,
ledgerPort: Int,
clientChannelConfig: LedgerClientChannelConfiguration,
nonRepudiationConfig: nonrepudiation.Configuration.Cli,
)(implicit executionContext: ExecutionContext): Future[NettyChannelBuilder] = {
val base = NettyChannelBuilder.forAddress(ledgerHost, ledgerPort)
val base = clientChannelConfig.builderFor(ledgerHost, ledgerPort)
Future
.fromTry(nonRepudiationConfig.validated)
.map(_.fold(base) { config =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -4,7 +4,10 @@
package com.daml.http

import com.daml.grpc.adapter.ExecutionSequencerFactory
import com.daml.ledger.client.configuration.LedgerClientConfiguration
import com.daml.ledger.client.configuration.{
LedgerClientChannelConfiguration,
LedgerClientConfiguration,
}
import com.daml.scalautil.ExceptionOps._
import com.daml.ledger.client.withoutledgerid.{LedgerClient => DamlLedgerClient}
import io.grpc.netty.NettyChannelBuilder
Expand Down Expand Up @@ -32,13 +35,15 @@ trait LedgerClientBase {
protected def channelBuilder(
ledgerHost: String,
ledgerPort: Int,
clientChannelConfig: LedgerClientChannelConfiguration,
nonRepudiationConfig: nonrepudiation.Configuration.Cli,
)(implicit executionContext: ExecutionContext): Future[NettyChannelBuilder]

private def buildLedgerClient(
ledgerHost: String,
ledgerPort: Int,
clientConfig: LedgerClientConfiguration,
clientChannelConfig: LedgerClientChannelConfiguration,
nonRepudiationConfig: nonrepudiation.Configuration.Cli,
)(implicit
ec: ExecutionContext,
Expand All @@ -47,13 +52,15 @@ trait LedgerClientBase {
channelBuilder(
ledgerHost,
ledgerPort,
clientChannelConfig,
nonRepudiationConfig,
).map(builder => DamlLedgerClient.fromBuilder(builder, clientConfig))

def fromRetried(
ledgerHost: String,
ledgerPort: Int,
clientConfig: LedgerClientConfiguration,
clientChannelConfig: LedgerClientChannelConfiguration,
nonRepudiationConfig: nonrepudiation.Configuration.Cli,
maxInitialConnectRetryAttempts: Int,
)(implicit
Expand All @@ -66,7 +73,13 @@ trait LedgerClientBase {
)
RetryStrategy
.constant(maxInitialConnectRetryAttempts, 1.seconds) { (i, _) =>
val client = buildLedgerClient(ledgerHost, ledgerPort, clientConfig, nonRepudiationConfig)
val client = buildLedgerClient(
ledgerHost,
ledgerPort,
clientConfig,
clientChannelConfig,
nonRepudiationConfig,
)
client.onComplete {
case Success(_) =>
logger.info(s"""Attempt $i/$maxInitialConnectRetryAttempts succeeded!""")
Expand All @@ -90,12 +103,19 @@ trait LedgerClientBase {
ledgerHost: String,
ledgerPort: Int,
clientConfig: LedgerClientConfiguration,
clientChannelConfig: LedgerClientChannelConfiguration,
nonRepudiationConfig: nonrepudiation.Configuration.Cli,
)(implicit
ec: ExecutionContext,
aesf: ExecutionSequencerFactory,
): Future[LedgerClientBase.Error \/ DamlLedgerClient] =
buildLedgerClient(ledgerHost, ledgerPort, clientConfig, nonRepudiationConfig)
buildLedgerClient(
ledgerHost,
ledgerPort,
clientConfig,
clientChannelConfig,
nonRepudiationConfig,
)
.map(_.right)
.recover { case NonFatal(e) =>
\/.left(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,7 @@ package com.daml.http

import java.io.File
import java.time.Instant

import akka.actor.ActorSystem
import akka.http.scaladsl.Http
import akka.http.scaladsl.Http.ServerBinding
Expand Down Expand Up @@ -37,6 +38,7 @@ import com.daml.ledger.api.tls.TlsConfiguration
import com.daml.ledger.api.v1.{value => v}
import com.daml.ledger.client.configuration.{
CommandClientConfiguration,
LedgerClientChannelConfiguration,
LedgerClientConfiguration,
LedgerIdRequirement,
}
Expand Down Expand Up @@ -122,7 +124,8 @@ object HttpServiceTestFixture extends LazyLogging with Assertions with Inside {
val client = DamlLedgerClient.singleHost(
"localhost",
ledgerPort.value,
clientConfig(applicationId, useTls = useTls),
clientConfig(applicationId),
clientChannelConfig(useTls),
)

val codecsF: Future[(DomainJsonEncoder, DomainJsonDecoder)] = for {
Expand Down Expand Up @@ -192,7 +195,8 @@ object HttpServiceTestFixture extends LazyLogging with Assertions with Inside {
} yield DamlLedgerClient.singleHost(
"localhost",
ledgerPort.value,
clientConfig(applicationId, token, useTls),
clientConfig(applicationId, token),
clientChannelConfig(useTls),
)

val fa: Future[A] = for {
Expand Down Expand Up @@ -228,19 +232,24 @@ object HttpServiceTestFixture extends LazyLogging with Assertions with Inside {
seeding = Some(Seeding.Weak),
)

private def clientConfig[A](
private def clientConfig(
applicationId: ApplicationId,
token: Option[String] = None,
useTls: UseTls,
): LedgerClientConfiguration =
LedgerClientConfiguration(
applicationId = ApplicationId.unwrap(applicationId),
ledgerIdRequirement = LedgerIdRequirement.none,
commandClient = CommandClientConfiguration.default,
sslContext = if (useTls) clientTlsConfig.client() else None,
token = token,
)

private def clientChannelConfig(useTls: UseTls): LedgerClientChannelConfiguration =
if (useTls) {
LedgerClientChannelConfiguration(clientTlsConfig.client())
} else {
LedgerClientChannelConfiguration.InsecureDefaults
}

def jsonCodecs(
client: DamlLedgerClient,
ledgerId: LedgerId,
Expand Down
Loading

0 comments on commit 4af48bb

Please sign in to comment.