2
0
mirror of https://github.com/VinylDNS/vinyldns synced 2025-08-22 10:10:12 +00:00

Parallelizing the build (#230)

Using the `all` command in sbt allows us to run certain tasks
in parallel.  Not everything can be done in parallel, so we have to use
judgement here.  Some things like dockerCompose cannot be done in parallel.

With the adjustments, local `;validate;verify` went from 12 minutes to 9
minutes, a savings of 3 full minutes!

**build.sbt**
Most changes are here.

Allow parallelExecution by default.  The reason is that test suites will be run
in parallel.  The play framework by default turns this off.  Some of the changes
made in here around the `InMemoryBatchChangeRepository` were necessary when I
flipped this on.

We cannot run in parallel by default for IntegrationTest.  The reason is that
several of the api integration tests use the same zone repo, so they wind up
stomping on each other.

Added a `killDocker` task that is much faster to run than `dockerComposeStop`

Enabled parallelExecution in IntegrationTest in dynamodb.  The integration tests
do not conflict with each other here.

Changed the command aliases to use parallel `all`

**InMemoryBatchCHangeRepository**
This used to be a singleton, which prevents running unit tests in parallel.
Made this a class, and updated unit tests to use the class instead.

**logback-test.xml**
We were still logging in odd ways in places.  Removed this to turn logging off
by default for tests.

**.jvmopts**
Kept running out of metaspace.  Increased the memory needed to help slow that
down.
This commit is contained in:
Paul Cleary 2018-09-21 16:24:49 -04:00 committed by GitHub
parent 56746e9763
commit e3fd1bc43a
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
10 changed files with 46 additions and 198 deletions

View File

@ -1,7 +1,8 @@
-Xms512M
-Xmx2048M
-Xmx4096M
-Xss2M
-XX:MaxMetaspaceSize=1024M
-XX:MaxMetaspaceSize=2048M
-XX:ReservedCodeCacheSize=1024M
-Dsbt.gigahorse=false
-Dhttp.port=9001
-Djava.net.preferIPv4Stack=true

View File

@ -70,7 +70,7 @@ lazy val sharedSettings = Seq(
)
lazy val testSettings = Seq(
parallelExecution in Test := false,
parallelExecution in Test := true,
parallelExecution in IntegrationTest := false,
fork in IntegrationTest := false,
testOptions in Test += Tests.Argument("-oDNCXEHPQRMIK"),
@ -196,6 +196,7 @@ lazy val api = (project in file("modules/api"))
.settings(inConfig(IntegrationTest)(scalafmtConfigSettings))
.dependsOn(core, dynamodb % "compile->compile;it->it")
val killDocker = TaskKey[Unit]("killDocker", "Kills all vinyldns docker containers")
lazy val root = (project in file(".")).enablePlugins(AutomateHeaderPlugin)
.configs(IntegrationTest)
.settings(headerSettings(IntegrationTest))
@ -203,7 +204,11 @@ lazy val root = (project in file(".")).enablePlugins(AutomateHeaderPlugin)
.settings(
inConfig(IntegrationTest)(scalafmtConfigSettings),
(scalastyleConfig in Test) := baseDirectory.value / "scalastyle-test-config.xml",
(scalastyleConfig in IntegrationTest) := baseDirectory.value / "scalastyle-test-config.xml"
(scalastyleConfig in IntegrationTest) := baseDirectory.value / "scalastyle-test-config.xml",
killDocker := {
import scala.sys.process._
"./bin/remove-vinyl-containers.sh" !
},
)
.aggregate(core, api, portal, dynamodb)
@ -271,7 +276,9 @@ lazy val dynamodb = (project in file("modules/dynamodb"))
organization := "io.vinyldns",
coverageMinimum := 85,
coverageFailOnMinimum := true,
coverageHighlighting := true
coverageHighlighting := true,
parallelExecution in Test := true,
parallelExecution in IntegrationTest := true
).dependsOn(core % "compile->compile;test->test")
val preparePortal = TaskKey[Unit]("preparePortal", "Runs NPM to prepare portal for start")
@ -427,24 +434,26 @@ releaseProcess :=
sonatypePublishStage ++
finalReleaseStage
// Validate runs static checks and compile to make sure we can go
addCommandAlias("validate-api",
";project api; clean; headerCheck; test:headerCheck; it:headerCheck; scalastyle; test:scalastyle; " +
"it:scalastyle; compile; test:compile; it:compile")
addCommandAlias("validate-dynamodb",
";project dynamodb; clean; headerCheck; test:headerCheck; it:headerCheck; scalastyle; test:scalastyle; " +
"it:scalastyle; compile; test:compile; it:compile")
addCommandAlias("validate-core",
";project core; clean; headerCheck; test:headerCheck; scalastyle; test:scalastyle; compile; test:compile")
addCommandAlias("validate-portal",
";project portal; clean; headerCheck; test:headerCheck; compile; test:compile; createJsHeaders; checkJsHeaders")
addCommandAlias("validate", ";validate-core;validate-dynamodb;validate-api;validate-portal")
// Let's do things in parallel!
addCommandAlias("validate", "; root/clean; " +
"all core/headerCheck core/test:headerCheck " +
"api/headerCheck api/test:headerCheck api/it:headerCheck " +
"dynamodb/headerCheck dynamodb/test:headerCheck dynamodb/it:headerCheck " +
"portal/headerCheck portal/test:headerCheck; " +
"all core/scalastyle core/test:scalastyle " +
"api/scalastyle api/test:scalastyle api/it:scalastyle " +
"dynamodb/scalastyle dynamodb/test:scalastyle dynamodb/it:scalastyle" +
"portal/scalastyle portal/test:scalastyle;" +
"portal/createJsHeaders;portal/checkJsHeaders;" +
"root/compile;root/test:compile;root/it:compile"
)
// Verify runs all tests and code coverage
addCommandAlias("dockerComposeUpAll",";project api;dockerComposeUp;project dynamodb;dockerComposeUp;project root")
addCommandAlias("dockerComposeStopAll",";project api;dockerComposeStop;project dynamodb;dockerComposeStop;project root")
addCommandAlias("verify",
";dockerComposeUpAll;project root;coverage;test;it:test;coverageReport;coverageAggregate;dockerComposeStopAll")
addCommandAlias("verify", "; project root; killDocker; " +
"project api; dockerComposeUp; project dynamodb; dockerComposeUp; " +
"project root; coverage; " +
"all core/test dynamodb/test api/test dynamodb/it:test api/it:test portal/test; " +
"project root; coverageReport; coverageAggregate; killDocker"
)
// Build the artifacts for release
addCommandAlias("build-api", ";project api;clean;assembly")

View File

@ -4,15 +4,8 @@
<pattern>%msg%n</pattern>
</encoder>
</appender>
<appender name="FILE" class="ch.qos.logback.core.FileAppender">
<file>target/test/test.log</file>
<append>true</append>
<encoder>
<pattern>%-4relative [%thread] %-5level %logger{35} - %msg%n</pattern>
</encoder>
</appender>
<root level="DEBUG">
<appender-ref ref="FILE"/>
<root level="OFF">
<appender-ref ref="CONSOLE"/>
</root>
</configuration>

View File

@ -4,16 +4,8 @@
<pattern>%msg%n</pattern>
</encoder>
</appender>
<appender name="FILE" class="ch.qos.logback.core.FileAppender">
<file>target/test/test.log</file>
<append>true</append>
<encoder>
<pattern>%-4relative [%thread] %-5level %logger{35} - %msg%n</pattern>
</encoder>
16
</appender>
<root level="DEBUG">
<appender-ref ref="FILE"/>
<root level="OFF">
<appender-ref ref="CONSOLE"/>
</root>
</configuration>

View File

@ -50,7 +50,7 @@ class MembershipAuthPrincipalProviderSpec
.when(mockMembershipRepo)
.getGroupsForUser(any[String])
val result = await[Option[AuthPrincipal]](underTest.getAuthPrincipal(accessKey))
val result = underTest.getAuthPrincipal(accessKey).unsafeRunSync()
result.map { authPrincipal =>
authPrincipal.signedInUser shouldBe okUser
authPrincipal.memberGroupIds should contain theSameElementsAs Seq(okGroup.id, dummyGroup.id)

View File

@ -214,7 +214,8 @@ class BatchChangeConverterSpec
}
}
private val underTest = new BatchChangeConverter(InMemoryBatchChangeRepository, SqsWithFail)
private val batchChangeRepo = new InMemoryBatchChangeRepository
private val underTest = new BatchChangeConverter(batchChangeRepo, SqsWithFail)
"convertAndSendBatchForProcessing" should {
"successfully generate add RecordSetChange and map IDs for all adds" in {
@ -322,7 +323,7 @@ class BatchChangeConverterSpec
// check the batch has been stored in the DB
val savedBatch: Option[BatchChange] =
await(InMemoryBatchChangeRepository.getBatchChange(batchChange.id))
await(batchChangeRepo.getBatchChange(batchChange.id))
savedBatch shouldBe Some(batchChange)
}
@ -356,7 +357,7 @@ class BatchChangeConverterSpec
// check the update has been made in the DB
val savedBatch: Option[BatchChange] =
await(InMemoryBatchChangeRepository.getBatchChange(batchChangeWithBadSqs.id))
await(batchChangeRepo.getBatchChange(batchChangeWithBadSqs.id))
savedBatch shouldBe Some(returnedBatch)
}
@ -370,7 +371,7 @@ class BatchChangeConverterSpec
result shouldBe an[BatchConversionError]
val notSaved: Option[BatchChange] =
await(InMemoryBatchChangeRepository.getBatchChange(batchChangeUnsupported.id))
await(batchChangeRepo.getBatchChange(batchChangeUnsupported.id))
notSaved shouldBe None
}
}

