Integrate non-repudiation into HTTP JSON API (#9180)

* Integrate non-repudiation into HTTP JSON API

changelog_begin
changelog_end

Closes #9094

* Address https://github.com/digital-asset/daml/pull/9180#discussion_r596400828

* Address https://github.com/digital-asset/daml/pull/9180#discussion_r596408663

* Address https://github.com/digital-asset/daml/pull/9180#discussion_r596402533

* Address https://github.com/digital-asset/daml/pull/9180#discussion_r596624708

* Address https://github.com/digital-asset/daml/pull/9180#discussion_r596624174

* Solve compilation error in daml-script

* Address https://github.com/digital-asset/daml/pull/9180#issuecomment-801875959

changelog_begin
- [JSON API] The JAR for the HTTP JSON API is no longer published on
Maven, use the fat JAR as indicated in the documentation.
changelog_end
This commit is contained in:
Stefano Baghino 2021-03-18 16:15:52 +01:00 committed by GitHub
parent dbd8806848
commit f0c8b1240d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
11 changed files with 356 additions and 11 deletions

View File

@ -12,6 +12,7 @@ import akka.stream.Materializer
import com.daml.bazeltools.BazelRunfiles._
import com.daml.grpc.adapter.{AkkaExecutionSequencerPool, ExecutionSequencerFactory}
import com.daml.http.HttpService
import com.daml.http.nonrepudiation
import com.daml.jwt.domain.DecodedJwt
import com.daml.jwt.{HMAC256Verifier, JwtSigner}
import com.daml.ledger.api.auth.{AuthServiceJWT, AuthServiceJWTCodec, AuthServiceJWTPayload}
@ -144,6 +145,7 @@ trait JsonApiFixture
override val wsConfig = None
override val accessTokenFile = Some(jsonAccessTokenFile)
override val allowNonHttps = true
override val nonRepudiation = nonrepudiation.Configuration.Cli.Empty
}
HttpService
.start(config)(

View File

@ -68,6 +68,7 @@ object HttpServiceTestFixture extends LazyLogging with Assertions with Inside {
leakPasswords: LeakPasswords = LeakPasswords.FiresheepStyle,
useTls: UseTls = UseTls.NoTls,
wsConfig: Option[WebsocketConfig] = None,
nonRepudiation: nonrepudiation.Configuration.Cli = nonrepudiation.Configuration.Cli.Empty,
)(testFn: (Uri, DomainJsonEncoder, DomainJsonDecoder, LedgerClient) => Future[A])(implicit
asys: ActorSystem,
mat: Materializer,
@ -93,6 +94,7 @@ object HttpServiceTestFixture extends LazyLogging with Assertions with Inside {
allowNonHttps = leakPasswords,
staticContentConfig = staticContentConfig,
packageReloadInterval = doNotReloadPackages,
nonRepudiation = nonRepudiation,
)
httpService <- stripLeft(
HttpService.start(

View File

@ -67,6 +67,7 @@ da_scala_library(
"//libs-scala/doobie-slf4j",
"//libs-scala/ports",
"//libs-scala/scala-utils",
"//runtime-components/non-repudiation-client",
],
)
@ -248,6 +249,7 @@ da_scala_test(
"@maven//:org_scalaz_scalaz_core",
"@maven//:org_tpolecat_doobie_core",
"@maven//:org_tpolecat_doobie_free",
"@maven//:org_tpolecat_doobie_hikari",
"@maven//:org_typelevel_cats_core",
"@maven//:org_typelevel_cats_effect",
"@maven//:org_typelevel_cats_free",
@ -269,9 +271,15 @@ da_scala_test(
"//ledger/ledger-api-auth",
"//ledger/ledger-api-common",
"//libs-scala/auth-utils",
"//libs-scala/doobie-slf4j",
"//libs-scala/ports",
"//libs-scala/postgresql-testing",
"//libs-scala/resources",
"//libs-scala/resources-grpc",
"//libs-scala/scala-utils",
"//runtime-components/non-repudiation",
"//runtime-components/non-repudiation-postgresql",
"//runtime-components/non-repudiation-testing",
],
)

View File

@ -0,0 +1,171 @@
// Copyright (c) 2021 Digital Asset (Switzerland) GmbH and/or its affiliates. All rights reserved.
// SPDX-License-Identifier: Apache-2.0
package com.daml.http
import java.nio.file.Files
import java.security.cert.X509Certificate
import java.time.Clock
import java.util.UUID
import akka.http.scaladsl.model.{StatusCodes, Uri}
import com.daml.doobie.logging.Slf4jLogHandler
import com.daml.http.AbstractHttpServiceIntegrationTestFuns.{dar1, dar2}
import com.daml.http.json.{DomainJsonDecoder, DomainJsonEncoder}
import com.daml.ledger.api.v1.command_service.CommandServiceGrpc
import com.daml.ledger.api.v1.command_submission_service.{
CommandSubmissionServiceGrpc,
SubmitRequest,
}
import com.daml.ledger.api.v1.value.Value.Sum
import com.daml.ledger.api.v1.value.{RecordField, Value, Variant}
import com.daml.ledger.client.LedgerClient
import com.daml.nonrepudiation.postgresql.{Tables, createTransactor}
import com.daml.nonrepudiation.testing.generateKeyAndCertificate
import com.daml.nonrepudiation.{CommandIdString, NonRepudiationProxy}
import com.daml.ports.{FreePort, Port}
import com.daml.resources.grpc.GrpcResourceOwnerFactories
import com.daml.testing.postgresql.PostgresAroundEach
import io.grpc.Server
import io.grpc.netty.{NettyChannelBuilder, NettyServerBuilder}
import org.scalatest.freespec.AsyncFreeSpec
import org.scalatest.matchers.should.Matchers
import org.scalatest.{Assertion, BeforeAndAfterEach, Inside}
import scala.concurrent.Future
import scala.concurrent.duration.DurationInt
@SuppressWarnings(Array("org.wartremover.warts.NonUnitStatements"))
final class NonRepudiationTest
extends AsyncFreeSpec
with Matchers
with Inside
with BeforeAndAfterEach
with AbstractHttpServiceIntegrationTestFuns
with PostgresAroundEach {
import HttpServiceTestFixture._
private var nonRepudiation: nonrepudiation.Configuration.Cli = _
private var certificate: X509Certificate = _
override protected def beforeEach(): Unit = {
super.beforeEach()
val (key, cert) = generateKeyAndCertificate()
certificate = cert
val certificatePath = Files.createTempFile("non-repudiation-test", "certificate")
val privateKeyPath = Files.createTempFile("non-repudiation-test", "key")
Files.write(certificatePath, cert.getEncoded)
Files.write(privateKeyPath, key.getEncoded)
nonRepudiation =
nonrepudiation.Configuration.Cli(certificatePath, privateKeyPath, key.getAlgorithm)
}
override protected def afterEach(): Unit = {
super.afterEach()
nonRepudiation.certificateFile.foreach(Files.delete)
nonRepudiation.privateKeyFile.foreach(Files.delete)
}
override val jdbcConfig: Option[JdbcConfig] = None
override val staticContentConfig: Option[StaticContentConfig] = None
override val useTls: UseTls = UseTls.NoTls
override val wsConfig: Option[WebsocketConfig] = None
"correctly sign a command" in withSetup { (db, uri, encoder) =>
val expectedParty = "Alice"
val expectedNumber = "abc123"
val expectedCommandId = UUID.randomUUID.toString
val meta = Some(domain.CommandMeta(commandId = Some(domain.CommandId(expectedCommandId))))
val domainParty = domain.Party(expectedParty)
val command = accountCreateCommand(domainParty, expectedNumber).copy(meta = meta)
postCreateCommand(command, encoder, uri)
.flatMap { case (status, _) =>
status shouldBe StatusCodes.OK
val payloads = db.signedPayloads.get(CommandIdString.wrap(expectedCommandId))
payloads should have size 1
val signedCommand = SubmitRequest.parseFrom(payloads.head.payload.unsafeArray)
val commands = signedCommand.getCommands.commands
commands should have size 1
val actualFields = commands.head.getCreate.getCreateArguments.fields.map(stripIdentifiers)
val expectedFields = command.payload.fields.map(stripIdentifiers)
actualFields should contain theSameElementsAs expectedFields
}
}
private def stripIdentifiers(field: RecordField): RecordField =
field.copy(value = Some(stripIdentifiers(field.getValue)))
// Doesn't aim at being complete, neither in stripping identifiers recursively
// Only covers variant because it's the only case interesting for the test cases here
private def stripIdentifiers(value: Value): Value =
value match {
case Value(Sum.Variant(Variant(Some(_), constructor, value))) =>
Value(Sum.Variant(Variant(None, constructor, value)))
case _ => value
}
private def withParticipant[A] =
HttpServiceTestFixture.withLedger[A](List(dar1, dar2), testId, None, useTls) _
private def withJsonApi[A](participantPort: Port) =
HttpServiceTestFixture.withHttpService[A](
testId,
participantPort,
jdbcConfig,
staticContentConfig,
LeakPasswords.No,
useTls,
wsConfig,
nonRepudiation,
) _
private def withSetup[A](test: (Tables, Uri, DomainJsonEncoder) => Future[Assertion]) =
withParticipant { case (participantPort, _: LedgerClient) =>
val participantChannelBuilder =
NettyChannelBuilder
.forAddress("localhost", participantPort.value)
.usePlaintext()
val proxyPort = FreePort.find()
val proxyBuilder = NettyServerBuilder.forPort(proxyPort.value)
val setup =
for {
participantChannel <- GrpcResourceOwnerFactories.forChannel(
participantChannelBuilder,
shutdownTimeout = 5.seconds,
)
transactor <- createTransactor(
postgresDatabase.url,
postgresDatabase.userName,
postgresDatabase.password,
maxPoolSize = 10,
GrpcResourceOwnerFactories,
)
db = Tables.initialize(transactor)(Slf4jLogHandler(getClass))
_ = db.certificates.put(certificate)
proxy <- NonRepudiationProxy.owner(
participantChannel,
proxyBuilder,
db.certificates,
db.signedPayloads,
Clock.systemUTC(),
CommandServiceGrpc.CommandService.scalaDescriptor.fullName,
CommandSubmissionServiceGrpc.CommandSubmissionService.scalaDescriptor.fullName,
)
} yield (proxy, db)
setup.use { case (_: Server, db: Tables) =>
withJsonApi(proxyPort) { (uri, encoder, _: DomainJsonDecoder, _: LedgerClient) =>
test(db, uri, encoder)
}
}
}
}

View File

@ -47,9 +47,9 @@ import scala.concurrent.{ExecutionContext, Future}
import scala.util.{Success, Try}
object AbstractHttpServiceIntegrationTestFuns {
private val dar1 = requiredResource("docs/quickstart-model.dar")
private[http] val dar1 = requiredResource("docs/quickstart-model.dar")
private val dar2 = requiredResource("ledger-service/http-json/Account.dar")
private[http] val dar2 = requiredResource("ledger-service/http-json/Account.dar")
private[http] val dar3 = requiredResource(com.daml.ledger.test.TestDars.fileNames("model"))
@ -87,7 +87,7 @@ trait AbstractHttpServiceIntegrationTestFuns extends StrictLogging {
protected def testId: String = this.getClass.getSimpleName
protected val metdata2: MetadataReader.LfMetadata =
protected val metadata2: MetadataReader.LfMetadata =
MetadataReader.readFromDar(dar2).valueOr(e => fail(s"Cannot read dar2 metadata: $e"))
protected val jwt: Jwt = jwtForParties(List("Alice"), List(), testId)
@ -1427,7 +1427,7 @@ abstract class AbstractHttpServiceIntegrationTest
accountCreateCommand(owner, accountNumber, now)
val packageId: Ref.PackageId = MetadataReader
.templateByName(metdata2)(Ref.QualifiedName.assertFromString("Account:Account"))
.templateByName(metadata2)(Ref.QualifiedName.assertFromString("Account:Account"))
.headOption
.map(_._1)
.getOrElse(fail(s"Cannot retrieve packageId"))

View File

@ -166,5 +166,39 @@ object Cli extends StrictLogging {
.optional()
.valueName(WebsocketConfig.usage)
.text(s"Optional websocket configuration string. ${WebsocketConfig.help}")
opt[String]("non-repudiation-certificate-path")
.action((path, config) =>
config.copy(nonRepudiation =
config.nonRepudiation.copy(certificateFile = Some(Paths.get(path)))
)
)
.text(nonRepudiationText)
.hidden()
opt[String]("non-repudiation-private-key-path")
.action((path, config) =>
config.copy(nonRepudiation =
config.nonRepudiation.copy(privateKeyFile = Some(Paths.get(path)))
)
)
.text(nonRepudiationText)
.hidden()
opt[String]("non-repudiation-private-key-algorithm")
.action((algorithm, config) =>
config.copy(nonRepudiation =
config.nonRepudiation.copy(privateKeyAlgorithm = Some(algorithm))
)
)
.text(nonRepudiationText)
.hidden()
}
private val nonRepudiationText: String =
"""EARLY ACCESS FEATURE
|--non-repudiation-certificate-path, --non-repudiation-private-key-path and --non-repudiation-certificate-path must be passed together.
|All commands issued by the HTTP JSON API will be signed with the private key and the X.509 certificate at the provided paths.
|This is relevant exclusively if you are using the non-repudiation middleware.""".stripMargin
}

View File

@ -35,6 +35,7 @@ private[http] final case class Config(
allowNonHttps: Boolean = false,
accessTokenFile: Option[Path] = None,
wsConfig: Option[WebsocketConfig] = None,
nonRepudiation: nonrepudiation.Configuration.Cli = nonrepudiation.Configuration.Cli.Empty,
) extends HttpService.StartSettings
private[http] object Config {

View File

@ -3,7 +3,10 @@
package com.daml.http
import java.nio.file.Path
import java.nio.file.{Files, Path}
import java.security.cert.{CertificateFactory, X509Certificate}
import java.security.spec.PKCS8EncodedKeySpec
import java.security.{KeyFactory, PrivateKey}
import akka.actor.{ActorSystem, Cancellable}
import akka.http.scaladsl.Http
@ -35,15 +38,18 @@ import com.daml.ledger.client.configuration.{
import com.daml.ledger.client.services.pkg.PackageClient
import com.daml.ledger.service.LedgerReader
import com.daml.ledger.service.LedgerReader.PackageStore
import com.daml.nonrepudiation.client.SigningInterceptor
import com.daml.ports.{Port, PortFiles}
import com.daml.scalautil.Statement.discard
import com.daml.util.ExceptionOps._
import com.typesafe.scalalogging.StrictLogging
import io.grpc.netty.NettyChannelBuilder
import scalaz.Scalaz._
import scalaz._
import scala.concurrent.duration.FiniteDuration
import scala.concurrent.{Await, ExecutionContext, Future}
import scala.util.Try
import scala.util.control.NonFatal
object HttpService extends StrictLogging {
@ -77,6 +83,7 @@ object HttpService extends StrictLogging {
val packageMaxInboundMessageSize: Option[Int]
val maxInboundMessageSize: Int
val healthTimeoutSeconds: Int
val nonRepudiation: nonrepudiation.Configuration.Cli
}
trait DefaultStartSettings extends StartSettings {
@ -87,6 +94,31 @@ object HttpService extends StrictLogging {
override val healthTimeoutSeconds: Int = DefaultHealthTimeoutSeconds
}
def loadCertificate(
path: Path
): Try[X509Certificate] = {
val newInputStream = Try(Files.newInputStream(path))
val certificate =
for {
input <- newInputStream
factory <- Try(CertificateFactory.getInstance("X.509"))
certificate <- Try(factory.generateCertificate(input).asInstanceOf[X509Certificate])
} yield certificate
newInputStream.foreach(_.close())
certificate
}
def loadPrivateKey(
path: Path,
algorithm: String,
): Try[PrivateKey] =
for {
bytes <- Try(Files.readAllBytes(path))
keySpec <- Try(new PKCS8EncodedKeySpec(bytes))
factory <- Try(KeyFactory.getInstance(algorithm))
key <- Try(factory.generatePrivate(keySpec))
} yield key
def start(
startSettings: StartSettings,
contractDao: Option[ContractDao] = None,
@ -114,7 +146,12 @@ object HttpService extends StrictLogging {
val bindingEt: EitherT[Future, Error, ServerBinding] = for {
client <- eitherT(
ledgerClient(ledgerHost, ledgerPort, clientConfig)
ledgerClient(
ledgerHost,
ledgerPort,
clientConfig,
startSettings.nonRepudiation,
)
): ET[LedgerClient]
pkgManagementClient <- eitherT(
@ -124,6 +161,7 @@ object HttpService extends StrictLogging {
packageMaxInboundMessageSize.fold(clientConfig)(size =>
clientConfig.copy(maxInboundMessageSize = size)
),
startSettings.nonRepudiation,
)
): ET[LedgerClient]
@ -328,17 +366,48 @@ object HttpService extends StrictLogging {
})
}
private def channelBuilder(
ledgerHost: String,
ledgerPort: Int,
nonRepudiationConfig: nonrepudiation.Configuration.Cli,
)(implicit executionContext: ExecutionContext): Future[NettyChannelBuilder] = {
val base = NettyChannelBuilder.forAddress(ledgerHost, ledgerPort)
Future
.fromTry(nonRepudiationConfig.validated)
.map(_.fold(base) { config =>
val channelWithInterceptor =
for {
certificate <- loadCertificate(config.certificateFile)
key <- loadPrivateKey(config.privateKeyFile, config.privateKeyAlgorithm)
} yield base.intercept(SigningInterceptor.signCommands(key, certificate))
channelWithInterceptor.get
})
}
private def ledgerClient(
ledgerHost: String,
ledgerPort: Int,
clientConfig: LedgerClientConfiguration,
)(implicit ec: ExecutionContext, aesf: ExecutionSequencerFactory): Future[Error \/ LedgerClient] =
LedgerClient
.singleHost(ledgerHost, ledgerPort, clientConfig)
nonRepudiationConfig: nonrepudiation.Configuration.Cli,
)(implicit
ec: ExecutionContext,
aesf: ExecutionSequencerFactory,
): Future[Error \/ LedgerClient] = {
val client = for {
builder <- channelBuilder(
ledgerHost,
ledgerPort,
nonRepudiationConfig,
)
client <- LedgerClient.fromBuilder(builder, clientConfig)
} yield client
client
.map(_.right)
.recover { case NonFatal(e) =>
\/.left(Error(s"Cannot connect to the ledger server, error: ${e.description}"))
}
}
private def createPortFile(
file: Path,

View File

@ -52,6 +52,9 @@ object Main extends StrictLogging {
s", allowNonHttps=${config.allowNonHttps.shows}" +
s", accessTokenFile=${config.accessTokenFile: Option[Path]}" +
s", wsConfig=${config.wsConfig.shows}" +
s", nonRepudiationCertificateFile=${config.nonRepudiation.certificateFile: Option[Path]}" +
s", nonRepudiationPrivateKeyFile=${config.nonRepudiation.privateKeyFile: Option[Path]}" +
s", nonRepudiationPrivateKeyAlgorithm=${config.nonRepudiation.privateKeyAlgorithm: Option[String]}" +
")"
)

View File

@ -0,0 +1,57 @@
// Copyright (c) 2021 Digital Asset (Switzerland) GmbH and/or its affiliates. All rights reserved.
// SPDX-License-Identifier: Apache-2.0
package com.daml.http.nonrepudiation
import java.nio.file.Path
import scala.util.{Failure, Success, Try}
sealed abstract class Configuration[F[_]] {
def certificateFile: F[Path]
def privateKeyFile: F[Path]
def privateKeyAlgorithm: F[String]
}
object Configuration {
type Id[X] = X
final case class Validated(
certificateFile: Path,
privateKeyFile: Path,
privateKeyAlgorithm: String,
) extends Configuration[Id]
final case class Cli(
certificateFile: Option[Path],
privateKeyFile: Option[Path],
privateKeyAlgorithm: Option[String],
) extends Configuration[Option] {
lazy val validated: Try[Option[Validated]] =
(certificateFile, privateKeyFile, privateKeyAlgorithm) match {
case (None, None, None) =>
Success(None)
case (Some(cf), Some(kf), Some(ka)) =>
Success(Some(Validated(cf, kf, ka)))
case _ =>
Failure(validationError())
}
}
object Cli {
def apply(
certificateFile: Path,
privateKeyFile: Path,
privateKeyAlgorithm: String,
): Cli = Cli(Some(certificateFile), Some(privateKeyFile), Some(privateKeyAlgorithm))
val Empty: Cli = Cli(None, None, None)
}
private def validationError(): IllegalArgumentException =
new IllegalArgumentException("Either all or none of the non-repudiation options must be passed")
}

View File

@ -153,8 +153,6 @@
type: jar-scala
- target: //ledger-service/db-backend:db-backend
type: jar-scala
- target: //ledger-service/http-json:http-json
type: jar-scala
- target: //ledger-service/lf-value-json:lf-value-json
type: jar-scala
- target: //ledger-service/cli-opts:cli-opts