View File

@ -103,7 +103,7 @@ class BatchChangeServiceSpec
None,
None)
private val batchChangeRepo = InMemoryBatchChangeRepository
private val batchChangeRepo = new InMemoryBatchChangeRepository
object EmptyBatchConverter extends BatchChangeConverterAlgebra {
def sendBatchForProcessing(

View File

@ -53,7 +53,7 @@ class RecordSetChangeHandlerSpec
private val rsRepoCaptor = ArgumentCaptor.forClass(classOf[ChangeSet])
private val changeRepoCaptor = ArgumentCaptor.forClass(classOf[ChangeSet])
private val batchRepo = InMemoryBatchChangeRepository
private val batchRepo = new InMemoryBatchChangeRepository
private val rs = completeCreateAAAA.recordSet
@ -302,7 +302,7 @@ class RecordSetChangeHandlerSpec
// resolve called once when validating, 12x for retries
verify(mockConn, times(13)).resolve(rs.name, rsChange.zone.name, rs.typ)
val batchChangeUpdates = await(batchRepo.getBatchChange(batchChange.id))
val batchChangeUpdates = batchRepo.getBatchChange(batchChange.id).unsafeRunSync()
val updatedSingleChanges = completeCreateAAAASingleChanges.map { ch =>
ch.copy(
status = SingleChangeStatus.Failed,

View File

@ -23,7 +23,7 @@ import vinyldns.core.domain.batch._
import scala.collection.concurrent
import cats.effect._
object InMemoryBatchChangeRepository extends BatchChangeRepository {
class InMemoryBatchChangeRepository extends BatchChangeRepository {
implicit def dateTimeOrdering: Ordering[DateTime] = Ordering.fromLessThan(_.isAfter(_))

View File

@ -1,148 +0,0 @@
/*
* Copyright 2018 Comcast Cable Communications Management, LLC
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package vinyldns.api.repository
import org.joda.time.DateTime
import org.scalatest.{Matchers, WordSpec}
import vinyldns.api.CatsHelpers
import vinyldns.core.domain.batch.SingleChangeStatus.Pending
import vinyldns.core.domain.batch._
import vinyldns.core.domain.record.AData
import vinyldns.core.domain.record.RecordType._
class InMemoryBatchChangeRepositorySpec extends WordSpec with Matchers with CatsHelpers {
private val addChange1 = SingleAddChange(
"zoneid",
"zoneName",
"apex.test.com.",
"apex.test.com.",
A,
100,
AData("1.1.1.1"),
Pending,
None,
None,
None,
"addChangeId1")
private val addChange2 =
addChange1.copy(inputName = "test.test.com.", recordName = "test", id = "addChangeId2")
private val batchChange = BatchChange(
"userId",
"username",
None,
DateTime.now,
List(addChange1, addChange2),
"batchChangeId")
private val failedChange = addChange1.copy(status = SingleChangeStatus.Failed)
private val completeChange = addChange2.copy(status = SingleChangeStatus.Complete)
private val underTest = InMemoryBatchChangeRepository
"InMemoryBatchChangeRepository" should {
"save a batch change, retrieve" in {
val saved = await(underTest.save(batchChange))
saved shouldBe batchChange
val checkBatch = await(underTest.getBatchChange(batchChange.id))
checkBatch shouldBe Some(batchChange)
val checkSingleChanges = await(underTest.getSingleChanges(List(addChange1.id, addChange2.id)))
checkSingleChanges.length shouldBe 2
checkSingleChanges should contain theSameElementsAs List(addChange1, addChange2)
}
"update single changes" in {
val update1 =
addChange1.copy(recordChangeId = Some("aRecordChange"), status = SingleChangeStatus.Pending)
val update2 = addChange2.copy(
recordChangeId = Some("aRecordChangeAgain"),
status = SingleChangeStatus.Pending)
await(underTest.updateSingleChanges(List(update1, update2)))
val checkBatch = await(underTest.getBatchChange(batchChange.id))
val expected = batchChange.copy(changes = List(update1, update2))
checkBatch shouldBe Some(expected)
val checkSingleChanges = await(underTest.getSingleChanges(List(addChange1.id, addChange2.id)))
checkSingleChanges.length shouldBe 2
checkSingleChanges should contain theSameElementsAs List(update1, update2)
}
}
"list batch change summaries with correct status when complete" in {
underTest.clear()
val changes = List(completeChange, completeChange)
val completeBatchChange = batchChange.copy(changes = changes)
val saved = await(underTest.save(completeBatchChange))
saved shouldBe completeBatchChange
val batchChangeSummaryList = await(underTest.getBatchChangeSummariesByUserId("userId"))
val expected = BatchChangeSummaryList(List(BatchChangeSummary(saved)))
batchChangeSummaryList shouldBe expected
batchChangeSummaryList.batchChanges(0).status shouldBe BatchChangeStatus.Complete
}
"list batch change summaries with correct status when pending" in {
underTest.clear()
val changes = List(completeChange, addChange1)
val pendingBatchChange = batchChange.copy(changes = changes)
val saved = await(underTest.save(pendingBatchChange))
saved shouldBe pendingBatchChange
val batchChangeSummaryList = await(underTest.getBatchChangeSummariesByUserId("userId"))
val expected = BatchChangeSummaryList(List(BatchChangeSummary(saved)))
batchChangeSummaryList shouldBe expected
batchChangeSummaryList.batchChanges(0).status shouldBe BatchChangeStatus.Pending
}
"list batch change summaries with correct status when failed" in {
underTest.clear()
val changes = List(failedChange, failedChange)
val failedBatchChange = batchChange.copy(changes = changes)
val saved = await(underTest.save(failedBatchChange))
saved shouldBe failedBatchChange
val batchChangeSummaryList = await(underTest.getBatchChangeSummariesByUserId("userId"))
val expected = BatchChangeSummaryList(List(BatchChangeSummary(saved)))
batchChangeSummaryList shouldBe expected
batchChangeSummaryList.batchChanges(0).status shouldBe BatchChangeStatus.Failed
}
"list batch change summaries with correct status when partial failure" in {
underTest.clear()
val changes = List(completeChange, failedChange)
val partiallyFailedBatchChange = batchChange.copy(changes = changes)
val saved = await(underTest.save(partiallyFailedBatchChange))
saved shouldBe partiallyFailedBatchChange
val batchChangeSummaryList = await(underTest.getBatchChangeSummariesByUserId("userId"))
val expected = BatchChangeSummaryList(List(BatchChangeSummary(saved)))
batchChangeSummaryList shouldBe expected
batchChangeSummaryList.batchChanges(0).status shouldBe BatchChangeStatus.PartialFailure
}
}